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 2015/07/23 01:01:01 UTC

[1/3] mesos git commit: A few style fixes in containerizer_tests.cpp.

Repository: mesos
Updated Branches:
  refs/heads/master 47071a628 -> ac0cedb4d


A few style fixes in containerizer_tests.cpp.


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

Branch: refs/heads/master
Commit: ac0cedb4df3465b204954542816b911e62822eae
Parents: a0304ca
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Jul 22 11:29:45 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Jul 22 15:48:40 2015 -0700

----------------------------------------------------------------------
 src/tests/containerizer_tests.cpp | 152 ++++++++++++++++++---------------
 1 file changed, 85 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ac0cedb4/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 29114e7..88c46e7 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -45,10 +45,27 @@
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
-using namespace mesos::internal::slave;
-using namespace mesos::internal::slave::state;
+using namespace process;
 
-using namespace mesos::slave;
+using mesos::internal::master::Master;
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::Launcher;
+using mesos::internal::slave::MesosContainerizer;
+using mesos::internal::slave::MesosContainerizerProcess;
+using mesos::internal::slave::PosixLauncher;
+using mesos::internal::slave::Provisioner;
+using mesos::internal::slave::Slave;
+
+using mesos::internal::slave::state::ExecutorState;
+using mesos::internal::slave::state::FrameworkState;
+using mesos::internal::slave::state::RunState;
+using mesos::internal::slave::state::SlaveState;
+
+using mesos::slave::ExecutorRunState;
+using mesos::slave::Isolator;
+using mesos::slave::IsolatorProcess;
+using mesos::slave::Limitation;
 
 using std::list;
 using std::map;
@@ -64,9 +81,8 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-
 class MesosContainerizerIsolatorPreparationTest :
-  public tests::TemporaryDirectoryTest
+  public TemporaryDirectoryTest
 {
 public:
   // Construct a MesosContainerizer with TestIsolator(s) which use the provided
@@ -103,7 +119,6 @@ public:
         hashmap<ContainerInfo::Image::Type, Owned<Provisioner>>());
   }
 
-
   Try<MesosContainerizer*> CreateContainerizer(
       Fetcher* fetcher,
       const Option<CommandInfo>& prepare)
@@ -127,26 +142,28 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
   Try<MesosContainerizer*> containerizer = CreateContainerizer(
       &fetcher,
       CREATE_COMMAND_INFO("touch " + file));
+
   CHECK_SOME(containerizer);
 
   ContainerID containerId;
   containerId.set_value("test_container");
 
