You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2018/08/14 23:57:52 UTC

[mesos] branch master updated (c633bf6 -> 4a42147)

This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from c633bf6  Added MESOS-9142 to 1.5.2 CHANGELOG.
     new 374f772  Updated `volume/host_path` isolator to honor volume mode.
     new 782ef70  Added a test `VolumeHostPathIsolatorTest.ROOT_ReadOnlyVolumeFromHost`.
     new 1631ce9  Updated `volume/sandbox_path` isolator to honor volume mode.
     new cba4ef8  Added a test `VolumeSandboxPathIsolatorTest.ROOT_SelfTypeReadOnly`.
     new 83a9df6  Updated `volume/image` isolator to honor volume mode.
     new 6d1af98  Added a test `ROOT_ImageInReadOnlyVolumeWithoutRootFilesystem`.
     new 085e841  Updated `volume/secret` isolator to honor volume mode.
     new c275976  Updated the test `ROOT_SecretInVolumeWithRootFilesystem`.
     new 49d3cd4  Updated `docker/volume` isolator to honor volume mode.
     new 4a42147  Added 2 tests for `docker/volume` isolator to cover read-only volume.

The 10 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../mesos/isolators/docker/volume/isolator.cpp     |  30 ++-
 .../mesos/isolators/docker/volume/isolator.hpp     |   1 +
 .../mesos/isolators/volume/host_path.cpp           |   8 +-
 .../containerizer/mesos/isolators/volume/image.cpp |  16 ++
 .../containerizer/mesos/isolators/volume/image.hpp |   1 +
 .../mesos/isolators/volume/sandbox_path.cpp        |  14 ++
 .../mesos/isolators/volume/secret.cpp              |  13 +
 .../containerizer/docker_volume_isolator_tests.cpp | 267 +++++++++++++++++++++
 .../volume_host_path_isolator_tests.cpp            |  56 +++++
 .../containerizer/volume_image_isolator_tests.cpp  |  85 +++++++
 .../volume_sandbox_path_isolator_tests.cpp         |  53 ++++
 .../containerizer/volume_secret_isolator_tests.cpp |  33 ++-
 12 files changed, 561 insertions(+), 16 deletions(-)


[mesos] 04/10: Added a test `VolumeSandboxPathIsolatorTest.ROOT_SelfTypeReadOnly`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit cba4ef850c343a9f41b32608c6df8f9615c6f623
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:25 2018 -0700

    Added a test `VolumeSandboxPathIsolatorTest.ROOT_SelfTypeReadOnly`.
    
    Review: https://reviews.apache.org/r/68215/
---
 .../volume_sandbox_path_isolator_tests.cpp         | 53 ++++++++++++++++++++++
 1 file changed, 53 insertions(+)

diff --git a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
index 97b35a4..cbe6778 100644
--- a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
@@ -108,6 +108,59 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfType)
 
   EXPECT_SOME_EQ("abc\n", os::read(path::join(directory, "tmp", "file")));
 }
+
+
+// This test verifies that a container launched with a rootfs cannot
+// write to a read-only SANDBOX_PATH volume with SELF type.
+TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfTypeReadOnly)
+{
+  string registry = path::join(sandbox.get(), "registry");
+  AWAIT_READY(DockerArchive::create(registry, "test_image"));
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,volume/sandbox_path,docker/runtime";
+  flags.docker_registry = registry;
+  flags.docker_store_dir = path::join(sandbox.get(), "store");
+  flags.image_providers = "docker";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "test_executor",
+      "echo abc > /tmp/file");
+
+  executor.mutable_container()->CopyFrom(createContainerInfo(
+      "test_image",
+      {createVolumeSandboxPath("/tmp", "tmp", Volume::RO)}));
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait->get().has_status());
+  EXPECT_WEXITSTATUS_NE(0, wait->get().status());
+}
 #endif // __linux__
 
 


