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 17:48:22 UTC

svn commit: r1382948 [1/2] - in /incubator/mesos/trunk: src/ src/common/ src/configurator/ src/files/ src/launcher/ src/linux/ src/local/ src/logging/ src/master/ src/slave/ src/tests/ src/webui/master/static/ third_party/libprocess/ third_party/libpro...

Author: benh
Date: Mon Sep 10 15:48:20 2012
New Revision: 1382948

URL: http://svn.apache.org/viewvc?rev=1382948&view=rev
Log:
Added a process::read to EOF; added a process::http::get; updated of
strings::tokenize and strings::split; created a new HTTP response
decoder via http_parser; updated process::http::Request to include
"query" map; added hashmap::get to return an Option; added JSONP
support to OK; implemented "browse" functionality for Files; injected
and more properly integrated Files (and added Files tests); added some
ASSERT/EXPECT_FUTURE_* functions (contributed by Ben Mahler,
https://reviews.apache.org/r/6617).

Removed:
    incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp
    incubator/mesos/trunk/third_party/libprocess/src/tokenize.cpp
    incubator/mesos/trunk/third_party/libprocess/src/tokenize.hpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/common/attributes.cpp
    incubator/mesos/trunk/src/common/resources.cpp
    incubator/mesos/trunk/src/common/values.cpp
    incubator/mesos/trunk/src/configurator/configuration.hpp
    incubator/mesos/trunk/src/configurator/configurator.cpp
    incubator/mesos/trunk/src/files/files.cpp
    incubator/mesos/trunk/src/files/files.hpp
    incubator/mesos/trunk/src/launcher/main.cpp
    incubator/mesos/trunk/src/linux/cgroups.cpp
    incubator/mesos/trunk/src/local/local.cpp
    incubator/mesos/trunk/src/logging/logging.cpp
    incubator/mesos/trunk/src/master/http.cpp
    incubator/mesos/trunk/src/master/main.cpp
    incubator/mesos/trunk/src/master/master.cpp
    incubator/mesos/trunk/src/master/master.hpp
    incubator/mesos/trunk/src/master/slaves_manager.cpp
    incubator/mesos/trunk/src/slave/http.cpp
    incubator/mesos/trunk/src/slave/main.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/tests/allocator_tests.cpp
    incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
    incubator/mesos/trunk/src/tests/configurator_tests.cpp
    incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
    incubator/mesos/trunk/src/tests/gc_tests.cpp
    incubator/mesos/trunk/src/tests/master_detector_tests.cpp
    incubator/mesos/trunk/src/tests/master_tests.cpp
    incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
    incubator/mesos/trunk/src/tests/stout_tests.cpp
    incubator/mesos/trunk/src/tests/utils.hpp
    incubator/mesos/trunk/src/webui/master/static/controllers.js
    incubator/mesos/trunk/third_party/libprocess/Makefile.am
    incubator/mesos/trunk/third_party/libprocess/include/process/http.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/io.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/pid.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/hashmap.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/json.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/numify.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/path.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp
    incubator/mesos/trunk/third_party/libprocess/src/decoder.hpp
    incubator/mesos/trunk/third_party/libprocess/src/encoder.hpp
    incubator/mesos/trunk/third_party/libprocess/src/process.cpp
    incubator/mesos/trunk/third_party/libprocess/src/statistics.cpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Mon Sep 10 15:48:20 2012
@@ -768,6 +768,7 @@ mesos_tests_SOURCES = tests/main.cpp tes
 	              tests/gc_tests.cpp				\
 	              tests/resource_offers_tests.cpp			\
 	              tests/fault_tolerance_tests.cpp			\
+	              tests/files_tests.cpp				\
 	              tests/flags_tests.cpp tests/log_tests.cpp		\
 	              tests/resources_tests.cpp				\
 	              tests/external_tests.cpp				\

Modified: incubator/mesos/trunk/src/common/attributes.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/attributes.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/attributes.cpp (original)
+++ incubator/mesos/trunk/src/common/attributes.cpp Mon Sep 10 15:48:20 2012
@@ -72,10 +72,10 @@ Attributes Attributes::parse(const strin
   // Tokenize and parse the value of "attributes".
   Attributes attributes;
 
-  vector<string> tokens = strings::split(s, ";\n");
+  vector<string> tokens = strings::tokenize(s, ";\n");
 
   for (size_t i = 0; i < tokens.size(); i++) {
-    const vector<string>& pairs = strings::split(tokens[i], ":");
+    const vector<string>& pairs = strings::tokenize(tokens[i], ":");
     if (pairs.size() != 2) {
       LOG(FATAL) << "Bad value for attributes, missing ':' within " << pairs[0];
     }

Modified: incubator/mesos/trunk/src/common/resources.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/resources.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/resources.cpp (original)
+++ incubator/mesos/trunk/src/common/resources.cpp Mon Sep 10 15:48:20 2012
@@ -214,10 +214,10 @@ Resources Resources::parse(const string&
   // Tokenize and parse the value of "resources".
   Resources resources;
 
-  vector<string> tokens = strings::split(s, ";\n");
+  vector<string> tokens = strings::tokenize(s, ";\n");
 
   for (size_t i = 0; i < tokens.size(); i++) {
-    const vector<string>& pairs = strings::split(tokens[i], ":");
+    const vector<string>& pairs = strings::tokenize(tokens[i], ":");
     if (pairs.size() != 2) {
       LOG(FATAL) << "Bad value for resources, missing ':' within " << pairs[0];
     }

Modified: incubator/mesos/trunk/src/common/values.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/values.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/values.cpp (original)
+++ incubator/mesos/trunk/src/common/values.cpp Mon Sep 10 15:48:20 2012
@@ -68,7 +68,7 @@ Try<Value> parse(const std::string& text
   if (index == 0) {
     // This is a ranges.
     Value::Ranges ranges;
-    const vector<string>& tokens = strings::split(temp, "[]-,\n");
+    const vector<string>& tokens = strings::tokenize(temp, "[]-,\n");
     if (tokens.size() % 2 != 0) {
       return Try<Value>::error("Error parsing value: " + text +
                                ", expect one or more \"ranges \"");
@@ -96,7 +96,7 @@ Try<Value> parse(const std::string& text
     if (index == 0) {
       // This is a set.
       Value::Set set;
-      const vector<string>& tokens = strings::split(temp, "{},\n");
+      const vector<string>& tokens = strings::tokenize(temp, "{},\n");
       for (size_t i = 0; i < tokens.size(); i++) {
         set.add_item(tokens[i]);
       }

Modified: incubator/mesos/trunk/src/configurator/configuration.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/configurator/configuration.hpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/configurator/configuration.hpp (original)
+++ incubator/mesos/trunk/src/configurator/configuration.hpp Mon Sep 10 15:48:20 2012
@@ -80,9 +80,9 @@ public:
    */
   void loadString(const std::string& str)
   {
-    std::vector<std::string> lines = strings::split(str, "\n\r");
+    std::vector<std::string> lines = strings::tokenize(str, "\n\r");
     foreach (std::string& line, lines) {
-      std::vector<std::string> parts = strings::split(line, "=");
+      std::vector<std::string> parts = strings::tokenize(line, "=");
       if (parts.size() != 2) {
         const Try<std::string>& error =
           strings::format("Failed to parse '%s'", line);

Modified: incubator/mesos/trunk/src/configurator/configurator.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/configurator/configurator.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/configurator/configurator.cpp (original)
+++ incubator/mesos/trunk/src/configurator/configurator.cpp Mon Sep 10 15:48:20 2012
@@ -255,7 +255,7 @@ void Configurator::loadConfigFile(const 
       continue;
     }
     // Split line by = and trim to get key and value
-    vector<string> tokens = strings::split(line, "=");
+    vector<string> tokens = strings::tokenize(line, "=");
     if (tokens.size() != 2) {
       string message = "Malformed line in config file: '" +
                        strings::trim(originalLine) + "'";

Modified: incubator/mesos/trunk/src/files/files.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/files/files.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/files/files.cpp (original)
+++ incubator/mesos/trunk/src/files/files.cpp Mon Sep 10 15:48:20 2012
@@ -1,3 +1,5 @@
+#include <sys/stat.h>
+
 #include <map>
 #include <string>
 #include <vector>
@@ -12,6 +14,8 @@
 #include <stout/numify.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/result.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
@@ -30,7 +34,6 @@ using process::http::NotFound;
 using process::http::OK;
 using process::http::Response;
 using process::http::Request;
-using process::http::ServiceUnavailable;
 
 using std::map;
 using std::string;
@@ -43,10 +46,9 @@ class FilesProcess : public Process<File
 {
 public:
   FilesProcess();
-  virtual ~FilesProcess();
 
   // Files implementation.
-  Future<bool> attach(const string& path, const string& name);
+  Future<Nothing> attach(const string& path, const string& name);
   void detach(const string& name);
 
 protected:
@@ -56,6 +58,14 @@ private:
   // HTTP endpoints.
   Future<Response> browse(const Request& request);
   Future<Response> read(const Request& request);
+  Future<Response> debug(const Request& request);
+
+  // Resolves the virtual path to an actual path.
+  // Returns the actual path if found.
+  // Returns None if the file is not found.
+  // Returns Error if we find the file but it cannot be resolved or it breaks
+  // out of the chroot.
+  Result<std::string> resolve(const string& path);
 
   hashmap<string, string> paths;
 };
@@ -66,38 +76,41 @@ FilesProcess::FilesProcess()
 {}
 
 
-FilesProcess::~FilesProcess()
-{}
-
-
 void FilesProcess::initialize()
 {
   route("/browse.json", &FilesProcess::browse);
   route("/read.json", &FilesProcess::read);
+  route("/debug.json", &FilesProcess::debug);
 }
 
 
-Future<bool> FilesProcess::attach(const string& path, const string& name)
+Future<Nothing> FilesProcess::attach(const string& path, const string& name)
 {
   Try<string> result = os::realpath(path);
 
   if (result.isError()) {
-    return Future<bool>::failed(result.error());
+    LOG(ERROR) << "Error attaching path '" << path << "': " << result.error();
+    return Future<Nothing>::failed(result.error());
   }
 
-  // Make sure we have permissions to "touch" the file (TODO(benh): We
-  // really only need permissions to read the file).
-  Try<bool> touched = os::touch(result.get());
+  // Make sure we have permissions to read the file/dir.
+  Try<bool> access = os::access(result.get(), R_OK);
 
-  if (touched.isError()) {
-    return Future<bool>::failed(touched.error());
+  if (access.isError()) {
+    LOG(WARNING) << "Error attaching path '" << path << "': " << access.error();
+    return Future<Nothing>::failed(access.error());
+  } else if (!access.get()) {
+    LOG(WARNING) << "Do not have read permission to attach path: " << path;
+    return Future<Nothing>::failed(access.error());
   }
 
-  if (touched.get()) {
-    paths[name] = result.get();
-  }
+  // To simplify the read/browse logic, strip any trailing / from the name.
+  string cleanedName = strings::remove(name, "/", strings::SUFFIX);
+
+  // TODO(bmahler): Do we want to always wipe out the previous path?
+  paths[cleanedName] = result.get();
 
-  return touched.get();
+  return Nothing();
 }
 
 
@@ -109,182 +122,229 @@ void FilesProcess::detach(const string& 
 
 Future<Response> FilesProcess::browse(const Request& request)
 {
-  return ServiceUnavailable();
-}
+  Option<string> path = request.query.get("path");
 
+  if (!path.isSome() || path.get().empty()) {
+    return BadRequest("Expecting 'path=value' in query.\n");
+  }
 
-Future<Response> FilesProcess::read(const Request& request)
-{
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ";&", "=");
+  Result<string> resolvedPath = resolve(path.get());
+
+  if (resolvedPath.isError()) {
+    return InternalServerError(resolvedPath.error());
+  } else if (resolvedPath.isNone()) {
+    return NotFound();
+  }
 
-  Option<string> name;
+  // The result will be a sorted (on path) array of files and dirs:
+  // [{"name": "README", "path": "dir/README" "dir":False, "size":42}, ...]
+  map<string, JSON::Object> files;
+  foreach(const string& filename, os::ls(resolvedPath.get())) {
+    struct stat s;
+    string fullPath = path::join(resolvedPath.get(), filename);
+
+    if (stat(fullPath.c_str(), &s) < 0) {
+      PLOG(WARNING) << "Found " << fullPath << " in ls but stat failed";
+      continue;
+    }
 
-  if (pairs.count("name") > 0 && pairs["name"].size() > 0) {
-    name = pairs["name"].back();
+    files[fullPath] = jsonFileInfo(path::join(path.get(), filename),
+                                   S_ISDIR(s.st_mode),
+                                   S_ISDIR(s.st_mode) ? 0 : s.st_size);
   }
 
-  if (!name.isSome()) {
-    return BadRequest();
+  JSON::Array listing;
+  foreachvalue(const JSON::Object& file, files) {
+    listing.values.push_back(file);
+  }
+
+  return OK(listing, request.query.get("jsonp"));
+}
+
+
+Future<Response> FilesProcess::read(const Request& request)
+{
+  Option<string> path = request.query.get("path");
+
+  if (!path.isSome() || path.get().empty()) {
+    return BadRequest("Expecting 'path=value' in query.\n");
   }
 
   off_t offset = -1;
 
-  if (pairs.count("offset") > 0 && pairs["offset"].size() > 0) {
-    Try<off_t> result = numify<off_t>(pairs["offset"].back());
+  if (request.query.get("offset").isSome()) {
+    Try<off_t> result = numify<off_t>(request.query.get("offset").get());
     if (result.isError()) {
-      LOG(WARNING) << "Failed to \"numify\" the 'offset' ("
-                   << pairs["offset"].back() << "): "
-                   << result.error();
-      return InternalServerError();
+      return BadRequest("Failed to parse offset: " + result.error());
     }
     offset = result.get();
   }
 
   ssize_t length = -1;
 
-  if (pairs.count("length") > 0) {
-    CHECK(pairs["length"].size() > 0);
-    Try<ssize_t> result = numify<ssize_t>(pairs["length"].back());
+  if (request.query.get("length").isSome()) {
+    Try<ssize_t> result = numify<ssize_t>(request.query.get("length").get());
     if (result.isError()) {
-      LOG(WARNING) << "Failed to \"numify\" the 'length' ("
-                   << pairs["length"].back() << "): "
-                   << result.error();
-      return InternalServerError();
+      return BadRequest("Failed to parse length: " + result.error());
     }
     length = result.get();
   }
 
-  Option<string> jsonp;
+  Result<string> resolvedPath = resolve(path.get());
 
-  if (pairs.count("jsonp") > 0 && pairs["jsonp"].size() > 0) {
-    jsonp = pairs["jsonp"].back();
+  if (resolvedPath.isError()) {
+    return BadRequest(resolvedPath.error());
+  } else if (!resolvedPath.isSome()) {
+    return NotFound();
   }
 
-  // Now try and see if this name has been attached. We check for the
-  // longest possible prefix match and if found append any suffix to
-  // the attached path (provided the path is to a directory).
-  string suffix;
+  // Don't read directories.
+  if (os::isdir(resolvedPath.get())) {
+    return BadRequest("Cannot read a directory.");
+  }
 
-  foreach (const string& s, strings::split(name.get(), "/")) {
-    string prefix = name.get().substr(0, name.get().length() - suffix.length());
+  // TODO(benh): Cache file descriptors so we aren't constantly
+  // opening them and paging the data in from disk.
+  Try<int> fd = os::open(resolvedPath.get(), O_RDONLY);
+
+  if (fd.isError()) {
+    string error = strings::format("Failed to open file at '%s': %s",
+        resolvedPath.get(), fd.error()).get();
+    LOG(WARNING) << error;
+    return InternalServerError(error);
+  }
 
-    if (!paths.contains(prefix)) {
-      suffix = s + "/" + suffix;
-    } else {
-      // Determine the final path: if it's a directory, append the
-      // suffix, if it's not a directory and there is a suffix, return
-      // '404 Not Found'.
-      string path = paths[prefix];
-      if (os::exists(path, true)) {
-        path += "/" + suffix;
-
-        // Canonicalize the absolute path and make sure the result
-        // doesn't break out of the chroot (i.e., resolving any '..'
-        // in the suffix should yield a resulting path that still
-        // contains the attached path as it's prefix).
-        Try<string> result = os::realpath(path);
-        if (result.isError()) {
-          return NotFound();
-        } else if (result.get().find(paths[prefix]) != 0) {
-          return NotFound();
-        }
-
-        path = result.get();
-      } else if (suffix != "") {
-        // Request is assuming attached path is a directory, but it is
-        // not! Rather than 'Bad Request', treat this as 'Not Found'.
-        return NotFound();
-      }
+  off_t size = lseek(fd.get(), 0, SEEK_END);
 
-      // TODO(benh): Cache file descriptors so we aren't constantly
-      // opening them and paging the data in from disk.
-      Try<int> fd = os::open(path, O_RDONLY);
-
-      if (fd.isError()) {
-        LOG(WARNING) << "Failed to open file at "
-                     << path << ": " << fd.error();
-        return InternalServerError();
-      }
+  if (size == -1) {
+    string error = strings::format("Failed to open file at '%s': %s",
+        resolvedPath.get(), strerror(errno)).get();
+    LOG(WARNING) << error;
+    close(fd.get());
+    return InternalServerError(error);
+  }
 
-      off_t size = lseek(fd.get(), 0, SEEK_END);
+  if (offset == -1) {
+    offset = size;
+  }
 
-      if (size == -1) {
-        PLOG(WARNING) << "Failed to seek in the file at " << path;
-        close(fd.get());
-        return InternalServerError();
-      }
+  if (length == -1) {
+    length = size - offset;
+  }
 
-      if (offset == -1) {
-        offset = size;
-      }
+  JSON::Object object;
 
-      if (length == -1) {
-        length = size - offset;
-      }
+  if (offset < size) {
+    // Seek to the offset we want to read from.
+    if (lseek(fd.get(), offset, SEEK_SET) == -1) {
+      string error = strings::format("Failed to seek file at '%s': %s",
+          resolvedPath.get(), strerror(errno)).get();
+      LOG(WARNING) << error;
+      close(fd.get());
+      return InternalServerError(error);
+    }
 
-      JSON::Object object;
+    // Read length bytes (or to EOF).
+    char* temp = new char[length];
 
-      if (offset < size) {
-        // Seek to the offset we want to read from.
-        if (lseek(fd.get(), offset, SEEK_SET) == -1) {
-          PLOG(WARNING) << "Failed to seek in the file at " << path;
-          close(fd.get());
-          return InternalServerError();
-        }
-
-        // Read length bytes (or to EOF).
-        char* temp = new char[length];
-
-        length = ::read(fd.get(), temp, length);
-
-        if (length == 0) {
-          object.values["offset"] = offset;
-          object.values["length"] = 0;
-          delete[] temp;
-        } else if (length == -1) {
-          PLOG(WARNING) << "Failed to read from the file at " << path;
-          delete[] temp;
-          close(fd.get());
-          return InternalServerError();
-        } else {
-          object.values["offset"] = offset;
-          object.values["length"] = length;
-          object.values["data"] = string(temp, length);
-          delete[] temp;
-        }
-      } else {
-        object.values["offset"] = size;
-        object.values["length"] = 0;
-      }
+    // TODO(bmahler): Change this to use async process::read.
+    length = ::read(fd.get(), temp, length);
 
+    if (length == 0) {
+      object.values["offset"] = offset;
+      object.values["length"] = 0;
+      delete[] temp;
+    } else if (length == -1) {
+      string error = strings::format("Failed to read file at '%s': %s",
+          resolvedPath.get(), strerror(errno)).get();
+      LOG(WARNING) << error;
+      delete[] temp;
       close(fd.get());
+      return InternalServerError(error);
+    } else {
+      object.values["offset"] = offset;
+      object.values["length"] = length;
+      object.values["data"] = string(temp, length);
+      delete[] temp;
+    }
+  } else {
+    object.values["offset"] = size;
+    object.values["length"] = 0;
+  }
 
-      std::ostringstream out;
+  close(fd.get());
 
-      if (jsonp.isSome()) {
-        out << jsonp.get() << "(";
-      }
+  return OK(object, request.query.get("jsonp"));
+}
+
+
+Future<Response> FilesProcess::debug(const Request& request)
+{
+  JSON::Object object;
+  foreachpair(const string& name, const string& path, paths) {
+    object.values[name] = path;
+  }
+  return OK(object, request.query.get("jsonp"));
+}
 
-      JSON::render(out, object);
 
-      OK response;
+Result<string> FilesProcess::resolve(const string& path)
+{
+  // Suppose we have: /1/2/hello_world.txt
+  // And we attach: /1/2 as /sandbox
+  // Then this function would resolve the following virtual path
+  // into the actual path:
+  // input: /sandbox/hello_world.txt
+  // output: /1/2/hello_world.txt
+  //
+  // Try and see if this path has been attached. We check for the
+  // longest possible prefix match and if found append any suffix to
+  // the attached path (provided the path is to a directory).
+  vector<string> tokens = strings::split(
+      strings::remove(path, "/", strings::SUFFIX), "/");
+
+  string suffix;
+  while (!tokens.empty()) {
+    string prefix = path::join(tokens);
 
-      if (jsonp.isSome()) {
-        out << ");";
-        response.headers["Content-Type"] = "text/javascript";
+    if (!paths.contains(prefix)) {
+      if (suffix.empty()) {
+        suffix = tokens.back();
       } else {
-        response.headers["Content-Type"] = "application/json";
+        suffix = path::join(tokens.back(), suffix);
       }
 
-      response.headers["Content-Length"] = stringify(out.str().size());
-      response.body = out.str().data();
+      tokens.pop_back();
+      continue;
+    }
+
+    // Determine the final path: if it's a directory, append the
+    // suffix, if it's not a directory and there is a suffix, return
+    // 'Not Found'.
+    string path = paths[prefix];
+    if (os::isdir(path)) {
+      path = path::join(path, suffix);
+
+      // Canonicalize the absolute path and make sure the result is
+      // accessible.
+      Try<string> result = os::realpath(path);
+      if (result.isError()) {
+        return Result<string>::error("Cannot resolve path.");
+      } else if (!strings::startsWith(result.get(), paths[prefix])) {
+        return Result<string>::error("Resolved path is inaccessible.");
+      }
 
-      return response;
+      path = result.get();
+    } else if (suffix != "") {
+      // Request is assuming attached path is a directory, but it is
+      // not! Rather than 'Bad Request', treat this as 'Not Found'.
+      return Result<string>::none();
     }
+
+    return path;
   }
 
-  return NotFound();
+  return Result<string>::none();
 }
 
 
@@ -299,10 +359,11 @@ Files::~Files()
 {
   terminate(process);
   wait(process);
+  delete process;
 }
 
 
-Future<bool> Files::attach(const string& path, const string& name)
+Future<Nothing> Files::attach(const string& path, const string& name)
 {
   return dispatch(process, &FilesProcess::attach, path, name);
 }
@@ -313,5 +374,11 @@ void Files::detach(const string& name)
   dispatch(process, &FilesProcess::detach, name);
 }
 
+
+PID<> Files::pid()
+{
+  return process->self();
+}
+
 } // namespace internal {
 } // namespace mesos {

Modified: incubator/mesos/trunk/src/files/files.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/files/files.hpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/files/files.hpp (original)
+++ incubator/mesos/trunk/src/files/files.hpp Mon Sep 10 15:48:20 2012
@@ -22,6 +22,12 @@
 #include <string>
 
 #include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/pid.hpp>
+
+#include <stout/json.hpp>
+#include <stout/nothing.hpp>
+#include <stout/path.hpp>
 
 namespace mesos {
 namespace internal {
@@ -44,17 +50,40 @@ public:
 
   // Returns the result of trying to attach the specified path
   // (directory or file) at the specified name.
-  process::Future<bool> attach(
+  process::Future<Nothing> attach(
       const std::string& path,
       const std::string& name);
 
   // Removes the specified name.
   void detach(const std::string& name);
 
+  // Returns the pid for the FilesProcess.
+  // NOTE: This has been made visible for testing.
+  process::PID<> pid();
+
 private:
   FilesProcess* process;
 };
 
+
+// Returns our JSON representation of a file or directory.
+inline JSON::Object jsonFileInfo(const std::string& path,
+                                 bool isDir,
+                                 size_t size)
+{
+  JSON::Object file;
+  file.values["path"] = path;
+  file.values["size"] = size;
+
+  if (isDir) {
+    file.values["dir"] = JSON::True();
+  } else {
+    file.values["dir"] = JSON::False();
+  }
+
+  return file;
+}
+
 } // namespace internal {
 } // namespace mesos {
 

Modified: incubator/mesos/trunk/src/launcher/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/launcher/main.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/launcher/main.cpp (original)
+++ incubator/mesos/trunk/src/launcher/main.cpp Mon Sep 10 15:48:20 2012
@@ -42,7 +42,7 @@ int main(int argc, char** argv)
 
   // Construct URIs from the encoded environment string.
   const std::string& uris = os::getenv("MESOS_EXECUTOR_URIS");
-  foreach (const std::string& token, strings::split(uris, " ")) {
+  foreach (const std::string& token, strings::tokenize(uris, " ")) {
     size_t pos = token.rfind("+"); // Delim between uri and exec permission.
     CHECK(pos != std::string::npos) << "Invalid executor uri token in env "
                                     << token;

Modified: incubator/mesos/trunk/src/linux/cgroups.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/cgroups.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/cgroups.cpp (original)
+++ incubator/mesos/trunk/src/linux/cgroups.cpp Mon Sep 10 15:48:20 2012
@@ -313,7 +313,7 @@ bool enabled()
 
 Try<bool> enabled(const std::string& subsystems)
 {
-  std::vector<std::string> names = strings::split(subsystems, ",");
+  std::vector<std::string> names = strings::tokenize(subsystems, ",");
   if (names.empty()) {
     return Try<bool>::error("No subsystem is specified");
   }
@@ -344,7 +344,7 @@ Try<bool> enabled(const std::string& sub
 
 Try<bool> busy(const std::string& subsystems)
 {
-  std::vector<std::string> names = strings::split(subsystems, ",");
+  std::vector<std::string> names = strings::tokenize(subsystems, ",");
   if (names.empty()) {
     return Try<bool>::error("No subsystem is specified");
   }
@@ -536,7 +536,7 @@ Try<bool> checkHierarchy(const std::stri
   }
 
   std::set<std::string> names = namesResult.get();
-  foreach (const std::string& name, strings::split(subsystems, ",")) {
+  foreach (const std::string& name, strings::tokenize(subsystems, ",")) {
     if (names.find(name) == names.end()) {
       return Try<bool>::error(
           "Subsystem " + name + " is not found or enabled");

Modified: incubator/mesos/trunk/src/local/local.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/local.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/local.cpp (original)
+++ incubator/mesos/trunk/src/local/local.cpp Mon Sep 10 15:48:20 2012
@@ -64,6 +64,7 @@ static AllocatorProcess* allocator = NUL
 static Master* master = NULL;
 static map<IsolationModule*, Slave*> slaves;
 static MasterDetector* detector = NULL;
+static Files* files = NULL;
 
 
 PID<Master> launch(int numSlaves,
@@ -85,7 +86,8 @@ PID<Master> launch(int numSlaves,
 }
 
 
-PID<Master> launch(const Configuration& configuration, AllocatorProcess* _allocator)
+PID<Master> launch(const Configuration& configuration,
+                   AllocatorProcess* _allocator)
 {
   int numSlaves = configuration.get<int>("num_slaves", 1);
 
@@ -102,10 +104,12 @@ PID<Master> launch(const Configuration& 
     allocator = NULL;
   }
 
+  files = new Files();
+
   {
     flags::Flags<logging::Flags, master::Flags> flags;
     flags.load(configuration.getMap());
-    master = new Master(_allocator, flags);
+    master = new Master(_allocator, files, flags);
   }
 
   PID<Master> pid = process::spawn(master);
@@ -119,7 +123,7 @@ PID<Master> launch(const Configuration& 
     // TODO(benh): Create a local isolation module?
     ProcessBasedIsolationModule* isolationModule =
       new ProcessBasedIsolationModule();
-    Slave* slave = new Slave(flags, true, isolationModule);
+    Slave* slave = new Slave(flags, true, isolationModule, files);
     slaves[isolationModule] = slave;
     pids.push_back(process::spawn(slave));
   }
@@ -156,6 +160,9 @@ void shutdown()
 
     delete detector;
     detector = NULL;
+
+    delete files;
+    files = NULL;
   }
 }
 

Modified: incubator/mesos/trunk/src/logging/logging.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/logging/logging.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/logging/logging.cpp (original)
+++ incubator/mesos/trunk/src/logging/logging.cpp Mon Sep 10 15:48:20 2012
@@ -73,20 +73,8 @@ protected:
 private:
   Future<Response> toggle(const Request& request)
   {
-    map<string, vector<string> > pairs =
-      strings::pairs(request.query, ";&", "=");
-
-    Option<string> level;
-
-    if (pairs.count("level") > 0 && pairs["level"].size() > 0) {
-      level = pairs["level"].back();
-    }
-
-    Option<string> duration;
-
-    if (pairs.count("duration") > 0 && pairs["duration"].size() > 0) {
-      duration = pairs["duration"].back();
-    }
+    Option<string> level = request.query.get("level");
+    Option<string> duration = request.query.get("duration");
 
     if (level.isNone() && duration.isNone()) {
       return OK(stringify(FLAGS_v) + "\n");

Modified: incubator/mesos/trunk/src/master/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/http.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/http.cpp (original)
+++ incubator/mesos/trunk/src/master/http.cpp Mon Sep 10 15:48:20 2012
@@ -190,12 +190,7 @@ Future<Response> vars(
     "build_flags " << build::FLAGS << "\n";
 
   // TODO(benh): Output flags.
-
-  OK response;
-  response.headers["Content-Type"] = "text/plain";
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(out.str(), request.query.get("jsonp"));
 }
 
 Future<Response> redirect(
@@ -264,15 +259,7 @@ Future<Response> stats(
     }
   }
 
-  std::ostringstream out;
-
-  JSON::render(out, object);
-
-  OK response;
-  response.headers["Content-Type"] = "application/json";
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(object, request.query.get("jsonp"));
 }
 
 
@@ -334,15 +321,7 @@ Future<Response> state(
     object.values["completed_frameworks"] = array;
   }
 
-  std::ostringstream out;
-
-  JSON::render(out, object);
-
-  OK response;
-  response.headers["Content-Type"] = "application/json";
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(object, request.query.get("jsonp"));
 }
 
 } // namespace json {

Modified: incubator/mesos/trunk/src/master/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/main.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/main.cpp (original)
+++ incubator/mesos/trunk/src/master/main.cpp Mon Sep 10 15:48:20 2012
@@ -117,8 +117,8 @@ int main(int argc, char** argv)
 
   AllocatorProcess* allocator = AllocatorProcess::create(flags.user_sorter,
 							 flags.framework_sorter);
-
-  Master* master = new Master(allocator, flags);
+  Files files;
+  Master* master = new Master(allocator, &files, flags);
   process::spawn(master);
 
   Try<MasterDetector*> detector =

Modified: incubator/mesos/trunk/src/master/master.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.cpp (original)
+++ incubator/mesos/trunk/src/master/master.cpp Mon Sep 10 15:48:20 2012
@@ -21,6 +21,7 @@
 #include <list>
 #include <sstream>
 
+#include <process/defer.hpp>
 #include <process/delay.hpp>
 #include <process/id.hpp>
 #include <process/run.hpp>
@@ -94,7 +95,7 @@ protected:
         whitelist = Option<hashset<string> >::some(hashset<string>());
       } else {
         hashset<string> hostnames;
-        vector<string> lines = strings::split(result.get(), "\n");
+        vector<string> lines = strings::tokenize(result.get(), "\n");
         foreach (const string& hostname, lines) {
           hostnames.insert(hostname);
         }
@@ -255,17 +256,20 @@ struct SlaveReregistrar
 };
 
 
-Master::Master(AllocatorProcess* _allocator)
+Master::Master(AllocatorProcess* _allocator, Files* _files)
   : ProcessBase("master"),
     flags(),
-    allocator(_allocator) {}
+    allocator(_allocator),
+    files(_files) {}
 
 
 Master::Master(AllocatorProcess* _allocator,
+               Files* _files,
                const flags::Flags<logging::Flags, master::Flags>& _flags)
   : ProcessBase("master"),
     flags(_flags),
-    allocator(_allocator) {}
+    allocator(_allocator),
+    files(_files) {}
 
 
 Master::~Master()
@@ -452,8 +456,13 @@ void Master::initialize()
   provide("static", path::join(flags.webui_dir, "master/static"));
 
   // TODO(benh): Ask glog for file name (i.e., mesos-master.INFO).
+  // Blocked on http://code.google.com/p/google-glog/issues/detail?id=116
+  // Alternatively, initialize() could take the executable name.
   if (flags.log_dir.isSome()) {
-    files.attach(flags.log_dir.get() + "/mesos-master.INFO", "/log");
+    Future<Nothing> result = files->attach(
+        path::join(flags.log_dir.get(), "mesos-master.INFO"),
+        "/log");
+    result.onAny(defer(self(), &Self::fileAttached, result));
   }
 }
 
@@ -517,6 +526,17 @@ void Master::exited(const UPID& pid)
 }
 
 
+void Master::fileAttached(const Future<Nothing>& result)
+{
+  CHECK(!result.isDiscarded());
+  if (result.isReady()) {
+    LOG(INFO) << "Master attached log file successfully";
+  } else {
+    LOG(ERROR) << "Failed to attach log file: " << result.failure();
+  }
+}
+
+
 void Master::submitScheduler(const string& name)
 {
   LOG(INFO) << "Scheduler submit request for " << name;

Modified: incubator/mesos/trunk/src/master/master.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.hpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.hpp (original)
+++ incubator/mesos/trunk/src/master/master.hpp Mon Sep 10 15:48:20 2012
@@ -67,8 +67,9 @@ class WhitelistWatcher;
 class Master : public ProtobufProcess<Master>
 {
 public:
-  Master(AllocatorProcess* allocator);
+  Master(AllocatorProcess* allocator, Files* files);
   Master(AllocatorProcess* allocator,
+         Files* files,
          const flags::Flags<logging::Flags, master::Flags>& flags);
 
   virtual ~Master();
@@ -122,6 +123,8 @@ protected:
   virtual void finalize();
   virtual void exited(const UPID& pid);
 
+  void fileAttached(const Future<Nothing>& result);
+
   // Return connected frameworks that are not in the process of being removed
   std::vector<Framework*> getActiveFrameworks() const;
 
@@ -207,6 +210,7 @@ private:
   AllocatorProcess* allocator;
   SlavesManager* slavesManager;
   WhitelistWatcher* whitelistWatcher;
+  Files* files;
 
   MasterInfo info;
 
@@ -232,8 +236,6 @@ private:
   } stats;
 
   double startTime; // Start time used to calculate uptime.
-
-  Files files;
 };
 
 

Modified: incubator/mesos/trunk/src/master/slaves_manager.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/slaves_manager.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/slaves_manager.cpp (original)
+++ incubator/mesos/trunk/src/master/slaves_manager.cpp Mon Sep 10 15:48:20 2012
@@ -27,6 +27,7 @@
 #include <process/dispatch.hpp>
 
 #include <stout/fatal.hpp>
+#include <stout/numify.hpp>
 #include <stout/strings.hpp>
 #include <stout/utils.hpp>
 
@@ -49,6 +50,7 @@ using process::PID;
 using process::Process;
 using process::UPID;
 
+using process::http::BadRequest;
 using process::http::InternalServerError;
 using process::http::NotFound;
 using process::http::OK;
@@ -566,9 +568,9 @@ bool ZooKeeperSlavesManagerStorage::pars
 
   const string& temp = s.substr(begin + key.size(), length);
 
-  const vector<string>& tokens = strings::split(temp, ",");
+  const vector<string>& tokens = strings::tokenize(temp, ",");
   foreach (const string& token, tokens) {
-    const vector<string>& pairs = strings::split(token, ":");
+    const vector<string>& pairs = strings::tokenize(token, ":");
     if (pairs.size() != 2) {
       LOG(WARNING) << "Slaves manager storage found bad data in '" << znode
                    << "', could not parse " << token;
@@ -619,9 +621,9 @@ SlavesManager::SlavesManager(const Flags
   } else {
     // Parse 'slaves' as initial active hostname:port pairs.
     if (slaves != "*") {
-      const vector<string>& tokens = strings::split(slaves, ",");
+      const vector<string>& tokens = strings::tokenize(slaves, ",");
       foreach (const string& token, tokens) {
-        const vector<string>& pairs = strings::split(token, ":");
+        const vector<string>& pairs = strings::tokenize(token, ":");
         if (pairs.size() != 2) {
           fatal("Failed to parse \"%s\" in option 'slaves'", token.c_str());
         }
@@ -810,173 +812,145 @@ void SlavesManager::updateInactive(
 
 Future<Response> SlavesManager::add(const Request& request)
 {
-  // Parse the query to get out the slave hostname and port.
-  string hostname = "";
-  uint16_t port = 0;
-
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ",", "=");
+  Option<string> hostname = request.query.get("hostname");
+  Option<string> portString = request.query.get("port");
 
   // Make sure there is at least a 'hostname=' and 'port='.
-  if (pairs.count("hostname") == 0) {
-    LOG(WARNING) << "Slaves manager expecting 'hostname' in query string"
-                 << " when trying to add a slave";
-    return NotFound();
-  } else if (pairs.count("port") == 0) {
-    LOG(WARNING) << "Slaves manager expecting 'port' in query string"
-                 << " when trying to add a slave";
-    return NotFound();
+  if (hostname.isNone()) {
+    string error = "Slaves manager expecting 'hostname' in query string"
+                   " when trying to add a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
+  } else if (portString.isNone()) {
+    string error = "Slaves manager expecting 'port' in query string"
+                   " when trying to add a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
-  hostname = pairs["hostname"].front();
-
   // Check that 'port' is valid.
-  try {
-    port = lexical_cast<uint16_t>(pairs["port"].front());
-  } catch (const bad_lexical_cast&) {
-    LOG(WARNING) << "Slaves manager failed to parse 'port = "
-		 << pairs["port"].front()
-                 << "'  when trying to add a slave";
-    return NotFound();
+  Try<uint16_t> port = numify<uint16_t>(portString.get());
+  if (port.isError()) {
+    string error = string("Slaves manager failed to parse "
+        "'port = ") + portString.get() + "' when trying to add a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
   LOG(INFO) << "Slaves manager received HTTP request to add slave at "
-	    << hostname << ":" << port;
+            << hostname.get() << ":" << port.get();
 
-  if (add(hostname, port)) {
+  if (add(hostname.get(), port.get())) {
     return OK();
-  } else {
-    return InternalServerError();
   }
+  return InternalServerError();
 }
 
 
 Future<Response> SlavesManager::remove(const Request& request)
 {
-  // Parse the query to get out the slave hostname and port.
-  string hostname = "";
-  uint16_t port = 0;
-
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ",", "=");
+  Option<string> hostname = request.query.get("hostname");
+  Option<string> portString = request.query.get("port");
 
   // Make sure there is at least a 'hostname=' and 'port='.
-  if (pairs.count("hostname") == 0) {
-    LOG(WARNING) << "Slaves manager expecting 'hostname' in query string"
-                 << " when trying to remove a slave";
-    return NotFound();
-  } else if (pairs.count("port") == 0) {
-    LOG(WARNING) << "Slaves manager expecting 'port' in query string"
-                 << " when trying to remove a slave";
-    return NotFound();
+  if (hostname.isNone()) {
+    string error = "Slaves manager expecting 'hostname' in query string"
+                   " when trying to remove a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
+  } else if (portString.isNone()) {
+    string error = "Slaves manager expecting 'port' in query string"
+                   " when trying to remove a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
-  hostname = pairs["hostname"].front();
-
   // Check that 'port' is valid.
-  try {
-    port = lexical_cast<uint16_t>(pairs["port"].front());
-  } catch (const bad_lexical_cast&) {
-    LOG(WARNING) << "Slaves manager failed to parse 'port = "
-		 << pairs["port"].front()
-                 << "'  when trying to remove a slave";
-    return NotFound();
+  Try<uint16_t> port = numify<uint16_t>(portString.get());
+  if (port.isError()) {
+    string error = string("Slaves manager failed to parse "
+        "'port = ") + portString.get() + "' when trying to remove a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
   LOG(INFO) << "Slaves manager received HTTP request to remove slave at "
-	    << hostname << ":" << port;
+            << hostname.get() << ":" << port.get();
 
-  if (remove(hostname, port)) {
+  if (remove(hostname.get(), port.get())) {
     return OK();
-  } else {
-    return InternalServerError();
   }
+  return InternalServerError();
 }
 
 
 Future<Response> SlavesManager::activate(const Request& request)
 {
-  // Parse the query to get out the slave hostname and port.
-  string hostname = "";
-  uint16_t port = 0;
-
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ",", "=");
+  Option<string> hostname = request.query.get("hostname");
+  Option<string> portString = request.query.get("port");
 
   // Make sure there is at least a 'hostname=' and 'port='.
-  if (pairs.count("hostname") == 0) {
+  if (hostname.isNone()) {
     LOG(WARNING) << "Slaves manager expecting 'hostname' in query string"
                  << " when trying to activate a slave";
     return NotFound();
-  } else if (pairs.count("port") == 0) {
+  } else if (portString.isNone()) {
     LOG(WARNING) << "Slaves manager expecting 'port' in query string"
                  << " when trying to activate a slave";
     return NotFound();
   }
 
-  hostname = pairs["hostname"].front();
-
   // Check that 'port' is valid.
-  try {
-    port = lexical_cast<uint16_t>(pairs["port"].front());
-  } catch (const bad_lexical_cast&) {
-    LOG(WARNING) << "Slaves manager failed to parse 'port = "
-		 << pairs["port"].front()
-                 << "'  when trying to activate a slave";
-    return NotFound();
+  Try<uint16_t> port = numify<uint16_t>(portString.get());
+  if (port.isError()) {
+    string error = string("Slaves manager failed to parse "
+        "'port = ") + portString.get() + "' when trying to activate a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
   LOG(INFO) << "Slaves manager received HTTP request to activate slave at "
-	    << hostname << ":" << port;
+            << hostname.get() << ":" << port.get();
 
-  if (activate(hostname, port)) {
+  if (activate(hostname.get(), port.get())) {
     return OK();
-  } else {
-    return InternalServerError();
   }
+  return InternalServerError();
 }
 
 
 Future<Response> SlavesManager::deactivate(const Request& request)
 {
-  // Parse the query to get out the slave hostname and port.
-  string hostname = "";
-  uint16_t port = 0;
-
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ",", "=");
+  Option<string> hostname = request.query.get("hostname");
+  Option<string> portString = request.query.get("port");
 
   // Make sure there is at least a 'hostname=' and 'port='.
-  if (pairs.count("hostname") == 0) {
+  if (hostname.isNone()) {
     LOG(WARNING) << "Slaves manager expecting 'hostname' in query string"
                  << " when trying to deactivate a slave";
     return NotFound();
-  } else if (pairs.count("port") == 0) {
+  } else if (portString.isNone()) {
     LOG(WARNING) << "Slaves manager expecting 'port' in query string"
                  << " when trying to deactivate a slave";
     return NotFound();
   }
 
-  hostname = pairs["hostname"].front();
-
   // Check that 'port' is valid.
-  try {
-    port = lexical_cast<uint16_t>(pairs["port"].front());
-  } catch (const bad_lexical_cast&) {
-    LOG(WARNING) << "Slaves manager failed to parse 'port = "
-		 << pairs["port"].front()
-                 << "'  when trying to deactivate a slave";
-    return NotFound();
+  Try<uint16_t> port = numify<uint16_t>(portString.get());
+  if (port.isError()) {
+    string error = string("Slaves manager failed to parse "
+        "'port = ") + portString.get() + "' when trying to deactivate a slave";
+    LOG(WARNING) << error;
+    return BadRequest(error);
   }
 
   LOG(INFO) << "Slaves manager received HTTP request to deactivate slave at "
-	    << hostname << ":" << port;
+            << hostname.get() << ":" << port.get();
 
-  if (deactivate(hostname, port)) {
+  if (deactivate(hostname.get(), port.get())) {
     return OK();
-  } else {
-    return InternalServerError();
   }
+  return InternalServerError();
 }
 
 
@@ -990,11 +964,7 @@ Future<Response> SlavesManager::activate
     out << hostname << ":" << port << "\n";
   }
 
-  OK response;
-  response.headers["Content-Type"] = "text/plain";
-  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(out.str());
 }
 
 
@@ -1008,11 +978,7 @@ Future<Response> SlavesManager::deactiva
     out << hostname << ":" << port << "\n";
   }
 
-  OK response;
-  response.headers["Content-Type"] = "text/plain";
-  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(out.str());
 }
 
 } // namespace master {

Modified: incubator/mesos/trunk/src/slave/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/http.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/http.cpp (original)
+++ incubator/mesos/trunk/src/slave/http.cpp Mon Sep 10 15:48:20 2012
@@ -138,12 +138,7 @@ Future<Response> vars(
     "build_flags " << build::FLAGS << "\n";
 
   // TODO(benh): Output flags.
-
-  OK response;
-  response.headers["Content-Type"] = "text/plain";
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(out.str(), request.query.get("jsonp"));
 }
 
 
@@ -167,15 +162,7 @@ Future<Response> stats(
   object.values["valid_status_updates"] = slave.stats.validStatusUpdates;
   object.values["invalid_status_updates"] = slave.stats.invalidStatusUpdates;
 
-  std::ostringstream out;
-
-  JSON::render(out, object);
-
-  OK response;
-  response.headers["Content-Type"] = "application/json";
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-  return response;
+  return OK(object, request.query.get("jsonp"));
 }
 
 
@@ -185,15 +172,6 @@ Future<Response> state(
 {
   LOG(INFO) << "HTTP request for '" << request.path << "'";
 
-  map<string, vector<string> > pairs =
-    strings::pairs(request.query, ";&", "=");
-
-  Option<string> jsonp;
-
-  if (pairs.count("jsonp") > 0 && pairs["jsonp"].size() > 0) {
-    jsonp = pairs["jsonp"].back();
-  }
-
   JSON::Object object;
   object.values["build_date"] = build::DATE;
   object.values["build_user"] = build::USER;
@@ -214,28 +192,7 @@ Future<Response> state(
   }
 
   object.values["frameworks"] = array;
-
-  std::ostringstream out;
-
-  if (jsonp.isSome()) {
-    out << jsonp.get() << "(";
-  }
-
-  JSON::render(out, object);
-
-  OK response;
-
-  if (jsonp.isSome()) {
-    out << ");";
-    response.headers["Content-Type"] = "text/javascript";
-  } else {
-    response.headers["Content-Type"] = "application/json";
-  }
-
-  response.headers["Content-Length"] = stringify(out.str().size());
-  response.body = out.str().data();
-
-  return response;
+  return OK(object, request.query.get("jsonp"));
 }
 
 } // namespace json {

Modified: incubator/mesos/trunk/src/slave/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/main.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/main.cpp (original)
+++ incubator/mesos/trunk/src/slave/main.cpp Mon Sep 10 15:48:20 2012
@@ -131,7 +131,8 @@ int main(int argc, char** argv)
   LOG(INFO) << "Build: " << build::DATE << " by " << build::USER;
   LOG(INFO) << "Starting Mesos slave";
 
-  Slave* slave = new Slave(flags, false, isolationModule);
+  Files files;
+  Slave* slave = new Slave(flags, false, isolationModule, &files);
   process::spawn(slave);
 
   Try<MasterDetector*> detector =

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Mon Sep 10 15:48:20 2012
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <iomanip>
 
+#include <process/defer.hpp>
 #include <process/delay.hpp>
 #include <process/dispatch.hpp>
 #include <process/id.hpp>
@@ -92,21 +93,25 @@ bool isTerminalTaskState(TaskState state
 
 Slave::Slave(const Resources& _resources,
              bool _local,
-             IsolationModule* _isolationModule)
+             IsolationModule* _isolationModule,
+             Files* _files)
   : ProcessBase(ID::generate("slave")),
     flags(),
     local(_local),
     resources(_resources),
-    isolationModule(_isolationModule) {}
+    isolationModule(_isolationModule),
+    files(_files) {}
 
 
 Slave::Slave(const flags::Flags<logging::Flags, slave::Flags>& _flags,
              bool _local,
-             IsolationModule* _isolationModule)
+             IsolationModule* _isolationModule,
+             Files* _files)
   : ProcessBase(ID::generate("slave")),
     flags(_flags),
     local(_local),
-    isolationModule(_isolationModule)
+    isolationModule(_isolationModule),
+    files(_files)
 {
   if (flags.resources.isNone()) {
     // TODO(benh): Move this compuation into Flags as the "default".
@@ -301,8 +306,13 @@ void Slave::initialize()
   route("/state.json", bind(&http::json::state, cref(*this), params::_1));
 
   // TODO(benh): Ask glog for file name (i.e., mesos-slave.INFO).
+  // Blocked on http://code.google.com/p/google-glog/issues/detail?id=116
+  // Alternatively, initialize() could take the executable name.
   if (flags.log_dir.isSome()) {
-    files.attach(flags.log_dir.get() + "/mesos-slave.INFO", "/log");
+    Future<Nothing> result = files->attach(
+        path::join(flags.log_dir.get(), "mesos-slave.INFO"),
+        "/log");
+    result.onAny(defer(self(), &Self::fileAttached, result));
   }
 }
 
@@ -336,6 +346,17 @@ void Slave::shutdown()
 }
 
 
+void Slave::fileAttached(const Future<Nothing>& result)
+{
+  CHECK(!result.isDiscarded());
+  if (result.isReady()) {
+    LOG(INFO) << "Master attached log file successfully";
+  } else {
+    LOG(ERROR) << "Failed to attach log file: " << result.failure();
+  }
+}
+
+
 void Slave::newMasterDetected(const UPID& pid)
 {
   LOG(INFO) << "New master detected at " << pid;
@@ -372,7 +393,7 @@ void Slave::registered(const SlaveID& sl
     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()) {
+    if (os::isdir(path) && file != id.value()) {
 
       Try<long> time = os::mtime(path);
 

Modified: incubator/mesos/trunk/src/slave/slave.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.hpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.hpp (original)
+++ incubator/mesos/trunk/src/slave/slave.hpp Mon Sep 10 15:48:20 2012
@@ -59,11 +59,13 @@ class Slave : public ProtobufProcess<Sla
 public:
   Slave(const Resources& resources,
         bool local,
-        IsolationModule* isolationModule);
+        IsolationModule* isolationModule,
+        Files* files);
 
   Slave(const flags::Flags<logging::Flags, slave::Flags>& flags,
         bool local,
-        IsolationModule *isolationModule);
+        IsolationModule *isolationModule,
+        Files* files);
 
   virtual ~Slave();
 
@@ -127,6 +129,8 @@ protected:
   virtual void finalize();
   virtual void exited(const UPID& pid);
 
+  void fileAttached(const Future<Nothing>& result);
+
   // Helper routine to lookup a framework.
   Framework* getFramework(const FrameworkID& frameworkId);
 
@@ -186,6 +190,7 @@ private:
   hashmap<FrameworkID, Framework*> frameworks;
 
   IsolationModule* isolationModule;
+  Files* files;
 
   // Statistics (initialized in Slave::initialize).
   struct {
@@ -200,7 +205,6 @@ private:
 
   bool connected; // Flag to indicate if slave is registered.
 
-  Files files;
   GarbageCollector gc;
 };
 

Modified: incubator/mesos/trunk/src/tests/allocator_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_tests.cpp Mon Sep 10 15:48:20 2012
@@ -139,12 +139,13 @@ TEST(AllocatorTest, DRFAllocatorProcess)
   EXPECT_CALL(a, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(m);
 
   ProcessBasedIsolationModule isolationModule;
   Resources resources = Resources::parse("cpus:2;mem:1024");
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s);
 
   BasicMasterDetector detector(master, slave1, true);
@@ -186,7 +187,7 @@ TEST(AllocatorTest, DRFAllocatorProcess)
   WAIT_UNTIL(framework2Added);
 
   Resources resources2 = Resources::parse("cpus:1;mem:512");
-  Slave s2(resources2, true, &isolationModule);
+  Slave s2(resources2, true, &isolationModule, &files);
   PID<Slave> slave2 = process::spawn(s2);
 
   BasicMasterDetector detector2(master, slave2, true);
@@ -196,7 +197,7 @@ TEST(AllocatorTest, DRFAllocatorProcess)
   EXPECT_THAT(offers2, OfferEq(1, 512));
 
   Resources resources3 = Resources::parse("cpus:3;mem:2048");
-  Slave s3(resources3, true, &isolationModule);
+  Slave s3(resources3, true, &isolationModule, &files);
   PID<Slave> slave3 = process::spawn(s3);
   BasicMasterDetector detector3(master, slave3, true);
 
@@ -220,7 +221,7 @@ TEST(AllocatorTest, DRFAllocatorProcess)
   WAIT_UNTIL(framework3Added);
 
   Resources resources4 = Resources::parse("cpus:4;mem:4096");
-  Slave s4(resources4, true, &isolationModule);
+  Slave s4(resources4, true, &isolationModule, &files);
   PID<Slave> slave4 = process::spawn(s4);
   BasicMasterDetector detector4(master, slave4, true);
 
@@ -297,13 +298,14 @@ TYPED_TEST(AllocatorTest, MockAllocator)
 
   EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _));
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(&m);
 
   ProcessBasedIsolationModule isolationModule;
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -375,12 +377,13 @@ TYPED_TEST(AllocatorTest, ResourcesUnuse
   EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _))
     .WillRepeatedly(Return());
 
-  Master m(&(this->allocator));
+  Files files;
+  Master m(&(this->allocator), &files);
   PID<Master> master = process::spawn(m);
 
   ProcessBasedIsolationModule isolationModule;
   Resources resources1 = Resources::parse("cpus:2;mem:1024");
-  Slave s(resources1, true, &isolationModule);
+  Slave s(resources1, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s);
   BasicMasterDetector(master, slave1, true);
 
@@ -488,14 +491,15 @@ TYPED_TEST(AllocatorTest, OutOfOrderDisp
 		    SaveArg<2>(&savedResources)))
     .WillRepeatedly(DoDefault());
 
-  Master m(&(this->allocator));
+  Files files;
+  Master m(&(this->allocator), &files);
   PID<Master> master = process::spawn(&m);
 
   ProcessBasedIsolationModule isolationModule;
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -600,7 +604,8 @@ TYPED_TEST(AllocatorTest, SchedulerFailo
   EXPECT_CALL(this->allocator, resourcesUnused(_, _, _, _))
     .WillRepeatedly(InvokeUnusedWithFilters(&this->allocator, 0));
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -621,7 +626,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailo
   EXPECT_CALL(isolationModule, resourcesChanged(_, _, _));
 
   Resources resources = Resources::parse("cpus:3;mem:1024");
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
   BasicMasterDetector detector(master, slave, true);
 
@@ -727,7 +732,8 @@ TYPED_TEST(AllocatorTest, FrameworkExite
   EXPECT_CALL(this->allocator, resourcesUnused(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(m);
 
   MockExecutor exec;
@@ -752,7 +758,7 @@ TYPED_TEST(AllocatorTest, FrameworkExite
     .Times(2);
 
   Resources resources1 = Resources::parse("cpus:3;mem:1024");
-  Slave s1(resources1, true, &isolationModule);
+  Slave s1(resources1, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s1);
   BasicMasterDetector detector1(master, slave1, true);
 
@@ -853,7 +859,8 @@ TYPED_TEST(AllocatorTest, SlaveLost)
 		    Trigger(&slaveRemovedTrigger1)))
     .WillOnce(Trigger(&slaveRemovedTrigger2));
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(m);
 
   MockExecutor exec;
@@ -875,7 +882,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   EXPECT_CALL(isolationModule, resourcesChanged(_, _, _));
 
   Resources resources1 = Resources::parse("cpus:2;mem:1024");
-  Slave s1(resources1, true, &isolationModule);
+  Slave s1(resources1, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s1);
   BasicMasterDetector detector1(master, slave1, true);
 
@@ -923,7 +930,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
 
   ProcessBasedIsolationModule isolationModule2;
   Resources resources2 = Resources::parse("cpus:3;mem:256");
-  Slave s2(resources2, true, &isolationModule2);
+  Slave s2(resources2, true, &isolationModule2, &files);
   PID<Slave> slave2 = process::spawn(s2);
   BasicMasterDetector detector2(master, slave2, true);
 
@@ -975,7 +982,8 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(m);
 
   MockExecutor exec;
@@ -999,7 +1007,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   EXPECT_CALL(isolationModule, resourcesChanged(_, _, _));
 
   Resources resources1 = Resources::parse("cpus:3;mem:1024");
-  Slave s1(resources1, true, &isolationModule);
+  Slave s1(resources1, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s1);
   BasicMasterDetector detector1(master, slave1, true);
 
@@ -1044,7 +1052,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   WAIT_UNTIL(launchTaskTrigger);
 
   Resources resources2 = Resources::parse("cpus:4;mem:2048");
-  Slave s2(resources2, true, &isolationModule);
+  Slave s2(resources2, true, &isolationModule, &files);
   PID<Slave> slave2 = process::spawn(s2);
   BasicMasterDetector detector2(master, slave2, true);
 
@@ -1099,7 +1107,8 @@ TYPED_TEST(AllocatorTest, TaskFinished)
     .WillOnce(InvokeUnusedWithFilters(&this->allocator, 0))
     .WillRepeatedly(InvokeUnusedWithFilters(&this->allocator, 1));
 
-  Master m(&this->allocator);
+  Files files;
+  Master m(&this->allocator, &files);
   PID<Master> master = process::spawn(m);
 
   MockExecutor exec;
@@ -1127,7 +1136,7 @@ TYPED_TEST(AllocatorTest, TaskFinished)
     .Times(2);
 
   Resources resources1 = Resources::parse("cpus:3;mem:1024");
-  Slave s1(resources1, true, &isolationModule);
+  Slave s1(resources1, true, &isolationModule, &files);
   PID<Slave> slave1 = process::spawn(s1);
   BasicMasterDetector detector1(master, slave1, true);
 
@@ -1220,9 +1229,10 @@ TYPED_TEST(AllocatorTest, WhitelistSlave
   string path = "whitelist.txt";
   CHECK (os::write(path, hosts).isSome()) << "Error writing whitelist";
 
+  Files files;
   flags::Flags<logging::Flags, master::Flags> flags;
   flags.whitelist = "file://" + path; // TODO(benh): Put in /tmp.
-  Master m(&this->allocator, flags);
+  Master m(&this->allocator, &files, flags);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -1232,7 +1242,7 @@ TYPED_TEST(AllocatorTest, WhitelistSlave
 
   TestingIsolationModule isolationModule(execs);
   Resources resources = Resources::parse("cpus:2;mem:1024");
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);

Modified: incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp Mon Sep 10 15:48:20 2012
@@ -100,7 +100,8 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
   EXPECT_MESSAGE(filter, Eq(ReregisterSlaveMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
-  Master m(&this->allocator1);
+  Files files;
+  Master m(&this->allocator1, &files);
   PID<Master> master1 = process::spawn(&m);
 
   string zk = "zk://" + this->zks->connectString() + "/znode";
@@ -125,7 +126,7 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
 
   TestingIsolationModule isolationModule(execs);
   Resources resources = Resources::parse("cpus:2;mem:1024");
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   Try<MasterDetector*> slave_detector =
@@ -181,7 +182,8 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
 
   WAIT_UNTIL(shutdownMessageTrigger);
 
-  Master m2(&(this->allocator2));
+  Files files2;
+  Master m2(&(this->allocator2), &files2);
   PID<Master> master2 = process::spawn(m2);
 
   Try<MasterDetector*> detector2 =
@@ -250,7 +252,8 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
   EXPECT_MESSAGE(filter, Eq(ReregisterFrameworkMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
-  Master m(&this->allocator1);
+  Files files;
+  Master m(&this->allocator1, &files);
   PID<Master> master1 = process::spawn(&m);
 
   string zk = "zk://" + this->zks->connectString() + "/znode";
@@ -275,7 +278,7 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
 
   TestingIsolationModule isolationModule(execs);
   Resources resources = Resources::parse("cpus:2;mem:1024");
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   Try<MasterDetector*> slave_detector =
@@ -331,7 +334,8 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
 
   WAIT_UNTIL(shutdownMessageTrigger);
 
-  Master m2(&(this->allocator2));
+  Files files2;
+  Master m2(&(this->allocator2), &files2);
   PID<Master> master2 = process::spawn(m2);
 
   Try<MasterDetector*> detector2 =

Modified: incubator/mesos/trunk/src/tests/configurator_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/configurator_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/configurator_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/configurator_tests.cpp Mon Sep 10 15:48:20 2012
@@ -130,13 +130,12 @@ TEST(ConfiguratorTest, CommandLine)
 // Check whether specifying just MESOS_CONF allows a config file to be loaded
 TEST_WITH_WORKDIR(ConfiguratorTest, ConfigFileWithConfDir)
 {
-  if (mkdir("conf2", 0755) != 0)
-    FAIL() << "Failed to create directory conf2";
-  ofstream file("conf2/mesos.conf");
-  file << "test3=shake # sugar bomb\n";
-  file << "# just a comment\n";
-  file << "test4=milk\n";
-  file.close();
+  ASSERT_TRUE(os::mkdir("conf2"));
+  ASSERT_TRUE(os::write("conf2/mesos.conf",
+                        "test3=shake # sugar bomb\n"
+                        "# just a comment\n"
+                        "test4=milk\n").get());
+
   setenv("MESOS_CONF", "conf2", 1);
   Configurator conf;
   EXPECT_NO_THROW( conf.load() );
@@ -151,15 +150,12 @@ TEST_WITH_WORKDIR(ConfiguratorTest, Conf
 // we load values from the config file first and then the command line
 TEST_WITH_WORKDIR(ConfiguratorTest, CommandLineConfFlag)
 {
-  if (mkdir("bin", 0755) != 0)
-    FAIL() << "Failed to create directory bin";
-  if (mkdir("conf2", 0755) != 0)
-    FAIL() << "Failed to create directory conf2";
-  ofstream file("conf2/mesos.conf");
-  file << "a=1\n";
-  file << "b=2\n";
-  file << "c=3";
-  file.close();
+  ASSERT_TRUE(os::mkdir("bin"));
+  ASSERT_TRUE(os::mkdir("conf2"));
+  ASSERT_TRUE(os::write("conf2/mesos.conf",
+                        "a=1\n"
+                        "b=2\n"
+                        "c=3").get());
 
   const int ARGC = 4;
   char* argv[ARGC];
@@ -184,17 +180,13 @@ TEST_WITH_WORKDIR(ConfiguratorTest, Comm
 // second should be environment variables, and last should be the file.
 TEST_WITH_WORKDIR(ConfiguratorTest, LoadingPriorities)
 {
-  // Create a file which contains parameters a, b, c and d
-  if (mkdir("bin", 0755) != 0)
-    FAIL() << "Failed to create directory bin";
-  if (mkdir("conf", 0755) != 0)
-    FAIL() << "Failed to create directory conf";
-  ofstream file("conf/mesos.conf");
-  file << "a=fromFile\n";
-  file << "b=fromFile\n";
-  file << "c=fromFile\n";
-  file << "d=fromFile\n";
-  file.close();
+  ASSERT_TRUE(os::mkdir("bin"));
+  ASSERT_TRUE(os::mkdir("conf"));
+  ASSERT_TRUE(os::write("conf/mesos.conf",
+                        "a=fromFile\n"
+                        "b=fromFile\n"
+                        "c=fromFile\n"
+                        "d=fromFile\n").get());
 
   // Set environment to contain parameters a and b
   setenv("MESOS_A", "fromEnv", 1);
@@ -228,19 +220,17 @@ TEST_WITH_WORKDIR(ConfiguratorTest, Load
 // Check that spaces before and after the = signs in config files are ignored
 TEST_WITH_WORKDIR(ConfiguratorTest, ConfigFileSpacesIgnored)
 {
-  if (mkdir("conf", 0755) != 0)
-    FAIL() << "Failed to create directory conf";
-  ofstream file("conf/mesos.conf");
-  file << "test1=coffee # beans are tasty\n";
-  file << "# just a comment\n";
-  file << "  \t # comment with spaces in front\n";
-  file << "\n";
-  file << "test2 =tea\n";
-  file << "test3=  water\n";
-  file << "   test4 =  milk\n";
-  file << "  test5 =  hot  chocolate\t\n";
-  file << "\ttest6 =  juice# #\n";
-  file.close();
+  ASSERT_TRUE(os::mkdir("conf"));
+  ASSERT_TRUE(os::write("conf/mesos.conf",
+                        "test1=coffee # beans are tasty\n"
+                        "# just a comment\n"
+                        "  \t # comment with spaces in front\n"
+                        "\n"
+                        "test2 =tea\n"
+                        "test3=  water\n"
+                        "   test4 =  milk\n"
+                        "  test5 =  hot  chocolate\t\n"
+                        "\ttest6 =  juice# #\n").get());
 
   Configurator conf;
   setenv("MESOS_CONF", "conf", 1);
@@ -259,13 +249,11 @@ TEST_WITH_WORKDIR(ConfiguratorTest, Conf
 // Check that exceptions are thrown on invalid config file
 TEST_WITH_WORKDIR(ConfiguratorTest, MalformedConfigFile)
 {
-  if (mkdir("conf", 0755) != 0)
-    FAIL() << "Failed to create directory conf";
-  ofstream file("conf/mesos.conf");
-  file << "test1=coffee\n";
-  file << "JUNK\n";
-  file << "test2=tea\n";
-  file.close();
+  ASSERT_TRUE(os::mkdir("conf"));
+  ASSERT_TRUE(os::write("conf/mesos.conf",
+                        "test1=coffee\n"
+                        "JUNK\n"
+                        "test2=tea\n").get());
 
   setenv("MESOS_CONF", "conf", 1);
   Configurator conf;

Modified: incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp Mon Sep 10 15:48:20 2012
@@ -69,14 +69,15 @@ TEST(FaultToleranceTest, SlaveLost)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   ProcessBasedIsolationModule isolationModule;
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -395,7 +396,8 @@ TEST(FaultToleranceTest, DISABLED_TaskLo
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -415,7 +417,7 @@ TEST(FaultToleranceTest, DISABLED_TaskLo
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -499,7 +501,8 @@ TEST(FaultToleranceTest, SchedulerFailov
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -522,7 +525,7 @@ TEST(FaultToleranceTest, SchedulerFailov
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -631,7 +634,8 @@ TEST(FaultToleranceTest, SchedulerFailov
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -654,7 +658,7 @@ TEST(FaultToleranceTest, SchedulerFailov
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -753,7 +757,8 @@ TEST(FaultToleranceTest, SchedulerExit)
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   trigger statusUpdateMsg;
@@ -784,7 +789,7 @@ TEST(FaultToleranceTest, SchedulerExit)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -878,14 +883,15 @@ TEST(FaultToleranceTest, SlaveReliableRe
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   ProcessBasedIsolationModule isolationModule;
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -936,14 +942,15 @@ TEST(FaultToleranceTest, SlaveReregister
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   ProcessBasedIsolationModule isolationModule;
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);

Modified: incubator/mesos/trunk/src/tests/gc_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/gc_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/gc_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/gc_tests.cpp Mon Sep 10 15:48:20 2012
@@ -82,7 +82,8 @@ protected:
       .WillRepeatedly(Return(false));
 
     a = new TestAllocatorProcess();
-    m = new Master(a);
+    files = new Files();
+    m = new Master(a, files);
     master = process::spawn(m);
 
     execs[DEFAULT_EXECUTOR_ID] = &exec;
@@ -107,7 +108,7 @@ protected:
     process::wait(master);
     delete m;
     delete a;
-
+    delete files;
     process::filter(NULL);
 
     os::rmdir(flags.work_dir);
@@ -117,7 +118,7 @@ protected:
   {
     isolationModule = new TestingIsolationModule(execs);
 
-    s = new Slave(flags, true, isolationModule);
+    s = new Slave(flags, true, isolationModule, files);
     slave = process::spawn(s);
 
     detector = new BasicMasterDetector(master, slave, true);
@@ -168,6 +169,7 @@ protected:
   Master* m;
   TestingIsolationModule* isolationModule;
   Slave* s;
+  Files* files;
   BasicMasterDetector* detector;
   MockExecutor exec, exec1;
   map<ExecutorID, Executor*> execs;

Modified: incubator/mesos/trunk/src/tests/master_detector_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_detector_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_detector_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_detector_tests.cpp Mon Sep 10 15:48:20 2012
@@ -58,7 +58,8 @@ TEST(MasterDetector, File)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   map<ExecutorID, Executor*> execs;
@@ -67,7 +68,7 @@ TEST(MasterDetector, File)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   // Write "master" to a file and use the "file://" mechanism to

Modified: incubator/mesos/trunk/src/tests/master_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_tests.cpp Mon Sep 10 15:48:20 2012
@@ -75,7 +75,8 @@ TEST(MasterTest, TaskRunning)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -98,7 +99,7 @@ TEST(MasterTest, TaskRunning)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -168,7 +169,8 @@ TEST(MasterTest, KillTask)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -194,7 +196,7 @@ TEST(MasterTest, KillTask)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -265,7 +267,8 @@ TEST(MasterTest, RecoverResources)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -293,7 +296,7 @@ TEST(MasterTest, RecoverResources)
   Resources slaveResources = Resources::parse(
       "cpus:2;mem:1024;ports:[1-10, 20-30]");
 
-  Slave s(slaveResources, true, &isolationModule);
+  Slave s(slaveResources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -399,7 +402,8 @@ TEST(MasterTest, FrameworkMessage)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -429,7 +433,7 @@ TEST(MasterTest, FrameworkMessage)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -519,7 +523,8 @@ TEST(MasterTest, MultipleExecutors)
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec1;
@@ -566,7 +571,7 @@ TEST(MasterTest, MultipleExecutors)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -664,7 +669,8 @@ TEST(MasterTest, MasterInfo)
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -676,7 +682,7 @@ TEST(MasterTest, MasterInfo)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -726,7 +732,8 @@ TEST(MasterTest, MasterInfoOnReElection)
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -738,7 +745,7 @@ TEST(MasterTest, MasterInfoOnReElection)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -827,9 +834,10 @@ TEST_F(WhitelistFixture, WhitelistSlave)
   CHECK (os::write(path, hosts).isSome()) << "Error writing whitelist";
 
   TestAllocatorProcess a;
+  Files files;
   flags::Flags<logging::Flags, master::Flags> flags;
   flags.whitelist = "file://" + path;
-  Master m(&a, flags);
+  Master m(&a, &files, flags);
   PID<Master> master = process::spawn(&m);
 
   trigger slaveRegisteredMsg;
@@ -846,7 +854,7 @@ TEST_F(WhitelistFixture, WhitelistSlave)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);
@@ -910,7 +918,8 @@ TEST(MasterTest, MasterLost)
     .WillRepeatedly(Return(false));
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -922,7 +931,7 @@ TEST(MasterTest, MasterLost)
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);

Modified: incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/resource_offers_tests.cpp?rev=1382948&r1=1382947&r2=1382948&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/resource_offers_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/resource_offers_tests.cpp Mon Sep 10 15:48:20 2012
@@ -473,7 +473,8 @@ TEST(ResourceOffersTest, Request)
   EXPECT_CALL(allocator, slaveRemoved(_))
     .WillRepeatedly(Return());
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false, &allocator);
+  PID<Master> master = local::launch(
+      1, 2, 1 * Gigabyte, false, &allocator);
 
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
 
@@ -518,7 +519,8 @@ TEST(ResourceOffersTest, TasksExecutorIn
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
   TestAllocatorProcess a;
-  Master m(&a);
+  Files files;
+  Master m(&a, &files);
   PID<Master> master = process::spawn(&m);
 
   MockExecutor exec;
@@ -541,7 +543,7 @@ TEST(ResourceOffersTest, TasksExecutorIn
 
   Resources resources = Resources::parse("cpus:2;mem:1024");
 
-  Slave s(resources, true, &isolationModule);
+  Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
   BasicMasterDetector detector(master, slave, true);