You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2014/01/17 01:58:21 UTC

[01/10] Decoupled replicated log coordinator logic and made it asynchronous.

Updated Branches:
  refs/heads/master 2ff53088b -> 420e30bfe


http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/replica.cpp
----------------------------------------------------------------------
diff --git a/src/log/replica.cpp b/src/log/replica.cpp
index 82c2157..032180a 100644
--- a/src/log/replica.cpp
+++ b/src/log/replica.cpp
@@ -25,7 +25,7 @@
 #include <algorithm>
 
 #include <process/dispatch.hpp>
-#include <process/protobuf.hpp>
+#include <process/id.hpp>
 
 #include <stout/check.hpp>
 #include <stout/error.hpp>
@@ -59,18 +59,17 @@ namespace protocol {
 // Some replica protocol definitions.
 Protocol<PromiseRequest, PromiseResponse> promise;
 Protocol<WriteRequest, WriteResponse> write;
-Protocol<LearnRequest, LearnResponse> learn;
 
 } // namespace protocol {
 
 
 struct State
 {
-  uint64_t coordinator; // Last promise made to a coordinator.
+  uint64_t proposal; // Last promise made.
   uint64_t begin; // Beginning position of the log.
   uint64_t end; // Ending position of the log.
-  std::set<uint64_t> learned; // Positions present and learned
-  std::set<uint64_t> unlearned; // Positions present but unlearned.
+  set<uint64_t> learned; // Positions present and learned
+  set<uint64_t> unlearned; // Positions present but unlearned.
 };
 
 
@@ -237,7 +236,7 @@ Try<State> LevelDBStorage::recover(const string& path)
   LOG(INFO) << "Compacted db in " << stopwatch.elapsed();
 
   State state;
-  state.coordinator = 0;
+  state.proposal = 0;
   state.begin = 0;
   state.end = 0;
 
@@ -277,14 +276,14 @@ Try<State> LevelDBStorage::recover(const string& path)
     switch (record.type()) {
       case Record::METADATA: {
         CHECK(record.has_metadata());
-        state.coordinator = record.metadata().promised();
+        state.proposal = record.metadata().promised();
         break;
       }
 
       // DEPRECATED!
       case Record::PROMISE: {
         CHECK(record.has_promise());
-        state.coordinator = record.promise().id();
+        state.proposal = record.promise().proposal();
         break;
       }
 
@@ -484,7 +483,7 @@ class ReplicaProcess : public ProtobufProcess<ReplicaProcess>
 public:
   // Constructs a new replica process using specified path to a
   // directory for storing the underlying log.
-  ReplicaProcess(const std::string& path);
+  ReplicaProcess(const string& path);
 
   virtual ~ReplicaProcess();
 
@@ -498,11 +497,15 @@ public:
 
   // Returns all the actions between the specified positions, unless
   // those positions are invalid, in which case returns an error.
-  process::Future<std::list<Action> > read(uint64_t from, uint64_t to);
+  Future<list<Action> > read(uint64_t from, uint64_t to);
+
+  // Returns true if the specified position is missing in the log
+  // (i.e., unlearned or holes).
+  bool missing(uint64_t position);
 
   // Returns missing positions in the log (i.e., unlearned or holes)
-  // up to the specified position.
-  std::set<uint64_t> missing(uint64_t position);
+  // within the specified range [from, to].
+  set<uint64_t> missing(uint64_t from, uint64_t to);
 
   // Returns the beginning position of the log.
   uint64_t beginning();
@@ -514,17 +517,13 @@ public:
   uint64_t promised();
 
 private:
-  // Handles a request from a coordinator to promise not to accept
-  // writes from any other coordinator.
+  // Handles a request from a proposer to promise not to accept writes
+  // from any other proposer with lower proposal number.
   void promise(const PromiseRequest& request);
 
-  // Handles a request from a coordinator to write an action.
+  // Handles a request from a proposer to write an action.
   void write(const WriteRequest& request);
 
-  // Handles a request from a coordinator (or replica) to learn the
-  // specified position in the log.
-  void learn(uint64_t position);
-
   // Handles a message notifying of a learned action.
   void learned(const Action& action);
 
@@ -534,13 +533,13 @@ private:
   bool persist(const Action& action);
 
   // Helper routine to recover log (e.g., on restart).
-  void recover(const std::string& path);
+  void recover(const string& path);
 
   // Underlying storage for the log.
   Storage* storage;
 
-  // Last promise made to a coordinator.
-  uint64_t coordinator;
+  // Last promise made to a proposer.
+  uint64_t proposal;
 
   // Beginning position of log (after *learned* truncations).
   uint64_t begin;
@@ -549,19 +548,21 @@ private:
   uint64_t end;
 
   // Holes in the log.
-  std::set<uint64_t> holes;
+  set<uint64_t> holes;
 
   // Unlearned positions in the log.
-  std::set<uint64_t> unlearned;
+  set<uint64_t> unlearned;
 };
 
 
 ReplicaProcess::ReplicaProcess(const string& path)
-  : coordinator(0),
+  : ProcessBase(ID::generate("log-replica")),
+    proposal(0),
     begin(0),
     end(0)
 {
-  storage = new LevelDBStorage(); // TODO(benh): Factor out and expose storage.
+  // TODO(benh): Factor out and expose storage.
+  storage = new LevelDBStorage();
 
   recover(path);
 
@@ -575,10 +576,6 @@ ReplicaProcess::ReplicaProcess(const string& path)
   install<LearnedMessage>(
       &ReplicaProcess::learned,
       &LearnedMessage::action);
-
-  install<LearnRequest>(
-      &ReplicaProcess::learn,
-      &LearnRequest::position);
 }
 
 
@@ -613,9 +610,7 @@ Result<Action> ReplicaProcess::read(uint64_t position)
 
 // TODO(benh): Make this function actually return a Try once we change
 // the future semantics to not include failures.
