You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2017/10/03 17:46:03 UTC

mesos git commit: Added a test using DefaultExecutor + health check + persistent volume.

Repository: mesos
Updated Branches:
  refs/heads/master 0292d4216 -> 32ea75b84


Added a test using DefaultExecutor + health check + persistent volume.

This test verifies that the command health checks initiated by the
default executor are able to read files in a persistent volume.

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


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

Branch: refs/heads/master
Commit: 32ea75b8426fb3972680a397dc3a05ff266d166d
Parents: 0292d42
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Tue Oct 3 10:45:53 2017 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Tue Oct 3 10:45:53 2017 -0700

----------------------------------------------------------------------
 src/tests/default_executor_tests.cpp | 148 ++++++++++++++++++++++++++++++
 1 file changed, 148 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/32ea75b8/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index 14dedab..2b4c643 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -2300,6 +2300,154 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   EXPECT_SOME_EQ("abc\n", os::read(filePath));
 }
 
+
+// This test verifies that the command health checks initiated by the default
+// executor are able to read files in a persistent volume.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    PersistentVolumeDefaultExecutor, ROOT_HealthCheckUsingPersistentVolume)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = param.launcher;
+  flags.isolation = param.isolation;
+
+  Fetcher fetcher(flags);
+
+  // We have to explicitly create a `Containerizer` in non-local mode,
+  // because `LaunchNestedContainerSession` (used by command health
+  // checks) tries to start a IO switchboard, which doesn't work in
+  // local mode yet.
+  Try<MesosContainerizer*> _containerizer =
+    MesosContainerizer::create(flags, false, &fetcher);
+  ASSERT_SOME(_containerizer);
+
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+
+  v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
+
+  Future<Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<Event::Offers> offers;
+  EXPECT_CALL(*scheduler, offers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  v1::scheduler::TestMesos mesos(
+      master.get()->pid,
+      ContentType::PROTOBUF,
+      scheduler);
+
+  AWAIT_READY(subscribed);
+  v1::FrameworkID frameworkId(subscribed->framework_id());
+
+  v1::Resources unreserved =
+    v1::Resources::parse("cpus:0.1;mem:32;disk:32").get();
+
+  v1::ExecutorInfo executorInfo = v1::createExecutorInfo(
+      v1::DEFAULT_EXECUTOR_ID,
+      None(),
+      unreserved,
+      v1::ExecutorInfo::DEFAULT,
+      frameworkId);
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->offers().empty());
+
+  const v1::Offer& offer = offers->offers(0);
+
+  v1::Resource volume = v1::createPersistentVolume(
+      Megabytes(1),
+      frameworkInfo.role(),
+      "id1",
+      "task_volume_path",
+      frameworkInfo.principal(),
+      None(),
+      frameworkInfo.principal());
+
+  v1::Resources reserved =
+    unreserved.pushReservation(v1::createDynamicReservationInfo(
+        frameworkInfo.role(), frameworkInfo.principal()));
+
+  // Launch a task that expects the persistent volume to be
+  // mounted in its sandbox.
+  v1::TaskInfo taskInfo = v1::createTask(
+      offer.agent_id(),
+      reserved.apply(v1::CREATE(volume)).get(),
+      "echo abc > task_volume_path/file && sleep 31337");
+
+  // Create a health check that will only pass if it is able to read
+  // from the task's persistent volume.
+  v1::HealthCheck healthCheck;
+  healthCheck.set_type(v1::HealthCheck::COMMAND);
+  healthCheck.mutable_command()->set_value("cat task_volume_path/file");
+  healthCheck.set_delay_seconds(0);
+  healthCheck.set_interval_seconds(0);
+  healthCheck.set_grace_period_seconds(10);
+
+  taskInfo.mutable_health_check()->CopyFrom(healthCheck);
+
+  v1::Offer::Operation reserve = v1::RESERVE(reserved);
+  v1::Offer::Operation create = v1::CREATE(volume);
+  v1::Offer::Operation launchGroup = v1::LAUNCH_GROUP(
+      executorInfo,
+      v1::createTaskGroupInfo({taskInfo}));
+
+  Future<Event::Update> updateRunning;
+  Future<Event::Update> updateHealthy;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(
+        DoAll(
+            FutureArg<1>(&updateRunning),
+            v1::scheduler::SendAcknowledge(frameworkId, offer.agent_id())))
+    .WillOnce(
+        DoAll(
+            FutureArg<1>(&updateHealthy),
+            v1::scheduler::SendAcknowledge(frameworkId, offer.agent_id())));
+
+  mesos.send(v1::createCallAccept(
+      frameworkId,
+      offer,
+      {reserve, create, launchGroup}));
+
+  AWAIT_READY(updateRunning);
+  ASSERT_EQ(TASK_RUNNING, updateRunning->status().state());
+  ASSERT_EQ(taskInfo.task_id(), updateRunning->status().task_id());
+
+  AWAIT_READY(updateHealthy);
+  EXPECT_EQ(TASK_RUNNING, updateHealthy->status().state());
+  EXPECT_EQ(
+      TaskStatus::REASON_TASK_HEALTH_CHECK_STATUS_UPDATED,
+      updateHealthy->status().reason());
+  EXPECT_TRUE(updateHealthy->status().has_healthy());
+  EXPECT_TRUE(updateHealthy->status().healthy());
+
+  string volumePath = slave::paths::getPersistentVolumePath(
+      flags.work_dir,
+      devolve(volume));
+
+  string filePath = path::join(volumePath, "file");
+
+  // Ensure that the task was able to write to the persistent volume.
+  EXPECT_SOME_EQ("abc\n", os::read(filePath));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {