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 2014/11/08 04:28:11 UTC

mesos git commit: Abstracted out invoking 'mesos-fetcher'.

Repository: mesos
Updated Branches:
  refs/heads/master 36fd0111e -> fae5a6020


Abstracted out invoking 'mesos-fetcher'.

Manually rebasing and re-editing https://reviews.apache.org/r/21233/,
which is supposed to be replaced now by this patch.

Original description: "To test the mesos-fetcher (and the setting of
the environment) more cleanly I did some refactoring into a 'fetcher'
namespace."

Also moved fetcher environment tests to fetcher test file. Added two
fetcher tests.

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


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

Branch: refs/heads/master
Commit: fae5a6020ab90414bff3511ee355b634a0253e38
Parents: 36fd011
Author: Bernd Mathiske <be...@mesosphere.io>
Authored: Fri Nov 7 18:50:17 2014 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Fri Nov 7 19:27:18 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                                 |   2 +
 src/launcher/fetcher.cpp                        |   2 +-
 src/slave/containerizer/containerizer.cpp       |  35 --
 src/slave/containerizer/containerizer.hpp       |   8 -
 src/slave/containerizer/docker.cpp              |   6 +-
 src/slave/containerizer/fetcher.cpp             | 118 +++++++
 src/slave/containerizer/fetcher.hpp             |  77 ++++
 src/slave/containerizer/mesos/containerizer.cpp | 118 ++-----
 src/slave/containerizer/mesos/containerizer.hpp |  22 +-
 src/tests/containerizer_tests.cpp               | 205 -----------
 src/tests/fetcher_tests.cpp                     | 354 ++++++++++++++++++-
 11 files changed, 600 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 443554a..8ecaa54 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -294,6 +294,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/docker.cpp					\
 	slave/containerizer/docker.hpp					\
 	slave/containerizer/external_containerizer.cpp			\
+	slave/containerizer/fetcher.cpp					\
 	slave/containerizer/isolator.cpp				\
 	slave/containerizer/launcher.cpp				\
 	slave/containerizer/mesos/containerizer.cpp			\
@@ -443,6 +444,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/state.hpp							\
 	slave/status_update_manager.hpp					\
 	slave/containerizer/containerizer.hpp				\
+	slave/containerizer/fetcher.hpp					\
 	slave/containerizer/external_containerizer.hpp			\
 	slave/containerizer/isolator.hpp				\
 	slave/containerizer/launcher.hpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/launcher/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/fetcher.cpp b/src/launcher/fetcher.cpp
