You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/05/24 01:37:40 UTC

svn commit: r1342090 - in /incubator/mesos/trunk/src: Makefile.am common/utils.hpp slave/slave.cpp tests/protobuf_io_tests.cpp tests/utils_tests.cpp

Author: benh
Date: Wed May 23 23:37:39 2012
New Revision: 1342090

URL: http://svn.apache.org/viewvc?rev=1342090&view=rev
Log:
Updates to utils (contributed by Vinod Kone, https://reviews.apache.org/r/4899).

Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/common/utils.hpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp
    incubator/mesos/trunk/src/tests/utils_tests.cpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1342090&r1=1342089&r2=1342090&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Wed May 23 23:37:39 2012
@@ -186,7 +186,8 @@ libmesos_no_third_party_la_SOURCES += co
 	common/hashset.hpp common/json.hpp common/lock.hpp		\
 	common/logging.hpp common/lambda.hpp common/option.hpp		\
 	common/resources.hpp common/result.hpp common/multihashmap.hpp	\
-	common/process_utils.hpp common/seconds.hpp common/try.hpp	\
+	common/process_utils.hpp common/protobuf_utils.hpp		\
+	common/seconds.hpp common/try.hpp				\
 	common/type_utils.hpp common/thread.hpp common/timer.hpp	\
 	common/utils.hpp common/units.hpp common/uuid.hpp		\
 	common/strings.hpp common/values.hpp				\

Modified: incubator/mesos/trunk/src/common/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/utils.hpp?rev=1342090&r1=1342089&r2=1342090&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/utils.hpp (original)
+++ incubator/mesos/trunk/src/common/utils.hpp Wed May 23 23:37:39 2012
@@ -110,9 +110,9 @@ Try<T> numify(const std::string& s)
     return boost::lexical_cast<T>(s);
   } catch (const boost::bad_lexical_cast&) {
     const Try<std::string>& message = strings::format(
-        "Failed to convert '%s' to number", s.c_str());
+      "Failed to convert '%s' to number", s.c_str());
     return Try<T>::error(
-        message.isSome() ? message.get() : "Failed to convert to number");
+      message.isSome() ? message.get() : "Failed to convert to number");
   }
 }
 
@@ -165,32 +165,149 @@ inline void unsetenv(const std::string& 
 }
 
 
-inline Result<int> open(const std::string& path, int oflag, mode_t mode = 0)
+inline Try<int> open(const std::string& path, int oflag, mode_t mode = 0)
 {
   int fd = ::open(path.c_str(), oflag, mode);
 
   if (fd < 0) {
-    return Result<int>::error(strerror(errno));
+    return Try<int>::error(strerror(errno));
   }
 
-  return Result<int>::some(fd);
+  return Try<int>::some(fd);
 }
 
 
-inline Result<bool> close(int fd)
+inline Try<bool> close(int fd)
 {
   if (::close(fd) != 0) {
-    return Result<bool>::error(strerror(errno));
+    return Try<bool>::error(strerror(errno));
   }
 
   return true;
 }
 
 