[mesos] 08/10: Updated the test `ROOT_SecretInVolumeWithRootFilesystem`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit c27597660b3498c217ef0e237c6174669488025e
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:37 2018 -0700

    Updated the test `ROOT_SecretInVolumeWithRootFilesystem`.
    
    This test is updated to cover both read-write and read-only volume.
    
    Review: https://reviews.apache.org/r/68220/
---
 .../containerizer/volume_secret_isolator_tests.cpp | 33 ++++++++++++++++------
 1 file changed, 24 insertions(+), 9 deletions(-)

diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
index 11cd3b6..8b11c6f 100644
--- a/src/tests/containerizer/volume_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -65,7 +65,11 @@ enum CONTAINER_LAUNCH_STATUS {
 class VolumeSecretIsolatorTest :
   public MesosTest,
   public ::testing::WithParamInterface<std::tr1::tuple<
-      const char*, const char*, enum FS_TYPE, enum CONTAINER_LAUNCH_STATUS>>
+      const char*,
+      const char*,
+      enum FS_TYPE,
+      enum CONTAINER_LAUNCH_STATUS,
+      Volume::Mode>>
 
 {
 protected:
@@ -78,7 +82,7 @@ protected:
     fsType = std::tr1::get<2>(GetParam());
     expectedContainerLaunchStatus = std::tr1::get<3>(GetParam());
 
-    volume.set_mode(Volume::RW);
+    volume.set_mode(std::tr1::get<4>(GetParam()));
     volume.set_container_path(secretContainerPath);
 
     Volume::Source* source = volume.mutable_source();
@@ -114,7 +118,8 @@ INSTANTIATE_TEST_CASE_P(
     ::testing::Combine(::testing::Values(""),
                        ::testing::ValuesIn(paths),
                        ::testing::Values(WITHOUT_ROOTFS),
-                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS),
+                       ::testing::Values(Volume::RW, Volume::RO)));
 
 
 INSTANTIATE_TEST_CASE_P(
@@ -123,7 +128,8 @@ INSTANTIATE_TEST_CASE_P(
     ::testing::Combine(::testing::Values("/"),
                        ::testing::ValuesIn(paths),
                        ::testing::Values(WITHOUT_ROOTFS),
-                       ::testing::Values(CONTAINER_LAUNCH_FAILURE)));
+                       ::testing::Values(CONTAINER_LAUNCH_FAILURE),
+                       ::testing::Values(Volume::RW, Volume::RO)));
 
 
 INSTANTIATE_TEST_CASE_P(
@@ -132,7 +138,8 @@ INSTANTIATE_TEST_CASE_P(
     ::testing::Combine(::testing::Values(""),
                        ::testing::Values("/bin/touch"),
                        ::testing::Values(WITHOUT_ROOTFS),
-                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS),
+                       ::testing::Values(Volume::RW, Volume::RO)));
 
 
 INSTANTIATE_TEST_CASE_P(
@@ -141,7 +148,8 @@ INSTANTIATE_TEST_CASE_P(
     ::testing::Combine(::testing::Values("", "/"),
                        ::testing::ValuesIn(paths),
                        ::testing::Values(WITH_ROOTFS),
-                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS),
+                       ::testing::Values(Volume::RW, Volume::RO)));
 
 
 TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
@@ -224,8 +232,10 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
   nestedContainerId.set_value(id::UUID::random().toString());
 
   CommandInfo nestedCommand = createCommandInfo(
-      "secret=$(cat " + secretContainerPath + "); "
-      "test \"$secret\" = \"" + string(SECRET_VALUE) + "\"");
+      volume.mode() == Volume::RW
+        ? "secret=$(cat " + secretContainerPath + "); "
+          "test \"$secret\" = \"" + string(SECRET_VALUE) + "\""
+        : "echo abc > " + secretContainerPath);
 
   launch = containerizer->launch(
       nestedContainerId,
@@ -242,7 +252,12 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
   AWAIT_READY(wait);
   ASSERT_SOME(wait.get());
   ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  if (volume.mode() == Volume::RW) {
+    EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+  } else {
+    EXPECT_WEXITSTATUS_NE(0, wait.get()->status());
+  }
 
   // Now wait for parent container.
   Future<Option<ContainerTermination>> termination =


[mesos] 02/10: Added a test `VolumeHostPathIsolatorTest.ROOT_ReadOnlyVolumeFromHost`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 782ef70c20d6e93a045b8f5b4d533adf3f8869d6
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:18 2018 -0700

    Added a test `VolumeHostPathIsolatorTest.ROOT_ReadOnlyVolumeFromHost`.
    
    Review: https://reviews.apache.org/r/68213/
---
 .../volume_host_path_isolator_tests.cpp            | 56 ++++++++++++++++++++++
 1 file changed, 56 insertions(+)

diff --git a/src/tests/containerizer/volume_host_path_isolator_tests.cpp b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
index 3c925bc..81bf72e 100644
--- a/src/tests/containerizer/volume_host_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
@@ -116,6 +116,62 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_VolumeFromHost)
 }
 
 