index bd95928..400fadf 100644
--- a/src/launcher/fetcher.cpp
+++ b/src/launcher/fetcher.cpp
@@ -309,7 +309,7 @@ int main(int argc, char* argv[])
         EXIT(1) << "Failed to chmod " << fetched.get() << ": " << chmod.error();
       }
     } else if (uri.extract()) {
-      //TODO(idownes): Consider removing the archive once extracted.
+      // TODO(idownes): Consider removing the archive once extracted.
       // Try to extract the file if it's recognized as an archive.
       Try<bool> extracted = extract(fetched.get(), directory);
       if (extracted.isError()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 0254679..f234835 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -285,41 +285,6 @@ map<string, string> executorEnvironment(
 }
 
 
-// Helper method to build the environment map used to launch fetcher.
-map<string, string> fetcherEnvironment(
-    const CommandInfo& commandInfo,
-    const std::string& directory,
-    const Option<std::string>& user,
-    const Flags& flags)
-{
-  // Prepare the environment variables to pass to mesos-fetcher.
-  string uris = "";
-  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
-    uris += uri.value() + "+" +
-    (uri.has_executable() && uri.executable() ? "1" : "0") +
-    (uri.extract() ? "X" : "N");
-    uris += " ";
-  }
-  // Remove extra space at the end.
-  uris = strings::trim(uris);
-
-  map<string, string> environment;
-  environment["MESOS_EXECUTOR_URIS"] = uris;
-  environment["MESOS_WORK_DIRECTORY"] = directory;
-  if (user.isSome()) {
-    environment["MESOS_USER"] = user.get();
-  }
-  if (!flags.frameworks_home.empty()) {
-    environment["MESOS_FRAMEWORKS_HOME"] = flags.frameworks_home;
-  }
-  if (!flags.hadoop_home.empty()) {
-    environment["HADOOP_HOME"] = flags.hadoop_home;
-  }
-
-  return environment;
-}
-
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 8a66412..02754cd 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -134,14 +134,6 @@ std::map<std::string, std::string> executorEnvironment(
     bool checkpoint,
     const Duration& recoveryTimeout);
 
-
-std::map<std::string, std::string> fetcherEnvironment(
-    const CommandInfo& commandInfo,
-    const std::string& directory,
-    const Option<std::string>& user,
-    const Flags& flags);
-
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index a668920..37f422a 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -46,6 +46,8 @@
 
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/docker.hpp"
+#include "slave/containerizer/fetcher.hpp"
+
 
 #include "slave/containerizer/isolators/cgroups/constants.hpp"
 
@@ -554,7 +556,7 @@ Future<Nothing> DockerContainerizerProcess::fetch(
     return Failure("Could not fetch URIs: failed to find mesos-fetcher");
   }
 
-  map<string, string> fetcherEnv = fetcherEnvironment(
+  map<string, string> environment = fetcher::environment(
       commandInfo,
       container->directory,
       None(),
@@ -568,7 +570,7 @@ Future<Nothing> DockerContainerizerProcess::fetch(
       Subprocess::PIPE(),
       Subprocess::PATH(path::join(container->directory, "stdout")),
       Subprocess::PATH(path::join(container->directory, "stderr")),
-      fetcherEnv);
+      environment);
 
   if (fetcher.isError()) {
     return Failure("Failed to execute mesos-fetcher: " + fetcher.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.cpp b/src/slave/containerizer/fetcher.cpp
new file mode 100644
index 0000000..a04f0a8
--- /dev/null
+++ b/src/slave/containerizer/fetcher.cpp
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "slave/slave.hpp"
+
+#include "slave/containerizer/fetcher.hpp"
+
+using std::map;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace fetcher {
+
+map<string, string> environment(
+    const CommandInfo& commandInfo,
+    const string& directory,
+    const Option<string>& user,
+    const Flags& flags)
+{
+  // Prepare the environment variables to pass to mesos-fetcher.
+  string uris = "";
+  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
+    uris += uri.value() + "+" +
+      (uri.has_executable() && uri.executable() ? "1" : "0") +
+      (uri.extract() ? "X" : "N");
+    uris += " ";
+  }
+
+  // Remove extra space at the end.
+  uris = strings::trim(uris);
+
+  map<string, string> result;
+
+  result["MESOS_EXECUTOR_URIS"] = uris;
+  result["MESOS_WORK_DIRECTORY"] = directory;
+
+  if (user.isSome()) {
+    result["MESOS_USER"] = user.get();
+  }
+
+  if (!flags.frameworks_home.empty()) {
+    result["MESOS_FRAMEWORKS_HOME"] = flags.frameworks_home;
+  }
+
+  if (!flags.hadoop_home.empty()) {
+    result["HADOOP_HOME"] = flags.hadoop_home;
+  }
+
+  return result;
+}
+
+
+process::Future<Option<int>> run(
+    const CommandInfo& commandInfo,
+    const string& directory,
+    const Option<string>& user,
+    const Flags& flags,
+    const Option<int>& stdout,
+    const Option<int>& stderr)
+{
+  // Determine path for mesos-fetcher.
+  Result<string> realpath = os::realpath(
+      path::join(flags.launcher_dir, "mesos-fetcher"));
+
+  if (!realpath.isSome()) {
+    LOG(ERROR) << "Failed to determine the canonical path "
+                << "for the mesos-fetcher '"
+                << path::join(flags.launcher_dir, "mesos-fetcher")
+                << "': "
+                << (realpath.isError() ? realpath.error()
+                                       : "No such file or directory");
+    return Failure("Could not fetch URIs: failed to find mesos-fetcher");
+  }
+
+  // Now the actual mesos-fetcher command.
+  string command = realpath.get();
+
+  LOG(INFO) << "Fetching URIs using command '" << command << "'";
+
+  Try<Subprocess> fetcher = subprocess(
+    command,
+    Subprocess::PIPE(),
+    stdout.isSome()
+      ? Subprocess::FD(stdout.get())
+      : Subprocess::PIPE(),
+    stderr.isSome()
+      ? Subprocess::FD(stderr.get())
+      : Subprocess::PIPE(),
+    environment(commandInfo, directory, user, flags));
+
+  if (fetcher.isError()) {
+    return Failure("Failed to execute mesos-fetcher: " + fetcher.error());
+  }
+
+  return fetcher.get().status();
+}
+
+} // namespace fetcher {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/fetcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.hpp b/src/slave/containerizer/fetcher.hpp
new file mode 100644
index 0000000..7c57809
--- /dev/null
+++ b/src/slave/containerizer/fetcher.hpp
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __SLAVE_FETCHER_HPP__
+#define __SLAVE_FETCHER_HPP__
+
+#include <map>
+#include <string>
+
+#include <process/future.hpp>
+#include <process/io.hpp>
+#include <process/subprocess.hpp>
+
+#include <stout/lambda.hpp>
+#include <stout/option.hpp>
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/result.hpp>
+#include <stout/strings.hpp>
+#include <stout/try.hpp>
+
+#include <mesos/mesos.hpp>
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace fetcher {
+
+// Defines helpers for running the mesos-fetcher.
+// TODO(benh): Consider moving this into a 'fetcher' subdirectory as
+// well as the actual mesos-fetcher program (in launcher/fetcher.cpp
+// as of the writing of this comment).
+
+// Helper method to build the environment used to run mesos-fetcher.
+std::map<std::string, std::string> environment(
+    const CommandInfo& commandInfo,
+    const std::string& directory,
+    const Option<std::string>& user,
+    const Flags& flags);
+
+// Run the mesos-fetcher for the specified arguments. Note that if
+// 'stdout' and 'stderr' file descriptors are provided then respective
+// output from the mesos-fetcher will be redirected to the file
+// descriptors. The file descriptors are duplicated (via dup) because
+// redirecting might still be occuring even after the mesos-fetcher has
+// terminated since there still might be data to be read.
+process::Future<Option<int>> run(
+    const CommandInfo& commandInfo,
+    const std::string& directory,
+    const Option<std::string>& user,
+    const Flags& flags,
+    const Option<int>& stdout = None(),
+    const Option<int>& stderr = None());
+
+} // namespace fetcher {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __SLAVE_FETCHER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 4bd2665..562b03b 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -31,6 +31,7 @@
 #include "slave/slave.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
+#include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 #ifdef __linux__
@@ -115,7 +116,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   creators["network/port_mapping"] = &PortMappingIsolatorProcess::create;
 #endif
 
-  vector<Owned<Isolator> > isolators;
+  vector<Owned<Isolator>> isolators;
 
   foreach (const string& type, strings::split(isolation, ",")) {
     if (creators.contains(type)) {
@@ -171,7 +172,7 @@ MesosContainerizer::MesosContainerizer(
     const Flags& flags,
     bool local,
     const Owned<Launcher>& launcher,
-    const vector<Owned<Isolator> >& isolators)
+    const vector<Owned<Isolator>>& isolators)
 {
   process = new MesosContainerizerProcess(
       flags, local, launcher, isolators);
@@ -269,7 +270,7 @@ void MesosContainerizer::destroy(const ContainerID& containerId)
 }
 
 
-Future<hashset<ContainerID> > MesosContainerizer::containers()
+Future<hashset<ContainerID>> MesosContainerizer::containers()
 {
   return dispatch(process, &MesosContainerizerProcess::containers);
 }
@@ -339,7 +340,7 @@ Future<Nothing> MesosContainerizerProcess::_recover(
     const list<RunState>& recoverable)
 {
   // Then recover the isolators.
-  list<Future<Nothing> > futures;
+  list<Future<Nothing>> futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
     futures.push_back(isolator->recover(recoverable));
   }
@@ -357,12 +358,12 @@ Future<Nothing> MesosContainerizerProcess::__recover(
     CHECK_SOME(run.id);
     const ContainerID& containerId = run.id.get();
 
-    Owned<Promise<containerizer::Termination> > promise(
+    Owned<Promise<containerizer::Termination>> promise(
         new Promise<containerizer::Termination>());
     promises.put(containerId, promise);
 
     CHECK_SOME(run.forkedPid);
-    Future<Option<int > > status = process::reap(run.forkedPid.get());
+    Future<Option<int>> status = process::reap(run.forkedPid.get());
     statuses[containerId] = status;
     status.onAny(defer(self(), &Self::reaped, containerId));
 
@@ -414,7 +415,7 @@ Future<bool> MesosContainerizerProcess::launch(
     return false;
   }
 
-  Owned<Promise<containerizer::Termination> > promise(
+  Owned<Promise<containerizer::Termination>> promise(
       new Promise<containerizer::Termination>());
   promises.put(containerId, promise);
 
@@ -527,7 +528,8 @@ Future<Nothing> _fetch(
   if (status.isNone() || (status.get() != 0)) {
     return Failure("Failed to fetch URIs for container '" +
                    stringify(containerId) + "': exit status " +
-                   (status.isNone() ? "none" : stringify(status.get())));
+                   (status.isNone() ?
+                       "none" : stringify(status.get())));
   }
 
   // Chown the work directory if a user is provided.
@@ -548,55 +550,23 @@ Future<Nothing> MesosContainerizerProcess::fetch(
     const string& directory,
     const Option<string>& user)
 {
-  // Determine path for mesos-fetcher.
-  Result<string> realpath = os::realpath(
-      path::join(flags.launcher_dir, "mesos-fetcher"));
-
-  if (!realpath.isSome()) {
-    LOG(ERROR) << "Failed to determine the canonical path "
-                << "for the mesos-fetcher '"
-                << path::join(flags.launcher_dir, "mesos-fetcher")
-                << "': "
-                << (realpath.isError() ? realpath.error()
-                                       : "No such file or directory");
-    return Failure("Could not fetch URIs: failed to find mesos-fetcher");
-  }
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-
-  // Now the actual mesos-fetcher command.
-  string command = realpath.get();
-
-  LOG(INFO) << "Fetching URIs for container '" << containerId
-            << "' using command '" << command << "'";
-
-  Try<Subprocess> fetcher = subprocess(
-      command,
-      Subprocess::PIPE(),
-      Subprocess::PIPE(),
-      Subprocess::PIPE(),
-      environment);
-
-  if (fetcher.isError()) {
-    return Failure("Failed to execute mesos-fetcher: " + fetcher.error());
-  }
+  // Before we fetch let's make sure we create 'stdout' and 'stderr'
+  // files into which we can redirect the output of the mesos-fetcher
+  // (and later redirect the child's stdout/stderr).
 
-  // Redirect output (stdout and stderr) from the fetcher to log files
-  // in the executor work directory, chown'ing them if a user is
-  // specified.
   // TODO(tillt): Consider adding O_CLOEXEC for atomic close-on-exec.
-  // TODO(tillt): Consider adding an overload to io::redirect
-  // that accepts a file path as 'to' for further reducing code. We
-  // would however also need an owner user parameter for such overload
-  // to perfectly replace the below.
+  // TODO(tillt): Considering updating fetcher::run to take paths
+  // instead of file descriptors and then use Subprocess::PATH()
+  // instead of Subprocess::FD(). The reason this can't easily be done
+  // today is because we not only need to open the files but also
+  // chown them.
   Try<int> out = os::open(
       path::join(directory, "stdout"),
       O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK,
       S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
 
   if (out.isError()) {
-    return Failure("Failed to redirect stdout: " + out.error());
+    return Failure("Failed to create 'stdout' file: " + out.error());
   }
 
   if (user.isSome()) {
@@ -604,20 +574,10 @@ Future<Nothing> MesosContainerizerProcess::fetch(
         user.get(), path::join(directory, "stdout"));
     if (chown.isError()) {
       os::close(out.get());
-      return Failure(
-          "Failed to redirect stdout: Failed to chown: " +
-          chown.error());
+      return Failure("Failed to chown 'stdout' file: " + chown.error());
     }
   }
 
-  // Redirect takes care of nonblocking and close-on-exec for the
-  // supplied file descriptors.
-  io::redirect(fetcher.get().out().get(), out.get());
-
-  // Redirect does 'dup' the file descriptor, hence we can close the
-  // original now.
-  os::close(out.get());
-
   // Repeat for stderr.
   Try<int> err = os::open(
       path::join(directory, "stderr"),
@@ -625,9 +585,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
       S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
 
   if (err.isError()) {
-    return Failure(
-        "Failed to redirect stderr: Failed to open: " +
-        err.error());
+    return Failure("Failed to create 'stderr' file: " + err.error());
   }
 
   if (user.isSome()) {
@@ -635,17 +593,19 @@ Future<Nothing> MesosContainerizerProcess::fetch(
         user.get(), path::join(directory, "stderr"));
     if (chown.isError()) {
       os::close(err.get());
-      return Failure(
-          "Failed to redirect stderr: Failed to chown: " +
-          chown.error());
+      return Failure("Failed to chown 'stderr' file: " + chown.error());
     }
   }
 
-  io::redirect(fetcher.get().err().get(), err.get());
-
-  os::close(err.get());
-
-  return fetcher.get().status()
+  return fetcher::run(
+      commandInfo,
+      directory,
+      user,
+      flags,
+      out.get(),
+      err.get())
+    .onAny(lambda::bind(&os::close, out.get()))
+    .onAny(lambda::bind(&os::close, err.get()))
     .then(lambda::bind(&_fetch, containerId, directory, user, lambda::_1));
 }
 
@@ -658,7 +618,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     const SlaveID& slaveId,
     const PID<Slave>& slavePid,
     bool checkpoint,
-    const list<Option<CommandInfo> >& commands)
+    const list<Option<CommandInfo>>& commands)
 {
   // Prepare environment variables for the executor.
   map<string, string> env = executorEnvironment(
@@ -752,7 +712,7 @@ Future<bool> MesosContainerizerProcess::_launch(
 
   // Monitor the executor's pid. We keep the future because we'll
   // refer to it again during container destroy.
-  Future<Option<int> > status = process::reap(pid);
+  Future<Option<int>> status = process::reap(pid);
   statuses.put(containerId, status);
   status.onAny(defer(self(), &Self::reaped, containerId));
 
@@ -789,7 +749,7 @@ Future<bool> MesosContainerizerProcess::isolate(
   // NOTE: This is done is parallel and is not sequenced like prepare
   // or destroy because we assume there are no dependencies in
   // isolation.
-  list<Future<Nothing> > futures;
+  list<Future<Nothing>> futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
     futures.push_back(isolator->isolate(containerId, _pid));
   }
@@ -857,7 +817,7 @@ Future<Nothing> MesosContainerizerProcess::update(
   resources.put(containerId, _resources);
 
   // Update each isolator.
-  list<Future<Nothing> > futures;
+  list<Future<Nothing>> futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
     futures.push_back(isolator->update(containerId, _resources));
   }
@@ -874,7 +834,7 @@ Future<Nothing> MesosContainerizerProcess::update(
 Future<ResourceStatistics> _usage(
     const ContainerID& containerId,
     const Option<Resources>& resources,
-    const list<Future<ResourceStatistics> >& statistics)
+    const list<Future<ResourceStatistics>>& statistics)
 {
   ResourceStatistics result;
 
@@ -916,7 +876,7 @@ Future<ResourceStatistics> MesosContainerizerProcess::usage(
     return Failure("Unknown container: " + stringify(containerId));
   }
 
-  list<Future<ResourceStatistics> > futures;
+  list<Future<ResourceStatistics>> futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
     futures.push_back(isolator->usage(containerId));
   }
@@ -1023,7 +983,7 @@ static T reversed(const T& t)
 
 void MesosContainerizerProcess::__destroy(
     const ContainerID& containerId,
-    const Future<Option<int > >& status)
+    const Future<Option<int>>& status)
 {
   // We clean up each isolator in the reverse order they were
   // prepared (see comment in prepare()).
@@ -1147,7 +1107,7 @@ void MesosContainerizerProcess::limited(
 }
 
 
-Future<hashset<ContainerID> > MesosContainerizerProcess::containers()
+Future<hashset<ContainerID>> MesosContainerizerProcess::containers()
 {
   return promises.keys();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index ab3bb6f..3baea31 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -45,7 +45,7 @@ public:
       const Flags& flags,
       bool local,
       const process::Owned<Launcher>& launcher,
-      const std::vector<process::Owned<Isolator> >& isolators);
+      const std::vector<process::Owned<Isolator>>& isolators);
 
   virtual ~MesosContainerizer();
 
@@ -83,7 +83,7 @@ public:
 
   virtual void destroy(const ContainerID& containerId);
 
-  virtual process::Future<hashset<ContainerID> > containers();
+  virtual process::Future<hashset<ContainerID>> containers();
 
 private:
   MesosContainerizerProcess* process;
@@ -98,7 +98,7 @@ public:
       const Flags& _flags,
       bool _local,
       const process::Owned<Launcher>& _launcher,
-      const std::vector<process::Owned<Isolator> >& _isolators)
+      const std::vector<process::Owned<Isolator>>& _isolators)
     : flags(_flags),
       local(_local),
       launcher(_launcher),
@@ -140,7 +140,7 @@ public:
 
   void destroy(const ContainerID& containerId);
 
-  process::Future<hashset<ContainerID> > containers();
+  process::Future<hashset<ContainerID>> containers();
 
 private:
   process::Future<Nothing> _recover(
@@ -149,7 +149,7 @@ private:
   process::Future<Nothing> __recover(
       const std::list<state::RunState>& recovered);
 
-  process::Future<std::list<Option<CommandInfo> > > prepare(
+  process::Future<std::list<Option<CommandInfo>>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const std::string& directory,
@@ -169,7 +169,7 @@ private:
       const SlaveID& slaveId,
       const process::PID<Slave>& slavePid,
       bool checkpoint,
-      const std::list<Option<CommandInfo> >& scripts);
+      const std::list<Option<CommandInfo>>& scripts);
 
   process::Future<bool> isolate(
       const ContainerID& containerId,
@@ -187,13 +187,13 @@ private:
   // Continues '_destroy()' once we get the exit status of the executor.
   void __destroy(
       const ContainerID& containerId,
-      const process::Future<Option<int > >& status);
+      const process::Future<Option<int>>& status);
 
   // Continues (and completes) '__destroy()' once all isolators have completed
   // cleanup.
   void ___destroy(
       const ContainerID& containerId,
-      const process::Future<Option<int> >& status,
+      const process::Future<Option<int>>& status,
       const process::Future<std::list<process::Future<Nothing>>>& cleanups);
 
   // Call back for when an isolator limits a container and impacts the
@@ -209,17 +209,17 @@ private:
   const Flags flags;
   const bool local;
   const process::Owned<Launcher> launcher;
-  const std::vector<process::Owned<Isolator> > isolators;
+  const std::vector<process::Owned<Isolator>> isolators;
 
   // TODO(idownes): Consider putting these per-container variables into a
   // struct.
   // Promises for futures returned from wait().
   hashmap<ContainerID,
-    process::Owned<process::Promise<containerizer::Termination> > > promises;
+    process::Owned<process::Promise<containerizer::Termination>>> promises;
 
   // We need to keep track of the future exit status for each executor because
   // we'll only get a single notification when the executor exits.
-  hashmap<ContainerID, process::Future<Option<int> > > statuses;
+  hashmap<ContainerID, process::Future<Option<int>>> statuses;
 
   // We keep track of any limitations received from each isolator so we can
   // determine the cause of an executor termination.

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 2c90d2f..abe8074 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -47,211 +47,6 @@ using std::map;
 using std::string;
 using std::vector;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
-// Forward declaration.
-map<string, string> fetcherEnvironment(
-    const CommandInfo& commandInfo,
-    const string& directory,
-    const Option<string>& user,
-    const Flags& flags);
-
-}  // namespace slave {
-}  // namespace internal {
-}  // namespace mesos {
-
-class MesosContainerizerProcessTest : public ::testing::Test {};
-
-
-TEST_F(MesosContainerizerProcessTest, Simple)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user = "user";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "/tmp/hadoop";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-  EXPECT_EQ(5u, environment.size());
-  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, MultipleURIs)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri1");
-  uri.set_executable(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-  uri.set_value("hdfs:///uri2");
-  uri.set_executable(true);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user("user");
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "/tmp/hadoop";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-
-  EXPECT_EQ(5u, environment.size());
-  EXPECT_EQ(
-      "hdfs:///uri1+0X hdfs:///uri2+1X", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, NoUser)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "/tmp/hadoop";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, None(), flags);
-
-  EXPECT_EQ(4u, environment.size());
-  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, EmptyHadoop)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user = "user";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-
-  EXPECT_EQ(4u, environment.size());
-  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, NoHadoop)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user = "user";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-
-  EXPECT_EQ(4u, environment.size());
-  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, NoExtract)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(false);
-  uri.set_extract(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user = "user";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "/tmp/hadoop";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-  EXPECT_EQ(5u, environment.size());
-  EXPECT_EQ("hdfs:///uri+0N", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
-}
-
-
-TEST_F(MesosContainerizerProcessTest, NoExtractExecutable)
-{
-  CommandInfo commandInfo;
-  CommandInfo::URI uri;
-  uri.set_value("hdfs:///uri");
-  uri.set_executable(true);
-  uri.set_extract(false);
-  commandInfo.add_uris()->MergeFrom(uri);
-
-  string directory = "/tmp/directory";
-  Option<string> user = "user";
-
-  Flags flags;
-  flags.frameworks_home = "/tmp/frameworks";
-  flags.hadoop_home = "/tmp/hadoop";
-
-  map<string, string> environment =
-    fetcherEnvironment(commandInfo, directory, user, flags);
-  EXPECT_EQ(5u, environment.size());
-  EXPECT_EQ("hdfs:///uri+1N", environment["MESOS_EXECUTOR_URIS"]);
-  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
-  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
-  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
-  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
-}
-
-
 class MesosContainerizerIsolatorPreparationTest :
   public tests::TemporaryDirectoryTest
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fae5a602/src/tests/fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_tests.cpp b/src/tests/fetcher_tests.cpp
index e026e87..a01eec1 100644
--- a/src/tests/fetcher_tests.cpp
+++ b/src/tests/fetcher_tests.cpp
@@ -18,8 +18,7 @@
 
 #include <unistd.h>
 
-#include <gmock/gmock.h>
-
+#include <map>
 #include <string>
 
 #include <process/future.hpp>
@@ -35,12 +34,16 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include "slave/containerizer/fetcher.hpp"
+#include "slave/flags.hpp"
+
 #include "tests/environment.hpp"
 #include "tests/flags.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
+using namespace mesos::internal::slave;
 using namespace mesos::internal::tests;
 
 using namespace process;
@@ -50,6 +53,198 @@ using process::Future;
 using std::string;
 using std::map;
 
+
+class FetcherEnvironmentTest : public ::testing::Test {};
+
+
+TEST_F(FetcherEnvironmentTest, Simple)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user = "user";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "/tmp/hadoop";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+
+  EXPECT_EQ(5u, environment.size());
+  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, MultipleURIs)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri1");
+  uri.set_executable(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+  uri.set_value("hdfs:///uri2");
+  uri.set_executable(true);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user("user");
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "/tmp/hadoop";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+
+  EXPECT_EQ(5u, environment.size());
+  EXPECT_EQ(
+      "hdfs:///uri1+0X hdfs:///uri2+1X", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, NoUser)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "/tmp/hadoop";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, None(), flags);
+
+  EXPECT_EQ(4u, environment.size());
+  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, EmptyHadoop)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user = "user";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+
+  EXPECT_EQ(4u, environment.size());
+  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, NoHadoop)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user = "user";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+
+  EXPECT_EQ(4u, environment.size());
+  EXPECT_EQ("hdfs:///uri+0X", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, NoExtractNoExecutable)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(false);
+  uri.set_extract(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user = "user";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "/tmp/hadoop";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+  EXPECT_EQ(5u, environment.size());
+  EXPECT_EQ("hdfs:///uri+0N", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
+}
+
+
+TEST_F(FetcherEnvironmentTest, NoExtractExecutable)
+{
+  CommandInfo commandInfo;
+  CommandInfo::URI uri;
+  uri.set_value("hdfs:///uri");
+  uri.set_executable(true);
+  uri.set_extract(false);
+  commandInfo.add_uris()->MergeFrom(uri);
+
+  string directory = "/tmp/directory";
+  Option<string> user = "user";
+
+  slave::Flags flags;
+  flags.frameworks_home = "/tmp/frameworks";
+  flags.hadoop_home = "/tmp/hadoop";
+
+  map<string, string> environment =
+    fetcher::environment(commandInfo, directory, user, flags);
+  EXPECT_EQ(5u, environment.size());
+  EXPECT_EQ("hdfs:///uri+1N", environment["MESOS_EXECUTOR_URIS"]);
+  EXPECT_EQ(directory, environment["MESOS_WORK_DIRECTORY"]);
+  EXPECT_EQ(user.get(), environment["MESOS_USER"]);
+  EXPECT_EQ(flags.frameworks_home, environment["MESOS_FRAMEWORKS_HOME"]);
+  EXPECT_EQ(flags.hadoop_home, environment["HADOOP_HOME"]);
+}
+
+
 class FetcherTest : public TemporaryDirectoryTest {};
 
 
@@ -74,7 +269,7 @@ TEST_F(FetcherTest, FileURI)
       env);
 
   ASSERT_SOME(fetcherProcess);
