You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by an...@apache.org on 2016/07/09 05:14:53 UTC

[1/8] mesos git commit: Added negative number validation for 'files/read' offset parameter.

Repository: mesos
Updated Branches:
  refs/heads/master 1cbfdc3c1 -> aa17a6572


Added negative number validation for 'files/read' offset parameter.

In this patch:
\- Added negative number validation for 'files/read' offset parameter
in files.cpp. Previously, it used to fail with a 'InternalServerError'.
\- Corrected the indent of negative number check response for
length parameter in files.cpp.

Review: https://reviews.apache.org/r/49796/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/da674fc5
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/da674fc5
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/da674fc5

Branch: refs/heads/master
Commit: da674fc56bdf3a75c8d7d13a8bf214a743e1360d
Parents: 1cbfdc3
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:33:51 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 21:33:51 2016 -0700

----------------------------------------------------------------------
 src/files/files.cpp | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/da674fc5/src/files/files.cpp
----------------------------------------------------------------------
diff --git a/src/files/files.cpp b/src/files/files.cpp
index c8991c4..4b12f93 100644
--- a/src/files/files.cpp
+++ b/src/files/files.cpp
@@ -508,6 +508,11 @@ Future<http::Response> FilesProcess::read(
       return BadRequest("Failed to parse offset: " + result.error() + ".\n");
     }
 
+    if (result.get() < -1) {
+      return BadRequest(strings::format(
+          "Negative offset provided: %d.\n", result.get()).get());
+    }
+
     offset = result.get();
   }
 
@@ -527,8 +532,8 @@ Future<http::Response> FilesProcess::read(
     // read to the end of the file, up to the maximum read length.
     // Will change this logic in MESOS-5334.
     if (result.get() < -1) {
-      return BadRequest(
-        strings::format("Negative length provided: %d.\n", result.get()).get());
+      return BadRequest(strings::format(
+          "Negative length provided: %d.\n", result.get()).get());
     }
 
     if (result.get() > -1){


[4/8] mesos git commit: Added 'read()' method in 'FilesProcess'.

Posted by an...@apache.org.
Added 'read()' method in 'FilesProcess'.

This method is used later for implementing master/agent's
'READ_FILE' call in v1 operator API.

Review: https://reviews.apache.org/r/49243/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/12d15e47
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/12d15e47
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/12d15e47

Branch: refs/heads/master
Commit: 12d15e478a55fe7a786be848244aba3be15966bd
Parents: b7844ae
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:38:19 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 21:38:19 2016 -0700

----------------------------------------------------------------------
 src/files/files.cpp | 177 ++++++++++++++++++++++++++++++-----------------
 src/files/files.hpp |   7 ++
 2 files changed, 119 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/12d15e47/src/files/files.cpp
----------------------------------------------------------------------
diff --git a/src/files/files.cpp b/src/files/files.cpp
index 4b12f93..63f7c34 100644
--- a/src/files/files.cpp
+++ b/src/files/files.cpp
@@ -85,6 +85,7 @@ using process::wait; // Necessary on some OS's to disambiguate.
 using std::list;
 using std::map;
 using std::string;
+using std::tuple;
 using std::vector;
 
 namespace mesos {
@@ -109,6 +110,12 @@ public:
     const string& path,
     const Option<string>& principal);
 
+  Future<Try<tuple<size_t, string>, FilesError>> read(
+      const size_t offset,
+      const Option<size_t>& length,
+      const string& path,
+      const Option<string>& principal);
+
 protected:
   virtual void initialize();
 
@@ -133,18 +140,18 @@ private:
       const http::Request& request,
       const Option<string>& principal);
 
+  // Continuation of `read()`.
+  Future<Try<tuple<size_t, string>, FilesError>> _read(
+      size_t offset,
+      Option<size_t> length,
+      const string& path);
+
   // Reads data from a file at a given offset and for a given length.
   // See the jquery pailer for the expected behavior.
-  Future<http::Response> read(
+  Future<http::Response> __read(
       const http::Request& request,
       const Option<string>& principal);
 
-  Future<http::Response> _read(
-      off_t offset,
-      Option<size_t> length,
-      const string& path,
-      const Option<string>& jsonp);
-
   // Returns the raw file contents for a given path.
   // Requests have the following parameters:
   //   path: The directory to browse. Required.
@@ -202,7 +209,7 @@ void FilesProcess::initialize()
     route("/read.json",
           authenticationRealm.get(),
           FilesProcess::READ_HELP,
-          &FilesProcess::read);
+          &FilesProcess::__read);
     route("/download.json",
           authenticationRealm.get(),
           FilesProcess::DOWNLOAD_HELP,
@@ -219,7 +226,7 @@ void FilesProcess::initialize()
     route("/read",
           authenticationRealm.get(),
           FilesProcess::READ_HELP,
-          &FilesProcess::read);
+          &FilesProcess::__read);
     route("/download",
           authenticationRealm.get(),
           FilesProcess::DOWNLOAD_HELP,
@@ -236,7 +243,7 @@ void FilesProcess::initialize()
           lambda::bind(&FilesProcess::_browse, this, lambda::_1, None()));
     route("/read.json",
           FilesProcess::READ_HELP,
-          lambda::bind(&FilesProcess::read, this, lambda::_1, None()));
+          lambda::bind(&FilesProcess::__read, this, lambda::_1, None()));
     route("/download.json",
           FilesProcess::DOWNLOAD_HELP,
           lambda::bind(&FilesProcess::download, this, lambda::_1, None()));