+// This test verifies that a container launched with a
+// rootfs cannot write to a read-only HOST_PATH volume.
+TEST_F(VolumeHostPathIsolatorTest, ROOT_ReadOnlyVolumeFromHost)
+{
+  string registry = path::join(sandbox.get(), "registry");
+  AWAIT_READY(DockerArchive::create(registry, "test_image"));
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,docker/runtime";
+  flags.docker_registry = registry;
+  flags.docker_store_dir = path::join(sandbox.get(), "store");
+  flags.image_providers = "docker";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<Containerizer> containerizer(create.get());
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "test_executor",
+      "echo abc > /tmp/dir/file");
+
+  executor.mutable_container()->CopyFrom(createContainerInfo(
+      "test_image",
+      {createVolumeHostPath("/tmp", sandbox.get(), Volume::RO)}));
+
+  string dir = path::join(sandbox.get(), "dir");
+  ASSERT_SOME(os::mkdir(dir));
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait->get().has_status());
+  EXPECT_WEXITSTATUS_NE(0, wait->get().status());
+}
+
+
 // This test verifies that a file volume with an absolute host
 // path as well as an absolute container path is properly mounted
 // in the container's mount namespace.


[mesos] 10/10: Added 2 tests for `docker/volume` isolator to cover read-only volume.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 4a421476ef6e924fcea5b33087a8cd3310b7bd5f
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:44 2018 -0700

    Added 2 tests for `docker/volume` isolator to cover read-only volume.
    
    Review: https://reviews.apache.org/r/68222/
---
 .../containerizer/docker_volume_isolator_tests.cpp | 267 +++++++++++++++++++++
 1 file changed, 267 insertions(+)

diff --git a/src/tests/containerizer/docker_volume_isolator_tests.cpp b/src/tests/containerizer/docker_volume_isolator_tests.cpp
index c15a6fa..553f344 100644
--- a/src/tests/containerizer/docker_volume_isolator_tests.cpp
+++ b/src/tests/containerizer/docker_volume_isolator_tests.cpp
@@ -1114,6 +1114,273 @@ TEST_F(DockerVolumeIsolatorTest,
   driver.join();
 }
 
