blob: 3dd4a5b7a4b3bc56bdc690d6adf05f88c0d28273 [file] [log] [blame]
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package mesos;
option java_package = "org.apache.mesos";
option java_outer_classname = "Protos";
/**
* Status is used to indicate the state of the scheduler and executor
* driver after function calls.
*/
enum Status {
DRIVER_NOT_STARTED = 1;
DRIVER_RUNNING = 2;
DRIVER_ABORTED = 3;
DRIVER_STOPPED = 4;
}
/**
* A unique ID assigned to a framework. A framework can reuse this ID
* in order to do failover (see MesosSchedulerDriver).
*/
message FrameworkID {
required string value = 1;
}
/**
* A unique ID assigned to an offer.
*/
message OfferID {
required string value = 1;
}
/**
* A unique ID assigned to a slave. Currently, a slave gets a new ID
* whenever it (re)registers with Mesos. Framework writers shouldn't
* assume any binding between a slave ID and and a hostname.
*/
message SlaveID {
required string value = 1;
}
/**
* A framework generated ID to distinguish a task. The ID must remain
* unique while the task is active. However, a framework can reuse an
* ID _only_ if a previous task with the same ID has reached a
* terminal state (e.g., TASK_FINISHED, TASK_LOST, TASK_KILLED, etc.).
*/
message TaskID {
required string value = 1;
}
/**
* A framework generated ID to distinguish an executor. Only one
* executor with the same ID can be active on the same slave at a
* time.
*/
message ExecutorID {
required string value = 1;
}
/**
* A slave generated ID to distinguish a container. The ID must be unique
* between any active or completed containers on the slave. In particular,
* containers for different runs of the same (framework, executor) pair must be
* unique.
*/
message ContainerID {
required string value = 1;
}
/**
* Describes a framework.
*/
message FrameworkInfo {
// Used to determine the Unix user that an executor or task should
// be launched as. If the user field is set to an empty string Mesos
// will automagically set it to the current user.
required string user = 1;
// Name of the framework that shows up in the Mesos Web UI.
required string name = 2;
// Note that 'id' is only available after a framework has
// registered, however, it is included here in order to facilitate
// scheduler failover (i.e., if it is set then the
// MesosSchedulerDriver expects the scheduler is performing
// failover).
optional FrameworkID id = 3;
// The amount of time that the master will wait for the scheduler to
// failover before it tears down the framework by killing all its
// tasks/executors. This should be non-zero if a framework expects
// to reconnect after a failover and not lose its tasks/executors.
optional double failover_timeout = 4 [default = 0.0];
// If set, framework pid, executor pids and status updates are
// checkpointed to disk by the slaves. Checkpointing allows a
// restarted slave to reconnect with old executors and recover
// status updates, at the cost of disk I/O.
optional bool checkpoint = 5 [default = false];
// Used to group frameworks for allocation decisions, depending on
// the allocation policy being used.
optional string role = 6 [default = "*"];
// Used to indicate the current host from which the scheduler is
// registered in the Mesos Web UI. If set to an empty string Mesos
// will automagically set it to the current hostname.
optional string hostname = 7;
// This field should match the credential's principal the framework
// uses for authentication. This field is used for framework API
// rate limiting and dynamic reservations. It should be set even
// if authentication is not enabled if these features are desired.
optional string principal = 8;
// This field allows a framework to advertise its web UI, so that
// the Mesos web UI can link to it. It is expected to be a full URL,
// for example http://my-scheduler.example.com:8080/.
optional string webui_url = 9;
message Capability {
enum Type {
// Receive offers with revocable resources. See 'Resource'
// message for details.
// TODO(vinod): This is currently a no-op.
REVOCABLE_RESOURCES = 1;
}
required Type type = 1;
}
// This field allows a framework to advertise its set of
// capabilities (e.g., ability to receive offers for revocable
// resources).
repeated Capability capabilities = 10;
}
/**
* Describes a health check for a task or executor (or any arbitrary
* process/command). A "strategy" is picked by specifying one of the
* optional fields; currently only 'command' is supported.
* Specifying more than one strategy is an error.
*/
message HealthCheck {
// Describes an HTTP health check. This is not fully implemented and not
// recommended for use - see MESOS-2533.
message HTTP {
// Port to send the HTTP request.
required uint32 port = 1;
// HTTP request path.
optional string path = 2 [default = "/"];
// TODO(benh): Implement:
// Whether or not to use HTTPS.
// optional bool ssl = 3 [default = false];
// Expected response statuses. Not specifying any statuses implies
// that any returned status is acceptable.
repeated uint32 statuses = 4;
// TODO(benh): Include an 'optional bytes data' field for checking
// for specific data in the response.
}
// HTTP health check - not yet recommended for use, see MESOS-2533.
optional HTTP http = 1;
// TODO(benh): Consider adding a URL health check strategy which
// allows doing something similar to the HTTP strategy but
// encapsulates all the details in a single string field.
// TODO(benh): Other possible health check strategies could include
// one for TCP/UDP.
// Amount of time to wait until starting the health checks.
optional double delay_seconds = 2 [default = 15.0];
// Interval between health checks.
optional double interval_seconds = 3 [default = 10.0];
// Amount of time to wait for the health check to complete.
optional double timeout_seconds = 4 [default = 20.0];
// Number of consecutive failures until considered unhealthy.
optional uint32 consecutive_failures = 5 [default = 3];
// Amount of time to allow failed health checks since launch.
optional double grace_period_seconds = 6 [default = 10.0];
// Command health check.
optional CommandInfo command = 7;
}
/**
* Describes a command, executed via: '/bin/sh -c value'. Any URIs specified
* are fetched before executing the command. If the executable field for an
* uri is set, executable file permission is set on the downloaded file.
* Otherwise, if the downloaded file has a recognized archive extension
* (currently [compressed] tar and zip) it is extracted into the executor's
* working directory. This extraction can be disabled by setting `extract` to
* false. In addition, any environment variables are set before executing
* the command (so they can be used to "parameterize" your command).
*/
message CommandInfo {
message URI {
required string value = 1;
optional bool executable = 2;
// In case the fetched file is recognized as an archive, extract
// its contents into the sandbox. Note that a cached archive is
// not copied from the cache to the sandbox in case extraction
// originates from an archive in the cache.
optional bool extract = 3 [default = true];
// If this field is "true", the fetcher cache will be used. If not,
// fetching bypasses the cache and downloads directly into the
// sandbox directory, no matter whether a suitable cache file is
// available or not. The former directs the fetcher to download to
// the file cache, then copy from there to the sandbox. Subsequent
// fetch attempts with the same URI will omit downloading and copy
// from the cache as long as the file is resident there. Cache files
// may get evicted at any time, which then leads to renewed
// downloading. See also "docs/fetcher.md" and
// "docs/fetcher-cache-internals.md".
optional bool cache = 4;
}
// Describes a container.
// Not all containerizers currently implement ContainerInfo, so it
// is possible that a launched task will fail due to supplying this
// attribute.
// NOTE: The containerizer API is currently in an early beta or
// even alpha state. Some details, like the exact semantics of an
// "image" or "options" are not yet hardened.
// TODO(tillt): Describe the exact scheme and semantics of "image"
// and "options".
message ContainerInfo {
// URI describing the container image name.
required string image = 1;
// Describes additional options passed to the containerizer.
repeated string options = 2;
}
// NOTE: MesosContainerizer does currently not support this
// attribute and tasks supplying a 'container' will fail.
optional ContainerInfo container = 4;
repeated URI uris = 1;
optional Environment environment = 2;
// There are two ways to specify the command:
// 1) If 'shell == true', the command will be launched via shell
// (i.e., /bin/sh -c 'value'). The 'value' specified will be
// treated as the shell command. The 'arguments' will be ignored.
// 2) If 'shell == false', the command will be launched by passing
// arguments to an executable. The 'value' specified will be
// treated as the filename of the executable. The 'arguments'
// will be treated as the arguments to the executable. This is
// similar to how POSIX exec families launch processes (i.e.,
// execlp(value, arguments(0), arguments(1), ...)).
// NOTE: The field 'value' is changed from 'required' to 'optional'
// in 0.20.0. It will only cause issues if a new framework is
// connecting to an old master.
optional bool shell = 6 [default = true];
optional string value = 3;
repeated string arguments = 7;
// Enables executor and tasks to run as a specific user. If the user
// field is present both in FrameworkInfo and here, the CommandInfo
// user value takes precedence.
optional string user = 5;
}
/**
* Describes information about an executor. The 'data' field can be
* used to pass arbitrary bytes to an executor.
*/
message ExecutorInfo {
required ExecutorID executor_id = 1;
optional FrameworkID framework_id = 8; // TODO(benh): Make this required.
required CommandInfo command = 7;
// Executor provided with a container will launch the container
// with the executor's CommandInfo and we expect the container to
// act as a Mesos executor.
optional ContainerInfo container = 11;
repeated Resource resources = 5;
optional string name = 9;
// Source is an identifier style string used by frameworks to track
// the source of an executor. This is useful when it's possible for
// different executor ids to be related semantically.
// NOTE: Source is exposed alongside the resource usage of the
// executor via JSON on the slave. This allows users to import
// usage information into a time series database for monitoring.
optional string source = 10;
optional bytes data = 4;
// Service discovery information for the executor. It is not
// interpreted or acted upon by Mesos. It is up to a service
// discovery system to use this information as needed and to handle
// executors without service discovery information.
optional DiscoveryInfo discovery = 12;
}
/**
* Describes a master. This will probably have more fields in the
* future which might be used, for example, to link a framework webui
* to a master webui.
*/
message MasterInfo {
required string id = 1;
required uint32 ip = 2;
required uint32 port = 3 [default = 5050];
optional string pid = 4;
optional string hostname = 5;
optional string version = 6;
}
/**
* Describes a slave. Note that the 'id' field is only available after
* a slave is registered with the master, and is made available here
* to facilitate re-registration. If checkpoint is set, the slave is
* checkpointing its own information and potentially frameworks'
* information (if a framework has checkpointing enabled).
*/
message SlaveInfo {
required string hostname = 1;
optional int32 port = 8 [default = 5051];
repeated Resource resources = 3;
repeated Attribute attributes = 5;
optional SlaveID id = 6;
// TODO(joerg84): Remove checkpoint field as with 0.22.0
// slave checkpointing is enabled for all slaves (MESOS-2317).
optional bool checkpoint = 7 [default = false];
}
/**
* Describes an Attribute or Resource "value". A value is described
* using the standard protocol buffer "union" trick.
*/
message Value {
enum Type {
SCALAR = 0;
RANGES = 1;
SET = 2;
TEXT = 3;
}
message Scalar {
required double value = 1;
}
message Range {
required uint64 begin = 1;
required uint64 end = 2;
}
message Ranges {
repeated Range range = 1;
}
message Set {
repeated string item = 1;
}
message Text {
required string value = 1;
}
required Type type = 1;
optional Scalar scalar = 2;
optional Ranges ranges = 3;
optional Set set = 4;
optional Text text = 5;
}
/**
* Describes an attribute that can be set on a machine. For now,
* attributes and resources share the same "value" type, but this may
* change in the future and attributes may only be string based.
*/
message Attribute {
required string name = 1;
required Value.Type type = 2;
optional Value.Scalar scalar = 3;
optional Value.Ranges ranges = 4;
optional Value.Set set = 6;
optional Value.Text text = 5;
}
/**
* Describes a resource on a machine. A resource can take on one of
* three types: scalar (double), a list of finite and discrete ranges
* (e.g., [1-10, 20-30]), or a set of items. A resource is described
* using the standard protocol buffer "union" trick.
*
* TODO(benh): Add better support for "expected" resources (e.g.,
* cpus, memory, disk, network).
*/
message Resource {
required string name = 1;
required Value.Type type = 2;
optional Value.Scalar scalar = 3;
optional Value.Ranges ranges = 4;
optional Value.Set set = 5;
optional string role = 6 [default = "*"];
message ReservationInfo {
// Describes a dynamic reservation. A dynamic reservation is
// acquired by an operator via the '/reserve' HTTP endpoint or by
// a framework via the offer cycle by sending back an
// 'Offer::Operation::Reserve' message.
// NOTE: We currently do not allow frameworks with role "*" to
// make dynamic reservations.
// This field indicates the principal of the operator or framework
// that reserved this resource. It is used in conjunction with the
// "unreserve" ACL to determine whether the entity attempting to
// unreserve this resource is permitted to do so.
// NOTE: This field should match the FrameworkInfo.principal of
// the framework that reserved this resource.
required string principal = 1;
}
// If this is set, this resource was dynamically reserved by an
// operator or a framework. Otherwise, this resource is either unreserved
// or statically reserved by an operator via the --resources flag.
optional ReservationInfo reservation = 8;
message DiskInfo {
// Describes a persistent disk volume.
// A persistent disk volume will not be automatically garbage
// collected if the task/executor/slave terminates, but is
// re-offered to the framework(s) belonging to the 'role'.
// A framework can set the ID (if it is not set yet) to express
// the intention to create a new persistent disk volume from a
// regular disk resource. To reuse a previously created volume, a
// framework can launch a task/executor when it receives an offer
// with a persistent volume, i.e., ID is set.
// NOTE: Currently, we do not allow a persistent disk volume
// without a reservation (i.e., 'role' should not be '*').
message Persistence {
// A unique ID for the persistent disk volume.
// NOTE: The ID needs to be unique per role on each slave.
required string id = 1;
}
optional Persistence persistence = 1;
// Describes how this disk resource will be mounted in the
// container. If not set, the disk resource will be used as the
// sandbox. Otherwise, it will be mounted according to the
// 'container_path' inside 'volume'. The 'host_path' inside
// 'volume' is ignored.
// NOTE: If 'volume' is set but 'persistence' is not set, the
// volume will be automatically garbage collected after
// task/executor terminates. Currently, if 'persistence' is set,
// 'volume' must be set.
optional Volume volume = 2;
}
optional DiskInfo disk = 7;
message RevocableInfo {}
// If this is set, the resources are revocable, i.e., any tasks or
// executors launched using these resources could get preempted or
// throttled at any time. This could be used by frameworks to run
// best effort tasks that do not need strict uptime or performance
// guarantees. Note that if this is set, 'disk' or 'reservation'
// cannot be set.
optional RevocableInfo revocable = 9;
}
/**
* When the network bandwidth caps are enabled and the container
* is over its limit, outbound packets may be either delayed or
* dropped completely either because it exceeds the maximum bandwidth
* allocation for a single container (the cap) or because the combined
* network traffic of multiple containers on the host exceeds the
* transmit capacity of the host (the share). We can report the
* following statistics for each of these conditions exported directly
* from the Linux Traffic Control Queueing Discipline.
*
* id : name of the limiter, e.g. 'tx_bw_cap'
* backlog : number of packets currently delayed
* bytes : total bytes seen
* drops : number of packets dropped in total
* overlimits : number of packets which exceeded allocation
* packets : total packets seen
* qlen : number of packets currently queued
* rate_bps : throughput in bytes/sec
* rate_pps : throughput in packets/sec
* requeues : number of times a packet has been delayed due to
* locking or device contention issues
*
* More information on the operation of Linux Traffic Control can be
* found at http://www.lartc.org/lartc.html.
*/
message TrafficControlStatistics {
required string id = 1;
optional uint64 backlog = 2;
optional uint64 bytes = 3;
optional uint64 drops = 4;
optional uint64 overlimits = 5;
optional uint64 packets = 6;
optional uint64 qlen = 7;
optional uint64 ratebps = 8;
optional uint64 ratepps = 9;
optional uint64 requeues = 10;
}
/**
* A snapshot of resource usage statistics.
*/
message ResourceStatistics {
required double timestamp = 1; // Snapshot time, in seconds since the Epoch.
optional uint32 processes = 30;
optional uint32 threads = 31;
// CPU Usage Information:
// Total CPU time spent in user mode, and kernel mode.
optional double cpus_user_time_secs = 2;
optional double cpus_system_time_secs = 3;
// Number of CPUs allocated.
optional double cpus_limit = 4;
// cpu.stat on process throttling (for contention issues).
optional uint32 cpus_nr_periods = 7;
optional uint32 cpus_nr_throttled = 8;
optional double cpus_throttled_time_secs = 9;
// Memory Usage Information:
// mem_total_bytes was added in 0.23.0 to represent the total memory
// of a process in RAM (as opposed to in Swap). This was previously
// reported as mem_rss_bytes, which was also changed in 0.23.0 to
// represent only the anonymous memory usage, to keep in sync with
// Linux kernel's (arguably erroneous) use of terminology.
optional uint64 mem_total_bytes = 36;
// Total memory + swap usage. This is set if swap is enabled.
optional uint64 mem_total_memsw_bytes = 37;
// Hard memory limit for a container.
optional uint64 mem_limit_bytes = 6;
// Soft memory limit for a container.
optional uint64 mem_soft_limit_bytes = 38;
// Broken out memory usage information: pagecache, rss (anonymous),
// mmaped files and swap.
// TODO(chzhcn) mem_file_bytes and mem_anon_bytes are deprecated in
// 0.23.0 and will be removed in 0.24.0.
optional uint64 mem_file_bytes = 10;
optional uint64 mem_anon_bytes = 11;
// mem_cache_bytes is added in 0.23.0 to represent page cache usage.
optional uint64 mem_cache_bytes = 39;
// Since 0.23.0, mem_rss_bytes is changed to represent only
// anonymous memory usage. Note that neither its requiredness, type,
// name nor numeric tag has been changed.
optional uint64 mem_rss_bytes = 5;
optional uint64 mem_mapped_file_bytes = 12;
// This is only set if swap is enabled.
optional uint64 mem_swap_bytes = 40;
// Number of occurrences of different levels of memory pressure
// events reported by memory cgroup. Pressure listening (re)starts
// with these values set to 0 when slave (re)starts. See
// https://www.kernel.org/doc/Documentation/cgroups/memory.txt for
// more details.
optional uint64 mem_low_pressure_counter = 32;
optional uint64 mem_medium_pressure_counter = 33;
optional uint64 mem_critical_pressure_counter = 34;
// Disk Usage Information for executor working directory.
optional uint64 disk_limit_bytes = 26;
optional uint64 disk_used_bytes = 27;
// Perf statistics.
optional PerfStatistics perf = 13;
// Network Usage Information:
optional uint64 net_rx_packets = 14;
optional uint64 net_rx_bytes = 15;
optional uint64 net_rx_errors = 16;
optional uint64 net_rx_dropped = 17;
optional uint64 net_tx_packets = 18;
optional uint64 net_tx_bytes = 19;
optional uint64 net_tx_errors = 20;
optional uint64 net_tx_dropped = 21;
// The kernel keeps track of RTT (round-trip time) for its TCP
// sockets. RTT is a way to tell the latency of a container.
optional double net_tcp_rtt_microsecs_p50 = 22;
optional double net_tcp_rtt_microsecs_p90 = 23;
optional double net_tcp_rtt_microsecs_p95 = 24;
optional double net_tcp_rtt_microsecs_p99 = 25;
optional double net_tcp_active_connections = 28;
optional double net_tcp_time_wait_connections = 29;
// Network traffic flowing into or out of a container can be delayed
// or dropped due to congestion or policy inside and outside the
// container.
repeated TrafficControlStatistics net_traffic_control_statistics = 35;
}
/**
* Describes a snapshot of the resource usage for executors.
*/
message ResourceUsage {
message Executor {
required ExecutorInfo executor_info = 1;
// This includes resources used by the executor itself
// as well as its active tasks.
repeated Resource allocated = 2;
// Current resource usage. If absent, the containerizer
// cannot provide resource usage.
optional ResourceStatistics statistics = 3;
}
repeated Executor executors = 1;
// TODO(jieyu): Include slave's total resources here.
}
/**
* Describes a sample of events from "perf stat". Only available on
* Linux.
*
* NOTE: Each optional field matches the name of a perf event (see
* "perf list") with the following changes:
* 1. Names are downcased.
* 2. Hyphens ('-') are replaced with underscores ('_').
* 3. Events with alternate names use the name "perf stat" returns,
* e.g., for the event "cycles OR cpu-cycles" perf always returns
* cycles.
*/
message PerfStatistics {
required double timestamp = 1; // Start of sample interval, in seconds since the Epoch.
required double duration = 2; // Duration of sample interval, in seconds.
// Hardware event.
optional uint64 cycles = 3;
optional uint64 stalled_cycles_frontend = 4;
optional uint64 stalled_cycles_backend = 5;
optional uint64 instructions = 6;
optional uint64 cache_references = 7;
optional uint64 cache_misses = 8;
optional uint64 branches = 9;
optional uint64 branch_misses = 10;
optional uint64 bus_cycles = 11;
optional uint64 ref_cycles = 12;
// Software event.
optional double cpu_clock = 13;
optional double task_clock = 14;
optional uint64 page_faults = 15;
optional uint64 minor_faults = 16;
optional uint64 major_faults = 17;
optional uint64 context_switches = 18;
optional uint64 cpu_migrations = 19;
optional uint64 alignment_faults = 20;
optional uint64 emulation_faults = 21;
// Hardware cache event.
optional uint64 l1_dcache_loads = 22;
optional uint64 l1_dcache_load_misses = 23;
optional uint64 l1_dcache_stores = 24;
optional uint64 l1_dcache_store_misses = 25;
optional uint64 l1_dcache_prefetches = 26;
optional uint64 l1_dcache_prefetch_misses = 27;
optional uint64 l1_icache_loads = 28;
optional uint64 l1_icache_load_misses = 29;
optional uint64 l1_icache_prefetches = 30;
optional uint64 l1_icache_prefetch_misses = 31;
optional uint64 llc_loads = 32;
optional uint64 llc_load_misses = 33;
optional uint64 llc_stores = 34;
optional uint64 llc_store_misses = 35;
optional uint64 llc_prefetches = 36;
optional uint64 llc_prefetch_misses = 37;
optional uint64 dtlb_loads = 38;
optional uint64 dtlb_load_misses = 39;
optional uint64 dtlb_stores = 40;
optional uint64 dtlb_store_misses = 41;
optional uint64 dtlb_prefetches = 42;
optional uint64 dtlb_prefetch_misses = 43;
optional uint64 itlb_loads = 44;
optional uint64 itlb_load_misses = 45;
optional uint64 branch_loads = 46;
optional uint64 branch_load_misses = 47;
optional uint64 node_loads = 48;
optional uint64 node_load_misses = 49;
optional uint64 node_stores = 50;
optional uint64 node_store_misses = 51;
optional uint64 node_prefetches = 52;
optional uint64 node_prefetch_misses = 53;
}
/**
* Describes a request for resources that can be used by a framework
* to proactively influence the allocator. If 'slave_id' is provided
* then this request is assumed to only apply to resources on that
* slave.
*
* TODO(vinod): Remove this once the old driver is removed.
*/
message Request {
optional SlaveID slave_id = 1;
repeated Resource resources = 2;
}
/**
* Describes some resources available on a slave. An offer only
* contains resources from a single slave.
*/
message Offer {
required OfferID id = 1;
required FrameworkID framework_id = 2;
required SlaveID slave_id = 3;
required string hostname = 4;
repeated Resource resources = 5;
repeated Attribute attributes = 7;
repeated ExecutorID executor_ids = 6;
// Defines an operation that can be performed against offers.
message Operation {
enum Type {
LAUNCH = 1;
RESERVE = 2;
UNRESERVE = 3;
CREATE = 4;
DESTROY = 5;
}
message Launch {
repeated TaskInfo task_infos = 1;
}
message Reserve {
repeated Resource resources = 1;
}
message Unreserve {
repeated Resource resources = 1;
}
message Create {
repeated Resource volumes = 1;
}
message Destroy {
repeated Resource volumes = 1;
}
required Type type = 1;
optional Launch launch = 2;
optional Reserve reserve = 3;
optional Unreserve unreserve = 4;
optional Create create = 5;
optional Destroy destroy = 6;
}
}
/**
* Describes a task. Passed from the scheduler all the way to an
* executor (see SchedulerDriver::launchTasks and
* Executor::launchTask). Either ExecutorInfo or CommandInfo should be set.
* A different executor can be used to launch this task, and subsequent tasks
* meant for the same executor can reuse the same ExecutorInfo struct.
*/
message TaskInfo {
required string name = 1;
required TaskID task_id = 2;
required SlaveID slave_id = 3;
repeated Resource resources = 4;
optional ExecutorInfo executor = 5;
optional CommandInfo command = 7;
// Task provided with a container will launch the container as part
// of this task paired with the task's CommandInfo.
optional ContainerInfo container = 9;
optional bytes data = 6;
// A health check for the task (currently in *alpha* and initial
// support will only be for TaskInfo's that have a CommandInfo).
optional HealthCheck health_check = 8;
// Labels are free-form key value pairs which are exposed through
// master and slave endpoints. Labels will not be interpreted or
// acted upon by Mesos itself. As opposed to the data field, labels
// will be kept in memory on master and slave processes. Therefore,
// labels should be used to tag tasks with light-weight meta-data.
optional Labels labels = 10;
// Service discovery information for the task. It is not interpreted
// or acted upon by Mesos. It is up to a service discovery system
// to use this information as needed and to handle tasks without
// service discovery information.
optional DiscoveryInfo discovery = 11;
}
/**
* Describes possible task states. IMPORTANT: Mesos assumes tasks that
* enter terminal states (see below) imply the task is no longer
* running and thus clean up any thing associated with the task
* (ultimately offering any resources being consumed by that task to
* another task).
*/
enum TaskState {
TASK_STAGING = 6; // Initial state. Framework status updates should not use.
TASK_STARTING = 0;
TASK_RUNNING = 1;
TASK_FINISHED = 2; // TERMINAL. The task finished successfully.
TASK_FAILED = 3; // TERMINAL. The task failed to finish successfully.
TASK_KILLED = 4; // TERMINAL. The task was killed by the executor.
TASK_LOST = 5; // TERMINAL. The task failed but can be rescheduled.
TASK_ERROR = 7; // TERMINAL. The task description contains an error.
}
/**
* Describes the current status of a task.
*/
message TaskStatus {
// Describes the source of the task status update.
enum Source {
SOURCE_MASTER = 0;
SOURCE_SLAVE = 1;
SOURCE_EXECUTOR = 2;
}
// Detailed reason for the task status update.
//
// TODO(bmahler): Differentiate between slave removal reasons
// (e.g. unhealthy vs. unregistered for maintenance).
enum Reason {
REASON_COMMAND_EXECUTOR_FAILED = 0;
REASON_EXECUTOR_PREEMPTED = 17;
REASON_EXECUTOR_TERMINATED = 1;
REASON_EXECUTOR_UNREGISTERED = 2;
REASON_FRAMEWORK_REMOVED = 3;
REASON_GC_ERROR = 4;
REASON_INVALID_FRAMEWORKID = 5;
REASON_INVALID_OFFERS = 6;
REASON_MASTER_DISCONNECTED = 7;
REASON_MEMORY_LIMIT = 8;
REASON_RECONCILIATION = 9;
REASON_RESOURCES_UNKNOWN = 18;
REASON_SLAVE_DISCONNECTED = 10;
REASON_SLAVE_REMOVED = 11;
REASON_SLAVE_RESTARTED = 12;
REASON_SLAVE_UNKNOWN = 13;
REASON_TASK_INVALID = 14;
REASON_TASK_UNAUTHORIZED = 15;
REASON_TASK_UNKNOWN = 16;
}
required TaskID task_id = 1;
required TaskState state = 2;
optional string message = 4; // Possible message explaining state.
optional Source source = 9;
optional Reason reason = 10;
optional bytes data = 3;
optional SlaveID slave_id = 5;
optional ExecutorID executor_id = 7; // TODO(benh): Use in master/slave.
optional double timestamp = 6;
// Statuses that are delivered reliably to the scheduler will
// include a 'uuid'. The status is considered delivered once
// it is acknowledged by the scheduler. Schedulers can choose
// to either explicitly acknowledge statuses or let the scheduler
// driver implicitly acknowledge (default).
//
// TODO(bmahler): This is currently overwritten in the scheduler
// driver and executor driver, but executors will need to set this
// to a valid RFC-4122 UUID if using the HTTP API.
optional bytes uuid = 11;
// Describes whether the task has been determined to be healthy
// (true) or unhealthy (false) according to the HealthCheck field in
// the command info.
optional bool healthy = 8;
}
/**
* Describes possible filters that can be applied to unused resources
* (see SchedulerDriver::launchTasks) to influence the allocator.
*/
message Filters {
// Time to consider unused resources refused. Note that all unused
// resources will be considered refused and use the default value
// (below) regardless of whether Filters was passed to
// SchedulerDriver::launchTasks. You MUST pass Filters with this
// field set to change this behavior (i.e., get another offer which
// includes unused resources sooner or later than the default).
optional double refuse_seconds = 1 [default = 5.0];
}
/**
* Describes a collection of environment variables. This is used with
* CommandInfo in order to set environment variables before running a
* command.
*/
message Environment {
message Variable {
required string name = 1;
required string value = 2;
}
repeated Variable variables = 1;
}
/**
* A generic (key, value) pair used in various places for parameters.
*/
message Parameter {
required string key = 1;
required string value = 2;
}
/**
* Collection of Parameter.
*/
message Parameters {
repeated Parameter parameter = 1;
}
/**
* Credential used in various places for authentication and
* authorization.
*
* NOTE: A 'principal' is different from 'FrameworkInfo.user'. The
* former is used for authentication and authorization while the
* latter is used to determine the default user under which the
* framework's executors/tasks are run.
*/
message Credential {
required string principal = 1;
optional bytes secret = 2;
}
/**
* Credentials used for framework authentication, HTTP authentication
* (where the common 'username' and 'password' are captured as
* 'principal' and 'secret' respectively), etc.
*/
message Credentials {
repeated Credential credentials = 1;
}
/**
* ACLs used for authorization.
*/
message ACL {
// Entity is used to describe a subject(s) or an object(s) of an ACL.
// NOTE:
// To allow everyone access to an Entity set its type to 'ANY'.
// To deny access to an Entity set its type to 'NONE'.
message Entity {
enum Type {
SOME = 0;
ANY = 1;
NONE = 2;
}
optional Type type = 1 [default = SOME];
repeated string values = 2; // Ignored for ANY/NONE.
}
// ACLs.
message RegisterFramework {
// Subjects.
required Entity principals = 1; // Framework principals.
// Objects.
required Entity roles = 2; // Roles for resource offers.
}
message RunTask {
// Subjects.
required Entity principals = 1; // Framework principals.
// Objects.
required Entity users = 2; // Users to run the tasks/executors as.
}
// Which principals are authorized to shutdown frameworks of other
// principals.
message ShutdownFramework {
// Subjects.
required Entity principals = 1;
// Objects.
required Entity framework_principals = 2;
}
}
/**
* Collection of ACL.
*
* Each authorization request is evaluated against the ACLs in the order
* they are defined.
*
* For simplicity, the ACLs for a given action are not aggregated even
* when they have the same subjects or objects. The first ACL that
* matches the request determines whether that request should be
* permitted or not. An ACL matches iff both the subjects
* (e.g., clients, principals) and the objects (e.g., urls, users,
* roles) of the ACL match the request.
*
* If none of the ACLs match the request, the 'permissive' field
* determines whether the request should be permitted or not.
*
* TODO(vinod): Do aggregation of ACLs when possible.
*
*/
message ACLs {
optional bool permissive = 1 [default = true];
repeated ACL.RegisterFramework register_frameworks = 2;
repeated ACL.RunTask run_tasks = 3;
repeated ACL.ShutdownFramework shutdown_frameworks = 4;
}
/**
* Rate (queries per second, QPS) limit for messages from a framework to master.
* Strictly speaking they are the combined rate from all frameworks of the same
* principal.
*/
message RateLimit {
// Leaving QPS unset gives it unlimited rate (i.e., not throttled),
// which also implies unlimited capacity.
optional double qps = 1;
// Principal of framework(s) to be throttled. Should match
// FrameworkInfo.princpal and Credential.principal (if using authentication).
required string principal = 2;
// Max number of outstanding messages from frameworks of this principal
// allowed by master before the next message is dropped and an error is sent
// back to the sender. Messages received before the capacity is reached are
// still going to be processed after the error is sent.
// If unspecified, this principal is assigned unlimited capacity.
// NOTE: This value is ignored if 'qps' is not set.
optional uint64 capacity = 3;
}
/**
* Collection of RateLimit.
* Frameworks without rate limits defined here are not throttled unless
* 'aggregate_default_qps' is specified.
*/
message RateLimits {
// Items should have unique principals.
repeated RateLimit limits = 1;
// All the frameworks not specified in 'limits' get this default rate.
// This rate is an aggregate rate for all of them, i.e., their combined
// traffic is throttled together at this rate.
optional double aggregate_default_qps = 2;
// All the frameworks not specified in 'limits' get this default capacity.
// This is an aggregate value similar to 'aggregate_default_qps'.
optional uint64 aggregate_default_capacity = 3;
}
/**
* Describes a volume mapping either from host to container or vice
* versa. Both paths can either refer to a directory or a file.
*/
message Volume {
// Absolute path pointing to a directory or file in the container.
required string container_path = 1;
// Absolute path pointing to a directory or file on the host or a path
// relative to the container work directory.
optional string host_path = 2;
enum Mode {
RW = 1; // read-write.
RO = 2; // read-only.
}
required Mode mode = 3;
}
/**
* Describes a container configuration and allows extensible
* configurations for different container implementations.
*/
message ContainerInfo {
// All container implementation types.
enum Type {
DOCKER = 1;
MESOS = 2;
}
message DockerInfo {
// The docker image that is going to be passed to the registry.
required string image = 1;
// Network options.
enum Network {
HOST = 1;
BRIDGE = 2;
NONE = 3;
}
optional Network network = 2 [default = HOST];
message PortMapping {
required uint32 host_port = 1;
required uint32 container_port = 2;
// Protocol to expose as (ie: tcp, udp).
optional string protocol = 3;
}
repeated PortMapping port_mappings = 3;
optional bool privileged = 4 [default = false];
// Allowing arbitrary parameters to be passed to docker CLI.
// Note that anything passed to this field is not guaranteed
// to be supported moving forward, as we might move away from
// the docker CLI.
repeated Parameter parameters = 5;
// With this flag set to true, the docker containerizer will
// pull the docker image from the registry even if the image
// is already downloaded on the slave.
optional bool force_pull_image = 6;
}
required Type type = 1;
repeated Volume volumes = 2;
optional string hostname = 4;
optional DockerInfo docker = 3;
}
/**
* Collection of labels.
*/
message Labels {
repeated Label labels = 1;
}
/**
* Key, value pair used to store free form user-data.
*/
message Label {
required string key = 1;
optional string value = 2;
}
/**
* Named port used for service discovery.
*/
message Port {
required uint32 number = 1;
optional string name = 2;
optional string protocol = 3;
}
/**
* Collection of ports.
*/
message Ports {
repeated Port ports = 1;
}
/**
* Service discovery information.
* The visibility field restricts discovery within a framework
* (FRAMEWORK), within a Mesos cluster (CLUSTER), or places no
* restrictions (EXTERNAL).
* The environment, location, and version fields provide first class
* support for common attributes used to differentiate between
* similar services. The environment may receive values such as
* PROD/QA/DEV, the location field may receive values like
* EAST-US/WEST-US/EUROPE/AMEA, and the version field may receive
* values like v2.0/v0.9. The exact use of these fields is up to each
* service discovery system.
*/
message DiscoveryInfo {
enum Visibility {
FRAMEWORK = 0;
CLUSTER = 1;
EXTERNAL = 2;
}
required Visibility visibility = 1;
optional string name = 2;
optional string environment = 3;
optional string location = 4;
optional string version = 5;
optional Ports ports = 6;
optional Labels labels = 7;
}