You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jp...@apache.org on 2019/08/08 04:54:52 UTC

[mesos] 04/06: Update `disk/xfs` tests for rootfs quotas.

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

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

commit abf9ae28205947ec967d7d7a33321517b0ba06f3
Author: James Peach <jp...@apache.org>
AuthorDate: Wed Aug 7 20:22:10 2019 -0700

    Update `disk/xfs` tests for rootfs quotas.
    
    Parameterize the `disk/xfs` isolator tests on whether we
    are enforcing quotas on the sandbox or on the container
    rootfs. Separating out the rootfs and sandbox cases wth
    parameterization helps to ensure that quota is correctly
    applied in both cases and tests don't accidentally pass.
    
    Review: https://reviews.apache.org/r/71195/
---
 src/tests/containerizer/rootfs.cpp          |   1 +
 src/tests/containerizer/xfs_quota_tests.cpp | 419 ++++++++++++++++++++++------
 src/tests/mesos.cpp                         |  33 +++
 src/tests/mesos.hpp                         |  18 ++
 4 files changed, 390 insertions(+), 81 deletions(-)

diff --git a/src/tests/containerizer/rootfs.cpp b/src/tests/containerizer/rootfs.cpp
index 206cab6..35a5509 100644
--- a/src/tests/containerizer/rootfs.cpp
+++ b/src/tests/containerizer/rootfs.cpp
@@ -133,6 +133,7 @@ Try<process::Owned<Rootfs>> LinuxRootfs::create(const string& root)
     "/bin/ls",
     "/bin/mkdir",
     "/bin/ping",
+    "/bin/rm",
     "/bin/sh",
     "/bin/sleep",
   };
diff --git a/src/tests/containerizer/xfs_quota_tests.cpp b/src/tests/containerizer/xfs_quota_tests.cpp
index 96cd40d..6703a1c 100644
--- a/src/tests/containerizer/xfs_quota_tests.cpp
+++ b/src/tests/containerizer/xfs_quota_tests.cpp
@@ -49,14 +49,20 @@
 #include "slave/slave.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
+
 #include "slave/containerizer/mesos/containerizer.hpp"
+
 #include "slave/containerizer/mesos/isolators/xfs/disk.hpp"
 #include "slave/containerizer/mesos/isolators/xfs/utils.hpp"
 
+#include "slave/containerizer/mesos/provisioner/backends/overlay.hpp"
+
 #include "tests/environment.hpp"
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
+#include "tests/containerizer/docker_archive.hpp"
+
 using namespace mesos::internal::xfs;
 
 using namespace process;
@@ -205,8 +211,15 @@ public:
     // We only need an XFS-specific directory for the work directory. We
     // don't mind that other flags refer to a different temp directory.
     flags.work_dir = mountPoint.get();
-    flags.isolation = "disk/xfs,filesystem/linux";
+    flags.isolation = "disk/xfs,filesystem/linux,docker/runtime";
     flags.enforce_container_disk_quota = true;
+
+    // Note that the docker registry doesn't need to be on XFS. The
+    // provisioner store does, but we get that via work_dir.
+    flags.docker_registry = path::join(sandbox.get(), "registry");
+    flags.docker_store_dir = path::join(sandbox.get(), "store");
+    flags.image_providers = "docker";
+
     return flags;
   }
 
@@ -303,6 +316,23 @@ public:
 };
 
 
+class ROOT_XFS_QuotaEnforcement
+  : public ROOT_XFS_TestBase,
+    public ::testing::WithParamInterface<ParamDiskQuota::Type>
+{
+public:
+  ROOT_XFS_QuotaEnforcement()
+    : ROOT_XFS_TestBase("prjquota") {}
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    Enforcing,
+    ROOT_XFS_QuotaEnforcement,
+    ::testing::ValuesIn(ParamDiskQuota::parameters()),
+    ParamDiskQuota::Printer());
+
+
 TEST_F(ROOT_XFS_QuotaTest, QuotaGetSet)
 {
   prid_t projectId = 44;
@@ -454,14 +484,22 @@ TEST_F(ROOT_XFS_QuotaTest, DirectoryTree)
 // is only allowed to consume exactly the assigned resources. We tell dd
 // to write 2MB but only give it 1MB of resources and (roughly) verify that
 // it exits with a failure (that should be a write error).
-TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuota)
+TEST_P(ROOT_XFS_QuotaEnforcement, DiskUsageExceedsQuota)
 {
+  slave::Flags flags = CreateSlaveFlags();
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), CreateSlaveFlags());
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -484,10 +522,23 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuota)
 
   // Create a task which requests 1MB disk, but actually uses more
   // than 2MB disk.