-process::Future<list<Action> > ReplicaProcess::read(
-    uint64_t from,
-    uint64_t to)
+Future<list<Action> > ReplicaProcess::read(uint64_t from, uint64_t to)
 {
   if (to < from) {
     process::Promise<list<Action> > promise;
@@ -649,26 +644,46 @@ process::Future<list<Action> > ReplicaProcess::read(
 }
 
 
-set<uint64_t> ReplicaProcess::missing(uint64_t index)
+bool ReplicaProcess::missing(uint64_t position)
 {
-  // Start off with all the unlearned positions.
-  set<uint64_t> positions = unlearned;
-
-  // Add in a spoonful of holes.
-  foreach (uint64_t hole, holes) {
-    positions.insert(hole);
+  if (position < begin) {
+    return false; // Truncated positions are treated as learned.
+  } else if (position > end) {
+    return true;
+  } else {
+    if (unlearned.count(position) != 0 || holes.count(position) != 0) {
+      return true;
+    } else {
+      return false;
+    }
   }
+}
 
-  // And finally add all the unknown positions beyond our end.
-  for (; index >= end; index--) {
-    positions.insert(index);
 
-    // Don't wrap around 0!
-    if (index == 0) {
-      break;
+set<uint64_t> ReplicaProcess::missing(uint64_t from, uint64_t to)
+{
+  // TODO(jieyu): Optimize the performence for the common case.
+  set<uint64_t> positions;
+
+  // Add unlearned positions.
+  foreach (uint64_t p, unlearned) {
+    if (p >= from && p <= to) {
+      positions.insert(p);
     }
   }
 
+  // Add holes.
+  foreach (uint64_t p, holes) {
+    if (p >= from && p <= to) {
+      positions.insert(p);
+    }
+  }
+
+  // Add all the unknown positions beyond our end.
+  for (; to > end; to--) {
+    positions.insert(to);
+  }
+
   return positions;
 }
 
@@ -687,54 +702,58 @@ uint64_t ReplicaProcess::ending()
 
 uint64_t ReplicaProcess::promised()
 {
-  return coordinator;
+  return proposal;
 }
 
 
 // Note that certain failures that occur result in returning from the
-// current function but *NOT* sending a 'nack' back to the coordinator
-// because that implies a coordinator has been demoted. Not sending
+// current function but *NOT* sending a NACK back to the proposer
+// because that implies a proposer has been demoted. Not sending
 // anything is equivalent to pretending like the request never made it
 // here. TODO(benh): At some point, however, we might want to actually
 // "fail" more dramatically because there could be something rather
-// seriously wrong on this box that we are ignoring (like a bad
-// disk). This could be accomplished by changing most LOG(ERROR)
-// statements to LOG(FATAL), or by counting the number of errors and
-// after reaching some threshold aborting. In addition, sending the
-// error information back to the coordinator "might" help the
-// debugging procedure.
+// seriously wrong on this box that we are ignoring (like a bad disk).
+// This could be accomplished by changing most LOG(ERROR) statements
+// to LOG(FATAL), or by counting the number of errors and after
+// reaching some threshold aborting. In addition, sending the error
+// information back to the proposer "might" help the debugging
+// procedure.
 
 
 void ReplicaProcess::promise(const PromiseRequest& request)
 {
   if (request.has_position()) {
-    LOG(INFO) << "Replica received explicit promise request for "
-              << request.id() << " for position " << request.position();
-
-    // If the position has been truncated, tell the coordinator that
-    // it's a learned no-op. This can happen when a replica has missed
-    // some truncates and it's coordinator tries to fill some
-    // truncated positions on election. A learned no-op is safe since
-    // the coordinator should eventually learn that this position was
+    LOG(INFO) << "Replica received explicit promise request for position "
+              << request.position() << " with proposal " << request.proposal();
+
+    // If the position has been truncated, tell the proposer that it's
+    // a learned no-op. This can happen when a replica has missed some
+    // truncates and it's proposer tries to fill some truncated
+    // positions on election. A learned no-op is safe since the
+    // proposer should eventually learn that this position was
     // actually truncated. The action must be _learned_ so that the
-    // coordinator doesn't attempt to run a full Paxos round which
-    // will never succeed because this replica will not permit the
-    // write (because ReplicaProcess::write "ignores" writes on
-    // truncated positions).
+    // proposer doesn't attempt to run a full Paxos round which will
+    // never succeed because this replica will not permit the write
+    // (because ReplicaProcess::write "ignores" writes on truncated
+    // positions).
+    // TODO(jieyu): Think about whether we need to check proposal
+    // number so that we don't reply a proposer whose number is
+    // obviously smaller than most of the proposers in the system.
     if (request.position() < begin) {
       Action action;
       action.set_position(request.position());
-      action.set_promised(coordinator); // Use the last coordinator.
-      action.set_performed(coordinator); // Use the last coordinator.
+      action.set_promised(proposal); // Use the last promised proposal.
+      action.set_performed(proposal); // Use the last promised proposal.
       action.set_learned(true);
       action.set_type(Action::NOP);
       action.mutable_nop()->MergeFrom(Action::Nop());
 
       PromiseResponse response;
       response.set_okay(true);
-      response.set_id(request.id());
+      response.set_proposal(request.proposal());
       response.mutable_action()->MergeFrom(action);
       reply(response);
+      return;
     }
 
     // Need to get the action for the specified position.
@@ -744,63 +763,86 @@ void ReplicaProcess::promise(const PromiseRequest& request)
       LOG(ERROR) << "Error getting log record at " << request.position()
                  << ": " << result.error();
     } else if (result.isNone()) {
-      Action action;
-      action.set_position(request.position());
-      action.set_promised(request.id());
-
-      if (persist(action)) {
+      // This position has been implicitly promised to a proposer.
+      // Therefore, we should no longer give promise to a proposer
+      // with a lower (or equal) proposal number. If not, we may
+      // accept writes from both proposers, causing a potential
+      // inconsistency in the log. For example, there are three
+      // replicas R1, R2 and R3. Assume that log position 1 in all
+      // replicas are implicitly promised to proposer 2. Later,
+      // proposer 1 asks for explicit promises from R2 and R3 for log
+      // position 1. If we don't perform the following check, R2 and
+      // R3 will give their promises to R2 and R3 for log position 1.
+      // As a result, proposer 1 can successfully write a value X to
+      // log position 1 and thinks that X is agreed, while proposer 2
+      // can later write a value Y and also believes that Y is agreed.
+      if (request.proposal() <= proposal) {
+        // If a promise request is rejected because of the proposal
+        // number check, we reply with the currently promised proposal
+        // number so that the proposer can bump its proposal number
+        // and retry if needed to ensure liveness.
         PromiseResponse response;
-        response.set_okay(true);
-        response.set_id(request.id());
-        response.set_position(request.position());
+        response.set_okay(false);
+        response.set_proposal(proposal);
         reply(response);
+      } else {
+        Action action;
+        action.set_position(request.position());
+        action.set_promised(request.proposal());
+
+        if (persist(action)) {
+          PromiseResponse response;
+          response.set_okay(true);
+          response.set_proposal(request.proposal());
+          response.set_position(request.position());
+          reply(response);
+        }
       }
     } else {
       CHECK_SOME(result);
       Action action = result.get();
-      CHECK(action.position() == request.position());
+      CHECK_EQ(action.position(), request.position());
 
-      if (request.id() < action.promised()) {
+      if (request.proposal() <= action.promised()) {
         PromiseResponse response;
         response.set_okay(false);
-        response.set_id(request.id());
-        response.set_position(request.position());
+        response.set_proposal(action.promised());
         reply(response);
       } else {
         Action original = action;
-        action.set_promised(request.id());
+        action.set_promised(request.proposal());
 
         if (persist(action)) {
           PromiseResponse response;
           response.set_okay(true);
-          response.set_id(request.id());
+          response.set_proposal(request.proposal());
           response.mutable_action()->MergeFrom(original);
           reply(response);
         }
       }
     }
   } else {
-    LOG(INFO) << "Replica received implicit promise request for "
-              << request.id();
+    LOG(INFO) << "Replica received implicit promise request with proposal "
+              << request.proposal();
 
-    if (request.id() <= coordinator) { // Only make an implicit promise once!
-      LOG(INFO) << "Replica denying promise request for "
-                << request.id();
+    if (request.proposal() <= proposal) { // Only make an implicit promise once!
+      LOG(INFO) << "Replica denying promise request with proposal "
+                << request.proposal();
       PromiseResponse response;
       response.set_okay(false);
-      response.set_id(request.id());
+      response.set_proposal(proposal);
       reply(response);
     } else {
       Promise promise;
-      promise.set_id(request.id());
+      promise.set_proposal(request.proposal());
 
       if (persist(promise)) {
-        coordinator = request.id();
+        proposal = request.proposal();
 
         // Return the last position written.
         PromiseResponse response;
         response.set_okay(true);
-        response.set_id(request.id());
+        response.set_proposal(request.proposal());
         response.set_position(end);
         reply(response);
       }
@@ -811,7 +853,8 @@ void ReplicaProcess::promise(const PromiseRequest& request)
 
 void ReplicaProcess::write(const WriteRequest& request)
 {
-  LOG(INFO) << "Replica received write request for position " << request.position();
+  LOG(INFO) << "Replica received write request for position "
+            << request.position();
 
   Result<Action> result = read(request.position());
 
@@ -819,17 +862,17 @@ void ReplicaProcess::write(const WriteRequest& request)
     LOG(ERROR) << "Error getting log record at " << request.position()
                << ": " << result.error();
   } else if (result.isNone()) {
-    if (request.id() < coordinator) {
+    if (request.proposal() < proposal) {
       WriteResponse response;
       response.set_okay(false);
-      response.set_id(request.id());
+      response.set_proposal(proposal);
       response.set_position(request.position());
       reply(response);
     } else {
       Action action;
       action.set_position(request.position());
-      action.set_promised(coordinator);
-      action.set_performed(request.id());
+      action.set_promised(proposal);
+      action.set_performed(request.proposal());
       if (request.has_learned()) action.set_learned(request.learned());
       action.set_type(request.type());
 
@@ -840,11 +883,11 @@ void ReplicaProcess::write(const WriteRequest& request)
           break;
         case Action::APPEND:
           CHECK(request.has_append());
-          action.mutable_append()->MergeFrom(request.append());
+          action.mutable_append()->CopyFrom(request.append());
           break;
         case Action::TRUNCATE:
           CHECK(request.has_truncate());
-          action.mutable_truncate()->MergeFrom(request.truncate());
+          action.mutable_truncate()->CopyFrom(request.truncate());
           break;
         default:
           LOG(FATAL) << "Unknown Action::Type!";
@@ -853,25 +896,42 @@ void ReplicaProcess::write(const WriteRequest& request)
       if (persist(action)) {
         WriteResponse response;
         response.set_okay(true);
-        response.set_id(request.id());
+        response.set_proposal(request.proposal());
         response.set_position(request.position());
         reply(response);
       }
     }
   } else if (result.isSome()) {
     Action action = result.get();
-    CHECK(action.position() == request.position());
+    CHECK_EQ(action.position(), request.position());
 
-    if (request.id() < action.promised()) {
+    if (request.proposal() < action.promised()) {
       WriteResponse response;
       response.set_okay(false);
-      response.set_id(request.id());
+      response.set_proposal(action.promised());
       response.set_position(request.position());
       reply(response);
     } else {
       // TODO(benh): Check if this position has already been learned,
       // and if so, check that we are re-writing the same value!
-      action.set_performed(request.id());
+      //
+      // TODO(jieyu): Interestingly, in the presence of truncations,
+      // we may encounter a situation where this position has already
+      // been learned, but we are re-writing a different value. For
+      // example, assume that there are 5 replicas (R1 ~ R5). First,
+      // an append operation has been agreed at position 5 by R1, R2,
+      // R3 and R4, but only R1 receives a learned message. Later, a
+      // truncate operation has been agreed at position 10 by R1, R2
+      // and R3, but only R1 receives a learned message. Now, a leader
+      // failover happens and R5 is filled with a NOP at position 5
+      // because its coordinator receives a learned NOP at position 5
+      // from R1 (because of its learned truncation at position 10).
+      // Now, another leader failover happens and R4's coordinator
+      // tries to fill position 5. However, it is only able to contact
+      // R2, R3 and R4 during the explicit promise phase. As a result,
+      // it will try to write an append operation at position 5 to R5
+      // while R5 currently have a learned NOP stored at position 5.
+      action.set_performed(request.proposal());
       action.clear_learned();
       if (request.has_learned()) action.set_learned(request.learned());
       action.clear_type();
@@ -887,11 +947,11 @@ void ReplicaProcess::write(const WriteRequest& request)
           break;
         case Action::APPEND:
           CHECK(request.has_append());
-          action.mutable_append()->MergeFrom(request.append());
+          action.mutable_append()->CopyFrom(request.append());
           break;
         case Action::TRUNCATE:
           CHECK(request.has_truncate());
-          action.mutable_truncate()->MergeFrom(request.truncate());
+          action.mutable_truncate()->CopyFrom(request.truncate());
           break;
         default:
           LOG(FATAL) << "Unknown Action::Type!";
@@ -900,7 +960,7 @@ void ReplicaProcess::write(const WriteRequest& request)
       if (persist(action)) {
         WriteResponse response;
         response.set_okay(true);
-        response.set_id(request.id());
+        response.set_proposal(request.proposal());
         response.set_position(request.position());
         reply(response);
       }
@@ -911,42 +971,18 @@ void ReplicaProcess::write(const WriteRequest& request)
 
 void ReplicaProcess::learned(const Action& action)
 {
-  LOG(INFO) << "Replica received learned notice for position " << action.position();
+  LOG(INFO) << "Replica received learned notice for position "
+            << action.position();
 
   CHECK(action.learned());
 
   if (persist(action)) {
-    LOG(INFO) << "Replica learned "
-              << Action::Type_Name(action.type())
+    LOG(INFO) << "Replica learned " << Action::Type_Name(action.type())
               << " action at position " << action.position();
   }
 }
 
 
-void ReplicaProcess::learn(uint64_t position)
-{
-  LOG(INFO) << "Replica received learn request for position " << position;
-
-  Result<Action> result = read(position);
-
-  if (result.isError()) {
-    LOG(ERROR) << "Error getting log record at " << position
-               << ": " << result.error();
-  } else if (result.isSome() &&
-             result.get().has_learned() &&
-             result.get().learned()) {
-    LearnResponse response;
-    response.set_okay(true);
-    response.mutable_action()->MergeFrom(result.get());
-    reply(response);
-  } else {
-    LearnResponse response;
-    response.set_okay(false);
-    reply(response);
-  }
-}
-
-
 bool ReplicaProcess::persist(const Promise& promise)
 {
   Try<Nothing> persisted = storage->persist(promise);
@@ -956,7 +992,7 @@ bool ReplicaProcess::persist(const Promise& promise)
     return false;
   }
 
-  LOG(INFO) << "Persisted promise to " << promise.id();
+  LOG(INFO) << "Persisted promise to " << promise.proposal();
 
   return true;
 }
@@ -999,6 +1035,9 @@ bool ReplicaProcess::persist(const Action& action)
       // And update the beginning position.
       begin = std::max(begin, action.truncate().to());
     }
+  } else {
+    // We just introduced an unlearned position.
+    unlearned.insert(action.position());
   }
 
   // Update holes if we just wrote many positions past the last end.
@@ -1020,13 +1059,13 @@ void ReplicaProcess::recover(const string& path)
   CHECK_SOME(state) << "Failed to recover the log";
 
   // Pull out and save some of the state.
-  coordinator = state.get().coordinator;
+  proposal = state.get().proposal;
   begin = state.get().begin;
   end = state.get().end;
   unlearned = state.get().unlearned;
 
   // Only use the learned positions to help determine the holes.
-  const std::set<uint64_t>& learned = state.get().learned;
+  const set<uint64_t>& learned = state.get().learned;
 
   // We need to assume that position 0 is a hole for a brand new log
   // (a coordinator will simply fill it with a no-op when it first
@@ -1050,54 +1089,58 @@ void ReplicaProcess::recover(const string& path)
 }
 
 
-Replica::Replica(const std::string& path)
+Replica::Replica(const string& path)
 {
   process = new ReplicaProcess(path);
-  process::spawn(process);
+  spawn(process);
 }
 
 
 Replica::~Replica()
 {
-  process::terminate(process);
+  terminate(process);
   process::wait(process);
   delete process;
 }
 
 
-process::Future<std::list<Action> > Replica::read(
-    uint64_t from,
-    uint64_t to)
+Future<list<Action> > Replica::read(uint64_t from, uint64_t to) const
+{
+  return dispatch(process, &ReplicaProcess::read, from, to);
+}
+
+
+Future<bool> Replica::missing(uint64_t position) const
 {
-  return process::dispatch(process, &ReplicaProcess::read, from, to);
+  return dispatch(process, &ReplicaProcess::missing, position);
 }
 
 
-process::Future<std::set<uint64_t> > Replica::missing(uint64_t position)
+Future<set<uint64_t> > Replica::missing(uint64_t from, uint64_t to) const
 {
-  return process::dispatch(process, &ReplicaProcess::missing, position);
+  return dispatch(process, &ReplicaProcess::missing, from, to);
 }
 
 
-process::Future<uint64_t> Replica::beginning()
+Future<uint64_t> Replica::beginning() const
 {
-  return process::dispatch(process, &ReplicaProcess::beginning);
+  return dispatch(process, &ReplicaProcess::beginning);
 }
 
 
-process::Future<uint64_t> Replica::ending()
+Future<uint64_t> Replica::ending() const
 {
-  return process::dispatch(process, &ReplicaProcess::ending);
+  return dispatch(process, &ReplicaProcess::ending);
 }
 
 
-process::Future<uint64_t> Replica::promised()
+Future<uint64_t> Replica::promised() const
 {
-  return process::dispatch(process, &ReplicaProcess::promised);
+  return dispatch(process, &ReplicaProcess::promised);
 }
 
 
-process::PID<ReplicaProcess> Replica::pid()
+PID<ReplicaProcess> Replica::pid() const
 {
   return process->self();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/replica.hpp
----------------------------------------------------------------------
diff --git a/src/log/replica.hpp b/src/log/replica.hpp
index d1f5ead..4cc7031 100644
--- a/src/log/replica.hpp
+++ b/src/log/replica.hpp
@@ -23,11 +23,9 @@
 #include <set>
 #include <string>
 
-#include <process/process.hpp>
 #include <process/protobuf.hpp>
 
 #include <stout/result.hpp>
-#include <stout/try.hpp>
 
 #include "messages/log.hpp"
 
@@ -40,7 +38,6 @@ namespace protocol {
 // Some replica protocol declarations.
 extern Protocol<PromiseRequest, PromiseResponse> promise;
 extern Protocol<WriteRequest, WriteResponse> write;
-extern Protocol<LearnRequest, LearnResponse> learn;
 
 } // namespace protocol {
 
@@ -59,23 +56,31 @@ public:
 
   // Returns all the actions between the specified positions, unless
   // those positions are invalid, in which case returns an error.
-  process::Future<std::list<Action> > read(uint64_t from, uint64_t to);
+  process::Future<std::list<Action> > read(
+      uint64_t from,
+      uint64_t to) const;
+
+  // Returns true if the specified position is missing in the log
+  // (i.e., unlearned or holes).
+  process::Future<bool> missing(uint64_t position) const;
 
   // Returns missing positions in the log (i.e., unlearned or holes)
-  // up to the specified position.
-  process::Future<std::set<uint64_t> > missing(uint64_t position);
+  // within the specified range [from, to].
+  process::Future<std::set<uint64_t> > missing(
+      uint64_t from,
+      uint64_t to) const;
 
   // Returns the beginning position of the log.
-  process::Future<uint64_t> beginning();
+  process::Future<uint64_t> beginning() const;
 
   // Returns the last written position in the log.
-  process::Future<uint64_t> ending();
+  process::Future<uint64_t> ending() const;
 
   // Returns the highest implicit promise this replica has given.
-  process::Future<uint64_t> promised();
+  process::Future<uint64_t> promised() const;
 
   // Returns the PID associated with this replica.
-  process::PID<ReplicaProcess> pid();
+  process::PID<ReplicaProcess> pid() const;
 
 private:
   ReplicaProcess* process;

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/messages/log.proto
----------------------------------------------------------------------
diff --git a/src/messages/log.proto b/src/messages/log.proto
index e6460ab..8fa46ca 100644
--- a/src/messages/log.proto
+++ b/src/messages/log.proto
@@ -19,28 +19,28 @@
 package mesos.internal.log;
 
 
-// Represents a "promise" that a replica has made to a coordinator. A
-// promise is *implicitly* valid for _all_ future actions that get
-// performed on the replicated log (provided the action comes from the
-// same coordinator), until a new promise is made to a coordinator
-// with a higher id. Each replica writesevery promise it makes as a
+// Represents a "promise" that a replica has made. A promise is
+// *implicitly* valid for _all_ future actions that get performed on
+// the replicated log (provided the action comes from the same
+// proposer), until a new promise is made to a proposer with a higher
+// proposal number. Each replica writes every promise it makes as a
 // log record so that it can recover this information after a failure.
 // TODO(benh): Does the promise actually need to be written to stable
 // storage? Can we get away with looking at the last written action
 // and using it's promised value? In this case, what happens if we
 // make a promise but don't receive an action from that coordinator?
 message Promise {
-  required uint64 id = 1;
+  required uint64 proposal = 1;
 }
 
 
 // Represents an "action" performed on the log. Each action has an
 // associated position in the log. In addition, each action (i.e.,
-// position) will have been "promised" to a specific coordinator
+// position) will have been "promised" to a specific proposer
 // (implicitly or explicitly) and may have been "performed" from a
-// specific coordinator. An action may also be "learned" to have
-// reached consensus. There are three types of possible actions that
-// can be performed on the log: nop (no action), append, and truncate.
+// specific proposer. An action may also be "learned" to have reached
+// consensus. There are three types of possible actions that can be
+// performed on the log: nop (no action), append, and truncate.
 message Action {
   required uint64 position = 1;
   required uint64 promised = 2;
@@ -110,32 +110,35 @@ message Record {
 ////////////////////////////////////////////////////
 
 
-// Represents a "promise" request from a coordinator with the
-// specified id to a replica. Most such requests will occur after a
-// coordinator has failed and a new coordinator is elected. In such a
-// case the position that the coordinator is asking the replica to
-// promise is implicitly *all* positions that the replica has made no
-// promises (thus the position field is not be used). In other
-// instances, however, a coordinator might be explicitly trying to
-// request that a replica promise a specific position in the log (such
-// as when trying to fill holes discovered during a client read), and
-// then position will be present.
+// Represents a "promise" request from a proposer with the specified
+// 'proposal' to a replica. If the proposer is a coordinator, most
+// such requests will occur after a coordinator has failed and a new
+// coordinator is elected. In such a case, the position that the
+// coordinator is asking the replica to promise is implicitly *all*
+// positions that the replica has made no promises (thus the position
+// field is not be used). In other instances, however, a proposer
+// might be explicitly trying to request that a replica promise a
+// specific position in the log (such as when trying to fill holes
+// discovered during a client read), and then the 'position' field
+// will be present.
 message PromiseRequest {
-  required uint64 id = 1;
+  required uint64 proposal = 1;
   optional uint64 position = 2;
 }
 
 
-// Represents a "promise" response from a replica back to a
-// coordinator with the specified id. A replica represents a 'nack'
-// (because it has promised a coordinator with a higher id) by setting
-// the okay field to false. The replica either sends back the highest
-// position it has recorded in the log (using the position field) or
-// the specific action (if any) it has at the position requested in
-// PromiseRequest.
+// Represents a "promise" response from a replica back to a proposer.
+// A replica represents a NACK (because it has promised a proposer
+// with a higher proposal number) by setting the okay field to false.
+// The 'proposal' is either the aforementioned higher proposal number
+// when the response is a NACK, or the corresponding request's
+// proposal number if it is an ACK. The replica either sends back the
+// highest position it has recorded in the log (using the 'position'
+// field) or the specific action (if any) it has at the position
+// requested in PromiseRequest (using the 'action' field).
 message PromiseResponse {
   required bool okay = 1;
-  required uint64 id = 2;
+  required uint64 proposal = 2;
   optional uint64 position = 4;
   optional Action action = 3;
 }
@@ -146,7 +149,7 @@ message PromiseResponse {
 // fields that are not relevant to a write request (e.g., promised,
 // performed) and rather than ignore them we exclude them for safety.
 message WriteRequest {
-  required uint64 id = 1;
+  required uint64 proposal = 1;
   required uint64 position = 2;
   optional bool learned = 3;
   required Action.Type type = 4;
@@ -156,31 +159,21 @@ message WriteRequest {
 }
 
 
-// Represents a write response corresponding to a write request. If
-// okay is not true then the coordinator has been demoted. Both id and
-// position should always correspond to the id and position set in the
-// request.
+// Represents a write response corresponding to a write request. A
+// replica represents a NACK (because it has promised a proposer with
+// a higher proposal number) by setting the okay field to false. If
+// the proposer is a coordinator, then it has been demoted. The
+// 'position' should always correspond to the position set in the
+// request. The 'proposal' is either the same proposal number set in
+// the request in the case of an ACK, or the higher proposal number
+// this position has been promised to in the case of a NACK.
 message WriteResponse {
   required bool okay = 1;
-  required uint64 id = 2;
+  required uint64 proposal = 2;
   required uint64 position = 3;
 }
 
 
-// Represents a learn (i.e., read) request and response. Note that a
-// non-learned position will not be returned. TODO(benh): Allow
-// learning more than one position at a time.
-message LearnRequest {
-  required uint64 position = 1;
-}
-
-
-message LearnResponse {
-  required bool okay = 1;
-  optional Action action = 2;
-}
-
-
 // Represents a "learned" event, that is, when a particular action has
 // been agreed upon (reached consensus).
 message LearnedMessage {

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/tests/log_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/log_tests.cpp b/src/tests/log_tests.cpp
index ff5f86c..fb9bbd8 100644
--- a/src/tests/log_tests.cpp
+++ b/src/tests/log_tests.cpp
@@ -48,8 +48,13 @@ using namespace mesos::internal::log;
 using process::Clock;
 using process::Future;
 using process::Timeout;
+using process::Shared;
 using process::UPID;
 
+using std::list;
+using std::set;
+using std::string;
+
 using testing::_;
 using testing::Eq;
 using testing::Return;
@@ -63,7 +68,7 @@ class ReplicaTest : public TemporaryDirectoryTest {};
 
 TEST_F(ReplicaTest, Promise)
 {
-  const std::string path = os::getcwd() + "/.log";
+  const string path = os::getcwd() + "/.log";
 
   Replica replica(path);
 
@@ -71,7 +76,7 @@ TEST_F(ReplicaTest, Promise)
   PromiseResponse response;
   Future<PromiseResponse> future;
 
-  request.set_id(2);
+  request.set_proposal(2);
 
   future = protocol::promise(replica.pid(), request);
 
@@ -79,12 +84,12 @@ TEST_F(ReplicaTest, Promise)
 
   response = future.get();
   EXPECT_TRUE(response.okay());
-  EXPECT_EQ(2u, response.id());
+  EXPECT_EQ(2u, response.proposal());
   EXPECT_TRUE(response.has_position());
   EXPECT_EQ(0u, response.position());
   EXPECT_FALSE(response.has_action());
 
-  request.set_id(1);
+  request.set_proposal(1);
 
   future = protocol::promise(replica.pid(), request);
 
@@ -92,11 +97,11 @@ TEST_F(ReplicaTest, Promise)
 
   response = future.get();
   EXPECT_FALSE(response.okay());
-  EXPECT_EQ(1u, response.id());
+  EXPECT_EQ(2u, response.proposal()); // Highest proposal seen so far.
   EXPECT_FALSE(response.has_position());
   EXPECT_FALSE(response.has_action());
 
-  request.set_id(3);
+  request.set_proposal(3);
 
   future = protocol::promise(replica.pid(), request);
 
@@ -104,7 +109,7 @@ TEST_F(ReplicaTest, Promise)
 
   response = future.get();
   EXPECT_TRUE(response.okay());
-  EXPECT_EQ(3u, response.id());
+  EXPECT_EQ(3u, response.proposal());
   EXPECT_TRUE(response.has_position());
   EXPECT_EQ(0u, response.position());
   EXPECT_FALSE(response.has_action());
@@ -113,14 +118,14 @@ TEST_F(ReplicaTest, Promise)
 
 TEST_F(ReplicaTest, Append)
 {
-  const std::string path = os::getcwd() + "/.log";
+  const string path = os::getcwd() + "/.log";
 
   Replica replica(path);
 
-  const uint64_t id = 1;
+  const uint64_t proposal = 1;
 
   PromiseRequest request1;
-  request1.set_id(id);
+  request1.set_proposal(proposal);
 
   Future<PromiseResponse> future1 =
     protocol::promise(replica.pid(), request1);
@@ -129,13 +134,13 @@ TEST_F(ReplicaTest, Append)
 
   PromiseResponse response1 = future1.get();
   EXPECT_TRUE(response1.okay());
-  EXPECT_EQ(id, response1.id());
+  EXPECT_EQ(proposal, response1.proposal());
   EXPECT_TRUE(response1.has_position());
   EXPECT_EQ(0u, response1.position());
   EXPECT_FALSE(response1.has_action());
 
   WriteRequest request2;
-  request2.set_id(id);
+  request2.set_proposal(proposal);
   request2.set_position(1);
   request2.set_type(Action::APPEND);
   request2.mutable_append()->set_bytes("hello world");
@@ -147,10 +152,10 @@ TEST_F(ReplicaTest, Append)
 
   WriteResponse response2 = future2.get();
   EXPECT_TRUE(response2.okay());
-  EXPECT_EQ(id, response2.id());
+  EXPECT_EQ(proposal, response2.proposal());
   EXPECT_EQ(1u, response2.position());
 
-  Future<std::list<Action> > actions = replica.read(1, 1);
+  Future<list<Action> > actions = replica.read(1, 1);
 
   AWAIT_READY(actions);
   ASSERT_EQ(1u, actions.get().size());
@@ -172,14 +177,14 @@ TEST_F(ReplicaTest, Append)
 
 TEST_F(ReplicaTest, Recover)
 {
-  const std::string path = os::getcwd() + "/.log";
+  const string path = os::getcwd() + "/.log";
 
   Replica replica1(path);
 
-  const uint64_t id = 1;
+  const uint64_t proposal= 1;
 
   PromiseRequest request1;
-  request1.set_id(id);
+  request1.set_proposal(proposal);
 
   Future<PromiseResponse> future1 =
     protocol::promise(replica1.pid(), request1);
@@ -188,13 +193,13 @@ TEST_F(ReplicaTest, Recover)
 
   PromiseResponse response1 = future1.get();
   EXPECT_TRUE(response1.okay());
-  EXPECT_EQ(id, response1.id());
+  EXPECT_EQ(proposal, response1.proposal());
   EXPECT_TRUE(response1.has_position());
   EXPECT_EQ(0u, response1.position());
   EXPECT_FALSE(response1.has_action());
 
   WriteRequest request2;
-  request2.set_id(id);
+  request2.set_proposal(proposal);
   request2.set_position(1);
   request2.set_type(Action::APPEND);
   request2.mutable_append()->set_bytes("hello world");
@@ -206,10 +211,10 @@ TEST_F(ReplicaTest, Recover)
 
   WriteResponse response2 = future2.get();
   EXPECT_TRUE(response2.okay());
-  EXPECT_EQ(id, response2.id());
+  EXPECT_EQ(proposal, response2.proposal());
   EXPECT_EQ(1u, response2.position());
 
-  Future<std::list<Action> > actions1 = replica1.read(1, 1);
+  Future<list<Action> > actions1 = replica1.read(1, 1);
 
   AWAIT_READY(actions1);
   ASSERT_EQ(1u, actions1.get().size());
@@ -231,7 +236,7 @@ TEST_F(ReplicaTest, Recover)
 
   Replica replica2(path);
 
-  Future<std::list<Action> > actions2 = replica2.read(1, 1);
+  Future<list<Action> > actions2 = replica2.read(1, 1);
 
   AWAIT_READY(actions2);
   ASSERT_EQ(1u, actions2.get().size());
@@ -258,18 +263,19 @@ class CoordinatorTest : public TemporaryDirectoryTest {};
 
 TEST_F(CoordinatorTest, Elect)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -278,7 +284,7 @@ TEST_F(CoordinatorTest, Elect)
   }
 
   {
-    Future<std::list<Action> > actions = replica1.read(0, 0);
+    Future<list<Action> > actions = replica1->read(0, 0);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(0u, actions.get().front().position());
@@ -290,18 +296,19 @@ TEST_F(CoordinatorTest, Elect)
 
 TEST_F(CoordinatorTest, AppendRead)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -320,7 +327,7 @@ TEST_F(CoordinatorTest, AppendRead)
   }
 
   {
-    Future<std::list<Action> > actions = replica1.read(position, position);
+    Future<list<Action> > actions = replica1->read(position, position);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(position, actions.get().front().position());
@@ -333,18 +340,19 @@ TEST_F(CoordinatorTest, AppendRead)
 
 TEST_F(CoordinatorTest, AppendReadError)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -364,7 +372,7 @@ TEST_F(CoordinatorTest, AppendReadError)
 
   {
     position += 1;
-    Future<std::list<Action> > actions = replica1.read(position, position);
+    Future<list<Action> > actions = replica1->read(position, position);
     AWAIT_FAILED(actions);
     EXPECT_EQ("Bad read range (past end of log)", actions.failure());
   }
@@ -373,15 +381,16 @@ TEST_F(CoordinatorTest, AppendReadError)
 
 TEST_F(CoordinatorTest, ElectNoQuorum)
 {
-  const std::string path = os::getcwd() + "/.log";
+  const string path = os::getcwd() + "/.log";
 
-  Replica replica(path);
+  Shared<Replica> replica(new Replica(path));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica->pid());
 
-  network.add(replica.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica, &network);
+  Coordinator coord(2, replica, network);
 
   Clock::pause();
 
@@ -401,18 +410,19 @@ TEST_F(CoordinatorTest, ElectNoQuorum)
 
 TEST_F(CoordinatorTest, AppendNoQuorum)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -420,7 +430,9 @@ TEST_F(CoordinatorTest, AppendNoQuorum)
     EXPECT_EQ(0u, result.get());
   }
 
-  network.remove(replica2.pid());
+  process::terminate(replica2->pid());
+  process::wait(replica2->pid());
+  replica2.reset();
 
   Clock::pause();
 
@@ -440,18 +452,19 @@ TEST_F(CoordinatorTest, AppendNoQuorum)
 
 TEST_F(CoordinatorTest, Failover)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -469,12 +482,9 @@ TEST_F(CoordinatorTest, Failover)
     EXPECT_EQ(1u, position);
   }
 
-  Network network2;
+  Shared<Network> network2(new Network(pids));
 
-  network2.add(replica1.pid());
-  network2.add(replica2.pid());
-
-  Coordinator coord2(2, &replica2, &network2);
+  Coordinator coord2(2, replica2, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -483,7 +493,7 @@ TEST_F(CoordinatorTest, Failover)
   }
 
   {
-    Future<std::list<Action> > actions = replica2.read(position, position);
+    Future<list<Action> > actions = replica2->read(position, position);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(position, actions.get().front().position());
@@ -496,18 +506,19 @@ TEST_F(CoordinatorTest, Failover)
 
 TEST_F(CoordinatorTest, Demoted)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -525,12 +536,9 @@ TEST_F(CoordinatorTest, Demoted)
     EXPECT_EQ(1u, position);
   }
 
-  Network network2;
+  Shared<Network> network2(new Network(pids));
 
-  network2.add(replica1.pid());
-  network2.add(replica2.pid());
-
-  Coordinator coord2(2, &replica2, &network2);
+  Coordinator coord2(2, replica2, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -554,7 +562,7 @@ TEST_F(CoordinatorTest, Demoted)
   }
 
   {
-    Future<std::list<Action> > actions = replica2.read(position, position);
+    Future<list<Action> > actions = replica2->read(position, position);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(position, actions.get().front().position());
@@ -567,19 +575,20 @@ TEST_F(CoordinatorTest, Demoted)
 
 TEST_F(CoordinatorTest, Fill)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
-  const std::string path3 = os::getcwd() + "/.log3";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -597,14 +606,15 @@ TEST_F(CoordinatorTest, Fill)
     EXPECT_EQ(1u, position);
   }
 
-  Replica replica3(path3);
+  Shared<Replica> replica3(new Replica(path3));
 
-  Network network2;
+  pids.clear();
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
 
-  network2.add(replica2.pid());
-  network2.add(replica3.pid());
+  Shared<Network> network2(new Network(pids));
 
-  Coordinator coord2(2, &replica3, &network2);
+  Coordinator coord2(2, replica3, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -615,7 +625,7 @@ TEST_F(CoordinatorTest, Fill)
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(position, position);
+    Future<list<Action> > actions = replica3->read(position, position);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(position, actions.get().front().position());
@@ -628,21 +638,24 @@ TEST_F(CoordinatorTest, Fill)
 
 TEST_F(CoordinatorTest, NotLearnedFill)
 {
-  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, _);
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
 
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
-  const std::string path3 = os::getcwd() + "/.log3";
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  // Drop messages here in order to obtain the pid of replica2. We
+  // only want to drop learned message sent to replica2.
+  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, Eq(replica2->pid()));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -660,14 +673,15 @@ TEST_F(CoordinatorTest, NotLearnedFill)
     EXPECT_EQ(1u, position);
   }
 
-  Replica replica3(path3);
+  Shared<Replica> replica3(new Replica(path3));
 
-  Network network2;
+  pids.clear();
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
 
-  network2.add(replica2.pid());
-  network2.add(replica3.pid());
+  Shared<Network> network2(new Network(pids));
 
-  Coordinator coord2(2, &replica3, &network2);
+  Coordinator coord2(2, replica3, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -678,7 +692,7 @@ TEST_F(CoordinatorTest, NotLearnedFill)
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(position, position);
+    Future<list<Action> > actions = replica3->read(position, position);
     AWAIT_READY(actions);
     ASSERT_EQ(1u, actions.get().size());
     EXPECT_EQ(position, actions.get().front().position());
@@ -691,18 +705,19 @@ TEST_F(CoordinatorTest, NotLearnedFill)
 
 TEST_F(CoordinatorTest, MultipleAppends)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -718,7 +733,7 @@ TEST_F(CoordinatorTest, MultipleAppends)
   }
 
   {
-    Future<std::list<Action> > actions = replica1.read(1, 10);
+    Future<list<Action> > actions = replica1->read(1, 10);
     AWAIT_READY(actions);
     EXPECT_EQ(10u, actions.get().size());
     foreach (const Action& action, actions.get()) {
@@ -732,21 +747,24 @@ TEST_F(CoordinatorTest, MultipleAppends)
 
 TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
 {
-  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, _);
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
 
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
-  const std::string path3 = os::getcwd() + "/.log3";
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  // Drop messages here in order to obtain the pid of replica2. We
+  // only want to drop learned message sent to replica2.
+  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, Eq(replica2->pid()));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -761,14 +779,15 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
     EXPECT_EQ(position, result.get());
   }
 
-  Replica replica3(path3);
+  Shared<Replica> replica3(new Replica(path3));
 
-  Network network2;
+  pids.clear();
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
 
-  network2.add(replica2.pid());
-  network2.add(replica3.pid());
+  Shared<Network> network2(new Network(pids));
 
-  Coordinator coord2(2, &replica3, &network2);
+  Coordinator coord2(2, replica3, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -779,7 +798,7 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(1, 10);
+    Future<list<Action> > actions = replica3->read(1, 10);
     AWAIT_READY(actions);
     EXPECT_EQ(10u, actions.get().size());
     foreach (const Action& action, actions.get()) {
@@ -793,18 +812,19 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
 
 TEST_F(CoordinatorTest, Truncate)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network.add(replica1.pid());
-  network.add(replica2.pid());
+  Shared<Network> network(new Network(pids));
 
-  Coordinator coord(2, &replica1, &network);
+  Coordinator coord(2, replica1, network);
 
   {
     Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
@@ -826,13 +846,13 @@ TEST_F(CoordinatorTest, Truncate)
   }
 
   {
-    Future<std::list<Action> > actions = replica1.read(6, 10);
+    Future<list<Action> > actions = replica1->read(6, 10);
     AWAIT_FAILED(actions);
     EXPECT_EQ("Bad read range (truncated position)", actions.failure());
   }
 
   {
-    Future<std::list<Action> > actions = replica1.read(7, 10);
+    Future<list<Action> > actions = replica1->read(7, 10);
     AWAIT_READY(actions);
     EXPECT_EQ(4u, actions.get().size());
     foreach (const Action& action, actions.get()) {
@@ -846,21 +866,24 @@ TEST_F(CoordinatorTest, Truncate)
 
 TEST_F(CoordinatorTest, TruncateNotLearnedFill)
 {
-  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, _);
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
 
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
-  const std::string path3 = os::getcwd() + "/.log3";
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  // Drop messages here in order to obtain the pid of replica2. We
+  // only want to drop learned message sent to replica2.
+  DROP_MESSAGES(Eq(LearnedMessage().GetTypeName()), _, Eq(replica2->pid()));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -881,14 +904,15 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
     EXPECT_EQ(11u, result.get());
   }
 
-  Replica replica3(path3);
+  Shared<Replica> replica3(new Replica(path3));
 
-  Network network2;
+  pids.clear();
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
 
-  network2.add(replica2.pid());
-  network2.add(replica3.pid());
+  Shared<Network> network2(new Network(pids));
 
-  Coordinator coord2(2, &replica3, &network2);
+  Coordinator coord2(2, replica3, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -899,13 +923,13 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(6, 10);
+    Future<list<Action> > actions = replica3->read(6, 10);
     AWAIT_FAILED(actions);
     EXPECT_EQ("Bad read range (truncated position)", actions.failure());
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(7, 10);
+    Future<list<Action> > actions = replica3->read(7, 10);
     AWAIT_READY(actions);
     EXPECT_EQ(4u, actions.get().size());
     foreach (const Action& action, actions.get()) {
@@ -919,19 +943,20 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
 
 TEST_F(CoordinatorTest, TruncateLearnedFill)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
-  const std::string path3 = os::getcwd() + "/.log3";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
 
-  Replica replica1(path1);
-  Replica replica2(path2);
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
 
-  Network network1;
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
 
-  network1.add(replica1.pid());
-  network1.add(replica2.pid());
+  Shared<Network> network1(new Network(pids));
 
-  Coordinator coord1(2, &replica1, &network1);
+  Coordinator coord1(2, replica1, network1);
 
   {
     Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
@@ -952,14 +977,15 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
     EXPECT_EQ(11u, result.get());
   }
 
-  Replica replica3(path3);
+  Shared<Replica> replica3(new Replica(path3));
 
-  Network network2;
+  pids.clear();
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
 
-  network2.add(replica2.pid());
-  network2.add(replica3.pid());
+  Shared<Network> network2(new Network(pids));
 
-  Coordinator coord2(2, &replica3, &network2);
+  Coordinator coord2(2, replica3, network2);
 
   {
     Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
@@ -970,13 +996,13 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(6, 10);
+    Future<list<Action> > actions = replica3->read(6, 10);
     AWAIT_FAILED(actions);
     EXPECT_EQ("Bad read range (truncated position)", actions.failure());
   }
 
   {
-    Future<std::list<Action> > actions = replica3.read(7, 10);
+    Future<list<Action> > actions = replica3->read(7, 10);
     AWAIT_READY(actions);
     EXPECT_EQ(4u, actions.get().size());
     foreach (const Action& action, actions.get()) {
@@ -993,12 +1019,12 @@ class LogTest : public TemporaryDirectoryTest {};
 
 TEST_F(LogTest, WriteRead)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
   Replica replica1(path1);
 
-  std::set<UPID> pids;
+  set<UPID> pids;
   pids.insert(replica1.pid());
 
   Log log(2, path2, pids);
@@ -1012,7 +1038,7 @@ TEST_F(LogTest, WriteRead)
 
   Log::Reader reader(&log);
 
-  Result<std::list<Log::Entry> > entries =
+  Result<list<Log::Entry> > entries =
     reader.read(position.get(), position.get(), Timeout::in(Seconds(10)));
 
   ASSERT_SOME(entries);
@@ -1024,12 +1050,12 @@ TEST_F(LogTest, WriteRead)
 
 TEST_F(LogTest, Position)
 {
-  const std::string path1 = os::getcwd() + "/.log1";
-  const std::string path2 = os::getcwd() + "/.log2";
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
 
   Replica replica1(path1);
 
-  std::set<UPID> pids;
+  set<UPID> pids;
   pids.insert(replica1.pid());
 
   Log log(2, path2, pids);


[05/10] git commit: Added log recovery support.

Posted by be...@apache.org.
Added log recovery support.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/15802


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/f9b60c4c
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/f9b60c4c
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/f9b60c4c

Branch: refs/heads/master
Commit: f9b60c4cac503b2d7a1c0cb5403203e619e563e6
Parents: 6ea7c14
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:53:40 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:53:40 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am           |    3 +
 src/common/type_utils.hpp |   25 +
 src/log/coordinator.cpp   |  121 +---
 src/log/coordinator.hpp   |   42 +-
 src/log/log.cpp           | 1192 ++++++++++++++++++++++++----------------
 src/log/log.hpp           |  283 ++--------
 src/log/recover.cpp       |  403 ++++++++++++++
 src/log/recover.hpp       |   59 ++
 src/log/replica.cpp       |  214 ++++++--
 src/log/replica.hpp       |   13 +-
 src/messages/log.proto    |   18 +-
 src/tests/log_tests.cpp   |  419 ++++++++------
 12 files changed, 1727 insertions(+), 1065 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 17fbf83..60fcb31 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -301,11 +301,14 @@ liblog_la_SOURCES =							\
   log/catchup.cpp							\
   log/consensus.cpp							\
   log/coordinator.cpp							\
+  log/log.cpp								\
+  log/recover.cpp							\
   log/replica.cpp
 liblog_la_SOURCES +=							\
   log/catchup.hpp							\
   log/consensus.hpp							\
   log/coordinator.hpp							\
+  log/recover.hpp							\
   log/replica.hpp							\
   log/log.hpp								\
   log/network.hpp							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/common/type_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.hpp b/src/common/type_utils.hpp
index 3b05751..fe6bf71 100644
--- a/src/common/type_utils.hpp
+++ b/src/common/type_utils.hpp
@@ -30,6 +30,7 @@
 
 #include "common/attributes.hpp"
 
+#include "messages/log.hpp"
 #include "messages/messages.hpp"
 
 // This file includes definitions for operators on protobuf classes
@@ -371,4 +372,28 @@ inline std::ostream& operator << (
 
 }} // namespace mesos { namespace internal {
 
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+inline std::ostream& operator << (
+    std::ostream& stream,
+    const Action::Type& type)
+{
+  return stream << Action::Type_Name(type);
+}
+
+
+inline std::ostream& operator << (
+    std::ostream& stream,
+    const Metadata::Status& status)
+{
+  return stream << Metadata::Status_Name(status);
+}
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos
+
 #endif // __TYPE_UTILS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/coordinator.cpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.cpp b/src/log/coordinator.cpp
index 21f2865..bc85e66 100644
--- a/src/log/coordinator.cpp
+++ b/src/log/coordinator.cpp
@@ -20,13 +20,13 @@
 
 #include <process/defer.hpp>
 #include <process/dispatch.hpp>
-#include <process/future.hpp>
 #include <process/id.hpp>
 #include <process/process.hpp>
 
-#include <stout/error.hpp>
 #include <stout/none.hpp>
 
+#include "common/type_utils.hpp"
+
 #include "log/catchup.hpp"
 #include "log/consensus.hpp"
 #include "log/coordinator.hpp"
@@ -59,22 +59,10 @@ public:
 
   virtual ~CoordinatorProcess() {}
 
-  // Handles coordinator election. Returns the last committed log
-  // position if the operation succeeds. Returns none if the election
-  // is not successful, but can be retried.
+  // See comments in 'coordinator.hpp'.
   Future<Option<uint64_t> > elect();
-
-  // Handles coordinator demotion. Returns the last committed log
-  // position if the operation succeeds.
   Future<uint64_t> demote();
-
-  // Appends the specified bytes to the end of the log. Returns the
-  // position of the appended entry if the operation succeeds.
   Future<uint64_t> append(const string& bytes);
-
-  // Removes all log entries preceding the log entry at the given
-  // position (to). Returns the position at which the truncate
-  // operation is written if the operation succeeds.
   Future<uint64_t> truncate(uint64_t to);
 
 protected:
@@ -344,8 +332,7 @@ Future<uint64_t> CoordinatorProcess::truncate(uint64_t to)
 
 Future<uint64_t> CoordinatorProcess::write(const Action& action)
 {
-  LOG(INFO) << "Coordinator attempting to write "
-            << Action::Type_Name(action.type())
+  LOG(INFO) << "Coordinator attempting to write " << action.type()
             << " action at position " << action.position();
 
   CHECK_EQ(state, ELECTED);
@@ -455,111 +442,27 @@ Coordinator::~Coordinator()
 }
 
 
-Result<uint64_t> Coordinator::elect(const Timeout& timeout)
+Future<Option<uint64_t> > Coordinator::elect()
 {
-  LOG(INFO) << "Coordinator attempting to get elected within "
-            << timeout.remaining();
-
-  Future<Option<uint64_t> > electing =
-    dispatch(process, &CoordinatorProcess::elect);
-
-  electing.await(timeout.remaining());
-
-  CHECK(!electing.isDiscarded());
-
-  if (electing.isPending()) {
-    LOG(INFO) << "Coordinator timed out while trying to get elected";
-
-    electing.discard();
-    return None();
-  } else if (electing.isFailed()) {
-    LOG(ERROR) << "Coordinator failed to get elected: "
-               << electing.failure();
-
-    return Error(electing.failure());
-  } else {
-    if (electing.get().isNone()) {
-      LOG(INFO) << "Coordinator lost an election, but can be retried";
-
-      return None();
-    } else {
-      LOG(INFO) << "Coordinator elected with current position "
-                << electing.get().get();
-
-      return electing.get().get();
-    }
-  }
+  return dispatch(process, &CoordinatorProcess::elect);
 }
 
 
-Result<uint64_t> Coordinator::demote()
+Future<uint64_t> Coordinator::demote()
 {
-  Future<uint64_t> demoting =
-    dispatch(process, &CoordinatorProcess::demote);
-
-  demoting.await(); // TODO(jieyu): Use a timeout.
-
-  CHECK(!demoting.isDiscarded());
-
-  if (demoting.isFailed()) {
-    return Error(demoting.failure());
-  } else {
-    return demoting.get();
-  }
+  return dispatch(process, &CoordinatorProcess::demote);
 }
 
 
-Result<uint64_t> Coordinator::append(
-    const string& bytes,
-    const Timeout& timeout)
+Future<uint64_t> Coordinator::append(const string& bytes)
 {
-  Future<uint64_t> appending =
-    dispatch(process, &CoordinatorProcess::append, bytes);
-
-  appending.await(timeout.remaining());
-
-  CHECK(!appending.isDiscarded());
-
-  if (appending.isPending()) {
-    LOG(INFO) << "Coordinator timed out while trying to append";
-
-    appending.discard();
-    return None();
-  } else if (appending.isFailed()) {
-    LOG(ERROR) << "Coordinator failed to append the log: "
-               << appending.failure();
-
-    return Error(appending.failure());
-  } else {
-    return appending.get();
-  }
+  return dispatch(process, &CoordinatorProcess::append, bytes);
 }
 
 
-Result<uint64_t> Coordinator::truncate(
-    uint64_t to,
-    const Timeout& timeout)
+Future<uint64_t> Coordinator::truncate(uint64_t to)
 {
-  Future<uint64_t> truncating =
-    dispatch(process, &CoordinatorProcess::truncate, to);
-
-  truncating.await(timeout.remaining());
-
-  CHECK(!truncating.isDiscarded());
-
-  if (truncating.isPending()) {
-    LOG(INFO) << "Coordinator timed out while trying to truncate";
-
-    truncating.discard();
-    return None();
-  } else if (truncating.isFailed()) {
-    LOG(ERROR) << "Coordinator failed to truncate the log: "
-               << truncating.failure();
-
-    return Error(truncating.failure());
-  } else {
-    return truncating.get();
-  }
+  return dispatch(process, &CoordinatorProcess::truncate, to);
 }
 
 } // namespace log {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/coordinator.hpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.hpp b/src/log/coordinator.hpp
index 43cb530..35b68e9 100644
--- a/src/log/coordinator.hpp
+++ b/src/log/coordinator.hpp
@@ -23,10 +23,10 @@
 
 #include <string>
 
+#include <process/future.hpp>
 #include <process/shared.hpp>
-#include <process/timeout.hpp>
 
-#include <stout/result.hpp>
+#include <stout/option.hpp>
 
 #include "log/network.hpp"
 #include "log/replica.hpp"
@@ -49,25 +49,25 @@ public:
 
   ~Coordinator();
 
-  // Handles coordinator election/demotion. A result of none means the
-  // coordinator failed to achieve a quorum (e.g., due to timeout) but
-  // can be retried. A some result returns the last committed log
-  // position.
-  Result<uint64_t> elect(const process::Timeout& timeout);
-  Result<uint64_t> demote();
-
-  // Returns the result of trying to append the specified bytes. A
-  // result of none means the append failed (e.g., due to timeout),
-  // but can be retried.
-  Result<uint64_t> append(
-      const std::string& bytes,
-      const process::Timeout& timeout);
-
-  // Returns the result of trying to truncate the log (from the
-  // beginning to the specified position exclusive). A result of
-  // none means the truncate failed (e.g., due to timeout), but can be
-  // retried.
-  Result<uint64_t> truncate(uint64_t to, const process::Timeout& timeout);
+  // Handles coordinator election. Returns the last committed (a.k.a.,
+  // learned) log position if the operation succeeds. Returns none if
+  // the election is not successful, but can be retried.
+  process::Future<Option<uint64_t> > elect();
+
+  // Handles coordinator demotion. Returns the last committed (a.k.a.,
+  // learned) log position if the operation succeeds. One should only
+  // call this function if the coordinator has been elected, and no
+  // write (append or truncate) is in progress.
+  process::Future<uint64_t> demote();
+
+  // Appends the specified bytes to the end of the log. Returns the
+  // position of the appended entry if the operation succeeds.
+  process::Future<uint64_t> append(const std::string& bytes);
+
+  // Removes all log entries preceding the log entry at the given
+  // position (to). Returns the position at which the truncate
+  // operation is written if the operation succeeds.
+  process::Future<uint64_t> truncate(uint64_t to);
 
 private:
   CoordinatorProcess* process;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/log.cpp
----------------------------------------------------------------------
diff --git a/src/log/log.cpp b/src/log/log.cpp
index d057925..e83f822 100644
--- a/src/log/log.cpp
+++ b/src/log/log.cpp
@@ -16,639 +16,893 @@
  * limitations under the License.
  */
 
-// TODO(benh): Optimize LearnedMessage (and the "commit" stage in
-// general) by figuring out a way to not send the entire action
-// contents a second time (should cut bandwidth used in half).
-
-// TODO(benh): Provide a LearnRequest that requests more than one
-// position at a time, and a LearnResponse that returns as many
-// positions as it knows.
-
-// TODO(benh): Implement background catchup: have a new replica that
-// comes online become part of the group but don't respond to promises
-// or writes until it has caught up! The advantage to becoming part of
-// the group is that the new replica can see where the end of the log
-// is in order to continue to catch up.
-
-// TODO(benh): Add tests that deliberatly put the system in a state of
-// inconsistency by doing funky things to the underlying logs. Figure
-// out ways of bringing new replicas online that seem to check the
-// consistency of the other replicas.
-
-#include <list>
-#include <map>
-#include <set>
-#include <string>
-#include <vector>
-
-#include <boost/lexical_cast.hpp>
-
+#include <process/defer.hpp>
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
 #include <process/process.hpp>
-#include <process/run.hpp>
+#include <process/shared.hpp>
 
 #include <stout/check.hpp>
-#include <stout/duration.hpp>
-#include <stout/fatal.hpp>
+#include <stout/error.hpp>
 #include <stout/foreach.hpp>
-#include <stout/os.hpp>
-#include <stout/result.hpp>
-
-#include "zookeeper/zookeeper.hpp"
+#include <stout/lambda.hpp>
+#include <stout/nothing.hpp>
+#include <stout/set.hpp>
 
 #include "log/coordinator.hpp"
+#include "log/log.hpp"
+#include "log/network.hpp"
+#include "log/recover.hpp"
 #include "log/replica.hpp"
 
-using namespace mesos;
-using namespace mesos::internal;
-using namespace mesos::internal::log;
-
 using namespace process;
 
 using std::list;
-using std::map;
-using std::pair;
 using std::set;
 using std::string;
-using std::vector;
 
+namespace mesos {
+namespace internal {
+namespace log {
 
-// class Drop : public Filter
-// {
-// public:
-//   Drop
-//   virtual bool filter(Message* message)
-//   {
-//     return  == message->name;
-//   }
-// };
+class LogProcess : public Process<LogProcess>
+{
+public:
+  LogProcess(
+      size_t _quorum,
+      const string& path,
+      const set<UPID>& pids);
+
+  LogProcess(
+      size_t _quorum,
+      const string& path,
+      const string& servers,
+      const Duration& timeout,
+      const string& znode,
+      const Option<zookeeper::Authentication>& auth);
+
+  // Recovers the log by catching up if needed. Returns a shared
+  // pointer to the local replica if the recovery succeeds.
+  Future<Shared<Replica> > recover();
 
+protected:
+  virtual void initialize();
+  virtual void finalize();
 
-// class PeriodicFilter
+private:
+  friend class LogReaderProcess;
+  friend class LogWriterProcess;
 
+  // Continuations.
+  void _recover();
 
-char** args; // Command line arguments for doing a restart.
+  // TODO(benh): Factor this out into "membership renewer".
+  void watch(
+      const UPID& pid,
+      const set<zookeeper::Group::Membership>& memberships);
 
+  void failed(const string& message);
+  void discarded();
 
-void restart()
-{
-  LOG(INFO) << "Restarting ...";
-  execv(args[0], args);
-  fatalerror("Failed to exec");
-}
+  const size_t quorum;
+  Shared<Replica> replica;
+  Shared<Network> network;
 
+  // For replica recovery.
+  Option<Future<Owned<Replica> > > recovering;
+  process::Promise<Nothing> recovered;
+  list<process::Promise<Shared<Replica> >*> promises;
 
-bool coordinate(Coordinator* coordinator,
-                uint64_t id,
-                int end,
-                map<int, int> truncations)
+  // For renewing membership. We store a Group instance in order to
+  // continually renew the replicas membership (when using ZooKeeper).
+  zookeeper::Group* group;
+  Future<zookeeper::Group::Membership> membership;
+};
+
+
+class LogReaderProcess : public Process<LogReaderProcess>
 {
-  const int attempts = 3;
+public:
+  LogReaderProcess(Log* log);
 
-  uint64_t index;
+  Future<Log::Position> beginning();
+  Future<Log::Position> ending();
 
-  int attempt = 1;
-  while (true) {
-    Result<uint64_t> result = coordinator->elect(id);
-    if (result.isError()) {
-      restart();
-    } else if (result.isNone()) {
-      if (attempt == attempts) {
-        restart();
-      } else {
-        attempt++;
-        os::sleep(Seconds(1));
-      }
-    } else {
-      CHECK_SOME(result);
-      index = result.get();
-      break;
-    }
-  }
+  Future<list<Log::Entry> > read(
+      const Log::Position& from,
+      const Log::Position& to);
 
-  uint64_t value = 0;
-
-  if (index != 0) {
-    attempt = 1;
-    while (true) {
-      Result<list<pair<uint64_t, string> > > result =
-        coordinator->read(index, index);
-      if (result.isError()) {
-        LOG(INFO) << "Restarting due to read error";
-        restart();
-      } else if (result.isNone()) {
-        if (attempt == attempts) {
-          LOG(INFO) << "Restarting after too many attempts";
-          restart();
-        } else {
-          attempt++;
-          os::sleep(Seconds(1));
-        }
-      } else {
-        CHECK_SOME(result);
-        const list<pair<uint64_t, string> >& list = result.get();
-        if (list.size() != 1) {
-          index--;
-        } else {
-          try {
-            value = boost::lexical_cast<uint64_t>(list.front().second);
-          } catch (boost::bad_lexical_cast&) {
-            LOG(INFO) << "Restarting due to conversion error";
-            restart();
-          }
-          break;
-        }
-      }
-    }
-  }
+protected:
+  virtual void initialize();
+  virtual void finalize();
 
-  value++;
-
-  srand(time(NULL));
-
-  int writes = rand() % 500;
-
-  LOG(INFO) << "Attempting to do " << writes << " writes";
-
-  attempt = 1;
-  while (writes > 0 && value <= end) {
-    if (truncations.count(value) > 0) {
-      int to = truncations[value];
-      Result<uint64_t> result = coordinator->truncate(to);
-      if (result.isError()) {
-        LOG(INFO) << "Restarting due to truncate error";
-        restart();
-      } else if (result.isNone()) {
-        if (attempt == attempts) {
-          LOG(INFO) << "Restarting after too many attempts";
-          restart();
-        } else {
-          attempt++;
-          os::sleep(Seconds(1));
-          continue;
-        }
-      } else {
-        CHECK_SOME(result);
-        LOG(INFO) << "Truncated to " << to;
-        os::sleep(Seconds(1));
-        attempt = 1;
-      }
-    }
+private:
+  // Returns a position from a raw value.
+  static Log::Position position(uint64_t value);
 
-    Result<uint64_t> result = coordinator->append(stringify(value));
-    if (result.isError()) {
-      LOG(INFO) << "Restarting due to append error";
-      restart();
-    } else if (result.isNone()) {
-      if (attempt == attempts) {
-        LOG(INFO) << "Restarting after too many attempts";
-        restart();
-      } else {
-        attempt++;
-        os::sleep(Seconds(1));
-      }
-    } else {
-      CHECK_SOME(result);
-      LOG(INFO) << "Wrote " << value;
-      os::sleep(Seconds(1));
-      writes--;
-      value++;
-      attempt = 1;
-    }
-  }
+  // Returns a future which gets set when the log recovery has
+  // finished (either succeeded or failed).
+  Future<Nothing> recover();
 
-  exit(0);
-  return true;
-}
+  // Continuations.
+  void _recover();
 
+  Future<Log::Position> _beginning();
+  Future<Log::Position> _ending();
 
-class LogProcess : public Process<LogProcess>
+  Future<list<Log::Entry> > _read(
+      const Log::Position& from,
+      const Log::Position& to);
+
+  Future<list<Log::Entry> > __read(
+      const Log::Position& from,
+      const Log::Position& to,
+      const list<Action>& actions);
+
+  Future<Shared<Replica> > recovering;
+  list<process::Promise<Nothing>*> promises;
+};
+
+
+class LogWriterProcess : public Process<LogWriterProcess>
 {
 public:
-  LogProcess(int _quorum,
-             const string& _file,
-             const string& _servers,
-             const string& _znode,
-             int _end,
-             const map<int, int>& _truncations);
-
-  virtual ~LogProcess();
-
-  // ZooKeeper events. TODO(*): Use a ZooKeeper listener?
-  void connected();
-  void reconnecting();
-  void reconnected();
-  void expired();
-  void updated(const string& path);
+  LogWriterProcess(Log* log);
+
+  Future<Option<Log::Position> > elect();
+  Future<Log::Position> append(const string& bytes);
+  Future<Log::Position> truncate(const Log::Position& to);
 
 protected:
-  virtual void initialze();
+  virtual void initialize();
+  virtual void finalize();
 
 private:
-  // Updates the group.
-  void regroup();
+  // Returns a position from a raw value.
+  static Log::Position position(uint64_t value);
 
-  // Runs an election.
-  void elect();
+  // Returns a future which gets set when the log recovery has
+  // finished (either succeeded or failed).
+  Future<Nothing> recover();
 
-  // ZooKeeper bits and pieces.
-  string servers;
-  string znode;
-  ZooKeeper* zk;
-  Watcher* watcher;
+  // Continuations.
+  void _recover();
 
-  // Size of quorum.
-  int quorum;
+  Future<Option<Log::Position> > _elect();
+  Option<Log::Position> __elect(const Option<uint64_t>& result);
 
-  // Log file.
-  string file;
+  void failed(const string& message);
 
-  // Termination value (when to stop writing to the log).
-  int end;
+  const size_t quorum;
+  const Shared<Network> network;
 
-  // Truncation points.
-  map<int, int> truncations;
+  Future<Shared<Replica> > recovering;
+  list<process::Promise<Nothing>*> promises;
 
-  // Coordinator id.
-  uint64_t id;
+  Coordinator* coordinator;
+  Option<string> error;
+};
 
-  // Whether or not the coordinator has been elected.
-  bool elected;
 
-  // Group members.
-  set<UPID> members;
+/////////////////////////////////////////////////
+// Implementation of LogProcess.
+/////////////////////////////////////////////////
 
-  ReplicaProcess* replica;
-  GroupProcess* group;
-  Coordinator* coordinator;
-};
+
+LogProcess::LogProcess(
+    size_t _quorum,
+    const string& path,
+    const set<UPID>& pids)
+  : ProcessBase(ID::generate("log")),
+    quorum(_quorum),
+    replica(new Replica(path)),
+    network(new Network(pids + (UPID) replica->pid())),
+    group(NULL) {}
+
+
+LogProcess::LogProcess(
+    size_t _quorum,
+    const string& path,
+    const string& servers,
+    const Duration& timeout,
+    const string& znode,
+    const Option<zookeeper::Authentication>& auth)
+  : ProcessBase(ID::generate("log")),
+    quorum(_quorum),
+    replica(new Replica(path)),
+    network(new ZooKeeperNetwork(servers, timeout, znode, auth)),
+    group(new zookeeper::Group(servers, timeout, znode, auth)) {}
 
 
-class LogProcessWatcher : public Watcher
+void LogProcess::initialize()
 {
-public:
-  LogProcessWatcher(const PID<LogProcess>& _pid)
-    : pid(_pid), reconnect(false) {}
-
-  virtual ~LogProcessWatcher() {}
-
-  virtual void process(ZooKeeper* zk, int type, int state, const string& path)
-  {
-    if ((state == ZOO_CONNECTED_STATE) && (type == ZOO_SESSION_EVENT)) {
-      // Check if this is a reconnect.
-      if (!reconnect) {
-        // Initial connect.
-        dispatch(pid, &LogProcess::connected);
-      } else {
-        // Reconnected.
-        dispatch(pid, &LogProcess::reconnected);
-      }
-    } else if ((state == ZOO_CONNECTING_STATE) &&
-               (type == ZOO_SESSION_EVENT)) {
-      // The client library automatically reconnects, taking into
-      // account failed servers in the connection string,
-      // appropriately handling the "herd effect", etc.
-      reconnect = true;
-      dispatch(pid, &LogProcess::reconnecting);
-    } else if ((state == ZOO_EXPIRED_SESSION_STATE) &&
-               (type == ZOO_SESSION_EVENT)) {
-      dispatch(pid, &LogProcess::expired);
-
-      // If this watcher is reused, the next connect won't be a reconnect.
-      reconnect = false;
-    } else if ((state == ZOO_CONNECTED_STATE) && (type == ZOO_CHILD_EVENT)) {
-      dispatch(pid, &LogProcess::updated, path);
-    } else if ((state == ZOO_CONNECTED_STATE) && (type == ZOO_CHANGED_EVENT)) {
-      dispatch(pid, &LogProcess::updated, path);
-    } else {
-      LOG(FATAL) << "Unimplemented ZooKeeper event: (state is "
-                 << state << " and type is " << type << ")";
-    }
+  if (group != NULL) {
+    // Need to add our replica to the ZooKeeper group!
+    LOG(INFO) << "Attempting to join replica to ZooKeeper group";
+
+    membership = group->join(replica->pid())
+      .onFailed(defer(self(), &Self::failed, lambda::_1))
+      .onDiscarded(defer(self(), &Self::discarded));
+
+    // We save and pass the pid of the replica to the 'watch' function
+    // because the field member 'replica' is not available during
+    // recovery. We need the pid to renew the replicas membership.
+    group->watch()
+      .onReady(defer(self(), &Self::watch, replica->pid(), lambda::_1))
+      .onFailed(defer(self(), &Self::failed, lambda::_1))
+      .onDiscarded(defer(self(), &Self::discarded));
   }
 
-private:
-  const PID<LogProcess> pid;
-  bool reconnect;
-};
+  // Start the recovery.
+  recover();
+}
+
+
+void LogProcess::finalize()
+{
+  if (recovering.isSome()) {
+    // Stop the recovery if it is still pending.
+    recovering.get().discard();
+  }
 
+  // If there exist operations that are gated by the recovery, we fail
+  // all of them because the log is being deleted.
+  foreach (process::Promise<Shared<Replica> >* promise, promises) {
+    promise->fail("Log is being deleted");
+    delete promise;
+  }
+  promises.clear();
 
-LogProcess::LogProcess(int _quorum,
-                       const string& _file,
-                       const string& _servers,
-                       const string& _znode,
-                       int _end,
-                       const map<int, int>& _truncations)
-  : quorum(_quorum),
-    file(_file),
-    servers(_servers),
-    znode(_znode),
-    end(_end),
-    truncations(_truncations),
-    id(0),
-    elected(false),
-    replica(NULL),
-    group(NULL),
-    coordinator(NULL) {}
-
-
-LogProcess::~LogProcess()
-{
-  delete zk;
-  delete watcher;
-  delete replica;
   delete group;
-  delete coordinator;
+
+  // Wait for the shared pointers 'network' and 'replica' to become
+  // unique (i.e., no other reference to them). These calls should not
+  // be blocking for too long because at this moment, all operations
+  // should have been cancelled or are being cancelled. We do this
+  // because we want to make sure that after the log is deleted, all
+  // operations associated with this log are terminated.
+  network.own().await();
+  replica.own().await();
 }
 
 
-void LogProcess::connected()
+Future<Shared<Replica> > LogProcess::recover()
 {
-  LOG(INFO) << "Log connected to ZooKeeper";
+  // The future 'recovered' is used to mark the success (or the
+  // failure) of the recovery. We do not use the future 'recovering'
+  // to do that because it can be set in other process and thus has a
+  // race condition which we want to avoid. We deliberately do not
+  // save replica in 'recovered' because it will complicate our
+  // deleting logic (see 'finalize').
+  Future<Nothing> future = recovered.future();
+
+  if (future.isDiscarded()) {
+    return Failure("Not expecting discarded future");
+  } else if (future.isFailed()) {
+    return Failure(future.failure());
+  } else if (future.isReady()) {
+    return replica;
+  }
+
+  // Recovery has not finished yet. Create a promise and queue it such
+  // that it can get notified once the recovery has finished (either
+  // succeeded or failed).
+  process::Promise<Shared<Replica> >* promise =
+    new process::Promise<Shared<Replica> >();
+
+  promises.push_back(promise);
+
+  if (recovering.isNone()) {
+    // TODO(jieyu): At this moment, we haven't shared 'replica' to
+    // others yet. Therefore, the following 'replica.own()' call
+    // should not be blocking. In the future, we may wanna support
+    // 'release' in Shared which will provide this CHECK internally.
+    CHECK(replica.unique());
+
+    recovering = log::recover(quorum, replica.own().get(), network)
+      .onAny(defer(self(), &Self::_recover));
+  }
+
+  return promise->future();
+}
+
 
-  int ret;
-  string result;
+void LogProcess::_recover()
+{
+  CHECK_SOME(recovering);
 
-  // Assume the znode that was created does not end with a "/".
-  CHECK(znode.size() == 0 || znode.at(znode.size() - 1) != '/');
+  Future<Owned<Replica> > future = recovering.get();
 
-  // Create directory path znodes as necessary.
-  size_t index = znode.find("/", 0);
+  if (!future.isReady()) {
+    // The 'future' here can only be discarded in 'finalize'.
+    string failure = future.isFailed() ?
+      future.failure() :
+      "The future 'recovering' is unexpectedly discarded";
 
-  while (index < string::npos) {
-    // Get out the prefix to create.
-    index = znode.find("/", index + 1);
-    string prefix = znode.substr(0, index);
+    // Mark the failure of the recovery.
+    recovered.fail(failure);
 
-    LOG(INFO) << "Log trying to create znode '"
-              << prefix << "' in ZooKeeper";
+    foreach (process::Promise<Shared<Replica> >* promise, promises) {
+      promise->fail(failure);
+      delete promise;
+    }
+    promises.clear();
+  } else {
+    replica = future.get().share();
 
-    // Create the node (even if it already exists).
-    ret = zk->create(
-        prefix,
-        "",
-        ZOO_OPEN_ACL_UNSAFE,
-        // ZOO_CREATOR_ALL_ACL, // needs authentication
-        0,
-        &result);
+    // Mark the success of the recovery.
+    recovered.set(Nothing());
 
-    if (ret != ZOK && ret != ZNODEEXISTS) {
-      LOG(FATAL) << "Failed to create '" << prefix
-                 << "' in ZooKeeper: " << zk->message(ret);
+    foreach (process::Promise<Shared<Replica> >* promise, promises) {
+      promise->set(replica);
+      delete promise;
     }
+    promises.clear();
   }
+}
 
-  // Now create the "replicas" znode.
-  LOG(INFO) << "Log trying to create znode '" << znode
-            << "/replicas" << "' in ZooKeeper";
 
-  // Create the node (even if it already exists).
-  ret = zk->create(znode + "/replicas", "", ZOO_OPEN_ACL_UNSAFE,
-                   // ZOO_CREATOR_ALL_ACL, // needs authentication
-                   0, &result);
+void LogProcess::watch(
+    const UPID& pid,
+    const set<zookeeper::Group::Membership>& memberships)
+{
+  if (membership.isReady() && memberships.count(membership.get()) == 0) {
+    // Our replica's membership must have expired, join back up.
+    LOG(INFO) << "Renewing replica group membership";
 
-  if (ret != ZOK && ret != ZNODEEXISTS) {
-    LOG(FATAL) << "Failed to create '" << znode << "/replicas"
-               << "' in ZooKeeper: " << zk->message(ret);
+    membership = group->join(pid)
+      .onFailed(defer(self(), &Self::failed, lambda::_1))
+      .onDiscarded(defer(self(), &Self::discarded));
   }
 
-  // Now create the "coordinators" znode.
-  LOG(INFO) << "Log trying to create znode '" << znode
-            << "/coordinators" << "' in ZooKeeper";
+  group->watch(memberships)
+    .onReady(defer(self(), &Self::watch, pid, lambda::_1))
+    .onFailed(defer(self(), &Self::failed, lambda::_1))
+    .onDiscarded(defer(self(), &Self::discarded));
+}
 
-  // Create the node (even if it already exists).
-  ret = zk->create(znode + "/coordinators", "", ZOO_OPEN_ACL_UNSAFE,
-                   // ZOO_CREATOR_ALL_ACL, // needs authentication
-                   0, &result);
 
-  if (ret != ZOK && ret != ZNODEEXISTS) {
-    LOG(FATAL) << "Failed to create '" << znode << "/coordinators"
-               << "' in ZooKeeper: " << zk->message(ret);
-  }
+void LogProcess::failed(const string& message)
+{
+  LOG(FATAL) << "Failed to participate in ZooKeeper group: " << message;
+}
 
-  // Okay, create our replica, group, and coordinator.
-  replica = new ReplicaProcess(file);
-  spawn(replica);
 
-  group = new GroupProcess();
-  spawn(group);
+void LogProcess::discarded()
+{
+  LOG(FATAL) << "Not expecting future to get discarded!";
+}
 
-  coordinator = new Coordinator(quorum, replica, group);
 
-  // Set a watch on the replicas.
-  ret = zk->getChildren(znode + "/replicas", true, NULL);
+/////////////////////////////////////////////////
+// Implementation of LogReaderProcess.
+/////////////////////////////////////////////////
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to set a watch on '" << znode << "/replicas"
-               << "' in ZooKeeper: " << zk->message(ret);
-  }
 
-  // Set a watch on the coordinators.
-  ret = zk->getChildren(znode + "/coordinators", true, NULL);
+LogReaderProcess::LogReaderProcess(Log* log)
+  : ProcessBase(ID::generate("log-reader")),
+    recovering(dispatch(log->process, &LogProcess::recover)) {}
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to set a watch on '" << znode << "/replicas"
-               << "' in ZooKeeper: " << zk->message(ret);
-  }
 
-  // Add an ephemeral znode for our replica and coordinator.
-  ret = zk->create(znode + "/replicas/", replica->self(), ZOO_OPEN_ACL_UNSAFE,
-                   // ZOO_CREATOR_ALL_ACL, // needs authentication
-                   ZOO_SEQUENCE | ZOO_EPHEMERAL, &result);
+void LogReaderProcess::initialize()
+{
+  recovering.onAny(defer(self(), &Self::_recover));
+}
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to create an ephmeral node at '" << znode
-               << "/replica/" << "' in ZooKeeper: " << zk->message(ret);
+
+void LogReaderProcess::finalize()
+{
+  foreach (process::Promise<Nothing>* promise, promises) {
+    promise->fail("Log reader is being deleted");
+    delete promise;
   }
+  promises.clear();
+}
 
-  ret = zk->create(znode + "/coordinators/", "", ZOO_OPEN_ACL_UNSAFE,
-                   // ZOO_CREATOR_ALL_ACL, // needs authentication
-                   ZOO_SEQUENCE | ZOO_EPHEMERAL, &result);
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to create an ephmeral node at '" << znode
-               << "/replica/" << "' in ZooKeeper: " << zk->message(ret);
+Future<Nothing> LogReaderProcess::recover()
+{
+  if (recovering.isReady()) {
+    return Nothing();
+  } else if (recovering.isFailed()) {
+    return Failure(recovering.failure());
+  } else if (recovering.isDiscarded()) {
+    return Failure("The future 'recovering' is unexpectedly discarded");
   }
 
-  // Save the sequence id but only grab the basename, e.g.,
-  // "/path/to/znode/000000131" => "000000131".
-  result = utils::os::basename(result);
+  // At this moment, the future 'recovering' should most likely be
+  // pending. But it is also likely that it gets set after the above
+  // checks. Either way, we know that the continuation '_recover' has
+  // not been called yet (otherwise, we should not be able to reach
+  // here). The promise we are creating below will be properly
+  // set/failed when '_recover' is called.
+  process::Promise<Nothing>* promise = new process::Promise<Nothing>();
+  promises.push_back(promise);
+  return promise->future();
+}
+
 
-  try {
-    id = boost::lexical_cast<uint64_t>(result);
-  } catch (boost::bad_lexical_cast&) {
-    LOG(FATAL) << "Failed to convert '" << result << "' into an integer";
+void LogReaderProcess::_recover()
+{
+  if (!recovering.isReady()) {
+    foreach (process::Promise<Nothing>* promise, promises) {
+      promise->fail(
+          recovering.isFailed() ?
+          recovering.failure() :
+          "The future 'recovering' is unexpectedly discarded");
+      delete promise;
+    }
+    promises.clear();
+  } else {
+    foreach (process::Promise<Nothing>* promise, promises) {
+      promise->set(Nothing());
+      delete promise;
+    }
+    promises.clear();
   }
+}
+
 
-  // Run an election!
-  elect();
+Future<Log::Position> LogReaderProcess::beginning()
+{
+  return recover().then(defer(self(), &Self::_beginning));
 }
 
 
-void LogProcess::reconnecting()
+Future<Log::Position> LogReaderProcess::_beginning()
 {
-  LOG(INFO) << "Reconnecting to ZooKeeper";
+  CHECK(recovering.isReady());
+
+  return recovering.get()->beginning()
+    .then(lambda::bind(&Self::position, lambda::_1));
 }
 
 
-void LogProcess::reconnected()
+Future<Log::Position> LogReaderProcess::ending()
 {
-  LOG(INFO) << "Reconnected to ZooKeeper";
+  return recover().then(defer(self(), &Self::_ending));
 }
 
 
-void LogProcess::expired()
+Future<Log::Position> LogReaderProcess::_ending()
 {
-  restart();
+  CHECK(recovering.isReady());
+
+  return recovering.get()->ending()
+    .then(lambda::bind(&Self::position, lambda::_1));
 }
 
 
-void LogProcess::updated(const string& path)
+Future<list<Log::Entry> > LogReaderProcess::read(
+    const Log::Position& from,
+    const Log::Position& to)
 {
-  if (znode + "/replicas" == path) {
+  return recover().then(defer(self(), &Self::_read, from, to));
+}
 
-    regroup();
 
-    // Reset a watch on the replicas.
-    int ret = zk->getChildren(znode + "/replicas", true, NULL);
+Future<list<Log::Entry> > LogReaderProcess::_read(
+    const Log::Position& from,
+    const Log::Position& to)
+{
+  CHECK(recovering.isReady());
 
-    if (ret != ZOK) {
-      LOG(FATAL) << "Failed to set a watch on '" << znode << "/replicas"
-                 << "' in ZooKeeper: " << zk->message(ret);
-    }
-  } else {
-    CHECK(znode + "/coordinators" == path);
+  return recovering.get()->read(from.value, to.value)
+    .then(defer(self(), &Self::__read, from, to, lambda::_1));
+}
 
-    elect();
 
-    // Reset a watch on the coordinators.
-    int ret = zk->getChildren(znode + "/coordinators", true, NULL);
+Future<list<Log::Entry> > LogReaderProcess::__read(
+    const Log::Position& from,
+    const Log::Position& to,
+    const list<Action>& actions)
+{
+  list<Log::Entry> entries;
+
+  uint64_t position = from.value;
+
+  foreach (const Action& action, actions) {
+    // Ensure read range is valid.
+    if (!action.has_performed() ||
+        !action.has_learned() ||
+        !action.learned()) {
+      return Failure("Bad read range (includes pending entries)");
+    } else if (position++ != action.position()) {
+      return Failure("Bad read range (includes missing entries)");
+    }
 
-    if (ret != ZOK) {
-      LOG(FATAL) << "Failed to set a watch on '" << znode << "/replicas"
-                 << "' in ZooKeeper: " << zk->message(ret);
+    // And only return appends.
+    CHECK(action.has_type());
+    if (action.type() == Action::APPEND) {
+      entries.push_back(Log::Entry(action.position(), action.append().bytes()));
     }
   }
+
+  return entries;
 }
 
 
-void LogProcess::initalize()
+Log::Position LogReaderProcess::position(uint64_t value)
 {
-  // TODO(benh): Real testing requires injecting a ZooKeeper instance.
-  watcher = new LogProcessWatcher(self());
-  zk = new ZooKeeper(servers, 10000, watcher);
+  return Log::Position(value);
 }
 
 
-void LogProcess::regroup()
+/////////////////////////////////////////////////
+// Implementation of LogWriterProcess.
+/////////////////////////////////////////////////
+
+
+LogWriterProcess::LogWriterProcess(Log* log)
+  : ProcessBase(ID::generate("log-writer")),
+    quorum(log->process->quorum),
+    network(log->process->network),
+    recovering(dispatch(log->process, &LogProcess::recover)),
+    coordinator(NULL),
+    error(None()) {}
+
+
+void LogWriterProcess::initialize()
 {
-  vector<string> results;
+  recovering.onAny(defer(self(), &Self::_recover));
+}
 
-  int ret = zk->getChildren(znode + "/replicas", false, &results);
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to get children of '" << znode << "/replicas"
-               << "' in ZooKeeper: " << zk->message(ret);
+void LogWriterProcess::finalize()
+{
+  foreach (process::Promise<Nothing>* promise, promises) {
+    promise->fail("Log writer is being deleted");
+    delete promise;
   }
+  promises.clear();
+
+  delete coordinator;
+}
 
-  set<UPID> current;
-  set<UPID> added;
-  set<UPID> removed;
 
-  foreach (const string& result, results) {
-    string s;
-    int ret = zk->get(znode + "/replicas/" + result, false, &s, NULL);
-    UPID pid = s;
-    current.insert(pid);
+Future<Nothing> LogWriterProcess::recover()
+{
+  if (recovering.isReady()) {
+    return Nothing();
+  } else if (recovering.isFailed()) {
+    return Failure(recovering.failure());
+  } else if (recovering.isDiscarded()) {
+    return Failure("The future 'recovering' is unexpectedly discarded");
   }
 
-  foreach (const UPID& pid, current) {
-    if (members.count(pid) == 0) {
-      added.insert(pid);
+  // At this moment, the future 'recovering' should most likely be
+  // pending. But it is also likely that it gets set after the above
+  // checks. Either way, we know that the continuation '_recover' has
+  // not been called yet (otherwise, we should not be able to reach
+  // here). The promise we are creating below will be properly
+  // set/failed when '_recover' is called.
+  process::Promise<Nothing>* promise = new process::Promise<Nothing>();
+  promises.push_back(promise);
+  return promise->future();
+}
+
+
+void LogWriterProcess::_recover()
+{
+  if (!recovering.isReady()) {
+    foreach (process::Promise<Nothing>* promise, promises) {
+      promise->fail(
+          recovering.isFailed() ?
+          recovering.failure() :
+          "The future 'recovering' is unexpectedly discarded");
+      delete promise;
+    }
+    promises.clear();
+  } else {
+    foreach (process::Promise<Nothing>* promise, promises) {
+      promise->set(Nothing());
+      delete promise;
     }
+    promises.clear();
   }
+}
 
-  foreach (const UPID& pid, members) {
-    if (current.count(pid) == 0) {
-      removed.insert(pid);
-    }
+
+Future<Option<Log::Position> > LogWriterProcess::elect()
+{
+  return recover().then(defer(self(), &Self::_elect));
+}
+
+
+Future<Option<Log::Position> > LogWriterProcess::_elect()
+{
+  // We delete the existing coordinator (if exists) and create a new
+  // coordinator each time 'elect' is called.
+  delete coordinator;
+  error = None();
+
+  CHECK(recovering.isReady());
+
+  coordinator = new Coordinator(quorum, recovering.get(), network);
+
+  return coordinator->elect()
+    .then(defer(self(), &Self::__elect, lambda::_1))
+    .onFailed(defer(self(), &Self::failed, lambda::_1));
+}
+
+
+Option<Log::Position> LogWriterProcess::__elect(const Option<uint64_t>& result)
+{
+  if (result.isNone()) {
+    return None();
+  } else {
+    return position(result.get());
+  }
+}
+
+
+Future<Log::Position> LogWriterProcess::append(const string& bytes)
+{
+  if (coordinator == NULL) {
+    return Failure("No election has been performed");
   }
 
-  foreach (const UPID& pid, added) {
-    dispatch(group, &GroupProcess::add, pid);
-    members.insert(pid);
+  if (error.isSome()) {
+    return Failure(error.get());
   }
 
-  foreach (const UPID& pid, removed) {
-    dispatch(group, &GroupProcess::remove, pid);
-    members.erase(pid);
+  return coordinator->append(bytes)
+    .then(lambda::bind(&Self::position, lambda::_1))
+    .onFailed(defer(self(), &Self::failed, lambda::_1));
+}
+
+
+Future<Log::Position> LogWriterProcess::truncate(const Log::Position& to)
+{
+  if (coordinator == NULL) {
+    return Failure("No election has been performed");
   }
+
+  if (error.isSome()) {
+    return Failure(error.get());
+  }
+
+  return coordinator->truncate(to.value)
+    .then(lambda::bind(&Self::position, lambda::_1))
+    .onFailed(defer(self(), &Self::failed, lambda::_1));
 }
 
 
-void LogProcess::elect()
+void LogWriterProcess::failed(const string& message)
 {
-  vector<string> results;
+  error = message;
+}
 
-  int ret = zk->getChildren(znode + "/coordinators", false, &results);
 
-  if (ret != ZOK) {
-    LOG(FATAL) << "Failed to get children of '" << znode << "/coordinators"
-               << "' in ZooKeeper: " << zk->message(ret);
-  }
+Log::Position LogWriterProcess::position(uint64_t value)
+{
+  return Log::Position(value);
+}
+
+
+/////////////////////////////////////////////////
+// Public interfaces for Log.
+/////////////////////////////////////////////////
+
+
+Log::Log(
+    int quorum,
+    const string& path,
+    const set<UPID>& pids)
+{
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+
+  process = new LogProcess(quorum, path, pids);
+  spawn(process);
+}
+
+Log::Log(
+    int quorum,
+    const string& path,
+    const string& servers,
+    const Duration& timeout,
+    const string& znode,
+    const Option<zookeeper::Authentication>& auth)
+{
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+
+  process = new LogProcess(quorum, path, servers, timeout, znode, auth);
+  spawn(process);
+}
+
+
+Log::~Log()
+{
+  terminate(process);
+  process::wait(process);
+  delete process;
+}
+
 
-  // "Elect" the minimum ephemeral znode.
-  uint64_t min = LONG_MAX;
-  foreach (const string& result, results) {
-    try {
-      min = std::min(min, boost::lexical_cast<uint64_t>(result));
-    } catch (boost::bad_lexical_cast&) {
-      LOG(FATAL) << "Failed to convert '" << result << "' into an integer";
+/////////////////////////////////////////////////
+// Public interfaces for Log::Reader.
+/////////////////////////////////////////////////
+
+
+Log::Reader::Reader(Log* log)
+{
+  process = new LogReaderProcess(log);
+  spawn(process);
+}
+
+
+Log::Reader::~Reader()
+{
+  terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+Result<list<Log::Entry> > Log::Reader::read(
+    const Log::Position& from,
+    const Log::Position& to,
+    const Timeout& timeout)
+{
+  Future<list<Log::Entry> > future =
+    dispatch(process, &LogReaderProcess::read, from, to);
+
+  if (!future.await(timeout.remaining())) {
+    LOG(INFO) << "Timed out while trying to read the log";
+
+    future.discard();
+    return None();
+  } else {
+    if (!future.isReady()) {
+      string failure =
+        future.isFailed() ?
+        future.failure() :
+        "Not expecting discarded future";
+
+      LOG(ERROR) << "Failed to read the log: " << failure;
+
+      return Error(failure);
+    } else {
+      return future.get();
     }
   }
+}
 
-  if (id == min && !elected) {
-    elected = true;
-    process::run(&coordinate, coordinator, id, end, truncations);
-  } else if (elected) {
-    LOG(INFO) << "Restarting due to demoted";
-    restart();
-  }
+
+Log::Position Log::Reader::beginning()
+{
+  // TODO(benh): Take a timeout and return an Option.
+  return dispatch(process, &LogReaderProcess::beginning).get();
 }
 
 
-int main(int argc, char** argv)
+Log::Position Log::Reader::ending()
 {
-  if (argc < 6) {
-    fatal("Usage: %s <quorum> <file> <servers> <znode> <end> <at> <to> ...",
-          argv[0]);
-  }
+  // TODO(benh): Take a timeout and return an Option.
+  return dispatch(process, &LogReaderProcess::ending).get();
+}
 
-  args = argv;
 
-  int quorum = atoi(argv[1]);
-  string file = argv[2];
-  string servers = argv[3];
-  string znode = argv[4];
-  int end = atoi(argv[5]);
+/////////////////////////////////////////////////
+// Public interfaces for Log::Writer.
+/////////////////////////////////////////////////
 
-  map<int, int> truncations;
 
-  for (int i = 6; argv[i] != NULL; i += 2) {
-    if (argv[i + 1] == NULL) {
-      fatal("Expecting 'to' argument for truncation");
+Log::Writer::Writer(Log* log, const Duration& timeout, int retries)
+{
+  process = new LogWriterProcess(log);
+  spawn(process);
+
+  // Trying to get elected.
+  for (;;) {
+    LOG(INFO) << "Attempting to get elected within " << timeout;
+
+    Future<Option<Log::Position> > future =
+      dispatch(process, &LogWriterProcess::elect);
+
+    if (!future.await(timeout)) {
+      LOG(INFO) << "Timed out while trying to get elected";
+
+      // Cancel the election. It is likely that the election is done
+      // right after the timeout has been reached. In that case, we
+      // may unnecessarily rerun the election, but it is safe.
+      future.discard();
+    } else {
+      if (!future.isReady()) {
+        string failure =
+          future.isFailed() ?
+          future.failure() :
+          "Not expecting discarded future";
+
+        LOG(ERROR) << "Failed to get elected: " << failure;
+        break;
+      } else if (future.get().isNone()) {
+        LOG(INFO) << "Lost an election, but can be retried";
+      } else {
+        LOG(INFO) << "Elected with current position "
+                  << future.get().get().value;
+        return;
+      }
+    }
+
+    if (--retries < 0) {
+      LOG(ERROR) << "Retry limit has been reached during election";
+      break;
     }
+  }
+}
+
+
+Log::Writer::~Writer()
+{
+  terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+Result<Log::Position> Log::Writer::append(
+    const string& data,
+    const Timeout& timeout)
+{
+  LOG(INFO) << "Attempting to append " << data.size() << " bytes to the log";
+
+  Future<Log::Position> future =
+    dispatch(process, &LogWriterProcess::append, data);
+
+  if (!future.await(timeout.remaining())) {
+    LOG(INFO) << "Timed out while trying to append the log";
 
-    int at = atoi(argv[i]);
-    int to = atoi(argv[i + 1]);
+    future.discard();
+    return None();
+  } else {
+    if (!future.isReady()) {
+      string failure =
+        future.isFailed() ?
+        future.failure() :
+        "Not expecting discarded future";
+
+      LOG(ERROR) << "Failed to append the log: " << failure;
 
-    truncations[at] = to;
+      return Error(failure);
+    } else {
+      return future.get();
+    }
   }
+}
+
+
+Result<Log::Position> Log::Writer::truncate(
+    const Log::Position& to,
+    const Timeout& timeout)
+{
+  LOG(INFO) << "Attempting to truncate the log to " << to.value;
 
-  process::initialize();
+  Future<Log::Position> future =
+    dispatch(process, &LogWriterProcess::truncate, to);
 
-  LogProcess log(quorum, file, servers, znode, end, truncations);
-  spawn(log);
-  wait(log);
+  if (!future.await(timeout.remaining())) {
+    LOG(INFO) << "Timed out while trying to truncate the log";
 
-  return 0;
+    future.discard();
+    return None();
+  } else {
+    if (!future.isReady()) {
+      string failure =
+        future.isFailed() ?
+        future.failure() :
+        "Not expecting discarded future";
+
+      LOG(ERROR) << "Failed to truncate the log: " << failure;
+
+      return Error(failure);
+    } else {
+      return future.get();
+    }
+  }
 }
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/log.hpp
----------------------------------------------------------------------
diff --git a/src/log/log.hpp b/src/log/log.hpp
index 042f13b..1f0b30d 100644
--- a/src/log/log.hpp
+++ b/src/log/log.hpp
@@ -28,15 +28,10 @@
 #include <process/shared.hpp>
 #include <process/timeout.hpp>
 
-#include <stout/check.hpp>
-#include <stout/error.hpp>
-#include <stout/foreach.hpp>
+#include <stout/duration.hpp>
 #include <stout/none.hpp>
+#include <stout/option.hpp>
 #include <stout/result.hpp>
-#include <stout/try.hpp>
-
-#include "log/coordinator.hpp"
-#include "log/replica.hpp"
 
 #include "zookeeper/group.hpp"
 
@@ -44,6 +39,12 @@ namespace mesos {
 namespace internal {
 namespace log {
 
+// Forward declarations.
+class LogProcess;
+class LogReaderProcess;
+class LogWriterProcess;
+
+
 class Log
 {
 public:
@@ -98,9 +99,12 @@ public:
 
   private:
     friend class Log;
-    friend class Reader;
     friend class Writer;
+    friend class LogReaderProcess;
+    friend class LogWriterProcess;
+
     Position(uint64_t _value) : value(_value) {}
+
     uint64_t value;
   };
 
@@ -111,8 +115,8 @@ public:
     std::string data;
 
   private:
-    friend class Reader;
-    friend class Writer;
+    friend class LogReaderProcess;
+
     Entry(const Position& _position, const std::string& _data)
       : position(_position), data(_data) {}
   };
@@ -125,9 +129,10 @@ public:
 
     // Returns all entries between the specified positions, unless
     // those positions are invalid, in which case returns an error.
-    Result<std::list<Entry> > read(const Position& from,
-                                   const Position& to,
-                                   const process::Timeout& timeout);
+    Result<std::list<Entry> > read(
+        const Position& from,
+        const Position& to,
+        const process::Timeout& timeout);
 
     // Returns the beginning position of the log from the perspective
     // of the local replica (which may be out of date if the log has
@@ -141,7 +146,7 @@ public:
     Position ending();
 
   private:
-    process::Shared<Replica> replica;
+    LogReaderProcess* process;
   };
 
   class Writer
@@ -171,69 +176,28 @@ public:
         const process::Timeout& timeout);
 
   private:
-    Option<std::string> error;
-    Coordinator coordinator;
+    LogWriterProcess* process;
   };
 
   // Creates a new replicated log that assumes the specified quorum
-  // size, is backed by a file at the specified path, and coordiantes
+  // size, is backed by a file at the specified path, and coordinates
   // with other replicas via the set of process PIDs.
-  Log(int _quorum,
+  Log(int quorum,
       const std::string& path,
-      const std::set<process::UPID>& pids)
-    : group(NULL),
-      executor(NULL),
-      quorum(_quorum),
-      replica(new Replica(path))
-  {
-    GOOGLE_PROTOBUF_VERIFY_VERSION;
-
-    // Add our own replica to the network.
-    Network* _network = new Network(pids);
-    _network->add(replica->pid());
-
-    network.reset(_network);
-  }
+      const std::set<process::UPID>& pids);
 
   // Creates a new replicated log that assumes the specified quorum
-  // size, is backed by a file at the specified path, and coordiantes
+  // size, is backed by a file at the specified path, and coordinates
   // with other replicas associated with the specified ZooKeeper
   // servers, timeout, and znode.
-  Log(int _quorum,
+  Log(int quorum,
       const std::string& path,
       const std::string& servers,
       const Duration& timeout,
       const std::string& znode,
-      const Option<zookeeper::Authentication>& auth = None())
-    : group(new zookeeper::Group(servers, timeout, znode, auth)),
-      executor(new process::Executor()),
-      quorum(_quorum),
-      replica(new Replica(path)),
-      network(new ZooKeeperNetwork(servers, timeout, znode, auth))
-  {
-    GOOGLE_PROTOBUF_VERIFY_VERSION;
+      const Option<zookeeper::Authentication>& auth = None());
 
-    // Need to add our replica to the ZooKeeper group!
-    LOG(INFO) << "Attempting to join replica to ZooKeeper group";
-
-    membership = group->join(replica->pid())
-      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
-
-    group->watch()
-      .onReady(executor->defer(lambda::bind(&Log::watch, this, lambda::_1)))
-      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
-  }
-
-  ~Log()
-  {
-    network.own().await();
-    replica.own().await();
-
-    delete executor;
-    delete group;
-  }
+  ~Log();
 
   // Returns a position based off of the bytes recovered from
   // Position.identity().
@@ -252,196 +216,13 @@ public:
       ((uint64_t) (bytes[7] & 0xff));
     return Position(value);
   }
-private:
-  friend class Reader;
-  friend class Writer;
-
-  // TODO(benh): Factor this out into some sort of "membership renewer".
-  void watch(const std::set<zookeeper::Group::Membership>& memberships);
-  void failed(const std::string& message) const;
-  void discarded() const;
-
-  // We store a Group instance in order to continually renew the
-  // replicas membership (when using ZooKeeper).
-  zookeeper::Group* group;
-  process::Future<zookeeper::Group::Membership> membership;
-  process::Executor* executor;
-
-  int quorum;
-  process::Shared<Replica> replica;
-  process::Shared<Network> network;
-};
-
-
-Log::Reader::Reader(Log* log)
-  : replica(log->replica) {}
-
-
-Log::Reader::~Reader() {}
-
-
-Result<std::list<Log::Entry> > Log::Reader::read(
-    const Log::Position& from,
-    const Log::Position& to,
-    const process::Timeout& timeout)
-{
-  process::Future<std::list<Action> > actions =
-    replica->read(from.value, to.value);
-
-  if (!actions.await(timeout.remaining())) {
-    return None();
-  } else if (actions.isFailed()) {
-    return Error(actions.failure());
-  }
-
-  CHECK(actions.isReady()) << "Not expecting discarded future!";
-
-  std::list<Log::Entry> entries;
-
-  uint64_t position = from.value;
-
-  foreach (const Action& action, actions.get()) {
-    // Ensure read range is valid.
-    if (!action.has_performed() ||
-        !action.has_learned() ||
-        !action.learned()) {
-      return Error("Bad read range (includes pending entries)");
-    } else if (position++ != action.position()) {
-      return Error("Bad read range (includes missing entries)");
-    }
-
-    // And only return appends.
-    CHECK(action.has_type());
-    if (action.type() == Action::APPEND) {
-      entries.push_back(Entry(action.position(), action.append().bytes()));
-    }
-  }
-
-  return entries;
-}
-
-
-Log::Position Log::Reader::beginning()
-{
-  // TODO(benh): Take a timeout and return an Option.
-  process::Future<uint64_t> value = replica->beginning();
-  value.await();
-  CHECK(value.isReady()) << "Not expecting a failed or discarded future!";
-  return Log::Position(value.get());
-}
-
-
-Log::Position Log::Reader::ending()
-{
-  // TODO(benh): Take a timeout and return an Option.
-  process::Future<uint64_t> value = replica->ending();
-  value.await();
-  CHECK(value.isReady()) << "Not expecting a failed or discarded future!";
-  return Log::Position(value.get());
-}
-
-
-Log::Writer::Writer(Log* log, const Duration& timeout, int retries)
-  : error(None()),
-    coordinator(log->quorum, log->replica, log->network)
-{
-  do {
-    Result<uint64_t> result = coordinator.elect(process::Timeout::in(timeout));
-    if (result.isNone()) {
-      retries--;
-    } else if (result.isSome()) {
-      break;
-    } else {
-      error = result.error();
-      break;
-    }
-  } while (retries > 0);
-}
-
-
-Log::Writer::~Writer()
-{
-  coordinator.demote();
-}
-
-
-Result<Log::Position> Log::Writer::append(
-    const std::string& data,
-    const process::Timeout& timeout)
-{
-  if (error.isSome()) {
-    return Error(error.get());
-  }
-
-  LOG(INFO) << "Attempting to append " << data.size() << " bytes to the log";
-
-  Result<uint64_t> result = coordinator.append(data, timeout);
-
-  if (result.isError()) {
-    error = result.error();
-    return Error(error.get());
-  } else if (result.isNone()) {
-    return None();
-  }
-
-  CHECK_SOME(result);
-
-  return Log::Position(result.get());
-}
-
-
-Result<Log::Position> Log::Writer::truncate(
-    const Log::Position& to,
-    const process::Timeout& timeout)
-{
-  if (error.isSome()) {
-    return Error(error.get());
-  }
-
-  LOG(INFO) << "Attempting to truncate the log to " << to.value;
-
-  Result<uint64_t> result = coordinator.truncate(to.value, timeout);
-
-  if (result.isError()) {
-    error = result.error();
-    return Error(error.get());
-  } else if (result.isNone()) {
-    return None();
-  }
-
-  CHECK_SOME(result);
-
-  return Log::Position(result.get());
-}
-
-
-void Log::watch(const std::set<zookeeper::Group::Membership>& memberships)
-{
-  if (membership.isReady() && memberships.count(membership.get()) == 0) {
-    // Our replica's membership must have expired, join back up.
-    LOG(INFO) << "Renewing replica group membership";
-    membership = group->join(replica->pid())
-      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
-  }
-
-  group->watch(memberships)
-    .onReady(executor->defer(lambda::bind(&Log::watch, this, lambda::_1)))
-    .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
-    .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
-}
-
-
-void Log::failed(const std::string& message) const
-{
-  LOG(FATAL) << "Failed to participate in ZooKeeper group: " << message;
-}
 
+private:
+  friend class LogReaderProcess;
+  friend class LogWriterProcess;
 
-void Log::discarded() const
-{
-  LOG(FATAL) << "Not expecting future to get discarded!";
-}
+  LogProcess* process;
+};
 
 } // namespace log {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/recover.cpp
----------------------------------------------------------------------
diff --git a/src/log/recover.cpp b/src/log/recover.cpp
new file mode 100644
index 0000000..0ab8e95
--- /dev/null
+++ b/src/log/recover.cpp
@@ -0,0 +1,403 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdlib.h>
+
+#include <set>
+
+#include <process/defer.hpp>
+#include <process/delay.hpp>
+#include <process/id.hpp>
+#include <process/process.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/lambda.hpp>
+#include <stout/none.hpp>
+#include <stout/option.hpp>
+
+#include "common/type_utils.hpp"
+
+#include "log/catchup.hpp"
+#include "log/recover.hpp"
+
+#include "messages/log.hpp"
+
+using namespace process;
+
+using std::set;
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// This process is used to recover a replica. The flow of the recover
+// process is described as follows:
+// A) Check the status of the local replica.
+//    A1) If it is VOTING, exit.
+//    A2) If it is not VOTING, goto (B).
+// B) Broadcast a RecoverRequest to all replicas in the network.
+//    B1) <<< Catch-up >>> If a quorum of replicas are found in VOTING
+//        status (no matter what the status of the local replica is),
+//        set the status of the local replica to RECOVERING, and start
+//        doing catch-up. If the local replica has been caught-up, set
+//        the status of the local replica to VOTING and exit.
+//    B2) If a quorum is not found, goto (B).
+//
+// In the following, we list a few scenarios and show how the recover
+// process will respond in those scenarios. All the examples assume a
+// quorum size of 2. Remember that a new replica is always put in
+// EMPTY status initially.
+//
+// 1) Replica A, B and C are all in VOTING status. The operator adds
+//    replica D. In that case, D will go into RECOVERING status and
+//    then go into VOTING status. Therefore, we should avoid adding a
+//    new replica unless we know that one replica has been removed.
+//
+// 2) Replica A and B are in VOTING status. The operator adds replica
+//    C. In that case, C will go into RECOVERING status and then go
+//    into VOTING status, which is expected.
+//
+// 3) Replica A is in VOTING status. The operator adds replica B. In
+//    that case, B will stay in EMPTY status forever. This is expected
+//    because we cannot make progress if VOTING replicas are not
+//    enough (i.e., less than quorum).
+//
+// 4) Replica A is in VOTING status and B is in EMPTY status. The
+//    operator adds replica C. In that case, C will stay in EMPTY
+//    status forever similar to case 3).
+class RecoverProcess : public Process<RecoverProcess>
+{
+public:
+  RecoverProcess(
+      size_t _quorum,
+      const Owned<Replica>& _replica,
+      const Shared<Network>& _network)
+    : ProcessBase(ID::generate("log-recover")),
+      quorum(_quorum),
+      replica(_replica),
+      network(_network) {}
+
+  Future<Owned<Replica> > future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    LOG(INFO) << "Start recovering a replica";
+
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+          static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    // Check the current status of the local replica and decide if
+    // recovery is needed. Recovery is needed if the local replica is
+    // not in VOTING status.
+    replica->status().onAny(defer(self(), &Self::checked, lambda::_1));
+  }
+
+  virtual void finalize()
+  {
+    LOG(INFO) << "Recover process terminated";
+
+    // Cancel all operations if they are still pending.
+    discard(responses);
+    catching.discard();
+  }
+
+private:
+  void checked(const Future<Metadata::Status>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to get replica status: " + future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    status = future.get();
+
+    LOG(INFO) << "Replica is in " << status << " status";
+
+    if (status == Metadata::VOTING) {
+      promise.set(replica);
+      terminate(self());
+    } else {
+      recover();
+    }
+  }
+
+  void recover()
+  {
+    CHECK_NE(status, Metadata::VOTING);
+
+    // Broadcast recover request to all replicas.
+    network->broadcast(protocol::recover, RecoverRequest())
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
+  void broadcasted(const Future<set<Future<RecoverResponse> > >& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to broadcast the recover request: " + future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    responses = future.get();
+
+    if (responses.empty()) {
+      // Retry if no replica is currently in the network.
+      retry();
+    } else {
+      // Instead of using a for loop here, we use select to process
+      // responses one after another so that we can ignore the rest if
+      // we have collected enough responses.
+      select(responses)
+        .onReady(defer(self(), &Self::received, lambda::_1));
+
+      // Reset the counters.
+      responsesReceived.clear();
+      lowestBeginPosition = None();
+      highestEndPosition = None();
+    }
+  }
+
+  void received(const Future<RecoverResponse>& future)
+  {
+    // Enforced by the select semantics.
+    CHECK(future.isReady());
+
+    // Remove this future from 'responses' so that we do not listen on
+    // it the next time we invoke select.
+    responses.erase(future);
+
+    const RecoverResponse& response = future.get();
+
+    LOG(INFO) << "Received a recover response from a replica in "
+              << response.status() << " status";
+
+    responsesReceived[response.status()]++;
+
+    // We need to remember the lowest begin position and highest end
+    // position seen from VOTING replicas.
+    if (response.status() == Metadata::VOTING) {
+      CHECK(response.has_begin() && response.has_end());
+
+      lowestBeginPosition = min(lowestBeginPosition, response.begin());
+      highestEndPosition = max(highestEndPosition, response.end());
+    }
+
+    // If we got responses from a quorum of VOTING replicas, the local
+    // replica will be put in RECOVERING status and start catching up.
+    // It is likely that the local replica is in RECOVERING status
+    // already. This is the case where the replica crashes during
+    // catch-up. When it restarts, we need to recalculate the lowest
+    // begin position and the highest end position since we haven't
+    // persisted this information on disk.
+    if (responsesReceived[Metadata::VOTING] >= quorum) {
+      discard(responses);
+      update(Metadata::RECOVERING);
+      return;
+    }
+
+    if (responses.empty()) {
+      // All responses have been received but neither have we received
+      // enough responses from VOTING replicas to do catch-up, nor are
+      // we in start-up case. This is either because we don't have
+      // enough replicas in the network (e.g. ZooKeeper blip), or we
+      // don't have enough VOTING replicas to proceed. We will retry
+      // the recovery in both cases.
+      retry();
+    } else {
+      // Wait for the next response.
+      select(responses)
+        .onReady(defer(self(), &Self::received, lambda::_1));
+    }
+  }
+
+  void update(const Metadata::Status& _status)
+  {
+    LOG(INFO) << "Updating replica status from "
+              << status << " to " << _status;
+
+    replica->update(_status)
+      .onAny(defer(self(), &Self::updated, _status, lambda::_1));
+  }
+
+  void updated(const Metadata::Status& _status, const Future<bool>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to update replica status: " + future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    } else if (!future.get()) {
+      promise.fail("Failed to update replica status");
+      terminate(self());
+      return;
+    }
+
+    // The replica status has been updated successfully. Depending on
+    // the new status, we decide what the next action should be.
+    status = _status;
+
+    if (status == Metadata::VOTING) {
+      LOG(INFO) << "Successfully joined the Paxos group";
+
+      promise.set(replica);
+      terminate(self());
+    } else if (status == Metadata::RECOVERING) {
+      catchup();
+    } else {
+      // The replica should not be in any other status.
+      LOG(FATAL) << "Unexpected replica status";
+    }
+  }
+
+  void catchup()
+  {
+    // We reach here either because the log is empty (uninitialized),
+    // or the log is not empty but a previous unfinished catch-up
+    // attempt has been detected (the process crashes/killed when
+    // catching up). In either case, the local replica may have lost
+    // some data and Paxos states, and should not be allowed to vote.
+    // Otherwise, we may introduce inconsistency in the log as the
+    // local replica could have accepted a write which it would not
+    // have accepted if the data and the Paxos states were not lost.
+    // Now, the question is how many positions the local replica
+    // should catch up before it can be allowed to vote. We find that
+    // it is sufficient to catch-up positions from _begin_ to _end_
+    // where _begin_ is the smallest position seen in a quorum of
+    // VOTING replicas and _end_ is the largest position seen in a
+    // quorum of VOTING replicas. Here is the correctness argument.
+    // For a position _e_ larger than _end_, obviously no value has
+    // been agreed on for that position. Otherwise, we should find at
+    // least one VOTING replica in a quorum of replicas such that its
+    // end position is larger than _end_. For the same reason, a
+    // coordinator should not have collected enough promises for
+    // position _e_. Therefore, it's safe for the local replica to
+    // vote for that position. For a position _b_ smaller than
+    // _begin_, it should have already been truncated and the
+    // truncation should have already been agreed. Therefore, allowing
+    // the local replica to vote for that position is safe.
+    CHECK(lowestBeginPosition.isSome());
+    CHECK(highestEndPosition.isSome());
+    CHECK_LE(lowestBeginPosition.get(), highestEndPosition.get());
+
+    uint64_t begin = lowestBeginPosition.get();
+    uint64_t end = highestEndPosition.get();
+
+    set<uint64_t> positions;
+    for (uint64_t p = begin; p <= end; ++p) {
+      positions.insert(p);
+    }
+
+    // Share the ownership of the replica. From this point until the
+    // point where the ownership of the replica is regained, we should
+    // not access the 'replica' field.
+    Shared<Replica> shared = replica.share();
+
+    // Since we do not know what proposal number to use (the log is
+    // empty), we use proposal number 0 and leave log::catchup to
+    // automatically bump the proposal number.
+    catching = log::catchup(quorum, shared, network, 0, positions);
+    catching.onAny(defer(self(), &Self::caughtup, shared, lambda::_1));
+  }
+
+  void caughtup(Shared<Replica> shared, const Future<Nothing>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to catch-up: " + future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+    } else {
+      // Try to regain the ownership of the replica.
+      shared.own().onAny(defer(self(), &Self::owned, lambda::_1));
+    }
+  }
+
+  void owned(const Future<Owned<Replica> >& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to own the replica: " + future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+    } else {
+      // Allow the replica to vote once the catch-up is done.
+      replica = future.get();
+      update(Metadata::VOTING);
+    }
+  }
+
+  void retry()
+  {
+    // We add a random delay before each retry because we do not want
+    // to saturate the network/disk IO in some cases (e.g., network
+    // size is less than quorum). The delay is chosen randomly to
+    // reduce the likelihood of conflicts (i.e., a replica receives a
+    // recover request while it is changing its status).
+    static const Duration T = Milliseconds(500);
+    Duration d = T * (1.0 + (double) ::random() / RAND_MAX);
+    delay(d, self(), &Self::recover);
+  }
+
+  const size_t quorum;
+  Owned<Replica> replica;
+  const Shared<Network> network;
+
+  Metadata::Status status;
+  set<Future<RecoverResponse> > responses;
+  hashmap<Metadata::Status, size_t> responsesReceived;
+  Option<uint64_t> lowestBeginPosition;
+  Option<uint64_t> highestEndPosition;
+  Future<Nothing> catching;
+
+  process::Promise<Owned<Replica> > promise;
+};
+
+
+Future<Owned<Replica> > recover(
+    size_t quorum,
+    const Owned<Replica>& replica,
+    const Shared<Network>& network)
+{
+  RecoverProcess* process = new RecoverProcess(quorum, replica, network);
+  Future<Owned<Replica> > future = process->future();
+  spawn(process, true);
+  return future;
+}
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/recover.hpp
----------------------------------------------------------------------
diff --git a/src/log/recover.hpp b/src/log/recover.hpp
new file mode 100644
index 0000000..634bc06
--- /dev/null
+++ b/src/log/recover.hpp
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_RECOVER_HPP__
+#define __LOG_RECOVER_HPP__
+
+#include <stdint.h>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/shared.hpp>
+
+#include <stout/nothing.hpp>
+
+#include "log/network.hpp"
+#include "log/replica.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// Recovers a replica by catching up enough missing positions. A
+// replica starts with an empty log (e.g., in the case of a disk
+// failure) should not be allowed to vote. Otherwise, the new votes it
+// makes may contradict its lost votes, leading to potential
+// inconsistency in the log. Instead, the replica should be put in
+// non-voting status and catch up missing positions (and associated
+// Paxos states). The replica can be re-allowed to vote if the
+// following two conditions are met: 1) a sufficient amount of missing
+// positions are recovered such that if other replicas fail, the
+// remaining replicas can restore all the successfully written log
+// entries; 2) its future votes cannot not contradict its lost votes.
+// This function returns an owned pointer to the recovered replica if
+// the recovery is successful.
+extern process::Future<process::Owned<Replica> > recover(
+    size_t quorum,
+    const process::Owned<Replica>& replica,
+    const process::Shared<Network>& network);
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_RECOVER_HPP__


[06/10] git commit: Refactored log tools and added a tool to initialize the log.

Posted by be...@apache.org.
Refactored log tools and added a tool to initialize the log.

Also pulled storage related code out (I haven't changed them) from
replica.cpp.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/16433


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/e2fe5860
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/e2fe5860
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/e2fe5860

Branch: refs/heads/master
Commit: e2fe5860bc8542e5408bc86ac7322002326d41b3
Parents: f9b60c4
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:55:00 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:55:00 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am             |  14 +-
 src/log/leveldb.cpp         | 422 +++++++++++++++++++++++++++++++++++++
 src/log/leveldb.hpp         |  51 +++++
 src/log/main.cpp            | 132 +++++-------
 src/log/replica.cpp         | 439 +--------------------------------------
 src/log/replica.hpp         |   6 +-
 src/log/storage.hpp         |  61 ++++++
 src/log/tool.hpp            |  51 +++++
 src/log/tool/initialize.cpp | 148 +++++++++++++
 src/log/tool/initialize.hpp |  63 ++++++
 src/log/tool/read.cpp       | 188 +++++++++++++++++
 src/log/tool/read.hpp       |  65 ++++++
 12 files changed, 1120 insertions(+), 520 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 60fcb31..d58b46e 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -301,17 +301,25 @@ liblog_la_SOURCES =							\
   log/catchup.cpp							\
   log/consensus.cpp							\
   log/coordinator.cpp							\
+  log/leveldb.cpp							\
   log/log.cpp								\
   log/recover.cpp							\
-  log/replica.cpp
+  log/replica.cpp							\
+  log/tool/initialize.cpp						\
+  log/tool/read.cpp
 liblog_la_SOURCES +=							\
   log/catchup.hpp							\
   log/consensus.hpp							\
   log/coordinator.hpp							\
-  log/recover.hpp							\
-  log/replica.hpp							\
+  log/leveldb.hpp							\
   log/log.hpp								\
   log/network.hpp							\
+  log/recover.hpp							\
+  log/replica.hpp							\
+  log/storage.hpp							\
+  log/tool.hpp								\
+  log/tool/initialize.hpp						\
+  log/tool/read.hpp							\
   messages/log.hpp							\
   messages/log.proto
 nodist_liblog_la_SOURCES = $(LOG_PROTOS)

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/leveldb.cpp
----------------------------------------------------------------------
diff --git a/src/log/leveldb.cpp b/src/log/leveldb.cpp
new file mode 100644
index 0000000..7819963
--- /dev/null
+++ b/src/log/leveldb.cpp
@@ -0,0 +1,422 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <google/protobuf/io/zero_copy_stream_impl.h>
+
+#include <leveldb/comparator.h>
+#include <leveldb/write_batch.h>
+
+#include <stout/check.hpp>
+#include <stout/error.hpp>
+#include <stout/numify.hpp>
+#include <stout/stopwatch.hpp>
+#include <stout/strings.hpp>
+
+#include "log/leveldb.hpp"
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+class Varint64Comparator : public leveldb::Comparator
+{
+public:
+  virtual int Compare(
+      const leveldb::Slice& a,
+      const leveldb::Slice& b) const
+  {
+    // TODO(benh): Use varint comparator.
+    LOG(FATAL) << "Unimplemented";
+    // uint64_t left = position(a);
+    // uint64_t right = position(b);
+    // if (left < right) return -1;
+    // if (left == right) return 0;
+    // if (left > right) return 1;
+  }
+
+  virtual const char* Name() const
+  {
+    // Note that this name MUST NOT CHANGE across uses of this
+    // comparator with the same DB (the semantics of doing so are
+    // undefined if the database doesn't catch this first).
+    return "varint64";
+  }
+
+  virtual void FindShortestSeparator(
+      string* start,
+      const leveldb::Slice& limit) const
+  {
+    // Intentional no-op.
+  }
+
+  virtual void FindShortSuccessor(string* key) const
+  {
+    // Intentional no-op.
+  }
+};
+
+
+// TODO(benh): Use varint comparator.
+// static Varint64Comparator comparator;
+
+
+// Returns a string representing the specified position. Note that we
+// adjust the actual position by incrementing it by 1 because we
+// reserve 0 for storing the promise record (Record::Promise,
+// DEPRECATED!), or the metadata (Record::Metadata).
+static string encode(uint64_t position, bool adjust = true)
+{
+  // Adjusted stringified represenation is plus 1 of actual position.
+  position = adjust ? position + 1 : position;
+
+  // TODO(benh): Use varint encoding for VarInt64Comparator!
+  // string s;
+  // google::protobuf::io::StringOutputStream _stream(&s);
+  // google::protobuf::io::CodedOutputStream stream(&_stream);
+  // position = adjust ? position + 1 : position;
+  // stream.WriteVarint64(position);
+  // return s;
+
+  Try<string> s = strings::format("%.*d", 10, position);
+  CHECK_SOME(s);
+  return s.get();
+}
+
+
+// Returns the position as represented in the specified slice
+// (performing a decrement as necessary to determine the actual
+// position represented).
+static uint64_t decode(const leveldb::Slice& s)
+{
+  // TODO(benh): Use varint decoding for VarInt64Comparator!
+  // uint64_t position;
+  // google::protobuf::io::ArrayInputStream _stream(s.data(), s.size());
+  // google::protobuf::io::CodedInputStream stream(&_stream);
+  // bool success = stream.ReadVarint64(&position);
+  // CHECK(success);
+  // return position - 1; // Actual position is less 1 of stringified.
+  Try<uint64_t> position = numify<uint64_t>(string(s.data(), s.size()));
+  CHECK_SOME(position);
+  return position.get() - 1; // Actual position is less 1 of stringified.
+}
+
+
+LevelDBStorage::LevelDBStorage()
+  : db(NULL), first(0)
+{
+  // Nothing to see here.
+}
+
+
+LevelDBStorage::~LevelDBStorage()
+{
+  delete db; // Might be null if open failed in LevelDBStorage::recover.
+}
+
+
+Try<LevelDBStorage::State> LevelDBStorage::restore(const string& path)
+{
+  leveldb::Options options;
+  options.create_if_missing = true;
+
+  // TODO(benh): Can't use varint comparator until bug discussed at
+  // groups.google.com/group/leveldb/browse_thread/thread/17eac39168909ba7
+  // gets fixed. For now, we are using the default byte-wise
+  // comparator and *assuming* that the encoding from unsigned long to
+  // string produces a stable ordering. Checks below.
+  // options.comparator = &comparator;
+
+  const string& one = encode(1);
+  const string& two = encode(2);
+  const string& ten = encode(10);
+
+  CHECK(leveldb::BytewiseComparator()->Compare(one, two) < 0);
+  CHECK(leveldb::BytewiseComparator()->Compare(two, one) > 0);
+  CHECK(leveldb::BytewiseComparator()->Compare(one, ten) < 0);
+  CHECK(leveldb::BytewiseComparator()->Compare(ten, two) > 0);
+  CHECK(leveldb::BytewiseComparator()->Compare(ten, ten) == 0);
+
+  Stopwatch stopwatch;
+  stopwatch.start();
+
+  leveldb::Status status = leveldb::DB::Open(options, path, &db);
+
+  if (!status.ok()) {
+    // TODO(benh): Consider trying to repair the DB.
+    return Error(status.ToString());
+  }
+
+  LOG(INFO) << "Opened db in " << stopwatch.elapsed();
+
+  stopwatch.start(); // Restart the stopwatch.
+
+  // TODO(benh): Conditionally compact to avoid long recovery times?
+  db->CompactRange(NULL, NULL);
+
+  LOG(INFO) << "Compacted db in " << stopwatch.elapsed();
+
+  State state;
+  state.begin = 0;
+  state.end = 0;
+
+  // TODO(benh): Consider just reading the "promise" record (e.g.,
+  // 'encode(0, false)') and then iterating over the rest of the
+  // records and confirming that they are all indeed of type
+  // Record::Action.
+
+  stopwatch.start(); // Restart the stopwatch.
+
+  leveldb::Iterator* iterator = db->NewIterator(leveldb::ReadOptions());
+
+  LOG(INFO) << "Created db iterator in " << stopwatch.elapsed();
+
+  stopwatch.start(); // Restart the stopwatch.
+
+  iterator->SeekToFirst();
+
+  LOG(INFO) << "Seeked to beginning of db in " << stopwatch.elapsed();
+
+  stopwatch.start(); // Restart the stopwatch.
+
+  uint64_t keys = 0;
+
+  while (iterator->Valid()) {
+    keys++;
+    const leveldb::Slice& slice = iterator->value();
+
+    google::protobuf::io::ArrayInputStream stream(slice.data(), slice.size());
+
+    Record record;
+
+    if (!record.ParseFromZeroCopyStream(&stream)) {
+      return Error("Failed to deserialize record");
+    }
+
+    switch (record.type()) {
+      case Record::METADATA: {
+        CHECK(record.has_metadata());
+        state.metadata.CopyFrom(record.metadata());
+        break;
+      }
+
+      // DEPRECATED!
+      case Record::PROMISE: {
+        CHECK(record.has_promise());
+        // This replica is in old format. Set its status to VOTING
+        // since there is no catch-up logic in the old code and this
+        // replica is obviously not empty.
+        state.metadata.set_status(Metadata::VOTING);
+        state.metadata.set_promised(record.promise().proposal());
+        break;
+      }
+
+      case Record::ACTION: {
+        CHECK(record.has_action());
+        const Action& action = record.action();
+        if (action.has_learned() && action.learned()) {
+          state.learned.insert(action.position());
+          state.unlearned.erase(action.position());
+          if (action.has_type() && action.type() == Action::TRUNCATE) {
+            state.begin = std::max(state.begin, action.truncate().to());
+          }
+        } else {
+          state.learned.erase(action.position());
+          state.unlearned.insert(action.position());
+        }
+        state.end = std::max(state.end, action.position());
+        break;
+      }
+
+      default: {
+        return Error("Bad record");
+      }
+    }
+
+    iterator->Next();
+  }
+
+  LOG(INFO) << "Iterated through " << keys
+            << " keys in the db in " << stopwatch.elapsed();
+
+  // Determine the first position still in leveldb so during a
+  // truncation we can attempt to delete all positions from the first
+  // position up to the truncate position. Note that this is not the
+  // beginning position of the log, but rather the first position that
+  // remains (i.e., hasn't been deleted) in leveldb.
+  iterator->Seek(encode(0));
+
+  if (iterator->Valid()) {
+    first = decode(iterator->key());
+  }
+
+  delete iterator;
+
+  return state;
+}
+
+
+Try<Nothing> LevelDBStorage::persist(const Metadata& metadata)
+{
+  Stopwatch stopwatch;
+  stopwatch.start();
+
+  leveldb::WriteOptions options;
+  options.sync = true;
+
+  Record record;
+  record.set_type(Record::METADATA);
+  record.mutable_metadata()->CopyFrom(metadata);
+
+  string value;
+
+  if (!record.SerializeToString(&value)) {
+    return Error("Failed to serialize record");
+  }
+
+  leveldb::Status status = db->Put(options, encode(0, false), value);
+
+  if (!status.ok()) {
+    return Error(status.ToString());
+  }
+
+  LOG(INFO) << "Persisting metadata (" << value.size()
+            << " bytes) to leveldb took " << stopwatch.elapsed();
+
+  return Nothing();
+}
+
+
+Try<Nothing> LevelDBStorage::persist(const Action& action)
+{
+  Stopwatch stopwatch;
+  stopwatch.start();
+
+  Record record;
+  record.set_type(Record::ACTION);
+  record.mutable_action()->MergeFrom(action);
+
+  string value;
+
+  if (!record.SerializeToString(&value)) {
+    return Error("Failed to serialize record");
+  }
+
+  leveldb::WriteOptions options;
+  options.sync = true;
+
+  leveldb::Status status = db->Put(options, encode(action.position()), value);
+
+  if (!status.ok()) {
+    return Error(status.ToString());
+  }
+
+  LOG(INFO) << "Persisting action (" << value.size()
+            << " bytes) to leveldb took " << stopwatch.elapsed();
+
+  // Delete positions if a truncate action has been *learned*. Note
+  // that we do this in a best-effort fashion (i.e., we ignore any
+  // failures to the database since we can always try again).
+  if (action.has_type() && action.type() == Action::TRUNCATE &&
+      action.has_learned() && action.learned()) {
+    CHECK(action.has_truncate());
+
+    stopwatch.start(); // Restart the stopwatch.
+
+    // To actually perform the truncation in leveldb we need to remove
+    // all the keys that represent positions no longer in the log. We
+    // do this by attempting to delete all keys that represent the
+    // first position we know is still in leveldb up to (but
+    // excluding) the truncate position. Note that this works because
+    // the semantics of WriteBatch are such that even if the position
+    // doesn't exist (which is possible because this replica has some
+    // holes), we can attempt to delete the key that represents it and
+    // it will just ignore that key. This is *much* cheaper than
+    // actually iterating through the entire database instead (which
+    // was, for posterity, the original implementation). In addition,
+    // caching the "first" position we know is in the database is
+    // cheaper than using an iterator to determine the first position
+    // (which was, for posterity, the second implementation).
+
+    leveldb::WriteBatch batch;
+
+    // Add positions up to (but excluding) the truncate position to
+    // the batch starting at the first position still in leveldb.
+    uint64_t index = 0;
+    while ((first + index) < action.truncate().to()) {
+      batch.Delete(encode(first + index));
+      index++;
+    }
+
+    // If we added any positions, attempt to delete them!
+    if (index > 0) {
+      // We do this write asynchronously (e.g., using default options).
+      leveldb::Status status = db->Write(leveldb::WriteOptions(), &batch);
+
+      if (!status.ok()) {
+        LOG(WARNING) << "Ignoring leveldb batch delete failure: "
+                     << status.ToString();
+      } else {
+        first = action.truncate().to(); // Save the new first position!
+
+        LOG(INFO) << "Deleting ~" << index
+                  << " keys from leveldb took " << stopwatch.elapsed();
+      }
+    }
+  }
+
+  return Nothing();
+}
+
+
+Try<Action> LevelDBStorage::read(uint64_t position)
+{
+  Stopwatch stopwatch;
+  stopwatch.start();
+
+  leveldb::ReadOptions options;
+
+  string value;
+
+  leveldb::Status status = db->Get(options, encode(position), &value);
+
+  if (!status.ok()) {
+    return Error(status.ToString());
+  }
+
+  google::protobuf::io::ArrayInputStream stream(value.data(), value.size());
+
+  Record record;
+
+  if (!record.ParseFromZeroCopyStream(&stream)) {
+    return Error("Failed to deserialize record");
+  }
+
+  if (record.type() != Record::ACTION) {
+    return Error("Bad record");
+  }
+
+  LOG(INFO) << "Reading position from leveldb took " << stopwatch.elapsed();
+
+  return record.action();
+}
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/leveldb.hpp
----------------------------------------------------------------------
diff --git a/src/log/leveldb.hpp b/src/log/leveldb.hpp
new file mode 100644
index 0000000..7eb51be
--- /dev/null
+++ b/src/log/leveldb.hpp
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_LEVELDB_HPP__
+#define __LOG_LEVELDB_HPP__
+
+#include <leveldb/db.h>
+
+#include "log/storage.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// Concrete implementation of the storage interface using leveldb.
+class LevelDBStorage : public Storage
+{
+public:
+  LevelDBStorage();
+  virtual ~LevelDBStorage();
+
+  virtual Try<State> restore(const std::string& path);
+  virtual Try<Nothing> persist(const Metadata& metadata);
+  virtual Try<Nothing> persist(const Action& action);
+  virtual Try<Action> read(uint64_t position);
+
+private:
+  leveldb::DB* db;
+  uint64_t first; // First position still in leveldb, used during truncation.
+};
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_LEVELDB_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/main.cpp
----------------------------------------------------------------------
diff --git a/src/log/main.cpp b/src/log/main.cpp
index f07bd10..c37dd6f 100644
--- a/src/log/main.cpp
+++ b/src/log/main.cpp
@@ -16,119 +16,89 @@
  * limitations under the License.
  */
 
+#include <string.h>
+
 #include <iostream>
-#include <list>
 #include <string>
 
-#include <process/process.hpp>
+#include <process/owned.hpp>
 
-#include <stout/check.hpp>
-#include <stout/flags.hpp>
 #include <stout/foreach.hpp>
-#include <stout/none.hpp>
-#include <stout/option.hpp>
-#include <stout/os.hpp>
-
-#include "log/replica.hpp"
+#include <stout/hashmap.hpp>
 
-#include "logging/flags.hpp"
-#include "logging/logging.hpp"
+#include "log/tool.hpp"
+#include "log/tool/initialize.hpp"
+#include "log/tool/read.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::log;
 
+using namespace process;
+
 using std::cerr;
-using std::cout;
 using std::endl;
 using std::string;
 
+// All the registered tools.
+static hashmap<string, Owned<tool::Tool> > tools;
 
-void usage(const char* argv0, const flags::FlagsBase& flags)
+
+static void add(const Owned<tool::Tool>& tool)
 {
-  cerr << "Usage: " << os::basename(argv0).get() << " [...] path/to/log"
-       << endl
-       << "Supported options:" << endl
-       << flags.usage();
+  tools[tool->name()] = tool;
 }
 
 
-int main(int argc, char** argv)
+static void usage(const char* argv0)
 {
-  flags::Flags<logging::Flags> flags;
-
-  Option<uint64_t> from;
-  flags.add(&from,
-            "from",
-            "Position from which to start reading in the log");
-
-  Option<uint64_t> to;
-  flags.add(&to,
-            "to",
-            "Position from which to stop reading in the log");
-
-  bool help;
-  flags.add(&help,
-            "help",
-            "Prints this help message",
-            false);
-
-  Try<Nothing> load = flags.load(None(), argc, argv);
-
-  if (load.isError()) {
-    cerr << load.error() << endl;
-    usage(argv[0], flags);
-    exit(1);
-  }
+  cerr << "Usage: " << argv0 << " <command> [OPTIONS]" << endl
+       << endl
+       << "Available commands:" << endl
+       << "    help" << endl;
 
-  if (help) {
-    usage(argv[0], flags);
-    exit(1);
+  // Get a list of available tools.
+  foreachkey (const string& name, tools) {
+    cerr << "    " << name << endl;
   }
+}
 
-  process::initialize();
-
-  logging::initialize(argv[0], flags);
-
-  string path = argv[argc - 1];
-
-  Replica replica(path);
-
-  process::Future<uint64_t> begin = replica.beginning();
-  process::Future<uint64_t> end = replica.ending();
-
-  begin.await();
-  end.await();
-
-  CHECK(begin.isReady());
-  CHECK(end.isReady());
 
-  if (!from.isSome()) {
-    from = begin.get();
-  }
+int main(int argc, char** argv)
+{
+  // Register log tools.
+  add(Owned<tool::Tool>(new tool::Initialize()));
+  add(Owned<tool::Tool>(new tool::Read()));
 
-  if (!to.isSome()) {
-    to = end.get();
+  if (argc < 2) {
+    usage(argv[0]);
+    return 1;
   }
 
-  CHECK_SOME(from);
-  CHECK_SOME(to);
-
-  cerr << endl << "Attempting to read the log from "
-       << from.get() << " to " << to.get() << endl << endl;
-
-  process::Future<std::list<Action> > actions =
-    replica.read(from.get(), to.get());
+  if (!strcmp(argv[1], "help")) {
+    if (argc == 2) {
+      usage(argv[0]);
+      return 0;
+    }
 
-  actions.await();
+    // 'mesos-log help command' => 'mesos-log command --help'
+    argv[1] = argv[2];
+    argv[2] = (char*) "--help";
+  }
 
-  CHECK(!actions.isFailed()) << actions.failure();
+  string command = argv[1];
 
-  CHECK(actions.isReady());
+  if (!tools.contains(command)) {
+    cerr << "Cannot find command '" << command << "'" << endl << endl;
+    usage(argv[0]);
+    return 1;
+  }
 
-  foreach (const Action& action, actions.get()) {
-    cout << "----------------------------------------------" << endl;
-    action.PrintDebugString();
+  // Execute the command.
+  Try<Nothing> execute = tools[command]->execute(argc, argv);
+  if (execute.isError()) {
+    cerr << execute.error() << endl;
+    return 1;
   }
 
   return 0;

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/replica.cpp
----------------------------------------------------------------------
diff --git a/src/log/replica.cpp b/src/log/replica.cpp
index da9310f..ec6e38c 100644
--- a/src/log/replica.cpp
+++ b/src/log/replica.cpp
@@ -16,12 +16,6 @@
  * limitations under the License.
  */
 
-#include <google/protobuf/io/zero_copy_stream_impl.h>
-
-#include <leveldb/comparator.h>
-#include <leveldb/db.h>
-#include <leveldb/write_batch.h>
-
 #include <algorithm>
 
 #include <process/dispatch.hpp>
@@ -32,22 +26,18 @@
 #include <stout/foreach.hpp>
 #include <stout/none.hpp>
 #include <stout/nothing.hpp>
-#include <stout/numify.hpp>
-#include <stout/stopwatch.hpp>
+#include <stout/result.hpp>
+#include <stout/try.hpp>
 #include <stout/utils.hpp>
 
 #include "common/type_utils.hpp"
 
+#include "log/leveldb.hpp"
 #include "log/replica.hpp"
-
-#include "logging/logging.hpp"
-
-#include "messages/log.hpp"
+#include "log/storage.hpp"
 
 using namespace process;
 
-using process::wait; // Necessary on some OS's to disambiguate.
-
 using std::list;
 using std::set;
 using std::string;
@@ -66,425 +56,6 @@ Protocol<RecoverRequest, RecoverResponse> recover;
 } // namespace protocol {
 
 
-struct State
-{
-  Metadata metadata; // The metadata for the replica.
-  uint64_t begin; // Beginning position of the log.
-  uint64_t end; // Ending position of the log.
-  set<uint64_t> learned; // Positions present and learned
-  set<uint64_t> unlearned; // Positions present but unlearned.
-};
-
-
-// Abstract interface for reading and writing records.
-class Storage
-{
-public:
-  virtual ~Storage() {}
-  virtual Try<State> restore(const string& path) = 0;
-  virtual Try<Nothing> persist(const Metadata& metadata) = 0;
-  virtual Try<Nothing> persist(const Action& action) = 0;
-  virtual Try<Action> read(uint64_t position) = 0;
-};
-
-
-// Concrete implementation of the storage interface using leveldb.
-class LevelDBStorage : public Storage
-{
-public:
-  LevelDBStorage();
-  virtual ~LevelDBStorage();
-
-  virtual Try<State> restore(const string& path);
-  virtual Try<Nothing> persist(const Metadata& metadata);
-  virtual Try<Nothing> persist(const Action& action);
-  virtual Try<Action> read(uint64_t position);
-
-private:
-  class Varint64Comparator : public leveldb::Comparator
-  {
-  public:
-    virtual int Compare(
-        const leveldb::Slice& a,
-        const leveldb::Slice& b) const
-    {
-      // TODO(benh): Use varint comparator.
-      LOG(FATAL) << "Unimplemented";
-      // uint64_t left = position(a);
-      // uint64_t right = position(b);
-      // if (left < right) return -1;
-      // if (left == right) return 0;
-      // if (left > right) return 1;
-    }
-
-    virtual const char* Name() const
-    {
-      // Note that this name MUST NOT CHANGE across uses of this
-      // comparator with the same DB (the semantics of doing so are
-      // undefined if the database doesn't catch this first).
-      return "varint64";
-    }
-
-    virtual void FindShortestSeparator(
-        string* start,
-        const leveldb::Slice& limit) const
-    {
-      // Intentional no-op.
-    }
-
-    virtual void FindShortSuccessor(string* key) const
-    {
-      // Intentional no-op.
-    }
-  };
-
-  // Returns a string representing the specified position. Note that
-  // we adjust the actual position by incrementing it by 1 because we
-  // reserve 0 for storing the promise record (Record::Promise,
-  // DEPRECATED!), or the metadata (Record::Metadata).
-  static string encode(uint64_t position, bool adjust = true)
-  {
-    // Adjusted stringified represenation is plus 1 of actual position.
-    position = adjust ? position + 1 : position;
-
-    // TODO(benh): Use varint encoding for VarInt64Comparator!
-    // string s;
-    // google::protobuf::io::StringOutputStream _stream(&s);
-    // google::protobuf::io::CodedOutputStream stream(&_stream);
-    // position = adjust ? position + 1 : position;
-    // stream.WriteVarint64(position);
-    // return s;
-
-    Try<string> s = strings::format("%.*d", 10, position);
-    CHECK_SOME(s);
-    return s.get();
-  }
-
-  // Returns the position as represented in the specified slice
-  // (performing a decrement as necessary to determine the actual
-  // position represented).
-  static uint64_t decode(const leveldb::Slice& s)
-  {
-    // TODO(benh): Use varint decoding for VarInt64Comparator!
-    // uint64_t position;
-    // google::protobuf::io::ArrayInputStream _stream(s.data(), s.size());
-    // google::protobuf::io::CodedInputStream stream(&_stream);
-    // bool success = stream.ReadVarint64(&position);
-    // CHECK(success);
-    // return position - 1; // Actual position is less 1 of stringified.
-    Try<uint64_t> position = numify<uint64_t>(string(s.data(), s.size()));
-    CHECK_SOME(position);
-    return position.get() - 1; // Actual position is less 1 of stringified.
-  }
-
-  // Varint64Comparator comparator; // TODO(benh): Use varint comparator.
-
-  leveldb::DB* db;
-
-  uint64_t first; // First position still in leveldb, used during truncation.
-};
-
-
-LevelDBStorage::LevelDBStorage()
-  : db(NULL), first(0)
-{
-  // Nothing to see here.
-}
-
-
-LevelDBStorage::~LevelDBStorage()
-{
-  delete db; // Might be null if open failed in LevelDBStorage::recover.
-}
-
-
-Try<State> LevelDBStorage::restore(const string& path)
-{
-  leveldb::Options options;
-  options.create_if_missing = true;
-
-  // TODO(benh): Can't use varint comparator until bug discussed at
-  // groups.google.com/group/leveldb/browse_thread/thread/17eac39168909ba7
-  // gets fixed. For now, we are using the default byte-wise
-  // comparator and *assuming* that the encoding from unsigned long to
-  // string produces a stable ordering. Checks below.
-  // options.comparator = &comparator;
-
-  const string& one = encode(1);
-  const string& two = encode(2);
-  const string& ten = encode(10);
-
-  CHECK(leveldb::BytewiseComparator()->Compare(one, two) < 0);
-  CHECK(leveldb::BytewiseComparator()->Compare(two, one) > 0);
-  CHECK(leveldb::BytewiseComparator()->Compare(one, ten) < 0);
-  CHECK(leveldb::BytewiseComparator()->Compare(ten, two) > 0);
-  CHECK(leveldb::BytewiseComparator()->Compare(ten, ten) == 0);
-
-  Stopwatch stopwatch;
-  stopwatch.start();
-
-  leveldb::Status status = leveldb::DB::Open(options, path, &db);
-
-  if (!status.ok()) {
-    // TODO(benh): Consider trying to repair the DB.
-    return Error(status.ToString());
-  }
-
-  LOG(INFO) << "Opened db in " << stopwatch.elapsed();
-
-  stopwatch.start(); // Restart the stopwatch.
-
-  // TODO(benh): Conditionally compact to avoid long recovery times?
-  db->CompactRange(NULL, NULL);
-
-  LOG(INFO) << "Compacted db in " << stopwatch.elapsed();
-
-  State state;
-  state.begin = 0;
-  state.end = 0;
-
-  // TODO(benh): Consider just reading the "promise" record (e.g.,
-  // 'encode(0, false)') and then iterating over the rest of the
-  // records and confirming that they are all indeed of type
-  // Record::Action.
-
-  stopwatch.start(); // Restart the stopwatch.
-
-  leveldb::Iterator* iterator = db->NewIterator(leveldb::ReadOptions());
-
-  LOG(INFO) << "Created db iterator in " << stopwatch.elapsed();
-
-  stopwatch.start(); // Restart the stopwatch.
-
-  iterator->SeekToFirst();
-
-  LOG(INFO) << "Seeked to beginning of db in " << stopwatch.elapsed();
-
-  stopwatch.start(); // Restart the stopwatch.
-
-  uint64_t keys = 0;
-
-  while (iterator->Valid()) {
-    keys++;
-    const leveldb::Slice& slice = iterator->value();
-
-    google::protobuf::io::ArrayInputStream stream(slice.data(), slice.size());
-
-    Record record;
-
-    if (!record.ParseFromZeroCopyStream(&stream)) {
-      return Error("Failed to deserialize record");
-    }
-
-    switch (record.type()) {
-      case Record::METADATA: {
-        CHECK(record.has_metadata());
-        state.metadata.CopyFrom(record.metadata());
-        break;
-      }
-
-      // DEPRECATED!
-      case Record::PROMISE: {
-        CHECK(record.has_promise());
-        // This replica is in old format. Set its status to VOTING
-        // since there is no catch-up logic in the old code and this
-        // replica is obviously not empty.
-        state.metadata.set_status(Metadata::VOTING);
-        state.metadata.set_promised(record.promise().proposal());
-        break;
-      }
-
-      case Record::ACTION: {
-        CHECK(record.has_action());
-        const Action& action = record.action();
-        if (action.has_learned() && action.learned()) {
-          state.learned.insert(action.position());
-          state.unlearned.erase(action.position());
-          if (action.has_type() && action.type() == Action::TRUNCATE) {
-            state.begin = std::max(state.begin, action.truncate().to());
-          }
-        } else {
-          state.learned.erase(action.position());
-          state.unlearned.insert(action.position());
-        }
-        state.end = std::max(state.end, action.position());
-        break;
-      }
-
-      default: {
-        return Error("Bad record");
-      }
-    }
-
-    iterator->Next();
-  }
-
-  LOG(INFO) << "Iterated through " << keys
-            << " keys in the db in " << stopwatch.elapsed();
-
-  // Determine the first position still in leveldb so during a
-  // truncation we can attempt to delete all positions from the first
-  // position up to the truncate position. Note that this is not the
-  // beginning position of the log, but rather the first position that
-  // remains (i.e., hasn't been deleted) in leveldb.
-  iterator->Seek(encode(0));
-
-  if (iterator->Valid()) {
-    first = decode(iterator->key());
-  }
-
-  delete iterator;
-
-  return state;
-}
-
-
-Try<Nothing> LevelDBStorage::persist(const Metadata& metadata)
-{
-  Stopwatch stopwatch;
-  stopwatch.start();
-
-  leveldb::WriteOptions options;
-  options.sync = true;
-
-  Record record;
-  record.set_type(Record::METADATA);
-  record.mutable_metadata()->CopyFrom(metadata);
-
-  string value;
-
-  if (!record.SerializeToString(&value)) {
-    return Error("Failed to serialize record");
-  }
-
-  leveldb::Status status = db->Put(options, encode(0, false), value);
-
-  if (!status.ok()) {
-    return Error(status.ToString());
-  }
-
-  LOG(INFO) << "Persisting metadata (" << value.size()
-            << " bytes) to leveldb took " << stopwatch.elapsed();
-
-  return Nothing();
-}
-
-
-Try<Nothing> LevelDBStorage::persist(const Action& action)
-{
-  Stopwatch stopwatch;
-  stopwatch.start();
-
-  Record record;
-  record.set_type(Record::ACTION);
-  record.mutable_action()->MergeFrom(action);
-
-  string value;
-
-  if (!record.SerializeToString(&value)) {
-    return Error("Failed to serialize record");
-  }
-
-  leveldb::WriteOptions options;
-  options.sync = true;
-
-  leveldb::Status status = db->Put(options, encode(action.position()), value);
-
-  if (!status.ok()) {
-    return Error(status.ToString());
-  }
-
-  LOG(INFO) << "Persisting action (" << value.size()
-            << " bytes) to leveldb took " << stopwatch.elapsed();
-
-  // Delete positions if a truncate action has been *learned*. Note
-  // that we do this in a best-effort fashion (i.e., we ignore any
-  // failures to the database since we can always try again).
-  if (action.has_type() && action.type() == Action::TRUNCATE &&
-      action.has_learned() && action.learned()) {
-    CHECK(action.has_truncate());
-
-    stopwatch.start(); // Restart the stopwatch.
-
-    // To actually perform the truncation in leveldb we need to remove
-    // all the keys that represent positions no longer in the log. We
-    // do this by attempting to delete all keys that represent the
-    // first position we know is still in leveldb up to (but
-    // excluding) the truncate position. Note that this works because
-    // the semantics of WriteBatch are such that even if the position
-    // doesn't exist (which is possible because this replica has some
-    // holes), we can attempt to delete the key that represents it and
-    // it will just ignore that key. This is *much* cheaper than
-    // actually iterating through the entire database instead (which
-    // was, for posterity, the original implementation). In addition,
-    // caching the "first" position we know is in the database is
-    // cheaper than using an iterator to determine the first position
-    // (which was, for posterity, the second implementation).
-
-    leveldb::WriteBatch batch;
-
-    // Add positions up to (but excluding) the truncate position to
-    // the batch starting at the first position still in leveldb.
-    uint64_t index = 0;
-    while ((first + index) < action.truncate().to()) {
-      batch.Delete(encode(first + index));
-      index++;
-    }
-
-    // If we added any positions, attempt to delete them!
-    if (index > 0) {
-      // We do this write asynchronously (e.g., using default options).
-      leveldb::Status status = db->Write(leveldb::WriteOptions(), &batch);
-
-      if (!status.ok()) {
-        LOG(WARNING) << "Ignoring leveldb batch delete failure: "
-                     << status.ToString();
-      } else {
-        first = action.truncate().to(); // Save the new first position!
-
-        LOG(INFO) << "Deleting ~" << index
-                  << " keys from leveldb took " << stopwatch.elapsed();
-      }
-    }
-  }
-
-  return Nothing();
-}
-
-
-Try<Action> LevelDBStorage::read(uint64_t position)
-{
-  Stopwatch stopwatch;
-  stopwatch.start();
-
-  leveldb::ReadOptions options;
-
-  string value;
-
-  leveldb::Status status = db->Get(options, encode(position), &value);
-
-  if (!status.ok()) {
-    return Error(status.ToString());
-  }
-
-  google::protobuf::io::ArrayInputStream stream(value.data(), value.size());
-
-  Record record;
-
-  if (!record.ParseFromZeroCopyStream(&stream)) {
-    return Error("Failed to deserialize record");
-  }
-
-  if (record.type() != Record::ACTION) {
-    return Error("Bad record");
-  }
-
-  LOG(INFO) << "Reading position from leveldb took " << stopwatch.elapsed();
-
-  return record.action();
-}
-
-
 class ReplicaProcess : public ProtobufProcess<ReplicaProcess>
 {
 public:
@@ -1140,7 +711,7 @@ bool ReplicaProcess::persist(const Action& action)
 
 void ReplicaProcess::restore(const string& path)
 {
-  Try<State> state = storage->restore(path);
+  Try<Storage::State> state = storage->restore(path);
 
   CHECK_SOME(state) << "Failed to recover the log";
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/replica.hpp
----------------------------------------------------------------------
diff --git a/src/log/replica.hpp b/src/log/replica.hpp
index ecb126d..467d0d9 100644
--- a/src/log/replica.hpp
+++ b/src/log/replica.hpp
@@ -19,14 +19,16 @@
 #ifndef __LOG_REPLICA_HPP__
 #define __LOG_REPLICA_HPP__
 
+#include <stdint.h>
+
 #include <list>
 #include <set>
 #include <string>
 
+#include <process/future.hpp>
+#include <process/pid.hpp>
 #include <process/protobuf.hpp>
 
-#include <stout/result.hpp>
-
 #include "messages/log.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/storage.hpp
----------------------------------------------------------------------
diff --git a/src/log/storage.hpp b/src/log/storage.hpp
new file mode 100644
index 0000000..663146f
--- /dev/null
+++ b/src/log/storage.hpp
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_STORAGE_HPP__
+#define __LOG_STORAGE_HPP__
+
+#include <stdint.h>
+
+#include <set>
+#include <string>
+
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+#include "messages/log.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// Abstract interface for reading and writing records.
+class Storage
+{
+public:
+  struct State
+  {
+    Metadata metadata; // The metadata for the replica.
+    uint64_t begin; // Beginning position of the log.
+    uint64_t end; // Ending position of the log.
+    std::set<uint64_t> learned; // Positions present and learned
+    std::set<uint64_t> unlearned; // Positions present but unlearned.
+  };
+
+  virtual ~Storage() {}
+
+  virtual Try<State> restore(const std::string& path) = 0;
+  virtual Try<Nothing> persist(const Metadata& metadata) = 0;
+  virtual Try<Nothing> persist(const Action& action) = 0;
+  virtual Try<Action> read(uint64_t position) = 0;
+};
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_STORAGE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/tool.hpp
----------------------------------------------------------------------
diff --git a/src/log/tool.hpp b/src/log/tool.hpp
new file mode 100644
index 0000000..656d3f6
--- /dev/null
+++ b/src/log/tool.hpp
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_TOOL_HPP__
+#define __LOG_TOOL_HPP__
+
+#include <string>
+
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace internal {
+namespace log {
+namespace tool {
+
+// Represents a tool for processing a log file.
+class Tool
+{
+public:
+  virtual ~Tool() {}
+
+  virtual std::string name() const = 0;
+
+  // Executes the tool. The tool can be configured by passing in
+  // command line arguments. If command line arguments are not
+  // specified, the default configuration will be used.
+  virtual Try<Nothing> execute(int argc, char** argv) = 0;
+};
+
+} // namespace tool {
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_TOOL_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/tool/initialize.cpp
----------------------------------------------------------------------
diff --git a/src/log/tool/initialize.cpp b/src/log/tool/initialize.cpp
new file mode 100644
index 0000000..ccda7fb
--- /dev/null
+++ b/src/log/tool/initialize.cpp
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <iostream>
+#include <sstream>
+
+#include <process/process.hpp>
+#include <process/timeout.hpp>
+
+#include <stout/error.hpp>
+
+#include "log/replica.hpp"
+#include "log/tool/initialize.hpp"
+
+#include "logging/logging.hpp"
+
+using namespace process;
+
+using std::endl;
+using std::ostringstream;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace log {
+namespace tool {
+
+Initialize::Flags::Flags()
+{
+  add(&Flags::path,
+      "path",
+      "Path to the log");
+
+  add(&Flags::timeout,
+      "timeout",
+      "Maximum time allowed for the command to finish\n"
+      "(e.g., 500ms, 1sec, etc.)");
+
+  add(&Flags::help,
+      "help",
+      "Prints the help message",
+      false);
+}
+
+
+string Initialize::usage(const string& argv0) const
+{
+  ostringstream out;
+
+  out << "Usage: " << argv0 << " " << name() << " [OPTIONS]" << endl
+      << endl
+      << "This command is used to initialize the log" << endl
+      << endl
+      << "Supported OPTIONS:" << endl
+      << flags.usage();
+
+  return out.str();
+}
+
+
+Try<Nothing> Initialize::execute(int argc, char** argv)
+{
+  // Configure the tool by parsing command line arguments.
+  if (argc > 0 && argv != NULL) {
+    Try<Nothing> load = flags.load(None(), argc, argv);
+    if (load.isError()) {
+      return Error(load.error() + "\n\n" + usage(argv[0]));
+    }
+
+    if (flags.help) {
+      return Error(usage(argv[0]));
+    }
+
+    process::initialize();
+    logging::initialize(argv[0], flags);
+  }
+
+  if (flags.path.isNone()) {
+    return Error("Missing flag: '--path'");
+  }
+
+  // Setup the timeout if specified.
+  Option<Timeout> timeout = None();
+  if (flags.timeout.isSome()) {
+    timeout = Timeout::in(flags.timeout.get());
+  }
+
+  Replica replica(flags.path.get());
+
+  // Get the current status of the replica.
+  Future<Metadata::Status> status = replica.status();
+  if (timeout.isSome()) {
+    status.await(timeout.get().remaining());
+  } else {
+    status.await();
+  }
+
+  if (status.isPending()) {
+    return Error("Timed out while getting replica status");
+  } else if (status.isDiscarded()) {
+    return Error("Failed to get status of replica (discarded future)");
+  } else if (status.isFailed()) {
+    return Error(status.failure());
+  }
+
+  // We only initialize a log if it is empty.
+  if (status.get() != Metadata::EMPTY) {
+    return Error("The log is not empty");
+  }
+
+  // Update the status of the replica to VOTING.
+  Future<bool> update = replica.update(Metadata::VOTING);
+  if (timeout.isSome()) {
+    update.await(timeout.get().remaining());
+  } else {
+    update.await();
+  }
+
+  if (update.isPending()) {
+    return Error("Timed out while setting replica status");
+  } else if (update.isDiscarded()) {
+    return Error("Failed to set replica status (discarded future)");
+  } else if (update.isFailed()) {
+    return Error(update.failure());
+  }
+
+  return Nothing();
+}
+
+} // namespace tool {
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/tool/initialize.hpp
----------------------------------------------------------------------
diff --git a/src/log/tool/initialize.hpp b/src/log/tool/initialize.hpp
new file mode 100644
index 0000000..10ac269
--- /dev/null
+++ b/src/log/tool/initialize.hpp
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_TOOL_INITIALIZE_HPP__
+#define __LOG_TOOL_INITIALIZE_HPP__
+
+#include <stout/duration.hpp>
+#include <stout/flags.hpp>
+#include <stout/option.hpp>
+
+#include "log/tool.hpp"
+
+#include "logging/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+namespace tool {
+
+class Initialize : public Tool
+{
+public:
+  class Flags : public logging::Flags
+  {
+  public:
+    Flags();
+
+    Option<std::string> path;
+    Option<Duration> timeout;
+    bool help;
+  };
+
+  virtual std::string name() const { return "initialize"; }
+  virtual Try<Nothing> execute(int argc = 0, char** argv = NULL);
+
+  // Users can change the default configuration by setting this flags.
+  Flags flags;
+
+private:
+  std::string usage(const std::string& argv0) const;
+};
+
+} // namespace tool {
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_TOOL_INITIALIZE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/tool/read.cpp
----------------------------------------------------------------------
diff --git a/src/log/tool/read.cpp b/src/log/tool/read.cpp
new file mode 100644
index 0000000..ab6068d
--- /dev/null
+++ b/src/log/tool/read.cpp
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <iostream>
+#include <sstream>
+
+#include <process/process.hpp>
+#include <process/timeout.hpp>
+
+#include <stout/error.hpp>
+
+#include "log/replica.hpp"
+#include "log/tool/read.hpp"
+
+#include "logging/logging.hpp"
+
+using namespace process;
+
+using std::cout;
+using std::endl;
+using std::list;
+using std::ostringstream;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace log {
+namespace tool {
+
+Read::Flags::Flags()
+{
+  add(&Flags::path,
+      "path",
+      "Path to the log");
+
+  add(&Flags::from,
+      "from",
+      "Position from which to start reading the log");
+
+  add(&Flags::to,
+      "to",
+      "Position from which to stop reading the log");
+
+  add(&Flags::timeout,
+      "timeout",
+      "Maximum time allowed for the command to finish\n"
+      "(e.g., 500ms, 1sec, etc.)");
+
+  add(&Flags::help,
+      "help",
+      "Prints the help message",
+      false);
+}
+
+
+string Read::usage(const string& argv0) const
+{
+  ostringstream out;
+
+  out << "Usage: " << argv0 << " " << name() << " [OPTIONS]" << endl
+      << endl
+      << "This command is used to read the log" << endl
+      << endl
+      << "Supported OPTIONS:" << endl
+      << flags.usage();
+
+  return out.str();
+}
+
+
+Try<Nothing> Read::execute(int argc, char** argv)
+{
+  // Configure the tool by parsing command line arguments.
+  if (argc > 0 && argv != NULL) {
+    Try<Nothing> load = flags.load(None(), argc, argv);
+    if (load.isError()) {
+      return Error(load.error() + "\n\n" + usage(argv[0]));
+    }
+
+    if (flags.help) {
+      return Error(usage(argv[0]));
+    }
+
+    process::initialize();
+    logging::initialize(argv[0], flags);
+  }
+
+  if (flags.path.isNone()) {
+    return Error("Missing flag '--path'");
+  }
+
+  // Setup the timeout if specified.
+  Option<Timeout> timeout = None();
+  if (flags.timeout.isSome()) {
+    timeout = Timeout::in(flags.timeout.get());
+  }
+
+  Replica replica(flags.path.get());
+
+  // Get the beginning of the replica.
+  Future<uint64_t> begin = replica.beginning();
+  if (timeout.isSome()) {
+    begin.await(timeout.get().remaining());
+  } else {
+    begin.await();
+  }
+
+  if (begin.isPending()) {
+    return Error("Timed out while getting the beginning of the replica");
+  } else if (begin.isDiscarded()) {
+    return Error(
+        "Failed to get the beginning of the replica (discarded future)");
+  } else if (begin.isFailed()) {
+    return Error(begin.failure());
+  }
+
+  // Get the ending of the replica.
+  Future<uint64_t> end = replica.ending();
+  if (timeout.isSome()) {
+    end.await(timeout.get().remaining());
+  } else {
+    end.await();
+  }
+
+  if (end.isPending()) {
+    return Error("Timed out while getting the ending of the replica");
+  } else if (end.isDiscarded()) {
+    return Error(
+        "Failed to get the ending of the replica (discarded future)");
+  } else if (end.isFailed()) {
+    return Error(end.failure());
+  }
+
+  Option<uint64_t> from = flags.from;
+  if (from.isNone()) {
+    from = begin.get();
+  }
+
+  Option<uint64_t> to = flags.to;
+  if (to.isNone()) {
+    to = end.get();
+  }
+
+  LOG(INFO) << "Attempting to read the log from "
+            << from.get() << " to " << to.get() << endl;
+
+  Future<list<Action> > actions = replica.read(from.get(), to.get());
+  if (timeout.isSome()) {
+    actions.await(timeout.get().remaining());
+  } else {
+    actions.await();
+  }
+
+  if (actions.isPending()) {
+    return Error("Timed out while reading the replica");
+  } else if (actions.isDiscarded()) {
+    return Error("Failed to read the replica (discarded future)");
+  } else if (actions.isFailed()) {
+    return Error(actions.failure());
+  }
+
+  foreach (const Action& action, actions.get()) {
+    cout << "----------------------------------------------" << endl;
+    action.PrintDebugString();
+  }
+
+  return Nothing();
+}
+
+} // namespace tool {
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2fe5860/src/log/tool/read.hpp
----------------------------------------------------------------------
diff --git a/src/log/tool/read.hpp b/src/log/tool/read.hpp
new file mode 100644
index 0000000..74faec0
--- /dev/null
+++ b/src/log/tool/read.hpp
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_TOOL_READ_HPP__
+#define __LOG_TOOL_READ_HPP__
+
+#include <stout/duration.hpp>
+#include <stout/flags.hpp>
+#include <stout/option.hpp>
+
+#include "log/tool.hpp"
+
+#include "logging/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+namespace tool {
+
+class Read : public Tool
+{
+public:
+  class Flags : public logging::Flags
+  {
+  public:
+    Flags();
+
+    Option<std::string> path;
+    Option<uint64_t> from;
+    Option<uint64_t> to;
+    Option<Duration> timeout;
+    bool help;
+  };
+
+  virtual std::string name() const { return "read"; }
+  virtual Try<Nothing> execute(int argc = 0, char** argv = NULL);
+
+  // Users can change the default configuration by setting this flags.
+  Flags flags;
+
+private:
+  std::string usage(const std::string& argv0) const;
+};
+
+} // namespace tool {
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_TOOL_READ_HPP__


[02/10] git commit: Decoupled replicated log coordinator logic and made it asynchronous.

Posted by be...@apache.org.
Decoupled replicated log coordinator logic and made it asynchronous.

This is the first patch of a series of patches that implement a
catch-up mechanism for replicated log. See the following ticket for
more details: https://issues.apache.org/jira/browse/MESOS-736.

Here is a brief summary of this patch: (Sorry for the fact that we are
not able to break it into smaller patches :().

1) Pulled the original Coordinator logic out and divides it into
several Paxos phases (see src/log/consensus.hpp). Instead of using a
blocking semantics, we implemented all the logics asynchronously.

2) In order to ensure the liveness of a catch-uper, we implemented a
retry logic by bumping the proposal number. This also requires us to
slightly change the existing replica protocol.

3) Made the "fill" operation independent of the underlying
replica. Instead, introduced a catchup (see src/log/catchup.hpp)
function to make sure the underlying local replica has learned each
write.

4) Modified the log tests to adapt to the new semantics (see (3)
above).

This is a joint work with Yan Xu.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/14631


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/19ad88b7
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/19ad88b7
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/19ad88b7

Branch: refs/heads/master
Commit: 19ad88b7c45164c1272001493bdd176d80a88b91
Parents: 2ff5308
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:51:45 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:51:45 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am         |  17 +-
 src/log/catchup.cpp     | 286 +++++++++++++++++
 src/log/catchup.hpp     |  54 ++++
 src/log/consensus.cpp   | 711 +++++++++++++++++++++++++++++++++++++++++++
 src/log/consensus.hpp   | 136 +++++++++
 src/log/coordinator.cpp | 472 ++++++----------------------
 src/log/coordinator.hpp |  58 +---
 src/log/log.hpp         |  69 ++---
 src/log/network.hpp     |  57 ++--
 src/log/replica.cpp     | 351 +++++++++++----------
 src/log/replica.hpp     |  25 +-
 src/messages/log.proto  |  91 +++---
 src/tests/log_tests.cpp | 420 +++++++++++++------------
 13 files changed, 1866 insertions(+), 881 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index cf0c8c6..17fbf83 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -297,9 +297,20 @@ libmesos_no_3rdparty_la_LIBADD += libbuild.la
 # Convenience library for building the replicated log in order to
 # include the leveldb headers.
 noinst_LTLIBRARIES += liblog.la
-liblog_la_SOURCES = log/coordinator.cpp log/replica.cpp
-liblog_la_SOURCES += log/coordinator.hpp log/replica.hpp log/log.hpp	\
-  log/network.hpp messages/log.hpp messages/log.proto
+liblog_la_SOURCES =							\
+  log/catchup.cpp							\
+  log/consensus.cpp							\
+  log/coordinator.cpp							\
+  log/replica.cpp
+liblog_la_SOURCES +=							\
+  log/catchup.hpp							\
+  log/consensus.hpp							\
+  log/coordinator.hpp							\
+  log/replica.hpp							\
+  log/log.hpp								\
+  log/network.hpp							\
+  messages/log.hpp							\
+  messages/log.proto
 nodist_liblog_la_SOURCES = $(LOG_PROTOS)
 liblog_la_CPPFLAGS = -I../$(LEVELDB)/include $(MESOS_CPPFLAGS)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/catchup.cpp
----------------------------------------------------------------------
diff --git a/src/log/catchup.cpp b/src/log/catchup.cpp
new file mode 100644
index 0000000..5825eae
--- /dev/null
+++ b/src/log/catchup.cpp
@@ -0,0 +1,286 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <list>
+
+#include <process/collect.hpp>
+#include <process/id.hpp>
+#include <process/process.hpp>
+
+#include <stout/lambda.hpp>
+#include <stout/stringify.hpp>
+
+#include "log/catchup.hpp"
+#include "log/consensus.hpp"
+
+#include "messages/log.hpp"
+
+using namespace process;
+
+using std::list;
+using std::set;
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+class CatchUpProcess : public Process<CatchUpProcess>
+{
+public:
+  CatchUpProcess(
+      size_t _quorum,
+      const Shared<Replica>& _replica,
+      const Shared<Network>& _network,
+      uint64_t _proposal,
+      uint64_t _position)
+    : ProcessBase(ID::generate("log-catch-up")),
+      quorum(_quorum),
+      replica(_replica),
+      network(_network),
+      position(_position),
+      proposal(_proposal) {}
+
+  virtual ~CatchUpProcess() {}
+
+  Future<uint64_t> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    check();
+  }
+
+  virtual void finalize()
+  {
+    checking.discard();
+    filling.discard();
+  }
+
+private:
+  void check()
+  {
+    checking = replica->missing(position);
+    checking.onAny(defer(self(), &Self::checked));
+  }
+
+  void checked()
+  {
+    // The future 'checking' can only be discarded in 'finalize'.
+    CHECK(!checking.isDiscarded());
+
+    if (checking.isFailed()) {
+      promise.fail("Failed to get missing positions: " + checking.failure());
+      terminate(self());
+    } else if (!checking.get()) {
+      // The position has been learned.
+      promise.set(proposal);
+      terminate(self());
+    } else {
+      // Still missing, try to fill it.
+      fill();
+    }
+  }
+
+  void fill()
+  {
+    filling = log::fill(quorum, network, proposal, position);
+    filling.onAny(defer(self(), &Self::filled));
+  }
+
+  void filled()
+  {
+    // The future 'filling' can only be discarded in 'finalize'.
+    CHECK(!filling.isDiscarded());
+
+    if (filling.isFailed()) {
+      promise.fail("Failed to fill missing position: " + filling.failure());
+      terminate(self());
+    } else {
+      // Update the proposal number so that we can save a proposal
+      // number bump round trip if we need to invoke fill again.
+      CHECK(filling.get().promised() >= proposal);
+      proposal = filling.get().promised();
+
+      check();
+    }
+  }
+
+  const size_t quorum;
+  const Shared<Replica> replica;
+  const Shared<Network> network;
+  const uint64_t position;
+
+  uint64_t proposal;
+
+  process::Promise<uint64_t> promise;
+  Future<bool> checking;
+  Future<Action> filling;
+};
+
+
+// Catches-up a single log position in the local replica. This
+// function returns the highest proposal number seen. The returned
+// proposal number can be used to save extra proposal number bumps.
+static Future<uint64_t> catchup(
+    size_t quorum,
+    const Shared<Replica>& replica,
+    const Shared<Network>& network,
+    uint64_t proposal,
+    uint64_t position)
+{
+  CatchUpProcess* process =
+    new CatchUpProcess(
+        quorum,
+        replica,
+        network,
+        proposal,
+        position);
+
+  Future<uint64_t> future = process->future();
+  spawn(process, true);
+  return future;
+}
+
+
+// TODO(jieyu): Our current implementation catches-up each position in
+// the set sequentially. In the future, we may want to parallelize it
+// to improve the performance. Also, we may want to implement rate
+// control here so that we don't saturate the network or disk.
+class BulkCatchUpProcess : public Process<BulkCatchUpProcess>
+{
+public:
+  BulkCatchUpProcess(
+      size_t _quorum,
+      const Shared<Replica>& _replica,
+      const Shared<Network>& _network,
+      uint64_t _proposal,
+      const set<uint64_t>& _positions)
+    : ProcessBase(ID::generate("log-bulk-catch-up")),
+      quorum(_quorum),
+      replica(_replica),
+      network(_network),
+      positions(_positions),
+      proposal(_proposal) {}
+
+  virtual ~BulkCatchUpProcess() {}
+
+  Future<Nothing> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    // Catch-up each position in the set sequentially.
+    it = positions.begin();
+
+    catchup();
+  }
+
+  virtual void finalize()
+  {
+    catching.discard();
+  }
+
+private:
+  void catchup()
+  {
+    if (it == positions.end()) {
+      promise.set(Nothing());
+      terminate(self());
+      return;
+    }
+
+    // Store the future so that we can discard it if the user wants to
+    // cancel the catch-up operation.
+    catching = log::catchup(quorum, replica, network, proposal, *it);
+    catching.onAny(defer(self(), &Self::caughtup));
+  }
+
+  void caughtup()
+  {
+    // No one can discard the future 'catching' except the 'finalize'.
+    CHECK(!catching.isDiscarded());
+
+    if (catching.isFailed()) {
+      promise.fail(
+          "Failed to catch-up position " + stringify(*it) +
+          ": " + catching.failure());
+      terminate(self());
+      return;
+    }
+
+    ++it;
+
+    // The single position catch-up function: 'log::catchup' will
+    // return the highest proposal number seen so far. We use this
+    // proposal number for the next 'catchup' as it is highly likely
+    // that this number is high enough, saving potentially unnecessary
+    // proposal number bumps.
+    proposal = catching.get();
+
+    catchup();
+  }
+
+  const size_t quorum;
+  const Shared<Replica> replica;
+  const Shared<Network> network;
+  const set<uint64_t> positions;
+
+  uint64_t proposal;
+  set<uint64_t>::iterator it;
+
+  process::Promise<Nothing> promise;
+  Future<uint64_t> catching;
+};
+
+
+/////////////////////////////////////////////////
+// Public interfaces below.
+/////////////////////////////////////////////////
+
+
+Future<Nothing> catchup(
+    size_t quorum,
+    const Shared<Replica>& replica,
+    const Shared<Network>& network,
+    uint64_t proposal,
+    const set<uint64_t>& positions)
+{
+  BulkCatchUpProcess* process =
+    new BulkCatchUpProcess(
+        quorum,
+        replica,
+        network,
+        proposal,
+        positions);
+
+  Future<Nothing> future = process->future();
+  spawn(process, true);
+  return future;
+}
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/catchup.hpp
----------------------------------------------------------------------
diff --git a/src/log/catchup.hpp b/src/log/catchup.hpp
new file mode 100644
index 0000000..3652830
--- /dev/null
+++ b/src/log/catchup.hpp
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_CATCHUP_HPP__
+#define __LOG_CATCHUP_HPP__
+
+#include <stdint.h>
+
+#include <set>
+
+#include <process/future.hpp>
+#include <process/shared.hpp>
+
+#include <stout/nothing.hpp>
+
+#include "log/network.hpp"
+#include "log/replica.hpp"
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// Catches-up a set of log positions in the local replica. The user of
+// this function can provide a hint on the proposal number that will
+// be used for Paxos. This could potentially save us a few Paxos
+// rounds. However, if the user has no idea what proposal number to
+// use, he can just use an arbitrary proposal number (e.g., 0).
+extern process::Future<Nothing> catchup(
+    size_t quorum,
+    const process::Shared<Replica>& replica,
+    const process::Shared<Network>& network,
+    uint64_t proposal,
+    const std::set<uint64_t>& positions);
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_CATCHUP_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/consensus.cpp
----------------------------------------------------------------------
diff --git a/src/log/consensus.cpp b/src/log/consensus.cpp
new file mode 100644
index 0000000..5eb90e7
--- /dev/null
+++ b/src/log/consensus.cpp
@@ -0,0 +1,711 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdlib.h>
+
+#include <set>
+
+#include <process/defer.hpp>
+#include <process/delay.hpp>
+#include <process/id.hpp>
+#include <process/process.hpp>
+
+#include <stout/check.hpp>
+#include <stout/duration.hpp>
+#include <stout/lambda.hpp>
+#include <stout/nothing.hpp>
+#include <stout/foreach.hpp>
+
+#include "log/consensus.hpp"
+#include "log/replica.hpp"
+
+using namespace process;
+
+using std::set;
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+class ExplicitPromiseProcess : public Process<ExplicitPromiseProcess>
+{
+public:
+  ExplicitPromiseProcess(
+      size_t _quorum,
+      const Shared<Network>& _network,
+      uint64_t _proposal,
+      uint64_t _position)
+    : ProcessBase(ID::generate("log-explicit-promise")),
+      quorum(_quorum),
+      network(_network),
+      proposal(_proposal),
+      position(_position),
+      responsesReceived(0) {}
+
+  virtual ~ExplicitPromiseProcess() {}
+
+  Future<PromiseResponse> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    request.set_proposal(proposal);
+    request.set_position(position);
+
+    network->broadcast(protocol::promise, request)
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
+  virtual void finalize()
+  {
+    // This process will be terminated when we get responses from a
+    // quorum of replicas. In that case, we no longer care about
+    // responses from other replicas, thus discarding them here.
+    discard(responses);
+  }
+
+private:
+  void broadcasted(const Future<set<Future<PromiseResponse> > >& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to broadcast explicit promise request: " + future.failure() :
+          "Not expecting discarded future");
+      terminate(self());
+      return;
+    }
+
+    responses = future.get();
+    foreach (const Future<PromiseResponse>& response, responses) {
+      response.onReady(defer(self(), &Self::received, lambda::_1));
+    }
+  }
+
+  void received(const PromiseResponse& response)
+  {
+    responsesReceived++;
+
+    if (!response.okay()) {
+      // Failed to get the promise from a replica for this position
+      // because it has been promised to a proposer with a higher
+      // proposal number. The 'proposal' field in the response
+      // specifies the proposal number. It is found to be larger than
+      // the proposal number used in this phase.
+      if (highestNackProposal.isNone() ||
+          highestNackProposal.get() < response.proposal()) {
+        highestNackProposal = response.proposal();
+      }
+    } else if (highestNackProposal.isSome()) {
+      // We still want to wait for more potential NACK responses so we
+      // can return the highest proposal number seen but we don't care
+      // about any more ACK responses.
+    } else {
+      // The position has been promised to us so the 'proposal' field
+      // should match the proposal we sent in the request.
+      CHECK_EQ(response.proposal(), request.proposal());
+
+      if (response.has_action()) {
+        CHECK_EQ(response.action().position(), position);
+        if (response.action().has_learned() && response.action().learned()) {
+          // Received a learned action. Note that there is no checking
+          // that we get the _same_ learned action in the event we get
+          // multiple responses with learned actions, we just take the
+          // "first". In fact, there is a specific instance in which
+          // learned actions will NOT be the same! In this instance,
+          // one replica may return that the action is a learned no-op
+          // because it knows the position has been truncated while
+          // another replica (that hasn't learned the truncation yet)
+          // might return the actual action at this position. Picking
+          // either action is _correct_, since eventually we know this
+          // position will be truncated. Fun!
+          promise.set(response);
+
+          // The remaining responses will be discarded in 'finalize'.
+          terminate(self());
+          return;
+        } else if (response.action().has_performed()) {
+          // An action has already been performed in this position, we
+          // need to save the action with the highest proposal number.
+          if (highestAckAction.isNone() ||
+              (highestAckAction.get().performed() <
+               response.action().performed())) {
+            highestAckAction = response.action();
+          }
+        } else {
+          // Received a response for a position that had previously
+          // been promised to some other proposer but an action had
+          // not been performed or learned. The position is now
+          // promised to us. No need to do anything here.
+        }
+      } else {
+        // Received a response without an action associated with. This
+        // is the case where this proposer is this first one who asks
+        // promise for this log position.
+        CHECK(response.has_position());
+        CHECK_EQ(response.position(), position);
+      }
+    }
+
+    if (responsesReceived >= quorum) {
+      // A quorum of replicas have replied.
+      PromiseResponse result;
+
+      if (highestNackProposal.isSome()) {
+        result.set_okay(false);
+        result.set_proposal(highestNackProposal.get());
+      } else {
+        result.set_okay(true);
+        if (highestAckAction.isSome()) {
+          result.mutable_action()->CopyFrom(highestAckAction.get());
+        }
+      }
+
+      promise.set(result);
+      terminate(self());
+    }
+  }
+
+  const size_t quorum;
+  const Shared<Network> network;
+  const uint64_t proposal;
+  const uint64_t position;
+
+  PromiseRequest request;
+  set<Future<PromiseResponse> > responses;
+  size_t responsesReceived;
+  Option<uint64_t> highestNackProposal;
+  Option<Action> highestAckAction;
+
+  process::Promise<PromiseResponse> promise;
+};
+
+
+class ImplicitPromiseProcess : public Process<ImplicitPromiseProcess>
+{
+public:
+  ImplicitPromiseProcess(
+      size_t _quorum,
+      const Shared<Network>& _network,
+      uint64_t _proposal)
+    : ProcessBase(ID::generate("log-implicit-promise")),
+      quorum(_quorum),
+      network(_network),
+      proposal(_proposal),
+      responsesReceived(0) {}
+
+  virtual ~ImplicitPromiseProcess() {}
+
+  Future<PromiseResponse> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    request.set_proposal(proposal);
+
+    network->broadcast(protocol::promise, request)
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
+  virtual void finalize()
+  {
+    // This process will be terminated when we get responses from a
+    // quorum of replicas. In that case, we no longer care about
+    // responses from other replicas, thus discarding them here.
+    discard(responses);
+  }
+
+private:
+  void broadcasted(const Future<set<Future<PromiseResponse> > >& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to broadcast implicit promise request: " + future.failure() :
+          "Not expecting discarded future");
+      terminate(self());
+      return;
+    }
+
+    responses = future.get();
+    foreach (const Future<PromiseResponse>& response, responses) {
+      response.onReady(defer(self(), &Self::received, lambda::_1));
+    }
+  }
+
+  void received(const PromiseResponse& response)
+  {
+    responsesReceived++;
+
+    if (!response.okay()) {
+      // Failed to get the promise from a replica because it has
+      // promised a proposer with a higher proposal number. The
+      // 'proposal' field in the response specifies the proposal
+      // number. It is found to be larger than the proposal number
+      // used in this phase.
+      if (highestNackProposal.isNone() ||
+          highestNackProposal.get() < response.proposal()) {
+        highestNackProposal = response.proposal();
+      }
+    } else if (highestNackProposal.isSome()) {
+      // We still want to wait for more potential NACK responses so we
+      // can return the highest proposal number seen but we don't care
+      // about any more ACK responses.
+    } else {
+      CHECK(response.has_position());
+      if (highestEndPosition.isNone() ||
+          highestEndPosition.get() < response.position()) {
+        highestEndPosition = response.position();
+      }
+    }
+
+    if (responsesReceived >= quorum) {
+      // A quorum of replicas have replied.
+      PromiseResponse result;
+
+      if (highestNackProposal.isSome()) {
+        result.set_okay(false);
+        result.set_proposal(highestNackProposal.get());
+      } else {
+        CHECK_SOME(highestEndPosition);
+
+        result.set_okay(true);
+        result.set_position(highestEndPosition.get());
+      }
+
+      promise.set(result);
+      terminate(self());
+    }
+  }
+
+  const size_t quorum;
+  const Shared<Network> network;
+  const uint64_t proposal;
+
+  PromiseRequest request;
+  set<Future<PromiseResponse> > responses;
+  size_t responsesReceived;
+  Option<uint64_t> highestNackProposal;
+  Option<uint64_t> highestEndPosition;
+
+  process::Promise<PromiseResponse> promise;
+};
+
+
+class WriteProcess : public Process<WriteProcess>
+{
+public:
+  WriteProcess(
+      size_t _quorum,
+      const Shared<Network>& _network,
+      uint64_t _proposal,
+      const Action& _action)
+    : ProcessBase(ID::generate("log-write")),
+      quorum(_quorum),
+      network(_network),
+      proposal(_proposal),
+      action(_action),
+      responsesReceived(0) {}
+
+  virtual ~WriteProcess() {}
+
+  Future<WriteResponse> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    request.set_proposal(proposal);
+    request.set_position(action.position());
+    request.set_type(action.type());
+    switch (action.type()) {
+      case Action::NOP:
+        CHECK(action.has_nop());
+        request.mutable_nop();
+        break;
+      case Action::APPEND:
+        CHECK(action.has_append());
+        request.mutable_append()->CopyFrom(action.append());
+        break;
+      case Action::TRUNCATE:
+        CHECK(action.has_truncate());
+        request.mutable_truncate()->CopyFrom(action.truncate());
+        break;
+      default:
+        LOG(FATAL) << "Unknown Action::Type " << action.type();
+    }
+
+    network->broadcast(protocol::write, request)
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
+  virtual void finalize()
+  {
+    // This process will be terminated when we get responses from a
+    // quorum of replicas. In that case, we no longer care about
+    // responses from other replicas, thus discarding them here.
+    discard(responses);
+  }
+
+private:
+  void broadcasted(const Future<set<Future<WriteResponse> > >& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Failed to broadcast the write request: " + future.failure() :
+          "Not expecting discarded future");
+      terminate(self());
+      return;
+    }
+
+    responses = future.get();
+    foreach (const Future<WriteResponse>& response, responses) {
+      response.onReady(defer(self(), &Self::received, lambda::_1));
+    }
+  }
+
+  void received(const WriteResponse& response)
+  {
+    CHECK_EQ(response.position(), request.position());
+
+    responsesReceived++;
+
+    if (!response.okay()) {
+      // A replica rejects the write request because this position has
+      // been promised to a proposer with a higher proposal number.
+      // The 'proposal' field in the response specifies the proposal
+      // number. It is found to be larger than the proposal number
+      // used in this phase.
+      if (highestNackProposal.isNone() ||
+          highestNackProposal.get() < response.proposal()) {
+        highestNackProposal = response.proposal();
+      }
+    }
+
+    if (responsesReceived >= quorum) {
+      // A quorum of replicas have replied.
+      WriteResponse result;
+
+      if (highestNackProposal.isSome()) {
+        result.set_okay(false);
+        result.set_proposal(highestNackProposal.get());
+      } else {
+        result.set_okay(true);
+      }
+
+      promise.set(result);
+      terminate(self());
+    }
+  }
+
+  const size_t quorum;
+  const Shared<Network> network;
+  const uint64_t proposal;
+  const Action action;
+
+  WriteRequest request;
+  set<Future<WriteResponse> > responses;
+  size_t responsesReceived;
+  Option<uint64_t> highestNackProposal;
+
+  process::Promise<WriteResponse> promise;
+};
+
+
+class FillProcess : public Process<FillProcess>
+{
+public:
+  FillProcess(
+      size_t _quorum,
+      const Shared<Network>& _network,
+      uint64_t _proposal,
+      uint64_t _position)
+    : ProcessBase(ID::generate("log-fill")),
+      quorum(_quorum),
+      network(_network),
+      position(_position),
+      proposal(_proposal) {}
+
+  virtual ~FillProcess() {}
+
+  Future<Action> future() { return promise.future(); }
+
+protected:
+  virtual void initialize()
+  {
+    // Stop when no one cares.
+    promise.future().onDiscarded(lambda::bind(
+        static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
+
+    runPromisePhase();
+  }
+
+  virtual void finalize()
+  {
+    promising.discard();
+    writing.discard();
+  }
+
+private:
+  void runPromisePhase()
+  {
+    promising = log::promise(quorum, network, proposal, position);
+    promising.onAny(defer(self(), &Self::checkPromisePhase));
+  }
+
+  void checkPromisePhase()
+  {
+    // The future 'promising' can only be discarded in 'finalize'
+    CHECK(!promising.isDiscarded());
+
+    if (promising.isFailed()) {
+      promise.fail("Explicit promise phase failed: " + promising.failure());
+      terminate(self());
+    } else {
+      const PromiseResponse& response = promising.get();
+      if (!response.okay()) {
+        // Retry with a higher proposal number.
+        retry(response.proposal());
+      } else if (response.has_action()) {
+        // A previously performed write has been found. Paxos
+        // restricts us to write the same value.
+        Action action = response.action();
+
+        CHECK_EQ(action.position(), position);
+        CHECK(action.has_type());
+        action.set_promised(proposal);
+        action.set_performed(proposal);
+
+        if (action.has_learned() && action.learned()) {
+          // If the promise phase returns a learned action, we simply
+          // learn the action by broadcasting a learned message. We
+          // don't check if a quorum of replicas acknowledge the
+          // learned message. Because of that, a catch-up replica
+          // needs to make sure that all positions it needs to recover
+          // have been learned before it can re-join the Paxos (i.e.,
+          // invoking log::catchup). Otherwise, we may not have a
+          // quorum of replicas remember an agreed value, leading to
+          // potential inconsistency in the log.
+          runLearnPhase(action);
+        } else {
+          runWritePhase(action);
+        }
+      } else {
+        // No previously performed write has been found. We can
+        // write any value. We choose to write a NOP.
+        Action action;
+        action.set_position(position);
+        action.set_promised(proposal);
+        action.set_performed(proposal);
+        action.set_type(Action::NOP);
+        action.mutable_nop();
+
+        runWritePhase(action);
+      }
+    }
+  }
+
+  void runWritePhase(const Action& action)
+  {
+    CHECK(!action.has_learned() || !action.learned());
+
+    writing = log::write(quorum, network, proposal, action);
+    writing.onAny(defer(self(), &Self::checkWritePhase, action));
+  }
+
+  void checkWritePhase(const Action& action)
+  {
+    // The future 'writing' can only be discarded in 'finalize'.
+    CHECK(!writing.isDiscarded());
+
+    if (writing.isFailed()) {
+      promise.fail("Write phase failed: " + writing.failure());
+      terminate(self());
+    } else {
+      const WriteResponse& response = writing.get();
+      if (!response.okay()) {
+        // Retry with a higher proposal number.
+        retry(response.proposal());
+      } else {
+        // The write has been accepted (and thus performed) by a
+        // quorum of replicas. A consensus has been reached.
+        Action learnedAction = action;
+        learnedAction.set_learned(true);
+
+        runLearnPhase(learnedAction);
+      }
+    }
+  }
+
+  void runLearnPhase(const Action& action)
+  {
+    CHECK(action.has_learned() && action.learned());
+
+    // We need to make sure that the learned message has been
+    // broadcasted before the fill process completes. Some users may
+    // rely on this invariant (e.g. checking if the local replica has
+    // learned the action).
+    log::learn(network, action)
+      .onAny(defer(self(), &Self::checkLearnPhase, action, lambda::_1));
+  }
+
+  void checkLearnPhase(const Action& action, const Future<Nothing>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          "Write phase failed: " + future.failure() :
+          "Not expecting discarded future");
+      terminate(self());
+    } else {
+      promise.set(action);
+      terminate(self());
+    }
+  }
+
+  void retry(uint64_t highestNackProposal)
+  {
+    // See comments below.
+    static const Duration T = Milliseconds(100);
+
+    // Bump the proposal number.
+    CHECK(highestNackProposal >= proposal);
+    proposal = highestNackProposal + 1;
+
+    // Randomized back-off. Generate a random delay in [T, 2T). T has
+    // to be chosen carefully. We want T >> broadcast time such that
+    // one proposer usually times out and wins before others wake up.
+    // On the other hand, we want T to be as small as possible such
+    // that we can reduce the wait time.
+    Duration d = T * (1.0 + (double) ::random() / RAND_MAX);
+    delay(d, self(), &Self::runPromisePhase);
+  }
+
+  const size_t quorum;
+  const Shared<Network> network;
+  const uint64_t position;
+
+  uint64_t proposal;
+
+  process::Promise<Action> promise;
+  Future<PromiseResponse> promising;
+  Future<WriteResponse> writing;
+};
+
+
+/////////////////////////////////////////////////
+// Public interfaces below.
+/////////////////////////////////////////////////
+
+
+Future<PromiseResponse> promise(
+    size_t quorum,
+    const Shared<Network>& network,
+    uint64_t proposal,
+    const Option<uint64_t>& position)
+{
+  if (position.isNone()) {
+    ImplicitPromiseProcess* process =
+      new ImplicitPromiseProcess(
+          quorum,
+          network,
+          proposal);
+
+    Future<PromiseResponse> future = process->future();
+    spawn(process, true);
+    return future;
+  } else {
+    ExplicitPromiseProcess* process =
+      new ExplicitPromiseProcess(
+          quorum,
+          network,
+          proposal,
+          position.get());
+
+    Future<PromiseResponse> future = process->future();
+    spawn(process, true);
+    return future;
+  }
+}
+
+
+Future<WriteResponse> write(
+    size_t quorum,
+    const Shared<Network>& network,
+    uint64_t proposal,
+    const Action& action)
+{
+  WriteProcess* process =
+    new WriteProcess(
+        quorum,
+        network,
+        proposal,
+        action);
+
+  Future<WriteResponse> future = process->future();
+  spawn(process, true);
+  return future;
+}
+
+
+Future<Nothing> learn(const Shared<Network>& network, const Action& action)
+{
+  LearnedMessage message;
+  message.mutable_action()->CopyFrom(action);
+
+  if (!action.has_learned() || !action.learned()) {
+    message.mutable_action()->set_learned(true);
+  }
+
+  return network->broadcast(message);
+}
+
+
+Future<Action> fill(
+    size_t quorum,
+    const Shared<Network>& network,
+    uint64_t proposal,
+    uint64_t position)
+{
+  FillProcess* process =
+    new FillProcess(
+        quorum,
+        network,
+        proposal,
+        position);
+
+  Future<Action> future = process->future();
+  spawn(process, true);
+  return future;
+}
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/consensus.hpp
----------------------------------------------------------------------
diff --git a/src/log/consensus.hpp b/src/log/consensus.hpp
new file mode 100644
index 0000000..ba41601
--- /dev/null
+++ b/src/log/consensus.hpp
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_CONSENSUS_HPP__
+#define __LOG_CONSENSUS_HPP__
+
+#include <stdint.h>
+
+#include <process/future.hpp>
+#include <process/shared.hpp>
+
+#include <stout/none.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+
+#include "log/network.hpp"
+
+#include "messages/log.hpp"
+
+// We use Paxos consensus protocol to agree on the value of each entry
+// in the replicated log. In our system, each replica is both an
+// acceptor and a learner. There are several types of proposers in the
+// system. Coordinator is one type of proposers we use to append new
+// log entries. The 'log::fill' function below creates an internal
+// proposer each time it is called. These internal proposers are used
+// to agree on previously written entries in the log.
+
+namespace mesos {
+namespace internal {
+namespace log {
+
+// Runs the promise phase (a.k.a., the prepare phase) in Paxos. This
+// phase has two purposes. First, the proposer asks promises from a
+// quorum of replicas not to accept writes from proposers with lower
+// proposal numbers. Second, the proposer looks for potential
+// previously agreed values. Only these values can be written in the
+// next phase. This restriction is used by Paxos to make sure that if
+// a value has been agreed on for a log position, subsequent writes to
+// this log position will always have the same value. We can run the
+// promise phase either for a specified log position ("explicit"
+// promise), or for all positions that have not yet been promised to
+// any proposer ("implicit" promise). The latter is a well known
+// optimization called Multi-Paxos. If the leader is relatively
+// stable, we can skip the promise phase for future instance of the
+// protocol with the same leader.
+//
+// We re-use PromiseResponse to specify the return value of this
+// phase. In the case of explicit promise, if a learned action has
+// been found in a response, this phase succeeds immediately with the
+// 'okay' field set to true and the 'action' field set to the learned
+// action. If no learned action has been found in a quorum of
+// replicas, we first check if some of them reply Nack (i.e., they
+// refuse to give promise). If yes, we set the 'okay' field to false
+// and set the 'proposal' field to be the highest proposal number seen
+// in these Nack responses. If none of them replies Nack, we set the
+// 'okay' field to true and set the 'action' field to be the action
+// that is performed by the proposer with the highest proposal number
+// in these responses. If no action has been found in these responses,
+// we leave the 'action' field unset.
+//
+// In the case of implicit promise, we must wait until a quorum of
+// replicas have replied. If some of them reply Nack, we set the
+// 'okay' field to false and set the 'proposal' field to be the
+// highest proposal number seen in these Nack responses. If none of
+// them replies Nack, we set the 'okay' field to true and set the
+// 'position' field to be the highest position (end position) seen in
+// these responses.
+extern process::Future<PromiseResponse> promise(
+    size_t quorum,
+    const process::Shared<Network>& network,
+    uint64_t proposal,
+    const Option<uint64_t>& position = None());
+
+
+// Runs the write phase (a.k.a., the propose phase) in Paxos. In this
+// phase, the proposer broadcasts a write to replicas. This phase
+// succeeds if a quorum of replicas accept the write. A proposer
+// cannot write if it hasn't gained enough (i.e., a quorum of)
+// promises from replicas. We re-use WriteResponse to specify the
+// return value of this phase. We must wait until a quorum of replicas
+// have replied. If some of them reply Nack, we set the 'okay' field
+// to false and set the 'proposal' field to be the highest proposal
+// number seen in these Nack responses. If none of them replies Nack,
+// we set the 'okay' field to true.
+extern process::Future<WriteResponse> write(
+    size_t quorum,
+    const process::Shared<Network>& network,
+    uint64_t proposal,
+    const Action& action);
+
+
+// Runs the learn phase (a.k.a, the commit phase) in Paxos. In fact,
+// this phase is not required, but treated as an optimization. In this
+// phase, a proposer broadcasts a learned message to replicas,
+// indicating that a consensus has already been reached for the given
+// log position. No need to wait for responses from replicas. When
+// the future is ready, the learned message has been broadcasted.
+extern process::Future<Nothing> learn(
+    const process::Shared<Network>& network,
+    const Action& action);
+
+
+// Tries to reach consensus for the given log position by running a
+// full Paxos round (i.e., promise -> write -> learn). If no value has
+// been previously agreed on for the given log position, a NOP will be
+// proposed. This function will automatically retry by bumping the
+// proposal number if the specified proposal number is found to be not
+// high enough. To ensure liveness, it will inject a random delay
+// before retrying. A learned action will be returned when the
+// operation succeeds.
+extern process::Future<Action> fill(
+    size_t quorum,
+    const process::Shared<Network>& network,
+    uint64_t proposal,
+    uint64_t position);
+
+} // namespace log {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LOG_CONSENSUS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/coordinator.cpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.cpp b/src/log/coordinator.cpp
index 6e6466f..b2ead8e 100644
--- a/src/log/coordinator.cpp
+++ b/src/log/coordinator.cpp
@@ -18,38 +18,33 @@
 
 #include <algorithm>
 
-#include <process/dispatch.hpp>
-#include <process/future.hpp>
-
-#include <stout/check.hpp>
-#include <stout/duration.hpp>
 #include <stout/error.hpp>
-#include <stout/foreach.hpp>
 #include <stout/none.hpp>
 
+#include "log/catchup.hpp"
+#include "log/consensus.hpp"
 #include "log/coordinator.hpp"
-#include "log/replica.hpp"
+
+#include "messages/log.hpp"
 
 using namespace process;
 
-using std::list;
-using std::pair;
 using std::set;
 using std::string;
 
-
 namespace mesos {
 namespace internal {
 namespace log {
 
-Coordinator::Coordinator(int _quorum,
-                         Replica* _replica,
-                         Network* _network)
-  : elected(false),
-    quorum(_quorum),
+Coordinator::Coordinator(
+    size_t _quorum,
+    const Shared<Replica>& _replica,
+    const Shared<Network>& _network)
+  : quorum(_quorum),
     replica(_replica),
     network(_network),
-    id(0),
+    elected(false),
+    proposal(0),
     index(0) {}
 
 
@@ -67,94 +62,76 @@ Result<uint64_t> Coordinator::elect(const Timeout& timeout)
   }
 
   // Get the highest known promise from our local replica.
-  Future<uint64_t> promise = replica->promised();
+  Future<uint64_t> promised = replica->promised();
 
-  if (!promise.await(timeout.remaining())) {
+  if (!promised.await(timeout.remaining())) {
+    promised.discard();
     return None();
-  } else if (promise.isFailed()) {
-    return Error(promise.failure());
+  } else if (promised.isFailed()) {
+    return Error(promised.failure());
   }
 
-  CHECK(promise.isReady()) << "Not expecting a discarded future!";
-
-  id = std::max(id, promise.get()) + 1; // Try the next highest!
-
-  PromiseRequest request;
-  request.set_id(id);
-
-  // Broadcast the request to the network.
-  set<Future<PromiseResponse> > futures =
-    broadcast(protocol::promise, request);
-
-  uint32_t okays = 0;
-
-  do {
-    Future<Future<PromiseResponse> > future = select(futures);
-    if (future.await(timeout.remaining())) {
-      CHECK(future.get().isReady());
-      const PromiseResponse& response = future.get().get();
-      if (!response.okay()) {
-        return None(); // Lost an election, but can retry.
-      } else if (response.okay()) {
-        CHECK(response.has_position());
-        index = std::max(index, response.position());
-        okays++;
-        if (okays >= quorum) {
-          break;
-        }
-      }
-      futures.erase(future.get());
-    }
-  } while (timeout.remaining() > Seconds(0));
+  CHECK(promised.isReady()) << "Not expecting a discarded future!";
+
+  proposal = std::max(proposal, promised.get()) + 1; // Try the next highest!
+
+  // Run the implicit promise phase.
+  Future<PromiseResponse> promising = log::promise(quorum, network, proposal);
+
+  if (!promising.await(timeout.remaining())) {
+    promising.discard();
+    return None();
+  } else if (promising.isFailed()) {
+    return Error(promising.failure());
+  }
 
-  // Discard the remaining futures.
-  discard(futures);
+  CHECK(promising.isReady()) << "Not expecting a discarded future!";
 
-  // Either we have a quorum or we timed out.
-  if (okays >= quorum) {
+  const PromiseResponse& response = promising.get();
+  if (!response.okay()) {
+    // Lost an election, but can retry.
+    proposal = response.proposal();
+    return None();
+  } else {
     LOG(INFO) << "Coordinator elected, attempting to fill missing positions";
-    elected = true;
 
-    // Need to "catchup" local replica (i.e., fill in any unlearned
+    CHECK(response.has_position());
+
+    index = response.position();
+
+    // Need to "catch-up" local replica (i.e., fill in any unlearned
     // and/or missing positions) so that we can do local reads.
     // Usually we could do this lazily, however, a local learned
     // position might have been truncated, so we actually need to
-    // catchup the local replica all the way to the end of the log
+    // catch-up the local replica all the way to the end of the log
     // before we can perform any up-to-date local reads.
 
-    Future<set<uint64_t> > positions = replica->missing(index);
+    Future<set<uint64_t> > positions = replica->missing(0, index);
 
     if (!positions.await(timeout.remaining())) {
-      elected = false;
+      positions.discard();
       return None();
     } else if (positions.isFailed()) {
-      elected = false;
       return Error(positions.failure());
     }
 
     CHECK(positions.isReady()) << "Not expecting a discarded future!";
 
-    foreach (uint64_t position, positions.get()) {
-      Result<Action> result = fill(position, timeout);
-      if (result.isError()) {
-        elected = false;
-        return Error(result.error());
-      } else if (result.isNone()) {
-        elected = false;
-        return None();
-      } else {
-        CHECK_SOME(result);
-        CHECK(result.get().position() == position);
-      }
+    Future<Nothing> catching =
+      log::catchup(quorum, replica, network, proposal, positions.get());
+
+    if (!catching.await(timeout.remaining())) {
+      catching.discard();
+      return None();
+    } else if (catching.isFailed()) {
+      return Error(catching.failure());
     }
 
-    index += 1;
-    return index - 1;
-  }
+    CHECK(catching.isReady()) << "Not expecting a discarded future!";
 
-  // Timed out ...
-  LOG(INFO) << "Coordinator timed out while trying to get elected";
-  return None();
+    elected = true;
+    return index++;
+  }
 }
 
 
@@ -175,8 +152,8 @@ Result<uint64_t> Coordinator::append(
 
   Action action;
   action.set_position(index);
-  action.set_promised(id);
-  action.set_performed(id);
+  action.set_promised(proposal);
+  action.set_performed(proposal);
   action.set_type(Action::APPEND);
   Action::Append* append = action.mutable_append();
   append->set_bytes(bytes);
@@ -184,7 +161,7 @@ Result<uint64_t> Coordinator::append(
   Result<uint64_t> result = write(action, timeout);
 
   if (result.isSome()) {
-    CHECK(result.get() == index);
+    CHECK_EQ(result.get(), index);
     index++;
   }
 
@@ -202,8 +179,8 @@ Result<uint64_t> Coordinator::truncate(
 
   Action action;
   action.set_position(index);
-  action.set_promised(id);
-  action.set_performed(id);
+  action.set_promised(proposal);
+  action.set_performed(proposal);
   action.set_type(Action::TRUNCATE);
   Action::Truncate* truncate = action.mutable_truncate();
   truncate->set_to(to);
@@ -211,7 +188,7 @@ Result<uint64_t> Coordinator::truncate(
   Result<uint64_t> result = write(action, timeout);
 
   if (result.isSome()) {
-    CHECK(result.get() == index);
+    CHECK_EQ(result.get(), index);
     index++;
   }
 
@@ -233,309 +210,60 @@ Result<uint64_t> Coordinator::write(
   CHECK(action.has_performed());
   CHECK(action.has_type());
 
-  // TODO(benh): Eliminate this special case hack?
-  if (quorum == 1) {
-    Result<uint64_t> result = commit(action);
-    if (result.isError()) {
-      return Error(result.error());
-    } else if (result.isNone()) {
-      return None();
-    } else {
-      CHECK_SOME(result);
-      return action.position();
-    }
-  }
-
-  WriteRequest request;
-  request.set_id(id);
-  request.set_position(action.position());
-  request.set_type(action.type());
-  switch (action.type()) {
-    case Action::NOP:
-      CHECK(action.has_nop());
-      request.mutable_nop();
-      break;
-    case Action::APPEND:
-      CHECK(action.has_append());
-      request.mutable_append()->MergeFrom(action.append());
-      break;
-    case Action::TRUNCATE:
-      CHECK(action.has_truncate());
-      request.mutable_truncate()->MergeFrom(action.truncate());
-      break;
-    default:
-      LOG(FATAL) << "Unknown Action::Type!";
-  }
-
-  // Broadcast the request to the network *excluding* the local replica.
-  set<Future<WriteResponse> > futures =
-    remotecast(protocol::write, request);
-
-  uint32_t okays = 0;
-
-  do {
-    Future<Future<WriteResponse> > future = select(futures);
-    if (future.await(timeout.remaining())) {
-      CHECK(future.get().isReady());
-      const WriteResponse& response = future.get().get();
-      CHECK(response.id() == request.id());
-      CHECK(response.position() == request.position());
-      if (!response.okay()) {
-        elected = false;
-        return Error("Coordinator demoted");
-      } else if (response.okay()) {
-        if (++okays >= (quorum - 1)) { // N.B. Using (quorum - 1) here!
-          // Got enough remote okays, discard the remaining futures
-          // and try and commit the action locally.
-          discard(futures);
-          Result<uint64_t> result = commit(action);
-          if (result.isError()) {
-            return Error(result.error());
-          } else if (result.isNone()) {
-            return None();
-          } else {
-            CHECK_SOME(result);
-            return action.position();
-          }
-        }
-      }
-      futures.erase(future.get());
-    }
-  } while (timeout.remaining() > Seconds(0));
-
-  // Timed out ... discard remaining futures.
-  LOG(INFO) << "Coordinator timed out while attempting to write "
-            << Action::Type_Name(action.type())
-            << " action at position " << action.position();
-  discard(futures);
-  return None();
-}
-
-
-Result<uint64_t> Coordinator::commit(const Action& action)
-{
-  LOG(INFO) << "Coordinator attempting to commit "
-            << Action::Type_Name(action.type())
-            << " action at position " << action.position();
-
-  CHECK(elected);
-
-  WriteRequest request;
-  request.set_id(id);
-  request.set_position(action.position());
-  request.set_learned(true); // A commit is just a learned write.
-  request.set_type(action.type());
-  switch (action.type()) {
-    case Action::NOP:
-      CHECK(action.has_nop());
-      request.mutable_nop();
-      break;
-    case Action::APPEND:
-      CHECK(action.has_append());
-      request.mutable_append()->MergeFrom(action.append());
-      break;
-    case Action::TRUNCATE:
-      CHECK(action.has_truncate());
-      request.mutable_truncate()->MergeFrom(action.truncate());
-      break;
-    default:
-      LOG(FATAL) << "Unknown Action::Type!";
-  }
-
-  //  TODO(benh): Add a non-message based way to do this write.
-  Future<WriteResponse> future = protocol::write(replica->pid(), request);
-
-  // We send a write request to the *local* replica just as the
-  // others: asynchronously via messages. However, rather than add the
-  // complications of dealing with timeouts for local operations
-  // (especially since we are trying to commit something), we make
-  // things simpler and block on the response from the local replica.
-  // Maybe we can let it timeout, but consider it a failure? This
-  // might be sound because we don't send the learned messages ... so
-  // this should be the same as if we just failed before we even do
-  // the write ... a client should just retry this write later.
+  Future<WriteResponse> writing =
+    log::write(quorum, network, proposal, action);
 
-  future.await(); // TODO(benh): Don't wait forever, see comment above.
-
-  if (future.isFailed()) {
-    return Error(future.failure());
+  if (!writing.await(timeout.remaining())) {
+    writing.discard();
+    return None();
+  } else if (writing.isFailed()) {
+    return Error(writing.failure());
   }
 
-  CHECK(future.isReady()) << "Not expecting a discarded future!";
-
-  const WriteResponse& response = future.get();
-  CHECK(response.id() == request.id());
-  CHECK(response.position() == request.position());
+  CHECK(writing.isReady()) << "Not expecting a discarded future!";
 
+  const WriteResponse& response = writing.get();
   if (!response.okay()) {
     elected = false;
+    proposal = response.proposal();
     return Error("Coordinator demoted");
-  }
-
-  // Commit successful, send a learned message to the network
-  // *excluding* the local replica and return the position.
-
-  LearnedMessage message;
-  message.mutable_action()->MergeFrom(action);
-
-  if (!action.has_learned() || !action.learned()) {
-    message.mutable_action()->set_learned(true);
-  }
-
-  LOG(INFO) << "Telling other replicas of learned action at position "
-            << action.position();
-
-  remotecast(message);
-
-  return action.position();
-}
-
-
-Result<Action> Coordinator::fill(uint64_t position, const Timeout& timeout)
-{
-  LOG(INFO) << "Coordinator attempting to fill position "
-            << position << " in the log";
-
-  CHECK(elected);
-
-  PromiseRequest request;
-  request.set_id(id);
-  request.set_position(position);
-
-  // Broadcast the request to the network.
-  set<Future<PromiseResponse> > futures =
-    broadcast(protocol::promise, request);
-
-  list<PromiseResponse> responses;
-
-  do {
-    Future<Future<PromiseResponse> > future = select(futures);
-    if (future.await(timeout.remaining())) {
-      CHECK(future.get().isReady());
-      const PromiseResponse& response = future.get().get();
-      CHECK(response.id() == request.id());
-      if (!response.okay()) {
-        elected = false;
-        return Error("Coordinator demoted");
-      } else if (response.okay()) {
-        responses.push_back(response);
-        if (responses.size() >= quorum) {
-          break;
-        }
-      }
-      futures.erase(future.get());
-    }
-  } while (timeout.remaining() > Seconds(0));
-
-  // Discard the remaining futures.
-  discard(futures);
-
-  // Either have a quorum or we timed out.
-  if (responses.size() >= quorum) {
-    // Check the responses for a learned action, otherwise, pick the
-    // action with the higest performed id or a no-op if no responses
-    // include performed actions.
-    Action action;
-    foreach (const PromiseResponse& response, responses) {
-      if (response.has_action()) {
-        CHECK(response.action().position() == position);
-        if (response.action().has_learned() && response.action().learned()) {
-          // Received a learned action, try and commit locally. Note
-          // that there is no checking that we get the _same_ learned
-          // action in the event we get multiple responses with
-          // learned actions, we just take the "first". In fact, there
-          // is a specific instance in which learned actions will NOT
-          // be the same! In this instance, one replica may return
-          // that the action is a learned no-op because it knows the
-          // position has been truncated while another replica (that
-          // hasn't learned the truncation yet) might return the
-          // actual action at this position. Picking either action is
-          // _correct_, since eventually we know this position will be
-          // truncated. Fun!
-          Result<uint64_t> result = commit(response.action());
-          if (result.isError()) {
-            return Error(result.error());
-          } else if (result.isNone()) {
-            return None();
-          } else {
-            CHECK_SOME(result);
-            return response.action();
-          }
-        } else if (response.action().has_performed() &&
-                   (!action.has_performed() ||
-                    response.action().performed() > action.performed())) {
-          action = response.action();
-        }
-      } else {
-        CHECK(response.has_position());
-        CHECK(response.position() == position);
-      }
+  } else {
+    // TODO(jieyu): Currently, each log operation (append or truncate)
+    // will write the same log content to the local disk twice: one
+    // from log::write() and one from log::learn(). In the future, we
+    // may want to use checksum to eliminate the duplicate disk write.
+    Future<Nothing> learning = log::learn(network, action);
+
+    // We need to make sure that learned message has been broadcasted,
+    // thus has been enqueued.  Otherwise, our "missing" check below
+    // will fail sometimes due to race condition.
+    if (!learning.await(timeout.remaining())) {
+      learning.discard();
+      return None();
+    } else if (learning.isFailed()) {
+      return Error(learning.failure());
     }
 
-    // Use a no-op if no known action has been performed.
-    if (!action.has_performed()) {
-      action.set_position(position);
-      action.set_promised(id);
-      action.set_performed(id);
-      action.set_type(Action::NOP);
-      action.mutable_nop();
-    } else {
-      action.set_performed(id);
-    }
+    CHECK(learning.isReady()) << "Not expecting a discarded future!";
 
-    Result<uint64_t> result = write(action, timeout);
+    // Make sure that the local replica has learned the newly written
+    // log entry. Since messages are delivered and dispatched in order
+    // locally, we should always have the new entry learned by now.
+    Future<bool> checking = replica->missing(action.position());
 
-    if (result.isError()) {
-      return Error(result.error());
-    } else if (result.isNone()) {
+    if (!checking.await(timeout.remaining())) {
+      checking.discard();
       return None();
-    } else {
-      CHECK_SOME(result);
-      return action;
+    } else if (checking.isFailed()) {
+      return Error(checking.failure());
     }
-  }
-
-  // Timed out ...
-  LOG(INFO) << "Coordinator timed out attempting to fill position "
-            << position << " in the log";
-  return None();
-}
 
+    CHECK(checking.isReady()) << "Not expecting a discarded future!";
 
-template <typename Req, typename Res>
-set<Future<Res> > Coordinator::broadcast(
-    const Protocol<Req, Res>& protocol,
-    const Req& req)
-{
-  Future<set<Future<Res> > > futures =
-    network->broadcast(protocol, req);
-  futures.await();
-  CHECK(futures.isReady());
-  return futures.get();
-}
-
+    CHECK(!checking.get());
 
-template <typename Req, typename Res>
-set<Future<Res> > Coordinator::remotecast(
-    const Protocol<Req, Res>& protocol,
-    const Req& req)
-{
-  set<UPID> filter;
-  filter.insert(replica->pid());
-  Future<set<Future<Res> > > futures =
-    network->broadcast(protocol, req, filter);
-  futures.await();
-  CHECK(futures.isReady());
-  return futures.get();
-}
-
-
-template <typename M>
-void Coordinator::remotecast(const M& m)
-{
-  set<UPID> filter;
-  filter.insert(replica->pid());
-  network->broadcast(m, filter);
+    return action.position();
+  }
 }
 
 } // namespace log {

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/coordinator.hpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.hpp b/src/log/coordinator.hpp
index 3f6fb7c..b0ff8df 100644
--- a/src/log/coordinator.hpp
+++ b/src/log/coordinator.hpp
@@ -19,10 +19,11 @@
 #ifndef __LOG_COORDINATOR_HPP__
 #define __LOG_COORDINATOR_HPP__
 
+#include <stdint.h>
+
 #include <string>
-#include <vector>
 
-#include <process/process.hpp>
+#include <process/shared.hpp>
 #include <process/timeout.hpp>
 
 #include <stout/result.hpp>
@@ -30,9 +31,6 @@
 #include "log/network.hpp"
 #include "log/replica.hpp"
 
-#include "messages/log.hpp"
-
-
 namespace mesos {
 namespace internal {
 namespace log {
@@ -40,7 +38,10 @@ namespace log {
 class Coordinator
 {
 public:
-  Coordinator(int quorum, Replica* replica, Network* group);
+  Coordinator(
+      size_t _quorum,
+      const process::Shared<Replica>& _replica,
+      const process::Shared<Network>& _network);
 
   ~Coordinator();
 
@@ -65,45 +66,16 @@ public:
   Result<uint64_t> truncate(uint64_t to, const process::Timeout& timeout);
 
 private:
-  // Helper that tries to achieve consensus of the specified action. A
-  // result of none means the write failed (e.g., due to timeout), but
-  // can be retried.
-  Result<uint64_t> write(const Action& action, const process::Timeout& timeout);
-
-  // Helper that handles commiting an action (i.e., writing to the
-  // local replica and then sending out learned messages).
-  Result<uint64_t> commit(const Action& action);
-
-  // Helper that tries to fill a position in the log.
-  Result<Action> fill(uint64_t position, const process::Timeout& timeout);
-
-  // Helper that uses the specified protocol to broadcast a request to
-  // our group and return a set of futures.
-  template <typename Req, typename Res>
-  std::set<process::Future<Res> > broadcast(
-      const Protocol<Req, Res>& protocol,
-      const Req& req);
-
-  // Helper like broadcast, but excludes our local replica.
-  template <typename Req, typename Res>
-  std::set<process::Future<Res> > remotecast(
-      const Protocol<Req, Res>& protocol,
-      const Req& req);
-
-  // Helper like remotecast but ignores any responses.
-  template <typename M>
-  void remotecast(const M& m);
-
-  bool elected; // True if this coordinator has been elected.
-
-  const uint32_t quorum; // Quorum size.
-
-  Replica* replica; // Local log replica.
-
-  Network* network; // Used to broadcast requests and messages to replicas.
+  Result<uint64_t> write(
+      const Action& action,
+      const process::Timeout& timeout);
 
-  uint64_t id; // Coordinator ID.
+  const size_t quorum;
+  const process::Shared<Replica> replica;
+  const process::Shared<Network> network;
 
+  bool elected; // True if this coordinator has been elected.
+  uint64_t proposal; // Currently used proposal number.
   uint64_t index; // Last position written in the log.
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/log.hpp
----------------------------------------------------------------------
diff --git a/src/log/log.hpp b/src/log/log.hpp
index 77edc7a..042f13b 100644
--- a/src/log/log.hpp
+++ b/src/log/log.hpp
@@ -23,7 +23,9 @@
 #include <set>
 #include <string>
 
+#include <process/owned.hpp>
 #include <process/process.hpp>
+#include <process/shared.hpp>
 #include <process/timeout.hpp>
 
 #include <stout/check.hpp>
@@ -139,7 +141,7 @@ public:
     Position ending();
 
   private:
-    Replica* replica;
+    process::Shared<Replica> replica;
   };
 
   class Writer
@@ -179,18 +181,18 @@ public:
   Log(int _quorum,
       const std::string& path,
       const std::set<process::UPID>& pids)
-    : group(NULL)
+    : group(NULL),
+      executor(NULL),
+      quorum(_quorum),
+      replica(new Replica(path))
   {
     GOOGLE_PROTOBUF_VERIFY_VERSION;
 
-    quorum = _quorum;
+    // Add our own replica to the network.
+    Network* _network = new Network(pids);
+    _network->add(replica->pid());
 
-    replica = new Replica(path);
-
-    network = new Network(pids);
-
-    // Don't forget to add our own replica!
-    network->add(replica->pid());
+    network.reset(_network);
   }
 
   // Creates a new replicated log that assumes the specified quorum
@@ -203,36 +205,34 @@ public:
       const Duration& timeout,
       const std::string& znode,
       const Option<zookeeper::Authentication>& auth = None())
+    : group(new zookeeper::Group(servers, timeout, znode, auth)),
+      executor(new process::Executor()),
+      quorum(_quorum),
+      replica(new Replica(path)),
+      network(new ZooKeeperNetwork(servers, timeout, znode, auth))
   {
     GOOGLE_PROTOBUF_VERIFY_VERSION;
 
-    quorum = _quorum;
-
-    LOG(INFO) << "Creating a new log replica";
-
-    replica = new Replica(path);
-
-    group = new zookeeper::Group(servers, timeout, znode, auth);
-    network = new ZooKeeperNetwork(group);
-
     // Need to add our replica to the ZooKeeper group!
     LOG(INFO) << "Attempting to join replica to ZooKeeper group";
 
     membership = group->join(replica->pid())
-      .onFailed(executor.defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor.defer(lambda::bind(&Log::discarded, this)));
+      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
+      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
 
     group->watch()
-      .onReady(executor.defer(lambda::bind(&Log::watch, this, lambda::_1)))
-      .onFailed(executor.defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor.defer(lambda::bind(&Log::discarded, this)));
+      .onReady(executor->defer(lambda::bind(&Log::watch, this, lambda::_1)))
+      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
+      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
   }
 
   ~Log()
   {
-    delete network;
+    network.own().await();
+    replica.own().await();
+
+    delete executor;
     delete group;
-    delete replica;
   }
 
   // Returns a position based off of the bytes recovered from
@@ -261,14 +261,15 @@ private:
   void failed(const std::string& message) const;
   void discarded() const;
 
+  // We store a Group instance in order to continually renew the
+  // replicas membership (when using ZooKeeper).
   zookeeper::Group* group;
   process::Future<zookeeper::Group::Membership> membership;
-  process::Executor executor;
+  process::Executor* executor;
 
   int quorum;
-
-  Replica* replica;
-  Network* network;
+  process::Shared<Replica> replica;
+  process::Shared<Network> network;
 };
 
 
@@ -420,14 +421,14 @@ void Log::watch(const std::set<zookeeper::Group::Membership>& memberships)
     // Our replica's membership must have expired, join back up.
     LOG(INFO) << "Renewing replica group membership";
     membership = group->join(replica->pid())
-      .onFailed(executor.defer(lambda::bind(&Log::failed, this, lambda::_1)))
-      .onDiscarded(executor.defer(lambda::bind(&Log::discarded, this)));
+      .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
+      .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
   }
 
   group->watch(memberships)
-    .onReady(executor.defer(lambda::bind(&Log::watch, this, lambda::_1)))
-    .onFailed(executor.defer(lambda::bind(&Log::failed, this, lambda::_1)))
-    .onDiscarded(executor.defer(lambda::bind(&Log::discarded, this)));
+    .onReady(executor->defer(lambda::bind(&Log::watch, this, lambda::_1)))
+    .onFailed(executor->defer(lambda::bind(&Log::failed, this, lambda::_1)))
+    .onDiscarded(executor->defer(lambda::bind(&Log::discarded, this)));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/19ad88b7/src/log/network.hpp
----------------------------------------------------------------------
diff --git a/src/log/network.hpp b/src/log/network.hpp
index d34cf78..2b674f6 100644
--- a/src/log/network.hpp
+++ b/src/log/network.hpp
@@ -33,6 +33,7 @@
 #include <stout/duration.hpp>
 #include <stout/foreach.hpp>
 #include <stout/lambda.hpp>
+#include <stout/nothing.hpp>
 
 #include "logging/logging.hpp"
 
@@ -67,13 +68,14 @@ public:
   process::Future<std::set<process::Future<Res> > > broadcast(
       const Protocol<Req, Res>& protocol,
       const Req& req,
-      const std::set<process::UPID>& filter = std::set<process::UPID>());
+      const std::set<process::UPID>& filter = std::set<process::UPID>()) const;
 
-  // Sends a message to each member of the network.
+  // Sends a message to each member of the network. The returned
+  // future is set when the message is broadcasted.
   template <typename M>
-  void broadcast(
+  process::Future<Nothing> broadcast(
       const M& m,
-      const std::set<process::UPID>& filter = std::set<process::UPID>());
+      const std::set<process::UPID>& filter = std::set<process::UPID>()) const;
 
 private:
   // Not copyable, not assignable.
@@ -87,11 +89,19 @@ private:
 class ZooKeeperNetwork : public Network
 {
 public:
-  ZooKeeperNetwork(zookeeper::Group* group);
+  ZooKeeperNetwork(
+      const std::string& servers,
+      const Duration& timeout,
+      const std::string& znode,
+      const Option<zookeeper::Authentication>& auth);
 
 private:
   typedef ZooKeeperNetwork This;
 
+  // Not copyable, not assignable.
+  ZooKeeperNetwork(const ZooKeeperNetwork&);
+  ZooKeeperNetwork& operator = (const ZooKeeperNetwork&);
+
   // Helper that sets up a watch on the group.
   void watch(const std::set<zookeeper::Group::Membership>& expected);
 
@@ -101,9 +111,13 @@ private:
   // Invoked when group members data has been collected.
   void collected(const process::Future<std::list<std::string> >& datas);
 
-  zookeeper::Group* group;
-  process::Executor executor;
+  zookeeper::Group group;
   process::Future<std::set<zookeeper::Group::Membership> > memberships;
+
+  // NOTE: The declaration order here is important. We want to delete
+  // the 'executor' before we delete the 'group' so that we don't get
+  // spurious fatal errors when the 'group' is being deleted.
+  process::Executor executor;
 };
 
 
@@ -157,7 +171,7 @@ public:
   }
 
   template <typename M>
-  void broadcast(
+  Nothing broadcast(
       const M& m,
       const std::set<process::UPID>& filter)
   {
@@ -168,6 +182,7 @@ public:
         process::post(pid, m);
       }
     }
+    return Nothing();
   }
 
 private:
@@ -223,7 +238,7 @@ template <typename Req, typename Res>
 process::Future<std::set<process::Future<Res> > > Network::broadcast(
     const Protocol<Req, Res>& protocol,
     const Req& req,
-    const std::set<process::UPID>& filter)
+    const std::set<process::UPID>& filter) const
 {
   return process::dispatch(process, &NetworkProcess::broadcast<Req, Res>,
                            protocol, req, filter);
@@ -231,20 +246,24 @@ process::Future<std::set<process::Future<Res> > > Network::broadcast(
 
 
 template <typename M>
-void Network::broadcast(
+process::Future<Nothing> Network::broadcast(
     const M& m,
-    const std::set<process::UPID>& filter)
+    const std::set<process::UPID>& filter) const
 {
   // Need to disambiguate overloaded function.
-  void (NetworkProcess::*broadcast)(const M&, const std::set<process::UPID>&) =
-    &NetworkProcess::broadcast<M>;
+  Nothing (NetworkProcess::*broadcast)(const M&, const std::set<process::UPID>&)
+    = &NetworkProcess::broadcast<M>;
 
-  process::dispatch(process, broadcast, m, filter);
+  return process::dispatch(process, broadcast, m, filter);
 }
 
 
-inline ZooKeeperNetwork::ZooKeeperNetwork(zookeeper::Group* _group)
-  : group(_group)
+inline ZooKeeperNetwork::ZooKeeperNetwork(
+    const std::string& servers,
+    const Duration& timeout,
+    const std::string& znode,
+    const Option<zookeeper::Authentication>& auth)
+  : group(servers, timeout, znode, auth)
 {
   watch(std::set<zookeeper::Group::Membership>());
 }
@@ -253,7 +272,7 @@ inline ZooKeeperNetwork::ZooKeeperNetwork(zookeeper::Group* _group)
 inline void ZooKeeperNetwork::watch(
     const std::set<zookeeper::Group::Membership>& expected)
 {
-  memberships = group->watch(expected);
+  memberships = group.watch(expected);
   memberships
     .onAny(executor.defer(lambda::bind(&This::watched, this, lambda::_1)));
 }
@@ -264,7 +283,7 @@ inline void ZooKeeperNetwork::watched(
 {
   if (memberships.isFailed()) {
     // We can't do much here, we could try creating another Group but
-    // that might just continue indifinitely, so we fail early
+    // that might just continue indefinitely, so we fail early
     // instead. Note that Group handles all retryable/recoverable
     // ZooKeeper errors internally.
     LOG(FATAL) << "Failed to watch ZooKeeper group: " << memberships.failure();
@@ -278,7 +297,7 @@ inline void ZooKeeperNetwork::watched(
   std::list<process::Future<std::string> > futures;
 
   foreach (const zookeeper::Group::Membership& membership, memberships.get()) {
-    futures.push_back(group->data(membership));
+    futures.push_back(group.data(membership));
   }
 
   process::collect(futures, process::Timeout::in(Seconds(5)))


[10/10] git commit: Changed log recover implementation to only broadcast requests when there are enough replicas in the network.

Posted by be...@apache.org.
Changed log recover implementation to only broadcast requests when
there are enough replicas in the network.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/16982


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/420e30bf
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/420e30bf
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/420e30bf

Branch: refs/heads/master
Commit: 420e30bfe5ce4c8b14bdccedff21f66475d91f18
Parents: 3e33188
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:56:46 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:56:46 2014 -0800

----------------------------------------------------------------------
 src/log/recover.cpp | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/420e30bf/src/log/recover.cpp
----------------------------------------------------------------------
diff --git a/src/log/recover.cpp b/src/log/recover.cpp
index 0ab8e95..0f827d7 100644
--- a/src/log/recover.cpp
+++ b/src/log/recover.cpp
@@ -148,6 +148,26 @@ private:
   {
     CHECK_NE(status, Metadata::VOTING);
 
+    // Wait until there are enough (i.e., quorum of) replicas in the
+    // network to avoid unnecessary retries.
+    network->watch(quorum, Network::GREATER_THAN_OR_EQUAL_TO)
+      .onAny(defer(self(), &Self::watched, lambda::_1));
+  }
+
+  void watched(const Future<size_t>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    CHECK_GE(future.get(), quorum);
+
     // Broadcast recover request to all replicas.
     network->broadcast(protocol::recover, RecoverRequest())
       .onAny(defer(self(), &Self::broadcasted, lambda::_1));


[08/10] git commit: Added a watch function to watch for network size changes.

Posted by be...@apache.org.
Added a watch function to watch for network size changes.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/16064


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/c1e3b741
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/c1e3b741
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/c1e3b741

Branch: refs/heads/master
Commit: c1e3b741d2936340b6ea7170a1737d1e5d838d07
Parents: fa5d450
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:55:56 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:55:56 2014 -0800

----------------------------------------------------------------------
 src/log/network.hpp     | 110 +++++++++++++++++++++++++++++++++++++++++++
 src/tests/log_tests.cpp |  49 +++++++++++++++++++
 2 files changed, 159 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c1e3b741/src/log/network.hpp
----------------------------------------------------------------------
diff --git a/src/log/network.hpp b/src/log/network.hpp
index 2b674f6..9c76bf8 100644
--- a/src/log/network.hpp
+++ b/src/log/network.hpp
@@ -22,6 +22,7 @@
 // TODO(benh): Eventually move and associate this code with the
 // libprocess protobuf code rather than keep it here.
 
+#include <list>
 #include <set>
 #include <string>
 
@@ -49,6 +50,16 @@ class NetworkProcess;
 class Network
 {
 public:
+  enum WatchMode
+  {
+    EQUAL_TO,
+    NOT_EQUAL_TO,
+    LESS_THAN,
+    LESS_THAN_OR_EQUAL_TO,
+    GREATER_THAN,
+    GREATER_THAN_OR_EQUAL_TO
+  };
+
   Network();
   Network(const std::set<process::UPID>& pids);
   virtual ~Network();
@@ -62,6 +73,14 @@ public:
   // Set the PIDs that are part of this network.
   void set(const std::set<process::UPID>& pids);
 
+  // Returns a future which gets set when the network size satisfies
+  // the constraint specified by 'size' and 'mode'. For example, if
+  // 'size' is 2 and 'mode' is GREATER_THAN, then the returned future
+  // will get set when the size of the network is greater than 2.
+  process::Future<size_t> watch(
+      size_t size,
+      WatchMode mode = NOT_EQUAL_TO) const;
+
   // Sends a request to each member of the network and returns a set
   // of futures that represent their responses.
   template <typename Req, typename Res>
@@ -135,12 +154,18 @@ public:
   {
     link(pid); // Try and keep a socket open (more efficient).
     pids.insert(pid);
+
+    // Update any pending watches.
+    update();
   }
 
   void remove(const process::UPID& pid)
   {
     // TODO(benh): unlink(pid);
     pids.erase(pid);
+
+    // Update any pending watches.
+    update();
   }
 
   void set(const std::set<process::UPID>& _pids)
@@ -149,6 +174,23 @@ public:
     foreach (const process::UPID& pid, _pids) {
       add(pid); // Also does a link.
     }
+
+    // Update any pending watches.
+    update();
+  }
+
+  process::Future<size_t> watch(size_t size, Network::WatchMode mode)
+  {
+    if (satisfied(size, mode)) {
+      return pids.size();
+    }
+
+    Watch* watch = new Watch(size, mode);
+    watches.push_back(watch);
+
+    // TODO(jieyu): Consider deleting 'watch' if the returned future
+    // is discarded by the user.
+    return watch->promise.future();
   }
 
   // Sends a request to each of the groups members and returns a set
@@ -185,12 +227,73 @@ public:
     return Nothing();
   }
 
+protected:
+  virtual void finalize()
+  {
+    foreach (Watch* watch, watches) {
+      watch->promise.fail("Network is being terminated");
+      delete watch;
+    }
+    watches.clear();
+  }
+
 private:
+  struct Watch
+  {
+    Watch(size_t _size, Network::WatchMode _mode)
+      : size(_size), mode(_mode) {}
+
+    size_t size;
+    Network::WatchMode mode;
+    process::Promise<size_t> promise;
+  };
+
   // Not copyable, not assignable.
   NetworkProcess(const NetworkProcess&);
   NetworkProcess& operator = (const NetworkProcess&);
 
+  // Notifies the change of the network.
+  void update()
+  {
+    const size_t size = watches.size();
+    for (size_t i = 0; i < size; i++) {
+      Watch* watch = watches.front();
+      watches.pop_front();
+
+      if (satisfied(watch->size, watch->mode)) {
+        watch->promise.set(pids.size());
+        delete watch;
+      } else {
+        watches.push_back(watch);
+      }
+    }
+  }
+
+  // Returns true if the current size of the network satisfies the
+  // constraint specified by 'size' and 'mode'.
+  bool satisfied(size_t size, Network::WatchMode mode)
+  {
+    switch (mode) {
+      case Network::EQUAL_TO:
+        return pids.size() == size;
+      case Network::NOT_EQUAL_TO:
+        return pids.size() != size;
+      case Network::LESS_THAN:
+        return pids.size() < size;
+      case Network::LESS_THAN_OR_EQUAL_TO:
+        return pids.size() <= size;
+      case Network::GREATER_THAN:
+        return pids.size() > size;
+      case Network::GREATER_THAN_OR_EQUAL_TO:
+        return pids.size() >= size;
+      default:
+        LOG(FATAL) << "Invalid watch mode";
+        break;
+    }
+  }
+
   std::set<process::UPID> pids;
+  std::list<Watch*> watches;
 };
 
 
@@ -234,6 +337,13 @@ inline void Network::set(const std::set<process::UPID>& pids)
 }
 
 
+inline process::Future<size_t> Network::watch(
+    size_t size, Network::WatchMode mode) const
+{
+  return process::dispatch(process, &NetworkProcess::watch, size, mode);
+}
+
+
 template <typename Req, typename Res>
 process::Future<std::set<process::Future<Res> > > Network::broadcast(
     const Protocol<Req, Res>& protocol,

http://git-wip-us.apache.org/repos/asf/mesos/blob/c1e3b741/src/tests/log_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/log_tests.cpp b/src/tests/log_tests.cpp
index f866dde..033e8e5 100644
--- a/src/tests/log_tests.cpp
+++ b/src/tests/log_tests.cpp
@@ -28,6 +28,7 @@
 #include <process/gtest.hpp>
 #include <process/owned.hpp>
 #include <process/pid.hpp>
+#include <process/process.hpp>
 #include <process/protobuf.hpp>
 #include <process/shared.hpp>
 
@@ -68,6 +69,54 @@ using testing::Eq;
 using testing::Return;
 
 
+TEST(NetworkTest, Watch)
+{
+  UPID pid1 = ProcessBase().self();
+  UPID pid2 = ProcessBase().self();
+
+  Network network;
+
+  // Test the default parameter.
+  Future<size_t> future = network.watch(1u);
+  AWAIT_READY(future);
+  EXPECT_EQ(0u, future.get());
+
+  future = network.watch(2u, Network::NOT_EQUAL_TO);
+  AWAIT_READY(future);
+  EXPECT_EQ(0u, future.get());
+
+  future = network.watch(0u, Network::GREATER_THAN_OR_EQUAL_TO);
+  AWAIT_READY(future);
+  EXPECT_EQ(0u, future.get());
+
+  future = network.watch(1u, Network::LESS_THAN);
+  AWAIT_READY(future);
+  EXPECT_EQ(0u, future.get());
+
+  network.add(pid1);
+
+  future = network.watch(1u, Network::EQUAL_TO);
+  AWAIT_READY(future);
+  EXPECT_EQ(1u, future.get());
+
+  future = network.watch(1u, Network::GREATER_THAN);
+  ASSERT_TRUE(future.isPending());
+
+  network.add(pid2);
+
+  AWAIT_READY(future);
+  EXPECT_EQ(2u, future.get());
+
+  future = network.watch(1u, Network::LESS_THAN_OR_EQUAL_TO);
+  ASSERT_TRUE(future.isPending());
+
+  network.remove(pid2);
+
+  AWAIT_READY(future);
+  EXPECT_EQ(1u, future.get());
+}
+
+
 class ReplicaTest : public TemporaryDirectoryTest
 {
 protected:


[09/10] git commit: Only broadcast requests when there are enough replicas in the network.

Posted by be...@apache.org.
Only broadcast requests when there are enough replicas in the network.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/16065


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/3e331889
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/3e331889
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/3e331889

Branch: refs/heads/master
Commit: 3e3318892c666195f1b4ebd6357f416717454fcf
Parents: c1e3b74
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:56:20 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:56:20 2014 -0800

----------------------------------------------------------------------
 src/log/consensus.cpp | 99 +++++++++++++++++++++++++++++++++++++---------
 1 file changed, 81 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3e331889/src/log/consensus.cpp
----------------------------------------------------------------------
diff --git a/src/log/consensus.cpp b/src/log/consensus.cpp
index 5eb90e7..b89673a 100644
--- a/src/log/consensus.cpp
+++ b/src/log/consensus.cpp
@@ -68,11 +68,11 @@ protected:
     promise.future().onDiscarded(lambda::bind(
         static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
 
-    request.set_proposal(proposal);
-    request.set_position(position);
-
-    network->broadcast(protocol::promise, request)
-      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+    // Wait until there are enough (i.e., quorum of) replicas in the
+    // network. This is because if there are less than quorum number
+    // of replicas in the network, the operation will not finish.
+    network->watch(quorum, Network::GREATER_THAN_OR_EQUAL_TO)
+      .onAny(defer(self(), &Self::watched, lambda::_1));
   }
 
   virtual void finalize()
@@ -84,6 +84,27 @@ protected:
   }
 
 private:
+  void watched(const Future<size_t>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    CHECK_GE(future.get(), quorum);
+
+    request.set_proposal(proposal);
+    request.set_position(position);
+
+    network->broadcast(protocol::promise, request)
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
   void broadcasted(const Future<set<Future<PromiseResponse> > >& future)
   {
     if (!future.isReady()) {
@@ -224,10 +245,11 @@ protected:
     promise.future().onDiscarded(lambda::bind(
         static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
 
-    request.set_proposal(proposal);
-
-    network->broadcast(protocol::promise, request)
-      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+    // Wait until there are enough (i.e., quorum of) replicas in the
+    // network. This is because if there are less than quorum number
+    // of replicas in the network, the operation will not finish.
+    network->watch(quorum, Network::GREATER_THAN_OR_EQUAL_TO)
+      .onAny(defer(self(), &Self::watched, lambda::_1));
   }
 
   virtual void finalize()
@@ -239,6 +261,26 @@ protected:
   }
 
 private:
+  void watched(const Future<size_t>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    CHECK_GE(future.get(), quorum);
+
+    request.set_proposal(proposal);
+
+    network->broadcast(protocol::promise, request)
+      .onAny(defer(self(), &Self::broadcasted, lambda::_1));
+  }
+
   void broadcasted(const Future<set<Future<PromiseResponse> > >& future)
   {
     if (!future.isReady()) {
@@ -341,6 +383,36 @@ protected:
     promise.future().onDiscarded(lambda::bind(
         static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
 
+    // Wait until there are enough (i.e., quorum of) replicas in the
+    // network. This is because if there are less than quorum number
+    // of replicas in the network, the operation will not finish.
+    network->watch(quorum, Network::GREATER_THAN_OR_EQUAL_TO)
+      .onAny(defer(self(), &Self::watched, lambda::_1));
+  }
+
+  virtual void finalize()
+  {
+    // This process will be terminated when we get responses from a
+    // quorum of replicas. In that case, we no longer care about
+    // responses from other replicas, thus discarding them here.
+    discard(responses);
+  }
+
+private:
+  void watched(const Future<size_t>& future)
+  {
+    if (!future.isReady()) {
+      promise.fail(
+          future.isFailed() ?
+          future.failure() :
+          "Not expecting discarded future");
+
+      terminate(self());
+      return;
+    }
+
+    CHECK_GE(future.get(), quorum);
+
     request.set_proposal(proposal);
     request.set_position(action.position());
     request.set_type(action.type());
@@ -365,15 +437,6 @@ protected:
       .onAny(defer(self(), &Self::broadcasted, lambda::_1));
   }
 
-  virtual void finalize()
-  {
-    // This process will be terminated when we get responses from a
-    // quorum of replicas. In that case, we no longer care about
-    // responses from other replicas, thus discarding them here.
-    discard(responses);
-  }
-
-private:
   void broadcasted(const Future<set<Future<WriteResponse> > >& future)
   {
     if (!future.isReady()) {


[03/10] git commit: Libprocessify the coordinator.

Posted by be...@apache.org.
Libprocessify the coordinator.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/14902


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/6ea7c14e
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/6ea7c14e
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/6ea7c14e

Branch: refs/heads/master
Commit: 6ea7c14e633feaa372d5565eaf6c9b57aff12b73
Parents: 19ad88b
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:53:10 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:53:10 2014 -0800

----------------------------------------------------------------------
 src/log/coordinator.cpp | 572 ++++++++++++++++++++++++++++++++-----------
 src/log/coordinator.hpp |  16 +-
 2 files changed, 439 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6ea7c14e/src/log/coordinator.cpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.cpp b/src/log/coordinator.cpp
index b2ead8e..21f2865 100644
--- a/src/log/coordinator.cpp
+++ b/src/log/coordinator.cpp
@@ -18,6 +18,12 @@
 
 #include <algorithm>
 
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/process.hpp>
+
 #include <stout/error.hpp>
 #include <stout/none.hpp>
 
@@ -36,67 +42,178 @@ namespace mesos {
 namespace internal {
 namespace log {
 
-Coordinator::Coordinator(
-    size_t _quorum,
-    const Shared<Replica>& _replica,
-    const Shared<Network>& _network)
-  : quorum(_quorum),
-    replica(_replica),
-    network(_network),
-    elected(false),
-    proposal(0),
-    index(0) {}
+class CoordinatorProcess : public Process<CoordinatorProcess>
+{
+public:
+  CoordinatorProcess(
+      size_t _quorum,
+      const Shared<Replica>& _replica,
+      const Shared<Network>& _network)
+    : ProcessBase(ID::generate("log-coordinator")),
+      quorum(_quorum),
+      replica(_replica),
+      network(_network),
+      state(INITIAL),
+      proposal(0),
+      index(0) {}
+
+  virtual ~CoordinatorProcess() {}
+
+  // Handles coordinator election. Returns the last committed log
+  // position if the operation succeeds. Returns none if the election
+  // is not successful, but can be retried.
+  Future<Option<uint64_t> > elect();
+
+  // Handles coordinator demotion. Returns the last committed log
+  // position if the operation succeeds.
+  Future<uint64_t> demote();
+
+  // Appends the specified bytes to the end of the log. Returns the
+  // position of the appended entry if the operation succeeds.
+  Future<uint64_t> append(const string& bytes);
+
+  // Removes all log entries preceding the log entry at the given
+  // position (to). Returns the position at which the truncate
+  // operation is written if the operation succeeds.
+  Future<uint64_t> truncate(uint64_t to);
+
+protected:
+  virtual void finalize()
+  {
+    electing.discard();
+    writing.discard();
+  }
 
+private:
+  /////////////////////////////////
+  // Election related functions. //
+  /////////////////////////////////
+
+  Future<uint64_t> getLastProposal();
+  Future<Nothing> updateProposal(uint64_t promised);
+  Future<PromiseResponse> runPromisePhase();
+  Future<Option<uint64_t> > checkPromisePhase(const PromiseResponse& response);
+  Future<set<uint64_t> > getMissingPositions();
+  Future<Nothing> catchupMissingPositions(const set<uint64_t>& positions);
+  Future<Option<uint64_t> > updateIndexAfterElected();
+  void electingFinished(const Option<uint64_t>& position);
+  void electingFailed();
+  void electingAborted();
+
+  /////////////////////////////////
+  // Writing related functions.  //
+  /////////////////////////////////
+
+  Future<uint64_t> write(const Action& action);
+  Future<WriteResponse> runWritePhase(const Action& action);
+  Future<Nothing> checkWritePhase(const WriteResponse& response);
+  Future<Nothing> runLearnPhase(const Action& action);
+  Future<bool> checkLearnPhase(const Action& action);
+  Future<uint64_t> updateIndexAfterWritten(bool missing);
+  void writingFinished();
+  void writingFailed();
+  void writingAborted();
+
+  const size_t quorum;
+  const Shared<Replica> replica;
+  const Shared<Network> network;
+
+  // The current state of the coordinator. A coordinator needs to be
+  // elected first to perform append and truncate operations. If one
+  // tries to do an append or a truncate while the coordinator is not
+  // elected, a failed future will be returned immediately. A
+  // coordinator does not declare itself as elected until it wins the
+  // election and has filled all existing positions. A coordinator is
+  // put in electing state after it decides to go for an election and
+  // before it is elected.
+  enum {
+    INITIAL,
+    ELECTING,
+    ELECTED,
+    WRITING,
+  } state;
+
+  // The current proposal number used by this coordinator.
+  uint64_t proposal;
+
+  // The position to which the next entry will be written.
+  uint64_t index;
+
+  Future<Option<uint64_t> > electing;
+  Future<uint64_t> writing;
+};
+
+
+/////////////////////////////////////////////////
+// Handles elect/demote in CoordinatorProcess.
+/////////////////////////////////////////////////
+
+
+Future<Option<uint64_t> > CoordinatorProcess::elect()
+{
+  if (state == ELECTING) {
+    return Future<Option<uint64_t> >::failed(
+        "Coordinator already being elected");
+  } else if (state == ELECTED) {
+    return Future<Option<uint64_t> >::failed(
+        "Coordinator already elected");
+  } else if (state == WRITING) {
+    return Future<Option<uint64_t> >::failed(
+        "Coordinator already elected, and is currently writing");
+  }
 
-Coordinator::~Coordinator() {}
+  CHECK_EQ(state, INITIAL);
 
+  state = ELECTING;
 
-Result<uint64_t> Coordinator::elect(const Timeout& timeout)
-{
-  LOG(INFO) << "Coordinator attempting to get elected within "
-            << timeout.remaining();
+  electing = getLastProposal()
+    .then(defer(self(), &Self::updateProposal, lambda::_1))
+    .then(defer(self(), &Self::runPromisePhase))
+    .then(defer(self(), &Self::checkPromisePhase, lambda::_1))
+    .onReady(defer(self(), &Self::electingFinished, lambda::_1))
+    .onFailed(defer(self(), &Self::electingFailed))
+    .onDiscarded(defer(self(), &Self::electingAborted));
 
-  if (elected) {
-    // TODO(benh): No-op instead of error?
-    return Error("Coordinator already elected");
-  }
+  return electing;
+}
 
-  // Get the highest known promise from our local replica.
-  Future<uint64_t> promised = replica->promised();
 
-  if (!promised.await(timeout.remaining())) {
-    promised.discard();
-    return None();
-  } else if (promised.isFailed()) {
-    return Error(promised.failure());
-  }
+Future<uint64_t> CoordinatorProcess::getLastProposal()
+{
+  return replica->promised();
+}
 
-  CHECK(promised.isReady()) << "Not expecting a discarded future!";
 
-  proposal = std::max(proposal, promised.get()) + 1; // Try the next highest!
+Future<Nothing> CoordinatorProcess::updateProposal(uint64_t promised)
+{
+  // It is possible that we have already tried an election and lost.
+  // We save the proposal number used in the last election in field
+  // 'proposal', and will try at least the proposal number we had
+  // before or greater in the next election.
+  proposal = std::max(proposal, promised) + 1;
+  return Nothing();
+}
 
-  // Run the implicit promise phase.
-  Future<PromiseResponse> promising = log::promise(quorum, network, proposal);
 
-  if (!promising.await(timeout.remaining())) {
-    promising.discard();
-    return None();
-  } else if (promising.isFailed()) {
-    return Error(promising.failure());
-  }
+Future<PromiseResponse> CoordinatorProcess::runPromisePhase()
+{
+  return log::promise(quorum, network, proposal);
+}
 
-  CHECK(promising.isReady()) << "Not expecting a discarded future!";
 
-  const PromiseResponse& response = promising.get();
+Future<Option<uint64_t> > CoordinatorProcess::checkPromisePhase(
+    const PromiseResponse& response)
+{
   if (!response.okay()) {
-    // Lost an election, but can retry.
+    // Lost an election, but can be retried. We save the proposal
+    // number here so that most likely we will have a high enough
+    // proposal number when we retry.
+    CHECK_LE(proposal, response.proposal());
     proposal = response.proposal();
+
     return None();
   } else {
-    LOG(INFO) << "Coordinator elected, attempting to fill missing positions";
-
     CHECK(response.has_position());
-
     index = response.position();
 
     // Need to "catch-up" local replica (i.e., fill in any unlearned
@@ -105,49 +222,90 @@ Result<uint64_t> Coordinator::elect(const Timeout& timeout)
     // position might have been truncated, so we actually need to
     // catch-up the local replica all the way to the end of the log
     // before we can perform any up-to-date local reads.
+    return getMissingPositions()
+      .then(defer(self(), &Self::catchupMissingPositions, lambda::_1))
+      .then(defer(self(), &Self::updateIndexAfterElected));
+   }
+}
 
-    Future<set<uint64_t> > positions = replica->missing(0, index);
 
-    if (!positions.await(timeout.remaining())) {
-      positions.discard();
-      return None();
-    } else if (positions.isFailed()) {
-      return Error(positions.failure());
-    }
+Future<set<uint64_t> > CoordinatorProcess::getMissingPositions()
+{
+  return replica->missing(0, index);
+}
 
-    CHECK(positions.isReady()) << "Not expecting a discarded future!";
 
-    Future<Nothing> catching =
-      log::catchup(quorum, replica, network, proposal, positions.get());
+Future<Nothing> CoordinatorProcess::catchupMissingPositions(
+    const set<uint64_t>& positions)
+{
+  LOG(INFO) << "Coordinator attemping to fill missing position";
 
-    if (!catching.await(timeout.remaining())) {
-      catching.discard();
-      return None();
-    } else if (catching.isFailed()) {
-      return Error(catching.failure());
-    }
+  return log::catchup(quorum, replica, network, proposal, positions);
+}
+
+
+Future<Option<uint64_t> > CoordinatorProcess::updateIndexAfterElected()
+{
+  return Option<uint64_t>(index++);
+}
 
-    CHECK(catching.isReady()) << "Not expecting a discarded future!";
 
-    elected = true;
-    return index++;
+void CoordinatorProcess::electingFinished(const Option<uint64_t>& position)
+{
+  CHECK_EQ(state, ELECTING);
+
+  if (position.isNone()) {
+    state = INITIAL;
+  } else {
+    state = ELECTED;
   }
 }
 
 
-Result<uint64_t> Coordinator::demote()
+void CoordinatorProcess::electingFailed()
+{
+  CHECK_EQ(state, ELECTING);
+  state = INITIAL;
+}
+
+
+void CoordinatorProcess::electingAborted()
 {
-  elected = false;
+  CHECK_EQ(state, ELECTING);
+  state = INITIAL;
+}
+
+
+Future<uint64_t> CoordinatorProcess::demote()
+{
+  if (state == INITIAL) {
+    return Future<uint64_t>::failed("Coordinator is not elected");
+  } else if (state == ELECTING) {
+    return Future<uint64_t>::failed("Coordinator is being elected");
+  } else if (state == WRITING) {
+    return Future<uint64_t>::failed("Coordinator is currently writing");
+  }
+
+  CHECK_EQ(state, ELECTED);
+
+  state = INITIAL;
   return index - 1;
 }
 
 
-Result<uint64_t> Coordinator::append(
-    const string& bytes,
-    const Timeout& timeout)
+/////////////////////////////////////////////////
+// Handles write in CoordinatorProcess.
+/////////////////////////////////////////////////
+
+
+Future<uint64_t> CoordinatorProcess::append(const string& bytes)
 {
-  if (!elected) {
-    return Error("Coordinator not elected");
+  if (state == INITIAL) {
+    return Future<uint64_t>::failed("Coordinator is not elected");
+  } else if (state == ELECTING) {
+    return Future<uint64_t>::failed("Coordinator is being elected");
+  } else if (state == WRITING) {
+    return Future<uint64_t>::failed("Coordinator is currently writing");
   }
 
   Action action;
@@ -158,23 +316,18 @@ Result<uint64_t> Coordinator::append(
   Action::Append* append = action.mutable_append();
   append->set_bytes(bytes);
 
-  Result<uint64_t> result = write(action, timeout);
-
-  if (result.isSome()) {
-    CHECK_EQ(result.get(), index);
-    index++;
-  }
-
-  return result;
+  return write(action);
 }
 
 
-Result<uint64_t> Coordinator::truncate(
-    uint64_t to,
-    const Timeout& timeout)
+Future<uint64_t> CoordinatorProcess::truncate(uint64_t to)
 {
-  if (!elected) {
-    return Error("Coordinator not elected");
+  if (state == INITIAL) {
+    return Future<uint64_t>::failed("Coordinator is not elected");
+  } else if (state == ELECTING) {
+    return Future<uint64_t>::failed("Coordinator is being elected");
+  } else if (state == WRITING) {
+    return Future<uint64_t>::failed("Coordinator is currently writing");
   }
 
   Action action;
@@ -185,84 +338,227 @@ Result<uint64_t> Coordinator::truncate(
   Action::Truncate* truncate = action.mutable_truncate();
   truncate->set_to(to);
 
-  Result<uint64_t> result = write(action, timeout);
-
-  if (result.isSome()) {
-    CHECK_EQ(result.get(), index);
-    index++;
-  }
-
-  return result;
+  return write(action);
 }
 
 
-Result<uint64_t> Coordinator::write(
-    const Action& action,
-    const Timeout& timeout)
+Future<uint64_t> CoordinatorProcess::write(const Action& action)
 {
   LOG(INFO) << "Coordinator attempting to write "
             << Action::Type_Name(action.type())
-            << " action at position " << action.position()
-            << " within " << timeout.remaining();
+            << " action at position " << action.position();
 
-  CHECK(elected);
+  CHECK_EQ(state, ELECTED);
+  CHECK(action.has_performed() && action.has_type());
 
-  CHECK(action.has_performed());
-  CHECK(action.has_type());
+  state = WRITING;
 
-  Future<WriteResponse> writing =
-    log::write(quorum, network, proposal, action);
+  writing = runWritePhase(action)
+    .then(defer(self(), &Self::checkWritePhase, lambda::_1))
+    .then(defer(self(), &Self::runLearnPhase, action))
+    .then(defer(self(), &Self::checkLearnPhase, action))
+    .then(defer(self(), &Self::updateIndexAfterWritten, lambda::_1))
+    .onReady(defer(self(), &Self::writingFinished))
+    .onFailed(defer(self(), &Self::writingFailed))
+    .onDiscarded(defer(self(), &Self::writingAborted));
 
-  if (!writing.await(timeout.remaining())) {
-    writing.discard();
-    return None();
-  } else if (writing.isFailed()) {
-    return Error(writing.failure());
-  }
+  return writing;
+}
 
-  CHECK(writing.isReady()) << "Not expecting a discarded future!";
 
-  const WriteResponse& response = writing.get();
-  if (!response.okay()) {
-    elected = false;
+Future<WriteResponse> CoordinatorProcess::runWritePhase(const Action& action)
+{
+  return log::write(quorum, network, proposal, action);
+}
+
+
+Future<Nothing> CoordinatorProcess::checkWritePhase(
+    const WriteResponse& response)
+{
+   if (!response.okay()) {
+    // Received a NACK. Save the proposal number.
+    CHECK_LE(proposal, response.proposal());
     proposal = response.proposal();
-    return Error("Coordinator demoted");
+
+    return Future<Nothing>::failed("Coordinator demoted");
   } else {
-    // TODO(jieyu): Currently, each log operation (append or truncate)
-    // will write the same log content to the local disk twice: one
-    // from log::write() and one from log::learn(). In the future, we
-    // may want to use checksum to eliminate the duplicate disk write.
-    Future<Nothing> learning = log::learn(network, action);
-
-    // We need to make sure that learned message has been broadcasted,
-    // thus has been enqueued.  Otherwise, our "missing" check below
-    // will fail sometimes due to race condition.
-    if (!learning.await(timeout.remaining())) {
-      learning.discard();
-      return None();
-    } else if (learning.isFailed()) {
-      return Error(learning.failure());
-    }
+    return Nothing();
+  }
+}
 
-    CHECK(learning.isReady()) << "Not expecting a discarded future!";
 
-    // Make sure that the local replica has learned the newly written
-    // log entry. Since messages are delivered and dispatched in order
-    // locally, we should always have the new entry learned by now.
-    Future<bool> checking = replica->missing(action.position());
+Future<Nothing> CoordinatorProcess::runLearnPhase(const Action& action)
+{
+  return log::learn(network, action);
+}
+
+
+Future<bool> CoordinatorProcess::checkLearnPhase(const Action& action)
+{
+  // Make sure that the local replica has learned the newly written
+  // log entry. Since messages are delivered and dispatched in order
+  // locally, we should always have the new entry learned by now.
+  return replica->missing(action.position());
+}
+
+
+Future<uint64_t> CoordinatorProcess::updateIndexAfterWritten(bool missing)
+{
+  CHECK(!missing) << "Not expecting local replica to be missing position "
+                  << index << " after the writing is done";
+
+  return index++;
+}
+
+
+void CoordinatorProcess::writingFinished()
+{
+  CHECK_EQ(state, WRITING);
+  state = ELECTED;
+}
+
+
+void CoordinatorProcess::writingFailed()
+{
+  CHECK_EQ(state, WRITING);
+  state = INITIAL;
+}
+
+
+void CoordinatorProcess::writingAborted()
+{
+  CHECK_EQ(state, WRITING);
+  state = ELECTED;
+}
+
+
+/////////////////////////////////////////////////
+// Coordinator implementation.
+/////////////////////////////////////////////////
+
+
+Coordinator::Coordinator(
+    size_t quorum,
+    const Shared<Replica>& replica,
+    const Shared<Network>& network)
+{
+  process = new CoordinatorProcess(quorum, replica, network);
+  spawn(process);
+}
+
+
+Coordinator::~Coordinator()
+{
+  terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+Result<uint64_t> Coordinator::elect(const Timeout& timeout)
+{
+  LOG(INFO) << "Coordinator attempting to get elected within "
+            << timeout.remaining();
+
+  Future<Option<uint64_t> > electing =
+    dispatch(process, &CoordinatorProcess::elect);
+
+  electing.await(timeout.remaining());
+
+  CHECK(!electing.isDiscarded());
+
+  if (electing.isPending()) {
+    LOG(INFO) << "Coordinator timed out while trying to get elected";
+
+    electing.discard();
+    return None();
+  } else if (electing.isFailed()) {
+    LOG(ERROR) << "Coordinator failed to get elected: "
+               << electing.failure();
+
+    return Error(electing.failure());
+  } else {
+    if (electing.get().isNone()) {
+      LOG(INFO) << "Coordinator lost an election, but can be retried";
 
-    if (!checking.await(timeout.remaining())) {
-      checking.discard();
       return None();
-    } else if (checking.isFailed()) {
-      return Error(checking.failure());
+    } else {
+      LOG(INFO) << "Coordinator elected with current position "
+                << electing.get().get();
+
+      return electing.get().get();
     }
+  }
+}
+
+
+Result<uint64_t> Coordinator::demote()
+{
+  Future<uint64_t> demoting =
+    dispatch(process, &CoordinatorProcess::demote);
+
+  demoting.await(); // TODO(jieyu): Use a timeout.
+
+  CHECK(!demoting.isDiscarded());
+
+  if (demoting.isFailed()) {
+    return Error(demoting.failure());
+  } else {
+    return demoting.get();
+  }
+}
+
+
+Result<uint64_t> Coordinator::append(
+    const string& bytes,
+    const Timeout& timeout)
+{
+  Future<uint64_t> appending =
+    dispatch(process, &CoordinatorProcess::append, bytes);
+
+  appending.await(timeout.remaining());
 
-    CHECK(checking.isReady()) << "Not expecting a discarded future!";
+  CHECK(!appending.isDiscarded());
 
-    CHECK(!checking.get());
+  if (appending.isPending()) {
+    LOG(INFO) << "Coordinator timed out while trying to append";
 
-    return action.position();
+    appending.discard();
+    return None();
+  } else if (appending.isFailed()) {
+    LOG(ERROR) << "Coordinator failed to append the log: "
+               << appending.failure();
+
+    return Error(appending.failure());
+  } else {
+    return appending.get();
+  }
+}
+
+
+Result<uint64_t> Coordinator::truncate(
+    uint64_t to,
+    const Timeout& timeout)
+{
+  Future<uint64_t> truncating =
+    dispatch(process, &CoordinatorProcess::truncate, to);
+
+  truncating.await(timeout.remaining());
+
+  CHECK(!truncating.isDiscarded());
+
+  if (truncating.isPending()) {
+    LOG(INFO) << "Coordinator timed out while trying to truncate";
+
+    truncating.discard();
+    return None();
+  } else if (truncating.isFailed()) {
+    LOG(ERROR) << "Coordinator failed to truncate the log: "
+               << truncating.failure();
+
+    return Error(truncating.failure());
+  } else {
+    return truncating.get();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6ea7c14e/src/log/coordinator.hpp
----------------------------------------------------------------------
diff --git a/src/log/coordinator.hpp b/src/log/coordinator.hpp
index b0ff8df..43cb530 100644
--- a/src/log/coordinator.hpp
+++ b/src/log/coordinator.hpp
@@ -35,6 +35,10 @@ namespace mesos {
 namespace internal {
 namespace log {
 
+// Forward declaration.
+class CoordinatorProcess;
+
+
 class Coordinator
 {
 public:
@@ -66,17 +70,7 @@ public:
   Result<uint64_t> truncate(uint64_t to, const process::Timeout& timeout);
 
 private:
-  Result<uint64_t> write(
-      const Action& action,
-      const process::Timeout& timeout);
-
-  const size_t quorum;
-  const process::Shared<Replica> replica;
-  const process::Shared<Network> network;
-
-  bool elected; // True if this coordinator has been elected.
-  uint64_t proposal; // Currently used proposal number.
-  uint64_t index; // Last position written in the log.
+  CoordinatorProcess* process;
 };
 
 } // namespace log {


[07/10] git commit: Adjusted log tests to use the new log tool.

Posted by be...@apache.org.
Adjusted log tests to use the new log tool.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/16946


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/fa5d450c
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/fa5d450c
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/fa5d450c

Branch: refs/heads/master
Commit: fa5d450c2355031296c5486b87cb3b5dcc68e470
Parents: e2fe586
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Jan 16 16:55:36 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Jan 16 16:55:36 2014 -0800

----------------------------------------------------------------------
 src/tests/log_tests.cpp | 223 ++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 219 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fa5d450c/src/tests/log_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/log_tests.cpp b/src/tests/log_tests.cpp
index aeebfb7..f866dde 100644
--- a/src/tests/log_tests.cpp
+++ b/src/tests/log_tests.cpp
@@ -43,9 +43,15 @@
 #include "log/network.hpp"
 #include "log/recover.hpp"
 #include "log/replica.hpp"
+#include "log/tool/initialize.hpp"
 
+#include "tests/environment.hpp"
 #include "tests/utils.hpp"
 
+#ifdef MESOS_HAS_JAVA
+#include "tests/zookeeper.hpp"
+#endif
+
 using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::log;
@@ -62,12 +68,19 @@ using testing::Eq;
 using testing::Return;
 
 
-class ReplicaTest : public TemporaryDirectoryTest {};
+class ReplicaTest : public TemporaryDirectoryTest
+{
+protected:
+  // For initializing the log.
+  tool::Initialize initializer;
+};
 
 
 TEST_F(ReplicaTest, Promise)
 {
   const string path = os::getcwd() + "/.log";
+  initializer.flags.path = path;
+  initializer.execute();
 
   Replica replica(path);
 
@@ -118,6 +131,8 @@ TEST_F(ReplicaTest, Promise)
 TEST_F(ReplicaTest, Append)
 {
   const string path = os::getcwd() + "/.log";
+  initializer.flags.path = path;
+  initializer.execute();
 
   Replica replica(path);
 
@@ -177,6 +192,8 @@ TEST_F(ReplicaTest, Append)
 TEST_F(ReplicaTest, Restore)
 {
   const string path = os::getcwd() + "/.log";
+  initializer.flags.path = path;
+  initializer.execute();
 
   Replica replica1(path);
 
@@ -257,13 +274,23 @@ TEST_F(ReplicaTest, Restore)
 }
 
 
-class CoordinatorTest : public TemporaryDirectoryTest {};
+class CoordinatorTest : public TemporaryDirectoryTest
+{
+protected:
+  // For initializing the log.
+  tool::Initialize initializer;
+};
 
 
 TEST_F(CoordinatorTest, Elect)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -297,7 +324,12 @@ TEST_F(CoordinatorTest, Elect)
 TEST_F(CoordinatorTest, AppendRead)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -341,7 +373,12 @@ TEST_F(CoordinatorTest, AppendRead)
 TEST_F(CoordinatorTest, AppendReadError)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -382,6 +419,8 @@ TEST_F(CoordinatorTest, AppendReadError)
 TEST_F(CoordinatorTest, ElectNoQuorum)
 {
   const string path = os::getcwd() + "/.log";
+  initializer.flags.path = path;
+  initializer.execute();
 
   Shared<Replica> replica(new Replica(path));
 
@@ -408,7 +447,12 @@ TEST_F(CoordinatorTest, ElectNoQuorum)
 TEST_F(CoordinatorTest, AppendNoQuorum)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -448,7 +492,12 @@ TEST_F(CoordinatorTest, AppendNoQuorum)
 TEST_F(CoordinatorTest, Failover)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -503,7 +552,12 @@ TEST_F(CoordinatorTest, Failover)
 TEST_F(CoordinatorTest, Demoted)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -571,8 +625,16 @@ TEST_F(CoordinatorTest, Demoted)
 TEST_F(CoordinatorTest, Fill)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -637,8 +699,16 @@ TEST_F(CoordinatorTest, Fill)
 TEST_F(CoordinatorTest, NotLearnedFill)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -707,7 +777,12 @@ TEST_F(CoordinatorTest, NotLearnedFill)
 TEST_F(CoordinatorTest, MultipleAppends)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -749,8 +824,16 @@ TEST_F(CoordinatorTest, MultipleAppends)
 TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -817,7 +900,12 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
 TEST_F(CoordinatorTest, Truncate)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -871,8 +959,16 @@ TEST_F(CoordinatorTest, Truncate)
 TEST_F(CoordinatorTest, TruncateNotLearnedFill)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -951,8 +1047,16 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
 TEST_F(CoordinatorTest, TruncateLearnedFill)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
 
   Shared<Replica> replica1(new Replica(path1));
   Shared<Replica> replica2(new Replica(path2));
@@ -1024,15 +1128,29 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
 }
 
 
-class RecoverTest : public TemporaryDirectoryTest {};
+class RecoverTest : public TemporaryDirectoryTest
+{
+protected:
+  // For initializing the log.
+  tool::Initialize initializer;
+};
 
 
 // Two logs both need recovery compete with each other.
 TEST_F(RecoverTest, RacingCatchup)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
   const string path3 = os::getcwd() + "/.log3";
+  initializer.flags.path = path3;
+  initializer.execute();
+
   const string path4 = os::getcwd() + "/.log4";
   const string path5 = os::getcwd() + "/.log5";
 
@@ -1121,13 +1239,23 @@ TEST_F(RecoverTest, RacingCatchup)
 }
 
 
-class LogTest : public TemporaryDirectoryTest {};
+class LogTest : public TemporaryDirectoryTest
+{
+protected:
+  // For initializing the log.
+  tool::Initialize initializer;
+};
 
 
 TEST_F(LogTest, WriteRead)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Replica replica1(path1);
 
@@ -1158,7 +1286,12 @@ TEST_F(LogTest, WriteRead)
 TEST_F(LogTest, Position)
 {
   const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
   const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
 
   Replica replica1(path1);
 
@@ -1178,6 +1311,88 @@ TEST_F(LogTest, Position)
 }
 
 
+#ifdef MESOS_HAS_JAVA
+// TODO(jieyu): We copy the code from TemporaryDirectoryTest here
+// because we cannot inherit from two test fixtures. In this future,
+// we need a way to compose multiple test fixtures together.
+class LogZooKeeperTest : public ZooKeeperTest
+{
+protected:
+  virtual void SetUp()
+  {
+    ZooKeeperTest::SetUp();
+
+    // Save the current working directory.
+    cwd = os::getcwd();
+
+    // Create a temporary directory for the test.
+    Try<string> directory = environment->mkdtemp();
+
+    ASSERT_SOME(directory) << "Failed to mkdtemp";
+
+    sandbox = directory.get();
+
+    LOG(INFO) << "Using temporary directory '" << sandbox.get() << "'";
+
+    // Run the test out of the temporary directory we created.
+    ASSERT_TRUE(os::chdir(sandbox.get()))
+      << "Failed to chdir into '" << sandbox.get() << "'";
+  }
+
+  virtual void TearDown()
+  {
+    // Return to previous working directory and cleanup the sandbox.
+    ASSERT_TRUE(os::chdir(cwd));
+
+    if (sandbox.isSome()) {
+      ASSERT_SOME(os::rmdir(sandbox.get()));
+    }
+  }
+
+  // For initializing the log.
+  tool::Initialize initializer;
+
+private:
+  string cwd;
+  Option<string> sandbox;
+};
+
+
+TEST_F(LogZooKeeperTest, WriteRead)
+{
+  const string path1 = os::getcwd() + "/.log1";
+  initializer.flags.path = path1;
+  initializer.execute();
+
+  const string path2 = os::getcwd() + "/.log2";
+  initializer.flags.path = path2;
+  initializer.execute();
+
+  string servers = server->connectString();
+
+  Log log1(2, path1, servers, NO_TIMEOUT, "/log/", None());
+  Log log2(2, path2, servers, NO_TIMEOUT, "/log/", None());
+
+  Log::Writer writer(&log2, Seconds(10));
+
+  Result<Log::Position> position =
+    writer.append("hello world", Timeout::in(Seconds(10)));
+
+  ASSERT_SOME(position);
+
+  Log::Reader reader(&log2);
+
+  Result<list<Log::Entry> > entries =
+    reader.read(position.get(), position.get(), Timeout::in(Seconds(10)));
+
+  ASSERT_SOME(entries);
+  ASSERT_EQ(1u, entries.get().size());
+  EXPECT_EQ(position.get(), entries.get().front().position);
+  EXPECT_EQ("hello world", entries.get().front().data);
+}
+#endif // MESOS_HAS_JAVA
+
+
 TEST_F(CoordinatorTest, RacingElect) {}
 
 TEST_F(CoordinatorTest, FillNoQuorum) {}


[04/10] Added log recovery support.

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/replica.cpp
----------------------------------------------------------------------
diff --git a/src/log/replica.cpp b/src/log/replica.cpp
index 032180a..da9310f 100644
--- a/src/log/replica.cpp
+++ b/src/log/replica.cpp
@@ -36,6 +36,8 @@
 #include <stout/stopwatch.hpp>
 #include <stout/utils.hpp>
 
+#include "common/type_utils.hpp"
+
 #include "log/replica.hpp"
 
 #include "logging/logging.hpp"
@@ -59,13 +61,14 @@ namespace protocol {
 // Some replica protocol definitions.
 Protocol<PromiseRequest, PromiseResponse> promise;
 Protocol<WriteRequest, WriteResponse> write;
+Protocol<RecoverRequest, RecoverResponse> recover;
 
 } // namespace protocol {
 
 
 struct State
 {
-  uint64_t proposal; // Last promise made.
+  Metadata metadata; // The metadata for the replica.
   uint64_t begin; // Beginning position of the log.
   uint64_t end; // Ending position of the log.
   set<uint64_t> learned; // Positions present and learned
@@ -78,8 +81,8 @@ class Storage
 {
 public:
   virtual ~Storage() {}
-  virtual Try<State> recover(const string& path) = 0;
-  virtual Try<Nothing> persist(const Promise& promise) = 0;
+  virtual Try<State> restore(const string& path) = 0;
+  virtual Try<Nothing> persist(const Metadata& metadata) = 0;
   virtual Try<Nothing> persist(const Action& action) = 0;
   virtual Try<Action> read(uint64_t position) = 0;
 };
@@ -92,8 +95,8 @@ public:
   LevelDBStorage();
   virtual ~LevelDBStorage();
 
-  virtual Try<State> recover(const string& path);
-  virtual Try<Nothing> persist(const Promise& promise);
+  virtual Try<State> restore(const string& path);
+  virtual Try<Nothing> persist(const Metadata& metadata);
   virtual Try<Nothing> persist(const Action& action);
   virtual Try<Action> read(uint64_t position);
 
@@ -137,7 +140,8 @@ private:
 
   // Returns a string representing the specified position. Note that
   // we adjust the actual position by incrementing it by 1 because we
-  // reserve 0 for storing the promise record (Record::Promise).
+  // reserve 0 for storing the promise record (Record::Promise,
+  // DEPRECATED!), or the metadata (Record::Metadata).
   static string encode(uint64_t position, bool adjust = true)
   {
     // Adjusted stringified represenation is plus 1 of actual position.
@@ -194,7 +198,7 @@ LevelDBStorage::~LevelDBStorage()
 }
 
 
-Try<State> LevelDBStorage::recover(const string& path)
+Try<State> LevelDBStorage::restore(const string& path)
 {
   leveldb::Options options;
   options.create_if_missing = true;
@@ -236,7 +240,6 @@ Try<State> LevelDBStorage::recover(const string& path)
   LOG(INFO) << "Compacted db in " << stopwatch.elapsed();
 
   State state;
-  state.proposal = 0;
   state.begin = 0;
   state.end = 0;
 
@@ -276,14 +279,18 @@ Try<State> LevelDBStorage::recover(const string& path)
     switch (record.type()) {
       case Record::METADATA: {
         CHECK(record.has_metadata());
-        state.proposal = record.metadata().promised();
+        state.metadata.CopyFrom(record.metadata());
         break;
       }
 
       // DEPRECATED!
       case Record::PROMISE: {
         CHECK(record.has_promise());
-        state.proposal = record.promise().proposal();
+        // This replica is in old format. Set its status to VOTING
+        // since there is no catch-up logic in the old code and this
+        // replica is obviously not empty.
+        state.metadata.set_status(Metadata::VOTING);
+        state.metadata.set_promised(record.promise().proposal());
         break;
       }
 
@@ -332,7 +339,7 @@ Try<State> LevelDBStorage::recover(const string& path)
 }
 
 
-Try<Nothing> LevelDBStorage::persist(const Promise& promise)
+Try<Nothing> LevelDBStorage::persist(const Metadata& metadata)
 {
   Stopwatch stopwatch;
   stopwatch.start();
@@ -341,8 +348,8 @@ Try<Nothing> LevelDBStorage::persist(const Promise& promise)
   options.sync = true;
 
   Record record;
-  record.set_type(Record::PROMISE);
-  record.mutable_promise()->MergeFrom(promise);
+  record.set_type(Record::METADATA);
+  record.mutable_metadata()->CopyFrom(metadata);
 
   string value;
 
@@ -356,7 +363,7 @@ Try<Nothing> LevelDBStorage::persist(const Promise& promise)
     return Error(status.ToString());
   }
 
-  LOG(INFO) << "Persisting promise (" << value.size()
+  LOG(INFO) << "Persisting metadata (" << value.size()
             << " bytes) to leveldb took " << stopwatch.elapsed();
 
   return Nothing();
@@ -513,9 +520,16 @@ public:
   // Returns the last written position in the log.
   uint64_t ending();
 
+  // Returns the current status of the this replica.
+  Metadata::Status status();
+
   // Returns the highest implicit promise this replica has given.
   uint64_t promised();
 
+  // Updates the status of this replica. The update will persisted on
+  // the disk. Returns true on success and false otherwise.
+  bool update(const Metadata::Status& status);
+
 private:
   // Handles a request from a proposer to promise not to accept writes
   // from any other proposer with lower proposal number.
@@ -524,22 +538,30 @@ private:
   // Handles a request from a proposer to write an action.
   void write(const WriteRequest& request);
 
+  // Handles a request from a recover process.
+  void recover(const RecoverRequest& request);
+
   // Handles a message notifying of a learned action.
   void learned(const Action& action);
 
   // Helper routines that write a record corresponding to the
   // specified argument. Returns true on success and false otherwise.
-  bool persist(const Promise& promise);
   bool persist(const Action& action);
 
-  // Helper routine to recover log (e.g., on restart).
-  void recover(const string& path);
+  // Helper routines that update metadata corresponding to the
+  // specified argument. The update will be persisted on the disk.
+  // Returns true on success and false otherwise.
+  bool update(uint64_t promised);
+
+  // Helper routine to restore log (e.g., on restart).
+  void restore(const string& path);
 
   // Underlying storage for the log.
   Storage* storage;
 
-  // Last promise made to a proposer.
-  uint64_t proposal;
+  // The cached metadata for this replica. It includes the current
+  // status of the replica and the last promise it made.
+  Metadata metadata;
 
   // Beginning position of log (after *learned* truncations).
   uint64_t begin;
@@ -557,14 +579,13 @@ private:
 
 ReplicaProcess::ReplicaProcess(const string& path)
   : ProcessBase(ID::generate("log-replica")),
-    proposal(0),
     begin(0),
     end(0)
 {
   // TODO(benh): Factor out and expose storage.
   storage = new LevelDBStorage();
 
-  recover(path);
+  restore(path);
 
   // Install protobuf handlers.
   install<PromiseRequest>(
@@ -573,6 +594,9 @@ ReplicaProcess::ReplicaProcess(const string& path)
   install<WriteRequest>(
       &ReplicaProcess::write);
 
+  install<RecoverRequest>(
+      &ReplicaProcess::recover);
+
   install<LearnedMessage>(
       &ReplicaProcess::learned,
       &LearnedMessage::action);
@@ -700,9 +724,59 @@ uint64_t ReplicaProcess::ending()
 }
 
 
+Metadata::Status ReplicaProcess::status()
+{
+  return metadata.status();
+}
+
+
 uint64_t ReplicaProcess::promised()
 {
-  return proposal;
+  return metadata.promised();
+}
+
+
+bool ReplicaProcess::update(const Metadata::Status& status)
+{
+  Metadata metadata_;
+  metadata_.set_status(status);
+  metadata_.set_promised(promised());
+
+  Try<Nothing> persisted = storage->persist(metadata_);
+
+  if (persisted.isError()) {
+    LOG(ERROR) << "Error writing to log: " << persisted.error();
+    return false;
+  }
+
+  LOG(INFO) << "Persisted replica status to " << status;
+
+  // Update the cached metadata.
+  metadata.set_status(status);
+
+  return true;
+}
+
+
+bool ReplicaProcess::update(uint64_t promised)
+{
+  Metadata metadata_;
+  metadata_.set_status(status());
+  metadata_.set_promised(promised);
+
+  Try<Nothing> persisted = storage->persist(metadata_);
+
+  if (persisted.isError()) {
+    LOG(ERROR) << "Error writing to log: " << persisted.error();
+    return false;
+  }
+
+  LOG(INFO) << "Persisted promised to " << promised;
+
+  // Update the cached metadata.
+  metadata.set_promised(promised);
+
+  return true;
 }
 
 
@@ -722,6 +796,13 @@ uint64_t ReplicaProcess::promised()
 
 void ReplicaProcess::promise(const PromiseRequest& request)
 {
+  // Ignore promise requests if this replica is not in VOTING status.
+  if (status() != Metadata::VOTING) {
+    LOG(INFO) << "Replica ignoring promise request as it is in "
+              << status() << " status";
+    return;
+  }
+
   if (request.has_position()) {
     LOG(INFO) << "Replica received explicit promise request for position "
               << request.position() << " with proposal " << request.proposal();
@@ -742,8 +823,8 @@ void ReplicaProcess::promise(const PromiseRequest& request)
     if (request.position() < begin) {
       Action action;
       action.set_position(request.position());
-      action.set_promised(proposal); // Use the last promised proposal.
-      action.set_performed(proposal); // Use the last promised proposal.
+      action.set_promised(promised()); // Use the last promised proposal.
+      action.set_performed(promised()); // Use the last promised proposal.
       action.set_learned(true);
       action.set_type(Action::NOP);
       action.mutable_nop()->MergeFrom(Action::Nop());
@@ -776,14 +857,14 @@ void ReplicaProcess::promise(const PromiseRequest& request)
       // As a result, proposer 1 can successfully write a value X to
       // log position 1 and thinks that X is agreed, while proposer 2
       // can later write a value Y and also believes that Y is agreed.
-      if (request.proposal() <= proposal) {
+      if (request.proposal() <= promised()) {
         // If a promise request is rejected because of the proposal
         // number check, we reply with the currently promised proposal
         // number so that the proposer can bump its proposal number
         // and retry if needed to ensure liveness.
         PromiseResponse response;
         response.set_okay(false);
-        response.set_proposal(proposal);
+        response.set_proposal(promised());
         reply(response);
       } else {
         Action action;
@@ -825,20 +906,16 @@ void ReplicaProcess::promise(const PromiseRequest& request)
     LOG(INFO) << "Replica received implicit promise request with proposal "
               << request.proposal();
 
-    if (request.proposal() <= proposal) { // Only make an implicit promise once!
+    if (request.proposal() <= promised()) {
+      // Only make an implicit promise once!
       LOG(INFO) << "Replica denying promise request with proposal "
                 << request.proposal();
       PromiseResponse response;
       response.set_okay(false);
-      response.set_proposal(proposal);
+      response.set_proposal(promised());
       reply(response);
     } else {
-      Promise promise;
-      promise.set_proposal(request.proposal());
-
-      if (persist(promise)) {
-        proposal = request.proposal();
-
+      if (update(request.proposal())) {
         // Return the last position written.
         PromiseResponse response;
         response.set_okay(true);
@@ -853,6 +930,13 @@ void ReplicaProcess::promise(const PromiseRequest& request)
 
 void ReplicaProcess::write(const WriteRequest& request)
 {
+  // Ignore write requests if this replica is not in VOTING status.
+  if (status() != Metadata::VOTING) {
+    LOG(INFO) << "Replica ignoring write request as it is in "
+              << status() << " status";
+    return;
+  }
+
   LOG(INFO) << "Replica received write request for position "
             << request.position();
 
@@ -862,16 +946,16 @@ void ReplicaProcess::write(const WriteRequest& request)
     LOG(ERROR) << "Error getting log record at " << request.position()
                << ": " << result.error();
   } else if (result.isNone()) {
-    if (request.proposal() < proposal) {
+    if (request.proposal() < promised()) {
       WriteResponse response;
       response.set_okay(false);
-      response.set_proposal(proposal);
+      response.set_proposal(promised());
       response.set_position(request.position());
       reply(response);
     } else {
       Action action;
       action.set_position(request.position());
-      action.set_promised(proposal);
+      action.set_promised(promised());
       action.set_performed(request.proposal());
       if (request.has_learned()) action.set_learned(request.learned());
       action.set_type(request.type());
@@ -969,32 +1053,34 @@ void ReplicaProcess::write(const WriteRequest& request)
 }
 
 
-void ReplicaProcess::learned(const Action& action)
+void ReplicaProcess::recover(const RecoverRequest& request)
 {
-  LOG(INFO) << "Replica received learned notice for position "
-            << action.position();
+  LOG(INFO) << "Replica in " << status()
+            << " status received a broadcasted recover request";
 
-  CHECK(action.learned());
+  RecoverResponse response;
+  response.set_status(status());
 
-  if (persist(action)) {
-    LOG(INFO) << "Replica learned " << Action::Type_Name(action.type())
-              << " action at position " << action.position();
+  if (status() == Metadata::VOTING) {
+    response.set_begin(begin);
+    response.set_end(end);
   }
+
+  reply(response);
 }
 
 
-bool ReplicaProcess::persist(const Promise& promise)
+void ReplicaProcess::learned(const Action& action)
 {
-  Try<Nothing> persisted = storage->persist(promise);
-
-  if (persisted.isError()) {
-    LOG(ERROR) << "Error writing to log: " << persisted.error();
-    return false;
-  }
+  LOG(INFO) << "Replica received learned notice for position "
+            << action.position();
 
-  LOG(INFO) << "Persisted promise to " << promise.proposal();
+  CHECK(action.learned());
 
-  return true;
+  if (persist(action)) {
+    LOG(INFO) << "Replica learned " << action.type()
+              << " action at position " << action.position();
+  }
 }
 
 
@@ -1052,14 +1138,14 @@ bool ReplicaProcess::persist(const Action& action)
 }
 
 
-void ReplicaProcess::recover(const string& path)
+void ReplicaProcess::restore(const string& path)
 {
-  Try<State> state = storage->recover(path);
+  Try<State> state = storage->restore(path);
 
   CHECK_SOME(state) << "Failed to recover the log";
 
   // Pull out and save some of the state.
-  proposal = state.get().proposal;
+  metadata = state.get().metadata;
   begin = state.get().begin;
   end = state.get().end;
   unlearned = state.get().unlearned;
@@ -1134,12 +1220,28 @@ Future<uint64_t> Replica::ending() const
 }
 
 
+Future<Metadata::Status> Replica::status() const
+{
+  return dispatch(process, &ReplicaProcess::status);
+}
+
+
 Future<uint64_t> Replica::promised() const
 {
   return dispatch(process, &ReplicaProcess::promised);
 }
 
 
+Future<bool> Replica::update(const Metadata::Status& status)
+{
+  // Need to disambiguate overloaded function.
+  bool (ReplicaProcess::*update)(const Metadata::Status& status) =
+    &ReplicaProcess::update;
+
+  return dispatch(process, update, status);
+}
+
+
 PID<ReplicaProcess> Replica::pid() const
 {
   return process->self();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/log/replica.hpp
----------------------------------------------------------------------
diff --git a/src/log/replica.hpp b/src/log/replica.hpp
index 4cc7031..ecb126d 100644
--- a/src/log/replica.hpp
+++ b/src/log/replica.hpp
@@ -38,6 +38,7 @@ namespace protocol {
 // Some replica protocol declarations.
 extern Protocol<PromiseRequest, PromiseResponse> promise;
 extern Protocol<WriteRequest, WriteResponse> write;
+extern Protocol<RecoverRequest, RecoverResponse> recover;
 
 } // namespace protocol {
 
@@ -50,7 +51,11 @@ class Replica
 {
 public:
   // Constructs a new replica process using specified path to a
-  // directory for storing the underlying log.
+  // directory for storing the underlying log. If a replica starts
+  // with an empty log, it will not be allowed to vote (i.e., cannot
+  // reply to any request except the recover request). The recover
+  // process will later decide if this replica can be re-allowed to
+  // vote depending on the status of other replicas.
   Replica(const std::string& path);
   ~Replica();
 
@@ -76,9 +81,15 @@ public:
   // Returns the last written position in the log.
   process::Future<uint64_t> ending() const;
 
+  // Returns the current status of this replica.
+  process::Future<Metadata::Status> status() const;
+
   // Returns the highest implicit promise this replica has given.
   process::Future<uint64_t> promised() const;
 
+  // Updates the status of this replica.
+  process::Future<bool> update(const Metadata::Status& status);
+
   // Returns the PID associated with this replica.
   process::PID<ReplicaProcess> pid() const;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/messages/log.proto
----------------------------------------------------------------------
diff --git a/src/messages/log.proto b/src/messages/log.proto
index 8fa46ca..d73b33f 100644
--- a/src/messages/log.proto
+++ b/src/messages/log.proto
@@ -79,8 +79,8 @@ message Metadata {
   enum Status {
     VOTING = 1;      // Normal voting member in Paxos group.
     RECOVERING = 2;  // In the process of catching up.
-    STARTING = 3;    // Transient state between EMPTY and RECOVERING.
-    EMPTY = 4;       // Initial state if start with an empty log.
+    STARTING = 3;    // The log has been initialized.
+    EMPTY = 4;       // The log is empty and is not initialized.
   }
 
   required Status status = 1 [default = EMPTY];
@@ -179,3 +179,17 @@ message WriteResponse {
 message LearnedMessage {
   required Action action = 1;
 }
+
+
+// Represents a recover request. A recover request is used to initiate
+// the recovery (by broadcasting it).
+message RecoverRequest {}
+
+
+// When a replica receives a RecoverRequest, it will reply with its
+// current status, and the begin and the end of its current log.
+message RecoverResponse {
+  required Metadata.Status status = 1;
+  optional uint64 begin = 2;
+  optional uint64 end = 3;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/f9b60c4c/src/tests/log_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/log_tests.cpp b/src/tests/log_tests.cpp
index fb9bbd8..aeebfb7 100644
--- a/src/tests/log_tests.cpp
+++ b/src/tests/log_tests.cpp
@@ -18,6 +18,7 @@
 
 #include <gmock/gmock.h>
 
+#include <list>
 #include <set>
 #include <string>
 
@@ -25,31 +26,32 @@
 #include <process/future.hpp>
 #include <process/gmock.hpp>
 #include <process/gtest.hpp>
+#include <process/owned.hpp>
 #include <process/pid.hpp>
 #include <process/protobuf.hpp>
-#include <process/timeout.hpp>
+#include <process/shared.hpp>
 
 #include <stout/gtest.hpp>
+#include <stout/none.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
-
-#include "common/type_utils.hpp"
+#include <stout/path.hpp>
+#include <stout/try.hpp>
 
 #include "log/coordinator.hpp"
 #include "log/log.hpp"
+#include "log/network.hpp"
+#include "log/recover.hpp"
 #include "log/replica.hpp"
 
-#include "messages/messages.hpp"
+#include "tests/utils.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::log;
+using namespace mesos::internal::tests;
 
-using process::Clock;
-using process::Future;
-using process::Timeout;
-using process::Shared;
-using process::UPID;
+using namespace process;
 
 using std::list;
 using std::set;
@@ -59,9 +61,6 @@ using testing::_;
 using testing::Eq;
 using testing::Return;
 
-#include "tests/utils.hpp"
-
-using namespace mesos::internal::tests;
 
 class ReplicaTest : public TemporaryDirectoryTest {};
 
@@ -175,7 +174,7 @@ TEST_F(ReplicaTest, Append)
 }
 
 
-TEST_F(ReplicaTest, Recover)
+TEST_F(ReplicaTest, Restore)
 {
   const string path = os::getcwd() + "/.log";
 
@@ -278,9 +277,10 @@ TEST_F(CoordinatorTest, Elect)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   {
@@ -311,18 +311,18 @@ TEST_F(CoordinatorTest, AppendRead)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result2 =
-      coord.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result2);
-    position = result2.get();
+    Future<uint64_t> appending = coord.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -355,18 +355,18 @@ TEST_F(CoordinatorTest, AppendReadError)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result2 =
-      coord.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result2);
-    position = result2.get();
+    Future<uint64_t> appending = coord.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -394,15 +394,12 @@ TEST_F(CoordinatorTest, ElectNoQuorum)
 
   Clock::pause();
 
-  // Create a timeout here so that we can advance time.
-  Timeout timeout = Timeout::in(Seconds(10));
+  Future<Option<uint64_t> > electing = coord.elect();
 
   Clock::advance(Seconds(10));
+  Clock::settle();
 
-  {
-    Result<uint64_t> result = coord.elect(timeout);
-    EXPECT_TRUE(result.isNone());
-  }
+  EXPECT_TRUE(electing.isPending());
 
   Clock::resume();
 }
@@ -425,9 +422,10 @@ TEST_F(CoordinatorTest, AppendNoQuorum)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   process::terminate(replica2->pid());
@@ -436,15 +434,12 @@ TEST_F(CoordinatorTest, AppendNoQuorum)
 
   Clock::pause();
 
-  // Create a timeout here so that we can advance time.
-  Timeout timeout = Timeout::in(Seconds(10));
+  Future<uint64_t> appending = coord.append("hello world");
 
   Clock::advance(Seconds(10));
+  Clock::settle();
 
-  {
-    Result<uint64_t> result = coord.append("hello world", timeout);
-    EXPECT_TRUE(result.isNone());
-  }
+  EXPECT_TRUE(appending.isPending());
 
   Clock::resume();
 }
@@ -467,18 +462,18 @@ TEST_F(CoordinatorTest, Failover)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result =
-      coord1.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    position = result.get();
+    Future<uint64_t> appending = coord1.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -487,9 +482,10 @@ TEST_F(CoordinatorTest, Failover)
   Coordinator coord2(2, replica2, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(position, electing.get().get());
   }
 
   {
@@ -521,18 +517,18 @@ TEST_F(CoordinatorTest, Demoted)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result =
-      coord1.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    position = result.get();
+    Future<uint64_t> appending = coord1.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -541,23 +537,22 @@ TEST_F(CoordinatorTest, Demoted)
   Coordinator coord2(2, replica2, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(position, electing.get().get());
   }
 
   {
-    Result<uint64_t> result =
-      coord1.append("hello moto", Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isError());
-    EXPECT_EQ("Coordinator demoted", result.error());
+    Future<uint64_t> appending = coord1.append("hello moto");
+    AWAIT_FAILED(appending);
+    EXPECT_EQ("Coordinator demoted", appending.failure());
   }
 
   {
-    Result<uint64_t> result =
-      coord2.append("hello hello", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    position = result.get();
+    Future<uint64_t> appending = coord2.append("hello hello");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(2u, position);
   }
 
@@ -591,18 +586,18 @@ TEST_F(CoordinatorTest, Fill)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result =
-      coord1.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    position = result.get();
+    Future<uint64_t> appending = coord1.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -617,11 +612,14 @@ TEST_F(CoordinatorTest, Fill)
   Coordinator coord2(2, replica3, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isNone());
-    result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(position, electing.get().get());
   }
 
   {
@@ -658,18 +656,18 @@ TEST_F(CoordinatorTest, NotLearnedFill)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   uint64_t position;
 
   {
-    Result<uint64_t> result =
-      coord1.append("hello world", Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    position = result.get();
+    Future<uint64_t> appending = coord1.append("hello world");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    position = appending.get();
     EXPECT_EQ(1u, position);
   }
 
@@ -684,11 +682,14 @@ TEST_F(CoordinatorTest, NotLearnedFill)
   Coordinator coord2(2, replica3, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isNone());
-    result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(position, electing.get().get());
   }
 
   {
@@ -720,16 +721,16 @@ TEST_F(CoordinatorTest, MultipleAppends)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result =
-      coord.append(stringify(position), Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<uint64_t> appending = coord.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
   }
 
   {
@@ -767,16 +768,16 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result =
-      coord1.append(stringify(position), Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<uint64_t> appending = coord1.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
   }
 
   Shared<Replica> replica3(new Replica(path3));
@@ -790,11 +791,14 @@ TEST_F(CoordinatorTest, MultipleAppendsNotLearnedFill)
   Coordinator coord2(2, replica3, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isNone());
-    result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(10u, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(10u, electing.get().get());
   }
 
   {
@@ -827,22 +831,22 @@ TEST_F(CoordinatorTest, Truncate)
   Coordinator coord(2, replica1, network);
 
   {
-    Result<uint64_t> result = coord.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result =
-      coord.append(stringify(position), Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<uint64_t> appending = coord.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
   }
 
   {
-    Result<uint64_t> result = coord.truncate(7, Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(11u, result.get());
+    Future<uint64_t> truncating = coord.truncate(7);
+    AWAIT_READY_FOR(truncating, Seconds(10));
+    EXPECT_EQ(11u, truncating.get());
   }
 
   {
@@ -886,22 +890,22 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result =
-      coord1.append(stringify(position), Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<uint64_t> appending = coord1.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
   }
 
   {
-    Result<uint64_t> result = coord1.truncate(7, Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(11u, result.get());
+    Future<uint64_t> truncating = coord1.truncate(7);
+    AWAIT_READY_FOR(truncating, Seconds(10));
+    EXPECT_EQ(11u, truncating.get());
   }
 
   Shared<Replica> replica3(new Replica(path3));
@@ -915,11 +919,14 @@ TEST_F(CoordinatorTest, TruncateNotLearnedFill)
   Coordinator coord2(2, replica3, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isNone());
-    result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(11u, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(11u, electing.get().get());
   }
 
   {
@@ -959,22 +966,22 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
   Coordinator coord1(2, replica1, network1);
 
   {
-    Result<uint64_t> result = coord1.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(0u, result.get());
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result =
-      coord1.append(stringify(position), Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(position, result.get());
+    Future<uint64_t> appending = coord1.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
   }
 
   {
-    Result<uint64_t> result = coord1.truncate(7, Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(11u, result.get());
+    Future<uint64_t> truncating = coord1.truncate(7);
+    AWAIT_READY_FOR(truncating, Seconds(10));
+    EXPECT_EQ(11u, truncating.get());
   }
 
   Shared<Replica> replica3(new Replica(path3));
@@ -988,11 +995,14 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
   Coordinator coord2(2, replica3, network2);
 
   {
-    Result<uint64_t> result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_TRUE(result.isNone());
-    result = coord2.elect(Timeout::in(Seconds(10)));
-    ASSERT_SOME(result);
-    EXPECT_EQ(11u, result.get());
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(11u, electing.get().get());
   }
 
   {
@@ -1014,6 +1024,103 @@ TEST_F(CoordinatorTest, TruncateLearnedFill)
 }
 
 
+class RecoverTest : public TemporaryDirectoryTest {};
+
+
+// Two logs both need recovery compete with each other.
+TEST_F(RecoverTest, RacingCatchup)
+{
+  const string path1 = os::getcwd() + "/.log1";
+  const string path2 = os::getcwd() + "/.log2";
+  const string path3 = os::getcwd() + "/.log3";
+  const string path4 = os::getcwd() + "/.log4";
+  const string path5 = os::getcwd() + "/.log5";
+
+  Shared<Replica> replica1(new Replica(path1));
+  Shared<Replica> replica2(new Replica(path2));
+  Shared<Replica> replica3(new Replica(path3));
+
+  set<UPID> pids;
+  pids.insert(replica1->pid());
+  pids.insert(replica2->pid());
+  pids.insert(replica3->pid());
+
+  Shared<Network> network1(new Network(pids));
+
+  Coordinator coord1(3, replica1, network1);
+
+  {
+    Future<Option<uint64_t> > electing = coord1.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(0u, electing.get().get());
+  }
+
+  for (uint64_t position = 1; position <= 10; position++) {
+    Future<uint64_t> appending = coord1.append(stringify(position));
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(position, appending.get());
+  }
+
+  // Two replicas both want to recover.
+  Owned<Replica> replica4(new Replica(path4));
+  Owned<Replica> replica5(new Replica(path5));
+
+  pids.insert(replica4->pid());
+  pids.insert(replica5->pid());
+
+  Shared<Network> network2(new Network(pids));
+
+  Future<Owned<Replica> > recovering4 = recover(3, replica4, network2);
+  Future<Owned<Replica> > recovering5 = recover(3, replica5, network2);
+
+  // Wait until recovery is done.
+  AWAIT_READY(recovering4);
+  AWAIT_READY(recovering5);
+
+  Shared<Replica> shared4 = recovering4.get().share();
+  Coordinator coord2(2, shared4, network2);
+
+  {
+    Future<Option<uint64_t> > electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_NONE(electing.get());
+
+    electing = coord2.elect();
+    AWAIT_READY_FOR(electing, Seconds(10));
+    ASSERT_SOME(electing.get());
+    EXPECT_EQ(10u, electing.get().get());
+  }
+
+  {
+    Future<list<Action> > actions = shared4->read(1, 10);
+    AWAIT_READY(actions);
+    EXPECT_EQ(10u, actions.get().size());
+    foreach (const Action& action, actions.get()) {
+      ASSERT_TRUE(action.has_type());
+      ASSERT_EQ(Action::APPEND, action.type());
+      EXPECT_EQ(stringify(action.position()), action.append().bytes());
+    }
+  }
+
+  {
+    Future<uint64_t> appending = coord2.append("hello hello");
+    AWAIT_READY_FOR(appending, Seconds(10));
+    EXPECT_EQ(11u, appending.get());
+  }
+
+  {
+    Future<list<Action> > actions = shared4->read(11u, 11u);
+    AWAIT_READY(actions);
+    ASSERT_EQ(1u, actions.get().size());
+    EXPECT_EQ(11u, actions.get().front().position());
+    ASSERT_TRUE(actions.get().front().has_type());
+    ASSERT_EQ(Action::APPEND, actions.get().front().type());
+    EXPECT_EQ("hello hello", actions.get().front().append().bytes());
+  }
+}
+
+
 class LogTest : public TemporaryDirectoryTest {};