You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2017/04/18 06:19:52 UTC

[1/4] mesos git commit: Overwriting Directories with Files in Copy Provisioner.

Repository: mesos
Updated Branches:
  refs/heads/master 15873bac3 -> 45811356c


Overwriting Directories with Files in Copy Provisioner.

When a layer overwrites a directory with a regular file or symbolic
link (or vice versa), the old dir/file need to be removed before
copying the layer into the rootfs. This is processed together with
whiteout: The copy provisioner find all files to remove, including
files marked as whiteout and the files described above, and remove
them before the copy process.

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


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

Branch: refs/heads/master
Commit: bc12a5835590178112ec0d46bbbcb014ed246f3b
Parents: 15873ba
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
Authored: Tue Apr 18 14:18:09 2017 +0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 18 14:18:21 2017 +0800

----------------------------------------------------------------------
 .../mesos/provisioner/backends/copy.cpp         | 119 +++++++++++--------
 1 file changed, 71 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bc12a583/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
index 584cc65..68178cb 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
@@ -147,60 +147,83 @@ Future<Nothing> CopyBackendProcess::_provision(
   vector<string> whiteouts;
   for (FTSENT *node = ::fts_read(tree);
        node != nullptr; node = ::fts_read(tree)) {
-    if (node->fts_info != FTS_F) {
+    string ftsPath = string(node->fts_path);
+
+    if (node->fts_info == FTS_DNR ||
+        node->fts_info == FTS_ERR ||
+        node->fts_info == FTS_NS) {
+      return Failure(
+          "Failed to read '" + ftsPath + "': " + os::strerror(node->fts_errno));
+    }
+
+    // Skip the postorder visit of a directory.
+    // See the manpage of fts_read in the following link:
+    //   http://man7.org/linux/man-pages/man3/fts_read.3.html
+    if (node->fts_info == FTS_DP) {
       continue;
     }
 
-    if (!strings::startsWith(node->fts_name, docker::spec::WHITEOUT_PREFIX)) {
+    if (ftsPath == layer) {
       continue;
     }
 
-    string ftsPath = string(node->fts_path);
-    Path whiteout = Path(ftsPath.substr(layer.length() + 1));
-
-    // Keep the relative paths of the whiteout files, we will
-    // remove them from rootfs after layer is copied to rootfs.
-    whiteouts.push_back(whiteout.string());
-
-    if (node->fts_name == string(docker::spec::WHITEOUT_OPAQUE_PREFIX)) {
-      const string path = path::join(rootfs, Path(whiteout).dirname());
-
-      // Remove the entries under the directory labeled
-      // as opaque whiteout from rootfs.
-      Try<Nothing> rmdir = os::rmdir(path, true, false);
-      if (rmdir.isError()) {
-        ::fts_close(tree);
-        return Failure(
-            "Failed to remove the entries under the directory labeled as"
-            " opaque whiteout '" + path + "': " + rmdir.error());
+    string layerPath = ftsPath.substr(layer.length() + 1);
+    string rootfsPath = path::join(rootfs, layerPath);
+    Option<string> removePath;
+
+    // Handle whiteout files.
+    if (node->fts_info == FTS_F &&
+        strings::startsWith(node->fts_name, docker::spec::WHITEOUT_PREFIX)) {
+      Path whiteout = Path(layerPath);
+
+      // Keep the absolute paths of the whiteout files, we will
+      // remove them from rootfs after layer is copied to rootfs.
+      whiteouts.push_back(rootfsPath);
+
+      if (node->fts_name == string(docker::spec::WHITEOUT_OPAQUE_PREFIX)) {
+        removePath = path::join(rootfs, whiteout.dirname());
+      } else {
+        removePath = path::join(
+            rootfs,
+            whiteout.dirname(),
+            whiteout.basename().substr(strlen(docker::spec::WHITEOUT_PREFIX)));
       }
-    } else {
-      const string path = path::join(
-          rootfs,
-          whiteout.dirname(),
-          whiteout.basename().substr(strlen(docker::spec::WHITEOUT_PREFIX)));
-
-      // The file/directory labeled as whiteout may have already been
-      // removed with the code above due to its parent directory labeled
-      // as opaque whiteout, so here we need to check if it still exists
-      // before trying to remove it.
-      if (os::exists(path)) {
-        if (os::stat::isdir(path)) {
-          Try<Nothing> rmdir = os::rmdir(path);
-          if (rmdir.isError()) {
-            ::fts_close(tree);
-            return Failure(
-                "Failed to remove the directory labeled as whiteout '" +
-                path + "': " + rmdir.error());
-          }
-        } else {
-          Try<Nothing> rm = os::rm(path);
-          if (rm.isError()) {
-            ::fts_close(tree);
-            return Failure(
-                "Failed to remove the file labeled as whiteout '" +
-                path + "': " + rm.error());
-          }
+    }
+
+    // Handle overwriting between directories and non-directories.
+    // Note: If a symbolic link is overwritten by a directory, the
+    // symbolic link must be removed before the directory is traversed
+    // so the following case won't cause a security issue:
+    //   ROOTFS: /bad@ -> /usr
+    //   LAYER:  /bad/bin/.wh.wh.opq
+    bool ftsIsDir = node->fts_info == FTS_D || node->fts_info == FTS_DC;
+    if (os::exists(rootfsPath) && os::stat::isdir(rootfsPath) != ftsIsDir) {
+      removePath = rootfsPath;
+    }
+
+    // The file/directory referred to by removePath may be empty or have
+    // already been removed because its parent directory is labeled as
+    // opaque whiteout or overwritten by a file, so here we need to
+    // check if it exists before trying to remove it.
+    if (removePath.isSome() && os::exists(removePath.get())) {
+      if (os::stat::isdir(removePath.get())) {
+        // It is OK to remove the entire directory labeled as opaque
+        // whiteout, since the same directory exists in this layer and
+        // will be copied back to rootfs.
+        Try<Nothing> rmdir = os::rmdir(removePath.get());
+        if (rmdir.isError()) {
+          ::fts_close(tree);
+          return Failure(
+              "Failed to remove directory '" +
+              removePath.get() + "': " + rmdir.error());
+        }
+      } else {
+        Try<Nothing> rm = os::rm(removePath.get());
+        if (rm.isError()) {
+          ::fts_close(tree);
+          return Failure(
+              "Failed to remove file '" +
+              removePath.get() + "': " + rm.error());
         }
       }
     }
@@ -258,7 +281,7 @@ Future<Nothing> CopyBackendProcess::_provision(
 
       // Remove the whiteout files from rootfs.
       foreach (const string whiteout, whiteouts) {
-        Try<Nothing> rm = os::rm(path::join(rootfs, whiteout));
+        Try<Nothing> rm = os::rm(whiteout);
         if (rm.isError()) {
           return Failure(
               "Failed to remove whiteout file '" +


[2/4] mesos git commit: Overwriting Symbolic Links with Files in Copy Provisioner.

Posted by ji...@apache.org.
Overwriting Symbolic Links with Files in Copy Provisioner.

When a layer overwrites a symbolic link with a regular file, the link
must be removed first, otherwise 'cp' would follow the link and
overwrite the target instead of the link itself.

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


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

Branch: refs/heads/master
Commit: 3c8deedc9a1bce617965c3442713ebdc6691d1ae
Parents: bc12a58
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
Authored: Tue Apr 18 14:18:45 2017 +0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 18 14:18:45 2017 +0800

----------------------------------------------------------------------
 .../mesos/provisioner/backends/copy.cpp         | 28 +++++++++++++-------
 1 file changed, 19 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3c8deedc/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
index 68178cb..69faa03 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
@@ -190,15 +190,25 @@ Future<Nothing> CopyBackendProcess::_provision(
       }
     }
 
-    // Handle overwriting between directories and non-directories.
-    // Note: If a symbolic link is overwritten by a directory, the
-    // symbolic link must be removed before the directory is traversed
-    // so the following case won't cause a security issue:
-    //   ROOTFS: /bad@ -> /usr
-    //   LAYER:  /bad/bin/.wh.wh.opq
-    bool ftsIsDir = node->fts_info == FTS_D || node->fts_info == FTS_DC;
-    if (os::exists(rootfsPath) && os::stat::isdir(rootfsPath) != ftsIsDir) {
-      removePath = rootfsPath;
+    if (os::exists(rootfsPath)) {
+      bool ftsIsDir = node->fts_info == FTS_D || node->fts_info == FTS_DC;
+      if (os::stat::isdir(rootfsPath) != ftsIsDir) {
+        // Handle overwriting between a directory and a non-directory.
+        // Note: If a symlink is overwritten by a directory, the symlink
+        // must be removed before the directory is traversed so the
+        // following case won't cause a security issue:
+        //   ROOTFS: /bad@ -> /usr
+        //   LAYER:  /bad/bin/.wh.wh.opq
+        removePath = rootfsPath;
+      } else if (os::stat::islink(rootfsPath)) {
+        // Handle overwriting a symlink with a regular file.
+        // Note: The symlink must be removed, or 'cp' would follow the
+        // link and overwrite the target instead of the link itself,
+        // which would cause a security issue in the following case:
+        //   ROOTFS: /bad@ -> /usr/bin/python
+        //   LAYER:  /bad is a malicious executable
+        removePath = rootfsPath;
+      }
     }
 
     // The file/directory referred to by removePath may be empty or have


[4/4] mesos git commit: Renaming for Provisioner Tests.

Posted by ji...@apache.org.
Renaming for Provisioner Tests.

Renamed ProvisionerDockerPullerTest to ProvisionerDockeTest.
Renamed ProvisionerDockerWhiteoutTest to
  ProvisionerDockerBackendTest.ROOT_INTERNET_CURL_Whiteout.
Renamed ProvisionerDockerOverwriteTest to
  ProvisionerDockerBackendTest.ROOT_INTERNET_CURL_Overwrite.

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


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

Branch: refs/heads/master
Commit: 45811356c03d50cb759ddfb914ea016727f7fc25
Parents: 6a3b424
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
Authored: Tue Apr 18 14:19:28 2017 +0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 18 14:19:28 2017 +0800

----------------------------------------------------------------------
 .../containerizer/provisioner_docker_tests.cpp  | 57 +++++---------------
 1 file changed, 13 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/45811356/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 2620de7..c26e1f9 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -369,14 +369,14 @@ TEST_F(ProvisionerDockerLocalStoreTest, PullingSameImageSimutanuously)
 
 
 #ifdef __linux__
-class ProvisionerDockerPullerTest
+class ProvisionerDockerTest
   : public MesosTest,
     public WithParamInterface<string> {};
 
 
 // This test verifies that local docker image can be pulled and
 // provisioned correctly, and shell command should be executed.
-TEST_F(ProvisionerDockerPullerTest, ROOT_LocalPullerSimpleCommand)
+TEST_F(ProvisionerDockerTest, ROOT_LocalPullerSimpleCommand)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -456,7 +456,7 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_LocalPullerSimpleCommand)
 // puller normalize docker official images if necessary.
 INSTANTIATE_TEST_CASE_P(
     ImageAlpine,
-    ProvisionerDockerPullerTest,
+    ProvisionerDockerTest,
     ::testing::ValuesIn(vector<string>({
         "alpine", // Verifies the normalization of the Docker repository name.
         "library/alpine",
@@ -466,7 +466,7 @@ INSTANTIATE_TEST_CASE_P(
 
 // TODO(jieyu): This is a ROOT test because of MESOS-4757. Remove the
 // ROOT restriction after MESOS-4757 is resolved.
-TEST_P(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
+TEST_P(ProvisionerDockerTest, ROOT_INTERNET_CURL_SimpleCommand)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -548,7 +548,7 @@ TEST_P(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
 // This test verifies that the scratch based docker image (that
 // only contain a single binary and its dependencies) can be
 // launched correctly.
-TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ScratchImage)
+TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_ScratchImage)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -619,7 +619,7 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ScratchImage)
 }
 
 
-class ProvisionerDockerWhiteoutTest
+class ProvisionerDockerBackendTest
   : public MesosTest,
     public WithParamInterface<string>
 {
@@ -646,13 +646,13 @@ public:
 
 INSTANTIATE_TEST_CASE_P(
     BackendFlag,
-    ProvisionerDockerWhiteoutTest,
-    ::testing::ValuesIn(ProvisionerDockerWhiteoutTest::parameters()));
+    ProvisionerDockerBackendTest,
+    ::testing::ValuesIn(ProvisionerDockerBackendTest::parameters()));
 
 
 // This test verifies that a docker image containing whiteout files
 // will be processed correctly by copy, aufs and overlay backends.
-TEST_P(ProvisionerDockerWhiteoutTest, ROOT_INTERNET_CURL_Whiteout)
+TEST_P(ProvisionerDockerBackendTest, ROOT_INTERNET_CURL_Whiteout)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -731,41 +731,10 @@ TEST_P(ProvisionerDockerWhiteoutTest, ROOT_INTERNET_CURL_Whiteout)
 }
 
 
-class ProvisionerDockerOverwriteTest
-  : public MesosTest,
-    public WithParamInterface<string>
-{
-public:
-  // Returns the supported backends.
-  static vector<string> parameters()
-  {
-    vector<string> backends = {COPY_BACKEND};
-
-    Try<bool> aufsSupported = fs::supported("aufs");
-    if (aufsSupported.isSome() && aufsSupported.get()) {
-      backends.push_back(AUFS_BACKEND);
-    }
-
-    Try<bool> overlayfsSupported = fs::supported("overlayfs");
-    if (overlayfsSupported.isSome() && overlayfsSupported.get()) {
-      backends.push_back(OVERLAY_BACKEND);
-    }
-
-    return backends;
-  }
-};
-
-
-INSTANTIATE_TEST_CASE_P(
-    BackendFlag,
-    ProvisionerDockerOverwriteTest,
-    ::testing::ValuesIn(ProvisionerDockerOverwriteTest::parameters()));
-
-
 // This test verifies that the provisioner correctly overwrites a
 // directory in underlying layers with a with a regular file or symbolic
 // link of the same name in an upper layer, and vice versa.
-TEST_P(ProvisionerDockerOverwriteTest, ROOT_INTERNET_CURL_Overwrite)
+TEST_P(ProvisionerDockerBackendTest, ROOT_INTERNET_CURL_Overwrite)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -850,7 +819,7 @@ TEST_P(ProvisionerDockerOverwriteTest, ROOT_INTERNET_CURL_Overwrite)
 
 // This test verifies that Docker image can be pulled from the
 // repository by digest.
-TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ImageDigest)
+TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_ImageDigest)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -934,7 +903,7 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ImageDigest)
 // command runs as the specified user 'nobody' and the sandbox of
 // the command task is writtable by the specified user. It also
 // verifies that stdout/stderr are owned by the specified user.
-TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_CommandTaskUser)
+TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_CommandTaskUser)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -1020,7 +989,7 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_CommandTaskUser)
 // directory still survives. The recursive `provisioner::destroy()`
 // can make sure that a child container is always cleaned up
 // before its parent container.
-TEST_F(ProvisionerDockerPullerTest, ROOT_RecoverNestedOnReboot)
+TEST_F(ProvisionerDockerTest, ROOT_RecoverNestedOnReboot)
 {
   const string directory = path::join(os::getcwd(), "archives");
 


[3/4] mesos git commit: Unit test for file/symlink/directory overwriting in provisioners.

Posted by ji...@apache.org.
Unit test for file/symlink/directory overwriting in provisioners.

The test is based on the following image:
  https://hub.docker.com/r/chhsiao/overwrite/

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


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

Branch: refs/heads/master
Commit: 6a3b4248924a7473d64b7da789897193c6927c7f
Parents: 3c8deed
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
Authored: Tue Apr 18 14:19:09 2017 +0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 18 14:19:09 2017 +0800

----------------------------------------------------------------------
 .../containerizer/provisioner_docker_tests.cpp  | 117 +++++++++++++++++++
 1 file changed, 117 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6a3b4248/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index d1224d8..2620de7 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -731,6 +731,123 @@ TEST_P(ProvisionerDockerWhiteoutTest, ROOT_INTERNET_CURL_Whiteout)
 }
 
 
+class ProvisionerDockerOverwriteTest
+  : public MesosTest,
+    public WithParamInterface<string>
+{
+public:
+  // Returns the supported backends.
+  static vector<string> parameters()
+  {
+    vector<string> backends = {COPY_BACKEND};
+
+    Try<bool> aufsSupported = fs::supported("aufs");
+    if (aufsSupported.isSome() && aufsSupported.get()) {
+      backends.push_back(AUFS_BACKEND);
+    }
+
+    Try<bool> overlayfsSupported = fs::supported("overlayfs");
+    if (overlayfsSupported.isSome() && overlayfsSupported.get()) {
+      backends.push_back(OVERLAY_BACKEND);
+    }
+
+    return backends;
+  }
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    BackendFlag,
+    ProvisionerDockerOverwriteTest,
+    ::testing::ValuesIn(ProvisionerDockerOverwriteTest::parameters()));
+
+
+// This test verifies that the provisioner correctly overwrites a
+// directory in underlying layers with a with a regular file or symbolic
+// link of the same name in an upper layer, and vice versa.
+TEST_P(ProvisionerDockerOverwriteTest, ROOT_INTERNET_CURL_Overwrite)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "docker/runtime,filesystem/linux";
+  flags.image_providers = "docker";
+  flags.image_provisioner_backend = GetParam();
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_EQ(1u, offers->size());
+
+  const Offer& offer = offers.get()[0];
+
+  // We are using the docker image 'chhsiao/overwrite' to verify that:
+  //   1. The '/merged' directory is merged.
+  //   2. All '/replaced*' files/directories are correctly overwritten.
+  //   3. The '/bar' symlink and '/baz' file are correctly overwritten.
+  // See more details in the following link:
+  //   https://hub.docker.com/r/chhsiao/overwrite/
+  CommandInfo command = createCommandInfo(
+      "test -f /replaced1 &&"
+      "test -L /replaced2 &&"
+      "test -f /replaced2/m1 &&"
+      "test -f /replaced2/m2 &&"
+      "! test -e /replaced2/r2 &&"
+      "test -d /replaced3 &&"
+      "test -d /replaced4 &&"
+      "! test -e /replaced4/m1 &&"
+      "test -f /foo &&"
+      "! test -L /bar &&"
+      "test -L /baz");
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128").get(),
+      command);
+
+  Image image = createDockerImage("chhsiao/overwrite");
+
+  ContainerInfo* container = task.mutable_container();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFinished));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY_FOR(statusRunning, Seconds(60));
+  EXPECT_EQ(task.task_id(), statusRunning->task_id());
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  AWAIT_READY(statusFinished);
+  EXPECT_EQ(task.task_id(), statusFinished->task_id());
+  EXPECT_EQ(TASK_FINISHED, statusFinished->state());
+
+  driver.stop();
+  driver.join();
+}
+
+
 // This test verifies that Docker image can be pulled from the
 // repository by digest.
 TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ImageDigest)