-  TaskInfo task = createTask(
-      offer.slave_id(),
-      Resources::parse("cpus:1;mem:128;disk:1").get(),
-      "dd if=/dev/zero of=file bs=1048576 count=2");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=/tmp/file bs=1048576 count=2");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=file bs=1048576 count=2");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -843,16 +894,23 @@ TEST_F(ROOT_XFS_QuotaTest, VolumeUsageExceedsQuotaWithKill)
 
 // This is the same logic as DiskUsageExceedsQuota except we turn off disk quota
 // enforcement, so exceeding the quota should be allowed.
-TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaNoEnforce)
+TEST_P(ROOT_XFS_QuotaEnforcement, DiskUsageExceedsQuotaNoEnforce)
 {
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.enforce_container_disk_quota = false;
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
-  slave::Flags flags = CreateSlaveFlags();
-  flags.enforce_container_disk_quota = false;
-
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
@@ -876,10 +934,24 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaNoEnforce)
 
   // Create a task which requests 1MB disk, but actually uses more
   // than 2MB disk.
-  TaskInfo task = createTask(
-      offer.slave_id(),
-      Resources::parse("cpus:1;mem:128;disk:1").get(),
-      "dd if=/dev/zero of=file bs=1048576 count=2");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=/tmp/file bs=1048576 count=2");
+
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=file bs=1048576 count=2");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -912,13 +984,8 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaNoEnforce)
 
 // Verify that when the `xfs_kill_containers` flag is enabled, tasks that
 // exceed their disk quota are killed with the correct container limitation.
-TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaWithKill)
+TEST_P(ROOT_XFS_QuotaEnforcement, DiskUsageExceedsQuotaWithKill)
 {
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  Owned<MasterDetector> detector = master.get()->createDetector();
-
   slave::Flags flags = CreateSlaveFlags();
 
   // Enable killing containers on disk quota violations.
@@ -928,6 +995,17 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaWithKill)
   // the quota violation as soon as possible.
   flags.container_disk_watch_interval = Milliseconds(1);
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
   Try<Owned<cluster::Slave>> slave =
     StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
@@ -953,10 +1031,23 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaWithKill)
   // Create a task which requests 1MB disk, but actually uses 2MB. This
   // waits a long time to ensure that the task lives long enough for the
   // isolator to impose a container limitation.
-  TaskInfo task = createTask(
-      offer.slave_id(),
-      Resources::parse("cpus:1;mem:128;disk:1").get(),
-      "dd if=/dev/zero of=file bs=1048576 count=2 && sleep 100000");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=/tmp/file bs=1048576 count=2");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offer.slave_id(),
+          Resources::parse("cpus:1;mem:128;disk:1").get(),
+          "pwd; dd if=/dev/zero of=file bs=1048576 count=2");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -1007,18 +1098,25 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaWithKill)
 
 // Verify that we can get accurate resource statistics from the XFS
 // disk isolator.
-TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)
+TEST_P(ROOT_XFS_QuotaEnforcement, ResourceStatistics)
 {
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.resources = strings::format(
+      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  slave::Flags flags = CreateSlaveFlags();
-  flags.resources = strings::format(
-      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
-
   Fetcher fetcher(flags);
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -1062,15 +1160,33 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)
     createDiskResource("3", DEFAULT_TEST_ROLE, None(), None()) +
     volume;
 
-  TaskInfo task = createTask(
-      offers->at(0).slave_id(),
-      taskResources,
-      "touch path1/working && "
-      "touch path1/started && "
-      "dd if=/dev/zero of=file bs=1048576 count=1 && "
-      "dd if=/dev/zero of=path1/file bs=1048576 count=1 && "
-      "rm path1/working && "
-      "sleep 1000");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "echo touch > path1/working && "
+          "echo touch > path1/started && "
+          "dd if=/dev/zero of=/tmp/file bs=1048576 count=1 && "
+          "dd if=/dev/zero of=path1/file bs=1048576 count=1 && "
+          "rm path1/working && "
+          "sleep 1000");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "echo touch > path1/working && "
+          "echo touch > path1/started && "
+          "dd if=/dev/zero of=file bs=1048576 count=1 && "
+          "dd if=/dev/zero of=path1/file bs=1048576 count=1 && "
+          "rm path1/working && "
+          "sleep 1000");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -1137,7 +1253,11 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)
       ASSERT_TRUE(statistics.has_used_bytes());
 
       EXPECT_EQ(Megabytes(3), Bytes(statistics.limit_bytes()));
