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 2011/06/05 11:17:56 UTC

svn commit: r1132276 - in /incubator/mesos/trunk/third_party/libprocess: process.cpp process.hpp

Author: benh
Date: Sun Jun  5 09:17:56 2011
New Revision: 1132276

URL: http://svn.apache.org/viewvc?rev=1132276&view=rev
Log:
Minor updates to libprocess names.

Modified:
    incubator/mesos/trunk/third_party/libprocess/process.cpp
    incubator/mesos/trunk/third_party/libprocess/process.hpp

Modified: incubator/mesos/trunk/third_party/libprocess/process.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/process.cpp?rev=1132276&r1=1132275&r2=1132276&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/process.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/process.cpp Sun Jun  5 09:17:56 2011
@@ -146,7 +146,7 @@ public:
   {
     if (process != NULL) {
       __sync_fetch_and_add(&(process->refs), 1);
-      if (process->state == ProcessBase::EXITING) {
+      if (process->state == ProcessBase::FINISHING) {
         __sync_fetch_and_sub(&(process->refs), 1);
         process = NULL;
       }
@@ -319,7 +319,7 @@ protected:
   {
     while (true) {
       serve();
-      if (name() == EXIT && proxies.count(from()) > 0) {
+      if (name() == EXITED && proxies.count(from()) > 0) {
         HttpProxy* proxy = proxies[from()];
         proxies.erase(from());
         delete proxy;
@@ -1234,7 +1234,7 @@ void LinkManager::link(ProcessBase *proc
 {
   // TODO(benh): The semantics we want to support for link are such
   // that if there is nobody to link to (local or remote) then a
-  // EXIT message gets generated. This functionality has only
+  // EXITED message gets generated. This functionality has only
   // been implemented when the link is local, not remote. Of course,
   // if there is nobody listening on the remote side, then this should
   // work remotely ... but if there is someone listening remotely just
@@ -1496,7 +1496,7 @@ void LinkManager::exited(const Node &nod
       if (pid.ip == node.ip && pid.port == node.port) {
         // N.B. If we call exited(pid) we might invalidate iteration.
         foreach (ProcessBase *process, processes) {
-          Message* message = encode(pid, process->pid, EXIT);
+          Message* message = encode(pid, process->pid, EXITED);
           process->enqueue(message);
         }
         removed.push_back(pid);
@@ -1524,7 +1524,7 @@ void LinkManager::exited(ProcessBase *pr
       set<ProcessBase *> &processes = it->second;
       foreach (ProcessBase *p, processes) {
         assert(process != p);
-        Message *message = encode(process->pid, p->pid, EXIT);
+        Message *message = encode(process->pid, p->pid, EXITED);
         // TODO(benh): Preserve happens-before when using clock.
         p->enqueue(message);
       }
@@ -1786,13 +1786,13 @@ void ProcessManager::link(ProcessBase *p
   } else {
     // Since the pid is local we want to get a reference to it's
     // underlying process so that while we are invoking the link
-    // manager we don't miss sending a possible EXIT.
+    // manager we don't miss sending a possible EXITED.
     if (ProcessReference _ = use(to)) {
       link_manager->link(process, to);
     } else {
       // Since the pid isn't valid it's process must have already died
       // (or hasn't been spawned yet) so send a process exit message.
-      Message *message = encode(to, process->pid, EXIT);
+      Message *message = encode(to, process->pid, EXITED);
       process->enqueue(message);
     }
   }
@@ -1940,7 +1940,7 @@ bool ProcessManager::wait(ProcessBase *p
   /* Now we can add the process to the waiters. */
   synchronized (processes) {
     if (processes.count(pid.id) > 0) {
-      assert(processes[pid.id]->state != ProcessBase::EXITED);
+      assert(processes[pid.id]->state != ProcessBase::FINISHED);
       waiters[processes[pid.id]].insert(process);
       waited = true;
     }
@@ -1987,7 +1987,7 @@ bool ProcessManager::external_wait(const
   synchronized (processes) {
     if (processes.count(pid.id) > 0) {
       ProcessBase *process = processes[pid.id];
-      assert(process->state != ProcessBase::EXITED);
+      assert(process->state != ProcessBase::FINISHED);
 
       /* Check and see if a gate already exists. */
       if (gates.find(process) == gates.end())
@@ -2136,7 +2136,7 @@ void ProcessManager::timedout(const UPID
 
         if (process->state != ProcessBase::RUNNING ||
             process->state != ProcessBase::INTERRUPTED ||
-            process->state != ProcessBase::EXITING) {
+            process->state != ProcessBase::FINISHING) {
           process_manager->enqueue(process);
         }
 
@@ -2145,7 +2145,7 @@ void ProcessManager::timedout(const UPID
         // that a process that was polling when selected from the
         // runq will fall out because of a timeout even though it also
         // received a message.
-        if (process->state != ProcessBase::EXITING) {
+        if (process->state != ProcessBase::FINISHING) {
           process->state = ProcessBase::TIMEDOUT;
         }
       }
@@ -2205,7 +2205,7 @@ void ProcessManager::cleanup(ProcessBase
   Gate *gate = NULL;
 
   // Stop new process references from being created.
-  process->state = ProcessBase::EXITING;
+  process->state = ProcessBase::FINISHING;
 
   /* Remove process. */
   synchronized (processes) {
@@ -2256,9 +2256,9 @@ void ProcessManager::cleanup(ProcessBase
         // N.B. The last thread that leaves the gate also free's it.
         gates.erase(it);
       }
-        
+
       assert(process->refs == 0);
-      process->state = ProcessBase::EXITED;
+      process->state = ProcessBase::FINISHED;
     }
     process->unlock();
   }
@@ -2289,7 +2289,7 @@ void ProcessManager::cleanup(ProcessBase
       // TODO(benh): Once we actually run multiple processes at a
       // time (using multiple threads) this logic will need to get
       // made thread safe (in particular, a process may be
-      // EXITING).
+      // FINISHING).
       assert(p->state == ProcessBase::RUNNING ||
              p->state == ProcessBase::WAITING);
       if (p->state == ProcessBase::RUNNING) {
@@ -2380,7 +2380,7 @@ void Clock::pause()
     // process). This Means that we have to take special care to
     // ensure happens-before timing (currently done for local message
     // sends and spawning new processes, not currently done for
-    // EXIT messages).
+    // EXITED messages).
     if (clk == NULL) {
       clk = new InternalClock();
 
@@ -2479,7 +2479,7 @@ void ProcessBase::enqueue(Message* messa
 
   lock();
   {
-    assert(state != EXITED);
+    assert(state != FINISHED);
 
     messages.push_back(message);
 
@@ -2498,7 +2498,7 @@ void ProcessBase::enqueue(Message* messa
            state == WAITING ||
            state == INTERRUPTED ||
            state == TIMEDOUT ||
-           state == EXITING);
+           state == FINISHING);
   }
   unlock();
 }
@@ -2512,7 +2512,7 @@ void ProcessBase::enqueue(pair<HttpReque
 
   lock();
   {
-    assert(state != EXITED);
+    assert(state != FINISHED);
 
     requests.push_back(request);
 
@@ -2532,7 +2532,7 @@ void ProcessBase::enqueue(pair<HttpReque
            state == WAITING ||
            state == INTERRUPTED ||
            state == TIMEDOUT ||
-           state == EXITING);
+           state == FINISHING);
   }
   unlock();
 }
@@ -2546,7 +2546,7 @@ void ProcessBase::enqueue(function<void(
 
   lock();
   {
-    assert(state != EXITED);
+    assert(state != FINISHED);
 
     delegators.push_back(delegator);
 
@@ -2566,7 +2566,7 @@ void ProcessBase::enqueue(function<void(
            state == WAITING ||
            state == INTERRUPTED ||
            state == TIMEDOUT ||
-           state == EXITING);
+           state == FINISHING);
   }
   unlock();
 }

Modified: incubator/mesos/trunk/third_party/libprocess/process.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/process.hpp?rev=1132276&r1=1132275&r2=1132276&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/process.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/process.hpp Sun Jun  5 09:17:56 2011
@@ -21,10 +21,10 @@
 
 namespace process {
 
-const std::string ERROR = "error";
-const std::string TIMEOUT = "timeout";
-const std::string EXIT = "exit";
-const std::string TERMINATE = "terminate";
+const std::string ERROR = "__process_error__";
+const std::string TIMEOUT = "__process_timeout__";
+const std::string EXITED = "__process_exited__";
+const std::string TERMINATE = "__process_terminate__";
 
 
 struct Message {
@@ -138,8 +138,8 @@ private:
 	 WAITING,
 	 INTERRUPTED,
 	 TIMEDOUT,
-         EXITING,
-	 EXITED } state;
+         FINISHING,
+	 FINISHED } state;
 
   /* Lock/mutex protecting internals. */
   pthread_mutex_t m;