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 2012/09/10 00:02:07 UTC

svn commit: r1382588 - in /incubator/mesos/trunk: src/detector/ src/java/jni/ src/linux/ src/log/ src/logging/ src/master/ src/slave/ src/state/ src/tests/ src/zookeeper/ third_party/libprocess/ third_party/libprocess/include/process/ third_party/libpr...

Author: benh
Date: Sun Sep  9 22:02:06 2012
New Revision: 1382588

URL: http://svn.apache.org/viewvc?rev=1382588&view=rev
Log:
Replaced the "amount of time" abstractions (e.g., seconds, hours, etc)
with a Duration type and Seconds, Hours, etc subtypes (contributed by
Vinod Kone, https://reviews.apache.org/r/6755).

Added:
    incubator/mesos/trunk/third_party/libprocess/include/stout/duration.hpp
Removed:
    incubator/mesos/trunk/third_party/libprocess/include/stout/time.hpp
Modified:
    incubator/mesos/trunk/src/detector/detector.cpp
    incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
    incubator/mesos/trunk/src/java/jni/org_apache_mesos_state_ZooKeeperState.cpp
    incubator/mesos/trunk/src/linux/cgroups.hpp
    incubator/mesos/trunk/src/log/coordinator.hpp
    incubator/mesos/trunk/src/log/log.hpp
    incubator/mesos/trunk/src/log/network.hpp
    incubator/mesos/trunk/src/logging/logging.cpp
    incubator/mesos/trunk/src/master/frameworks_manager.cpp
    incubator/mesos/trunk/src/master/frameworks_manager.hpp
    incubator/mesos/trunk/src/master/slaves_manager.cpp
    incubator/mesos/trunk/src/slave/gc.cpp
    incubator/mesos/trunk/src/slave/gc.hpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/state/zookeeper.cpp
    incubator/mesos/trunk/src/state/zookeeper.hpp
    incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp
    incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp
    incubator/mesos/trunk/src/tests/gc_tests.cpp
    incubator/mesos/trunk/src/tests/log_tests.cpp
    incubator/mesos/trunk/src/tests/master_tests.cpp
    incubator/mesos/trunk/src/tests/stout_tests.cpp
    incubator/mesos/trunk/src/zookeeper/group.cpp
    incubator/mesos/trunk/src/zookeeper/group.hpp
    incubator/mesos/trunk/src/zookeeper/zookeeper.cpp
    incubator/mesos/trunk/src/zookeeper/zookeeper.hpp
    incubator/mesos/trunk/third_party/libprocess/Makefile.am
    incubator/mesos/trunk/third_party/libprocess/include/process/statistics.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/timer.hpp
    incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp
    incubator/mesos/trunk/third_party/libprocess/src/statistics_tests.cpp

Modified: incubator/mesos/trunk/src/detector/detector.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/detector/detector.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/detector/detector.cpp (original)
+++ incubator/mesos/trunk/src/detector/detector.cpp Sun Sep  9 22:02:06 2012
@@ -273,7 +273,7 @@ ZooKeeperMasterDetector::ZooKeeperMaster
     : ZOO_OPEN_ACL_UNSAFE;
 
   // Start up the ZooKeeper connection!
-  zk = new ZooKeeper(url.servers, milliseconds(10000), this);
+  zk = new ZooKeeper(url.servers, Milliseconds(10000), this);
 }
 
 ZooKeeperMasterDetector::~ZooKeeperMasterDetector()
@@ -408,7 +408,7 @@ void ZooKeeperMasterDetector::expired()
   CHECK(zk != NULL);
   delete zk;
 
-  zk = new ZooKeeper(url.servers, milliseconds(10000), this);
+  zk = new ZooKeeper(url.servers, Milliseconds(10000), this);
 }
 
 

Modified: incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp (original)
+++ incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp Sun Sep  9 22:02:06 2012
@@ -145,7 +145,7 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   Result<std::list<Log::Entry> > entries = reader->read(from, to, timeout);
 
@@ -295,7 +295,7 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   Result<Log::Position> position = writer->append(data, timeout);
 
@@ -348,7 +348,7 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   Result<Log::Position> position = writer->truncate(to, timeout);
 
@@ -402,7 +402,7 @@ JNIEXPORT void JNICALL Java_org_apache_m
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   int retries = jretries;
 
@@ -515,7 +515,7 @@ JNIEXPORT void JNICALL Java_org_apache_m
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   std::string znode = construct<std::string>(env, jznode);
 
@@ -559,7 +559,7 @@ JNIEXPORT void JNICALL Java_org_apache_m
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   std::string znode = construct<std::string>(env, jznode);
 

Modified: incubator/mesos/trunk/src/java/jni/org_apache_mesos_state_ZooKeeperState.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/jni/org_apache_mesos_state_ZooKeeperState.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/jni/org_apache_mesos_state_ZooKeeperState.cpp (original)
+++ incubator/mesos/trunk/src/java/jni/org_apache_mesos_state_ZooKeeperState.cpp Sun Sep  9 22:02:06 2012
@@ -5,6 +5,8 @@
 
 #include <process/future.hpp>
 
+#include <stout/duration.hpp>
+
 #include "state/state.hpp"
 #include "state/zookeeper.hpp"
 
@@ -42,7 +44,7 @@ JNIEXPORT void JNICALL Java_org_apache_m
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   string znode = construct<string>(env, jznode);
 
@@ -80,7 +82,7 @@ JNIEXPORT void JNICALL Java_org_apache_m
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
   string znode = construct<string>(env, jznode);
 