-      EXPECT_EQ(Megabytes(1), Bytes(statistics.used_bytes()));
+
+      // We can't precisely control how much ephemeral space is consumed
+      // by the rootfs, so check a reasonable range.
+      EXPECT_GE(Bytes(statistics.used_bytes()), Megabytes(1));
+      EXPECT_LE(Bytes(statistics.used_bytes()), Kilobytes(1400));
 
       EXPECT_EQ("id1", statistics.persistence().id());
       EXPECT_EQ(
@@ -1161,19 +1281,27 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)
 // This is the same logic as ResourceStatistics, except the task should
 // be allowed to exceed the disk quota, and usage statistics should report
 // that the quota was exceeded.
-TEST_F(ROOT_XFS_QuotaTest, ResourceStatisticsNoEnforce)
+TEST_P(ROOT_XFS_QuotaEnforcement, ResourceStatisticsNoEnforce)
 {
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.enforce_container_disk_quota = false;
+
+  flags.resources = strings::format(
+      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  slave::Flags flags = CreateSlaveFlags();
-  flags.enforce_container_disk_quota = false;
-  flags.resources = strings::format(
-      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
-
   Fetcher fetcher(flags);
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -1217,15 +1345,33 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatisticsNoEnforce)
     createDiskResource("1", DEFAULT_TEST_ROLE, None(), None()) +
     volume;
 
-  TaskInfo task = createTask(
-      offers->at(0).slave_id(),
-      taskResources,
-      "touch path1/working && "
-      "touch path1/started && "
-      "dd if=/dev/zero of=file bs=1048576 count=2 && "
-      "dd if=/dev/zero of=path1/file bs=1048576 count=2 && "
-      "rm path1/working && "
-      "sleep 1000");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "echo touch > path1/working && "
+          "echo touch > path1/started && "
+          "dd if=/dev/zero of=/tmp/file bs=1048576 count=2 && "
+          "dd if=/dev/zero of=path1/file bs=1048576 count=2 && "
+          "rm path1/working && "
+          "sleep 1000");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "echo touch > path1/working && "
+          "echo touch > path1/started && "
+          "dd if=/dev/zero of=file bs=1048576 count=2 && "
+          "dd if=/dev/zero of=path1/file bs=1048576 count=2 && "
+          "rm path1/working && "
+          "sleep 1000");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -1291,7 +1437,10 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatisticsNoEnforce)
       ASSERT_TRUE(statistics.has_used_bytes());
 
       EXPECT_EQ(Megabytes(1), Bytes(statistics.limit_bytes()));
-      EXPECT_EQ(Megabytes(2), Bytes(statistics.used_bytes()));
+
+      // We can't precisely control how much ephemeral space is consumed
+      // by the rootfs, so check a reasonable range.
+      EXPECT_GE(Bytes(statistics.used_bytes()), Megabytes(2));
 
       EXPECT_EQ("id1", statistics.persistence().id());
       EXPECT_EQ(
@@ -1315,18 +1464,25 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatisticsNoEnforce)
 // In this test, the framework is not checkpointed. This ensures that when we
 // stop the slave, the executor is killed and we will need to recover the
 // working directories without getting any checkpointed recovery state.
-TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
+TEST_P(ROOT_XFS_QuotaEnforcement, NoCheckpointRecovery)
 {
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.resources = strings::format(
+      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  slave::Flags flags = CreateSlaveFlags();
-  flags.resources = strings::format(
-      "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
-
   Fetcher fetcher(flags);
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -1374,10 +1530,23 @@ TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
     createDiskResource("1", DEFAULT_TEST_ROLE, None(), None()) +
     volume;
 
-  TaskInfo task = createTask(
-      offers->at(0).slave_id(),
-      taskResources,
-      "dd if=/dev/zero of=file bs=1048576 count=1; sleep 1000");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "dd if=/dev/zero of=/tmp/file bs=1048576 count=1; sleep 1000");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "dd if=/dev/zero of=file bs=1048576 count=1; sleep 1000");
+      break;
+  }
 
   Future<TaskStatus> runningStatus;
   Future<TaskStatus> startingStatus;