+
+// This test verifies that a container launched without
+// a rootfs cannot write to a read-only docker volume.
+TEST_F(DockerVolumeIsolatorTest, ROOT_CommandTaskNoRootfsWithReadOnlyVolume)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  MockDockerVolumeDriverClient* mockClient = new MockDockerVolumeDriverClient;
+
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer(flags, Owned<DriverClient>(mockClient));
+
+  ASSERT_SOME(containerizer);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      containerizer->get(),
+      flags);
+
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      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);
+  ASSERT_FALSE(offers->empty());
+
+  const Offer& offer = offers.get()[0];
+
+  const string key = "iops";
+  const string value = "150";
+
+  hashmap<string, string> options = {{key, value}};
+
+  // Create a volume with relative path.
+  const string _driver = "driver";
+  const string name = "name";
+  const string containerPath = "tmp/foo";
+
+  Volume volume = createDockerVolume(_driver, name, containerPath, options);
+  volume.set_mode(Volume::RO);
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      offer.resources(),
+      "echo 'hello' > " + containerPath + "/file");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_volumes()->CopyFrom(volume);
+
+  task.mutable_container()->CopyFrom(containerInfo);
+
+  // Create mount point for volume.
+  const string mountPoint = path::join(os::getcwd(), "volume");
+  ASSERT_SOME(os::mkdir(mountPoint));
+
+  Future<string> mountName;
+  Future<hashmap<string, string>> mountOptions;
+
+  EXPECT_CALL(*mockClient, mount(_driver, _, _))
+    .WillOnce(DoAll(FutureArg<1>(&mountName),
+                    FutureArg<2>(&mountOptions),
+                    Return(mountPoint)));
+
+  Future<string> unmountName;
+
+  EXPECT_CALL(*mockClient, unmount(_driver, _))
+    .WillOnce(DoAll(FutureArg<1>(&unmountName),
+                    Return(Nothing())));
+
+  Future<TaskStatus> statusStarting;
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFailed;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusStarting))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFailed));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(statusStarting);
+  EXPECT_EQ(TASK_STARTING, statusStarting->state());
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  // Make sure the docker volume mount parameters are same with the
+  // parameters in `containerInfo`.
+  AWAIT_EXPECT_EQ(name, mountName);
+
+  AWAIT_READY(mountOptions);
+  EXPECT_SOME_EQ(value, mountOptions->get(key));
+
+  AWAIT_READY(statusFailed);
+  EXPECT_EQ(TASK_FAILED, statusFailed->state());
+
+  // Make sure the docker volume unmount parameters are same with
+  // the parameters in `containerInfo`.
+  AWAIT_EXPECT_EQ(name, unmountName);
+
+  driver.stop();
+  driver.join();
+}
+
+
+// This test verifies that a container launched with
+// a rootfs cannot write to a read-only docker volume.
+TEST_F(DockerVolumeIsolatorTest,
+       ROOT_INTERNET_CURL_CommandTaskRootfsWithReadOnlyVolume)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "docker/volume,docker/runtime,filesystem/linux";
+  flags.image_providers = "docker";
+
+  MockDockerVolumeDriverClient* mockClient = new MockDockerVolumeDriverClient;
+
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer(flags, Owned<DriverClient>(mockClient));
+
+  ASSERT_SOME(containerizer);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      containerizer->get(),
+      flags);
+
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      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);
+  ASSERT_FALSE(offers->empty());
+
+  const Offer& offer = offers.get()[0];
+
+  const string key = "iops";
+  const string value = "150";
+
+  hashmap<string, string> options = {{key, value}};
+
+  // Create a volume with relative path.
+  const string _driver = "driver";
+  const string name = "name";
+  const string containerPath = "tmp/foo";
+
+  Volume volume = createDockerVolume(_driver, name, containerPath, options);
+  volume.set_mode(Volume::RO);
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      offer.resources(),
+      "echo 'hello' > " + containerPath + "/file");
+
+  Image image;
+  image.set_type(Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_volumes()->CopyFrom(volume);
+
+  containerInfo.mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  task.mutable_container()->CopyFrom(containerInfo);
+
+  // Create mount point for volume.
+  const string mountPoint = path::join(os::getcwd(), "volume");
+  ASSERT_SOME(os::mkdir(mountPoint));
+
+  Future<string> mountName;
+  Future<hashmap<string, string>> mountOptions;
+
+  EXPECT_CALL(*mockClient, mount(_driver, _, _))
+    .WillOnce(DoAll(FutureArg<1>(&mountName),
+                    FutureArg<2>(&mountOptions),
+                    Return(mountPoint)));
+
+  Future<string> unmountName;
+
+  EXPECT_CALL(*mockClient, unmount(_driver, _))
+    .WillOnce(DoAll(FutureArg<1>(&unmountName),
+                    Return(Nothing())));
+
+  Future<TaskStatus> statusStarting;
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFailed;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusStarting))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFailed));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(statusStarting);
+  EXPECT_EQ(TASK_STARTING, statusStarting->state());
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  // Make sure the docker volume mount parameters are same with the
+  // parameters in `containerInfo`.
+  AWAIT_EXPECT_EQ(name, mountName);
+
+  AWAIT_READY(mountOptions);
+  EXPECT_SOME_EQ(value, mountOptions->get(key));
+
+  AWAIT_READY(statusFailed);
+  EXPECT_EQ(TASK_FAILED, statusFailed->state());
+
+  // Make sure the docker volume unmount parameters are same with
+  // the parameters in `containerInfo`.
+  AWAIT_EXPECT_EQ(name, unmountName);
+
+  driver.stop();
+  driver.join();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 05/10: Updated `volume/image` isolator to honor volume mode.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 83a9df6c6dd93c26c9cf9b5e70628102342ffdbe
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:28 2018 -0700

    Updated `volume/image` isolator to honor volume mode.
    
    Review: https://reviews.apache.org/r/68216/