@@ -249,7 +256,7 @@ void FilesProcess::initialize()
           lambda::bind(&FilesProcess::_browse, this, lambda::_1, None()));
     route("/read",
           FilesProcess::READ_HELP,
-          lambda::bind(&FilesProcess::read, this, lambda::_1, None()));
+          lambda::bind(&FilesProcess::__read, this, lambda::_1, None()));
     route("/download",
           FilesProcess::DOWNLOAD_HELP,
           lambda::bind(&FilesProcess::download, this, lambda::_1, None()));
@@ -449,21 +456,6 @@ Future<Try<list<FileInfo>, FilesError>> FilesProcess::browse(
 }
 
 
-// TODO(benh): Remove 'const &' from size after fixing libprocess.
-Future<http::Response> __read(int fd,
-                       const size_t& size,
-                       off_t offset,
-                       const boost::shared_array<char>& data,
-                       const Option<string>& jsonp) {
-  JSON::Object object;
-
-  object.values["offset"] = offset;
-  object.values["data"] = string(data.get(), size);
-
-  return OK(object, jsonp);
-}
-
-
 const string FilesProcess::READ_HELP = HELP(
     TLDR(
         "Reads data from a file."),
@@ -489,7 +481,7 @@ const string FilesProcess::READ_HELP = HELP(
         "See authorization documentation for details."));
 
 
