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/05/01 00:34:22 UTC

svn commit: r1332453 - in /incubator/mesos/trunk: src/exec/ src/sched/ src/slave/ third_party/libprocess/ third_party/libprocess/include/process/ third_party/libprocess/src/

Author: benh
Date: Mon Apr 30 22:34:21 2012
New Revision: 1332453

URL: http://svn.apache.org/viewvc?rev=1332453&view=rev
Log:
Added the ability to generate IDs for libprocess processes and named a bunch of the ones in Mesos.

Added:
    incubator/mesos/trunk/third_party/libprocess/include/process/id.hpp
Modified:
    incubator/mesos/trunk/src/exec/exec.cpp
    incubator/mesos/trunk/src/sched/sched.cpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
    incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
    incubator/mesos/trunk/src/slave/reaper.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/third_party/libprocess/Makefile.am
    incubator/mesos/trunk/third_party/libprocess/include/process/executor.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/protobuf.hpp
    incubator/mesos/trunk/third_party/libprocess/src/latch.cpp
    incubator/mesos/trunk/third_party/libprocess/src/process.cpp

Modified: incubator/mesos/trunk/src/exec/exec.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/exec/exec.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/exec/exec.cpp (original)
+++ incubator/mesos/trunk/src/exec/exec.cpp Mon Apr 30 22:34:21 2012
@@ -18,8 +18,6 @@
 
 #include <signal.h>
 
-#include <glog/logging.h>
-
 #include <iostream>
 #include <string>
 #include <sstream>
@@ -27,6 +25,7 @@
 #include <mesos/executor.hpp>
 
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 
@@ -62,7 +61,8 @@ public:
                   const ExecutorID& _executorId,
                   bool _local,
                   const std::string& _directory)
-    : slave(_slave),
+    : ProcessBase(ID::generate("executor")),
+      slave(_slave),
       driver(_driver),
       executor(_executor),
       frameworkId(_frameworkId),

Modified: incubator/mesos/trunk/src/sched/sched.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/sched/sched.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/sched/sched.cpp (original)
+++ incubator/mesos/trunk/src/sched/sched.cpp Mon Apr 30 22:34:21 2012
@@ -33,6 +33,7 @@
 #include <mesos/scheduler.hpp>
 
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 #include <process/timer.hpp>
@@ -81,7 +82,8 @@ public:
                    const FrameworkInfo& _framework,
                    pthread_mutex_t* _mutex,
                    pthread_cond_t* _cond)
-    : driver(_driver),
+    : ProcessBase(ID::generate("scheduler")),
+      driver(_driver),
       scheduler(_scheduler),
       framework(_framework),
       mutex(_mutex),

Modified: incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp Mon Apr 30 22:34:21 2012
@@ -57,7 +57,8 @@ const int64_t MIN_MEMORY_MB = 128 * Mega
 
 
 LxcIsolationModule::LxcIsolationModule()
