You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ya...@apache.org on 2016/04/23 07:40:07 UTC

mesos git commit: Add subdirectory support to URI.output_file field.

Repository: mesos
Updated Branches:
  refs/heads/master 653eca74f -> 3d2d3edb9


Add subdirectory support to URI.output_file field.

URI.output_file allows the user to specify the path of the file that'll
be saved in the sandbox when the URI is fetched, but previously it would
fail at fetch time if "filename" had a directory component. This change
allows users to specify a relative path for custom ouput targets within
the sandbox.

The protobuf field 'filename' is renamed 'output_file' due to this
usage change.

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


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

Branch: refs/heads/master
Commit: 3d2d3edb96c9c428f988653cdd4428a05690e747
Parents: 653eca7
Author: Michael Browning <mr...@uber.com>
Authored: Fri Apr 22 17:48:24 2016 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Fri Apr 22 22:34:16 2016 -0700

----------------------------------------------------------------------
 CHANGELOG                           |  2 +-
 docs/fetcher.md                     |  7 +--
 include/mesos/mesos.proto           |  6 ++-
 include/mesos/v1/mesos.proto        |  6 ++-
 src/launcher/fetcher.cpp            | 49 +++++++++++++-----
 src/slave/containerizer/fetcher.cpp | 24 ++++++---
 src/slave/containerizer/fetcher.hpp |  4 +-
 src/tests/fetcher_cache_tests.cpp   | 53 ++++++++++++++++++--
 src/tests/fetcher_tests.cpp         | 85 ++++++++++++++++++++++++++++----
 9 files changed, 194 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index d2e902f..57f5457 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -47,7 +47,7 @@ Deprecations:
 Additional API Changes:
   * [MESOS-4580] - Returning `202` (Accepted) for /reserve and related endpoints.
 
-  * [MESOS-4735] - Add 'filename' field to CommandInfo.URI in Scheduler API
+  * [MESOS-4735] - Add 'output_file' field to CommandInfo.URI in Scheduler API
     and v1 Scheduler HTTP API.
 
   * [MESOS-5014] - Changes Call and Event Type enums in scheduler.proto

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/docs/fetcher.md
----------------------------------------------------------------------
diff --git a/docs/fetcher.md b/docs/fetcher.md
index fd6d8a7..b23fdf2 100644
--- a/docs/fetcher.md
+++ b/docs/fetcher.md
@@ -86,8 +86,9 @@ has no effect.
 
 If the "cache" field is true, the fetcher cache is to be used for the URI.
 
-If the "filename" field is set, the fetcher will use that name for the copy
-stored in the sandbox directory.
+If the "output_file" field is set, the fetcher will use that name for the copy
+stored in the sandbox directory. "output_file" may contain a directory
+component, in which case the path described must be a relative path.
 
 ### Specifying a user name
 
@@ -131,7 +132,7 @@ In case the cache is bypassed, both the archive and the unpacked results will be
 found together in the sandbox. In case a cache file is unpacked, only the
 extraction result will be found in the sandbox.
 
-The "filename" field is useful here for cases where the URI ends with query
+The "output_file" field is useful here for cases where the URI ends with query
 parameters, since these will otherwise end up in the file copied to the sandbox
 and will subsequently fail to be recognized as archives.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 87af4a0..9a18030 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -416,8 +416,10 @@ message CommandInfo {
 
     // The fetcher's default behavior is to use the URI string's basename to
     // name the local copy. If this field is provided, the local copy will be
-    // named with its value instead.
-    optional string filename = 5;
+    // named with its value instead. If there is a directory component (which
+    // must be a relative path), the local copy will be stored in that
+    // subdirectory inside the sandbox.
+    optional string output_file = 5;
   }
 
   repeated URI uris = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 34da0a1..44b4f8a 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -416,8 +416,10 @@ message CommandInfo {
 
     // The fetcher's default behavior is to use the URI string's basename to
     // name the local copy. If this field is provided, the local copy will be
-    // named with its value instead.
-    optional string filename = 5;
+    // named with its value instead. If there is a directory component (which
+    // must be a relative path), the local copy will be stored in that
+    // subdirectory inside the sandbox.
+    optional string output_file = 5;
   }
 
   repeated URI uris = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/src/launcher/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/fetcher.cpp b/src/launcher/fetcher.cpp