@@ -1475,6 +1644,28 @@ TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
     EXPECT_SOME(xfs::getProjectQuota(path, projectId.get()));
   }
 
+  // Since we are not checkpointing, the rootfs should be gone.
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    int count = 0;
+
+    Try<list<string>> dirs =
+      slave::OverlayBackend::listEphemeralVolumes(flags.work_dir);
+
+    ASSERT_SOME(dirs);
+
+    foreach (const string& dir, dirs.get()) {
+      Result<prid_t> projectId = xfs::getProjectId(dir);
+      ASSERT_FALSE(projectId.isError()) << projectId.error();
+
+      EXPECT_NONE(projectId);
+      if (projectId.isSome()) {
+        ++count;
+      }
+    }
+
+    EXPECT_EQ(0, count);
+  }
+
   // We should have project IDs still allocated for the persistent volume and
   // for the task sandbox (since it is not GC'd yet).
   JSON::Object metrics = Metrics();
@@ -1492,16 +1683,23 @@ TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
 // In this test, the framework is checkpointed so we expect the executor to
 // persist across the slave restart and to have the same resource usage before
 // and after.
-TEST_F(ROOT_XFS_QuotaTest, CheckpointRecovery)
+TEST_P(ROOT_XFS_QuotaEnforcement, CheckpointRecovery)
 {
-  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
-  frameworkInfo.set_checkpoint(true);
-
   slave::Flags flags = CreateSlaveFlags();
+
   flags.resources = strings::format(
       "disk(%s):%d", DEFAULT_TEST_ROLE, DISK_SIZE_MB).get();
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
+  frameworkInfo.set_checkpoint(true);
+
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
@@ -1539,10 +1737,23 @@ TEST_F(ROOT_XFS_QuotaTest, CheckpointRecovery)
     createDiskResource("1", DEFAULT_TEST_ROLE, None(), None()) +
     volume;
 
-  TaskInfo task = createTask(
-      offers->at(0).slave_id(),
-      taskResources,
-      "dd if=/dev/zero of=file bs=1048576 count=1; sleep 1000");
+  TaskInfo task;
+
+  switch (GetParam()) {
+    case ParamDiskQuota::ROOTFS:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "dd if=/dev/zero of=/tmp/file bs=1048576 count=1; sleep 1000");
+      task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+      break;
+    case ParamDiskQuota::SANDBOX:
+      task = createTask(
+          offers->at(0).slave_id(),
+          taskResources,
+          "dd if=/dev/zero of=file bs=1048576 count=1; sleep 1000");
+      break;
+  }
 
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
@@ -1624,6 +1835,28 @@ TEST_F(ROOT_XFS_QuotaTest, CheckpointRecovery)
     EXPECT_SOME(xfs::getProjectQuota(path, projectId.get()));
   }
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    int count = 0;
+    Try<list<string>> dirs =
+      slave::OverlayBackend::listEphemeralVolumes(flags.work_dir);
+
+    ASSERT_SOME(dirs);
+    EXPECT_FALSE(dirs->empty());
+
+    foreach (const string& dir, dirs.get()) {
+      Result<prid_t> projectId = xfs::getProjectId(dir);
+      ASSERT_FALSE(projectId.isError()) << projectId.error();
+
+      if (projectId.isSome()) {
+        ++count;
+        EXPECT_SOME(xfs::getProjectQuota(dir, projectId.get()));
+      }
+    }
+
+    EXPECT_GT(1, count)
+      << "overlay provisioner backend is missing project IDs";
+  }
+
   JSON::Object metrics = Metrics();
   EXPECT_EQ(
       metrics.at<JSON::Number>("containerizer/mesos/disk/project_ids_total")
@@ -1638,9 +1871,9 @@ TEST_F(ROOT_XFS_QuotaTest, CheckpointRecovery)
 
 // In this test, the agent initially doesn't enable disk isolation
 // but then restarts with XFS disk isolation enabled. We verify that
-// the old container launched before the agent restart is
+// the old container that was launched before the agent restart is
 // successfully recovered.
-TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
+TEST_P(ROOT_XFS_QuotaEnforcement, RecoverOldContainers)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -1650,8 +1883,14 @@ TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
   slave::Flags flags = CreateSlaveFlags();
 
   // `CreateSlaveFlags()` enables `disk/xfs` so here we reset
-  // `isolation` to empty.
-  flags.isolation.clear();
+  // `isolation` to remove it.
+  flags.isolation = "filesystem/linux,docker/runtime";
+
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
 
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
@@ -1677,11 +1916,17 @@ TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
 
   Offer offer = offers.get()[0];
 
-  TaskInfo task = createTask(
+  TaskInfo task;
+
+  task = createTask(
       offer.slave_id(),
       Resources::parse("cpus:1;mem:128;disk:1").get(),
       "dd if=/dev/zero of=file bs=1024 count=1; sleep 1000");
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+  }
+
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningstatus;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -1736,6 +1981,7 @@ TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
     const ResourceUsage_Executor& executor = usage->executors().Get(0);
     ASSERT_TRUE(executor.has_statistics());
     ASSERT_FALSE(executor.statistics().has_disk_limit_bytes());
+    ASSERT_FALSE(executor.statistics().has_disk_used_bytes());
   }
 
   // Verify that we haven't allocated any project IDs.
