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/08/11 00:00:19 UTC

svn commit: r1371830 - in /incubator/mesos/trunk: src/ src/linux/ src/slave/ src/tests/ third_party/libprocess/include/stout/

Author: benh
Date: Fri Aug 10 22:00:19 2012
New Revision: 1371830

URL: http://svn.apache.org/viewvc?rev=1371830&view=rev
Log:
Factored garbage collection out of slave/executors directories into a
GarbageCollector abstraction (https://reviews.apache.org/r/6476).

Added:
    incubator/mesos/trunk/src/slave/gc.cpp
    incubator/mesos/trunk/src/slave/gc.hpp
    incubator/mesos/trunk/src/tests/gc_tests.cpp
      - copied, changed from r1371362, incubator/mesos/trunk/src/tests/slave_tests.cpp
Removed:
    incubator/mesos/trunk/src/tests/slave_tests.cpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/linux/proc.cpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/tests/stout_tests.cpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Fri Aug 10 22:00:19 2012
@@ -153,8 +153,8 @@ nodist_libmesos_no_third_party_la_SOURCE
 libmesos_no_third_party_la_SOURCES = sched/sched.cpp local/local.cpp	\
 	master/master.cpp master/http.cpp master/slaves_manager.cpp	\
 	master/frameworks_manager.cpp					\
-	master/dominant_share_allocator.cpp slave/slave.cpp		\
-	slave/http.cpp slave/isolation_module.cpp			\
+	master/dominant_share_allocator.cpp slave/gc.cpp		\
+	slave/slave.cpp slave/http.cpp slave/isolation_module.cpp	\
 	slave/process_based_isolation_module.cpp slave/reaper.cpp	\
 	launcher/launcher.cpp exec/exec.cpp common/lock.cpp		\
 	detector/detector.cpp configurator/configurator.cpp		\
@@ -202,7 +202,7 @@ libmesos_no_third_party_la_SOURCES += co
 	master/master.hpp master/dominant_share_allocator.hpp		\
 	master/slaves_manager.hpp master/webui.hpp			\
 	messages/messages.hpp slave/constants.hpp slave/flags.hpp	\
-	slave/http.hpp slave/isolation_module.hpp			\
+	slave/gc.hpp slave/http.hpp slave/isolation_module.hpp		\
 	slave/isolation_module_factory.hpp				\
 	slave/cgroups_isolation_module.hpp				\
 	slave/lxc_isolation_module.hpp					\
@@ -764,7 +764,7 @@ check_PROGRAMS += mesos-tests
 
 mesos_tests_SOURCES = tests/main.cpp tests/utils.cpp			\
 	              tests/master_tests.cpp tests/state_tests.cpp	\
-	              tests/slave_tests.cpp				\
+	              tests/gc_tests.cpp				\
 	              tests/resource_offers_tests.cpp			\
 	              tests/fault_tolerance_tests.cpp			\
 	              tests/flags_tests.cpp tests/log_tests.cpp		\

Modified: incubator/mesos/trunk/src/linux/proc.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/proc.cpp?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/proc.cpp (original)
+++ incubator/mesos/trunk/src/linux/proc.cpp Fri Aug 10 22:00:19 2012
@@ -22,10 +22,10 @@ Try<set<pid_t> > pids()
 {
   set<pid_t> pids;
 
-  foreach (const string& path, os::listdir("/proc")) {
-    Try<pid_t> pid = numify<pid_t>(path);
+  foreach (const string& file, os::ls("/proc")) {
+    Try<pid_t> pid = numify<pid_t>(file);
 
-    // Ignore paths that can't be numified.
+    // Ignore files that can't be numified.
     if (pid.isSome()) {
       pids.insert(pid.get());
     }

Added: incubator/mesos/trunk/src/slave/gc.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/gc.cpp?rev=1371830&view=auto
==============================================================================
--- incubator/mesos/trunk/src/slave/gc.cpp (added)
+++ incubator/mesos/trunk/src/slave/gc.cpp Fri Aug 10 22:00:19 2012
@@ -0,0 +1,106 @@
+/**
+ * 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 <string>
+
+#include <process/delay.hpp>
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/process.hpp>
+
+#include <stout/os.hpp>
+#include <stout/time.hpp>
+
+#include "logging/logging.hpp"
+
+#include "slave/gc.hpp"
+
+using namespace process;
+
+using process::wait; // Necessary on some OS's to disambiguate.
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class GarbageCollectorProcess : public Process<GarbageCollectorProcess>
+{
+public:
+  // GarbageCollector implementation.
+  Future<bool> schedule(const seconds& s, const string& path);
+
+private:
+  void remove(const string& path, Promise<bool>* promise);
+};
+
+
+Future<bool> GarbageCollectorProcess::schedule(
+    const seconds& s,
+    const string& path)
+{
+  LOG(INFO) << "Scheduling " << path << " for removal";
+
+  Promise<bool>* promise = new Promise<bool>();
+
+  delay(s.value, self(), &Self::remove, path, promise);
+
+  return promise->future();
+}
+
+
+void GarbageCollectorProcess::remove(
+    const string& path,
+    Promise<bool>* promise)
+{
+  LOG(INFO) << "Removing " << path;
+
+  // TODO(benh): Check error conditions of 'rmdir', e.g., permission
+  // denied, file no longer exists, etc.
+  bool result = os::rmdir(path);
+
+  promise->set(result);
+  delete promise;
+}
+
+
+GarbageCollector::GarbageCollector()
+{
+  process = new GarbageCollectorProcess();
+  spawn(process);
+}
+
+
+GarbageCollector::~GarbageCollector()
+{
+  terminate(process);
+  wait(process);
+}
+
+
+Future<bool> GarbageCollector::schedule(
+    const seconds& s,
+    const string& path)
+{
+  return dispatch(process, &GarbageCollectorProcess::schedule, s, path);
+}
+
+} // namespace mesos {
+} // namespace internal {
+} // namespace slave {

Added: incubator/mesos/trunk/src/slave/gc.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/gc.hpp?rev=1371830&view=auto
==============================================================================
--- incubator/mesos/trunk/src/slave/gc.hpp (added)
+++ incubator/mesos/trunk/src/slave/gc.hpp Fri Aug 10 22:00:19 2012
@@ -0,0 +1,62 @@
+/**
+ * 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 __SLAVE_GC_HPP__
+#define __SLAVE_GC_HPP__
+
+#include <string>
+
+#include <process/future.hpp>
+
+#include <stout/time.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declarations.
+class GarbageCollectorProcess;
+
+// Provides an abstraction for removing files and directories after
+// some point at which they are no longer considered necessary to keep
+// around. The intent with this abstraction is to also easily enable
+// implementations that may actually copy files and directories to
+// "more" permanent storage (or provide any other hooks that might be
+// useful, e.g., emailing users some time before their files are
+// scheduled for removal).
+class GarbageCollector
+{
+public:
+  GarbageCollector();
+  ~GarbageCollector();
+
+  // Schedules the specified path for removal after the specified
+  // number of seconds have 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);
+
+private:
+  GarbageCollectorProcess* process;
+};
+
+} // namespace mesos {
+} // namespace internal {
+} // namespace slave {
+
+#endif // __SLAVE_GC_HPP__

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=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp Fri Aug 10 22:00:19 2012
@@ -161,7 +161,7 @@ void LxcIsolationModule::launchExecutor(
     // Close unnecessary file descriptors. Note that we are assuming
     // stdin, stdout, and stderr can ONLY be found at the POSIX
     // specified file numbers (0, 1, 2).
-    foreach (const string& entry, os::listdir("/proc/self/fd")) {
+    foreach (const string& entry, os::ls("/proc/self/fd")) {
       if (entry != "." && entry != "..") {
         try {
           int fd = boost::lexical_cast<int>(entry);

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Fri Aug 10 22:00:19 2012
@@ -364,7 +364,33 @@ void Slave::registered(const SlaveID& sl
 
   connected = true;
 
-  garbageCollectSlaveDirs(path::join(flags.work_dir, "slaves"));
+  // 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);
+
+  const string& directory = path::join(flags.work_dir, "slaves");
+
+  foreach (const string& file, os::ls(directory)) {
+    const string& path = path::join(directory, file);
+
+    // Check that this path is a directory but not our directory!
+    if (os::exists(path, true) && file != id.value()) {
+
+      Try<long> time = os::mtime(path);
+
+      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()));
+        gc.schedule(delta, path);
+      } else {
+        LOG(WARNING) << "Failed to get the modification time of "
+                     << path << ": " << time.error();
+        gc.schedule(timeout, path);
+      }
+    }
+  }
 }
 
 
@@ -1411,7 +1437,9 @@ void Slave::executorExited(const Framewo
     send(master, message);
   }
 
-  garbageCollectExecutorDir(executor->directory);
+  // Schedule the executor directory to get garbage collected.
+  gc.schedule(hours(flags.gc_timeout_hours), executor->directory);
+
   framework->destroyExecutor(executor->id);
 }
 
@@ -1454,7 +1482,9 @@ void Slave::shutdownExecutorTimeout(cons
              &IsolationModule::killExecutor,
              framework->id, executor->id);
 
-    garbageCollectExecutorDir(executor->directory);
+    // Schedule the executor directory to get garbage collected.
+    gc.schedule(hours(flags.gc_timeout_hours), executor->directory);
+
     framework->destroyExecutor(executor->id);
   }
 
@@ -1466,48 +1496,6 @@ void Slave::shutdownExecutorTimeout(cons
 }
 
 
-void Slave::garbageCollectExecutorDir(const string& dir)
-{
-  hours timeout(flags.gc_timeout_hours);
-  std::list<string> result;
-
-  LOG(INFO) << "Scheduling executor directory " << dir << " for deletion";
-  result.push_back(dir);
-
-  delay(timeout.secs(), self(), &Slave::garbageCollect, result);
-}
-
-
-void Slave::garbageCollectSlaveDirs(const string& dir)
-{
-  hours timeout(flags.gc_timeout_hours);
-
-  std::list<string> result;
-
-  foreach (const string& d, os::listdir(dir)) {
-    if (d != "." && d != ".." && d != id.value()) {
-      const string& path = dir + "/" + d;
-      Try<long> modtime = os::modtime(path);
-      if (os::exists(path, true) && // Check if its a directory.
-        modtime.isSome() && (Clock::now() - modtime.get()) > timeout.secs()) {
-        LOG(INFO) << "Scheduling slave directory " << path << " for deletion";
-        result.push_back(path);
-      }
-    }
-  }
-  garbageCollect(result); // Delete these right away.
-}
-
-
-void Slave::garbageCollect(const std::list<string>& directories)
-{
-  foreach (const string& dir, directories) {
-    LOG(INFO) << "Deleting directory " << dir;
-    os::rmdir(dir);
-  }
-}
-
-
 // void Slave::recover()
 // {
 //   // if we find an executor that is no longer running and it's last

Modified: incubator/mesos/trunk/src/slave/slave.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.hpp?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.hpp (original)
+++ incubator/mesos/trunk/src/slave/slave.hpp Fri Aug 10 22:00:19 2012
@@ -30,6 +30,7 @@
 
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
+#include "slave/gc.hpp"
 #include "slave/http.hpp"
 #include "slave/isolation_module.hpp"
 
@@ -142,16 +143,6 @@ protected:
                                const ExecutorID& executorId,
                                const UUID& uuid);
 
-  // Schedules the executor directory for deletion after a timeout.
-  void garbageCollectExecutorDir(const std::string& directory);
-
-  // Schedules old slave directories under the given directory root
-  // for deletion after a timeout.
-  void garbageCollectSlaveDirs(const std::string& directory);
-
-  // Actually deletes the directories.
-  void garbageCollect(const std::list<std::string>& directories);
-
 //   // Create a new status update stream.
 //   StatusUpdates* createStatusUpdateStream(const StatusUpdateStreamID& streamId,
 //                                           const string& directory);
@@ -210,6 +201,7 @@ private:
   bool connected; // Flag to indicate if slave is registered.
 
   Files files;
+  GarbageCollector gc;
 };
 
 

Copied: incubator/mesos/trunk/src/tests/gc_tests.cpp (from r1371362, incubator/mesos/trunk/src/tests/slave_tests.cpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/gc_tests.cpp?p2=incubator/mesos/trunk/src/tests/gc_tests.cpp&p1=incubator/mesos/trunk/src/tests/slave_tests.cpp&r1=1371362&r2=1371830&rev=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/slave_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/gc_tests.cpp Fri Aug 10 22:00:19 2012
@@ -66,7 +66,7 @@ using testing::Eq;
 using testing::Return;
 using testing::SaveArg;
 
-class SlaveTest : public ::testing::Test
+class GarbageCollectorTest : public ::testing::Test
 {
 protected:
   static void SetUpTestCase()
@@ -176,6 +176,7 @@ protected:
   MockScheduler sched;
   MesosSchedulerDriver* driver;
   trigger resourceOffersCall;
+  SlaveRegisteredMessage registeredMsg;
   vector<Offer> offers;
   TaskStatus status;
   vector<TaskInfo> tasks;
@@ -187,18 +188,19 @@ protected:
 
 
 // Initialize static members here.
-flags::Flags<logging::Flags, slave::Flags> SlaveTest::flags;
+flags::Flags<logging::Flags, slave::Flags> GarbageCollectorTest::flags;
 
 
-TEST_F(SlaveTest, GarbageCollectSlaveDirs)
+TEST_F(GarbageCollectorTest, Restart)
 {
   process::Message message;
-  trigger slaveRegisteredMsg;
+  trigger slaveRegisteredMsg1, slaveRegisteredMsg2;
   EXPECT_MESSAGE(filter, Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
-    .WillRepeatedly(DoAll(
+    .WillOnce(DoAll(
         SaveArgField<0>(&process::MessageEvent::message, &message),
-        Trigger(&slaveRegisteredMsg),
-        Return(false)));
+        Trigger(&slaveRegisteredMsg1),
+        Return(false)))
+    .WillOnce(DoAll(Trigger(&slaveRegisteredMsg2), Return(false)));
 
   trigger lostSlaveMsg;
   EXPECT_MESSAGE(filter, Eq(LostSlaveMessage().GetTypeName()), _, _)
@@ -210,67 +212,60 @@ TEST_F(SlaveTest, GarbageCollectSlaveDir
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillRepeatedly(DoAll(SaveArg<1>(&offers), Trigger(&resourceOffersCall)));
 
-  trigger statusUpdateCall, statusUpdateCall2;
+  trigger statusUpdateCall;
   EXPECT_CALL(sched, statusUpdate(_, _))
     .WillOnce(DoAll(SaveArg<1>(&status), Trigger(&statusUpdateCall)))
-    .WillOnce(Return()) // Ignore the TASK_LOST update.
-    .WillOnce(Trigger(&statusUpdateCall2));
+    .WillRepeatedly(Return()); // Ignore remaining updates (e.g., TASK_LOST).
 
   EXPECT_CALL(sched, slaveLost(_, _))
     .Times(1);
 
-  // Start the slave.
   startSlave();
 
+  WAIT_UNTIL(slaveRegisteredMsg1);
+
+  // Capture the slave id.
+  registeredMsg.ParseFromString(message.body);
+  SlaveID slaveId = registeredMsg.slave_id();
+
   driver->start();
 
   WAIT_UNTIL(resourceOffersCall);
+
   launchTask();
 
   WAIT_UNTIL(statusUpdateCall);
 
   EXPECT_EQ(TASK_RUNNING, status.state());
 
-  SlaveRegisteredMessage registeredMsg;
-
-  WAIT_UNTIL(slaveRegisteredMsg); // Capture the slave id.
-  registeredMsg.ParseFromString(message.body);
-  SlaveID slaveId = registeredMsg.slave_id();
-
-  // Make sure directory exists.
+  // Make sure directory exists. Need to do this AFTER getting a
+  // status update for a task because the directory won't get created
+  // until the SlaveRegisteredMessage has been received.
   const std::string& slaveDir = flags.work_dir + "/slaves/" + slaveId.value();
   ASSERT_TRUE(os::exists(slaveDir));
 
   Clock::pause();
 
-  // Advance the clock and restart the slave.
   stopSlave();
 
-  hours timeout(slave::GC_TIMEOUT_HOURS);
-  Clock::advance(timeout.secs());
-
   WAIT_UNTIL(lostSlaveMsg);
 
-  // Reset the triggers.
-  slaveRegisteredMsg.value = false;
-  resourceOffersCall.value = false;
-
   startSlave();
 
-  WAIT_UNTIL(slaveRegisteredMsg); // Capture the new slave id.
-  registeredMsg.ParseFromString(message.body);
-  SlaveID slaveId2 = registeredMsg.slave_id();
+  // In order to make sure the slave has scheduled some directories to
+  // get garbaged collected we need to wait until the slave has been
+  // registered. TODO(benh): We really need to wait until the
+  // GarbageCollectorProcess has dispatched a message back to itself.
+  WAIT_UNTIL(slaveRegisteredMsg2);
 
-  WAIT_UNTIL(resourceOffersCall);
-  launchTask();
+  sleep(1);
+
+  Clock::advance(hours(flags.gc_timeout_hours).secs());
 
-  WAIT_UNTIL(statusUpdateCall2);
+  Clock::settle();
 
-  // By this time the old slave directory should be cleaned up and
-  // the new directory should exist.
+  // By this time the old slave directory should be cleaned up.
   ASSERT_FALSE(os::exists(slaveDir));
-  ASSERT_TRUE(os::exists(flags.work_dir + "/slaves/" +
-                                slaveId2.value()));
 
   Clock::resume();
 
@@ -279,10 +274,11 @@ TEST_F(SlaveTest, GarbageCollectSlaveDir
 }
 
 
-TEST_F(SlaveTest, GarbageCollectExecutorDir)
+TEST_F(GarbageCollectorTest, ExitedExecutor)
 {
-  trigger runTaskMsg;
-  process::Message message;
+  trigger exitedExecutorMsg;
+  EXPECT_MESSAGE(filter, Eq(ExitedExecutorMessage().GetTypeName()), _, _)
+    .WillOnce(DoAll(Trigger(&exitedExecutorMsg), Return(false)));
 
   FrameworkID frameworkId;
   EXPECT_CALL(sched, registered(_, _, _))
@@ -291,60 +287,51 @@ TEST_F(SlaveTest, GarbageCollectExecutor
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillRepeatedly(DoAll(SaveArg<1>(&offers), Trigger(&resourceOffersCall)));
 
-  trigger statusUpdateCall, statusUpdateCall2;
+  trigger statusUpdateCall;
   EXPECT_CALL(sched, statusUpdate(_, _))
     .WillOnce(DoAll(SaveArg<1>(&status), Trigger(&statusUpdateCall)))
     .WillOnce(Return()) // Ignore the TASK_LOST update.
     .WillRepeatedly(Trigger(&statusUpdateCall));
 
-  // Start the slave.
   startSlave();
 
   driver->start();
 
   WAIT_UNTIL(resourceOffersCall);
+
   launchTask();
 
   WAIT_UNTIL(statusUpdateCall);
 
   EXPECT_EQ(TASK_RUNNING, status.state());
 
-  const std::string executorDir =
-      isolationModule->directories[DEFAULT_EXECUTOR_ID];
+  const std::string& executorDir =
+    isolationModule->directories[DEFAULT_EXECUTOR_ID];
 
   ASSERT_TRUE(os::exists(executorDir));
 
-  statusUpdateCall.value = false;
-  resourceOffersCall.value = false;
+  Clock::pause();
 
   // Kill the executor and inform the slave.
   isolationModule->killExecutor(frameworkId, DEFAULT_EXECUTOR_ID);
+
   process::dispatch(slave, &Slave::executorExited, frameworkId,
                     DEFAULT_EXECUTOR_ID, 0);
 
-  statusUpdateCall.value = false;
-  WAIT_UNTIL(resourceOffersCall);
-  launchTask();
-  WAIT_UNTIL(statusUpdateCall); // TASK_RUNNING
+  // In order to make sure the slave has scheduled the executor
+  // directory to get garbage collected we need to wait until the
+  // slave has sent the ExecutorExited message. TODO(benh): We really
+  // need to wait until the GarbageCollectorProcess has dispatched a
+  // message back to itself.
+  WAIT_UNTIL(exitedExecutorMsg);
 
-  Clock::pause();
+  sleep(1);
 
-  hours timeout(slave::GC_TIMEOUT_HOURS);
-  Clock::advance(timeout.secs());
+  Clock::advance(hours(flags.gc_timeout_hours).secs());
 
-  // Kill the new executor and inform the slave.
-  // We do this to make sure we can capture an event (TASK_LOST) that tells us
-  // that the slave has processed its previous message (garbageCollect).
-  isolationModule->killExecutor(frameworkId, DEFAULT_EXECUTOR_ID);
-  process::dispatch(slave, &Slave::executorExited, frameworkId,
-                    DEFAULT_EXECUTOR_ID, 0);
-
-  statusUpdateCall.value = false;
-  WAIT_UNTIL(resourceOffersCall);
-  launchTask();
-  WAIT_UNTIL(statusUpdateCall); // TASK_LOST
+  Clock::settle();
 
-  // First executor's directory should be gced by now.
+  // First executor's directory should be gc'ed by now.
   ASSERT_FALSE(os::exists(executorDir));
 
   Clock::resume();

Modified: incubator/mesos/trunk/src/tests/stout_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/stout_tests.cpp?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/stout_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/stout_tests.cpp Fri Aug 10 22:00:19 2012
@@ -172,7 +172,7 @@ TEST(StoutStringsTest, Contains)
 static hashset<std::string> listfiles(const std::string& dir)
 {
   hashset<std::string> fileset;
-  foreach (const std::string& file, os::listdir(dir)) {
+  foreach (const std::string& file, os::ls(dir)) {
     fileset.insert(file);
   }
   return fileset;
@@ -200,41 +200,39 @@ protected:
 TEST_F(StoutUtilsTest, rmdir)
 {
   // TODO(John Sirois): It would be good to use something like mkdtemp, but
-  //abstract away a proper platform independent /tmp dir.
+  // abstract away a proper platform independent /tmp dir.
 
-  hashset<std::string> emptyListing;
-  emptyListing.insert(".");
-  emptyListing.insert("..");
+  const hashset<std::string> EMPTY;
 
-  hashset<std::string> expectedListing = emptyListing;
+  hashset<std::string> expectedListing = EMPTY;
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
 
   os::mkdir(tmpdir + "/a/b/c");
   os::mkdir(tmpdir + "/a/b/d");
   os::mkdir(tmpdir + "/e/f");
 
-  expectedListing = emptyListing;
+  expectedListing = EMPTY;
   expectedListing.insert("a");
   expectedListing.insert("e");
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
 
-  expectedListing = emptyListing;
+  expectedListing = EMPTY;
   expectedListing.insert("b");
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/a"));
 
-  expectedListing = emptyListing;
+  expectedListing = EMPTY;
   expectedListing.insert("c");
   expectedListing.insert("d");
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/a/b"));
 
-  expectedListing = emptyListing;
+  expectedListing = EMPTY;
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/a/b/c"));
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/a/b/d"));
 
   expectedListing.insert("f");
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/e"));
 
-  expectedListing = emptyListing;
+  expectedListing = EMPTY;
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/e/f"));
 }
 

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp?rev=1371830&r1=1371829&r2=1371830&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp Fri Aug 10 22:00:19 2012
@@ -12,6 +12,7 @@
 #include <stddef.h>
 #include <stdio.h>
 #include <stdlib.h>
+#include <string.h>
 #include <unistd.h>
 
 #include <glog/logging.h>
@@ -315,12 +316,13 @@ inline bool exists(const std::string& pa
 }
 
 
-inline Try<long> modtime(const std::string& path)
+// TODO(benh): Put this in the 'paths' or 'files' or 'fs' namespace.
+inline Try<long> mtime(const std::string& path)
 {
   struct stat s;
 
   if (::stat(path.c_str(), &s) < 0) {
-    return Try<long>::error("Cannot stat " + path + " for modification time");
+    return Try<long>::error(strerror(errno));
   }
 
   return s.st_mtime;
@@ -480,7 +482,7 @@ inline std::string getcwd()
 }
 
 
-inline std::list<std::string> listdir(const std::string& directory)
+inline std::list<std::string> ls(const std::string& directory)
 {
   std::list<std::string> result;
 
@@ -519,6 +521,9 @@ inline std::list<std::string> listdir(co
   int error;
 
   while ((error = readdir_r(dir, temp, &entry)) == 0 && entry != NULL) {
+    if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0) {
+      continue;
+    }
     result.push_back(entry->d_name);
   }
 
@@ -548,7 +553,7 @@ inline Try<std::list<std::string> > find
     return Try<std::list<std::string> >::error("Directory " + directory + " doesn't exist!");
   }
 
-  foreach (const std::string& entry, listdir(directory)) {
+  foreach (const std::string& entry, ls(directory)) {
     if (entry == "." || entry == "..") {
       continue;
     }