You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2017/02/15 01:10:26 UTC

aurora git commit: Add DSL and E2E changes for per task volume mounts.

Repository: aurora
Updated Branches:
  refs/heads/master 0e9c0864e -> 9ea897978


Add DSL and E2E changes for per task volume mounts.

Enables the client DSL to set per task volume mounts. This also adds a E2E test
that tests per task volume mounting.

Testing Done:
sh ./src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh

Bugs closed: AURORA-1107

Reviewed at https://reviews.apache.org/r/53333/


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

Branch: refs/heads/master
Commit: 9ea897978a41d71121033fe5f2115465ec76581c
Parents: 0e9c086
Author: Zameer Manji <zm...@apache.org>
Authored: Tue Feb 14 17:09:37 2017 -0800
Committer: Zameer Manji <zm...@apache.org>
Committed: Tue Feb 14 17:09:37 2017 -0800

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  1 +
 docs/reference/configuration.md                 |  9 +++++++
 examples/vagrant/upstart/aurora-scheduler.conf  |  1 +
 .../python/apache/aurora/config/schema/base.py  |  7 ++++++
 src/main/python/apache/aurora/config/thrift.py  | 23 ++++++++++++++++--
 .../python/apache/aurora/config/test_thrift.py  | 25 +++++++++++++++++++-
 .../sh/org/apache/aurora/e2e/Dockerfile.netcat  |  1 +
 src/test/sh/org/apache/aurora/e2e/check-fs.sh   |  6 +++++
 .../apache/aurora/e2e/http/http_example.aurora  | 12 ++++++++--
 src/test/sh/org/apache/aurora/e2e/run-server.sh |  1 -
 10 files changed, 80 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index ff382ff..2391d32 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -5,6 +5,7 @@
 
 - Add message parameter to `killTasks` RPC.
 - Add prune_tasks endpoint to aurora_admin. See aurora_admin prune_tasks -h for usage information.
+- Add support for per-task volume mounts for Mesos containers to the Aurora config DSL.
 
 0.17.0
 ======

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/docs/reference/configuration.md
----------------------------------------------------------------------
diff --git a/docs/reference/configuration.md b/docs/reference/configuration.md
index 6c71142..0040de1 100644
--- a/docs/reference/configuration.md
+++ b/docs/reference/configuration.md
@@ -468,6 +468,15 @@ unified-container, the container can be omitted from your job config.
   param            | type                           | description
   -----            | :----:                         | -----------
   ```image```      | Choice(AppcImage, DockerImage) | An optional filesystem image to use within this container.
+  ```volumes```    | List(Volume)                   | An optional list of volume mounts for this container.
+
+### Volume Object
+
+  param                  | type     | description
+  -----                  | :----:   | -----------
+  ```container_path```   | String   | Path on the host to mount.
+  ```volume_path```      | String   | Mount point in the container.
+  ```mode```             | Enum     | Mode of the mount, can be 'RW' or 'RO'.
 
 ### AppcImage
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/examples/vagrant/upstart/aurora-scheduler.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/aurora-scheduler.conf b/examples/vagrant/upstart/aurora-scheduler.conf
index 91f0f0c..49fdcbd 100644
--- a/examples/vagrant/upstart/aurora-scheduler.conf
+++ b/examples/vagrant/upstart/aurora-scheduler.conf
@@ -53,4 +53,5 @@ exec bin/aurora-scheduler \
   -receive_revocable_resources=true \
   -enable_revocable_ram=true \
   -allow_gpu_resource=true \
+  -allow_container_volumes=true \
   -offer_filter_duration=0secs

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/main/python/apache/aurora/config/schema/base.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/schema/base.py b/src/main/python/apache/aurora/config/schema/base.py
index b15b939..b2692a6 100644
--- a/src/main/python/apache/aurora/config/schema/base.py
+++ b/src/main/python/apache/aurora/config/schema/base.py
@@ -132,9 +132,16 @@ class DockerImage(Struct):
   name = Required(String)
   tag = Required(String)
 