-  : initialized(false)
+  : ProcessBase(ID::generate("lxc-isolation-module")),
+    initialized(false)
 {
   // Spawn the reaper, note that it might send us a message before we
   // actually get spawned ourselves, but that's okay, the message will

Modified: incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp Mon Apr 30 22:34:21 2012
@@ -21,6 +21,7 @@
 #include <map>
 
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
 
 #include "process_based_isolation_module.hpp"
 
@@ -44,7 +45,8 @@ using process::wait; // Necessary on som
 
 
 ProcessBasedIsolationModule::ProcessBasedIsolationModule()
-  : initialized(false)
+  : ProcessBase(ID::generate("process-isolation-module")),
+    initialized(false)
 {
   // Spawn the reaper, note that it might send us a message before we
   // actually get spawned ourselves, but that's okay, the message will

Modified: incubator/mesos/trunk/src/slave/reaper.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/reaper.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/reaper.cpp (original)
+++ incubator/mesos/trunk/src/slave/reaper.cpp Mon Apr 30 22:34:21 2012
@@ -20,20 +20,21 @@
 #include <sys/wait.h>
 
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
 #include <process/timer.hpp>
 
-#include "reaper.hpp"
-
 #include "common/foreach.hpp"
 
-using namespace process;
+#include "slave/reaper.hpp"
 
+using namespace process;
 
 namespace mesos {
 namespace internal {
 namespace slave {
 
-Reaper::Reaper() {}
+Reaper::Reaper()
+  : ProcessBase(ID::generate("reaper")) {}
 
 
 Reaper::~Reaper() {}

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Mon Apr 30 22:34:21 2012
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <iomanip>
 
+#include <process/id.hpp>
 #include <process/timer.hpp>
 
 #include "common/build.hpp"
@@ -84,7 +85,7 @@ bool isTerminalTaskState(TaskState state
 Slave::Slave(const Resources& _resources,
              bool _local,
              IsolationModule* _isolationModule)
-  : ProcessBase("slave"),
+  : ProcessBase(ID::generate("slave")),
     resources(_resources),
     local(_local),
     isolationModule(_isolationModule)
@@ -94,7 +95,7 @@ Slave::Slave(const Resources& _resources
 Slave::Slave(const Configuration& _conf,
              bool _local,
              IsolationModule* _isolationModule)
-  : ProcessBase("slave"),
+  : ProcessBase(ID::generate("slave")),
     conf(_conf),
     local(_local),
     isolationModule(_isolationModule)

Modified: incubator/mesos/trunk/third_party/libprocess/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/Makefile.am?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/Makefile.am (original)
+++ incubator/mesos/trunk/third_party/libprocess/Makefile.am Mon Apr 30 22:34:21 2012
@@ -48,8 +48,11 @@ libprocess_la_SOURCES += $(top_srcdir)/i
 	$(top_srcdir)/include/process/future.hpp			\
 	$(top_srcdir)/include/process/gc.hpp				\
 	$(top_srcdir)/include/process/http.hpp				\
+	$(top_srcdir)/include/process/id.hpp				\
 	$(top_srcdir)/include/process/latch.hpp				\
 	$(top_srcdir)/include/process/message.hpp			\
+	$(top_srcdir)/include/process/nothing.hpp			\
+	$(top_srcdir)/include/process/once.hpp				\
 	$(top_srcdir)/include/process/option.hpp			\
 	$(top_srcdir)/include/process/pid.hpp				\
 	$(top_srcdir)/include/process/preprocessor.hpp			\

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/executor.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/executor.hpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/executor.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/executor.hpp Mon Apr 30 22:34:21 2012
@@ -3,6 +3,7 @@
 
 #include <process/deferred.hpp>
 #include <process/dispatch.hpp>
+#include <process/id.hpp>
 #include <process/preprocessor.hpp>
 
 namespace process {
@@ -14,7 +15,7 @@ class ExecutorProcess : public process::
 private:
   friend class Executor;
 
-  ExecutorProcess() {}
+  ExecutorProcess() : ProcessBase(ID::generate("__executor__")) {}
   ~ExecutorProcess() {}
 
   // Not copyable, not assignable.

Added: incubator/mesos/trunk/third_party/libprocess/include/process/id.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/id.hpp?rev=1332453&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/id.hpp (added)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/id.hpp Mon Apr 30 22:34:21 2012
@@ -0,0 +1,16 @@
+#ifndef __PROCESS_ID_HPP__
+#define __PROCESS_ID_HPP__
+
+#include <string>
+
+namespace process {
+namespace ID {
+
+// Returns 'prefix(N)' where N represents the number of instances
+// where this prefix has been used to generate an ID.
+std::string generate(const std::string& prefix = "");
+
+} // namespace ID {
+} // namespace process {
+
+#endif // __PROCESS_ID_HPP__

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp Mon Apr 30 22:34:21 2012
@@ -216,7 +216,7 @@ private:
 template <typename T>
 class Process : public virtual ProcessBase {
 public:
-  Process(const std::string& id = "") : ProcessBase(id) {}
+  virtual ~Process() {}
 
   // Returns pid of process; valid even before calling spawn.
   PID<T> self() const { return PID<T>(dynamic_cast<const T*>(this)); }

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/protobuf.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/protobuf.hpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/protobuf.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/protobuf.hpp Mon Apr 30 22:34:21 2012
@@ -66,9 +66,6 @@ template <typename T>
 class ProtobufProcess : public process::Process<T>
 {
 public:
-  ProtobufProcess(const std::string& id = "")
-    : process::Process<T>(id) {}
-
   virtual ~ProtobufProcess() {}
 
 protected:

Modified: incubator/mesos/trunk/third_party/libprocess/src/latch.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/latch.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/latch.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/latch.cpp Mon Apr 30 22:34:21 2012
@@ -1,10 +1,10 @@
+#include <process/id.hpp>
 #include <process/latch.hpp>
 #include <process/process.hpp>
 
-
 namespace process {
 
-// TODO(benh): Provided an "optimized" implementation of a latch that
+// TODO(benh): Provide an "optimized" implementation of a latch that
 // is libprocess aware. That is, allow integrate "waiting" on a latch
 // within libprocess such that it doesn't cost a memory allocation, a
 // spawn, a message send, a wait, and two user-space context-switchs.
@@ -18,7 +18,7 @@ Latch::Latch()
   // deleting thread is holding. Hence, we only save the PID for
   // triggering the latch and let the GC actually do the deleting
   // (thus no waiting is necessary, and deadlocks are avoided).
-  pid = spawn(new ProcessBase(), true);
+  pid = spawn(new ProcessBase(ID::generate("__latch__")), true);
 }
 
 

Modified: incubator/mesos/trunk/third_party/libprocess/src/process.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/process.cpp?rev=1332453&r1=1332452&r2=1332453&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/process.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/process.cpp Mon Apr 30 22:34:21 2012
@@ -51,6 +51,7 @@
 #include <process/filter.hpp>
 #include <process/future.hpp>
 #include <process/gc.hpp>
+#include <process/id.hpp>
 #include <process/process.hpp>
 #include <process/socket.hpp>
 #include <process/timer.hpp>
@@ -126,6 +127,18 @@ public:
 
 namespace process {
 
+namespace ID {
+
+string generate(const string& prefix)
+{
+  static map<string, int> prefixes;
+  stringstream out;
+  out << __sync_add_and_fetch(&prefixes[prefix], 1);
+  return prefix + "(" + out.str() + ")";
+}
+
+} // namespace ID {
+
 // Provides reference counting semantics for a process pointer.
 class ProcessReference
 {
@@ -1356,7 +1369,8 @@ void initialize(const string& delegate)
 
 
 HttpProxy::HttpProxy(const Socket& _socket)
-  : socket(_socket) {}
+  : ProcessBase(ID::generate("__http__")),
+    socket(_socket) {}
 
 
 HttpProxy::~HttpProxy()
@@ -2522,7 +2536,7 @@ bool cancel(const Timer& timer)
 } // namespace timers {
 
 
-ProcessBase::ProcessBase(const std::string& _id)
+ProcessBase::ProcessBase(const std::string& id)
 {
   process::initialize();
 
@@ -2532,15 +2546,7 @@ ProcessBase::ProcessBase(const std::stri
 
   refs = 0;
 
-  if (_id != "") {
-    pid.id = _id;
-  } else {
-    // Generate string representation of unique id for process.
-    stringstream out;
-    out << __sync_add_and_fetch(&__id__, 1);
-    pid.id = out.str();
-  }
-
+  pid.id = id != "" ? id : ID::generate();
   pid.ip = __ip__;
   pid.port = __port__;
 
@@ -2798,7 +2804,10 @@ class WaitWaiter : public Process<WaitWa
 {
 public:
   WaitWaiter(const UPID& _pid, double _secs, bool* _waited)
-    : pid(_pid), secs(_secs), waited(_waited) {}
+    : ProcessBase(ID::generate("__waiter__")),
+      pid(_pid),
+      secs(_secs),
+      waited(_waited) {}
 
   virtual void initialize()
   {