-inline Result<bool> rm(const std::string& path)
+inline Try<bool> touch(const std::string& path)
+{
+  Try<int> fd = open(path, O_RDWR | O_CREAT, S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+
+  if (fd.isError()) {
+    return Try<bool>::error("Failed to open file " + path);
+  }
+
+  Try<bool> result = close(fd.get());
+
+  if (result.isError()) {
+    return Try<bool>::error("Failed to close file " + path);
+  }
+
+  return true;
+}
+
+
+// Write out the string to the file at the current fd position.
+inline Try<bool> write(int fd, const std::string& message)
+{
+  ssize_t length = ::write(fd, message.data(), message.length());
+
+  if (length == -1) {
+    return Try<bool>::error(strerror(errno));
+  }
+
+  CHECK(length != 0);
+  CHECK(length == message.length()); // TODO(benh): Handle a non-blocking fd?
+
+  return true;
+}
+
+
+// A wrapper function that wraps the above write() with
+// open and closing the file.
+inline Try<bool> write(const std::string& path,
+                       const std::string& message)
+{
+  Try<int> fd = os::open(path, O_WRONLY | O_CREAT | O_TRUNC,
+                         S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+  if (fd.isError()) {
+    return Try<bool>::error("Failed to open file " + path);
+  }
+
+  Try<bool> result = write(fd.get(), message);
+
+  // NOTE: We ignore the return value of close(). This is because users calling
+  // this function are interested in the return value of write(). Also an
+  // unsuccessful close() doesn't affect the write.
+  os::close(fd.get());
+
+  return result;
+}
+
+
+// Read the contents of the file from its current offset
+// and return it as a string.
+inline Result<std::string> read(int fd)
+{
+  // Get the size of the file.
+  off_t offset = lseek(fd, 0, SEEK_CUR);
+  if (offset == -1) {
+    return Result<std::string>::error("Error seeking to SEEK_CUR");
+  }
+
+  off_t size = lseek(fd, 0, SEEK_END);
+  if (size == -1) {
+      return Result<std::string>::error("Error seeking to SEEK_END");
+  }
+
+  if (lseek(fd, offset, SEEK_SET) == -1) {
+    return Result<std::string>::error("Error seeking to SEEK_SET");
+  }
+
+  // Allocate memory.
+  char* buffer = new char[size];
+
+  ssize_t length = ::read(fd, buffer, size);
+
+  if (length == 0) {
+    return Result<std::string>::none();
+  } else if (length == -1) {
+    // Save the error, reset the file offset, and return the error.
+    return Result<std::string>::error(strerror(errno));
+  } else if (length != size) {
+    return Result<std::string>::error("Couldn't read the entire file");
+  }
+
+  std::string result(buffer, size);
+
+  return result;
+}
+
+
+// A wrapper function that wraps the above read() with
+// open and closing the file.
+inline Result<std::string> read(const std::string& path)
+{
+  Try<int> fd = os::open(path, O_RDONLY,
+                         S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+
+  if (fd.isError()) {
+    return Result<std::string>::error("Failed to open file " + path);
+  }
+
+  Result<std::string> result = read(fd.get());
+
+  // NOTE: We ignore the return value of close(). This is because users calling
+  // this function are interested in the return value of read(). Also an
+  // unsuccessful close() doesn't affect the read.
+  os::close(fd.get());
+
+  return result;
+}
+
+
+inline Try<bool> rm(const std::string& path)
 {
   if (::remove(path.c_str()) != 0) {
-    return Result<bool>::error(strerror(errno));
+    return Try<bool>::error(strerror(errno));
   }
 
   return true;
@@ -203,13 +320,19 @@ inline std::string basename(const std::s
 }
 
 
+inline std::string dirname(const std::string& path)
+{
+  return ::dirname(const_cast<char*>(path.c_str()));
+}
+
+
 inline Try<std::string> realpath(const std::string& path)
 {
   char temp[PATH_MAX];
   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);
 }
@@ -363,8 +486,8 @@ inline std::list<std::string> listdir(co
     (size_t) offsetof(dirent, d_name) + name_max + 1;
 
   size_t size = (name_end > sizeof(dirent)
-                 ? name_end
-                 : sizeof(dirent));
+    ? name_end
+    : sizeof(dirent));
 
   dirent* temp = (dirent*) malloc(size);
 
@@ -393,6 +516,44 @@ inline std::list<std::string> listdir(co
 }
 
 
+// Return the list of file paths that match the given pattern by recursively
+// searching the given directory. A match is successful if the pattern is a
+// substring of the file name.
+// 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)
+{
+  std::list<std::string> results;
+
+  if (!exists(directory, true)) {
+    return Try<std::list<std::string> >::error("Directory " + directory + " doesn't exist!");
+  }
+
+  foreach (const std::string& entry, listdir(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 (exists(result, true)) { // 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);
+      }
+    } else {
+      if (basename(result).find(pattern) != std::string::npos) {
+        results.push_back(result); // Matched the file pattern!
+      }
+    }
+  }
+
+  return results;
+}
+
+
 inline std::string user()
 {
   passwd* passwd;
@@ -404,12 +565,12 @@ inline std::string user()
 }
 
 
-inline Result<std::string> hostname()
+inline Try<std::string> hostname()
 {
   char host[512];
 
   if (gethostname(host, sizeof(host)) < 0) {
-    return Result<std::string>::error(strerror(errno));
+    return Try<std::string>::error(strerror(errno));
   }
 
   struct hostent he, *hep;
@@ -423,7 +584,7 @@ inline Result<std::string> hostname()
   temp = new char[length];
 
   while ((result = gethostbyname2_r(host, AF_INET, &he, temp,
-            length, &hep, &herrno)) == ERANGE) {
+                                    length, &hep, &herrno)) == ERANGE) {
     // Enlarge the buffer.
     delete[] temp;
     length *= 2;
@@ -432,12 +593,12 @@ inline Result<std::string> hostname()
 
   if (result != 0 || hep == NULL) {
     delete[] temp;
-    return Result<std::string>::error(hstrerror(herrno));
+    return Try<std::string>::error(hstrerror(herrno));
   }
 
   std::string hostname = hep->h_name;
   delete[] temp;
-  return Result<std::string>::some(hostname);
+  return Try<std::string>::some(hostname);
 }
 
 
@@ -473,7 +634,7 @@ inline Try<int> shell(std::ostream* os, 
   }
 
   if (ferror(file) != 0) {
-    std::string error =
+    const std::string& error =
       "Error reading output of '" + cmdline.get() + "': " + strerror(errno);
     pclose(file); // Ignoring result since we already have an error.
     return Try<int>::error(error);
@@ -523,7 +684,7 @@ namespace protobuf {
 // Write out the given protobuf to the specified file descriptor by
 // first writing out the length of the protobuf followed by the
 // contents.
-inline Result<bool> write(int fd, const google::protobuf::Message& message)
+inline Try<bool> write(int fd, const google::protobuf::Message& message)
 {
   if (!message.IsInitialized()) {
     LOG(ERROR) << "Failed to write protocol buffer to file, "
@@ -538,7 +699,7 @@ inline Result<bool> write(int fd, const 
   if (length == -1) {
     std::string error = strerror(errno);
     error = error + " (" + __FILE__ + ":" + utils::stringify(__LINE__) + ")";
-    return Result<bool>::error(error);
+    return Try<bool>::error(error);
   }
 
   CHECK(length != 0);
@@ -550,17 +711,16 @@ inline Result<bool> write(int fd, const 
 
 // A wrapper function that wraps the above write() with
 // open and closing the file.
-inline Result<bool> write(const std::string& path,
-                          const google::protobuf::Message& message)
+inline Try<bool> write(const std::string& path,
+                       const google::protobuf::Message& message)
 {
-  Result<int> fd = os::open(path, O_WRONLY | O_CREAT | O_TRUNC,
-                            S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+  Try<int> fd = os::open(path, O_WRONLY | O_CREAT | O_TRUNC,
+                         S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
   if (fd.isError()) {
-    LOG(ERROR) << "Failed to open file: " << path;
-    return Result<bool>::error("Failed to open file.");
+    return Try<bool>::error("Failed to open file " + path);
   }
 
-  Result<bool> result = write(fd.get(), message);
+  Try<bool> result = write(fd.get(), message);
 
   // NOTE: We ignore the return value of close(). This is because users calling
   // this function are interested in the return value of write(). Also an
@@ -654,12 +814,11 @@ inline Result<bool> read(int fd, google:
 inline Result<bool> read(const std::string& path,
                          google::protobuf::Message* message)
 {
-  Result<int> fd = os::open(path, O_RDONLY,
-                            S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+  Try<int> fd = os::open(path, O_RDONLY,
+                         S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
 
   if (fd.isError()) {
-    LOG(ERROR) << "Failed to open file: " << path;
-    return Result<bool>::error("Failed to open file.");
+    return Result<bool>::error("Failed to open file " + path);
   }
 
   Result<bool> result = read(fd.get(), message);
@@ -685,8 +844,8 @@ inline Try<int> download(const std::stri
 #ifndef HAVE_LIBCURL
   return Try<int>::error("Downloading via HTTP/FTP is not supported");
 #else
-  Result<int> fd = utils::os::open(path, O_CREAT | O_WRONLY,
-                                   S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+  Try<int> fd = utils::os::open(path, O_CREAT | O_WRONLY,
+                                S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
 
   CHECK(!fd.isNone());
 

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1342090&r1=1342089&r2=1342090&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Wed May 23 23:37:39 2012
@@ -203,14 +203,12 @@ void Slave::initialize()
   LOG(INFO) << "Slave resources: " << resources;
 
   // Determine our hostname.
-  Result<string> result = utils::os::hostname();
+  Try<string> result = utils::os::hostname();
 
   if (result.isError()) {
     LOG(FATAL) << "Failed to get hostname: " << result.error();
   }
 
-  CHECK(result.isSome());
-
   string hostname = result.get();
 
   // Check and see if we have a different public DNS name. Normally
@@ -734,7 +732,7 @@ void Slave::statusUpdateAcknowledgement(
 //     // Write the update out to disk.
 //     CHECK(stream->acknowledged != NULL);
 
-//     Result<bool> result =
+//     Try<bool> result =
 //       utils::protobuf::write(stream->acknowledged, update);
 
 //     if (result.isError()) {

Modified: incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp?rev=1342090&r1=1342089&r2=1342090&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp Wed May 23 23:37:39 2012
@@ -33,10 +33,8 @@ TEST(ProtobufIOTest, Basic)
 {
   const std::string file = ".protobuf_io_test_basic";
 
-  Result<int> result = Result<int>::none();
-
-  result = utils::os::open(file, O_CREAT | O_WRONLY | O_SYNC,
-                           S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+  Try<int> result = utils::os::open(file, O_CREAT | O_WRONLY | O_SYNC,
+                                    S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
 
   ASSERT_TRUE(result.isSome());
   int fdw = result.get();
@@ -52,7 +50,7 @@ TEST(ProtobufIOTest, Basic)
   for (int i = 0; i < writes; i++) {
     FrameworkID frameworkId;
     frameworkId.set_value(utils::stringify(i));
-    Result<bool> result = utils::protobuf::write(fdw, frameworkId);
+    Try<bool> result = utils::protobuf::write(fdw, frameworkId);
     ASSERT_TRUE(result.isSome());
     EXPECT_TRUE(result.get());
   }

Modified: incubator/mesos/trunk/src/tests/utils_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils_tests.cpp?rev=1342090&r1=1342089&r2=1342090&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/utils_tests.cpp Wed May 23 23:37:39 2012
@@ -38,17 +38,34 @@ static hashset<std::string> listfiles(co
 }
 
 
-TEST(UtilsTest, rmdir)
+class UtilsTest : public ::testing::Test
+{
+protected:
+  virtual void SetUp()
+  {
+    tmpdir = "/tmp/" + UUID::random().toString();
+    mkdir(tmpdir);
+  }
+
+  virtual void TearDown()
+  {
+    rmdir(tmpdir);
+  }
+
+  std::string tmpdir;
+};
+
+
+TEST_F(UtilsTest, rmdir)
 {
   // TODO(John Sirois): It would be good to use something like mkdtemp, but
   //abstract away a proper platform independent /tmp dir.
-  std::string tmpdir = "/tmp/zks-" + UUID::random().toString();
 
   hashset<std::string> emptyListing;
   emptyListing.insert(".");
   emptyListing.insert("..");
 
-  hashset<std::string> expectedListing;
+  hashset<std::string> expectedListing = emptyListing;
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
 
   mkdir(tmpdir + "/a/b/c");
@@ -78,11 +95,61 @@ TEST(UtilsTest, rmdir)
 
   expectedListing = emptyListing;
   EXPECT_EQ(expectedListing, listfiles(tmpdir + "/e/f"));
+}
 
-  rmdir(tmpdir);
 
-  expectedListing.clear();
-  EXPECT_EQ(expectedListing, listfiles(tmpdir));
+TEST_F(UtilsTest, touch)
+{
+  const std::string& testfile  = tmpdir + "/" + UUID::random().toString();
+  Try<bool> result = touch(testfile);
+
+  ASSERT_TRUE(result.get());
+  ASSERT_TRUE(exists(testfile));
+}
+
+
+TEST_F(UtilsTest, readWriteString)
+{
+  const std::string& testfile  = tmpdir + "/" + UUID::random().toString();
+  const std::string& teststr = "test";
+
+  Try<bool> result = write(testfile, teststr);
+  ASSERT_TRUE(result.get());
+
+  Result<std::string> readstr = read(testfile);
+
+  ASSERT_TRUE(readstr.isSome());
+  EXPECT_EQ(teststr, readstr.get());
+}
+
+
+TEST_F(UtilsTest, find)
+{
+  const std::string& testdir  = tmpdir + "/" + UUID::random().toString();
+  const std::string& subdir = testdir + "/test1";
+  ASSERT_TRUE(mkdir(subdir)); // Create the directories.
+
+  // Now write some files.
+  const std::string& file1 = testdir + "/file1.txt";
+  const std::string& file2 = subdir + "/file2.txt";
+  const std::string& file3 = subdir + "/file3.jpg";
+
+  ASSERT_TRUE(touch(file1).get());
+  ASSERT_TRUE(touch(file2).get());
+  ASSERT_TRUE(touch(file3).get());
+
+  // Find "*.txt" files.
+  Try<std::list<std::string> > result = find(testdir, ".txt");
+  ASSERT_TRUE(result.isSome());
+
+  hashset<std::string> files;
+  foreach (const std::string& file, result.get()) {
+    files.insert(file);
+  }
+
+  ASSERT_EQ(2, files.size());
+  ASSERT_TRUE(files.contains(file1));
+  ASSERT_TRUE(files.contains(file2));
 }
 
 } // namespace os