@@ -1753,7 +1999,7 @@ TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
 
 // Verify that XFS project IDs are reclaimed when sandbox directories they were
 // set on are garbage collected.
-TEST_F(ROOT_XFS_QuotaTest, ProjectIdReclaiming)
+TEST_P(ROOT_XFS_QuotaEnforcement, ProjectIdReclaiming)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -1761,9 +2007,16 @@ TEST_F(ROOT_XFS_QuotaTest, ProjectIdReclaiming)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   slave::Flags flags = CreateSlaveFlags();
+
   flags.gc_delay = Seconds(10);
   flags.disk_watch_interval = Seconds(10);
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    flags.image_provisioner_backend = "overlay";
+
+    AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+  }
+
   Try<Resource> projects =
     Resources::parse("projects", flags.xfs_project_range, "*");
   ASSERT_SOME(projects);
@@ -1813,6 +2066,10 @@ TEST_F(ROOT_XFS_QuotaTest, ProjectIdReclaiming)
       Resources::parse("cpus:1;mem:128;disk:2").get(),
       "dd if=/dev/zero of=file bs=1048576 count=1 && sleep 1000");
 
+  if (GetParam() == ParamDiskQuota::ROOTFS) {
+    task.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+  }
+
   Future<TaskStatus> startingStatus;
   Future<TaskStatus> runningStatus;
   Future<TaskStatus> exitStatus;
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index df363ab..0396ce7 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -35,6 +35,7 @@
 
 #ifdef __linux__
 #include "linux/cgroups.hpp"
+#include "linux/fs.hpp"
 #endif
 
 #ifdef ENABLE_PORT_MAPPING_ISOLATOR
@@ -56,6 +57,7 @@ using mesos::master::detector::MasterDetector;
 using std::list;
 using std::shared_ptr;
 using std::string;
+using std::vector;
 
 using testing::_;
 
@@ -935,6 +937,37 @@ void ContainerizerTest<slave::MesosContainerizer>::TearDown()
 }
 #endif // __linux__
 
+
+string ParamDiskQuota::Printer::operator()(
+  const ::testing::TestParamInfo<ParamDiskQuota::Type>& info) const
+{
+  switch (info.param) {
+    case SANDBOX:
+      return "Sandbox";
+    case ROOTFS:
+      return "Rootfs";
+    default:
+      UNREACHABLE();
+  }
+}
+
+
+vector<ParamDiskQuota::Type> ParamDiskQuota::parameters()
+{
+  vector<Type> params{SANDBOX};
+
+  // ROOTFS tests depend on overlayfs being available, since that is
+  // the only provisioner backend that supports ephemeral volumes.
+#if __linux__
+  Try<bool> overlayfsSupported = fs::supported("overlayfs");
+  if (overlayfsSupported.isSome() && overlayfsSupported.get()) {
+    params.push_back(ROOTFS);
+  }
+#endif
+
+  return params;
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index cd50673..73b6e42 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -3919,6 +3919,24 @@ private:
   Type type;
 };
 
+
+struct ParamDiskQuota
+{
+  enum Type
+  {
+    SANDBOX,
+    ROOTFS,
+  };
+
+  struct Printer
+  {
+    std::string operator()(
+        const ::testing::TestParamInfo<ParamDiskQuota::Type>& info) const;
+  };
+
+  static std::vector<Type> parameters();
+};
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {