You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by dl...@apache.org on 2014/03/25 07:10:07 UTC

svn commit: r1581246 [5/6] - in /incubator/aurora/site: ./ publish/ publish/community/ publish/developers/ publish/docs/gettingstarted/ publish/docs/howtocontribute/ publish/documentation/ publish/documentation/latest/ publish/documentation/latest/clie...

Added: incubator/aurora/site/source/documentation/latest/configurationtutorial.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/configurationtutorial.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/configurationtutorial.md (added)
+++ incubator/aurora/site/source/documentation/latest/configurationtutorial.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,1139 @@
+Aurora Configuration Tutorial
+=============================
+
+How to write Aurora configuration files, including feature descriptions
+and best practices. When writing a configuration file, make use of
+`aurora inspect`. It takes the same job key and configuration file
+arguments as `aurora create` or `aurora update`. It first ensures the
+configuration parses, then outputs it in human-readable form.
+
+You should read this after going through the general [Aurora Tutorial](/documentation/latest/tutorial/).
+
+[The Basics](#Basics)
+    [Use Bottom-To-Top Object Ordering](#Bottom)
+[An Example Configuration File](#Example)
+[Defining Process Objects](#Process)
+[Getting Your Code Into The Sandbox](#Sandbox)
+[Defining Task Objects](#Task)
+    [`SequentialTask`](#Sequential)
+    [`SimpleTask`](#Simple)
+    [`Tasks.concat` and `Tasks.combine`](#Concat)
+[Defining `Job` Objects](#Job)
+[Defining The `jobs` List](#jobs)
+[Templating](#Templating)
+[Templating 1: Binding in Pystachio](#Binding)
+[Structurals in Pystachio / Aurora](#Structurals)
+    [Mustaches Within Structurals](#Mustaches)
+[Templating 2: Structurals Are Factories](#Factories)
+    [A Second Way of Templating](#Second)
+[Advanced Binding](#AdvancedBinding)
+[Bind Syntax](#BindSyntax)
+    [Binding Complex Objects](#ComplexObjects)
+[Structural Binding](#StructuralBinding)
+[Configuration File Writing Tips And Best Practices](#Tips)
+    [Use As Few `.aurora` Files As Possible](#Few)
+    [Avoid Boilerplate](#Boilerplate)
+    [Thermos Uses bash, But Thermos Is Not bash](#Bash)
+    [Rarely Use Functions In Your Configurations](#Functions)
+
+The Basics
+----------
+
+To run a job on Aurora, you must specify a configuration file that tells
+Aurora what it needs to know to schedule the job, what Mesos needs to
+run the tasks the job is made up of, and what Thermos needs to run the
+processes that make up the tasks. This file must have
+a`.aurora` suffix.
+
+A configuration file defines a collection of objects, along with parameter
+values for their attributes. An Aurora configuration file contains the
+following three types of objects:
+
+- Job
+- Task
+- Process
+
+A configuration also specifies a list of `Job` objects assigned
+to the variable `jobs`.
+
+- jobs (list of defined Jobs to run)
+
+The `.aurora` file format is just Python. However, `Job`, `Task`,
+`Process`, and other classes are defined by a type-checked dictionary
+templating library called *Pystachio*, a powerful tool for
+configuration specification and reuse. Pystachio objects are tailored
+via {{}} surrounded templates.
+
+When writing your `.aurora` file, you may use any Pystachio datatypes, as
+well as any objects shown in the [*Aurora+Thermos Configuration
+Reference*](/documentation/latest/configurationreference/), without `import` statements - the
+Aurora config loader injects them automatically. Other than that, an `.aurora`
+file works like any other Python script.
+
+[*Aurora+Thermos Configuration Reference*](/documentation/latest/configurationreference/)
+has a full reference of all Aurora/Thermos defined Pystachio objects.
+
+### Use Bottom-To-Top Object Ordering
+
+A well-structured configuration starts with structural templates (if
+any). Structural templates encapsulate in their attributes all the
+differences between Jobs in the configuration that are not directly
+manipulated at the `Job` level, but typically at the `Process` or `Task`
+level. For example, if certain processes are invoked with slightly
+different settings or input.
+
+After structural templates, define, in order, `Process`es, `Task`s, and
+`Job`s.
+
+Structural template names should be *UpperCamelCased* and their
+instantiations are typically *UPPER\_SNAKE\_CASED*. `Process`, `Task`,
+and `Job` names are typically *lower\_snake\_cased*. Indentation is typically 2
+spaces.
+
+An Example Configuration File
+-----------------------------
+
+The following is a typical configuration file. Don't worry if there are
+parts you don't understand yet, but you may want to refer back to this
+as you read about its individual parts. Note that names surrounded by
+curly braces {{}} are template variables, which the system replaces with
+bound values for the variables.
+
+    # --- templates here ---
+	class Profile(Struct):
+	  package_version = Default(String, 'live')
+	  java_binary = Default(String, '/usr/lib/jvm/java-1.7.0-openjdk/bin/java')
+	  extra_jvm_options = Default(String, '')
+	  parent_environment = Default(String, 'prod')
+	  parent_serverset = Default(String,
+                                 '/foocorp/service/bird/{{parent_environment}}/bird')
+
+	# --- processes here ---
+	main = Process(
+	  name = 'application',
+	  cmdline = '{{profile.java_binary}} -server -Xmx1792m '
+	            '{{profile.extra_jvm_options}} '
+	            '-jar application.jar '
+	            '-upstreamService {{profile.parent_serverset}}'
+	)
+
+	# --- tasks ---
+	base_task = SequentialTask(
+	  name = 'application',
+	  processes = [
+	    Process(
+	      name = 'fetch',
+	      cmdline = 'curl -O
+                  https://packages.foocorp.com/{{profile.package_version}}/application.jar'),
+	  ]
+	)
+
+        # not always necessary but often useful to have separate task
+        # resource classes
+        staging_task = base_task(resources =
+                         Resources(cpu = 1.0,
+                                   ram = 2048*MB,
+                                   disk = 1*GB))
+	production_task = base_task(resources =
+                            Resources(cpu = 4.0,
+                                      ram = 2560*MB,
+                                      disk = 10*GB))
+
+	# --- job template ---
+	job_template = Job(
+	  name = 'application',
+	  role = 'myteam',
+	  contact = 'myteam-team@foocorp.com',
+	  instances = 20,
+	  service = True,
+	  task = production_task
+	)
+
+	# -- profile instantiations (if any) ---
+	PRODUCTION = Profile()
+	STAGING = Profile(
+	  extra_jvm_options = '-Xloggc:gc.log',
+	  parent_environment = 'staging'
+	)
+
+	# -- job instantiations --
+	jobs = [
+          job_template(cluster = 'cluster1', environment = 'prod')
+	               .bind(profile = PRODUCTION),
+
+          job_template(cluster = 'cluster2', environment = 'prod')
+	                .bind(profile = PRODUCTION),
+
+          job_template(cluster = 'cluster1',
+                        environment = 'staging',
+			service = False,
+			task = staging_task,
+			instances = 2)
+			.bind(profile = STAGING),
+	]
+
+## Defining Process Objects
+
+Processes are handled by the Thermos system. A process is a single
+executable step run as a part of an Aurora task, which consists of a
+bash-executable statement.
+
+The key (and required) `Process` attributes are:
+
+-   `name`: Any string which is a valid Unix filename (no slashes,
+    NULLs, or leading periods). The `name` value must be unique relative
+    to other Processes in a `Task`.
+-   `cmdline`: A command line run in a bash subshell, so you can use
+    bash scripts. Nothing is supplied for command-line arguments,
+    so `$*` is unspecified.
+
+Many tiny processes make managing configurations more difficult. For
+example, the following is a bad way to define processes.
+
+    copy = Process(
+      name = 'copy',
+      cmdline = 'curl -O https://packages.foocorp.com/app.zip'
+    )
+    unpack = Process(
+      name = 'unpack',
+      cmdline = 'unzip app.zip'
+    )
+    remove = Process(
+      name = 'remove',
+      cmdline = 'rm -f app.zip'
+    )
+    run = Process(
+      name = 'app',
+      cmdline = 'java -jar app.jar'
+    )
+    run_task = Task(
+      processes = [copy, unpack, remove, run],
+      constraints = order(copy, unpack, remove, run)
+    )
+
+Since `cmdline` runs in a bash subshell, you can chain commands
+with `&&` or `||`.
+
+When defining a `Task` that is just a list of Processes run in a
+particular order, use `SequentialTask`, as described in the [*Defining*
+`Task` *Objects*](#Task) section. The following simplifies and combines the
+above multiple `Process` definitions into just two.
+
+    stage = Process(
+      name = 'stage',
+      cmdline = 'curl -O https://packages.foocorp.com/app.zip && '
+                'unzip app.zip && rm -f app.zip')
+
+    run = Process(name = 'app', cmdline = 'java -jar app.jar')
+
+    run_task = SequentialTask(processes = [stage, run])
+
+`Process` also has five optional attributes, each with a default value
+if one isn't specified in the configuration:
+
+-   `max_failures`: Defaulting to `1`, the maximum number of failures
+    (non-zero exit statuses) before this `Process` is marked permanently
+    failed and not retried. If a `Process` permanently fails, Thermos
+    checks the `Process` object's containing `Task` for the task's
+    failure limit (usually 1) to determine whether or not the `Task`
+    should be failed. Setting `max_failures`to `0` means that this
+    process will keep retrying until a successful (zero) exit status is
+    achieved. Retries happen at most once every `min_duration` seconds
+    to prevent effectively mounting a denial of service attack against
+    the coordinating scheduler.
+
+-   `daemon`: Defaulting to `False`, if `daemon` is set to `True`, a
+    successful (zero) exit status does not prevent future process runs.
+    Instead, the `Process` reinvokes after `min_duration` seconds.
+    However, the maximum failure limit (`max_failures`) still
+    applies. A combination of `daemon=True` and `max_failures=0` retries
+    a `Process` indefinitely regardless of exit status. This should
+    generally be avoided for very short-lived processes because of the
+    accumulation of checkpointed state for each process run. When
+    running in Aurora, `max_failures` is capped at
+    100.
+
+-   `ephemeral`: Defaulting to `False`, if `ephemeral` is `True`, the
+    `Process`' status is not used to determine if its bound `Task` has
+    completed. For example, consider a `Task` with a
+    non-ephemeral webserver process and an ephemeral logsaver process
+    that periodically checkpoints its log files to a centralized data
+    store. The `Task` is considered finished once the webserver process
+    finishes, regardless of the logsaver's current status.
+
+-   `min_duration`: Defaults to `15`. Processes may succeed or fail
+    multiple times during a single Task. Each result is called a
+    *process run* and this value is the minimum number of seconds the
+    scheduler waits before re-running the same process.
+
+-   `final`: Defaulting to `False`, this is a finalizing `Process` that
+    should run last. Processes can be grouped into two classes:
+    *ordinary* and *finalizing*. By default, Thermos Processes are
+    ordinary. They run as long as the `Task` is considered
+    healthy (i.e. hasn't reached a failure limit). But once all regular
+    Thermos Processes have either finished or the `Task` has reached a
+    certain failure threshold, Thermos moves into a *finalization* stage
+    and runs all finalizing Processes. These are typically necessary for
+    cleaning up after the `Task`, such as log checkpointers, or perhaps
+    e-mail notifications of a completed Task. Finalizing processes may
+    not depend upon ordinary processes or vice-versa, however finalizing
+    processes may depend upon other finalizing processes and will
+    otherwise run as a typical process schedule.
+
+## Getting Your Code Into The Sandbox
+
+When using Aurora, you need to get your executable code into its "sandbox", specifically
+the Task sandbox where the code executes for the Processes that make up that Task.
+
+Each Task has a sandbox created when the Task starts and garbage
+collected when it finishes. All of a Task's processes run in its
+sandbox, so processes can share state by using a shared current
+working directory.
+
+Typically, you save this code somewhere. You then need to define a Process
+in your `.aurora` configuration file that fetches the code from that somewhere
+to where the slave can see it. For a public cloud, that can be anywhere public on
+the Internet, such as S3. For a private cloud internal storage, you need to put in
+on an accessible HDFS cluster or similar storage.
+
+The template for this Process is:
+
+    <name> = Process(
+      name = '<name>'
+      cmdline = '<command to copy and extract code archive into current working directory>'
+    )
+
+Note: Be sure the extracted code archive has an executable.
+
+## Defining Task Objects
+
+Tasks are handled by Mesos. A task is a collection of processes that
+runs in a shared sandbox. It's the fundamental unit Aurora uses to
+schedule the datacenter; essentially what Aurora does is find places
+in the cluster to run tasks.
+
+The key (and required) parts of a Task are:
+
+-   `name`: A string giving the Task's name. By default, if a Task is
+    not given a name, it inherits the first name in its Process list.
+
+-   `processes`: An unordered list of Process objects bound to the Task.
+    The value of the optional `constraints` attribute affects the
+    contents as a whole. Currently, the only constraint, `order`, determines if
+    the processes run in parallel or sequentially.
+
+-   `resources`: A `Resource` object defining the Task's resource
+        footprint. A `Resource` object has three attributes:
+        -   `cpu`: A Float, the fractional number of cores the Task
+        requires.
+        -   `ram`: An Integer, RAM bytes the Task requires.
+        -   `disk`: An integer, disk bytes the Task requires.
+
+A basic Task definition looks like:
+
+    Task(
+        name="hello_world",
+        processes=[Process(name = "hello_world", cmdline = "echo hello world")],
+        resources=Resources(cpu = 1.0,
+                            ram = 1*GB,
+                            disk = 1*GB))
+
+There are four optional Task attributes:
+
+-   `constraints`: A list of `Constraint` objects that constrain the
+    Task's processes. Currently there is only one type, the `order`
+    constraint. For example the following requires that the processes
+    run in the order `foo`, then `bar`.
+
+        constraints = [Constraint(order=['foo', 'bar'])]
+
+    There is an `order()` function that takes `order('foo', 'bar', 'baz')`
+    and converts it into `[Constraint(order=['foo', 'bar', 'baz'])]`.
+    `order()` accepts Process name strings `('foo', 'bar')` or the processes
+    themselves, e.g. `foo=Process(name='foo', ...)`, `bar=Process(name='bar', ...)`,
+    `constraints=order(foo, bar)`
+
+    Note that Thermos rejects tasks with process cycles.
+
+-   `max_failures`: Defaulting to `1`, the number of failed processes
+    needed for the `Task` to be marked as failed. Note how this
+    interacts with individual Processes' `max_failures` values. Assume a
+    Task has two Processes and a `max_failures` value of `2`. So both
+    Processes must fail for the Task to fail. Now, assume each of the
+    Task's Processes has its own `max_failures` value of `10`. If
+    Process "A" fails 5 times before succeeding, and Process "B" fails
+    10 times and is then marked as failing, their parent Task succeeds.
+    Even though there were 15 individual failures by its Processes, only
+    1 of its Processes was finally marked as failing. Since 1 is less
+    than the 2 that is the Task's `max_failures` value, the Task does
+    not fail.
+
+-   `max_concurrency`: Defaulting to `0`, the maximum number of
+    concurrent processes in the Task. `0` specifies unlimited
+    concurrency. For Tasks with many expensive but otherwise independent
+    processes, you can limit the amount of concurrency Thermos schedules
+    instead of artificially constraining them through `order`
+    constraints. For example, a test framework may generate a Task with
+    100 test run processes, but runs it in a Task with
+    `resources.cpus=4`. Limit the amount of parallelism to 4 by setting
+    `max_concurrency=4`.
+
+-   `finalization_wait`: Defaulting to `30`, the number of seconds
+    allocated for finalizing the Task's processes. A Task starts in
+    `ACTIVE` state when Processes run and stays there as long as the Task
+    is healthy and Processes run. When all Processes finish successfully
+    or the Task reaches its maximum process failure limit, it goes into
+    `CLEANING` state. In `CLEANING`, it sends `SIGTERMS` to any still running
+    Processes. When all Processes terminate, the Task goes into
+    `FINALIZING` state and invokes the schedule of all processes whose
+    final attribute has a True value. Everything from the end of `ACTIVE`
+    to the end of `FINALIZING` must happen within `finalization_wait`
+    number of seconds. If not, all still running Processes are sent
+    `SIGKILL`s (or if dependent on yet to be completed Processes, are
+    never invoked).
+
+### SequentialTask: Running Processes in Parallel or Sequentially
+
+By default, a Task with several Processes runs them in parallel. There
+are two ways to run Processes sequentially:
+
+-   Include an `order` constraint in the Task definition's `constraints`
+    attribute whose arguments specify the processes' run order:
+
+        Task( ... processes=[process1, process2, process3],
+	          constraints = order(process1, process2, process3), ...)
+
+-   Use `SequentialTask` instead of `Task`; it automatically runs
+    processes in the order specified in the `processes` attribute. No
+    `constraint` parameter is needed:
+
+        SequentialTask( ... processes=[process1, process2, process3] ...)
+
+### SimpleTask
+
+For quickly creating simple tasks, use the `SimpleTask` helper. It
+creates a basic task from a provided name and command line using a
+default set of resources. For example, in a .`aurora` configuration
+file:
+
+    SimpleTask(name="hello_world", command="echo hello world")
+
+is equivalent to
+
+    Task(name="hello_world",
+         processes=[Process(name = "hello_world", cmdline = "echo hello world")],
+         resources=Resources(cpu = 1.0,
+                             ram = 1*GB,
+                             disk = 1*GB))
+
+The simplest idiomatic Job configuration thus becomes:
+
+    import os
+    hello_world_job = Job(
+      task=SimpleTask(name="hello_world", command="echo hello world"),
+      role=os.getenv('USER'),
+      cluster="cluster1")
+
+When written to `hello_world.aurora`, you invoke it with a simple
+`aurora create cluster1/$USER/test/hello_world hello_world.aurora`.
+
+### Combining tasks
+
+`Tasks.concat`(synonym,`concat_tasks`) and
+`Tasks.combine`(synonym,`combine_tasks`) merge multiple Task definitions
+into a single Task. It may be easier to define complex Jobs
+as smaller constituent Tasks. But since a Job only includes a single
+Task, the subtasks must be combined before using them in a Job.
+Smaller Tasks can also be reused between Jobs, instead of having to
+repeat their definition for multiple Jobs.
+
+With both methods, the merged Task takes the first Task's name. The
+difference between the two is the result Task's process ordering.
+
+-   `Tasks.combine` runs its subtasks' processes in no particular order.
+    The new Task's resource consumption is the sum of all its subtasks'
+    consumption.
+
+-   `Tasks.concat` runs its subtasks in the order supplied, with each
+    subtask's processes run serially between tasks. It is analogous to
+    the `order` constraint helper, except at the Task level instead of
+    the Process level. The new Task's resource consumption is the
+    maximum value specified by any subtask for each Resource attribute
+    (cpu, ram and disk).
+
+For example, given the following:
+
+    setup_task = Task(
+      ...
+      processes=[download_interpreter, update_zookeeper],
+      # It is important to note that {{Tasks.concat}} has
+      # no effect on the ordering of the processes within a task;
+      # hence the necessity of the {{order}} statement below
+      # (otherwise, the order in which {{download_interpreter}}
+      # and {{update_zookeeper}} run will be non-deterministic)
+      constraints=order(download_interpreter, update_zookeeper),
+      ...
+    )
+
+    run_task = SequentialTask(
+      ...
+      processes=[download_application, start_application],
+      ...
+    )
+
+    combined_task = Tasks.concat(setup_task, run_task)
+
+The `Tasks.concat` command merges the two Tasks into a single Task and
+ensures all processes in `setup_task` run before the processes
+in `run_task`. Conceptually, the task is reduced to:
+
+    task = Task(
+      ...
+      processes=[download_interpreter, update_zookeeper,
+                 download_application, start_application],
+      constraints=order(download_interpreter, update_zookeeper,
+                        download_application, start_application),
+      ...
+    )
+
+In the case of `Tasks.combine`, the two schedules run in parallel:
+
+    task = Task(
+      ...
+      processes=[download_interpreter, update_zookeeper,
+                 download_application, start_application],
+      constraints=order(download_interpreter, update_zookeeper) +
+                        order(download_application, start_application),
+      ...
+    )
+
+In the latter case, each of the two sequences may operate in parallel.
+Of course, this may not be the intended behavior (for example, if
+the `start_application` Process implicitly relies
+upon `download_interpreter`). Make sure you understand the difference
+between using one or the other.
+
+## Defining Job Objects
+
+A job is a group of identical tasks that Aurora can run in a Mesos cluster.
+
+A `Job` object is defined by the values of several attributes, some
+required and some optional. The required attributes are:
+
+-   `task`: Task object to bind to this job. Note that a Job can
+    only take a single Task.
+
+-   `role`: Job's role account; in other words, the user account to run
+    the job as on a Mesos cluster machine. A common value is
+    `os.getenv('USER')`; using a Python command to get the user who
+    submits the job request. The other common value is the service
+    account that runs the job, e.g. `www-data`.
+
+-   `environment`: Job's environment, typical values
+    are `devel`, `test`, or `prod`.
+
+-   `cluster`: Aurora cluster to schedule the job in, defined in
+    `/etc/aurora/clusters.json` or `~/.clusters.json`. You can specify
+    jobs where the only difference is the `cluster`, then at run time
+    only run the Job whose job key includes your desired cluster's name.
+
+You usually see a `name` parameter. By default, `name` inherits its
+value from the Job's associated Task object, but you can override this
+default. For these four parameters, a Job definition might look like:
+
+    foo_job = Job( name = 'foo', cluster = 'cluster1',
+              role = os.getenv('USER'), environment = 'prod',
+              task = foo_task)
+
+In addition to the required attributes, there are several optional
+attributes. The first (strongly recommended) optional attribute is:
+
+-   `contact`: An email address for the Job's owner. For production
+    jobs, it is usually a team mailing list.
+
+Two more attributes deal with how to handle failure of the Job's Task:
+
+-   `max_task_failures`: An integer, defaulting to `1`, of the maximum
+    number of Task failures after which the Job is considered failed.
+    `-1` allows for infinite failures.
+
+-   `service`: A boolean, defaulting to `False`, which if `True`
+    restarts tasks regardless of whether they succeeded or failed. In
+    other words, if `True`, after the Job's Task completes, it
+    automatically starts again. This is for Jobs you want to run
+    continuously, rather than doing a single run.
+
+Three attributes deal with configuring the Job's Task:
+
+-   `instances`: Defaulting to `1`, the number of
+    instances/replicas/shards of the Job's Task to create.
+
+-   `priority`: Defaulting to `0`, the Job's Task's preemption priority,
+    for which higher values may preempt Tasks from Jobs with lower
+    values.
+
+-   `production`: a Boolean, defaulting to `False`, specifying that this
+    is a production job backed by quota. Tasks from production Jobs may
+    preempt tasks from any non-production job, and may only be preempted
+    by tasks from production jobs in the same role with higher
+    priority. **WARNING**: To run Jobs at this level, the Job role must
+    have the appropriate quota.
+
+The final three Job attributes each take an object as their value.
+
+-   `update_config`: An `UpdateConfig`
+    object provides parameters for controlling the rate and policy of
+    rolling updates. The `UpdateConfig` parameters are:
+    -   `batch_size`: An integer, defaulting to `1`, specifying the
+        maximum number of shards to update in one iteration.
+    -   `restart_threshold`: An integer, defaulting to `60`, specifying
+        the maximum number of seconds before a shard must move into the
+        `RUNNING` state before considered a failure.
+    -   `watch_secs`: An integer, defaulting to `30`, specifying the
+        minimum number of seconds a shard must remain in the `RUNNING`
+        state before considered a success.
+    -   `max_per_shard_failures`: An integer, defaulting to `0`,
+        specifying the maximum number of restarts per shard during an
+        update. When the limit is exceeded, it increments the total
+        failure count.
+    -   `max_total_failures`: An integer, defaulting to `0`, specifying
+        the maximum number of shard failures tolerated during an update.
+        Cannot be equal to or greater than the job's total number of
+        tasks.
+-   `health_check_config`: A `HealthCheckConfig` object that provides
+    parameters for controlling a Task's health checks via HTTP. Only
+    used if a health port was assigned with a command line wildcard. The
+    `HealthCheckConfig` parameters are:
+    -   `initial_interval_secs`: An integer, defaulting to `60`,
+        specifying the initial delay for doing an HTTP health check.
+    -   `interval_secs`: An integer, defaulting to `30`, specifying the
+        number of seconds in the interval between checking the Task's
+        health.
+    -   `timeout_secs`: An integer, defaulting to `1`, specifying the
+        number of seconds the application must respond to an HTTP health
+        check with `OK` before it is considered a failure.
+    -   `max_consecutive_failures`: An integer, defaulting to `0`,
+        specifying the maximum number of consecutive failures before a
+        task is unhealthy.
+-   `constraints`: A `dict` Python object, specifying Task scheduling
+    constraints. Most users will not need to specify constraints, as the
+    scheduler automatically inserts reasonable defaults. Please do not
+    set this field unless you are sure of what you are doing. See the
+    section in the Aurora + Thermos Reference manual on [Specifying
+    Scheduling Constraints](/documentation/latest/configurationreference/) for more information.
+
+## The jobs List
+
+At the end of your `.aurora` file, you need to specify a list of the
+file's defined Jobs to run in the order listed. For example, the
+following runs first `job1`, then `job2`, then `job3`.
+
+jobs = [job1, job2, job3]
+
+Templating
+----------
+
+The `.aurora` file format is just Python. However, `Job`, `Task`,
+`Process`, and other classes are defined by a templating library called
+*Pystachio*, a powerful tool for configuration specification and reuse.
+
+[Aurora+Thermos Configuration Reference](/documentation/latest/configurationreference/)
+has a full reference of all Aurora/Thermos defined Pystachio objects.
+
+When writing your `.aurora` file, you may use any Pystachio datatypes, as
+well as any objects shown in the *Aurora+Thermos Configuration
+Reference* without `import` statements - the Aurora config loader
+injects them automatically. Other than that the `.aurora` format
+works like any other Python script.
+
+### Templating 1: Binding in Pystachio
+
+Pystachio uses the visually distinctive {{}} to indicate template
+variables. These are often called "mustache variables" after the
+similarly appearing variables in the Mustache templating system and
+because the curly braces resemble mustaches.
+
+If you are familiar with the Mustache system, templates in Pystachio
+have significant differences. They have no nesting, joining, or
+inheritance semantics. On the other hand, when evaluated, templates
+are evaluated iteratively, so this affords some level of indirection.
+
+Let's start with the simplest template; text with one
+variable, in this case `name`;
+
+    Hello {{name}}
+
+If we evaluate this as is, we'd get back:
+
+    Hello
+
+If a template variable doesn't have a value, when evaluated it's
+replaced with nothing. If we add a binding to give it a value:
+
+    { "name" : "Tom" }
+
+We'd get back:
+
+    Hello Tom
+
+We can also use {{}} variables as sectional variables. Let's say we
+have:
+
+    {{#x}} Testing... {{/x}}
+
+If `x` evaluates to `True`, the text between the sectional tags is
+shown. If there is no value for `x` or it evaluates to `False`, the
+between tags text is not shown. So, at a basic level, a sectional
+variable acts as a conditional.
+
+However, if the sectional variable evaluates to a list, array, etc. it
+acts as a `foreach`. For example,
+
+    {{#x}} {{name}} {{/x}}
+
+with
+
+    { "x": [ { "name" : "tic" } { "name" : "tac" } { "name" : "toe" } ] }
+
+evaluates to
+
+    tic tac toe
+
+Every Pystachio object has an associated `.bind` method that can bind
+values to {{}} variables. Bindings are not immediately evaluated.
+Instead, they are evaluated only when the interpolated value of the
+object is necessary, e.g. for performing equality or serializing a
+message over the wire.
+
+Objects with and without mustache templated variables behave
+differently:
+
+    >>> Float(1.5)
+    Float(1.5)
+
+    >>> Float('{{x}}.5')
+    Float({{x}}.5)
+
+    >>> Float('{{x}}.5').bind(x = 1)
+    Float(1.5)
+
+    >>> Float('{{x}}.5').bind(x = 1) == Float(1.5)
+    True
+
+    >>> contextual_object = String('{{metavar{{number}}}}').bind(
+    ... metavar1 = "first", metavar2 = "second")
+
+    >>> contextual_object
+    String({{metavar{{number}}}})
+
+    >>> contextual_object.bind(number = 1)
+    String(first)
+
+    >>> contextual_object.bind(number = 2)
+    String(second)
+
+You usually bind simple key to value pairs, but you can also bind three
+other objects: lists, dictionaries, and structurals. These will be
+described in detail later.
+
+### Structurals in Pystachio / Aurora
+
+Most Aurora/Thermos users don't ever (knowingly) interact with `String`,
+`Float`, or `Integer` Pystashio objects directly. Instead they interact
+with derived structural (`Struct`) objects that are collections of
+fundamental and structural objects. The structural object components are
+called *attributes*. Aurora's most used structural objects are `Job`,
+`Task`, and `Process`:
+
+    class Process(Struct):
+      cmdline = Required(String)
+      name = Required(String)
+      max_failures = Default(Integer, 1)
+      daemon = Default(Boolean, False)
+      ephemeral = Default(Boolean, False)
+      min_duration = Default(Integer, 5)
+      final = Default(Boolean, False)
+
+Construct default objects by following the object's type with (). If you
+want an attribute to have a value different from its default, include
+the attribute name and value inside the parentheses.
+
+    >>> Process()
+    Process(daemon=False, max_failures=1, ephemeral=False,
+      min_duration=5, final=False)
+
+Attribute values can be template variables, which then receive specific
+values when creating the object.
+
+    >>> Process(cmdline = 'echo {{message}}')
+    Process(daemon=False, max_failures=1, ephemeral=False, min_duration=5,
+            cmdline=echo {{message}}, final=False)
+
+    >>> Process(cmdline = 'echo {{message}}').bind(message = 'hello world')
+    Process(daemon=False, max_failures=1, ephemeral=False, min_duration=5,
+            cmdline=echo hello world, final=False)
+
+A powerful binding property is that all of an object's children inherit its
+bindings:
+
+    >>> List(Process)([
+    ... Process(name = '{{prefix}}_one'),
+    ... Process(name = '{{prefix}}_two')
+    ... ]).bind(prefix = 'hello')
+    ProcessList(
+      Process(daemon=False, name=hello_one, max_failures=1, ephemeral=False, min_duration=5, final=False),
+      Process(daemon=False, name=hello_two, max_failures=1, ephemeral=False, min_duration=5, final=False)
+      )
+
+Remember that an Aurora Job contains Tasks which contain Processes. A
+Job level binding is inherited by its Tasks and all their Processes.
+Similarly a Task level binding is available to that Task and its
+Processes but is *not* visible at the Job level (inheritance is a
+one-way street.)
+
+#### Mustaches Within Structurals
+
+When you define a `Struct` schema, one powerful, but confusing, feature
+is that all of that structure's attributes are Mustache variables within
+the enclosing scope *once they have been populated*.
+
+For example, when `Process` is defined above, all its attributes such as
+{{`name`}}, {{`cmdline`}}, {{`max_failures`}} etc., are all immediately
+defined as Mustache variables, implicitly bound into the `Process`, and
+inherit all child objects once they are defined.
+
+Thus, you can do the following:
+
+    >>> Process(name = "installer", cmdline = "echo {{name}} is running")
+    Process(daemon=False, name=installer, max_failures=1, ephemeral=False, min_duration=5,
+            cmdline=echo installer is running, final=False)
+
+WARNING: This binding only takes place in one direction. For example,
+the following does NOT work and does not set the `Process` `name`
+attribute's value.
+
+    >>> Process().bind(name = "installer")
+    Process(daemon=False, max_failures=1, ephemeral=False, min_duration=5, final=False)
+
+The following is also not possible and results in an infinite loop that
+attempts to resolve `Process.name`.
+
+    >>> Process(name = '{{name}}').bind(name = 'installer')
+
+Do not confuse Structural attributes with bound Mustache variables.
+Attributes are implicitly converted to Mustache variables but not vice
+versa.
+
+### Templating 2: Structurals Are Factories
+
+#### A Second Way of Templating
+
+A second templating method is both as powerful as the aforementioned and
+often confused with it. This method is due to automatic conversion of
+Struct attributes to Mustache variables as described above.
+
+Suppose you create a Process object:
+
+    >>> p = Process(name = "process_one", cmdline = "echo hello world")
+
+    >>> p
+    Process(daemon=False, name=process_one, max_failures=1, ephemeral=False, min_duration=5,
+            cmdline=echo hello world, final=False)
+
+This `Process` object, "`p`", can be used wherever a `Process` object is
+needed. It can also be reused by changing the value(s) of its
+attribute(s). Here we change its `name` attribute from `process_one` to
+`process_two`.
+
+    >>> p(name = "process_two")
+    Process(daemon=False, name=process_two, max_failures=1, ephemeral=False, min_duration=5,
+            cmdline=echo hello world, final=False)
+
+Template creation is a common use for this technique:
+
+    >>> Daemon = Process(daemon = True)
+    >>> logrotate = Daemon(name = 'logrotate', cmdline = './logrotate conf/logrotate.conf')
+    >>> mysql = Daemon(name = 'mysql', cmdline = 'bin/mysqld --safe-mode')
+
+### Advanced Binding
+
+As described above, `.bind()` binds simple strings or numbers to
+Mustache variables. In addition to Structural types formed by combining
+atomic types, Pystachio has two container types; `List` and `Map` which
+can also be bound via `.bind()`.
+
+#### Bind Syntax
+
+The `bind()` function can take Python dictionaries or `kwargs`
+interchangeably (when "`kwargs`" is in a function definition, `kwargs`
+receives a Python dictionary containing all keyword arguments after the
+formal parameter list).
+
+    >>> String('{{foo}}').bind(foo = 'bar') == String('{{foo}}').bind({'foo': 'bar'})
+    True
+
+Bindings done "closer" to the object in question take precedence:
+
+    >>> p = Process(name = '{{context}}_process')
+    >>> t = Task().bind(context = 'global')
+    >>> t(processes = [p, p.bind(context = 'local')])
+    Task(processes=ProcessList(
+      Process(daemon=False, name=global_process, max_failures=1, ephemeral=False, final=False,
+              min_duration=5),
+      Process(daemon=False, name=local_process, max_failures=1, ephemeral=False, final=False,
+              min_duration=5)
+    ))
+
+#### Binding Complex Objects
+
+##### Lists
+
+    >>> fibonacci = List(Integer)([1, 1, 2, 3, 5, 8, 13])
+    >>> String('{{fib[4]}}').bind(fib = fibonacci)
+    String(5)
+
+##### Maps
+
+    >>> first_names = Map(String, String)({'Kent': 'Clark', 'Wayne': 'Bruce', 'Prince': 'Diana'})
+    >>> String('{{first[Kent]}}').bind(first = first_names)
+    String(Clark)
+
+##### Structurals
+
+    >>> String('{{p.cmdline}}').bind(p = Process(cmdline = "echo hello world"))
+    String(echo hello world)
+
+### Structural Binding
+
+Use structural templates when binding more than two or three individual
+values at the Job or Task level. For fewer than two or three, standard
+key to string binding is sufficient.
+
+Structural binding is a very powerful pattern and is most useful in
+Aurora/Thermos for doing Structural configuration. For example, you can
+define a job profile. The following profile uses `HDFS`, the Hadoop
+Distributed File System, to designate a file's location. `HDFS` does
+not come with Aurora, so you'll need to either install it separately
+or change the way the dataset is designated.
+
+    class Profile(Struct):
+      version = Required(String)
+      environment = Required(String)
+      dataset = Default(String, hdfs://home/aurora/data/{{environment}}')
+
+    PRODUCTION = Profile(version = 'live', environment = 'prod')
+    DEVEL = Profile(version = 'latest',
+                    environment = 'devel',
+                    dataset = 'hdfs://home/aurora/data/test')
+    TEST = Profile(version = 'latest', environment = 'test')
+
+    JOB_TEMPLATE = Job(
+      name = 'application',
+      role = 'myteam',
+      cluster = 'cluster1',
+      environment = '{{profile.environment}}',
+      task = SequentialTask(
+        name = 'task',
+        resources = Resources(cpu = 2, ram = 4*GB, disk = 8*GB),
+        processes = [
+	  Process(name = 'main', cmdline = 'java -jar application.jar -hdfsPath
+                 {{profile.dataset}}')
+        ]
+       )
+     )
+
+    jobs = [
+      JOB_TEMPLATE(instances = 100).bind(profile = PRODUCTION),
+      JOB_TEMPLATE.bind(profile = DEVEL),
+      JOB_TEMPLATE.bind(profile = TEST),
+     ]
+
+In this case, a custom structural "Profile" is created to self-document
+the configuration to some degree. This also allows some schema
+"type-checking", and for default self-substitution, e.g. in
+`Profile.dataset` above.
+
+So rather than a `.bind()` with a half-dozen substituted variables, you
+can bind a single object that has sensible defaults stored in a single
+place.
+
+Configuration File Writing Tips And Best Practices
+--------------------------------------------------
+
+### Use As Few .aurora Files As Possible
+
+When creating your `.aurora` configuration, try to keep all versions of
+a particular job within the same `.aurora` file. For example, if you
+have separate jobs for `cluster1`, `cluster1` staging, `cluster1`
+testing, and`cluster2`, keep them as close together as possible.
+
+Constructs shared across multiple jobs owned by your team (e.g.
+team-level defaults or structural templates) can be split into separate
+`.aurora`files and included via the `include` directive.
+
+### Avoid Boilerplate
+
+If you see repetition or find yourself copy and pasting any parts of
+your configuration, it's likely an opportunity for templating. Take the
+example below:
+
+`redundant.aurora` contains:
+
+    download = Process(
+      name = 'download',
+      cmdline = 'wget http://www.python.org/ftp/python/2.7.3/Python-2.7.3.tar.bz2',
+      max_failures = 5,
+      min_duration = 1)
+
+    unpack = Process(
+      name = 'unpack',
+      cmdline = 'rm -rf Python-2.7.3 && tar xzf Python-2.7.3.tar.bz2',
+      max_failures = 5,
+      min_duration = 1)
+
+    build = Process(
+      name = 'build',
+      cmdline = 'pushd Python-2.7.3 && ./configure && make && popd',
+      max_failures = 1)
+
+    email = Process(
+      name = 'email',
+      cmdline = 'echo Success | mail feynman@tmc.com',
+      max_failures = 5,
+      min_duration = 1)
+
+    build_python = Task(
+      name = 'build_python',
+      processes = [download, unpack, build, email],
+      constraints = [Constraint(order = ['download', 'unpack', 'build', 'email'])])
+
+As you'll notice, there's a lot of repetition in the `Process`
+definitions. For example, almost every process sets a `max_failures`
+limit to 5 and a `min_duration` to 1. This is an opportunity for factoring
+into a common process template.
+
+Furthermore, the Python version is repeated everywhere. This can be
+bound via structural templating as described in the [Advanced Binding](#AdvancedBinding)
+section.
+
+`less_redundant.aurora` contains:
+
+    class Python(Struct):
+      version = Required(String)
+      base = Default(String, 'Python-{{version}}')
+      package = Default(String, '{{base}}.tar.bz2')
+
+    ReliableProcess = Process(
+      max_failures = 5,
+      min_duration = 1)
+
+    download = ReliableProcess(
+      name = 'download',
+      cmdline = 'wget http://www.python.org/ftp/python/{{python.version}}/{{python.package}}')
+
+    unpack = ReliableProcess(
+      name = 'unpack',
+      cmdline = 'rm -rf {{python.base}} && tar xzf {{python.package}}')
+
+    build = ReliableProcess(
+      name = 'build',
+      cmdline = 'pushd {{python.base}} && ./configure && make && popd',
+      max_failures = 1)
+
+    email = ReliableProcess(
+      name = 'email',
+      cmdline = 'echo Success | mail {{role}}@foocorp.com')
+
+    build_python = SequentialTask(
+      name = 'build_python',
+      processes = [download, unpack, build, email]).bind(python = Python(version = "2.7.3"))
+
+### Thermos Uses bash, But Thermos Is Not bash
+
+#### Bad
+
+Many tiny Processes makes for harder to manage configurations.
+
+    copy = Process(
+      name = 'copy',
+      cmdline = 'rcp user@my_machine:my_application .'
+     )
+
+     unpack = Process(
+       name = 'unpack',
+       cmdline = 'unzip app.zip'
+     )
+
+     remove = Process(
+       name = 'remove',
+       cmdline = 'rm -f app.zip'
+     )
+
+     run = Process(
+       name = 'app',
+       cmdline = 'java -jar app.jar'
+     )
+
+     run_task = Task(
+       processes = [copy, unpack, remove, run],
+       constraints = order(copy, unpack, remove, run)
+     )
+
+#### Good
+
+Each `cmdline` runs in a bash subshell, so you have the full power of
+bash. Chaining commands with `&&` or `||` is almost always the right
+thing to do.
+
+Also for Tasks that are simply a list of processes that run one after
+another, consider using the `SequentialTask` helper which applies a
+linear ordering constraint for you.
+
+    stage = Process(
+      name = 'stage',
+      cmdline = 'rcp user@my_machine:my_application . && unzip app.zip && rm -f app.zip')
+
+    run = Process(name = 'app', cmdline = 'java -jar app.jar')
+
+    run_task = SequentialTask(processes = [stage, run])
+
+### Rarely Use Functions In Your Configurations
+
+90% of the time you define a function in a `.aurora` file, you're
+probably Doing It Wrong(TM).
+
+#### Bad
+
+    def get_my_task(name, user, cpu, ram, disk):
+      return Task(
+        name = name,
+        user = user,
+        processes = [STAGE_PROCESS, RUN_PROCESS],
+        constraints = order(STAGE_PROCESS, RUN_PROCESS),
+        resources = Resources(cpu = cpu, ram = ram, disk = disk)
+     )
+
+     task_one = get_my_task('task_one', 'feynman', 1.0, 32*MB, 1*GB)
+     task_two = get_my_task('task_two', 'feynman', 2.0, 64*MB, 1*GB)
+
+#### Good
+
+This one is more idiomatic. Forced keyword arguments prevents accidents,
+e.g. constructing a task with "32*MB" when you mean 32MB of ram and not
+disk. Less proliferation of task-construction techniques means
+easier-to-read, quicker-to-understand, and a more composable
+configuration.
+
+    TASK_TEMPLATE = SequentialTask(
+      user = 'wickman',
+      processes = [STAGE_PROCESS, RUN_PROCESS],
+    )
+
+    task_one = TASK_TEMPLATE(
+      name = 'task_one',
+      resources = Resources(cpu = 1.0, ram = 32*MB, disk = 1*GB) )
+
+    task_two = TASK_TEMPLATE(
+      name = 'task_two',
+      resources = Resources(cpu = 2.0, ram = 64*MB, disk = 1*GB)
+    )

Added: incubator/aurora/site/source/documentation/latest/contributing.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/contributing.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/contributing.md (added)
+++ incubator/aurora/site/source/documentation/latest/contributing.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,61 @@
+Getting your ReviewBoard Account
+--------------------------------
+Go to https://reviews.apache.org and create an account.
+
+Setting up your email account (committers)
+------------------------------------------
+Once your Apache ID has been set up you can configure your account and add ssh keys and
+setup an email forwarding address at
+
+  http://id.apache.org
+
+Additional instructions for setting up your new committer email can be found at
+
+  http://www.apache.org/dev/user-email.html
+
+The recommended setup is to configure all services (mailing lists, JIRA, ReviewBoard) to
+send emails to your @apache.org email address.
+
+Setting up your ReviewBoard Environment
+---------------------------------------
+Run `./rbt status`. The first time this runs it will bootstrap and you will be asked to login.
+Subsequent runs will cache your login credentials.
+
+Submitting a Patch for Review
+-----------------------------
+Post a review with `rbt`, fill out the fields in your browser and hit Publish.
+
+    ./rbt post -o -g
+
+Updating an Existing Review
+---------------------------
+Incorporate review feedback, make some more commits, update your existing review, fill out the
+fields in your browser and hit Publish.
+
+    ./rbt post -o -r <RB_ID>
+
+Merging Your Own Review (Committers)
+------------------------------------
+Once you have shipits from the right committers, merge your changes in a single commit and mark
+the review as submitted. The typical workflow is:
+
+    git checkout master
+    git pull origin master
+    ./rbt patch -c <RB_ID>  # Verify the automatically-generated commit message looks sane,
+                            # editing if necessary.
+    git show master         # Verify everything looks sane
+    git push origin master
+    ./rbt close <RB_ID>
+
+Note that even if you're developing using feature branches you will not use `git merge` - each
+commit will be an atomic change accompanied by a ReviewBoard entry.
+
+Merging Someone Else's Review
+-----------------------------
+Sometimes you'll need to merge someone else's RB. The typical workflow for this is
+
+    git checkout master
+    git pull origin master
+    ./rbt patch -c <RB_ID>
+    git show master  # Verify everything looks sane, author is correct
+    git push origin master

Added: incubator/aurora/site/source/documentation/latest/deploying-aurora-scheduler.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/deploying-aurora-scheduler.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/deploying-aurora-scheduler.md (added)
+++ incubator/aurora/site/source/documentation/latest/deploying-aurora-scheduler.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,146 @@
+The Aurora scheduler is responsible for scheduling new jobs, rescheduling failed jobs, and killing
+old jobs.
+
+Installing Aurora
+=================
+Aurora is a standalone Java server. As part of the build process it creates a bundle of all its
+dependencies, with the notable exceptions of the JVM and libmesos. Each target server should have
+a JVM (Java 7 or higher) and libmesos (0.17.0) installed.
+
+Creating the Distribution .zip File (Optional)
+----------------------------------------------
+To create a distribution for installation you will need build tools installed. On Ubuntu this can be
+done with `sudo apt-get install build-essential default-jdk`.
+
+    git clone http://git-wip-us.apache.org/repos/asf/incubator-aurora.git
+    cd incubator-aurora
+    ./gradlew distZip
+
+Copy the generated `dist/distributions/aurora-scheduler-*.zip` to each node that will run a scheduler.
+
+Installing Aurora
+-----------------
+Extract the aurora-scheduler zip file. The example configurations assume it is extracted to
+`/usr/local/aurora-scheduler`.
+
+    sudo unzip dist/distributions/aurora-scheduler-*.zip -d /usr/local
+    sudo ln -nfs "$(ls -dt /usr/local/aurora-scheduler-* | head -1)" /usr/local/aurora-scheduler
+
+Configuring Aurora
+==================
+
+A Note on Configuration
+-----------------------
+Like Mesos, Aurora uses command-line flags for runtime configuration. As such the Aurora
+"configuration file" is typically a `scheduler.sh` shell script of the form.
+
+    #!/bin/bash
+    AURORA_HOME=/usr/local/aurora-scheduler
+
+    # Flags controlling the JVM.
+    JAVA_OPTS=(
+      -Xmx2g
+      -Xms2g
+      # GC tuning, etc.
+    )
+
+    # Flags controlling the scheduler.
+    AURORA_FLAGS=(
+      -http_port=8081
+      -thrift_port=8082
+      # Log configuration, etc.
+    )
+
+    # Environment variables controlling libmesos
+    export JAVA_HOME=...
+    export GLOG_v=1
+    export LIBPROCESS_PORT=8083
+
+    JAVA_OPTS="${JAVA_OPTS[*]}" exec "$AURORA_HOME/bin/aurora-scheduler" "${AURORA_FLAGS[@]}"
+
+That way Aurora's current flags are visible in `ps` and in the `/vars` admin endpoint.
+
+Examples are available under `examples/scheduler/`. For a list of available Aurora flags and their
+documentation run
+
+    /usr/local/aurora-scheduler/bin/aurora-scheduler -help
+
+Replicated Log Configuration
+----------------------------
+All Aurora state is persisted to a replicated log. This includes all jobs Aurora is running
+including where in the cluster they are being run and the configuration for running them, as
+well as other information such as metadata needed to reconnect to the Mesos master, resource
+quotas, and any other locks in place.
+
+Aurora schedulers use ZooKeeper to discover log replicas and elect a leader. Only one scheduler is
+leader at a given time - the other schedulers follow log writes and prepare to take over as leader
+but do not communicate with the Mesos master. Either 3 or 5 schedulers are recommended in a
+production deployment depending on failure tolerance and they must have persistent storage.
+
+In a cluster with `N` schedulers, the flag `-native_log_quorum_size` should be set to
+`floor(N/2) + 1`. So in a cluster with 1 scheduler it should be set to `1`, in a cluster with 3 it
+should be set to `2`, and in a cluster of 5 it should be set to `3`.
+
+<table>
+  <thead>
+    <tr>
+      <th>Number of schedulers (N)
+      <th><code>-native_log_quorum_size</code> setting (<code>floor(N/2) + 1</code>)
+  <tbody>
+    <tr>
+      <td>1
+      <td>1
+    <tr>
+      <td>3
+      <td>2
+    <tr>
+      <td>5
+      <td>3
+    <tr>
+      <td>7
+      <td>4
+</table>
+
+*Incorrectly setting this flag will cause data corruption to occur!*
+
+Network considerations
+----------------------
+The Aurora scheduler listens on 3 ports - a Thrift port for client RPCs, an admin web UI, and a
+libprocess (HTTP+Protobuf) port used to communicate with the Mesos master and for the log
+replication protocol. These can be left unconfigured (the scheduler publishes all selected ports
+to ZooKeeper) or explicitly set in the startup script as follows:
+
+    # ...
+    AURORA_FLAGS=(
+      # ...
+      -http_port=8081
+      -thrift_port=8082
+      # ...
+    )
+    # ...
+    export LIBPROCESS_PORT=8083
+    # ...
+
+Running Aurora
+==============
+Configure a supervisor like [Monit](http://mmonit.com/monit/) or
+[supervisord](http://supervisord.org/) to run the created `scheduler.sh` file and restart it
+whenever it fails. Aurora expects to be restarted by an external process when it fails. Aurora
+supports an active health checking protocol on its admin HTTP interface - if a `GET /health` times
+out or returns anything other than `200 OK` the scheduler process is unhealthy and should be
+restarted.
+
+For example, monit can be configured with
+
+    if failed port 8081 send "GET /health HTTP/1.0\r\n" expect "OK\n" with timeout 2 seconds for 10 cycles then restart
+
+assuming you set `-http_port=8081`.
+
+Maintaining an Aurora Installation
+==================================
+
+Monitoring
+----------
+Aurora exports performance metrics via its HTTP interface `/vars` and `/vars.json` contain lots of
+useful data to help debug performance and configuration problems. These are all made available via
+[twitter.common.http](https://github.com/twitter/commons/tree/master/src/java/com/twitter/commons/http).

Added: incubator/aurora/site/source/documentation/latest/developing-aurora-scheduler.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/developing-aurora-scheduler.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/developing-aurora-scheduler.md (added)
+++ incubator/aurora/site/source/documentation/latest/developing-aurora-scheduler.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,100 @@
+Java code in the aurora repo is built with [Gradle](http://gradle.org).
+
+Getting Started
+===============
+You will need Java 7 installed and on your `PATH` or unzipped somewhere with `JAVA_HOME` set. Then
+
+    ./gradlew tasks
+
+will bootstrap the build system and show available tasks. This can take a while the first time you
+run it but subsequent runs will be much faster due to cached artifacts.
+
+Running the Tests
+-----------------
+Aurora has a comprehensive unit test suite. To run the tests use
+
+    ./gradlew build
+
+Gradle will only re-run tests when dependencies of them have changed. To force a re-run of all
+tests use
+
+    ./gradlew clean build
+
+Creating a bundle for deployment
+--------------------------------
+Gradle can create a zip file containing Aurora, all of its dependencies, and a launch script with
+
+    ./gradlew distZip
+
+or a tar file containing the same files with
+
+    ./gradlew distTar
+
+The output file will be written to `dist/distributions/aurora-scheduler.zip` or
+`dist/distributions/aurora-scheduler.tar`.
+
+Developing Aurora Java code
+===========================
+
+Setting up an IDE
+-----------------
+Gradle can generate project files for your IDE. To generate an IntelliJ IDEA project run
+
+    ./gradlew idea
+
+and import the generated `aurora.ipr` file.
+
+Adding or Upgrading a Dependency
+--------------------------------
+New dependencies can be added from Maven central by adding a `compile` dependency to `build.gradle`.
+For example, to add a dependency on `com.example`'s `example-lib` 1.0 add this block:
+
+    compile 'com.example:example-lib:1.0'
+
+NOTE: Anyone thinking about adding a new dependency should first familiarize themself with the
+Apache Foundation's third-party licensing
+[policy](http://www.apache.org/legal/resolved.html#category-x).
+
+Developing Aurora UI
+======================
+
+Installing bower (optional)
+----------------------------
+Third party JS libraries used in Aurora (located at 3rdparty/javascript/bower_components) are
+managed by bower, a JS dependency manager. Bower is only required if you plan to add, remove or
+update JS libraries. Bower can be installed using the following command:
+
+    npm install -g bower
+
+Bower depends on node.js and npm. The easiest way to install node on a mac is via brew:
+
+    brew install node
+
+For more node.js installation options refer to https://github.com/joyent/node/wiki/Installation.
+
+More info on installing and using bower can be found at: http://bower.io/. Once installed, you can
+use the following commands to view and modify the bower repo at
+3rdparty/javascript/bower_components
+
+    bower list
+    bower install <library name>
+    bower remove <library name>
+    bower update <library name>
+    bower help
+
+Developing the Aurora Build System
+==================================
+
+Bootstrapping Gradle
+--------------------
+The following files were autogenerated by `gradle wrapper` using gradle 1.8's
+[Wrapper](http://www.gradle.org/docs/1.8/dsl/org.gradle.api.tasks.wrapper.Wrapper.html) plugin and
+should not be modified directly:
+
+    ./gradlew
+    ./gradlew.bat
+    ./gradle/wrapper/gradle-wrapper.jar
+    ./gradle/wrapper/gradle-wrapper.properties
+
+To upgrade Gradle unpack the new version somewhere, run `/path/to/new/gradle wrapper` in the
+repository root and commit the changed files.

Added: incubator/aurora/site/source/documentation/latest/hooks.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/hooks.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/hooks.md (added)
+++ incubator/aurora/site/source/documentation/latest/hooks.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,270 @@
+# Hooks for Aurora Client API
+
+- [Introduction](#introduction)
+- [Hook Types](#hook-types)
+- [Execution Order](#execution-order)
+- [Hookable Methods](#hookable-methods)
+- [Activating and Using Hooks](#activating-and-using-hooks)
+- [.aurora Config File Settings](#aurora-config-file-settings)
+- [Command Line](#command-line)
+- [Hooks Protocol](#hooks-protocol)
+  - [pre_ Methods](#pre_-methods)
+  - [err_ Methods](#err_-methods)
+  - [post_ Methods](#post_-methods)
+- [Generic Hooks](#generic-hooks)
+- [Hooks Process Checklist](#hooks-process-checklist)
+
+## Introduction
+
+You can execute hook methods around Aurora API Client methods when they are called by the Aurora Command Line commands.
+
+Explaining how hooks work is a bit tricky because of some indirection about what they apply to. Basically, a hook is code that executes when a particular Aurora Client API method runs, letting you extend the method's actions. The hook executes on the client side, specifically on the machine executing Aurora commands.
+
+The catch is that hooks are associated with Aurora Client API methods, which users don't directly call. Instead, users call Aurora Command Line commands, which call Client API methods during their execution. Since which hooks run depend on which Client API methods get called, you will need to know which Command Line commands call which API methods. Later on, there is a table showing the various associations.
+
+**Terminology Note**: From now on, "method(s)" refer to Client API methods, and "command(s)" refer to Command Line commands.
+
+## Hook Types
+
+Hooks have three basic types, differing by when they run with respect to their associated method.
+
+`pre_<method_name>`: When its associated method is called, the `pre_` hook executes first, then the called method. If the `pre_` hook fails, the method never runs. Later code that expected the method to succeed may be affected by this, and result in terminating the Aurora client.
+
+Note that a `pre_` hook can error-trap internally so it does not
+return `False`. Designers/contributors of new `pre_` hooks should
+consider whether or not to error-trap them. You can error trap at the
+highest level very generally and always pass the `pre_` hook by
+returning `True`. For example:
+
+```python
+def pre_create(...):
+  do_something()  # if do_something fails with an exception, the create_job is not attempted!
+  return True
+
+# However...
+def pre_create(...):
+  try:
+    do_something()  # may cause exception
+  except Exception:  # generic error trap will catch it
+    pass  # and ignore the exception
+  return True  # create_job will run in any case!
+```
+
+`post_<method_name>`: A `post_` hook executes after its associated method successfully finishes running. If it fails, the already executed method is unaffected. A `post_` hook's error is trapped, and any later operations are unaffected.
+
+`err_<method_name>`: Executes only when its associated method returns a status other than OK or throws an exception. If an `err_` hook fails, the already executed method is unaffected. An `err_` hook's error is trapped, and any later operations are unaffected.
+
+## Execution Order
+
+A command with `pre_`, `post_`, and `err_` hooks defined and activated for its called method executes in the following order when the method successfully executes:
+
+1. Command called
+2. Command code executes
+3. Method Called
+4. `pre_` method hook runs
+5. Method runs and successfully finishes
+6. `post_` method hook runs
+7. Command code executes
+8. Command execution ends
+
+The following is what happens when, for the same command and hooks, the method associated with the command suffers an error and does not successfully finish executing:
+
+1. Command called
+2. Command code executes
+3. Method Called
+4. `pre_` method hook runs
+5. Method runs and fails
+6. `err_` method hook runs
+7. Command Code executes (if `err_` method does not end the command execution)
+8. Command execution ends
+
+Note that the `post_` and `err_` hooks for the same method can never both run for a single execution of that method.
+
+## Hookable Methods
+
+You can associate `pre_`, `post_`, and `err_` hooks with the following methods. Since you do not directly interact with the methods, but rather the Aurora Command Line commands that call them, for each method we also list the command(s) that can call the method. Note that a different method or methods may be called by a command depending on how the command's other code executes. Similarly, multiple commands can call the same method. We also list the methods' argument signatures, which are used by their associated hooks. <a name="Chart"></a>
+
+<table border="1" cellpadding="0" cellspacing="0">
+  <tbody>
+    <tr>
+      <th>Aurora Client API Method</td>
+      <th>Client API Method Argument Signature</td>
+      <th>Aurora Command Line Command</td>
+    </tr>
+    <tr>
+      <td><code>cancel_update</code></td>
+      <td><code>self</code>, <code>job_key</code></td>
+      <td><code>cancel_update</code></td>
+    </tr>
+    <tr>
+      <td><code>create_job</code></td>
+      <td><code>self</code>, <code>config</code></td>
+      <td><code>create</code>, <code>runtask</code></td>
+    </tr>
+    <tr>
+      <td><code>restart</code></td>
+      <td><code>self</code>, <code>job_key</code>, <code>shards</code>, <code>update_config</code>, <code>health_check_interval_seconds</code></td>
+      <td><code>restart</code></td>
+    </tr>
+    <tr>
+      <td><code>update_job</code></td>
+      <td><code>self</code>, <code>config</code>, <code>health_check_interval_seconds=3</code>, <code>shards=None</code></td>
+      <td><code>update</code></td>
+     </tr>
+     <tr>
+       <td><code>kill_job</code></td>
+       <td><code>self</code>, <code>job_key</code>, <code>shards=None</code></td>
+       <td><code>kill</code></td>
+     </tr>
+   </table>
+
+Some specific examples:
+
+* `pre_create_job` executes when a `create_job` method is called, and before the `create_job` method itself executes.
+
+* `post_cancel_update` executes after a `cancel_update` method has successfully finished running.
+
+* `err_kill_job` executes when the `kill_job` method is called, but doesn't successfully finish running.
+
+## Activating and Using Hooks
+
+By default, hooks are inactive. If you do not want to use hooks, you do not need to make any changes to your code. If you do want to use hooks, you will need to alter your `.aurora` config file to activate them both for the configuration as a whole as well as for individual `Job`s. And, of course, you will need to define in your config file what happens when a particular hook executes.
+
+## .aurora Config File Settings
+
+You can define a top-level `hooks` variable in any `.aurora` config file. `hooks` is a list of all objects that define hooks used by `Job`s defined in that config file. If you do not want to define any hooks for a configuration, `hooks` is optional.
+
+    hooks = [Object_with_defined_hooks1, Object_with_defined_hooks2]
+
+Be careful when assembling a config file using `include` on multiple smaller config files. If there are multiple files that assign a value to `hooks`, only the last assignment made will stick. For example, if `x.aurora` has `hooks = [a, b, c]` and `y.aurora` has `hooks = [d, e, f]` and `z.aurora` has, in this order, `include x.aurora` and `include y.aurora`, the `hooks` value will be `[d, e, f]`.
+
+Also, for any `Job` that you want to use hooks with, its `Job` definition in the `.aurora` config file must set an `enable_hooks` flag to `True` (it defaults to `False`). By default, hooks are disabled and you must enable them for `Job`s of your choice.
+
+To summarize, to use hooks for a particular job, you must both activate hooks for your config file as a whole, and for that job. Activating hooks only for individual jobs won't work, nor will only activating hooks for your config file as a whole. You must also specify the hooks' defining object in the `hooks` variable.
+
+Recall that `.aurora` config files are written in Pystachio. So the following turns on hooks for production jobs at cluster1 and cluster2, but leaves them off for similar jobs with a defined user role. Of course, you also need to list the objects that define the hooks in your config file's `hooks` variable.
+
+```python
+jobs = [
+        Job(enable_hooks = True, cluster = c, env = 'prod') for c in ('cluster1', 'cluster2')
+       ]
+jobs.extend(
+   Job(cluster = c, env = 'prod', role = getpass.getuser()) for c in ('cluster1', 'cluster2'))
+   # Hooks disabled for these jobs
+```
+
+## Command Line
+
+All Aurora Command Line commands now accept an `.aurora` config file as an optional parameter (some, of course, accept it as a required parameter). Whenever a command has a `.aurora` file parameter, any hooks specified and activated in the `.aurora` file can be used. For example:
+
+    aurora restart cluster1/role/env/app myapp.aurora
+
+The command activates any hooks specified and activated in `myapp.aurora`. For the `restart` command, that is the only thing the `myapp.aurora` parameter does. So, if the command was the following, since there is no `.aurora` config file to specify any hooks, no hooks on the `restart` command can run.
+
+    aurora restart cluster1/role/env/app
+
+## Hooks Protocol
+
+Any object defined in the `.aurora` config file can define hook methods. You should define your hook methods within a class, and then use the class name as a value in the `hooks` list in your config file.
+
+Note that you can define other methods in the class that its hook methods can call; all the logic of a hook does not have to be in its definition.
+
+The following example defines a class containing a `pre_kill_job` hook definition that calls another method defined in the class.
+
+```python
+# Defines a method pre_kill_job
+class KillConfirmer(object):
+  def confirm(self, msg):
+    return raw_input(msg).lower() == 'yes'
+
+  def pre_kill_job(self, job_key, shards=None):
+    shards = ('shards %s' % shards) if shards is not None else 'all shards'
+    return self.confirm('Are you sure you want to kill %s (%s)? (yes/no): '
+                        % (job_key, shards))
+```
+
+### pre_ Methods
+
+`pre_` methods have the signature:
+
+    pre_<API method name>(self, <associated method's signature>)
+
+`pre_` methods have the same signature as their associated method, with the addition of `self` as the first parameter. See the [chart](#Chart) above for the mapping of parameters to methods. When writing `pre_` methods, you can use the `*` and `**` syntax to designate that all unspecified parameters are passed in a list to the `*`ed variable and all named parameters with values are passed as name/value pairs to the `**`ed variable.
+
+If this method returns False, the API command call aborts.
+
+### err_ Methods
+
+`err_` methods have the signature:
+
+    err_<API method name>(self, exc, <associated method's signature>)
+
+`err_` methods have the same signature as their associated method, with the addition of a first parameter `self` and a second parameter `exc`. `exc` is either a result with responseCode other than `ResponseCode.OK` or an `Exception`. See the [chart](#Chart) above for the mapping of parameters to methods. When writing `err`_ methods, you can use the `*` and `**` syntax to designate that all unspecified parameters are passed in a list to the `*`ed variable and all named parameters with values are passed as name/value pairs to the `**`ed variable.
+
+`err_` method return codes are ignored.
+
+### post_ Methods
+
+`post_` methods have the signature:
+
+    post_<API method name>(self, result, <associated method signature>)
+
+`post_` method parameters are `self`, then `result`, followed by the same parameter signature as their associated method. `result` is the result of the associated method call. See the [chart](#chart) above for the mapping of parameters to methods. When writing `post_` methods, you can use the `*` and `**` syntax to designate that all unspecified arguments are passed in a list to the `*`ed parameter and all unspecified named arguments with values are passed as name/value pairs to the `**`ed parameter.
+
+`post_` method return codes are ignored.
+
+## Generic Hooks
+
+There are five Aurora API Methods which any of the three hook types can attach to. Thus, there are 15 possible hook/method combinations for a single `.aurora` config file. Say that you define `pre_` and `post_` hooks for the `restart` method. That leaves 13 undefined hook/method combinations; `err_restart` and the 3 `pre_`, `post_`, and `err_` hooks for each of the other 4 hookable methods. You can define what happens when any of these otherwise undefined 13 hooks execute via a generic hook, whose signature is:
+
+```python
+generic_hook(self, hook_config, event, method_name, result_or_err, args*, kw**)
+```
+
+where:
+
+* `hook_config` is a named tuple of `config` (the Pystashio `config` object) and `job_key`.
+
+* `event` is one of `pre`, `err`, or `post`, indicating which type of hook the genetic hook is standing in for. For example, assume no specific hooks were defined for the `restart` API command. If `generic_hook` is defined and activated, and `restart` is called, `generic_hook` will effectively run as `pre_restart`, `post_restart`, and `err_restart`. You can use a selection statement on this value so that `generic_hook` will act differently based on whether it is standing in for a `pre_`, `post_`, or `err_` hook.
+
+* `method_name` is the Client API method name whose execution is causing this execution of the `generic_hook`.
+
+* `args*`, `kw**` are the API method arguments and keyword arguments respectively.
+* `result_or_err` is a tri-state parameter taking one of these three values:
+  1. None for `pre_`hooks
+  2. `result` for `post_` nooks
+  3. `exc` for `err_` hooks
+
+Example:
+
+```python
+# Overrides the standard do-nothing generic_hook by adding a log writing operation.
+from twitter.common import log
+  class Logger(object):
+    '''Adds to the log every time a hookable API method is called'''
+    def generic_hook(self, hook_config, event, method_name, result_or_err, *args, **kw)
+      log.info('%s: %s_%s of %s'
+               % (self.__class__.__name__, event, method_name, hook_config.job_key))
+```
+
+## Hooks Process Checklist
+
+1. In your `.aurora` config file, add a `hooks` variable. Note that you may want to define a `.aurora` file only for hook definitions and then include this file in multiple other config files that you want to use the same hooks.
+
+```python
+hooks = []
+```
+
+2. In the `hooks` variable, list all objects that define hooks used by `Job`s defined in this config:
+
+```python
+hooks = [Object_hook_definer1, Object_hook_definer2]
+```
+
+3. For each job that uses hooks in this config file, add `enable_hooks = True` to the `Job` definition. Note that this is necessary even if you only want to use the generic hook.
+
+4. Write your `pre_`, `post_`, and `err_` hook definitions as part of an object definition in your `.aurora` config file.
+
+5. If desired, write your `generic_hook` definition as part of an object definition in your `.aurora` config file. Remember, the object must be listed as a member of `hooks`.
+
+6. If your Aurora command line command does not otherwise take an `.aurora` config file argument, add the appropriate `.aurora` file as an argument in order to define and activate the configuration's hooks.

Added: incubator/aurora/site/source/documentation/latest/images/CPUavailability.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/CPUavailability.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/CPUavailability.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/HelloWorldJob.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/HelloWorldJob.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/HelloWorldJob.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/RoleJobs.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/RoleJobs.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/RoleJobs.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/ScheduledJobs.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/ScheduledJobs.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/ScheduledJobs.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/TaskBreakdown.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/TaskBreakdown.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/TaskBreakdown.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/aurora_hierarchy.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/aurora_hierarchy.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/aurora_hierarchy.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/killedtask.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/killedtask.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/killedtask.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/lifeofatask.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/lifeofatask.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/lifeofatask.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/runningtask.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/runningtask.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/runningtask.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/stderr.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/stderr.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/stderr.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/images/stdout.png
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/images/stdout.png?rev=1581246&view=auto
==============================================================================
Binary file - no diff available.

Propchange: incubator/aurora/site/source/documentation/latest/images/stdout.png
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: incubator/aurora/site/source/documentation/latest/resourceisolation.md
URL: http://svn.apache.org/viewvc/incubator/aurora/site/source/documentation/latest/resourceisolation.md?rev=1581246&view=auto
==============================================================================
--- incubator/aurora/site/source/documentation/latest/resourceisolation.md (added)
+++ incubator/aurora/site/source/documentation/latest/resourceisolation.md Tue Mar 25 06:10:05 2014
@@ -0,0 +1,147 @@
+Resource Isolation and Sizing
+=============================
+
+**NOTE**: Resource Isolation and Sizing is very much a work in progress.
+Both user-facing aspects and how it works under the hood are subject to
+change.
+
+- [Introduction](#introduction)
+- [CPU Isolation](#cpu-isolation)
+- [CPU Sizing](#cpu-sizing)
+- [Memory Isolation](#memory-isolation)
+- [Memory Sizing](#memory-sizing)
+- [Disk Space](#disk-space)
+- [Disk Space Sizing](#disk-space-sizing)
+- [Other Resources](#other-resources)
+
+## Introduction
+
+Aurora is a multi-tenant system; a single software instance runs on a
+server, serving multiple clients/tenants. To share resources among
+tenants, it implements isolation of:
+
+* CPU
+* memory
+* disk space
+
+CPU is a soft limit, and handled differently from memory and disk space.
+Too low a CPU value results in throttling your application and
+slowing it down. Memory and disk space are both hard limits; when your
+application goes over these values, it's killed.
+
+Let's look at each resource type in more detail:
+
+## CPU Isolation
+
+Mesos uses a quota based CPU scheduler (the *Completely Fair Scheduler*)
+to provide consistent and predictable performance.  This is effectively
+a guarantee of resources -- you receive at least what you requested, but
+also no more than you've requested.
+
+The scheduler gives applications a CPU quota for every 100 ms interval.
+When an application uses its quota for an interval, it is throttled for
+the rest of the 100 ms. Usage resets for each interval and unused
+quota does not carry over.
+
+For example, an application specifying 4.0 CPU has access to 400 ms of
+CPU time every 100 ms. This CPU quota can be used in different ways,
+depending on the application and available resources. Consider the
+scenarios shown in this diagram.
+
+![CPU Availability](images/CPUavailability.png)
+
+* *Scenario A*: the application can use up to 4 cores continuously for
+every 100 ms interval. It is never throttled and starts processing
+new requests immediately.
+
+* *Scenario B* : the application uses up to 8 cores (depending on
+availability) but is throttled after 50 ms. The CPU quota resets at the
+start of each new 100 ms interval.
+
+* *Scenario C* : is like Scenario A, but there is a garbage collection
+event in the second interval that consumes all CPU quota. The
+application throttles for the remaining 75 ms of that interval and
+cannot service requests until the next interval. In this example, the
+garbage collection finished in one interval but, depending on how much
+garbage needs collecting, it may take more than one interval and further
+delay service of requests.
+
+*Technical Note*: Mesos considers logical cores, also known as
+hyperthreading or SMT cores, as the unit of CPU.
+
+## CPU Sizing
+
+To correctly size Aurora-run Mesos tasks, specify a per-shard CPU value
+that lets the task run at its desired performance when at peak load
+distributed across all shards. Include reserve capacity of at least 50%,
+possibly more, depending on how critical your service is (or how
+confident you are about your original estimate : -)), ideally by
+increasing the number of shards to also improve resiliency. When running
+your application, observe its CPU stats over time. If consistently at or
+near your quota during peak load, you should consider increasing either
+per-shard CPU or the number of shards.
+
+## Memory Isolation
+
+Mesos uses dedicated memory allocation. Your application always has
+access to the amount of memory specified in your configuration. The
+application's memory use is defined as the sum of the resident set size
+(RSS) of all processes in a shard. Each shard is considered
+independently.
+
+In other words, say you specified a memory size of 10GB. Each shard
+would receive 10GB of memory. If an individual shard's memory demands
+exceed 10GB, that shard is killed, but the other shards continue
+working.
+
+*Technical note*: Total memory size is not enforced at allocation time,
+so your application can request more than its allocation without getting
+an ENOMEM. However, it will be killed shortly after.
+
+## Memory Sizing
+
+Size for your application's peak requirement. Observe the per-instance
+memory statistics over time, as memory requirements can vary over
+different periods. Remember that if your application exceeds its memory
+value, it will be killed, so you should also add a safety margin of
+around 10-20%. If you have the ability to do so, you may also want to
+put alerts on the per-instance memory.
+
+## Disk Space
+
+Disk space used by your application is defined as the sum of the files'
+disk space in your application's directory, including the `stdout` and
+`stderr` logged from your application. Each shard is considered
+independently. You should use off-node storage for your application's
+data whenever possible.
+
+In other words, say you specified disk space size of 100MB. Each shard
+would receive 100MB of disk space. If an individual shard's disk space
+demands exceed 100MB, that shard is killed, but the other shards
+continue working.
+
+After your application finishes running, its allocated disk space is
+reclaimed. Thus, your job's final action should move any disk content
+that you want to keep, such as logs, to your home file system or other
+less transitory storage. Disk reclamation takes place an undefined
+period after the application finish time; until then, the disk contents
+are still available but you shouldn't count on them being so.
+
+*Technical note* : Disk space is not enforced at write so your
+application can write above its quota without getting an ENOSPC, but it
+will be killed shortly after. This is subject to change.
+
+## Disk Space Sizing
+
+Size for your application's peak requirement. Rotate and discard log
+files as needed to stay within your quota. When running a Java process,
+add the maximum size of the Java heap to your disk space requirement, in
+order to account for an out of memory error dumping the heap
+into the application's sandbox space.
+
+## Other Resources
+
+Other resources, such as network bandwidth, do not have any performance
+guarantees. For some resources, such as memory bandwidth, there are no
+practical sharing methods so some application combinations collocated on
+the same host may cause contention.