@@ -257,9 +259,9 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
-  if (future->await(timeout.value)) {
+  if (future->await(timeout.secs())) {
     if (future->isFailed()) {
       clazz = env->FindClass("java/util/concurrent/ExecutionException");
       env->ThrowNew(clazz, future->failure().c_str());
@@ -446,9 +448,9 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
-  if (future->await(timeout.value)) {
+  if (future->await(timeout.secs())) {
     if (future->isFailed()) {
       clazz = env->FindClass("java/util/concurrent/ExecutionException");
       env->ThrowNew(clazz, future->failure().c_str());
@@ -630,9 +632,9 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
 
-  seconds timeout(jseconds);
+  Seconds timeout(jseconds);
 
-  if (future->await(timeout.value)) {
+  if (future->await(timeout.secs())) {
     if (future->isFailed()) {
       clazz = env->FindClass("java/util/concurrent/ExecutionException");
       env->ThrowNew(clazz, future->failure().c_str());

Modified: incubator/mesos/trunk/src/linux/cgroups.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/cgroups.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/cgroups.hpp (original)
+++ incubator/mesos/trunk/src/linux/cgroups.hpp Sun Sep  9 22:02:06 2012
@@ -27,8 +27,8 @@
 
 #include <process/future.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 
 namespace cgroups {
@@ -267,7 +267,7 @@ process::Future<uint64_t> listenEvent(co
 //          Error if some unexpected happens.
 process::Future<bool> freezeCgroup(const std::string& hierarchy,
                                    const std::string& cgroup,
-                                   const seconds& interval = seconds(0.1));
+                                   const Duration& interval = Seconds(0.1));
 
 
 // Thaw the given cgroup. This is a revert operation of freezeCgroup. It will
@@ -282,7 +282,7 @@ process::Future<bool> freezeCgroup(const
 //          Error if some unexpected happens.
 process::Future<bool> thawCgroup(const std::string& hierarchy,
                                  const std::string& cgroup,
-                                 const seconds& interval = seconds(0.1));
+                                 const Duration& interval = Seconds(0.1));
 
 
 // Atomically kill all tasks in a given cgroup. This function will return a
@@ -301,7 +301,7 @@ process::Future<bool> thawCgroup(const s
 //          Error if some unexpected happens.
 process::Future<bool> killTasks(const std::string& hierarchy,
                                 const std::string& cgroup,
-                                const seconds& interval = seconds(0.1));
+                                const Duration& interval = Seconds(0.1));
 
 
 // Destroy a cgroup under a given hierarchy. This function is different from
@@ -319,7 +319,7 @@ process::Future<bool> killTasks(const st
 //          Error if some unexpected happens.
 process::Future<bool> destroyCgroup(const std::string& hierarchy,
                                     const std::string& cgroup = "/",
-                                    const seconds& interval = seconds(0.1));
+                                    const Duration& interval = Seconds(0.1));
 
 } // namespace cgroups {
 

Modified: incubator/mesos/trunk/src/log/coordinator.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/coordinator.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/coordinator.hpp (original)
+++ incubator/mesos/trunk/src/log/coordinator.hpp Sun Sep  9 22:02:06 2012
@@ -26,7 +26,6 @@
 #include <process/timeout.hpp>
 
 #include <stout/result.hpp>
-#include <stout/time.hpp>
 
 #include "log/network.hpp"
 #include "log/replica.hpp"

Modified: incubator/mesos/trunk/src/log/log.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/log.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/log.hpp (original)
+++ incubator/mesos/trunk/src/log/log.hpp Sun Sep  9 22:02:06 2012
@@ -26,9 +26,9 @@
 #include <process/process.hpp>
 #include <process/timeout.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/foreach.hpp>
 #include <stout/result.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 
 #include "log/coordinator.hpp"
@@ -123,7 +123,7 @@ public:
     // those positions are invalid, in which case returns an error.
     Result<std::list<Entry> > read(const Position& from,
                                    const Position& to,
-                                   const seconds& timeout);
+                                   const Duration& 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
@@ -147,20 +147,20 @@ public:
     // one writer (local and remote) is valid at a time. A writer
     // becomes invalid if any operation returns an error, and a new
     // writer must be created in order perform subsequent operations.
-    Writer(Log* log, const seconds& timeout, int retries = 3);
+    Writer(Log* log, const Duration& timeout, int retries = 3);
     ~Writer();
 
     // Attempts to append the specified data to the log. A none result
     // means the operation timed out, otherwise the new ending
     // position of the log is returned or an error. Upon error a new
     // Writer must be created.
-    Result<Position> append(const std::string& data, const seconds& timeout);
+    Result<Position> append(const std::string& data, const Duration& timeout);
 
     // Attempts to truncate the log up to but not including the
     // specificed position. A none result means the operation timed
     // out, otherwise the new ending position of the log is returned
     // or an error. Upon error a new Writer must be created.
-    Result<Position> truncate(const Position& to, const seconds& timeout);
+    Result<Position> truncate(const Position& to, const Duration& timeout);
 
   private:
     Option<std::string> error;
@@ -194,7 +194,7 @@ public:
   Log(int _quorum,
       const std::string& path,
       const std::string& servers,
-      const seconds& timeout,
+      const Seconds& timeout,
       const std::string& znode,
       const Option<zookeeper::Authentication>& auth
         = Option<zookeeper::Authentication>::none())
@@ -277,12 +277,12 @@ Log::Reader::~Reader() {}
 Result<std::list<Log::Entry> > Log::Reader::read(
     const Log::Position& from,
     const Log::Position& to,
-    const seconds& timeout)
+    const Duration& timeout)
 {
   process::Future<std::list<Action> > actions =
     replica->read(from.value, to.value);
 
-  if (!actions.await(timeout.value)) {
+  if (!actions.await(timeout.secs())) {
     return Result<std::list<Log::Entry> >::none();
   } else if (actions.isFailed()) {
     return Result<std::list<Log::Entry> >::error(actions.failure());
@@ -337,12 +337,12 @@ Log::Position Log::Reader::ending()
 }
 
 
-Log::Writer::Writer(Log* log, const seconds& timeout, int retries)
+Log::Writer::Writer(Log* log, const Duration& timeout, int retries)
   : error(Option<std::string>::none()),
     coordinator(log->quorum, log->replica, log->network)
 {
   do {
-    Result<uint64_t> result = coordinator.elect(Timeout(timeout.value));
+    Result<uint64_t> result = coordinator.elect(Timeout(timeout.secs()));
     if (result.isNone()) {
       retries--;
     } else if (result.isSome()) {
@@ -363,7 +363,7 @@ Log::Writer::~Writer()
 
 Result<Log::Position> Log::Writer::append(
     const std::string& data,
-    const seconds& timeout)
+    const Duration& timeout)
 {
   if (error.isSome()) {
     return Result<Log::Position>::error(error.get());
@@ -371,7 +371,7 @@ Result<Log::Position> Log::Writer::appen
 
   LOG(INFO) << "Attempting to append " << data.size() << " bytes to the log";
 
-  Result<uint64_t> result = coordinator.append(data, Timeout(timeout.value));
+  Result<uint64_t> result = coordinator.append(data, Timeout(timeout.secs()));
 
   if (result.isError()) {
     error = result.error();
@@ -388,7 +388,7 @@ Result<Log::Position> Log::Writer::appen
 
 Result<Log::Position> Log::Writer::truncate(
     const Log::Position& to,
-    const seconds& timeout)
+    const Duration& timeout)
 {
   if (error.isSome()) {
     return Result<Log::Position>::error(error.get());
@@ -397,7 +397,7 @@ Result<Log::Position> Log::Writer::trunc
   LOG(INFO) << "Attempting to truncate the log to " << to.value;
 
   Result<uint64_t> result =
-    coordinator.truncate(to.value, Timeout(timeout.value));
+    coordinator.truncate(to.value, Timeout(timeout.secs()));
 
   if (result.isError()) {
     error = result.error();

Modified: incubator/mesos/trunk/src/log/network.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/network.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/network.hpp (original)
+++ incubator/mesos/trunk/src/log/network.hpp Sun Sep  9 22:02:06 2012
@@ -32,7 +32,6 @@
 
 #include <stout/foreach.hpp>
 #include <stout/lambda.hpp>
-#include <stout/time.hpp>
 
 #include "logging/logging.hpp"
 

Modified: incubator/mesos/trunk/src/logging/logging.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/logging/logging.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/logging/logging.cpp (original)
+++ incubator/mesos/trunk/src/logging/logging.cpp Sun Sep  9 22:02:06 2012
@@ -29,10 +29,10 @@
 #include <process/process.hpp>
 #include <process/timeout.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/numify.hpp>
 #include <stout/os.hpp>
 #include <stout/stringify.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 
 #include "logging/logging.hpp"
@@ -110,7 +110,7 @@ private:
       return BadRequest("'" + stringify(v.get()) + "' < original level.\n");
     }
 
-    Try<seconds> s = seconds::parse(duration.get());
+    Try<Duration> s = Duration::parse(duration.get());
 
     if (s.isError()) {
       return BadRequest(s.error() + ".\n");
@@ -121,7 +121,7 @@ private:
 
     // Start a revert timer (if necessary).
     if (v.get() != original) {
-      timeout = s.get().value;
+      timeout = s.get().secs();
       delay(timeout.remaining(), this, &This::revert);
     }
 

Modified: incubator/mesos/trunk/src/master/frameworks_manager.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/frameworks_manager.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/frameworks_manager.cpp (original)
+++ incubator/mesos/trunk/src/master/frameworks_manager.cpp Sun Sep  9 22:02:06 2012
@@ -102,7 +102,7 @@ Result<bool> FrameworksManager::add(cons
 
 
 Future<Result<bool> > FrameworksManager::remove(const FrameworkID& id,
-                                                const seconds& s)
+                                                const Duration& timeout)
 {
   if (!cache()) {
     return Result<bool>::error("Error caching framework infos.");
@@ -113,13 +113,14 @@ Future<Result<bool> > FrameworksManager:
     return Result<bool>::error("Error removing non-existing framework.");
   }
 
-  LOG(INFO) << "Expiring framework " << id << " in " << s.value << " seconds";
+  LOG(INFO) << "Expiring framework " << id
+            << " in " << timeout.secs() << " seconds";
 
   // Set the option to contain the firing time of the message.
-  infos[id].second = Option<double>::some(Clock::now() + s.value);
+  infos[id].second = Option<double>::some(Clock::now() + timeout.secs());
 
   Promise<Result<bool> >* promise = new Promise<Result<bool> >();
-  delay(s.value, self(), &FrameworksManager::expire, id, promise);
+  delay(timeout.secs(), self(), &FrameworksManager::expire, id, promise);
   return promise->future();
 }
 

Modified: incubator/mesos/trunk/src/master/frameworks_manager.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/frameworks_manager.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/frameworks_manager.hpp (original)
+++ incubator/mesos/trunk/src/master/frameworks_manager.hpp Sun Sep  9 22:02:06 2012
@@ -25,9 +25,9 @@
 
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
 #include <stout/result.hpp>
-#include <stout/time.hpp>
 
 #include "messages/messages.hpp"
 
@@ -60,8 +60,8 @@ public:
   // Add a new framework.
   Result<bool> add(const FrameworkID& id, const FrameworkInfo& info);
 
-  // Remove a framework after a specified number of seconds.
-  Future<Result<bool> > remove(const FrameworkID& id, const seconds& s);
+  // Remove a framework after a specified amount of time.
+  Future<Result<bool> > remove(const FrameworkID& id, const Duration& timeout);
 
   // Resurrect the framework.
   Result<bool> resurrect(const FrameworkID& id);

Modified: incubator/mesos/trunk/src/master/slaves_manager.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/slaves_manager.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/slaves_manager.cpp (original)
+++ incubator/mesos/trunk/src/master/slaves_manager.cpp Sun Sep  9 22:02:06 2012
@@ -151,7 +151,7 @@ ZooKeeperSlavesManagerStorage::ZooKeeper
 {
   PID<ZooKeeperSlavesManagerStorage> pid(*this);
   watcher = new ZooKeeperSlavesManagerStorageWatcher(pid);
-  zk = new ZooKeeper(servers, milliseconds(10000), watcher);
+  zk = new ZooKeeper(servers, Milliseconds(10000), watcher);
 }
 
 
@@ -495,7 +495,7 @@ Future<bool> ZooKeeperSlavesManagerStora
   CHECK(zk != NULL);
   delete zk;
 
-  zk = new ZooKeeper(servers, milliseconds(10000), watcher);
+  zk = new ZooKeeper(servers, Milliseconds(10000), watcher);
 
   // TODO(benh): Put mechanisms in place such that reconnects may
   // fail (or just take too long).

Modified: incubator/mesos/trunk/src/slave/gc.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/gc.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/gc.cpp (original)
+++ incubator/mesos/trunk/src/slave/gc.cpp Sun Sep  9 22:02:06 2012
@@ -23,8 +23,8 @@
 #include <process/future.hpp>
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/os.hpp>
-#include <stout/time.hpp>
 
 #include "logging/logging.hpp"
 
@@ -44,7 +44,7 @@ class GarbageCollectorProcess : public P
 {
 public:
   // GarbageCollector implementation.
-  Future<bool> schedule(const seconds& s, const string& path);
+  Future<bool> schedule(const Duration& d, const string& path);
 
 private:
   void remove(const string& path, Promise<bool>* promise);
@@ -52,14 +52,14 @@ private:
 
 
 Future<bool> GarbageCollectorProcess::schedule(
-    const seconds& s,
+    const Duration& d,
     const string& path)
 {
   LOG(INFO) << "Scheduling " << path << " for removal";
 
   Promise<bool>* promise = new Promise<bool>();
 
-  delay(s.value, self(), &Self::remove, path, promise);
+  delay(d.secs(), self(), &Self::remove, path, promise);
 
   return promise->future();
 }
@@ -95,10 +95,10 @@ GarbageCollector::~GarbageCollector()
 
 
 Future<bool> GarbageCollector::schedule(
-    const seconds& s,
+    const Duration& d,
     const string& path)
 {
-  return dispatch(process, &GarbageCollectorProcess::schedule, s, path);
+  return dispatch(process, &GarbageCollectorProcess::schedule, d, path);
 }
 
 } // namespace mesos {

Modified: incubator/mesos/trunk/src/slave/gc.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/gc.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/gc.hpp (original)
+++ incubator/mesos/trunk/src/slave/gc.hpp Sun Sep  9 22:02:06 2012
@@ -23,7 +23,7 @@
 
 #include <process/future.hpp>
 
-#include <stout/time.hpp>
+#include <stout/duration.hpp>
 
 namespace mesos {
 namespace internal {
@@ -46,10 +46,10 @@ public:
   ~GarbageCollector();
 
   // Schedules the specified path for removal after the specified
-  // number of seconds have elapsed and returns true if the file was
+  // amount of time has elapsed and returns true if the file was
   // successfully removed and false if the file didn't exist (or an
   // error, e.g., permission denied).
-  process::Future<bool> schedule(const seconds& s, const std::string& path);
+  process::Future<bool> schedule(const Duration& d, const std::string& path);
 
 private:
   GarbageCollectorProcess* process;

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Sun Sep  9 22:02:06 2012
@@ -26,11 +26,11 @@
 #include <process/dispatch.hpp>
 #include <process/id.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/strings.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 #include <stout/utils.hpp>
 
@@ -366,7 +366,7 @@ void Slave::registered(const SlaveID& sl
   // Schedule all old slave directories to get garbage
   // collected. TODO(benh): It's unclear if we really need/want to
   // wait until the slave is registered to do this.
-  hours timeout(flags.gc_timeout_hours);
+  Hours timeout(flags.gc_timeout_hours);
 
   const string& directory = path::join(flags.work_dir, "slaves");
 
@@ -381,7 +381,7 @@ void Slave::registered(const SlaveID& sl
       if (time.isSome()) {
         // Schedule the directory to be removed after some remaining
         // delta of the timeout and last modification time.
-        seconds delta(timeout.secs() - (Clock::now() - time.get()));
+        Seconds delta(timeout.secs() - (Clock::now() - time.get()));
         gc.schedule(delta, path);
       } else {
         LOG(WARNING) << "Failed to get the modification time of "
@@ -1437,7 +1437,7 @@ void Slave::executorExited(const Framewo
   }
 
   // Schedule the executor directory to get garbage collected.
-  gc.schedule(hours(flags.gc_timeout_hours), executor->directory);
+  gc.schedule(Hours(flags.gc_timeout_hours), executor->directory);
 
   framework->destroyExecutor(executor->id);
 }
@@ -1482,7 +1482,7 @@ void Slave::shutdownExecutorTimeout(cons
              framework->id, executor->id);
 
     // Schedule the executor directory to get garbage collected.
-    gc.schedule(hours(flags.gc_timeout_hours), executor->directory);
+    gc.schedule(Hours(flags.gc_timeout_hours), executor->directory);
 
     framework->destroyExecutor(executor->id);
   }

Modified: incubator/mesos/trunk/src/state/zookeeper.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/state/zookeeper.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/state/zookeeper.cpp (original)
+++ incubator/mesos/trunk/src/state/zookeeper.cpp Sun Sep  9 22:02:06 2012
@@ -8,10 +8,10 @@
 #include <process/future.hpp>
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
 #include <stout/result.hpp>
 #include <stout/strings.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 #include <stout/uuid.hpp>
 
@@ -53,7 +53,7 @@ void fail(queue<T*>* queue, const string
 
 ZooKeeperStateProcess::ZooKeeperStateProcess(
     const string& _servers,
-    const seconds& _timeout,
+    const Duration& _timeout,
     const string& _znode,
     const Option<Authentication>& _auth)
   : servers(_servers),

Modified: incubator/mesos/trunk/src/state/zookeeper.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/state/zookeeper.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/state/zookeeper.hpp (original)
+++ incubator/mesos/trunk/src/state/zookeeper.hpp Sun Sep  9 22:02:06 2012
@@ -9,9 +9,9 @@
 #include <process/future.hpp>
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
 #include <stout/result.hpp>
-#include <stout/time.hpp>
 #include <stout/try.hpp>
 #include <stout/uuid.hpp>
 
@@ -39,7 +39,7 @@ public:
   // TODO(benh): Just take a zookeeper::URL.
   ZooKeeperState(
       const std::string& servers,
-      const seconds& timeout,
+      const Duration& timeout,
       const std::string& znode,
       const Option<zookeeper::Authentication>& auth =
       Option<zookeeper::Authentication>());
@@ -63,7 +63,7 @@ class ZooKeeperStateProcess : public pro
 public:
   ZooKeeperStateProcess(
       const std::string& servers,
-      const seconds& timeout,
+      const Duration& timeout,
       const std::string& znode,
       const Option<zookeeper::Authentication>& auth);
   virtual ~ZooKeeperStateProcess();
@@ -90,7 +90,7 @@ private:
   Result<bool> doSwap(const Entry& entry, const UUID& uuid);
 
   const std::string servers;
-  const seconds timeout;
+  const Duration timeout;
   const std::string znode;
 
   Option<zookeeper::Authentication> auth; // ZooKeeper authentication.
@@ -143,7 +143,7 @@ private:
 template <typename Serializer>
 ZooKeeperState<Serializer>::ZooKeeperState(
     const std::string& servers,
-    const seconds& timeout,
+    const Duration& timeout,
     const std::string& znode,
     const Option<zookeeper::Authentication>& auth)
 {

Modified: incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp (original)
+++ incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp Sun Sep  9 22:02:06 2012
@@ -44,7 +44,7 @@ namespace mesos {
 namespace internal {
 namespace test {
 
-const milliseconds BaseZooKeeperTest::NO_TIMEOUT(5000);
+const Milliseconds BaseZooKeeperTest::NO_TIMEOUT(5000);
 
 
 static void silenceServerLogs(Jvm* jvm)

Modified: incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp (original)
+++ incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp Sun Sep  9 22:02:06 2012
@@ -25,7 +25,7 @@
 
 #include <tr1/functional>
 
-#include <stout/time.hpp>
+#include <stout/duration.hpp>
 
 #include "jvm/jvm.hpp"
 
@@ -92,7 +92,7 @@ protected:
   virtual void TearDown();
 
   // A very long session timeout that simulates no timeout for test cases.
-  static const milliseconds NO_TIMEOUT;
+  static const Milliseconds NO_TIMEOUT;
 
   ZooKeeperServer* zks;
 

Modified: incubator/mesos/trunk/src/tests/gc_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/gc_tests.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/gc_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/gc_tests.cpp Sun Sep  9 22:02:06 2012
@@ -24,8 +24,8 @@
 #include <process/dispatch.hpp>
 #include <process/future.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/os.hpp>
-#include <stout/time.hpp>
 
 #include "detector/detector.hpp"
 
@@ -258,7 +258,7 @@ TEST_F(GarbageCollectorTest, Restart)
 
   sleep(1);
 
-  Clock::advance(hours(flags.gc_timeout_hours).secs());
+  Clock::advance(Hours(flags.gc_timeout_hours).secs());
 
   Clock::settle();
 
@@ -325,7 +325,7 @@ TEST_F(GarbageCollectorTest, ExitedExecu
 
   sleep(1);
 
-  Clock::advance(hours(flags.gc_timeout_hours).secs());
+  Clock::advance(Hours(flags.gc_timeout_hours).secs());
 
   Clock::settle();
 

Modified: incubator/mesos/trunk/src/tests/log_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/log_tests.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/log_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/log_tests.cpp Sun Sep  9 22:02:06 2012
@@ -1131,16 +1131,16 @@ TEST(LogTest, WriteRead)
 
   Log log(2, path2, pids);
 
-  Log::Writer writer(&log, seconds(1.0));
+  Log::Writer writer(&log, Seconds(1.0));
 
-  Result<Log::Position> position = writer.append("hello world", seconds(1.0));
+  Result<Log::Position> position = writer.append("hello world", Seconds(1.0));
 
   ASSERT_TRUE(position.isSome());
 
   Log::Reader reader(&log);
 
   Result<std::list<Log::Entry> > entries =
-    reader.read(position.get(), position.get(), seconds(1.0));
+    reader.read(position.get(), position.get(), Seconds(1.0));
 
   ASSERT_TRUE(entries.isSome());
   ASSERT_EQ(1u, entries.get().size());
@@ -1167,9 +1167,9 @@ TEST(LogTest, Position)
 
   Log log(2, path2, pids);
 
-  Log::Writer writer(&log, seconds(1.0));
+  Log::Writer writer(&log, Seconds(1.0));
 
-  Result<Log::Position> position = writer.append("hello world", seconds(1.0));
+  Result<Log::Position> position = writer.append("hello world", Seconds(1.0));
 
   ASSERT_TRUE(position.isSome());
 

Modified: incubator/mesos/trunk/src/tests/master_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_tests.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_tests.cpp Sun Sep  9 22:02:06 2012
@@ -1068,7 +1068,7 @@ TEST_F(FrameworksManagerTestFixture, Rem
   id.set_value("non-existent framework");
 
   Future<Result<bool> > future1 =
-    process::dispatch(manager, &FrameworksManager::remove, id, seconds(0));
+    process::dispatch(manager, &FrameworksManager::remove, id, Seconds(0));
 
   ASSERT_TRUE(future1.await(2.0));
   EXPECT_TRUE(future1.get().isError());
@@ -1092,7 +1092,7 @@ TEST_F(FrameworksManagerTestFixture, Rem
 
   // Now remove the added framework.
   Future<Result<bool> > future3 =
-    process::dispatch(manager, &FrameworksManager::remove, id2, seconds(1.0));
+    process::dispatch(manager, &FrameworksManager::remove, id2, Seconds(1.0));
 
   Clock::update(Clock::now(manager) + 1.0);
 
@@ -1169,7 +1169,7 @@ TEST_F(FrameworksManagerTestFixture, Res
 
   // Remove after 2 secs.
   Future<Result<bool> > future1 =
-    process::dispatch(manager, &FrameworksManager::remove, id, seconds(2.0));
+    process::dispatch(manager, &FrameworksManager::remove, id, Seconds(2.0));
 
   // Resurrect in the meanwhile.
   Future<Result<bool> > future2 =
@@ -1207,7 +1207,7 @@ TEST_F(FrameworksManagerTestFixture, Res
   Clock::pause();
 
   Future<Result<bool> > future1 =
-    process::dispatch(manager, &FrameworksManager::remove, id, seconds(2.0));
+    process::dispatch(manager, &FrameworksManager::remove, id, Seconds(2.0));
 
   // Resurrect in the meanwhile.
   Future<Result<bool> > future2 =
@@ -1215,7 +1215,7 @@ TEST_F(FrameworksManagerTestFixture, Res
 
   // Remove again.
   Future<Result<bool> > future3 =
-    process::dispatch(manager, &FrameworksManager::remove, id, seconds(1.0));
+    process::dispatch(manager, &FrameworksManager::remove, id, Seconds(1.0));
 
   ASSERT_TRUE(future2.await(2.0));
   EXPECT_TRUE(future2.get().get());
@@ -1284,7 +1284,7 @@ TEST(FrameworksManagerTest, CacheFailure
 
   // Remove framework should fail due to caching failure.
   Future<Result<bool> > future3 =
-    process::dispatch(manager, &FrameworksManager::remove, id, seconds(0));
+    process::dispatch(manager, &FrameworksManager::remove, id, Seconds(0));
 
   ASSERT_TRUE(future3.await(2.0));
   ASSERT_TRUE(future3.get().isError());

Modified: incubator/mesos/trunk/src/tests/stout_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/stout_tests.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/stout_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/stout_tests.cpp Sun Sep  9 22:02:06 2012
@@ -20,7 +20,6 @@
 #include <stout/result.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
-#include <stout/time.hpp>
 #include <stout/timer.hpp>
 #include <stout/try.hpp>
 #include <stout/uuid.hpp>

Modified: incubator/mesos/trunk/src/zookeeper/group.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/group.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/group.cpp (original)
+++ incubator/mesos/trunk/src/zookeeper/group.cpp Sun Sep  9 22:02:06 2012
@@ -8,6 +8,7 @@
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/numify.hpp>
 #include <stout/os.hpp>
 #include <stout/result.hpp>
@@ -40,7 +41,7 @@ class GroupProcess : public Process<Grou
 {
 public:
   GroupProcess(const string& servers,
-               const seconds& timeout,
+               const Duration& timeout,
                const string& znode,
                const Option<Authentication>& auth);
   virtual ~GroupProcess();
@@ -90,7 +91,7 @@ private:
   Option<string> error; // Potential non-retryable error.
 
   const string servers;
-  const seconds timeout;
+  const Duration timeout;
   const string znode;
 
   Option<Authentication> auth; // ZooKeeper authentication.
@@ -173,7 +174,7 @@ void fail(queue<T*>* queue, const string
 
 GroupProcess::GroupProcess(
     const string& _servers,
-    const seconds& _timeout,
+    const Duration& _timeout,
     const string& _znode,
     const Option<Authentication>& _auth)
   : servers(_servers),
@@ -839,7 +840,7 @@ void GroupProcess::abort()
 
 
 Group::Group(const string& servers,
-             const seconds& timeout,
+             const Duration& timeout,
              const string& znode,
              const Option<Authentication>& auth)
 {

Modified: incubator/mesos/trunk/src/zookeeper/group.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/group.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/group.hpp (original)
+++ incubator/mesos/trunk/src/zookeeper/group.hpp Sun Sep  9 22:02:06 2012
@@ -5,8 +5,8 @@
 
 #include "process/future.hpp"
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
-#include <stout/time.hpp>
 
 #include "zookeeper/authentication.hpp"
 
@@ -82,7 +82,7 @@ public:
   // Constructs this group using the specified ZooKeeper servers (list
   // of host:port) with the given timeout at the specified znode.
   Group(const std::string& servers,
-        const seconds& timeout,
+        const Duration& timeout,
         const std::string& znode,
         const Option<Authentication>& auth = Option<Authentication>::none());
   ~Group();

Modified: incubator/mesos/trunk/src/zookeeper/zookeeper.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/zookeeper.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/zookeeper.cpp (original)
+++ incubator/mesos/trunk/src/zookeeper/zookeeper.cpp Sun Sep  9 22:02:06 2012
@@ -26,6 +26,7 @@
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/fatal.hpp>
 
 #include "zookeeper/zookeeper.hpp"
@@ -162,8 +163,8 @@ class ZooKeeperImpl
 public:
   ZooKeeperImpl(ZooKeeper* zk,
                 const string& servers,
-                const milliseconds& timeout,
-		            Watcher* watcher)
+                const Duration& timeout,
+                Watcher* watcher)
     : servers(servers),
       timeout(timeout),
       zk(zk),
@@ -187,7 +188,7 @@ public:
     zh = zookeeper_init(
         servers.c_str(),
         event,
-        static_cast<int>(timeout.value),
+        static_cast<int>(timeout.ms()),
         NULL,
         this,
         0);
@@ -471,7 +472,7 @@ private:
   friend class ZooKeeper;
 
   const string servers; // ZooKeeper host:port pairs.
-  const milliseconds timeout; // ZooKeeper session timeout.
+  const Duration timeout; // ZooKeeper session timeout.
 
   ZooKeeper* zk; // ZooKeeper instance.
   zhandle_t* zh; // ZooKeeper connection handle.
@@ -482,7 +483,7 @@ private:
 
 
 ZooKeeper::ZooKeeper(const string& servers,
-                     const milliseconds& timeout,
+                     const Duration& timeout,
                      Watcher* watcher)
 {
   impl = new ZooKeeperImpl(this, servers, timeout, watcher);

Modified: incubator/mesos/trunk/src/zookeeper/zookeeper.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/zookeeper.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/zookeeper.hpp (original)
+++ incubator/mesos/trunk/src/zookeeper/zookeeper.hpp Sun Sep  9 22:02:06 2012
@@ -31,7 +31,7 @@
 #include <string>
 #include <vector>
 
-#include <stout/time.hpp>
+#include <stout/duration.hpp>
 
 
 /* Forward declarations of classes we are using. */
@@ -121,7 +121,7 @@ public:
    *    method will be invoked.
    */
   ZooKeeper(const std::string &servers,
-            const milliseconds& timeout,
+            const Duration& timeout,
             Watcher *watcher);
 
   ~ZooKeeper();

Modified: incubator/mesos/trunk/third_party/libprocess/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/Makefile.am?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/Makefile.am (original)
+++ incubator/mesos/trunk/third_party/libprocess/Makefile.am Sun Sep  9 22:02:06 2012
@@ -67,6 +67,7 @@ libprocess_la_SOURCES += $(top_srcdir)/i
 	$(top_srcdir)/include/process/timeout.hpp			\
 	$(top_srcdir)/include/process/timer.hpp				\
 	$(top_srcdir)/include/stout/cache.hpp				\
+	$(top_srcdir)/include/stout/duration.hpp			\
 	$(top_srcdir)/include/stout/fatal.hpp				\
 	$(top_srcdir)/include/stout/foreach.hpp				\
 	$(top_srcdir)/include/stout/format.hpp				\
@@ -84,7 +85,6 @@ libprocess_la_SOURCES += $(top_srcdir)/i
 	$(top_srcdir)/include/stout/result.hpp				\
 	$(top_srcdir)/include/stout/stringify.hpp			\
 	$(top_srcdir)/include/stout/strings.hpp				\
-	$(top_srcdir)/include/stout/time.hpp				\
 	$(top_srcdir)/include/stout/timer.hpp				\
 	$(top_srcdir)/include/stout/try.hpp				\
 	$(top_srcdir)/include/stout/utils.hpp				\

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/statistics.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/statistics.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/statistics.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/statistics.hpp Sun Sep  9 22:02:06 2012
@@ -3,8 +3,8 @@
 
 #include <process/future.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/option.hpp>
-#include <stout/time.hpp>
 
 namespace process {
 
@@ -17,14 +17,14 @@ class StatisticsProcess;
 class Statistics
 {
 public:
-  Statistics(const seconds& window);
+  Statistics(const Seconds& window);
   ~Statistics();
 
   // Returns the time series of a statistic.
-  process::Future<std::map<seconds, double> > get(
+  process::Future<std::map<Seconds, double> > get(
       const std::string& name,
-      const Option<seconds>& start = Option<seconds>::none(),
-      const Option<seconds>& stop = Option<seconds>::none());
+      const Option<Seconds>& start = Option<Seconds>::none(),
+      const Option<Seconds>& stop = Option<Seconds>::none());
 
   // Sets the current value of a statistic.
   void set(const std::string& name, double value);

Added: incubator/mesos/trunk/third_party/libprocess/include/stout/duration.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/duration.hpp?rev=1382588&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/duration.hpp (added)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/duration.hpp Sun Sep  9 22:02:06 2012
@@ -0,0 +1,150 @@
+#ifndef __STOUT_DURATION_HPP__
+#define __STOUT_DURATION_HPP__
+
+#include <ctype.h> // For 'isdigit'.
+
+#include <string>
+
+#include "numify.hpp"
+#include "try.hpp"
+
+
+class Duration
+{
+public:
+  double nanos() const { return value; }
+  double micros() const { return value / MICROSECONDS; }
+  double millis() const { return value / MILLISECONDS; }
+  double secs() const { return value / SECONDS; }
+  double mins() const { return value / MINUTES; }
+  double days() const { return value / DAYS; }
+  double weeks() const { return value / WEEKS; }
+
+  bool operator < (const Duration& that) const { return value < that.value; }
+
+  static Try<Duration> parse(const std::string& s)
+  {
+    // TODO(benh): Support negative durations (i.e., starts with '-').
+    size_t index = 0;
+    while (index < s.size()) {
+      if (isdigit(s[index]) || s[index] == '.') {
+        index++;
+        continue;
+      }
+
+      Try<double> value = numify<double>(s.substr(0, index));
+
+      if (value.isError()) {
+        return Try<Duration>::error(value.error());
+      }
+
+      const std::string& unit = s.substr(index);
+
+      if (unit == "ns") {
+        return Duration(value.get(), NANOSECONDS);
+      } else if (unit == "us") {
+        return Duration(value.get(), MICROSECONDS);
+      } else if (unit == "ms") {
+        return Duration(value.get(), MILLISECONDS);
+      } else if (unit == "secs") {
+        return Duration(value.get(), SECONDS);
+      } else if (unit == "mins") {
+        return Duration(value.get(), MINUTES);
+      } else if (unit == "hrs") {
+        return Duration(value.get(), HOURS);
+      } else if (unit == "days") {
+        return Duration(value.get(), DAYS);
+      } else if (unit == "weeks") {
+        return Duration(value.get(), WEEKS);
+      } else {
+        return Try<Duration>::error("Unknown duration unit '" + unit + "'");
+      }
+    }
+    return Try<Duration>::error("Invalid duration '" + s + "'");
+  }
+
+protected:
+  static const uint64_t NANOSECONDS = 1;
+  static const uint64_t MICROSECONDS = 1000 * NANOSECONDS;
+  static const uint64_t MILLISECONDS = 1000 * MICROSECONDS;
+  static const uint64_t SECONDS = 1000 * MILLISECONDS;
+  static const uint64_t MINUTES = 60 * SECONDS;
+  static const uint64_t HOURS = 60 * MINUTES;
+  static const uint64_t DAYS = 24 * HOURS;
+  static const uint64_t WEEKS = 7 * DAYS;
+
+  Duration(double _value, uint64_t _unit)
+  {
+    // Convert to the lowest unit (nanoseconds).
+    value = _value * _unit;
+  }
+
+private:
+  double value;
+};
+
+
+class Nanoseconds : public Duration
+{
+public:
+  explicit Nanoseconds(double value)
+    : Duration(value, NANOSECONDS) {}
+};
+
+
+class Microseconds : public Duration
+{
+public:
+  explicit Microseconds(double value)
+    : Duration(value, MICROSECONDS) {}
+};
+
+
+class Milliseconds : public Duration
+{
+public:
+  explicit Milliseconds(double value)
+    : Duration(value, MILLISECONDS) {}
+};
+
+
+class Seconds : public Duration
+{
+public:
+  explicit Seconds(double value)
+    : Duration(value, SECONDS) {}
+};
+
+
+class Minutes : public Duration
+{
+public:
+  explicit Minutes(double value)
+    : Duration(value, MINUTES) {}
+};
+
+
+class Hours : public Duration
+{
+public:
+  explicit Hours(double value)
+    : Duration(value, HOURS) {}
+};
+
+
+class Days : public Duration
+{
+public:
+  explicit Days(double value)
+    : Duration(value, DAYS) {}
+};
+
+
+class Weeks : public Duration
+{
+public:
+  explicit Weeks(double value)
+    : Duration(value, WEEKS) {}
+};
+
+#endif // __STOUT_DURATION_HPP__

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/timer.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/timer.hpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/timer.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/timer.hpp Sun Sep  9 22:02:06 2012
@@ -10,7 +10,7 @@
 
 #include <sys/time.h>
 
-#include "time.hpp"
+#include "duration.hpp"
 
 class Timer
 {
@@ -29,13 +29,13 @@ public:
     running = false;
   }
 
-  nanoseconds elapsed()
+  Nanoseconds elapsed()
   {
     if (!running) {
-      return nanoseconds(diff(stopped, started));
+      return Nanoseconds(diff(stopped, started));
     }
 
-    return nanoseconds(diff(now(), started));
+    return Nanoseconds(diff(now(), started));
   }
 
 private:

Modified: incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp Sun Sep  9 22:02:06 2012
@@ -13,6 +13,7 @@
 #include <process/process.hpp>
 #include <process/statistics.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/foreach.hpp>
 #include <stout/hashmap.hpp>
 #include <stout/json.hpp>
@@ -20,7 +21,6 @@
 #include <stout/option.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
-#include <stout/time.hpp>
 
 using namespace process;
 using namespace process::http;
@@ -34,7 +34,7 @@ namespace process {
 class StatisticsProcess : public Process<StatisticsProcess>
 {
 public:
-  StatisticsProcess(const seconds& _window)
+  StatisticsProcess(const Seconds& _window)
     : ProcessBase("statistics"),
       window(_window)
   {}
@@ -42,10 +42,10 @@ public:
   virtual ~StatisticsProcess() {}
 
   // Statistics implementation.
-  map<seconds, double> get(
+  map<Seconds, double> get(
       const string& name,
-      const Option<seconds>& start,
-      const Option<seconds>& stop);
+      const Option<Seconds>& start,
+      const Option<Seconds>& stop);
   void set(const string& name, double value);
   void increment(const string& name);
   void decrement(const string& name);
@@ -68,38 +68,38 @@ private:
   // Returns the time series of a statistic in JSON.
   Future<Response> series(const Request& request);
 
-  const seconds window;
+  const Seconds window;
 
   // We use a map instead of a hashmap to store the values because
   // that way we can retrieve a series in sorted order efficiently.
-  hashmap<string, map<seconds, double> > statistics;
+  hashmap<string, map<Seconds, double> > statistics;
 };
 
 
-map<seconds, double> StatisticsProcess::get(
+map<Seconds, double> StatisticsProcess::get(
     const string& name,
-    const Option<seconds>& start,
-    const Option<seconds>& stop)
+    const Option<Seconds>& start,
+    const Option<Seconds>& stop)
 {
   if (!statistics.contains(name)) {
-    return map<seconds, double>();
+    return map<Seconds, double>();
   }
 
-  const std::map<seconds, double>& values = statistics.find(name)->second;
+  const std::map<Seconds, double>& values = statistics.find(name)->second;
 
-  map<seconds, double>::const_iterator lower =
-    values.lower_bound(start.isSome() ? start.get() : seconds(0.0));
+  map<Seconds, double>::const_iterator lower =
+    values.lower_bound(start.isSome() ? start.get() : Seconds(0.0));
 
-  map<seconds, double>::const_iterator upper =
-    values.upper_bound(stop.isSome() ? stop.get() : seconds(DBL_MAX));
+  map<Seconds, double>::const_iterator upper =
+    values.upper_bound(stop.isSome() ? stop.get() : Seconds(DBL_MAX));
 
-  return map<seconds, double>(lower, upper);
+  return map<Seconds, double>(lower, upper);
 }
 
 
 void StatisticsProcess::set(const string& name, double value)
 {
-  statistics[name][seconds(Clock::now())] = value;
+  statistics[name][Seconds(Clock::now())] = value;
   truncate(name);
 }
 
@@ -108,9 +108,9 @@ void StatisticsProcess::increment(const 
 {
   if (statistics[name].size() > 0) {
     double d = statistics[name].rbegin()->second;
-    statistics[name][seconds(Clock::now())] = d + 1.0;
+    statistics[name][Seconds(Clock::now())] = d + 1.0;
   } else {
-    statistics[name][seconds(Clock::now())] = 1.0;
+    statistics[name][Seconds(Clock::now())] = 1.0;
   }
 
   truncate(name);
@@ -121,9 +121,9 @@ void StatisticsProcess::decrement(const 
 {
   if (statistics[name].size() > 0) {
     double d = statistics[name].rbegin()->second;
-    statistics[name][seconds(Clock::now())] = d - 1.0;
+    statistics[name][Seconds(Clock::now())] = d - 1.0;
   } else {
-    statistics[name][seconds(Clock::now())] = -1.0;
+    statistics[name][Seconds(Clock::now())] = -1.0;
   }
 
   truncate(name);
@@ -140,9 +140,9 @@ void StatisticsProcess::truncate(const s
     return;
   }
 
-  map<seconds, double>::iterator start = statistics[name].begin();
+  map<Seconds, double>::iterator start = statistics[name].begin();
 
-  while ((Clock::now() - start->first.value) > window.value) {
+  while ((Clock::now() - start->first.secs()) > window.secs()) {
     statistics[name].erase(start);
     if (statistics[name].size() == 1) {
       break;
@@ -160,7 +160,7 @@ Future<Response> StatisticsProcess::snap
     CHECK(statistics[name].size() > 0);
     JSON::Object object;
     object.values["name"] = name;
-    object.values["time"] = statistics[name].rbegin()->first.value;
+    object.values["time"] = statistics[name].rbegin()->first.secs();
     object.values["value"] = statistics[name].rbegin()->second;
     array.values.push_back(object);
   }
@@ -187,8 +187,8 @@ Future<Response> StatisticsProcess::seri
     ? Option<string>::some(pairs["name"].back())
     : Option<string>::none();
 
-  Option<seconds> start = Option<seconds>::none();
-  Option<seconds> stop = Option<seconds>::none();
+  Option<Seconds> start = Option<Seconds>::none();
+  Option<Seconds> stop = Option<Seconds>::none();
 
   if (pairs.count("start") > 0 && pairs["start"].size() > 0) {
     Try<double> result = numify<double>(pairs["start"].back());
@@ -198,7 +198,7 @@ Future<Response> StatisticsProcess::seri
                    << result.error();
       return BadRequest();
     }
-    start = Option<seconds>::some(seconds(result.get()));
+    start = Option<Seconds>::some(Seconds(result.get()));
   }
 
   if (pairs.count("stop") > 0 && pairs["stop"].size() > 0) {
@@ -209,17 +209,17 @@ Future<Response> StatisticsProcess::seri
                    << result.error();
       return BadRequest();
     }
-    stop = Option<seconds>::some(seconds(result.get()));
+    stop = Option<Seconds>::some(Seconds(result.get()));
   }
 
   if (name.isSome()) {
     JSON::Array array;
 
-    map<seconds, double> values = get(name.get(), start, stop);
+    map<Seconds, double> values = get(name.get(), start, stop);
 
-    foreachpair (const seconds& s, double value, values) {
+    foreachpair (const Seconds& s, double value, values) {
       JSON::Object object;
-      object.values["time"] = s.value;
+      object.values["time"] = s.secs();
       object.values["value"] = value;
       array.values.push_back(object);
     }
@@ -239,7 +239,7 @@ Future<Response> StatisticsProcess::seri
 }
 
 
-Statistics::Statistics(const seconds& window)
+Statistics::Statistics(const Seconds& window)
 {
   process = new StatisticsProcess(window);
   spawn(process);
@@ -253,10 +253,10 @@ Statistics::~Statistics()
 }
 
 
-Future<map<seconds, double> > Statistics::get(
+Future<map<Seconds, double> > Statistics::get(
     const string& name,
-    const Option<seconds>& start,
-    const Option<seconds>& stop)
+    const Option<Seconds>& start,
+    const Option<Seconds>& stop)
 {
   return dispatch(process, &StatisticsProcess::get, name, start, stop);
 }

Modified: incubator/mesos/trunk/third_party/libprocess/src/statistics_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/statistics_tests.cpp?rev=1382588&r1=1382587&r2=1382588&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/statistics_tests.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/statistics_tests.cpp Sun Sep  9 22:02:06 2012
@@ -24,7 +24,7 @@
 #include <process/future.hpp>
 #include <process/statistics.hpp>
 
-#include <stout/time.hpp>
+#include <stout/duration.hpp>
 
 using namespace process;
 
@@ -33,17 +33,17 @@ using std::map;
 
 TEST(Statistics, set)
 {
-  Statistics statistics(seconds(60*60*24));
+  Statistics statistics(Seconds(60*60*24));
 
   statistics.set("statistic", 3.14);
 
-  Future<map<seconds, double> > values = statistics.get("statistic");
+  Future<map<Seconds, double> > values = statistics.get("statistic");
 
   values.await();
 
   ASSERT_TRUE(values.isReady());
   EXPECT_EQ(1, values.get().size());
-  EXPECT_GE(Clock::now(), values.get().begin()->first.value);
+  EXPECT_GE(Clock::now(), values.get().begin()->first.secs());
   EXPECT_DOUBLE_EQ(3.14, values.get().begin()->second);
 }
 
@@ -52,17 +52,17 @@ TEST(Statistics, truncate)
 {
   Clock::pause();
 
-  Statistics statistics(seconds(60*60*24));
+  Statistics statistics(Seconds(60*60*24));
 
   statistics.set("statistic", 3.14);
 
-  Future<map<seconds, double> > values = statistics.get("statistic");
+  Future<map<Seconds, double> > values = statistics.get("statistic");
 
   values.await();
 
   ASSERT_TRUE(values.isReady());
   EXPECT_EQ(1, values.get().size());
-  EXPECT_GE(Clock::now(), values.get().begin()->first.value);
+  EXPECT_GE(Clock::now(), values.get().begin()->first.secs());
   EXPECT_DOUBLE_EQ(3.14, values.get().begin()->second);
 
   Clock::advance((60*60*24) + 1);
@@ -75,7 +75,7 @@ TEST(Statistics, truncate)
 
   ASSERT_TRUE(values.isReady());
   EXPECT_EQ(1, values.get().size());
-  EXPECT_GE(Clock::now(), values.get().begin()->first.value);
+  EXPECT_GE(Clock::now(), values.get().begin()->first.secs());
   EXPECT_DOUBLE_EQ(4.14, values.get().begin()->second);
 
   Clock::resume();