-Future<http::Response> FilesProcess::read(
+Future<http::Response> FilesProcess::__read(
     const http::Request& request,
     const Option<string>& principal)
 {
@@ -541,39 +533,87 @@ Future<http::Response> FilesProcess::read(
     }
   }
 
-  string requestedPath = path.get();
+  size_t offset_ = offset;
+
+  // The pailer in the webui sends `offset=-1` initially to determine the length
+  // of the file. This is equivalent to making a call to `read()` with an
+  // `offset`/`length` of 0.
+  if (offset == -1) {
+    offset_ = 0;
+    length = 0;
+  }
+
   Option<string> jsonp = request.url.query.get("jsonp");
 
-  return authorize(requestedPath, principal)
+  return read(offset_, length, path.get(), principal)
+    .then([offset, jsonp](const Try<tuple<size_t, string>, FilesError>& result)
+        -> Future<http::Response> {
+      if (result.isError()) {
+        const FilesError& error = result.error();
+
+        switch (error.type) {
+          case FilesError::Type::INVALID:
+            return BadRequest(error.message);
+
+          case FilesError::Type::NOT_FOUND:
+            return NotFound(error.message);
+
+          case FilesError::Type::UNAUTHORIZED:
+            return Forbidden(error.message);
+
+          case FilesError::Type::UNKNOWN:
+            return InternalServerError(error.message);
+        }
+
+        UNREACHABLE();
+      }
+
+      const tuple<size_t, string>& contents = result.get();
+
+      JSON::Object object;
+      object.values["offset"] = (offset == -1) ? std::get<0>(contents) : offset;
+      object.values["data"] = std::get<1>(contents);
+
+      return OK(object, jsonp);
+    });
+}
+
+
+Future<Try<tuple<size_t, string>, FilesError>> FilesProcess::read(
+    const size_t offset,
+    const Option<size_t>& length,
+    const string& path,
+    const Option<string>& principal)
+{
+  return authorize(path, principal)
     .then(defer(self(),
-        [this, offset, length, path, jsonp](bool authorized)
-          -> Future<http::Response> {
-      if (authorized) {
-        return _read(offset, length, path.get(), jsonp);
+        [this, offset, length, path](bool authorized)
+          -> Future<Try<tuple<size_t, string>, FilesError>> {
+      if (!authorized) {
+        return FilesError(FilesError::Type::UNAUTHORIZED);
       }
 
-      return Forbidden();
+      return _read(offset, length, path);
     }));
 }
 
 
-Future<http::Response> FilesProcess::_read(
-    off_t offset,
+Future<Try<tuple<size_t, string>, FilesError>> FilesProcess::_read(
+    size_t offset,
     Option<size_t> length,
-    const string& path,
-    const Option<string>& jsonp)
+    const string& path)
 {
   Result<string> resolvedPath = resolve(path);
 
   if (resolvedPath.isError()) {
-    return BadRequest(resolvedPath.error() + ".\n");
+    return FilesError(FilesError::Type::INVALID, resolvedPath.error() + ".\n");
   } else if (!resolvedPath.isSome()) {
-    return NotFound();
+    return FilesError(FilesError::Type::NOT_FOUND);
   }
 
   // Don't read directories.
   if (os::stat::isdir(resolvedPath.get())) {
-    return BadRequest("Cannot read a directory.\n");
+    return FilesError(FilesError::Type::INVALID, "Cannot read a directory.\n");
   }
 
   // TODO(benh): Cache file descriptors so we aren't constantly
@@ -586,10 +626,10 @@ Future<http::Response> FilesProcess::_read(
         resolvedPath.get(),
         fd.error()).get();
     LOG(WARNING) << error;
-    return InternalServerError(error + ".\n");
+    return FilesError(FilesError::Type::UNKNOWN, error + ".\n");
   }
 
-  off_t size = lseek(fd.get(), 0, SEEK_END);
+  const off_t size = lseek(fd.get(), 0, SEEK_END);
 
   if (size == -1) {
     string error = strings::format(
@@ -599,29 +639,26 @@ Future<http::Response> FilesProcess::_read(
 
     LOG(WARNING) << error;
     os::close(fd.get());
-    return InternalServerError(error + ".\n");
+    return FilesError(FilesError::Type::UNKNOWN, error + ".\n");
   }
 
-  if (offset == -1) {
-    offset = size;
+  if (offset >= static_cast<size_t>(size)) {
+    os::close(fd.get());
+    return std::make_tuple(size, "");
   }
 
   if (length.isNone()) {
     length = size - offset;
   }
 
+  // Return the size of file if length is 0.
+  if (length == 0) {
+    return std::make_tuple(size, "");
+  }
+
   // Cap the read length at 16 pages.
   length = std::min<size_t>(length.get(), os::pagesize() * 16);
 
-  if (offset >= size) {
-    os::close(fd.get());
-
-    JSON::Object object;
-    object.values["offset"] = size;
-    object.values["data"] = "";
-    return OK(object, jsonp);
-  }
-
   // Seek to the offset we want to read from.
   if (lseek(fd.get(), offset, SEEK_SET) == -1) {
     string error = strings::format(
@@ -631,7 +668,7 @@ Future<http::Response> FilesProcess::_read(
 
     LOG(WARNING) << error;
     os::close(fd.get());
-    return InternalServerError(error);
+    return FilesError(FilesError::Type::UNKNOWN, error);
   }
 
   Try<Nothing> nonblock = os::nonblock(fd.get());
@@ -640,21 +677,16 @@ Future<http::Response> FilesProcess::_read(
         "Failed to set file descriptor nonblocking: " + nonblock.error();
     LOG(WARNING) << error;
     os::close(fd.get());
-    return InternalServerError(error);
+    return FilesError(FilesError::Type::UNKNOWN, error);
   }
 
   // Read 'length' bytes (or to EOF).
   boost::shared_array<char> data(new char[length.get()]);
 
   return io::read(fd.get(), data.get(), length.get())
-    .then(lambda::bind(
-        __read,
-        fd.get(),
-        lambda::_1,
-        offset,
-        data,
-        jsonp))
-    .onAny([fd]() { os::close(fd.get()); });
+    .then([size, data, length]() -> Try<tuple<size_t, string>, FilesError> {
+      return std::make_tuple(size, string(data.get(), length.get()));
+    });
 }
 
 
@@ -885,5 +917,20 @@ Future<Try<list<FileInfo>, FilesError>> Files::browse(
   return dispatch(process, &FilesProcess::browse, path, principal);
 }
 
+
+Future<Try<tuple<size_t, string>, FilesError>> Files::read(
+    const size_t offset,
+    const Option<size_t>& length,
+    const string& path,
+    const Option<string>& principal)
+{
+  return dispatch(process,
+                  &FilesProcess::read,
+                  offset,
+                  length,
+                  path,
+                  principal);
+}
+
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/12d15e47/src/files/files.hpp
----------------------------------------------------------------------
diff --git a/src/files/files.hpp b/src/files/files.hpp
index c36ecfb..8cffc26 100644
--- a/src/files/files.hpp
+++ b/src/files/files.hpp
@@ -94,6 +94,13 @@ public:
       const std::string& path,
       const Option<std::string>& principal);
 
+  // Returns the size and data of file.
+  process::Future<Try<std::tuple<size_t, std::string>, FilesError>> read(
+      const size_t offset,
+      const Option<size_t>& length,
+      const std::string& path,
+      const Option<std::string>& principal);
+
 private:
   FilesProcess* process;
 };


[5/8] mesos git commit: Implemented 'READ_FILE' for v1 Master API.

Posted by an...@apache.org.
Implemented 'READ_FILE' for v1 Master API.

Review: https://reviews.apache.org/r/49244/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/91308287
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/91308287
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/91308287

Branch: refs/heads/master
Commit: 913082877c8f6c27839d8ad15a133a06ff2dcd20
Parents: 12d15e4
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:43:04 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 21:59:31 2016 -0700

----------------------------------------------------------------------
 src/master/http.cpp     |  52 +++++++++++++++++++-
 src/master/master.hpp   |   5 ++
 src/tests/api_tests.cpp | 111 +++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 167 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/91308287/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 8b7e917..42ba364 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -520,7 +520,7 @@ Future<Response> Master::Http::api(
       return listFiles(call, principal, acceptType);
 
     case mesos::master::Call::READ_FILE:
-      return NotImplemented();
+      return readFile(call, principal, acceptType);
 
     case mesos::master::Call::GET_STATE:
       return getState(call, principal, acceptType);
@@ -2755,6 +2755,56 @@ Future<Response> Master::Http::state(
 }
 
 
+Future<Response> Master::Http::readFile(
+    const mesos::master::Call& call,
+    const Option<string>& principal,
+    ContentType contentType) const
+{
+  CHECK_EQ(mesos::master::Call::READ_FILE, call.type());
+
+  const size_t offset = call.read_file().offset();
+  const string& path = call.read_file().path();
+
+  Option<size_t> length;
+  if (call.read_file().has_length()) {
+    length = call.read_file().length();
+  }
+
+  return master->files->read(offset, length, path, principal)
+    .then([contentType](const Try<tuple<size_t, string>, FilesError>& result)
+        -> Future<Response> {
+      if (result.isError()) {
+        const FilesError& error = result.error();
+
+        switch (error.type) {
+          case FilesError::Type::INVALID:
+            return BadRequest(error.message);
+
+          case FilesError::Type::UNAUTHORIZED:
+            return Forbidden(error.message);
+
+          case FilesError::Type::NOT_FOUND:
+            return NotFound(error.message);
+
+          case FilesError::Type::UNKNOWN:
+            return InternalServerError(error.message);
+        }
+
+        UNREACHABLE();
+      }
+
+      mesos::master::Response response;
+      response.set_type(mesos::master::Response::READ_FILE);
+
+      response.mutable_read_file()->set_size(std::get<0>(result.get()));
+      response.mutable_read_file()->set_data(std::get<1>(result.get()));
+
+      return OK(serialize(contentType, evolve(response)),
+                stringify(contentType));
+    });
+}
+
+
 // This abstraction has no side-effects. It factors out computing the
 // mapping from 'slaves' to 'frameworks' to answer the questions 'what
 // frameworks are running on a given slave?' and 'what slaves are

http://git-wip-us.apache.org/repos/asf/mesos/blob/91308287/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 845f2f6..ac998b1 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1521,6 +1521,11 @@ private:
         const Option<std::string>& principal,
         ContentType contentType) const;
 
+    process::Future<process::http::Response> readFile(
+        const mesos::master::Call& call,
+        const Option<std::string>& principal,
+        ContentType contentType) const;
+
     Master* master;
 
     // NOTE: The quota specific pieces of the Operator API are factored

http://git-wip-us.apache.org/repos/asf/mesos/blob/91308287/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index fe26351..239ab50 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -1972,6 +1972,117 @@ TEST_P(MasterAPITest, UpdateWeights)
 }
 
 
+// This test verifies if we can retrieve file data in the master.
+TEST_P(MasterAPITest, ReadFile)
+{
+  Files files;
+
+  // Now write a file.
+  ASSERT_SOME(os::write("file", "body"));
+  AWAIT_EXPECT_READY(files.attach("file", "myname"));
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  ContentType contentType = GetParam();
+
+  {
+    v1::master::Call v1Call;
+    v1Call.set_type(v1::master::Call::READ_FILE);
+
+    v1::master::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(1);
+    readFile->set_length(2);
+    readFile->set_path("myname");
+
+    Future<v1::master::Response> v1Response =
+        post(master.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::master::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("od", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+
+  // Read the file with `offset >= size`. This should return the size of file
+  // and empty data.
+  {
+    v1::master::Call v1Call;
+    v1Call.set_type(v1::master::Call::READ_FILE);
+
+    v1::master::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(5);
+    readFile->set_length(2);
+    readFile->set_path("myname");
+
+    Future<v1::master::Response> v1Response =
+        post(master.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::master::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+
+  // Read the file without length being set and `offset=0`. This should read
+  // the entire file.
+  {
+    v1::master::Call v1Call;
+    v1Call.set_type(v1::master::Call::READ_FILE);
+
+    v1::master::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(0);
+    readFile->set_path("myname");
+
+    Future<v1::master::Response> v1Response =
+        post(master.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::master::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("body", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+}
+
+
+// This test verifies that the client will receive a `NotFound` response when
+// it tries to make a `READ_FILE` call with an invalid path.
+TEST_P(MasterAPITest, ReadFileInvalidPath)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  // Read an invalid file.
+  v1::master::Call v1Call;
+  v1Call.set_type(v1::master::Call::READ_FILE);
+
+  v1::master::Call::ReadFile* readFile = v1Call.mutable_read_file();
+  readFile->set_offset(1);
+  readFile->set_length(2);
+  readFile->set_path("invalid_file");
+
+  ContentType contentType = GetParam();
+
+  Future<Response> response = process::http::post(
+    master.get()->pid,
+    "api/v1",
+    createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+    serialize(contentType, v1Call),
+    stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(NotFound().status, response);
+}
+
+
 class AgentAPITest
   : public MesosTest,
     public WithParamInterface<ContentType>


[2/8] mesos git commit: Updated v1 operator 'Call::ReadFile' message.

Posted by an...@apache.org.
Updated v1 operator 'Call::ReadFile' message.

Updated the type of offset and length of 'Call::ReadFile'
for v1 operator API.

Review: https://reviews.apache.org/r/49679/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/f2a8efbc
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/f2a8efbc
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/f2a8efbc

Branch: refs/heads/master
Commit: f2a8efbc275b287313e76e02468304fa5035b3db
Parents: da674fc
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:37:01 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 21:37:01 2016 -0700

----------------------------------------------------------------------
 include/mesos/agent/agent.proto      | 13 ++++++++++---
 include/mesos/master/master.proto    | 13 ++++++++++---
 include/mesos/v1/agent/agent.proto   | 13 ++++++++++---
 include/mesos/v1/master/master.proto | 13 ++++++++++---
 4 files changed, 40 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f2a8efbc/include/mesos/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/agent/agent.proto b/include/mesos/agent/agent.proto
index ef07b1f..1342293 100644
--- a/include/mesos/agent/agent.proto
+++ b/include/mesos/agent/agent.proto
@@ -45,7 +45,7 @@ message Call {
     SET_LOGGING_LEVEL = 6;  // See 'SetLoggingLevel' below.
 
     LIST_FILES = 7;
-    READ_FILE = 8;
+    READ_FILE = 8;          // See 'ReadFile' below.
 
     GET_STATE = 9;
 
@@ -77,10 +77,17 @@ message Call {
     required string path = 1;
   }
 
+  // Reads data from a file.
   message ReadFile {
+    // The path of file.
     required string path = 1;
-    optional string offset = 2;
-    optional string length = 3;
+
+    // Initial offset in file to start reading from.
+    required uint64 offset = 2;
+
+    // The maximum number of bytes to read. The read length is capped at 16
+    // memory pages.
+    optional uint64 length = 3;
   }
 
   optional Type type = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f2a8efbc/include/mesos/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/master/master.proto b/include/mesos/master/master.proto
index d82b245..25edb5a 100644
--- a/include/mesos/master/master.proto
+++ b/include/mesos/master/master.proto
@@ -51,7 +51,7 @@ message Call {
     SET_LOGGING_LEVEL = 6;  // See 'SetLoggingLevel' below.
 
     LIST_FILES = 7;
-    READ_FILE = 8;
+    READ_FILE = 8;          // See 'ReadFile' below.
 
     GET_STATE = 9;
     GET_STATE_SUMMARY = 10;
@@ -113,10 +113,17 @@ message Call {
     required string path = 1;
   }
 
+  // Reads data from a file.
   message ReadFile {
+    // The path of file.
     required string path = 1;
-    optional string offset = 2;
-    optional string length = 3;
+
+    // Initial offset in file to start reading from.
+    required uint64 offset = 2;
+
+    // The maximum number of bytes to read. The read length is capped at 16
+    // memory pages.
+    optional uint64 length = 3;
   }
 
   message UpdateWeights {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f2a8efbc/include/mesos/v1/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/agent/agent.proto b/include/mesos/v1/agent/agent.proto
index 8f84508..f64dbd2 100644
--- a/include/mesos/v1/agent/agent.proto
+++ b/include/mesos/v1/agent/agent.proto
@@ -45,7 +45,7 @@ message Call {
     SET_LOGGING_LEVEL = 6;  // See 'SetLoggingLevel' below.
 
     LIST_FILES = 7;
-    READ_FILE = 8;
+    READ_FILE = 8;          // See 'ReadFile' below.
 
     GET_STATE = 9;
 
@@ -77,10 +77,17 @@ message Call {
     required string path = 1;
   }
 
+  // Reads data from a file.
   message ReadFile {
+    // The path of file.
     required string path = 1;
-    optional string offset = 2;
-    optional string length = 3;
+
+    // Initial offset in file to start reading from.
+    required uint64 offset = 2;
+
+    // The maximum number of bytes to read. The read length is capped at 16
+    // memory pages.
+    optional uint64 length = 3;
   }
 
   optional Type type = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f2a8efbc/include/mesos/v1/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/master/master.proto b/include/mesos/v1/master/master.proto
index 711a664..aa1e8dd 100644
--- a/include/mesos/v1/master/master.proto
+++ b/include/mesos/v1/master/master.proto
@@ -51,7 +51,7 @@ message Call {
     SET_LOGGING_LEVEL = 6;  // See 'SetLoggingLevel' below.
 
     LIST_FILES = 7;
-    READ_FILE = 8;
+    READ_FILE = 8;          // See 'ReadFile' below.
 
     GET_STATE = 9;
     GET_STATE_SUMMARY = 10;
@@ -113,10 +113,17 @@ message Call {
     required string path = 1;
   }
 
+  // Reads data from a file.
   message ReadFile {
+    // The path of file.
     required string path = 1;
-    optional string offset = 2;
-    optional string length = 3;
+
+    // Initial offset in file to start reading from.
+    required uint64 offset = 2;
+
+    // The maximum number of bytes to read. The read length is capped at 16
+    // memory pages.
+    optional uint64 length = 3;
   }
 
   message UpdateWeights {


[3/8] mesos git commit: Added 'ReadFile' response message in v1 Operator API.

Posted by an...@apache.org.
Added 'ReadFile' response message in v1 Operator API.

Review: https://reviews.apache.org/r/49242/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/b7844ae2
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/b7844ae2
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/b7844ae2

Branch: refs/heads/master
Commit: b7844ae25ecbbb051a1f8b4d046c0668d377592f
Parents: f2a8efb
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:37:35 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 21:37:35 2016 -0700

----------------------------------------------------------------------
 include/mesos/agent/agent.proto      | 12 ++++++++----
 include/mesos/master/master.proto    | 12 ++++++++----
 include/mesos/v1/agent/agent.proto   | 12 ++++++++----
 include/mesos/v1/master/master.proto | 12 ++++++++----
 4 files changed, 32 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b7844ae2/include/mesos/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/agent/agent.proto b/include/mesos/agent/agent.proto
index 1342293..cfd117d 100644
--- a/include/mesos/agent/agent.proto
+++ b/include/mesos/agent/agent.proto
@@ -115,7 +115,7 @@ message Response {
     GET_LOGGING_LEVEL = 5;         // See 'GetLoggingLevel' below.
 
     LIST_FILES = 6;
-    READ_FILE = 7;
+    READ_FILE = 7;                 // See 'ReadFile' below.
 
     GET_STATE = 8;
 
@@ -153,8 +153,12 @@ message Response {
     repeated FileInfo file_infos = 1;
   }
 
-  message GetFileContents {
-    repeated bytes data = 1;
+  // Contains the file data.
+  message ReadFile {
+    // The size of file (in bytes).
+    required uint64 size = 1;
+
+    required bytes data = 2;
   }
 
   message GetState {
@@ -185,7 +189,7 @@ message Response {
   optional GetMetrics get_metrics = 5;
   optional GetLoggingLevel get_logging_level = 6;
   optional ListFiles list_files = 7;
-  optional GetFileContents get_file_contents = 8;
+  optional ReadFile read_file = 8;
   optional GetState get_state = 9;
   optional GetContainers get_containers = 10;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b7844ae2/include/mesos/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/master/master.proto b/include/mesos/master/master.proto
index 25edb5a..7fb7153 100644
--- a/include/mesos/master/master.proto
+++ b/include/mesos/master/master.proto
@@ -222,7 +222,7 @@ message Response {
     GET_LOGGING_LEVEL = 5;         // See 'GetLoggingLevel' below.
 
     LIST_FILES = 6;
-    READ_FILE = 7;
+    READ_FILE = 7;                 // See 'ReadFile' below.
 
     GET_STATE = 8;
     GET_STATE_SUMMARY = 9;
@@ -274,8 +274,12 @@ message Response {
     repeated FileInfo file_infos = 1;
   }
 
-  message GetFileContents {
-    repeated bytes data = 1;
+  // Contains the file data.
+  message ReadFile {
+    // The size of file (in bytes).
+    required uint64 size = 1;
+
+    required bytes data = 2;
   }
 
   // Contains full state of the master i.e. information about the tasks,
@@ -422,7 +426,7 @@ message Response {
   optional GetMetrics get_metrics = 5;
   optional GetLoggingLevel get_logging_level = 6;
   optional ListFiles list_files = 7;
-  optional GetFileContents get_file_contents = 8;
+  optional ReadFile read_file = 8;
   optional GetState get_state = 9;
   optional GetStateSummary get_state_summary = 10;
   optional GetAgents get_agents = 11;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b7844ae2/include/mesos/v1/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/agent/agent.proto b/include/mesos/v1/agent/agent.proto
index f64dbd2..213c428 100644
--- a/include/mesos/v1/agent/agent.proto
+++ b/include/mesos/v1/agent/agent.proto
@@ -115,7 +115,7 @@ message Response {
     GET_LOGGING_LEVEL = 5;         // See 'GetLoggingLevel' below.
 
     LIST_FILES = 6;
-    READ_FILE = 7;
+    READ_FILE = 7;                 // See 'ReadFile' below.
 
     GET_STATE = 8;
 
@@ -153,8 +153,12 @@ message Response {
     repeated FileInfo file_infos = 1;
   }
 
-  message GetFileContents {
-    repeated bytes data = 1;
+  // Contains the file data.
+  message ReadFile {
+    // The size of file (in bytes).
+    required uint64 size = 1;
+
+    required bytes data = 2;
   }
 
   message GetState {
@@ -185,7 +189,7 @@ message Response {
   optional GetMetrics get_metrics = 5;
   optional GetLoggingLevel get_logging_level = 6;
   optional ListFiles list_files = 7;
-  optional GetFileContents get_file_contents = 8;
+  optional ReadFile read_file = 8;
   optional GetState get_state = 9;
   optional GetContainers get_containers = 10;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b7844ae2/include/mesos/v1/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/master/master.proto b/include/mesos/v1/master/master.proto
index aa1e8dd..c9e4359 100644
--- a/include/mesos/v1/master/master.proto
+++ b/include/mesos/v1/master/master.proto
@@ -223,7 +223,7 @@ message Response {
     GET_LOGGING_LEVEL = 5;         // See 'GetLoggingLevel' below.
 
     LIST_FILES = 6;
-    READ_FILE = 7;
+    READ_FILE = 7;                 // See 'ReadFile' below.
 
     GET_STATE = 8;
     GET_STATE_SUMMARY = 9;
@@ -275,8 +275,12 @@ message Response {
     repeated FileInfo file_infos = 1;
   }
 
-  message GetFileContents {
-    repeated bytes data = 1;
+  // Contains the file data.
+  message ReadFile {
+    // The size of file (in bytes).
+    required uint64 size = 1;
+
+    required bytes data = 2;
   }
 
   // Contains full state of the master i.e. information about the tasks,
@@ -423,7 +427,7 @@ message Response {
   optional GetMetrics get_metrics = 5;
   optional GetLoggingLevel get_logging_level = 6;
   optional ListFiles list_files = 7;
-  optional GetFileContents get_file_contents = 8;
+  optional ReadFile read_file = 8;
   optional GetState get_state = 9;
   optional GetStateSummary get_state_summary = 10;
   optional GetAgents get_agents = 11;


[6/8] mesos git commit: Implemented 'READ_FILE' for v1 Agent API.

Posted by an...@apache.org.
Implemented 'READ_FILE' for v1 Agent API.

Review: https://reviews.apache.org/r/49245/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/4c6bf64e
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/4c6bf64e
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/4c6bf64e

Branch: refs/heads/master
Commit: 4c6bf64edfecd90d909cd923a0898298c8e09813
Parents: 9130828
Author: zhou xing <xi...@cn.ibm.com>
Authored: Fri Jul 8 21:43:31 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 22:03:04 2016 -0700

----------------------------------------------------------------------
 src/slave/http.cpp      |  52 ++++++++++++++++-
 src/slave/slave.hpp     |   5 ++
 src/tests/api_tests.cpp | 129 +++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 185 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4c6bf64e/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 362cf17..21c7ebf 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -382,7 +382,7 @@ Future<Response> Slave::Http::api(
       return listFiles(call, principal, acceptType);
 
     case agent::Call::READ_FILE:
-      return NotImplemented();
+      return readFile(call, principal, acceptType);
 
     case agent::Call::GET_STATE:
       return NotImplemented();
@@ -1387,6 +1387,56 @@ Try<string> Slave::Http::extractEndpoint(const process::http::URL& url) const
   return "/" + pathComponents[1];
 }
 
+
+Future<Response> Slave::Http::readFile(
+    const mesos::agent::Call& call,
+    const Option<string>& principal,
+    ContentType contentType) const
+{
+  CHECK_EQ(mesos::agent::Call::READ_FILE, call.type());
+
+  const size_t offset = call.read_file().offset();
+  const string& path = call.read_file().path();
+
+  Option<size_t> length;
+  if (call.read_file().has_length()) {
+    length = call.read_file().length();
+  }
+
+  return slave->files->read(offset, length, path, principal)
+    .then([contentType](const Try<tuple<size_t, string>, FilesError>& result)
+        -> Future<Response> {
+      if (result.isError()) {
+        const FilesError& error = result.error();
+
+        switch (error.type) {
+          case FilesError::Type::INVALID:
+            return BadRequest(error.message);
+
+          case FilesError::Type::UNAUTHORIZED:
+            return Forbidden(error.message);
+
+          case FilesError::Type::NOT_FOUND:
+            return NotFound(error.message);
+
+          case FilesError::Type::UNKNOWN:
+            return InternalServerError(error.message);
+        }
+
+        UNREACHABLE();
+      }
+
+      mesos::agent::Response response;
+      response.set_type(mesos::agent::Response::READ_FILE);
+
+      response.mutable_read_file()->set_size(std::get<0>(result.get()));
+      response.mutable_read_file()->set_data(std::get<1>(result.get()));
+
+      return OK(serialize(contentType, evolve(response)),
+                stringify(contentType));
+    });
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4c6bf64e/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index a8952f0..42afa9e 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -544,6 +544,11 @@ private:
         const Option<std::string>& principal,
         ContentType contentType) const;
 
+    process::Future<process::http::Response> readFile(
+        const mesos::agent::Call& call,
+        const Option<std::string>& principal,
+        ContentType contentType) const;
+
     Slave* slave;
 
     // Used to rate limit the statistics endpoint.

http://git-wip-us.apache.org/repos/asf/mesos/blob/4c6bf64e/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 239ab50..b9d4fe1 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -2512,6 +2512,135 @@ TEST_P(AgentAPITest, GetContainers)
   driver.join();
 }
 
+
+// This test verifies if we can retrieve file data in the agent.
+TEST_P(AgentAPITest, ReadFile)
+{
+  Files files;
+
+  // Now write a file.
+  ASSERT_SOME(os::write("file", "body"));
+  AWAIT_EXPECT_READY(files.attach("file", "myname"));
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+
+  StandaloneMasterDetector detector;
+  Try<Owned<cluster::Slave>> slave = StartSlave(&detector);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(__recover);
+
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
+  ContentType contentType = GetParam();
+
+  {
+    v1::agent::Call v1Call;
+    v1Call.set_type(v1::agent::Call::READ_FILE);
+
+    v1::agent::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(1);
+    readFile->set_length(2);
+    readFile->set_path("myname");
+
+    Future<v1::agent::Response> v1Response =
+        post(slave.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::agent::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("od", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+
+  // Read the file with `offset >= size`. This should return the size of file
+  // and empty data.
+  {
+    v1::agent::Call v1Call;
+    v1Call.set_type(v1::agent::Call::READ_FILE);
+
+    v1::agent::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(5);
+    readFile->set_length(2);
+    readFile->set_path("myname");
+
+    Future<v1::agent::Response> v1Response =
+        post(slave.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::agent::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+
+  // Read the file without length being set and `offset=0`. This should read
+  // the entire file.
+  {
+    v1::agent::Call v1Call;
+    v1Call.set_type(v1::agent::Call::READ_FILE);
+
+    v1::agent::Call::ReadFile* readFile = v1Call.mutable_read_file();
+    readFile->set_offset(0);
+    readFile->set_path("myname");
+
+    Future<v1::agent::Response> v1Response =
+        post(slave.get()->pid, v1Call, contentType);
+
+    AWAIT_READY(v1Response);
+
+    ASSERT_TRUE(v1Response.get().IsInitialized());
+    ASSERT_EQ(v1::agent::Response::READ_FILE, v1Response.get().type());
+
+    ASSERT_EQ("body", v1Response.get().read_file().data());
+    ASSERT_EQ(4, v1Response.get().read_file().size());
+  }
+}
+
+
+// This test verifies that the client will receive a `NotFound` response when
+// it tries to make a `READ_FILE` call with an invalid path.
+TEST_P(AgentAPITest, ReadFileInvalidPath)
+{
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+
+  StandaloneMasterDetector detector;
+  Try<Owned<cluster::Slave>> slave = StartSlave(&detector);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(__recover);
+
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
+  // Read an invalid file.
+  v1::agent::Call v1Call;
+  v1Call.set_type(v1::agent::Call::READ_FILE);
+
+  v1::agent::Call::ReadFile* readFile = v1Call.mutable_read_file();
+  readFile->set_offset(1);
+  readFile->set_length(2);
+  readFile->set_path("invalid_file");
+
+  ContentType contentType = GetParam();
+
+  Future<Response> response = process::http::post(
+    slave.get()->pid,
+    "api/v1",
+    createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+    serialize(contentType, v1Call),
+    stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(NotFound().status, response);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[8/8] mesos git commit: Fixed minor spacing issue in declaration of `browse()`.

Posted by an...@apache.org.
Fixed minor spacing issue in declaration of `browse()`.


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/aa17a657
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/aa17a657
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/aa17a657

Branch: refs/heads/master
Commit: aa17a6572ebb8650ec802a13b8bf0b50a8f0daf7
Parents: f4bbd86
Author: Anand Mazumdar <an...@apache.org>
Authored: Fri Jul 8 21:48:38 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 22:03:05 2016 -0700

----------------------------------------------------------------------
 src/files/files.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/aa17a657/src/files/files.cpp
----------------------------------------------------------------------
diff --git a/src/files/files.cpp b/src/files/files.cpp
index 63f7c34..8ab8f8a 100644
--- a/src/files/files.cpp
+++ b/src/files/files.cpp
@@ -107,8 +107,8 @@ public:
   void detach(const string& name);
 
   Future<Try<list<FileInfo>, FilesError>> browse(
-    const string& path,
-    const Option<string>& principal);
+      const string& path,
+      const Option<string>& principal);
 
   Future<Try<tuple<size_t, string>, FilesError>> read(
       const size_t offset,


[7/8] mesos git commit: Added a `Clock::settle()` to ensure agent has finished recovery.

Posted by an...@apache.org.
Added a `Clock::settle()` to ensure agent has finished recovery.

We were waiting on \`\_\_recover()\` to be invoked which does
\_not\_ mean that the function has completed. This can lead to
some flakiness i.e. a \`503 xx\` when the `post()` happens before
the agent has finished recovery e.g., due to a context switch.

Review: https://reviews.apache.org/r/49780/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/f4bbd862
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/f4bbd862
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/f4bbd862

Branch: refs/heads/master
Commit: f4bbd8620df2c8feef676ab6d729d56cf6b2d733
Parents: 4c6bf64
Author: Anand Mazumdar <an...@apache.org>
Authored: Fri Jul 8 21:44:19 2016 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Fri Jul 8 22:03:05 2016 -0700

----------------------------------------------------------------------
 src/tests/api_tests.cpp | 30 ++++++++++++++++++++++++------
 1 file changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f4bbd862/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index b9d4fe1..55e825e 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -2130,9 +2130,12 @@ TEST_P(AgentAPITest, GetFlags)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   v1::agent::Call v1Call;
   v1Call.set_type(v1::agent::Call::GET_FLAGS);
 
@@ -2155,9 +2158,12 @@ TEST_P(AgentAPITest, GetHealth)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   v1::agent::Call v1Call;
   v1Call.set_type(v1::agent::Call::GET_HEALTH);
 
@@ -2181,9 +2187,12 @@ TEST_P(AgentAPITest, GetVersion)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   v1::agent::Call v1Call;
   v1Call.set_type(v1::agent::Call::GET_VERSION);
 
@@ -2209,9 +2218,12 @@ TEST_P(AgentAPITest, GetMetrics)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   Duration timeout = Seconds(5);
 
   v1::agent::Call v1Call;
@@ -2249,9 +2261,12 @@ TEST_P(AgentAPITest, GetLoggingLevel)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   v1::agent::Call v1Call;
   v1Call.set_type(v1::agent::Call::GET_LOGGING_LEVEL);
 
@@ -2279,9 +2294,12 @@ TEST_P(AgentAPITest, SetLoggingLevel)
   Try<Owned<cluster::Slave>> slave = this->StartSlave(&detector);
   ASSERT_SOME(slave);
 
-  // Wait until the agent has finished recovery.
   AWAIT_READY(__recover);
 
+  // Wait until the agent has finished recovery.
+  Clock::pause();
+  Clock::settle();
+
   // We capture the original logging level first; it would be used to verify
   // the logging level revert works.
   uint32_t originalLevel = static_cast<uint32_t>(FLAGS_v);