---
 src/slave/containerizer/mesos/isolators/volume/image.cpp | 16 ++++++++++++++++
 src/slave/containerizer/mesos/isolators/volume/image.hpp |  1 +
 2 files changed, 17 insertions(+)

diff --git a/src/slave/containerizer/mesos/isolators/volume/image.cpp b/src/slave/containerizer/mesos/isolators/volume/image.cpp
index 3457724..53cbaef 100644
--- a/src/slave/containerizer/mesos/isolators/volume/image.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/image.cpp
@@ -113,7 +113,11 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::prepare(
     return Failure("Can only prepare image volumes for a MESOS container");
   }
 
+  // TODO(qianzhang): Here we use vector to ensure the order of mount target,
+  // mount source and volume mode which is kind of hacky, we could consider
+  // to introduce a dedicated struct for it in future.
   vector<string> targets;
+  vector<Volume::Mode> volumeModes;
   vector<Future<ProvisionInfo>> futures;
 
   for (int i = 0; i < containerConfig.container_info().volumes_size(); i++) {
@@ -188,6 +192,7 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::prepare(
     }
 
     targets.push_back(target);
+    volumeModes.push_back(volume.mode());
     futures.push_back(provisioner->provision(containerId, volume.image()));
   }
 
@@ -197,6 +202,7 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::prepare(
         &VolumeImageIsolatorProcess::_prepare,
         containerId,
         targets,
+        volumeModes,
         lambda::_1));
 }
 