-  Future<Option<int> > status = fetcherProcess.get().status();
+  Future<Option<int>> status = fetcherProcess.get().status();
 
   AWAIT_READY(status);
   ASSERT_SOME(status.get());
@@ -105,7 +300,7 @@ TEST_F(FetcherTest, FilePath)
       env);
 
   ASSERT_SOME(fetcherProcess);
-  Future<Option<int> > status = fetcherProcess.get().status();
+  Future<Option<int>> status = fetcherProcess.get().status();
 
   AWAIT_READY(status);
   ASSERT_SOME(status.get());
@@ -153,7 +348,7 @@ TEST_F(FetcherTest, OSNetUriTest)
       env);
 
   ASSERT_SOME(fetcherProcess);
-  Future<Option<int> > status = fetcherProcess.get().status();
+  Future<Option<int>> status = fetcherProcess.get().status();
 
   AWAIT_READY(status);
   ASSERT_SOME(status.get());
@@ -184,7 +379,7 @@ TEST_F(FetcherTest, FileLocalhostURI)
       env);
 
   ASSERT_SOME(fetcherProcess);
-  Future<Option<int> > status = fetcherProcess.get().status();
+  Future<Option<int>> status = fetcherProcess.get().status();
 
   AWAIT_READY(status);
   ASSERT_SOME(status.get());
@@ -192,3 +387,150 @@ TEST_F(FetcherTest, FileLocalhostURI)
   EXPECT_EQ(0, status.get().get());
   EXPECT_TRUE(os::exists(localFile));
 }
+
+
+TEST_F(FetcherTest, NoExtractNotExecutable)
+{
+  // First construct a temporary file that can be fetched.
+  Try<string> path = os::mktemp();
+
+  ASSERT_SOME(path);
+
+  CommandInfo commandInfo;
+  CommandInfo::URI* uri = commandInfo.add_uris();
+  uri->set_value(path.get());
+  uri->set_executable(false);
+  uri->set_extract(false);
+
+  Option<int> stdout = None();
+  Option<int> stderr = None();
+
+  // Redirect mesos-fetcher output if running the tests verbosely.
+  if (tests::flags.verbose) {
+    stdout = STDOUT_FILENO;
+    stderr = STDERR_FILENO;
+  }
+
+  slave::Flags flags;
+  flags.launcher_dir = path::join(tests::flags.build_dir, "src");
+
+  Future<Option<int>> run =
+    fetcher::run(commandInfo, os::getcwd(), None(), flags, stdout, stderr);
+
+  AWAIT_READY(run);
+  EXPECT_SOME_EQ(0, run.get());
+
+  Try<string> basename = os::basename(path.get());
+
+  ASSERT_SOME(basename);
+
+  Try<os::Permissions> permissions = os::permissions(basename.get());
+
+  ASSERT_SOME(permissions);
+  EXPECT_FALSE(permissions.get().owner.x);
+  EXPECT_FALSE(permissions.get().group.x);
+  EXPECT_FALSE(permissions.get().others.x);
+
+  ASSERT_SOME(os::rm(path.get()));
+}
+
+
+TEST_F(FetcherTest, NoExtractExecutable)
+{
+  // First construct a temporary file that can be fetched.
+  Try<string> path = os::mktemp();
+
+  ASSERT_SOME(path);
+
+  CommandInfo commandInfo;
+  CommandInfo::URI* uri = commandInfo.add_uris();
+  uri->set_value(path.get());
+  uri->set_executable(true);
+  uri->set_extract(false);
+
+  Option<int> stdout = None();
+  Option<int> stderr = None();
+
+  // Redirect mesos-fetcher output if running the tests verbosely.
+  if (tests::flags.verbose) {
+    stdout = STDOUT_FILENO;
+    stderr = STDERR_FILENO;
+  }
+
+  slave::Flags flags;
+  flags.launcher_dir = path::join(tests::flags.build_dir, "src");
+
+  Future<Option<int>> run =
+    fetcher::run(commandInfo, os::getcwd(), None(), flags, stdout, stderr);
+
+  AWAIT_READY(run);
+  EXPECT_SOME_EQ(0, run.get());
+
+  Try<string> basename = os::basename(path.get());
+
+  ASSERT_SOME(basename);
+
+  Try<os::Permissions> permissions = os::permissions(basename.get());
+
+  ASSERT_SOME(permissions);
+  EXPECT_TRUE(permissions.get().owner.x);
+  EXPECT_TRUE(permissions.get().group.x);
+  EXPECT_TRUE(permissions.get().others.x);
+
+  ASSERT_SOME(os::rm(path.get()));
+}
+
+
+TEST_F(FetcherTest, ExtractNotExecutable)
+{
+  // First construct a temporary file that can be fetched and archive
+  // with tar  gzip.
+  Try<string> path = os::mktemp();
+
+  ASSERT_SOME(path);
+
+  ASSERT_SOME(os::write(path.get(), "hello world"));
+
+  // TODO(benh): Update os::tar so that we can capture or ignore
+  // stdout/stderr output.
+
+  ASSERT_SOME(os::tar(path.get(), path.get() + ".tar.gz"));
+
+  CommandInfo commandInfo;
+  CommandInfo::URI* uri = commandInfo.add_uris();
+  uri->set_value(path.get() + ".tar.gz");
+  uri->set_executable(false);
+  uri->set_extract(true);
+
+  Option<int> stdout = None();
+  Option<int> stderr = None();
+
+  // Redirect mesos-fetcher output if running the tests verbosely.
+  if (tests::flags.verbose) {
+    stdout = STDOUT_FILENO;
+    stderr = STDERR_FILENO;
+  }
+
+  slave::Flags flags;
+  flags.launcher_dir = path::join(tests::flags.build_dir, "src");
+
+  Future<Option<int>> run =
+    fetcher::run(commandInfo, os::getcwd(), None(), flags, stdout, stderr);
+
+  AWAIT_READY(run);
+  EXPECT_SOME_EQ(0, run.get());
+
+  ASSERT_TRUE(os::exists(path::join(".", path.get())));
+
+  ASSERT_SOME_EQ("hello world", os::read(path::join(".", path.get())));
+
+  Try<os::Permissions> permissions =
+    os::permissions(path::join(".", path.get()));
+
+  ASSERT_SOME(permissions);
+  EXPECT_FALSE(permissions.get().owner.x);
+  EXPECT_FALSE(permissions.get().group.x);
+  EXPECT_FALSE(permissions.get().others.x);
+
+  ASSERT_SOME(os::rm(path.get()));
+}