+Mode = Enum('RO', 'RW')
+
+class Volume(Struct):
+  container_path = Required(String)
+  host_path = Required(String)
+  mode = Required(Mode)
 
 class Mesos(Struct):
   image = Choice([AppcImage, DockerImage])
+  volumes = Default(List(Volume), [])
 
 
 class Container(Struct):

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/main/python/apache/aurora/config/thrift.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/thrift.py b/src/main/python/apache/aurora/config/thrift.py
index 3539469..2158c3a 100644
--- a/src/main/python/apache/aurora/config/thrift.py
+++ b/src/main/python/apache/aurora/config/thrift.py
@@ -47,10 +47,12 @@ from gen.apache.aurora.api.ttypes import (
     LimitConstraint,
     MesosContainer,
     Metadata,
+    Mode,
     Resource,
     TaskConfig,
     TaskConstraint,
-    ValueConstraint
+    ValueConstraint,
+    Volume
 )
 
 __all__ = (
@@ -163,11 +165,28 @@ def create_container_config(container):
          else unwrapped)))
 
   if isinstance(unwrapped, Mesos):
-    return Container(MesosContainer(image_to_thrift(unwrapped.image())), None)
+    image = image_to_thrift(unwrapped.image())
+    volumes = volumes_to_thrift(unwrapped.volumes())
+
+    return Container(MesosContainer(image, volumes), None)
 
   raise InvalidConfig('If a container is specified it must set one type.')
 
 
+def volumes_to_thrift(volumes):
+  thrift_volumes = []
+  for v in volumes:
+    mode = parse_enum(Mode, v.mode())
+    thrift_volumes.append(
+      Volume(
+          containerPath=fully_interpolated(v.container_path()),
+          hostPath=fully_interpolated(v.host_path()),
+          mode=mode
+      )
+    )
+  return thrift_volumes
+
+
 def image_to_thrift(image):
   if image is Empty:
     return None

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/test/python/apache/aurora/config/test_thrift.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/config/test_thrift.py b/src/test/python/apache/aurora/config/test_thrift.py
index e213184..2a6a5d0 100644
--- a/src/test/python/apache/aurora/config/test_thrift.py
+++ b/src/test/python/apache/aurora/config/test_thrift.py
@@ -26,14 +26,17 @@ from apache.aurora.config.schema.base import (
     HealthCheckConfig,
     Job,
     Mesos,
+    Mode,
     Parameter,
-    SimpleTask
+    SimpleTask,
+    Volume
 )
 from apache.aurora.config.thrift import convert as convert_pystachio_to_thrift
 from apache.aurora.config.thrift import InvalidConfig, task_instance_from_job
 from apache.thermos.config.schema import Process, Resources, Task
 
 from gen.apache.aurora.api.constants import GOOD_IDENTIFIER_PATTERN_PYTHON
+from gen.apache.aurora.api.ttypes import Mode as ThriftMode
 from gen.apache.aurora.api.ttypes import CronCollisionPolicy, Identity, JobKey, Resource
 from gen.apache.aurora.test.constants import INVALID_IDENTIFIERS, VALID_IDENTIFIERS
 
@@ -108,6 +111,26 @@ def test_config_with_appc_image():
   assert job.taskConfig.container.mesos.image.appc.imageId == image_id
 
 
