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/15 00:58:48 UTC

[mesos] branch 1.7.x updated (d9931b8 -> 54ae2e3)

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

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


    from d9931b8  Added MESOS-9142 to 1.7.0 CHANGELOG.
     new b07bc76  Updated `volume/host_path` isolator to honor volume mode.
     new 0243667  Added a test `VolumeHostPathIsolatorTest.ROOT_ReadOnlyVolumeFromHost`.
     new 889dc45  Updated `volume/sandbox_path` isolator to honor volume mode.
     new 8ce21f2  Added a test `VolumeSandboxPathIsolatorTest.ROOT_SelfTypeReadOnly`.
     new f5245fa  Updated `volume/image` isolator to honor volume mode.
     new b5d983d  Added a test `ROOT_ImageInReadOnlyVolumeWithoutRootFilesystem`.
     new eb728f0  Updated `volume/secret` isolator to honor volume mode.
     new 6d92daa  Updated the test `ROOT_SecretInVolumeWithRootFilesystem`.
     new 4559aff  Updated `docker/volume` isolator to honor volume mode.
     new 54ae2e3  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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 54ae2e3050c00cff181528abe7d5d27ccf41fb0f
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit b07bc7628579e5d2d4b2a33438d4326d3d198c13
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 8ce21f2e8fe6d086b3d352ba049cb78f83c99e8b
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit b5d983dc5ea330f6d19c131f566c486a9e414f1a
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 4559aff41e3a8b5444b22fe2cbc79e56079429cd
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit eb728f0f57f66da98b79eee62c60622a96419130
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 0243667adda268ecc21c073cc6892b7726ef6024
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 6d92daa3d0fd78580981a08678b5ff8030fd39ac
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit f5245fa2a2367dfd47d9b9a4b52dc5b94025292a
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] 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 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 889dc4500189996eca2baf9bd89c1d7f738a9a2a
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(