-  process::Future<bool> launch = containerizer.get()->launch(
+  Future<bool> launch = containerizer.get()->launch(
       containerId,
       CREATE_EXECUTOR_INFO("executor", "exit 0"),
       directory,
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   // Wait until the launch completes.
   AWAIT_READY(launch);
 
   // Wait for the child (preparation script + executor) to complete.
-  process::Future<containerizer::Termination> wait =
+  Future<containerizer::Termination> wait =
     containerizer.get()->wait(containerId);
+
   AWAIT_READY(wait);
 
   // Check the child exited correctly.
@@ -174,6 +191,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
   Try<MesosContainerizer*> containerizer = CreateContainerizer(
       &fetcher,
       CREATE_COMMAND_INFO("touch " + file + " && exit 1"));
+
   CHECK_SOME(containerizer);
 
   ContainerID containerId;
@@ -185,7 +203,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
       directory,
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   // Wait until the launch completes.
@@ -194,6 +212,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
   // Wait for the child (preparation script + executor) to complete.
   Future<containerizer::Termination> wait =
     containerizer.get()->wait(containerId);
+
   AWAIT_READY(wait);
 
   // Check the child failed to exit correctly.
@@ -210,9 +229,9 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
 }
 
 
-// There are two isolators, one with a prepare command that succeeds and
-// another that fails. The execution order is not defined but the launch should
-// fail from the failing prepare command.
+// There are two isolators, one with a prepare command that succeeds
+// and another that fails. The execution order is not defined but the
+// launch should fail from the failing prepare command.
 TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 {
   string directory = os::getcwd(); // We're inside a temporary sandbox.
@@ -220,9 +239,11 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
   string file2 = path::join(directory, "child.script.executed.2");
 
   vector<Option<CommandInfo>> prepares;
-  // This isolator prepare command one will succeed if called first, otherwise
-  // it won't get run.
+
+  // This isolator prepare command one will succeed if called first,
+  // otherwise it won't get run.
   prepares.push_back(CREATE_COMMAND_INFO("touch " + file1 + " && exit 0"));
+
   // This will fail, either first or after the successful command.
   prepares.push_back(CREATE_COMMAND_INFO("touch " + file2 + " && exit 1"));
 
@@ -230,6 +251,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 
   Try<MesosContainerizer*> containerizer =
     CreateContainerizer(&fetcher, prepares);
+
   CHECK_SOME(containerizer);
 
   ContainerID containerId;
@@ -241,7 +263,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
       directory,
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   // Wait until the launch completes.
@@ -281,6 +303,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
   // Use local=false so std{err,out} are redirected to files.
   Try<MesosContainerizer*> containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
+
   ASSERT_SOME(containerizer);
 
   ContainerID containerId;
@@ -291,21 +314,22 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
   string command =
     "(echo '" + errMsg + "' 1>&2) && echo '" + outMsg + "'";
 
-  process::Future<bool> launch = containerizer.get()->launch(
+  Future<bool> launch = containerizer.get()->launch(
       containerId,
       CREATE_EXECUTOR_INFO("executor", command),
       directory,
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   // Wait for the launch to complete.
   AWAIT_READY(launch);
 
   // Wait on the container.
-  process::Future<containerizer::Termination> wait =
+  Future<containerizer::Termination> wait =
     containerizer.get()->wait(containerId);
+
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -335,17 +359,17 @@ public:
       const slave::Flags& flags,
       bool local,
       Fetcher* fetcher,
-      const process::Owned<Launcher>& launcher,
-      const vector<process::Owned<Isolator>>& isolators,
+      const Owned<Launcher>& launcher,
+      const vector<Owned<Isolator>>& isolators,
       const hashmap<ContainerInfo::Image::Type,
                     Owned<Provisioner>>& provisioners)
     : MesosContainerizerProcess(
-        flags,
-        local,
-        fetcher,
-        launcher,
-        isolators,
-        provisioners)
+          flags,
+          local,
+          fetcher,
+          launcher,
+          isolators,
+          provisioners)
   {
     // NOTE: See TestContainerizer::setup for why we use
     // 'EXPECT_CALL' and 'WillRepeatedly' here instead of
@@ -356,11 +380,11 @@ public:
 
   MOCK_METHOD2(
       exec,
-      process::Future<bool>(
+      Future<bool>(
           const ContainerID& containerId,
           int pipeWrite));
 
-  process::Future<bool> _exec(
+  Future<bool> _exec(
       const ContainerID& containerId,
       int pipeWrite)
   {
@@ -391,20 +415,20 @@ public:
 
   MOCK_METHOD2(
       recover,
-      process::Future<Nothing>(
-          const list<mesos::slave::ExecutorRunState>&,
+      Future<Nothing>(
+          const list<ExecutorRunState>&,
           const hashset<ContainerID>&));
 
   MOCK_METHOD5(
       prepare,
-      process::Future<Option<CommandInfo>>(
+      Future<Option<CommandInfo>>(
           const ContainerID&,
           const ExecutorInfo&,
           const string&,
           const Option<string>&,
           const Option<string>&));
 
-  virtual process::Future<Option<CommandInfo>> _prepare(
+  virtual Future<Option<CommandInfo>> _prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const string& directory,
@@ -416,25 +440,25 @@ public:
 
   MOCK_METHOD2(
       isolate,
-      process::Future<Nothing>(const ContainerID&, pid_t));
+      Future<Nothing>(const ContainerID&, pid_t));
 
   MOCK_METHOD1(
       watch,
-      process::Future<mesos::slave::Limitation>(const ContainerID&));
+      Future<Limitation>(const ContainerID&));
 
   MOCK_METHOD2(
       update,
-      process::Future<Nothing>(const ContainerID&, const Resources&));
+      Future<Nothing>(const ContainerID&, const Resources&));
 
   MOCK_METHOD1(
       usage,
-      process::Future<ResourceStatistics>(const ContainerID&));
+      Future<ResourceStatistics>(const ContainerID&));
 
   MOCK_METHOD1(
       cleanup,
-      process::Future<Nothing>(const ContainerID&));
+      Future<Nothing>(const ContainerID&));
 
-  Promise<mesos::slave::Limitation> watchPromise;
+  Promise<Limitation> watchPromise;
 };
 
 
@@ -443,9 +467,9 @@ public:
 TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
 {
   slave::Flags flags = CreateSlaveFlags();
+
   Try<Launcher*> launcher = PosixLauncher::create(flags);
   ASSERT_SOME(launcher);
-  vector<process::Owned<Isolator>> isolators;
 
   Fetcher fetcher;
 
@@ -454,11 +478,12 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
       true,
       &fetcher,
       Owned<Launcher>(launcher.get()),
-      isolators,
+      vector<Owned<Isolator>>(),
       hashmap<ContainerInfo::Image::Type, Owned<Provisioner>>());
 
   Future<Nothing> exec;
   Promise<bool> promise;
+
   // Letting exec hang to simulate a long fetch.
   EXPECT_CALL(*process, exec(_, _))
     .WillOnce(DoAll(FutureSatisfy(&exec),
@@ -480,7 +505,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
       os::getcwd(),
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -494,24 +519,20 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
 }
 
 
-// Destroying a mesos containerizer while it is preparing should
-// wait until isolators are finished preparing before destroying.
+// Destroying a mesos containerizer while it is preparing should wait
+// until isolators are finished preparing before destroying.
 TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 {
-  slave::Flags flags;
+  slave::Flags flags = CreateSlaveFlags();
+
   Try<Launcher*> launcher = PosixLauncher::create(flags);
   ASSERT_SOME(launcher);
-  vector<Owned<Isolator>> isolators;
 
   MockIsolatorProcess* isolatorProcess = new MockIsolatorProcess();
 
-  Owned<Isolator> isolator(
-      new Isolator(Owned<IsolatorProcess>((IsolatorProcess*)isolatorProcess)));
-
-  isolators.push_back(isolator);
-
   Future<Nothing> prepare;
   Promise<Option<CommandInfo>> promise;
+
   // Simulate a long prepare from the isolator.
   EXPECT_CALL(*isolatorProcess, prepare(_, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&prepare),
@@ -519,15 +540,13 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 
   Fetcher fetcher;
 
-  hashmap<ContainerInfo::Image::Type, Owned<Provisioner>> provisioners;
-
   MockMesosContainerizerProcess* process = new MockMesosContainerizerProcess(
       flags,
       true,
       &fetcher,
       Owned<Launcher>(launcher.get()),
-      isolators,
-      provisioners);
+      {Owned<Isolator>(new Isolator(Owned<IsolatorProcess>(isolatorProcess)))},
+      hashmap<ContainerInfo::Image::Type, Owned<Provisioner>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
 
@@ -545,7 +564,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
       os::getcwd(),
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -565,13 +584,12 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 
   containerizer::Termination termination = wait.get();
 
-  ASSERT_EQ(
+  EXPECT_EQ(
       "Container destroyed while preparing isolators",
       termination.message());
 
-  ASSERT_TRUE(termination.killed());
-
-  ASSERT_FALSE(termination.has_status());
+  EXPECT_TRUE(termination.killed());
+  EXPECT_FALSE(termination.has_status());
 }
 
 
@@ -590,22 +608,21 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 {
   // Create a TestLauncher backed by PosixLauncher.
   slave::Flags flags = CreateSlaveFlags();
+
   Try<Launcher*> launcher_ = PosixLauncher::create(flags);
   ASSERT_SOME(launcher_);
+
   TestLauncher* launcher = new TestLauncher(Owned<Launcher>(launcher_.get()));
 
-  vector<process::Owned<Isolator>> isolators;
   Fetcher fetcher;
 
-  hashmap<ContainerInfo::Image::Type, Owned<Provisioner>> provisioners;
-
   MesosContainerizerProcess* process = new MesosContainerizerProcess(
       flags,
       true,
       &fetcher,
       Owned<Launcher>(launcher),
-      isolators,
-      provisioners);
+      vector<Owned<Isolator>>(),
+      hashmap<ContainerInfo::Image::Type, Owned<Provisioner>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
 
@@ -629,7 +646,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
       os::getcwd(),
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      PID<Slave>(),
       false);
 
   AWAIT_READY(launch);
@@ -666,11 +683,12 @@ class MesosContainerizerRecoverTest : public MesosTest {};
 // that were started by another containerizer (e.g: Docker).
 TEST_F(MesosContainerizerRecoverTest, SkipRecoverNonMesosContainers)
 {
-  slave::Flags flags;
+  slave::Flags flags = CreateSlaveFlags();
   Fetcher fetcher;
 
   Try<MesosContainerizer*> containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
+
   ASSERT_SOME(containerizer);
 
   ExecutorID executorId;


[2/3] mesos git commit: Added a persistent volume test for slave recovery.

Posted by ji...@apache.org.
Added a persistent volume test for slave recovery.

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


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

Branch: refs/heads/master
Commit: a0304ca5ca888e4ecd3d83422983aed1ed295739
Parents: 7f29a72
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Jul 21 23:00:16 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Jul 22 15:48:40 2015 -0700

----------------------------------------------------------------------
 src/tests/persistent_volume_tests.cpp | 123 +++++++++++++++++++++++++++++
 1 file changed, 123 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a0304ca5/src/tests/persistent_volume_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_tests.cpp b/src/tests/persistent_volume_tests.cpp
index c5f72d5..8b791ac 100644
--- a/src/tests/persistent_volume_tests.cpp
+++ b/src/tests/persistent_volume_tests.cpp
@@ -591,6 +591,129 @@ TEST_F(PersistentVolumeTest, AccessPersistentVolume)
   Shutdown();
 }
 
+
+// This test verifies that persistent volumes are recovered properly
+// after the slave restarts. The idea is to launch a command which
+// keeps testing if the persistent volume exists, and fails if it does
+// not. So the framework should not receive a TASK_FAILED after the
+// slave finishes recovery.
+TEST_F(PersistentVolumeTest, SlaveRecovery)
+{
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_role("role1");
+  frameworkInfo.set_checkpoint(true);
+
+  Try<PID<Master>> master = StartMaster(MasterFlags({frameworkInfo}));
+  ASSERT_SOME(master);
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+
+  slaveFlags.resources = "cpus:2;mem:1024;disk(role1):1024";
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return());        // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_FALSE(offers.get().empty());
+
+  Offer offer = offers.get()[0];
+
+  Resources volume = createPersistentVolume(
+      Megabytes(64),
+      "role1",
+      "id1",
+      "path1");
+
+  // Create a task which writes a file in the persistent volume.
+  Resources taskResources =
+    Resources::parse("cpus:1;mem:128;disk(role1):32").get() + volume;
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      taskResources,
+      "while true; do test -d path1; done");
+
+  Future<TaskStatus> status1;
+  Future<TaskStatus> status2;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status1))
+    .WillOnce(FutureArg<1>(&status2));
+
+  Future<Nothing> ack =
+    FUTURE_DISPATCH(_, &Slave::_statusUpdateAcknowledgement);
+
+  driver.acceptOffers(
+      {offer.id()},
+      {CREATE(volume), LAUNCH({task})});
+
+  AWAIT_READY(status1);
+  EXPECT_EQ(task.task_id(), status1.get().task_id());
+  EXPECT_EQ(TASK_RUNNING, status1.get().state());
+
+  // Wait for the ACK to be checkpointed.
+  AWAIT_READY(ack);
+
+  // Restart the slave.
+  Stop(slave.get());
+
+  Future<SlaveReregisteredMessage> slaveReregisteredMessage =
+    FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);
+
+  Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
+
+  slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  Clock::pause();
+
+  AWAIT_READY(_recover);
+
+  // Wait for slave to schedule reregister timeout.
+  Clock::settle();
+
+  // Ensure the slave considers itself recovered.
+  Clock::advance(slave::EXECUTOR_REREGISTER_TIMEOUT);
+
+  Clock::resume();
+
+  // Wait for the slave to re-register.
+  AWAIT_READY(slaveReregisteredMessage);
+
+  // The framework should not receive a TASK_FAILED here since the
+  // persistent volume shouldn't be affected even if slave restarts.
+  ASSERT_TRUE(status2.isPending());
+
+  // NOTE: We kill the task and wait for TASK_KILLED here to make sure
+  // any pending status updates are received by the framework.
+  driver.killTask(task.task_id());
+
+  AWAIT_READY(status2);
+  EXPECT_EQ(task.task_id(), status2.get().task_id());
+  EXPECT_EQ(TASK_KILLED, status2.get().state());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[3/3] mesos git commit: Fixed a bug related to persistent volumes during slave recovery.

Posted by ji...@apache.org.
Fixed a bug related to persistent volumes during slave recovery.

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


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

Branch: refs/heads/master
Commit: 7f29a72602ad96861f1bb80cce466a82fa0e59e4
Parents: 47071a6
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Jul 21 23:42:19 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Jul 22 15:48:40 2015 -0700

----------------------------------------------------------------------
 .../isolators/filesystem/posix.cpp              | 38 +++++++++++++++-----
 1 file changed, 29 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7f29a726/src/slave/containerizer/isolators/filesystem/posix.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/posix.cpp b/src/slave/containerizer/isolators/filesystem/posix.cpp
index 1904279..b78c547 100644
--- a/src/slave/containerizer/isolators/filesystem/posix.cpp
+++ b/src/slave/containerizer/isolators/filesystem/posix.cpp
@@ -218,15 +218,35 @@ Future<Nothing> PosixFilesystemIsolatorProcess::update(
 
     string link = path::join(info->directory, containerPath);
 
-    LOG(INFO) << "Adding symlink from '" << original << "' to '"
-              << link << "' for persistent volume " << resource
-              << " of container " << containerId;
-
-    Try<Nothing> symlink = ::fs::symlink(original, link);
-    if (symlink.isError()) {
-      return Failure(
-          "Failed to symlink persistent volume from '" +
-          original + "' to '" + link + "'");
+    if (os::exists(link)) {
+      // NOTE: This is possible because 'info->resources' will be
+      // reset when slave restarts and recovers. When the slave calls
+      // 'containerizer->update' after the executor re-registers,
+      // we'll try to relink all the already symlinked volumes.
+      Result<string> realpath = os::realpath(link);
+      if (!realpath.isSome()) {
+        return Failure(
+            "Failed to get the realpath of symlink '" + link + "': " +
+            (realpath.isError() ? realpath.error() : "No such directory"));
+      }
+
+      // NOTE: A sanity check which we don't expect it to happen.
+      if (realpath.get() != original) {
+        return Failure(
+            "The existing symlink '" + link + "' points to '" + original +
+            "' and the new target is '" + realpath.get() + "'");
+      }
+    } else {
+      LOG(INFO) << "Adding symlink from '" << original << "' to '"
+                << link << "' for persistent volume " << resource
+                << " of container " << containerId;
+
+      Try<Nothing> symlink = ::fs::symlink(original, link);
+      if (symlink.isError()) {
+        return Failure(
+            "Failed to symlink persistent volume from '" +
+            original + "' to '" + link + "'");
+      }
     }
   }