index 47583ee..d323f63 100644
--- a/src/launcher/fetcher.cpp
+++ b/src/launcher/fetcher.cpp
@@ -249,16 +249,30 @@ static Try<string> fetchBypassingCache(
 {
   LOG(INFO) << "Fetching directly into the sandbox directory";
 
-  Try<string> basename = uri.has_filename()
-    ? uri.filename()
+  // TODO(mrbrowning): Factor out duplicated processing of "output_file" field
+  // here and in fetchFromCache into a separate helper function.
+  if (uri.has_output_file()) {
+    string dirname = Path(uri.output_file()).dirname();
+    if (dirname != ".") {
+      Try<Nothing> result =
+        os::mkdir(path::join(sandboxDirectory, dirname), true);
+
+      if (result.isError()) {
+        return Error(
+            "Unable to create subdirectory " + dirname + " in sandbox");
+      }
+    }
+  }
+
+  Try<string> outputFile = uri.has_output_file()
+    ? uri.output_file()
     : Fetcher::basename(uri.value());
 
-  if (basename.isError()) {
-    return Error("Failed to determine the basename of the URI '" +
-                 uri.value() + "' with error: " + basename.error());
+  if (outputFile.isError()) {
+    return Error(outputFile.error());
   }
 
-  string path = path::join(sandboxDirectory, basename.get());
+  string path = path::join(sandboxDirectory, outputFile.get());
 
   Try<string> downloaded = download(uri.value(), path, frameworksHome);
   if (downloaded.isError()) {
@@ -291,15 +305,28 @@ static Try<string> fetchFromCache(
 {
   LOG(INFO) << "Fetching from cache";
 
-  Try<string> basename = item.uri().has_filename()
-    ? item.uri().filename()
+  if (item.uri().has_output_file()) {
+    string dirname = Path(item.uri().output_file()).dirname();
+    if (dirname != ".") {
+      Try<Nothing> result =
+        os::mkdir(path::join(sandboxDirectory, dirname), true);
+
+      if (result.isError()) {
+        return Error(
+          "Unable to create subdirectory " + dirname + "in sandbox");
+      }
+    }
+  }
+
+  Try<string> outputFile = item.uri().has_output_file()
+    ? item.uri().output_file()
     : Fetcher::basename(item.uri().value());
 
-  if (basename.isError()) {
-    return Error(basename.error());
+  if (outputFile.isError()) {
+    return Error(outputFile.error());
   }
 
-  string destinationPath = path::join(sandboxDirectory, basename.get());
+  string destinationPath = path::join(sandboxDirectory, outputFile.get());
 
   // Non-empty cache filename is guaranteed by the callers of this function.
   CHECK(!item.cache_filename().empty());

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/src/slave/containerizer/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.cpp b/src/slave/containerizer/fetcher.cpp
index d5910ad..176d886 100644
--- a/src/slave/containerizer/fetcher.cpp
+++ b/src/slave/containerizer/fetcher.cpp
@@ -145,13 +145,23 @@ Try<Nothing> Fetcher::validateUri(const string& uri)
 }
 
 
-Try<Nothing> Fetcher::validateFilename(const string& filename)
+Try<Nothing> Fetcher::validateOutputFile(const string& path)
 {
-  Try<string> result = Path(filename).basename();
+  Try<string> result = Path(path).basename();
   if (result.isError()) {
     return Error(result.error());
   }
 
+  if (path.size() == 0) {
+    return Error("URI output file path is empty");
+  }
+
+  // TODO(mrbrowning): Check that the filename's directory component is
+  // actually a subdirectory of the sandbox, not just relative to it.
+  if (path.at(0) == '/') {
+    return Error("URI output file must be within the sandbox directory");
+  }
+
   return Nothing();
 }
 
@@ -164,11 +174,11 @@ static Try<Nothing> validateUris(const CommandInfo& commandInfo)
       return Error(uriValidation.error());
     }
 
-    if (uri.has_filename()) {
-      Try<Nothing> filenameValidation =
-        Fetcher::validateFilename(uri.filename());
-      if (filenameValidation.isError()) {
-        return Error(filenameValidation.error());
+    if (uri.has_output_file()) {
+      Try<Nothing> outputFileValidation =
+        Fetcher::validateOutputFile(uri.output_file());
+      if (outputFileValidation.isError()) {
+        return Error(outputFileValidation.error());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/src/slave/containerizer/fetcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.hpp b/src/slave/containerizer/fetcher.hpp
index eeb663e..9e3018d 100644
--- a/src/slave/containerizer/fetcher.hpp
+++ b/src/slave/containerizer/fetcher.hpp
@@ -67,8 +67,8 @@ public:
   // reported to the user.
   static Try<Nothing> validateUri(const std::string& uri);
 
-  // Checks to make sure the URI filename is valid.
-  static Try<Nothing> validateFilename(const std::string& filename);
+  // Checks to make sure the URI 'output_file' is valid.
+  static Try<Nothing> validateOutputFile(const std::string& path);
 
   // Determines if the given URI refers to a local file system path
   // and prepends frameworksHome if it is a relative path. Fails if

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/src/tests/fetcher_cache_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_cache_tests.cpp b/src/tests/fetcher_cache_tests.cpp
index 9ffcd23..ba7d3f8 100644
--- a/src/tests/fetcher_cache_tests.cpp
+++ b/src/tests/fetcher_cache_tests.cpp
@@ -668,15 +668,15 @@ TEST_F(FetcherCacheTest, CachedCustomFilename)
   driver->start();
 
   const int index = 0;
-  const string customFilename = "my-command";
+  const string customOutputFile = "my-command";
   CommandInfo::URI uri;
   uri.set_value(commandPath);
   uri.set_executable(true);
   uri.set_cache(true);
-  uri.set_filename(customFilename);
+  uri.set_output_file(customOutputFile);
 
   CommandInfo commandInfo;
-  commandInfo.set_value("./" + customFilename + " " + taskName(index));
+  commandInfo.set_value("./" + customOutputFile + " " + taskName(index));
   commandInfo.add_uris()->CopyFrom(uri);
 
   const Try<Task> task = launchTask(commandInfo, index);
@@ -688,9 +688,52 @@ TEST_F(FetcherCacheTest, CachedCustomFilename)
   ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
   EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags).get().size());
 
-  // Verify that the downloaded executable lives at our custom filename path.
+  // Verify that the downloaded executable lives at our custom output path.
   const string executablePath = path::join(
-      task.get().runDirectory.value, customFilename);
+    task.get().runDirectory.value, customOutputFile);
+
+  EXPECT_TRUE(isExecutable(executablePath));
+
+  // The script specified by COMMAND_SCRIPT just statically touches a file
+  // named $COMMAND_NAME + $1, so if we want to verify that it ran here we have
+  // to check this path in addition to the custom-named executable we saved.
+  const string outputPath = path::join(
+    task.get().runDirectory.value, COMMAND_NAME);
+
+  EXPECT_TRUE(os::exists(outputPath + taskName(index)));
+}
+
+
+TEST_F(FetcherCacheTest, CachedCustomOutputFileWithSubdirectory)
+{
+  startSlave();
+  driver->start();
+
+  const int index = 0;
+  const string customOutputFile = "subdir/my-command";
+  CommandInfo::URI uri;
+  uri.set_value(commandPath);
+  uri.set_executable(true);
+  uri.set_cache(true);
+  uri.set_output_file(customOutputFile);
+
+  CommandInfo commandInfo;
+  commandInfo.set_value("./" + customOutputFile + " " + taskName(index));
+  commandInfo.add_uris()->CopyFrom(uri);
+
+  const Try<Task> task = launchTask(commandInfo, index);
+  ASSERT_SOME(task);
+
+  AWAIT_READY(awaitFinished(task.get()));
+
+  EXPECT_EQ(1u, fetcherProcess->cacheSize());
+  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags).get().size());
+
+  // Verify that the downloaded executable lives at our custom output file
+  // path.
+  const string executablePath = path::join(
+      task.get().runDirectory.value, customOutputFile);
 
   EXPECT_TRUE(isExecutable(executablePath));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d2d3edb/src/tests/fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_tests.cpp b/src/tests/fetcher_tests.cpp
index 23a8dc5..d38ce6e 100644
--- a/src/tests/fetcher_tests.cpp
+++ b/src/tests/fetcher_tests.cpp
@@ -100,6 +100,73 @@ TEST_F(FetcherTest, FileURI)
 }
 
 
+TEST_F(FetcherTest, CustomOutputFileSubdirectory)
+{
+  string testFile = path::join(os::getcwd(), "test");
+  EXPECT_SOME(os::write(testFile, "data"));
+
+  string customOutputFile = "subdir/custom.txt";
+  string localFile = path::join(os::getcwd(), customOutputFile);
+  EXPECT_FALSE(os::exists(localFile));
+
+  slave::Flags flags;
+  flags.launcher_dir = getLauncherDir();
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  CommandInfo commandInfo;
+  CommandInfo::URI* uri = commandInfo.add_uris();
+  uri->set_value("file://" + testFile);
+  uri->set_output_file(customOutputFile);
+
+  Fetcher fetcher;
+  SlaveID slaveId;
+
+  Future<Nothing> fetch = fetcher.fetch(
+      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+  AWAIT_READY(fetch);
+
+  EXPECT_TRUE(os::exists(localFile));
+}
+
+
+// Negative test: invalid custom URI output file. If the user specifies a
+// path for the file saved in the sandbox that has a directory component,
+// it must be a relative path.
+TEST_F(FetcherTest, AbsoluteCustomSubdirectoryFails)
+{
+  string fromDir = path::join(os::getcwd(), "from");
+  ASSERT_SOME(os::mkdir(fromDir));
+  string testFile = path::join(fromDir, "test");
+  EXPECT_SOME(os::write(testFile, "data"));
+
+  string customOutputFile = "/subdir/custom.txt";
+  string localFile = path::join(os::getcwd(), customOutputFile);
+  EXPECT_FALSE(os::exists(localFile));
+
+  slave::Flags flags;
+  flags.launcher_dir = getLauncherDir();
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  CommandInfo commandInfo;
+  CommandInfo::URI* uri = commandInfo.add_uris();
+  uri->set_value("file://" + testFile);
+  uri->set_output_file(customOutputFile);
+
+  Fetcher fetcher;
+  SlaveID slaveId;
+
+  Future<Nothing> fetch = fetcher.fetch(
+      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+  AWAIT_FAILED(fetch);
+
+  EXPECT_FALSE(os::exists(localFile));
+}
+
+
 // Negative test: invalid user name. Copied from FileTest, so this
 // normally would succeed, but here a bogus user name is specified.
 // So we check for fetch failure.
@@ -807,7 +874,7 @@ TEST_F(FetcherTest, UNZIP_ExtractFileWithDuplicatedEntries)
 }
 
 
-TEST_F(FetcherTest, UseCustomFilename)
+TEST_F(FetcherTest, UseCustomOutputFile)
 {
   // First construct a temporary file that can be fetched.
   Try<string> dir =
@@ -822,12 +889,12 @@ TEST_F(FetcherTest, UseCustomFilename)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  const string customFilename = "custom.txt";
+  const string customOutputFile = "custom.txt";
   CommandInfo commandInfo;
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(path.get());
   uri->set_extract(true);
-  uri->set_filename(customFilename);
+  uri->set_output_file(customOutputFile);
 
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
@@ -840,14 +907,14 @@ TEST_F(FetcherTest, UseCustomFilename)
 
   AWAIT_READY(fetch);
 
-  ASSERT_TRUE(os::exists(path::join(".", customFilename)));
+  ASSERT_TRUE(os::exists(path::join(".", customOutputFile)));
 
   ASSERT_SOME_EQ(
-      "hello renamed file", os::read(path::join(".", customFilename)));
+      "hello renamed file", os::read(path::join(".", customOutputFile)));
 }
 
 
-TEST_F(FetcherTest, CustomGzipFilename)
+TEST_F(FetcherTest, CustomGzipOutputFile)
 {
   // First construct a temporary file that can be fetched.
   Try<string> dir =
@@ -863,12 +930,12 @@ TEST_F(FetcherTest, CustomGzipFilename)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  const string customFilename = "custom";
+  const string customOutputFile = "custom";
   CommandInfo commandInfo;
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(path.get() + ".gz");
   uri->set_extract(true);
-  uri->set_filename(customFilename + ".gz");
+  uri->set_output_file(customOutputFile + ".gz");
 
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
@@ -881,7 +948,7 @@ TEST_F(FetcherTest, CustomGzipFilename)
 
   AWAIT_READY(fetch);
 
-  string extractFile = path::join(".", customFilename);
+  string extractFile = path::join(".", customOutputFile);
   ASSERT_TRUE(os::exists(extractFile));
 
   ASSERT_SOME_EQ("hello renamed gzip file", os::read(extractFile));