You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@aurora.apache.org by "Justin Venus (JIRA)" <ji...@apache.org> on 2018/04/03 21:41:00 UTC

[jira] [Commented] (AURORA-1983) Support for Docker Volume Isolator

    [ https://issues.apache.org/jira/browse/AURORA-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16424650#comment-16424650 ] 

Justin Venus commented on AURORA-1983:
--------------------------------------

From Commit: 70d043ebbb31aad4b19aad4aba486b742d2ab274 [~rdelvalle] I asked about this yesterday in Slack.  It passes units ... I'll start testing in my environment soon.  It's missing the wiring for setting the driver options on the scheduler, but otherwise I think it is close to complete.
{code:java}
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index ef754e321..938ae0bd5 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -169,14 +169,54 @@ enum Mode {
RO = 2
}

+struct HostPath {
+ /** The path on the host that will serve as the source for the mount. */
+ 1: string path
+}
+
+/** Describes a parameter passed to docker cli */
+struct DockerParameter {
+ /** a parameter to pass to docker. (e.g. volume) */
+ 1: string name
+ /** the value to pass to a parameter (e.g. /src/webapp:/opt/webapp) */
+ 2: string value
+}
+
+struct DockerVolume {
+ /** The driver name on the host facilitate mounting the resource. */
+ 1: string driver
+ /** The resource name that will serve as the source for the mount. */
+ 2: string name
+ /** The specific options to pass on to the driver. */
+ 3: optional list<DockerParameter> options
+}
+
+/** The type of volume mount */
+enum VolumeType {
+ /** Represent container volume type 'HOST_PATH' requires isolator 'filesystem/linux' */
+ HOST_PATH = 1
+ /** Represent container volume type 'DOCKER_VOLUME' requires isolator 'docker/volume' */
+ DOCKER_VOLUME = 2
+}
+
+union VolumeSource {
+ 1: HostPath host
+ 2: DockerVolume docker
+}
+
+
/** A volume mount point within a container */
struct Volume {
/** The path inside the container where the mount will be created. */
1: string containerPath
- /** The path on the host that will serve as the source for the mount. */
+ /** The path on the host that will serve as the source for the mount. This is deprecated with AURORA-1983 */
2: string hostPath
/** The access mode */
3: Mode mode
+ /** The volume type - set optional for backwards compatibility */
+ 4: optional VolumeType volumeType
+ /** The volume source - set optional for backwards compatibility */
+ 5: optional VolumeSource source
}

/** Describes an image for use with the Mesos unified containerizer in the Docker format */
@@ -208,15 +248,6 @@ struct MesosContainer {
/** the optional list of volumes to mount into the task. */
2: optional list<Volume> volumes
}
-
-/** Describes a parameter passed to docker cli */
-struct DockerParameter {
- /** a parameter to pass to docker. (e.g. volume) */
- 1: string name
- /** the value to pass to a parameter (e.g. /src/webapp:/opt/webapp) */
- 2: string value
-}
-
/** Describes a docker container */
struct DockerContainer {
/** The container image to be run */
diff --git a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
index bcb2bbf88..bcee5dbc9 100644
--- a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
+++ b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
@@ -27,6 +27,7 @@ import com.google.protobuf.ByteString;

import org.apache.aurora.Protobufs;
import org.apache.aurora.codec.ThriftBinaryCodec;
+import org.apache.aurora.gen.VolumeType;
import org.apache.aurora.scheduler.base.JobKeys;
import org.apache.aurora.scheduler.base.SchedulerException;
import org.apache.aurora.scheduler.base.Tasks;
@@ -43,6 +44,7 @@ import org.apache.aurora.scheduler.storage.entities.IJobKey;
import org.apache.aurora.scheduler.storage.entities.IMesosContainer;
import org.apache.aurora.scheduler.storage.entities.IServerInfo;
import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
+import org.apache.aurora.scheduler.storage.entities.IVolume;
import org.apache.mesos.v1.Protos;
import org.apache.mesos.v1.Protos.CommandInfo;
import org.apache.mesos.v1.Protos.ContainerInfo;
@@ -251,11 +253,7 @@ public interface MesosTaskFactory {
ContainerInfo.MesosInfo.newBuilder();

Iterable<Protos.Volume> containerVolumes = Iterables.transform(mesosContainer.getVolumes(),
- input -> Protos.Volume.newBuilder()
- .setMode(Protos.Volume.Mode.valueOf(input.getMode().name()))
- .setHostPath(input.getHostPath())
- .setContainerPath(input.getContainerPath())
- .build());
+ input -> getContainerVolume(input));

Protos.Volume volume = Protos.Volume.newBuilder()
.setImage(imageBuilder)
@@ -274,6 +272,36 @@ public interface MesosTaskFactory {
return Optional.empty();
}

+ private Protos.Volume getContainerVolume(IVolume volume) {
+ requireNonNull(volume);
+ // deprecated see AURORA-1983
+ if (volume.getHostPath() != "/deprecated") {
+ return Protos.Volume.newBuilder()
+ .setMode(Protos.Volume.Mode.valueOf(volume.getMode().name()))
+ .setHostPath(volume.getHostPath())
+ .setContainerPath(volume.getContainerPath())
+ .build();
+ } else if (volume.getVolumeType() == VolumeType.HOST_PATH) {
+ return Protos.Volume.newBuilder()
+ .setMode(Protos.Volume.Mode.valueOf(volume.getMode().name()))
+ .setHostPath(volume.getSource().getHost().getPath())
+ .setContainerPath(volume.getContainerPath())
+ .build();
+ } else if (volume.getVolumeType() == VolumeType.DOCKER_VOLUME) {
+ return Protos.Volume.newBuilder()
+ .setMode(Protos.Volume.Mode.valueOf(volume.getMode().name()))
+ .setSource(Protos.Volume.Source.newBuilder()
+ .setDockerVolume(Protos.Volume.Source.DockerVolume.newBuilder()
+ .setDriver(volume.getSource().getDocker().getDriver())
+ .setName(volume.getSource().getDocker().getName())
+ .build())
+ .build())
+ .setContainerPath(volume.getContainerPath())
+ .build();
+ }
+ throw new SchedulerException("Task had no supported volume type.");
+ }
+
private ContainerInfo getDockerContainerInfo(
IDockerContainer config,
Optional<String> executorName) {
diff --git a/src/main/python/apache/aurora/config/schema/base.py b/src/main/python/apache/aurora/config/schema/base.py
index a629bcd12..0e94b9e1a 100644
--- a/src/main/python/apache/aurora/config/schema/base.py
+++ b/src/main/python/apache/aurora/config/schema/base.py
@@ -141,10 +141,24 @@ class DockerImage(Struct):

Mode = Enum('RO', 'RW')

+
+class HostPath(Struct):
+ path = Required(String)
+
+
+class DockerVolume(Struct):
+ driver = Required(String)
+ name = Required(String)
+ options = Default(List(Parameter), [])
+
+
+DEPRECATED_PATH_SENTINEL = "deprecated"
+
class Volume(Struct):
container_path = Required(String)
- host_path = Required(String)
+ host_path = Default(String, DEPRECATED_PATH_SENTINEL)
mode = Required(Mode)
+ source = Default(Choice([HostPath, DockerVolume]), HostPath(path='{{host_path}}'))

class Mesos(Struct):
image = Choice([AppcImage, DockerImage])
diff --git a/src/main/python/apache/aurora/config/thrift.py b/src/main/python/apache/aurora/config/thrift.py
index 6d2dde6e9..c30f02d8c 100644
--- a/src/main/python/apache/aurora/config/thrift.py
+++ b/src/main/python/apache/aurora/config/thrift.py
@@ -21,7 +21,10 @@ from twitter.common.lang import Compatibility
from apache.aurora.config.schema.base import AppcImage as PystachioAppcImage
from apache.aurora.config.schema.base import Container as PystachioContainer
from apache.aurora.config.schema.base import DockerImage as PystachioDockerImage
+from apache.aurora.config.schema.base import DockerVolume as PystachioDockerVolume
+from apache.aurora.config.schema.base import HostPath as PystachioHostPath
from apache.aurora.config.schema.base import (
+ DEPRECATED_PATH_SENTINEL,
Docker,
HealthCheckConfig,
Mesos,
@@ -39,7 +42,9 @@ from gen.apache.aurora.api.ttypes import (
DockerContainer,
DockerImage,
DockerParameter,
+ DockerVolume,
ExecutorConfig,
+ HostPath,
Identity,
Image,
JobConfiguration,
@@ -53,7 +58,8 @@ from gen.apache.aurora.api.ttypes import (
TaskConfig,
TaskConstraint,
ValueConstraint,
- Volume
+ Volume,
+ VolumeType
)

__all__ = (
@@ -174,15 +180,40 @@ def create_container_config(container):
raise InvalidConfig('If a container is specified it must set one type.')


+def source_volume_to_thrift(volume, hostPath):
+ # deprecated see AURORA-1983
+ if hostPath != DEPRECATED_PATH_SENTINEL:
+ return VolumeType.HOST_PATH, HostPath(path=hostPath)
+
+ source = volume.source().unwrap()
+ if isinstance(source, PystachioDockerVolume):
+ options = []
+ for option in source.options():
+ options.append(
+ DockerParameter(fully_interpolated(option.name()), fully_interpolated(option.value())))
+ return VolumeType.DOCKER_VOLUME, DockerVolume(
+ driver=fully_interpolated(source.driver()),
+ name=fully_interpolated(source.name()),
+ options=options)
+ elif isinstance(source, PystachioHostPath):
+ path = fully_interpolated(source.path())
+ if path != DEPRECATED_PATH_SENTINEL:
+ return VolumeType.HOST_PATH, HostPath(path=path)
+ raise InvalidConfig('If a volume is specified it must set one type.')
+
def volumes_to_thrift(volumes):
thrift_volumes = []
for v in volumes:
mode = parse_enum(Mode, v.mode())
+ hostPath = fully_interpolated(v.host_path())
+ type_, source = source_volume_to_thrift(v, hostPath)
thrift_volumes.append(
Volume(
containerPath=fully_interpolated(v.container_path()),
- hostPath=fully_interpolated(v.host_path()),
- mode=mode
+ hostPath=hostPath,
+ mode=mode,
+ source=source,
+ volumeType=type_
)
)
return thrift_volumes
diff --git a/src/test/python/apache/aurora/config/test_thrift.py b/src/test/python/apache/aurora/config/test_thrift.py
index 8e1d0e177..6c2ee6f1d 100644
--- a/src/test/python/apache/aurora/config/test_thrift.py
+++ b/src/test/python/apache/aurora/config/test_thrift.py
@@ -23,10 +23,13 @@ from apache.aurora.config.schema.base import PartitionPolicy as PystachioPartiti
from apache.aurora.config.schema.base import (
AppcImage,
Container,
+ DEPRECATED_PATH_SENTINEL,
Docker,
DockerImage,
+ DockerVolume,
ExecutorConfig,
HealthCheckConfig,
+ HostPath,
Job,
Mesos,
Metadata,
@@ -46,7 +49,8 @@ from gen.apache.aurora.api.ttypes import (
Identity,
JobKey,
PartitionPolicy,
- Resource
+ Resource,
+ VolumeType
)
from gen.apache.aurora.test.constants import INVALID_IDENTIFIERS, VALID_IDENTIFIERS

@@ -186,6 +190,7 @@ def test_config_with_appc_image():
assert job.taskConfig.container.mesos.image.appc.imageId == image_id


+# deprecated see AURORA-1983
def test_config_with_volumes():
image_name = 'some-image'
image_tag = 'some-tag'
@@ -203,8 +208,56 @@ def test_config_with_volumes():

assert thrift_volume.hostPath == host_path
assert thrift_volume.containerPath == container_path
+ assert thrift_volume.volumeType == VolumeType.HOST_PATH
assert thrift_volume.mode == ThriftMode.RO
+ assert thrift_volume.source.path == host_path

+def test_config_with_host_volumes():
+ image_name = 'some-image'
+ image_tag = 'some-tag'
+ host_path = '/etc/secrets/role/'
+ container_path = '/etc/secrets/'
+ source = HostPath(path=host_path)
+
+ volume = Volume(container_path=container_path, mode=Mode('RO'), source=source)
+
+ 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 == DEPRECATED_PATH_SENTINEL
+ assert thrift_volume.containerPath == container_path
+ assert thrift_volume.volumeType == VolumeType.HOST_PATH
+ assert thrift_volume.mode == ThriftMode.RO
+ assert thrift_volume.source.path == host_path
+
+def test_config_with_docker_volumes():
+ image_name = 'some-image'
+ image_tag = 'some-tag'
+ container_path = '/etc/secrets/'
+ source = DockerVolume(
+ driver="rexray", name="someEBSVolume", options=[Parameter(name="--foo", value="bar")])
+
+ volume = Volume(container_path=container_path, mode=Mode('RO'), source=source)
+
+ 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 == DEPRECATED_PATH_SENTINEL
+ assert thrift_volume.containerPath == container_path
+ assert thrift_volume.mode == ThriftMode.RO
+ assert thrift_volume.volumeType == VolumeType.DOCKER_VOLUME
+ assert thrift_volume.source.driver == source.driver().get()
+ assert thrift_volume.source.name == source.name().get()
+ assert thrift_volume.source.options[0].name == "--foo"
+ assert thrift_volume.source.options[0].value == "bar"

def test_docker_with_parameters():
helloworld = HELLO_WORLD(
{code}

> Support for Docker Volume Isolator
> ----------------------------------
>
>                 Key: AURORA-1983
>                 URL: https://issues.apache.org/jira/browse/AURORA-1983
>             Project: Aurora
>          Issue Type: Story
>            Reporter: Justin Venus
>            Priority: Minor
>
> It would be really useful to support [docker/volume|http://mesos.apache.org/documentation/latest/isolators/docker-volume/] isolation in Aurora.  This would allow for example ... operators in AWS to be able to easily attach EBS volumes to their containers.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)