@@ -204,6 +210,7 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::prepare(
 Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::_prepare(
     const ContainerID& containerId,
     const vector<string>& targets,
+    const vector<Volume::Mode>& volumeModes,
     const vector<Future<ProvisionInfo>>& futures)
 {
   ContainerLaunchInfo launchInfo;
@@ -225,10 +232,12 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::_prepare(
   }
 
   CHECK_EQ(sources.size(), targets.size());
+  CHECK_EQ(sources.size(), volumeModes.size());
 
   for (size_t i = 0; i < sources.size(); i++) {
     const string& source = sources[i];
     const string& target = targets[i];
+    const Volume::Mode volumeMode = volumeModes[i];
 
     LOG(INFO) << "Mounting image volume rootfs '" << source
               << "' to '" << target << "' for container " << containerId;
@@ -242,6 +251,13 @@ Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::_prepare(
     mount->set_source(source);
     mount->set_target(target);
     mount->set_flags(MS_BIND | MS_REC);
+
+    // If the mount needs to be read-only, do a remount.
+    if (volumeMode == Volume::RO) {
+      mount = launchInfo.add_mounts();
+      mount->set_target(target);
+      mount->set_flags(MS_BIND | MS_RDONLY | MS_REMOUNT);
+    }
   }
 
   return launchInfo;
diff --git a/src/slave/containerizer/mesos/isolators/volume/image.hpp b/src/slave/containerizer/mesos/isolators/volume/image.hpp
index 706b8ff..6f70c4d 100644
--- a/src/slave/containerizer/mesos/isolators/volume/image.hpp
+++ b/src/slave/containerizer/mesos/isolators/volume/image.hpp
@@ -58,6 +58,7 @@ private:
   process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
       const ContainerID& containerId,
       const std::vector<std::string>& targets,
+      const std::vector<Volume::Mode>& volumeModes,
       const std::vector<process::Future<ProvisionInfo>>& futures);
 
   const Flags flags;


[mesos] 09/10: Updated `docker/volume` isolator to honor volume mode.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 49d3cd43cd668bc37b8ed3ee563e13e95214b3b7
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:41 2018 -0700

    Updated `docker/volume` isolator to honor volume mode.
    
    Review: https://reviews.apache.org/r/68221/
---
 .../mesos/isolators/docker/volume/isolator.cpp     | 30 +++++++++++++++++-----
 .../mesos/isolators/docker/volume/isolator.hpp     |  1 +
 2 files changed, 25 insertions(+), 6 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
index ab749be..24c9fd6 100644
--- a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
@@ -326,7 +326,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
 
   // The hashset is used to check if there are duplicated docker
   // volume for the same container.
-  hashset<DockerVolume> volumes;
+  hashset<DockerVolume> volumeSet;
 
   // Represents mounts that will be sent to the driver client.
   struct Mount
@@ -335,11 +335,17 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
     hashmap<string, string> options;
   };
 
+
+  // TODO(qianzhang): Here we use vector to ensure the order of mount target,
+  // mount source and volume mode which is kind of hacky, we could consider
+  // to introduce a dedicated struct for it in future.
   vector<Mount> mounts;
 
   // The mount points in the container.
   vector<string> targets;
 
+  vector<Volume::Mode> volumeModes;
+
   foreach (const Volume& _volume, containerConfig.container_info().volumes()) {
     if (!_volume.has_source()) {
       continue;
@@ -366,7 +372,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
     volume.set_driver(driver);
     volume.set_name(name);
 
-    if (volumes.contains(volume)) {
+    if (volumeSet.contains(volume)) {
       return Failure(
           "Found duplicate docker volume with driver '" +
           driver + "' and name '" + name + "'");
@@ -443,15 +449,16 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
     mount.volume = volume;
     mount.options = options;
 
-    volumes.insert(volume);
+    volumeSet.insert(volume);
     mounts.push_back(mount);
     targets.push_back(target);
+    volumeModes.push_back(_volume.mode());
   }
 
   // It is possible that there is no external volume specified for
   // this container. We avoid checkpointing empty state and creating
   // an empty `Info`.
-  if (volumes.empty()) {
+  if (volumeSet.empty()) {
     return None();
   }
 
@@ -468,7 +475,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
 
   // Create DockerVolumes protobuf message to checkpoint.
   DockerVolumes state;
-  foreach (const DockerVolume& volume, volumes) {
+  foreach (const DockerVolume& volume, volumeSet) {
     state.add_volumes()->CopyFrom(volume);
   }
 
@@ -487,7 +494,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
 
   VLOG(1) << "Successfully created checkpoint at '" << volumesPath << "'";
 
-  infos.put(containerId, Owned<Info>(new Info(volumes)));
+  infos.put(containerId, Owned<Info>(new Info(volumeSet)));
 
   // Invoke driver client to create the mount.
   vector<Future<string>> futures;
@@ -508,6 +515,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
         &DockerVolumeIsolatorProcess::_prepare,
         containerId,
         targets,
+        volumeModes,
         lambda::_1));
 }
 
@@ -515,6 +523,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::prepare(
 Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::_prepare(
     const ContainerID& containerId,
     const vector<string>& targets,
+    const vector<Volume::Mode>& volumeModes,
     const vector<Future<string>>& futures)
 {
   ContainerLaunchInfo launchInfo;
@@ -536,10 +545,12 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::_prepare(
   }
 
   CHECK_EQ(sources.size(), targets.size());
+  CHECK_EQ(sources.size(), volumeModes.size());
 
   for (size_t i = 0; i < sources.size(); i++) {
     const string& source = sources[i];
     const string& target = targets[i];
+    const Volume::Mode volumeMode = volumeModes[i];
 
     LOG(INFO) << "Mounting docker volume mount point '" << source
               << "' to '" << target << "' for container " << containerId;
@@ -548,6 +559,13 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::_prepare(
     mount->set_source(source);
     mount->set_target(target);
     mount->set_flags(MS_BIND | MS_REC);
+
+    // If the mount needs to be read-only, do a remount.
+    if (volumeMode == Volume::RO) {
+      mount = launchInfo.add_mounts();
+      mount->set_target(target);
+      mount->set_flags(MS_BIND | MS_RDONLY | MS_REMOUNT);
+    }
   }
 
   return launchInfo;
diff --git a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.hpp b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.hpp
index 76f1a52..2fd0493 100644
--- a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.hpp
+++ b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.hpp
@@ -82,6 +82,7 @@ private:
   process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
       const ContainerID& containerId,
       const std::vector<std::string>& targets,
+      const std::vector<Volume::Mode>& volumeModes,
       const std::vector<process::Future<std::string>>& futures);
 
   process::Future<Nothing> _cleanup(


[mesos] 06/10: Added a test `ROOT_ImageInReadOnlyVolumeWithoutRootFilesystem`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 6d1af986c323295af71e7dbbf3319742a906bf77
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:31 2018 -0700

    Added a test `ROOT_ImageInReadOnlyVolumeWithoutRootFilesystem`.
    
    Review: https://reviews.apache.org/r/68218/
---
 .../containerizer/volume_image_isolator_tests.cpp  | 85 ++++++++++++++++++++++
 1 file changed, 85 insertions(+)

diff --git a/src/tests/containerizer/volume_image_isolator_tests.cpp b/src/tests/containerizer/volume_image_isolator_tests.cpp
index b49f0f9..e47df53 100644
--- a/src/tests/containerizer/volume_image_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_image_isolator_tests.cpp
@@ -244,6 +244,91 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   }
 }
 
+
+// This test verifies that a container launched without
+// a rootfs cannot write to a read-only IMAGE volume.
+TEST_P(VolumeImageIsolatorTest, ROOT_ImageInReadOnlyVolumeWithoutRootFilesystem)
+{
+  string registry = path::join(sandbox.get(), "registry");
+  AWAIT_READY(DockerArchive::create(registry, "test_image"));
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,volume/image,docker/runtime";
+  flags.docker_registry = registry;
+  flags.docker_store_dir = path::join(sandbox.get(), "store");
+  flags.image_providers = "docker";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<Containerizer> containerizer(create.get());
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ContainerInfo container = createContainerInfo(
+      None(),
+      {createVolumeFromDockerImage("rootfs", "test_image", Volume::RO)});
+
+  CommandInfo command = createCommandInfo("echo abc > rootfs/file");
+
+  ExecutorInfo executor = createExecutorInfo(
+      "test_executor",
+      nesting ? createCommandInfo("sleep 1000") : command);
+
+  if (!nesting) {
+    executor.mutable_container()->CopyFrom(container);
+  }
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  if (nesting) {
+    ContainerID nestedContainerId;
+    nestedContainerId.mutable_parent()->CopyFrom(containerId);
+    nestedContainerId.set_value(id::UUID::random().toString());
+
+    launch = containerizer->launch(
+        nestedContainerId,
+        createContainerConfig(command, container),
+        map<string, string>(),
+        None());
+
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+    wait = containerizer->wait(nestedContainerId);
+  }
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait->get().has_status());
+  EXPECT_WEXITSTATUS_NE(0, wait->get().status());
+
+  if (nesting) {
+    Future<Option<ContainerTermination>> termination =
+      containerizer->destroy(containerId);
+
+    AWAIT_READY(termination);
+    ASSERT_SOME(termination.get());
+    ASSERT_TRUE(termination->get().has_status());
+    EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+  }
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 01/10: Updated `volume/host_path` isolator to honor volume mode.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 374f7720bf2fdb2ab9dc584618c7ae51b2a9b416
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:13 2018 -0700

    Updated `volume/host_path` isolator to honor volume mode.
    
    Review: https://reviews.apache.org/r/68212/
---
 src/slave/containerizer/mesos/isolators/volume/host_path.cpp | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git a/src/slave/containerizer/mesos/isolators/volume/host_path.cpp b/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
index 9127cf4..2e03ef5 100644
--- a/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
@@ -309,11 +309,17 @@ Future<Option<ContainerLaunchInfo>> VolumeHostPathIsolatorProcess::prepare(
     // result, no need for the bind mount because the 'hostPath' is
     // already accessible in the container.
     if (hostPath.get() != mountPoint) {
-      // TODO(jieyu): Consider the mode in the volume.
       ContainerMountInfo* mount = launchInfo.add_mounts();
       mount->set_source(hostPath.get());
       mount->set_target(mountPoint);
       mount->set_flags(MS_BIND | MS_REC);
+
+      // If the mount needs to be read-only, do a remount.
+      if (volume.mode() == Volume::RO) {
+        mount = launchInfo.add_mounts();
+        mount->set_target(mountPoint);
+        mount->set_flags(MS_BIND | MS_RDONLY | MS_REMOUNT);
+      }
     }
   }
 


[mesos] 07/10: Updated `volume/secret` isolator to honor volume mode.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 085e8419830fa208d3ee982000426bb4cd651798
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:34 2018 -0700

    Updated `volume/secret` isolator to honor volume mode.
    
    Review: https://reviews.apache.org/r/68219/
---
 src/slave/containerizer/mesos/isolators/volume/secret.cpp | 13 +++++++++++++
 1 file changed, 13 insertions(+)

diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.cpp b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
index 663aafc..7a9bb82 100644
--- a/src/slave/containerizer/mesos/isolators/volume/secret.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
@@ -276,6 +276,19 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
     command->add_arguments(sandboxSecretPath);
     command->add_arguments(targetContainerPath);
 
+    // If the mount needs to be read-only, do a remount.
+    if (volume.mode() == Volume::RO) {
+      command = launchInfo.add_pre_exec_commands();
+      command->set_shell(false);
+      command->set_value("mount");
+      command->add_arguments("mount");
+      command->add_arguments("-n");
+      command->add_arguments("-o");
+      command->add_arguments("bind,ro,remount");
+      command->add_arguments(sandboxSecretPath);
+      command->add_arguments(targetContainerPath);
+    }
+
     Future<Nothing> future = secretResolver->resolve(secret)
       .then([hostSecretPath](const Secret::Value& value) -> Future<Nothing> {
         Try<Nothing> writeSecret = os::write(hostSecretPath, value.data());


[mesos] 03/10: Updated `volume/sandbox_path` isolator to honor volume mode.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 1631ce9a69f23dbc51985d00436715c11812f89f
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 14 16:19:21 2018 -0700

    Updated `volume/sandbox_path` isolator to honor volume mode.
    
    Review: https://reviews.apache.org/r/68214/
---
 .../containerizer/mesos/isolators/volume/sandbox_path.cpp  | 14 ++++++++++++++
 1 file changed, 14 insertions(+)

diff --git a/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp b/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
index 4896c68..21d9528 100644
--- a/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
@@ -379,12 +379,26 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
       mount->set_source(source);
       mount->set_target(target);
       mount->set_flags(MS_BIND | MS_REC);
+
+      // If the mount needs to be read-only, do a remount.
+      if (volume.mode() == Volume::RO) {
+        mount = launchInfo.add_mounts();
+        mount->set_target(target);
+        mount->set_flags(MS_BIND | MS_RDONLY | MS_REMOUNT);
+      }
 #endif // __linux__
     } else {
       LOG(INFO) << "Linking SANDBOX_PATH volume from "
                 << "'" << source << "' to '" << target << "' "
                 << "for container " << containerId;
 
+      // NOTE: We cannot enforce read-only access given the symlink without
+      // changing the source so we just log a warning here.
+      if (volume.mode() == Volume::RO) {
+        LOG(WARNING) << "Allowing read-write access to read-only volume '"
+                     << source << "' of container " << containerId;
+      }
+
       Try<Nothing> symlink = ::fs::symlink(source, target);
       if (symlink.isError()) {
         return Failure(