+def test_config_with_volumes():
+  image_name = 'some-image'
+  image_tag = 'some-tag'
+  host_path = '/etc/secrets/role/'
+  container_path = '/etc/secrets/'
+
+  volume = Volume(host_path=host_path, container_path=container_path, mode=Mode('RO'))
+
+  container = Mesos(image=DockerImage(name=image_name, tag=image_tag), volumes=[volume])
+
+  job = convert_pystachio_to_thrift(HELLO_WORLD(container=container))
+
+  assert len(job.taskConfig.container.mesos.volumes) == 1
+  thrift_volume = job.taskConfig.container.mesos.volumes[0]
+
+  assert thrift_volume.hostPath == host_path
+  assert thrift_volume.containerPath == container_path
+  assert thrift_volume.mode == ThriftMode.RO
+
+
 def test_docker_with_parameters():
   helloworld = HELLO_WORLD(
     container=Container(

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/test/sh/org/apache/aurora/e2e/Dockerfile.netcat
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/Dockerfile.netcat b/src/test/sh/org/apache/aurora/e2e/Dockerfile.netcat
index c8b2f46..17a028e 100644
--- a/src/test/sh/org/apache/aurora/e2e/Dockerfile.netcat
+++ b/src/test/sh/org/apache/aurora/e2e/Dockerfile.netcat
@@ -16,3 +16,4 @@ FROM buildpack-deps:jessie
 
 RUN apt-get update && apt-get install -y netcat-openbsd
 COPY run-server.sh /usr/local/bin
+COPY check-fs.sh /usr/local/bin

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/test/sh/org/apache/aurora/e2e/check-fs.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/check-fs.sh b/src/test/sh/org/apache/aurora/e2e/check-fs.sh
new file mode 100755
index 0000000..03221c0
--- /dev/null
+++ b/src/test/sh/org/apache/aurora/e2e/check-fs.sh
@@ -0,0 +1,6 @@
+#!/bin/bash
+echo "Checking volume mounts..."
+if [ ! -f "/etc/rsyslog.d.container/50-default.conf" ]; then
+    echo "Mounted file was not found";
+    exit 1
+fi

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
index b2b977b..de81792 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
@@ -70,6 +70,11 @@ if __name__ == "__main__":
 """
 )
 
+verify_file_system = Process(
+  name = 'check_fs',
+  cmdline = 'check-fs.sh'
+)
+
 test_task = SequentialTask(
   name = 'http_example',
   resources = Resources(cpu=0.5, ram=32*MB, disk=64*MB, gpu='{{profile.gpu}}'),
@@ -92,6 +97,7 @@ no_python_task = SequentialTask(
       echo_ports,
       verify_command_escaping,
       verify_file_mount,
+      verify_file_system,
       Process(name='run_server', cmdline='run-server.sh {{thermos.ports[http]}}'),
   ]
 )
@@ -121,6 +127,8 @@ job = Service(
   ),
 )
 
+volumes = [Volume(container_path="etc/rsyslog.d.container", host_path="/etc/rsyslog.d", mode=Mode("RO"))]
+
 jobs = [
   job(
     name = 'http_example'
@@ -142,13 +150,13 @@ jobs = [
   ).bind(profile=GpuProfile),
   job(
     name = 'http_example_unified_appc',
-    container = Mesos(image=AppcImage(name='http_example_netcat', image_id='{{appc_image_id}}')),
+    container = Mesos(image=AppcImage(name='http_example_netcat', image_id='{{appc_image_id}}'), volumes=volumes),
     task = no_python_task,
     health_check_config=shell_health_check_config
   ).bind(profile=DefaultProfile()),
   job(
     name = 'http_example_unified_docker',
-    container = Mesos(image=DockerImage(name='http_example_netcat', tag='latest')),
+    container = Mesos(image=DockerImage(name='http_example_netcat', tag='latest'), volumes=volumes),
     task = no_python_task,
     health_check_config=shell_health_check_config
   ).bind(profile=DefaultProfile())

http://git-wip-us.apache.org/repos/asf/aurora/blob/9ea89797/src/test/sh/org/apache/aurora/e2e/run-server.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/run-server.sh b/src/test/sh/org/apache/aurora/e2e/run-server.sh
index 1fe0909..834f723 100755
--- a/src/test/sh/org/apache/aurora/e2e/run-server.sh
+++ b/src/test/sh/org/apache/aurora/e2e/run-server.sh
@@ -1,5 +1,4 @@
 #!/bin/bash
-
 echo "Starting up server..."
 while true
 do