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/22 02:47:20 UTC

svn commit: r1388729 - in /incubator/mesos/trunk: src/ src/common/ src/examples/ src/launcher/ src/local/ src/log/ src/master/ src/mesos/ src/slave/ src/tests/ src/zookeeper/ third_party/libprocess/include/process/ third_party/libprocess/include/stout/

Author: benh
Date: Sat Sep 22 00:47:19 2012
New Revision: 1388729

URL: http://svn.apache.org/viewvc?rev=1388729&view=rev
Log:
Refactored os::dirname and os::basename to return a Try (and other
small fixes to unbreak the build) (contributed by Vinod Kone,
https://reviews.apache.org/r/7184).

Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/common/type_utils.hpp
    incubator/mesos/trunk/src/examples/balloon_executor.cpp
    incubator/mesos/trunk/src/launcher/launcher.cpp
    incubator/mesos/trunk/src/local/main.cpp
    incubator/mesos/trunk/src/log/main.cpp
    incubator/mesos/trunk/src/master/main.cpp
    incubator/mesos/trunk/src/mesos/main.cpp
    incubator/mesos/trunk/src/slave/main.cpp
    incubator/mesos/trunk/src/slave/paths.hpp
    incubator/mesos/trunk/src/slave/state.cpp
    incubator/mesos/trunk/src/tests/cgroups_tests.cpp
    incubator/mesos/trunk/src/tests/main.cpp
    incubator/mesos/trunk/src/tests/slave_state_tests.cpp
    incubator/mesos/trunk/src/zookeeper/group.cpp
    incubator/mesos/trunk/third_party/libprocess/include/process/async.hpp
    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=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Sat Sep 22 00:47:19 2012
@@ -189,6 +189,7 @@ EXTRA_DIST += slave/solaris_project_isol
 
 libmesos_no_third_party_la_SOURCES += common/attributes.hpp		\
 	common/build.hpp common/date_utils.hpp common/factory.hpp	\
+	common/protobuf_utils.hpp					\
 	common/lock.hpp common/resources.hpp common/process_utils.hpp	\
 	common/type_utils.hpp common/thread.hpp common/units.hpp	\
 	common/values.hpp configurator/configuration.hpp		\

Modified: incubator/mesos/trunk/src/common/type_utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/type_utils.hpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/type_utils.hpp (original)
+++ incubator/mesos/trunk/src/common/type_utils.hpp Sat Sep 22 00:47:19 2012
@@ -83,13 +83,6 @@ inline std::ostream& operator << (std::o
 }
 
 
-inline std::ostream& operator << (std::ostream& stream, const Value::Ranges& ranges)
-{
-  stream << ranges.DebugString();
-  return stream;
-}
-
-
 inline bool operator == (const FrameworkID& left, const FrameworkID& right)
 {
   return left.value() == right.value();

Modified: incubator/mesos/trunk/src/examples/balloon_executor.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/balloon_executor.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/balloon_executor.cpp (original)
+++ incubator/mesos/trunk/src/examples/balloon_executor.cpp Sat Sep 22 00:47:19 2012
@@ -50,8 +50,8 @@ static void balloon(size_t limit)
               << BALLOON_STEP_MB << " MB" << std::endl;
 
     // Allocate page-aligned virtual memory.
-    char* buffer = NULL;
-    if (posix_memalign((void**) &buffer, getpagesize(), chunk) != 0) {
+    void* buffer = NULL;
+    if (posix_memalign(&buffer, getpagesize(), chunk) != 0) {
       perror("Failed to allocate page-aligned memory, posix_memalign");
       abort();
     }

Modified: incubator/mesos/trunk/src/launcher/launcher.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/launcher/launcher.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/launcher/launcher.cpp (original)
+++ incubator/mesos/trunk/src/launcher/launcher.cpp Sat Sep 22 00:47:19 2012
@@ -33,6 +33,7 @@
 #include <stout/foreach.hpp>
 #include <stout/net.hpp>
 #include <stout/os.hpp>
+#include <stout/path.hpp>
 
 #include "launcher/launcher.hpp"
 
@@ -202,14 +203,20 @@ int ExecutorLauncher::fetchExecutors()
       // try looking for hadoop on the PATH.
       string hadoopScript;
       if (hadoopHome != "") {
-        hadoopScript = hadoopHome + "/bin/hadoop";
+        hadoopScript = path::join(hadoopHome, "bin/hadoop");
       } else if (getenv("HADOOP_HOME") != 0) {
-        hadoopScript = string(getenv("HADOOP_HOME")) + "/bin/hadoop";
+        hadoopScript = path::join(string(getenv("HADOOP_HOME")), "bin/hadoop");
       } else {
         hadoopScript = "hadoop"; // Look for hadoop on the PATH.
       }
 
-      string localFile = string("./") + basename((char *) resource.c_str());
+      Try<std::string> base = os::basename(resource);
+      if (base.isError()) {
+        cerr << base.error() << endl;
+        return -1;
+      }
+
+      string localFile = path::join(".", base.get());
       ostringstream command;
       command << hadoopScript << " fs -copyToLocal '" << resource
               << "' '" << localFile << "'";
@@ -229,7 +236,7 @@ int ExecutorLauncher::fetchExecutors()
       string path = resource.substr(resource.find("://") + 3);
       CHECK(path.find("/") != string::npos) << "Malformed URL (missing path)";
       CHECK(path.size() > path.find("/") + 1) << "Malformed URL (missing path)";
-      path =  "./" + path.substr(path.find_last_of("/") + 1);
+      path =  path::join(".", path.substr(path.find_last_of("/") + 1));
       cout << "Downloading " << resource << " to " << path << endl;
       Try<int> code = net::download(resource, path);
       if (code.isError()) {
@@ -245,9 +252,9 @@ int ExecutorLauncher::fetchExecutors()
       if (resource.find_first_of("/") != 0) {
         // We got a non-Hadoop and non-absolute path.
         if (frameworksHome != "") {
-          resource = frameworksHome + "/" + resource;
+          resource = path::join(frameworksHome, resource);
           cout << "Prepended configuration option frameworks_home to resource "
-            << "path, making it: " << resource << endl;
+               << "path, making it: " << resource << endl;
         } else {
           cerr << "A relative path was passed for the resource, but "
                << "the configuration option frameworks_home is not set. "
@@ -264,9 +271,17 @@ int ExecutorLauncher::fetchExecutors()
 
       int ret = os::system(command.str());
       if (ret != 0) {
-        fatal("Copy failed: return code %d", ret);
+        cerr << "Failed to copy " << resource << ": Exit code " << ret << endl;
+        return -1;
       }
-      resource = "./" + os::basename(resource);
+
+      Try<std::string> base = os::basename(resource);
+      if (base.isError()) {
+        cerr << base.error() << endl;
+        return -1;
+      }
+
+      resource = path::join(".", base.get());
     }
 
     if (shouldSwitchUser && !os::chown(user, resource)) {

Modified: incubator/mesos/trunk/src/local/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/main.cpp (original)
+++ incubator/mesos/trunk/src/local/main.cpp Sat Sep 22 00:47:19 2012
@@ -47,7 +47,7 @@ using std::string;
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...]" << endl
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
        << endl
        << "Launches a cluster within a single OS process."
        << endl

Modified: incubator/mesos/trunk/src/log/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/main.cpp (original)
+++ incubator/mesos/trunk/src/log/main.cpp Sat Sep 22 00:47:19 2012
@@ -48,7 +48,7 @@ using std::string;
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...] path/to/log"
+  cerr << "Usage: " << os::basename(argv0).get() << " [...] path/to/log"
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();

Modified: incubator/mesos/trunk/src/master/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/main.cpp (original)
+++ incubator/mesos/trunk/src/master/main.cpp Sat Sep 22 00:47:19 2012
@@ -46,7 +46,7 @@ using std::string;
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...]" << endl
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();

Modified: incubator/mesos/trunk/src/mesos/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/mesos/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/mesos/main.cpp (original)
+++ incubator/mesos/trunk/src/mesos/main.cpp Sat Sep 22 00:47:19 2012
@@ -19,7 +19,7 @@ using std::string;
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...]" << endl
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();

Modified: incubator/mesos/trunk/src/slave/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/main.cpp (original)
+++ incubator/mesos/trunk/src/slave/main.cpp Sat Sep 22 00:47:19 2012
@@ -46,7 +46,7 @@ using std::string;
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...]" << endl
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();

Modified: incubator/mesos/trunk/src/slave/paths.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/paths.hpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/paths.hpp (original)
+++ incubator/mesos/trunk/src/slave/paths.hpp Sat Sep 22 00:47:19 2012
@@ -199,14 +199,18 @@ inline std::string createUniqueExecutorW
 
   CHECK(!paths.isError()) << paths.error();
 
-  if (paths.isSome()) {
-    foreach (const std::string& path, paths.get()) {
-      Try<int> temp = numify<int>(os::basename(path));
-      if (temp.isError()) {
-        continue;
-      }
-      run = std::max(run, temp.get());
+  foreach (const std::string& path, paths.get()) {
+    Try<std::string> base = os::basename(path);
+    if (base.isError()) {
+      LOG(ERROR) << base.error();
+      continue;
     }
+
+    Try<int> temp = numify<int>(base.get());
+    if (temp.isError()) {
+      continue;
+    }
+    run = std::max(run, temp.get());
   }
 
   std::string path =

Modified: incubator/mesos/trunk/src/slave/state.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/state.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/state.cpp (original)
+++ incubator/mesos/trunk/src/slave/state.cpp Sat Sep 22 00:47:19 2012
@@ -37,7 +37,7 @@ SlaveState parse(const string& rootDir, 
 
   foreach (const string& path, frameworks.get()) {
     FrameworkID frameworkId;
-    frameworkId.set_value(os::basename(path));
+    frameworkId.set_value(os::basename(path).get());
 
     // Find the executors.
     Try<list<string> > executors =
@@ -52,7 +52,7 @@ SlaveState parse(const string& rootDir, 
 
     foreach (const string& path, executors.get()) {
       ExecutorID executorId;
-      executorId.set_value(os::basename(path));
+      executorId.set_value(os::basename(path).get());
 
       // Find the runs.
       Try<list<string> > runs =
@@ -65,7 +65,7 @@ SlaveState parse(const string& rootDir, 
       }
 
       foreach (const string& path, runs.get()) {
-        Try<int> result = numify<int>(os::basename(path));
+        Try<int> result = numify<int>(os::basename(path).get());
         if (!result.isSome()) {
           LOG(ERROR) << "Non-numeric run number in path " << path;
           continue;
@@ -91,7 +91,7 @@ SlaveState parse(const string& rootDir, 
 
         foreach (const string& path, tasks.get()) {
           TaskID taskId;
-          taskId.set_value(os::basename(path));
+          taskId.set_value(os::basename(path).get());
 
           state.frameworks[frameworkId].executors[executorId].runs[run].tasks
             .insert(taskId);
@@ -104,14 +104,15 @@ SlaveState parse(const string& rootDir, 
 
 
 // Helper functions for check-pointing slave data.
+
 void writeTask(Task* task, const string& taskDir)
 {
   const string& path = taskDir + "/task";
 
-  Try<Nothing> created = os::mkdir(os::dirname(path));
+  Try<Nothing> created = os::mkdir(taskDir);
 
   CHECK(created.isSome())
-    << "Error creating directory '" << os::dirname(path)
+    << "Error creating task directory '" << taskDir
     << "': " << created.error();
 
   LOG(INFO) << "Writing task description for task "
@@ -129,10 +130,10 @@ void writeSlaveID(const string& rootDir,
 {
   const string& path = paths::getSlaveIDPath(rootDir);
 
-  Try<Nothing> created = os::mkdir(os::dirname(path));
+  Try<Nothing> created = os::mkdir(os::dirname(path).get());
 
   CHECK(created.isSome())
-    << "Error creating directory '" << os::dirname(path)
+    << "Error creating directory '" << os::dirname(path).get()
     << "': " << created.error();
 
   LOG(INFO) << "Writing slave id " << slaveId << " to " << path;
@@ -173,10 +174,10 @@ void writeFrameworkPID(const string& met
   const string& path = paths::getFrameworkPIDPath(metaRootDir, slaveId,
                                                   frameworkId);
 
-  Try<Nothing> created = os::mkdir(os::dirname(path));
+  Try<Nothing> created = os::mkdir(os::dirname(path).get());
 
   CHECK(created.isSome())
-    << "Error creating directory '" << os::dirname(path)
+    << "Error creating directory '" << os::dirname(path).get()
     << "': " << created.error();
 
   LOG(INFO) << "Writing framework pid " << pid << " to " << path;

Modified: incubator/mesos/trunk/src/tests/cgroups_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/cgroups_tests.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/cgroups_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/cgroups_tests.cpp Sat Sep 22 00:47:19 2012
@@ -414,20 +414,20 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_ListenE
 
     // Blow up the memory.
     size_t limit = 1024 * 1024 * 512;
-    char* ptr = NULL;
+    void* buffer = NULL;
 
-    if (posix_memalign((void**) &ptr, getpagesize(), limit) != 0) {
+    if (posix_memalign(&buffer, getpagesize(), limit) != 0) {
       FAIL() << "Failed to allocate page-aligned memory, posix_memalign: "
              << strerror(errno);
     }
 
     // We use mlock and memset here to make sure that the memory
     // actually gets paged in and thus accounted for.
-    if (mlock(ptr, limit) != 0) {
+    if (mlock(buffer, limit) != 0) {
       FAIL() << "Failed to lock memory, mlock: " << strerror(errno);
     }
 
-    if (memset(ptr, 1, limit) != 0) {
+    if (memset(buffer, 1, limit) != 0) {
       FAIL() << "Failed to fill memory, memset: " << strerror(errno);
     }
 

Modified: incubator/mesos/trunk/src/tests/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/main.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/main.cpp (original)
+++ incubator/mesos/trunk/src/tests/main.cpp Sat Sep 22 00:47:19 2012
@@ -116,7 +116,7 @@ static void setupFilter()
 
 void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << os::basename(argv0) << " [...]" << endl
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();

Modified: incubator/mesos/trunk/src/tests/slave_state_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/slave_state_tests.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/slave_state_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/slave_state_tests.cpp Sat Sep 22 00:47:19 2012
@@ -44,7 +44,10 @@ protected:
     executorId.set_value("executor1");
     taskId.set_value("task1");
     run = 0;
-    rootDir = os::mkdtemp().get();
+
+    Try<string> path = os::mkdtemp();
+    CHECK(path.isSome()) << "Failed to mkdtemp " << path.error();
+    rootDir = path.get();
   }
 
   virtual ~SlaveStateFixture()

Modified: incubator/mesos/trunk/src/zookeeper/group.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/group.cpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/group.cpp (original)
+++ incubator/mesos/trunk/src/zookeeper/group.cpp Sat Sep 22 00:47:19 2012
@@ -558,9 +558,13 @@ Result<Group::Membership> GroupProcess::
 
   // Save the sequence number but only grab the basename. Example:
   // "/path/to/znode/0000000131" => "0000000131".
-  result = os::basename(result);
+  Try<string> base = os::basename(result);
+  if (base.isError()) {
+    return Result<Group::Membership>::error(
+        "Failed to get the sequence number: " + base.error());
+  }
 
-  Try<uint64_t> sequence = numify<uint64_t>(result);
+  Try<uint64_t> sequence = numify<uint64_t>(base.get());
   CHECK(sequence.isSome()) << sequence.error();
 
   Promise<bool>* cancelled = new Promise<bool>();

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/async.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/async.hpp?rev=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/async.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/async.hpp Sat Sep 22 00:47:19 2012
@@ -228,4 +228,4 @@ Future<typename std::tr1::result_of<F(A1
 
 } // namespace process {
 
-#endif __ASYNC_HPP__
+#endif // __ASYNC_HPP__

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=1388729&r1=1388728&r2=1388729&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp Sat Sep 22 00:47:19 2012
@@ -33,6 +33,7 @@
 
 #include "foreach.hpp"
 #include "nothing.hpp"
+#include "path.hpp"
 #include "result.hpp"
 #include "strings.hpp"
 #include "try.hpp"
@@ -189,10 +190,10 @@ inline Try<Nothing> touch(const std::str
 // Creates a temporary file under 'root' directory and returns its path.
 inline Try<std::string> mktemp(const std::string& root = "/tmp")
 {
-  const std::string path = root + "/XXXXXX";
+  const std::string path = path::join(root, "XXXXXX");
   char* temp = new char[path.size() + 1];
 
-  if (::mktemp(::strncpy(temp, path.c_str(), path.size())) != NULL) {
+  if (::mktemp(::strcpy(temp, path.c_str())) != NULL) {
     std::string result(temp);
     delete temp;
     return result;
@@ -224,7 +225,7 @@ inline Try<Nothing> write(int fd, const 
 // A wrapper function that wraps the above write() with
 // open and closing the file.
 inline Try<Nothing> write(const std::string& path,
-                       const std::string& message)
+                          const std::string& message)
 {
   Try<int> fd = os::open(path, O_WRONLY | O_CREAT | O_TRUNC,
                          S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
@@ -315,15 +316,35 @@ inline Try<Nothing> rm(const std::string
 }
 
 
-inline std::string basename(const std::string& path)
+inline Try<std::string> basename(const std::string& path)
 {
-  return ::basename(const_cast<char*>(path.c_str()));
+  char* temp = new char[path.size() + 1];
+  char* result = ::basename(::strcpy(temp, path.c_str()));
+  if (result == NULL) {
+    delete temp;
+    return Try<std::string>::error(
+        "Error getting basename of '" + path + "': " + strerror(errno));
+  }
+
+  std::string s(result);
+  delete temp;
+  return s;
 }
 
 
-inline std::string dirname(const std::string& path)
+inline Try<std::string> dirname(const std::string& path)
 {
-  return ::dirname(const_cast<char*>(path.c_str()));
+  char* temp = new char[path.size() + 1];
+  char* result = ::dirname(::strcpy(temp, path.c_str()));
+  if (result == NULL) {
+    delete temp;
+    return Try<std::string>::error(
+        "Error getting dirname of '" + path + "': " + strerror(errno));
+  }
+
+  std::string s(result);
+  delete temp;
+  return s;
 }
 
 
@@ -333,7 +354,7 @@ inline Try<std::string> realpath(const s
   if (::realpath(path.c_str(), temp) == NULL) {
     // TODO(benh): Include strerror(errno).
     return Try<std::string>::error(
-      "Failed to canonicalize " + path + " into an absolute path");
+        "Failed to canonicalize " + path + " into an absolute path");
   }
   return std::string(temp);
 }
@@ -414,10 +435,10 @@ inline Try<Nothing> mkdir(const std::str
 // Creates a temporary directory under 'root' directory and returns its path.
 inline Try<std::string> mkdtemp(const std::string& root = "/tmp")
 {
-  const std::string path = root + "/XXXXXX";
+  const std::string path = path::join(root, "XXXXXX");
   char* temp = new char[path.size() + 1];
 
-  if (::mkdtemp(::strncpy(temp, path.c_str(), path.size())) != NULL) {
+  if (::mkdtemp(::strcpy(temp, path.c_str())) != NULL) {
     std::string result(temp);
     delete temp;
     return result;
@@ -624,8 +645,9 @@ inline std::list<std::string> ls(const s
 // NOTE: Directory path should not end with '/'.
 // TODO(vinod): Support regular expressions for pattern.
 // TODO(vinod): Consider using ftw or a non-recursive approach.
-inline Try<std::list<std::string> > find(const std::string& directory,
-                                         const std::string& pattern)
+inline Try<std::list<std::string> > find(
+    const std::string& directory,
+    const std::string& pattern)
 {
   std::list<std::string> results;
 
@@ -635,21 +657,19 @@ inline Try<std::list<std::string> > find
   }
 
   foreach (const std::string& entry, ls(directory)) {
-    if (entry == "." || entry == "..") {
-      continue;
-    }
-    std::string result = directory + '/' + entry;
-    // This is just a hack to check whether this path is a regular file or
-    // a (sub) directory.
-    if (isdir(result)) { // If its a directory recurse.
-      CHECK(find(result, pattern).isSome())
-        << "Directory " << directory << " doesn't exist";
-      foreach (const std::string& path, find(result, pattern).get()) {
-        results.push_back(path);
+    std::string path = path::join(directory, entry);
+    // If it's a directory, recurse.
+    if (isdir(path)) {
+      Try<std::list<std::string> > matches = find(path, pattern);
+      if (matches.isError()) {
+        return matches;
+      }
+      foreach (const std::string& match, matches.get()) {
+        results.push_back(match);
       }
     } else {
-      if (basename(result).find(pattern) != std::string::npos) {
-        results.push_back(result); // Matched the file pattern!
+      if (entry.find(pattern) != std::string::npos) {
+        results.push_back(path); // Matched the file pattern!
       }
     }
   }