You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by re...@apache.org on 2018/04/03 23:54:50 UTC

svn commit: r1828294 [25/27] - in /aurora/site: publish/blog/aurora-0-19-1-released/ publish/blog/aurora-0-20-0-released/ publish/documentation/0.19.1/ publish/documentation/0.19.1/additional-resources/ publish/documentation/0.19.1/additional-resources...

Added: aurora/site/source/documentation/0.20.0/operations/configuration.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/configuration.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/configuration.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/configuration.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,339 @@
+# Scheduler Configuration
+
+The Aurora scheduler can take a variety of configuration options through command-line arguments.
+Examples are available under `examples/scheduler/`. For a list of available Aurora flags and their
+documentation, see [Scheduler Configuration Reference](../../reference/scheduler-configuration/).
+
+
+## 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=(
+      # Port for client RPCs and the web UI
+      -http_port=8081
+      # Log configuration, etc.
+    )
+
+    # Environment variables controlling libmesos
+    export JAVA_HOME=...
+    export GLOG_v=1
+    export LIBPROCESS_PORT=8083
+    export LIBPROCESS_IP=192.168.33.7
+
+    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.
+
+
+## JVM Configuration
+
+JVM settings are dependent on your environment and cluster size. They might require
+custom tuning. As a starting point, we recommend:
+
+* Ensure the initial (`-Xms`) and maximum (`-Xmx`) heap size are idential to prevent heap resizing
+  at runtime.
+* Either `-XX:+UseConcMarkSweepGC` or `-XX:+UseG1GC -XX:+UseStringDeduplication` are
+  sane defaults for the garbage collector.
+* `-Djava.net.preferIPv4Stack=true` makes sense in most cases as well.
+
+
+## Network Configuration
+
+By default, Aurora binds to all interfaces and auto-discovers its hostname. To reduce ambiguity
+it helps to hardcode them though:
+
+    -http_port=8081
+    -ip=192.168.33.7
+    -hostname="aurora1.us-east1.example.org"
+
+Two environment variables control the ip and port for the communication with the Mesos master
+and for the replicated log used by Aurora:
+
+    export LIBPROCESS_PORT=8083
+    export LIBPROCESS_IP=192.168.33.7
+
+It is important that those can be reached from all Mesos master and Aurora scheduler instances.
+
+
+## Replicated Log Configuration
+
+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.
+
+Below is a summary of scheduler storage configuration flags that either don't have default values
+or require attention before deploying in a production environment.
+
+### `-native_log_quorum_size`
+Defines the Mesos replicated log quorum size. 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`.
+
+  Number of schedulers (N) | ```-native_log_quorum_size``` setting (```floor(N/2) + 1```)
+  ------------------------ | -------------------------------------------------------------
+  1                        | 1
+  3                        | 2
+  5                        | 3
+  7                        | 4
+
+*Incorrectly setting this flag will cause data corruption to occur!*
+
+### `-native_log_file_path`
+Location of the Mesos replicated log files. For optimal and consistent performance, consider
+allocating a dedicated disk (preferably SSD) for the replicated log. Ensure that this disk is not
+used by anything else (e.g. no process logging) and in particular that it is a real disk
+and not just a partition.
+
+Even when a dedicated disk is used, switching from `CFQ` to `deadline` I/O scheduler of Linux kernel
+can furthermore help with storage performance in Aurora ([see this ticket for details](https://issues.apache.org/jira/browse/AURORA-1211)).
+
+### `-native_log_zk_group_path`
+ZooKeeper path used for Mesos replicated log quorum discovery.
+
+See [code](https://github.com/apache/aurora/blob/rel/0.20.0/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java) for
+other available Mesos replicated log configuration options and default values.
+
+### Changing the Quorum Size
+Special care needs to be taken when changing the size of the Aurora scheduler quorum.
+Since Aurora uses a Mesos replicated log, similar steps need to be followed as when
+[changing the Mesos quorum size](http://mesos.apache.org/documentation/latest/operational-guide).
+
+As a preparation, increase `-native_log_quorum_size` on each existing scheduler and restart them.
+When updating from 3 to 5 schedulers, the quorum size would grow from 2 to 3.
+
+When starting the new schedulers, use the `-native_log_quorum_size` set to the new value. Failing to
+first increase the quorum size on running schedulers can in some cases result in corruption
+or truncating of the replicated log used by Aurora. In that case, see the documentation on
+[recovering from backup](../backup-restore/).
+
+
+## Backup Configuration
+
+Configuration options for the Aurora scheduler backup manager.
+
+* `-backup_interval`: The interval on which the scheduler writes local storage backups.
+   The default is every hour.
+* `-backup_dir`: Directory to write backups to. As stated above, this should not be co-located on the
+   same disk as the replicated log.
+* `-max_saved_backups`: Maximum number of backups to retain before deleting the oldest backup(s).
+
+
+## Resource Isolation
+
+For proper CPU, memory, and disk isolation as mentioned in our [enduser documentation](../../features/resource-isolation/),
+we recommend to add the following isolators to the `--isolation` flag of the Mesos agent:
+
+* `cgroups/cpu`
+* `cgroups/mem`
+* `disk/du`
+
+In addition, we recommend to set the following [agent flags](http://mesos.apache.org/documentation/latest/configuration/):
+
+* `--cgroups_limit_swap` to enable memory limits on both memory and swap instead of just memory.
+  Alternatively, you could disable swap on your agent hosts.
+* `--cgroups_enable_cfs` to enable hard limits on CPU resources via the CFS bandwidth limiting
+  feature.
+* `--enforce_container_disk_quota` to enable disk quota enforcement for containers.
+
+To enable the optional GPU support in Mesos, please see the GPU related flags in the
+[Mesos configuration](http://mesos.apache.org/documentation/latest/configuration/).
+To enable the corresponding feature in Aurora, you have to start the scheduler with the
+flag
+
+    -allow_gpu_resource=true
+
+If you want to use revocable resources, first follow the
+[Mesos oversubscription documentation](http://mesos.apache.org/documentation/latest/oversubscription/)
+and then set set this Aurora scheduler flag to allow receiving revocable Mesos offers:
+
+    -receive_revocable_resources=true
+
+Both CPUs and RAM are supported as revocable resources. The former is enabled by the default,
+the latter needs to be enabled via:
+
+    -enable_revocable_ram=true
+
+Unless you want to use the [default](https://github.com/apache/aurora/blob/rel/0.20.0/src/main/resources/org/apache/aurora/scheduler/tiers.json)
+tier configuration, you will also have to specify a file path:
+
+    -tier_config=path/to/tiers/config.json
+
+
+## Multi-Framework Setup
+
+Aurora holds onto Mesos offers in order to provide efficient scheduling and
+[preemption](../../features/multitenancy/#preemption). This is problematic in multi-framework
+environments as Aurora might starve other frameworks.
+
+With a downside of increased scheduling latency, Aurora can be configured to be more cooperative:
+
+* Lowering `-min_offer_hold_time` (e.g. to `1mins`) can ensure unused offers are returned back to
+  Mesos more frequently.
+* Increasing `-offer_filter_duration` (e.g to `30secs`) will instruct Mesos
+  not to re-offer rejected resources for the given duration.
+
+Setting a [minimum amount of resources](http://mesos.apache.org/documentation/latest/quota/) for
+each Mesos role can furthermore help to ensure no framework is starved entirely.
+
+
+## Containers
+
+Both the Mesos and Docker containerizers require configuration of the Mesos agent.
+
+### Mesos Containerizer
+
+The minimal agent configuration requires to enable Docker and Appc image support for the Mesos
+containerizer:
+
+    --containerizers=mesos
+    --image_providers=appc,docker
+    --isolation=filesystem/linux,docker/runtime  # as an addition to your other isolators
+
+Further details can be found in the corresponding [Mesos documentation](http://mesos.apache.org/documentation/latest/container-image/).
+
+### Docker Containerizer
+
+The [Docker containerizer](http://mesos.apache.org/documentation/latest/docker-containerizer/)
+requires the Docker engine is installed on each agent host. In addition, it  must be enabled on the
+Mesos agents by launching them with the option:
+
+    --containerizers=mesos,docker
+
+If you would like to run a container with a read-only filesystem, it may also be necessary to use
+the scheduler flag `-thermos_home_in_sandbox` in order to set HOME to the sandbox
+before the executor runs. This will make sure that the executor/runner PEX extractions happens
+inside of the sandbox instead of the container filesystem root.
+
+If you would like to supply your own parameters to `docker run` when launching jobs in docker
+containers, you may use the following flags:
+
+    -allow_docker_parameters
+    -default_docker_parameters
+
+`-allow_docker_parameters` controls whether or not users may pass their own configuration parameters
+through the job configuration files. If set to `false` (the default), the scheduler will reject
+jobs with custom parameters. *NOTE*: this setting should be used with caution as it allows any job
+owner to specify any parameters they wish, including those that may introduce security concerns
+(`privileged=true`, for example).
+
+`-default_docker_parameters` allows a cluster operator to specify a universal set of parameters that
+should be used for every container that does not have parameters explicitly configured at the job
+level. The argument accepts a multimap format:
+
+    -default_docker_parameters="read-only=true,tmpfs=/tmp,tmpfs=/run"
+
+### Common Options
+
+The following Aurora options work for both containerizers.
+
+A scheduler flag, `-global_container_mounts` allows mounting paths from the host (i.e the agent machine)
+into all containers on that host. The format is a comma separated list of host_path:container_path[:mode]
+tuples. For example `-global_container_mounts=/opt/secret_keys_dir:/mnt/secret_keys_dir:ro` mounts
+`/opt/secret_keys_dir` from the agents into all launched containers. Valid modes are `ro` and `rw`.
+
+
+## Thermos Process Logs
+
+### Log destination
+By default, Thermos will write process stdout/stderr to log files in the sandbox. Process object
+configuration allows specifying alternate log file destinations like streamed stdout/stderr or
+suppression of all log output. Default behavior can be configured for the entire cluster with the
+following flag (through the `-thermos_executor_flags` argument to the Aurora scheduler):
+
+    --runner-logger-destination=both
+
+`both` configuration will send logs to files and stream to parent stdout/stderr outputs.
+
+See [Configuration Reference](../../reference/configuration/#logger) for all destination options.
+
+### Log rotation
+By default, Thermos will not rotate the stdout/stderr logs from child processes and they will grow
+without bound. An individual user may change this behavior via configuration on the Process object,
+but it may also be desirable to change the default configuration for the entire cluster.
+In order to enable rotation by default, the following flags can be applied to Thermos (through the
+`-thermos_executor_flags` argument to the Aurora scheduler):
+
+    --runner-logger-mode=rotate
+    --runner-rotate-log-size-mb=100
+    --runner-rotate-log-backups=10
+
+In the above example, each instance of the Thermos runner will rotate stderr/stdout logs once they
+reach 100 MiB in size and keep a maximum of 10 backups. If a user has provided a custom setting for
+their process, it will override these default settings.
+
+
+## Thermos Executor Wrapper
+
+If you need to do computation before starting the Thermos executor (for example, setting a different
+`--announcer-hostname` parameter for every executor), then the Thermos executor should be invoked
+inside a wrapper script. In such a case, the aurora scheduler should be started with
+`-thermos_executor_path` pointing to the wrapper script and `-thermos_executor_resources` set to a
+comma separated string of all the resources that should be copied into the sandbox (including the
+original Thermos executor). Ensure the wrapper script does not access resources outside of the
+sandbox, as when the script is run from within a Docker container those resources may not exist.
+
+For example, to wrap the executor inside a simple wrapper, the scheduler will be started like this
+`-thermos_executor_path=/path/to/wrapper.sh -thermos_executor_resources=/usr/share/aurora/bin/thermos_executor.pex`
+
+## Custom Executors
+
+The scheduler can be configured to utilize a custom executor by specifying the `-custom_executor_config` flag.
+The flag must be set to the path of a valid executor configuration file.
+
+For more information on this feature please see the custom executors [documentation](../../features/custom-executors/).
+
+## A note on increasing executor overhead
+
+Increasing executor overhead on an existing cluster, whether it be for custom executors or for Thermos,
+will result in degraded preemption performance until all task which began life with the previous
+executor configuration with less overhead are preempted/restarted.
+
+## Controlling MTTA via Update Affinity
+
+When there is high resource contention in your cluster you may experience noticably elevated job update
+times, as well as high task churn across the cluster. This is due to Aurora's first-fit scheduling
+algorithm. To alleviate this, you can enable update affinity where the Scheduler will make a best-effort
+attempt to reuse the same agent for the updated task (so long as the resources for the job are not being
+increased).
+
+To enable this in the Scheduler, you can set the following options:
+
+    --enable_update_affinity=true
+    --update_affinity_reservation_hold_time=3mins
+
+You will need to tune the hold time to match the behavior you see in your cluster. If you have extremely
+high update throughput, you might have to extend it as processing updates could easily add significant
+delays between scheduling attempts. You may also have to tune scheduling parameters to achieve the
+throughput you need in your cluster. Some relevant settings (with defaults) are:
+
+    --max_schedule_attempts_per_sec=40
+    --initial_schedule_penalty=1secs
+    --max_schedule_penalty=1mins
+    --scheduling_max_batch_size=3
+    --max_tasks_per_schedule_attempt=5
+
+There are metrics exposed by the Scheduler which can provide guidance on where the bottleneck is.
+Example metrics to look at:
+
+    - schedule_attempts_blocks (if this number is greater than 0, then task throughput is hitting
+                                limits controlled by --max_scheduler_attempts_per_sec)
+    - scheduled_task_penalty_* (metrics around scheduling penalties for tasks, if the numbers here are high
+                                then you could have high contention for resources)
+
+Most likely you'll run into limits with the number of update instances that can be processed per minute
+before you run into any other limits. So if your total work done per minute starts to exceed 2k instances,
+you may need to extend the update_affinity_reservation_hold_time.

Added: aurora/site/source/documentation/0.20.0/operations/installation.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/installation.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/installation.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/installation.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,256 @@
+# Installing Aurora
+
+Source and binary distributions can be found on our
+[downloads](https://aurora.apache.org/downloads/) page.  Installing from binary packages is
+recommended for most.
+
+- [Installing the scheduler](#installing-the-scheduler)
+- [Installing worker components](#installing-worker-components)
+- [Installing the client](#installing-the-client)
+- [Installing Mesos](#installing-mesos)
+- [Troubleshooting](#troubleshooting)
+
+If our binay packages don't suite you, our package build toolchain makes it easy to build your
+own packages. See the [instructions](https://github.com/apache/aurora-packaging) to learn how.
+
+
+## Machine profiles
+
+Given that many of these components communicate over the network, there are numerous ways you could
+assemble them to create an Aurora cluster.  The simplest way is to think in terms of three machine
+profiles:
+
+### Coordinator
+**Components**: ZooKeeper, Aurora scheduler, Mesos master
+
+A small number of machines (typically 3 or 5) responsible for cluster orchestration.  In most cases
+it is fine to co-locate these components in anything but very large clusters (> 1000 machines).
+Beyond that point, operators will likely want to manage these services on separate machines.
+In particular, you will want to use separate ZooKeeper ensembles for leader election and
+service discovery. Otherwise a service discovery error or outage can take down the entire cluster.
+
+In practice, 5 coordinators have been shown to reliably manage clusters with tens of thousands of
+machines.
+
+### Worker
+**Components**: Aurora executor, Aurora observer, Mesos agent
+
+The bulk of the cluster, where services will actually run.
+
+### Client
+**Components**: Aurora client, Aurora admin client
+
+Any machines that users submit jobs from.
+
+
+## Installing the scheduler
+### Ubuntu Trusty
+
+1. Install Mesos
+   Skip down to [install mesos](#mesos-on-ubuntu-trusty), then run:
+
+        sudo start mesos-master
+
+2. Install ZooKeeper
+
+        sudo apt-get install -y zookeeperd
+
+3. Install the Aurora scheduler
+
+        sudo add-apt-repository -y ppa:openjdk-r/ppa
+        sudo apt-get update
+        sudo apt-get install -y openjdk-8-jre-headless wget
+
+        sudo update-alternatives --set java /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
+
+        wget -c https://apache.bintray.com/aurora/ubuntu-trusty/aurora-scheduler_0.17.0_amd64.deb
+        sudo dpkg -i aurora-scheduler_0.17.0_amd64.deb
+
+### CentOS 7
+
+1. Install Mesos
+   Skip down to [install mesos](#mesos-on-centos-7), then run:
+
+        sudo systemctl start mesos-master
+
+2. Install ZooKeeper
+
+        sudo rpm -Uvh https://archive.cloudera.com/cdh4/one-click-install/redhat/6/x86_64/cloudera-cdh-4-0.x86_64.rpm
+        sudo yum install -y java-1.8.0-openjdk-headless zookeeper-server
+
+        sudo service zookeeper-server init
+        sudo systemctl start zookeeper-server
+
+3. Install the Aurora scheduler
+
+        sudo yum install -y wget
+
+        wget -c https://apache.bintray.com/aurora/centos-7/aurora-scheduler-0.17.0-1.el7.centos.aurora.x86_64.rpm
+        sudo yum install -y aurora-scheduler-0.17.0-1.el7.centos.aurora.x86_64.rpm
+
+### Finalizing
+By default, the scheduler will start in an uninitialized mode.  This is because external
+coordination is necessary to be certain operator error does not result in a quorum of schedulers
+starting up and believing their databases are empty when in fact they should be re-joining a
+cluster.
+
+Because of this, a fresh install of the scheduler will need intervention to start up.  First,
+stop the scheduler service.
+Ubuntu: `sudo stop aurora-scheduler`
+CentOS: `sudo systemctl stop aurora`
+
+Now initialize the database:
+
+    sudo -u aurora mkdir -p /var/lib/aurora/scheduler/db
+    sudo -u aurora mesos-log initialize --path=/var/lib/aurora/scheduler/db
+
+Now you can start the scheduler back up.
+Ubuntu: `sudo start aurora-scheduler`
+CentOS: `sudo systemctl start aurora`
+
+
+## Installing worker components
+### Ubuntu Trusty
+
+1. Install Mesos
+   Skip down to [install mesos](#mesos-on-ubuntu-trusty), then run:
+
+        start mesos-slave
+
+2. Install Aurora executor and observer
+
+        sudo apt-get install -y python2.7 wget
+
+        # NOTE: This appears to be a missing dependency of the mesos deb package and is needed
+        # for the python mesos native bindings.
+        sudo apt-get -y install libcurl4-nss-dev
+
+        wget -c https://apache.bintray.com/aurora/ubuntu-trusty/aurora-executor_0.17.0_amd64.deb
+        sudo dpkg -i aurora-executor_0.17.0_amd64.deb
+
+### CentOS 7
+
+1. Install Mesos
+   Skip down to [install mesos](#mesos-on-centos-7), then run:
+
+        sudo systemctl start mesos-slave
+
+2. Install Aurora executor and observer
+
+        sudo yum install -y python2 wget
+
+        wget -c https://apache.bintray.com/aurora/centos-7/aurora-executor-0.17.0-1.el7.centos.aurora.x86_64.rpm
+        sudo yum install -y aurora-executor-0.17.0-1.el7.centos.aurora.x86_64.rpm
+
+### Worker Configuration
+The executor typically does not require configuration.  Command line arguments can
+be passed to the executor using a command line argument on the scheduler.
+
+The observer needs to be configured to look at the correct mesos directory in order to find task
+sandboxes. You should 1st find the Mesos working directory by looking for the Mesos agent
+`--work_dir` flag. You should see something like:
+
+        ps -eocmd | grep "mesos-slave" | grep -v grep | tr ' ' '\n' | grep "\--work_dir"
+        --work_dir=/var/lib/mesos
+
+If the flag is not set, you can view the default value like so:
+
+        mesos-slave --help
+        Usage: mesos-slave [options]
+
+          ...
+          --work_dir=VALUE      Directory path to place framework work directories
+                                (default: /tmp/mesos)
+          ...
+
+The value you find for `--work_dir`, `/var/lib/mesos` in this example, should match the Aurora
+observer value for `--mesos-root`.  You can look for that setting in a similar way on a worker
+node by grepping for `thermos_observer` and `--mesos-root`.  If the flag is not set, you can view
+the default value like so:
+
+        thermos_observer -h
+        Options:
+          ...
+          --mesos-root=MESOS_ROOT
+                                The mesos root directory to search for Thermos
+                                executor sandboxes [default: /var/lib/mesos]
+          ...
+
+In this case the default is `/var/lib/mesos` and we have a match. If there is no match, you can
+either adjust the mesos-master start script(s) and restart the master(s) or else adjust the
+Aurora observer start scripts and restart the observers.  To adjust the Aurora observer:
+
+#### Ubuntu Trusty
+
+    sudo sh -c 'echo "MESOS_ROOT=/tmp/mesos" >> /etc/default/thermos'
+
+#### CentOS 7
+
+Make an edit to add the `--mesos-root` flag resulting in something like:
+
+    grep -A5 OBSERVER_ARGS /etc/sysconfig/thermos
+    OBSERVER_ARGS=(
+      --port=1338
+      --mesos-root=/tmp/mesos
+      --log_to_disk=NONE
+      --log_to_stderr=google:INFO
+    )
+
+
+## Installing the client
+### Ubuntu Trusty
+
+    sudo apt-get install -y python2.7 wget
+
+    wget -c https://apache.bintray.com/aurora/ubuntu-trusty/aurora-tools_0.17.0_amd64.deb
+    sudo dpkg -i aurora-tools_0.17.0_amd64.deb
+
+### CentOS 7
+
+    sudo yum install -y python2 wget
+
+    wget -c https://apache.bintray.com/aurora/centos-7/aurora-tools-0.17.0-1.el7.centos.aurora.x86_64.rpm
+    sudo yum install -y aurora-tools-0.17.0-1.el7.centos.aurora.x86_64.rpm
+
+### Mac OS X
+
+    brew upgrade
+    brew install aurora-cli
+
+### Client Configuration
+Client configuration lives in a json file that describes the clusters available and how to reach
+them.  By default this file is at `/etc/aurora/clusters.json`.
+
+Jobs may be submitted to the scheduler using the client, and are described with
+[job configurations](../../reference/configuration/) expressed in `.aurora` files.  Typically you will
+maintain a single job configuration file to describe one or more deployment environments (e.g.
+dev, test, prod) for a production job.
+
+
+## Installing Mesos
+Mesos uses a single package for the Mesos master and agent.  As a result, the package dependencies
+are identical for both.
+
+### Mesos on Ubuntu Trusty
+
+    sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E56151BF
+    DISTRO=$(lsb_release -is | tr '[:upper:]' '[:lower:]')
+    CODENAME=$(lsb_release -cs)
+
+    echo "deb http://repos.mesosphere.io/${DISTRO} ${CODENAME} main" | \
+      sudo tee /etc/apt/sources.list.d/mesosphere.list
+    sudo apt-get -y update
+
+    # Use `apt-cache showpkg mesos | grep [version]` to find the exact version.
+    sudo apt-get -y install mesos=1.1.0-2.0.107.ubuntu1404_amd64.deb
+
+### Mesos on CentOS 7
+
+    sudo rpm -Uvh https://repos.mesosphere.io/el/7/noarch/RPMS/mesosphere-el-repo-7-1.noarch.rpm
+    sudo yum -y install mesos-1.1.0
+
+
+## Troubleshooting
+
+So you've started your first cluster and are running into some issues? We've collected some common
+stumbling blocks and solutions in our [Troubleshooting guide](../troubleshooting/) to help get you moving.

Added: aurora/site/source/documentation/0.20.0/operations/monitoring.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/monitoring.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/monitoring.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/monitoring.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,181 @@
+# Monitoring your Aurora cluster
+
+Before you start running important services in your Aurora cluster, it's important to set up
+monitoring and alerting of Aurora itself.  Most of your monitoring can be against the scheduler,
+since it will give you a global view of what's going on.
+
+## Reading stats
+The scheduler exposes a *lot* of instrumentation data via its HTTP interface. You can get a quick
+peek at the first few of these in our vagrant image:
+
+    $ vagrant ssh -c 'curl -s localhost:8081/vars | head'
+    async_tasks_completed 1004
+    attribute_store_fetch_all_events 15
+    attribute_store_fetch_all_events_per_sec 0.0
+    attribute_store_fetch_all_nanos_per_event 0.0
+    attribute_store_fetch_all_nanos_total 3048285
+    attribute_store_fetch_all_nanos_total_per_sec 0.0
+    attribute_store_fetch_one_events 3391
+    attribute_store_fetch_one_events_per_sec 0.0
+    attribute_store_fetch_one_nanos_per_event 0.0
+    attribute_store_fetch_one_nanos_total 454690753
+
+These values are served as `Content-Type: text/plain`, with each line containing a space-separated metric
+name and value. Values may be integers, doubles, or strings (note: strings are static, others
+may be dynamic).
+
+If your monitoring infrastructure prefers JSON, the scheduler exports that as well:
+
+    $ vagrant ssh -c 'curl -s localhost:8081/vars.json | python -mjson.tool | head'
+    {
+        "async_tasks_completed": 1009,
+        "attribute_store_fetch_all_events": 15,
+        "attribute_store_fetch_all_events_per_sec": 0.0,
+        "attribute_store_fetch_all_nanos_per_event": 0.0,
+        "attribute_store_fetch_all_nanos_total": 3048285,
+        "attribute_store_fetch_all_nanos_total_per_sec": 0.0,
+        "attribute_store_fetch_one_events": 3409,
+        "attribute_store_fetch_one_events_per_sec": 0.0,
+        "attribute_store_fetch_one_nanos_per_event": 0.0,
+
+This will be the same data as above, served with `Content-Type: application/json`.
+
+## Viewing live stat samples on the scheduler
+The scheduler uses the Twitter commons stats library, which keeps an internal time-series database
+of exported variables - nearly everything in `/vars` is available for instant graphing.  This is
+useful for debugging, but is not a replacement for an external monitoring system.
+
+You can view these graphs on a scheduler at `/graphview`.  It supports some composition and
+aggregation of values, which can be invaluable when triaging a problem.  For example, if you have
+the scheduler running in vagrant, check out these links:
+[simple graph](http://192.168.33.7:8081/graphview?query=jvm_uptime_secs)
+[complex composition](http://192.168.33.7:8081/graphview?query=rate\(scheduler_log_native_append_nanos_total\)%2Frate\(scheduler_log_native_append_events\)%2F1e6)
+
+### Counters and gauges
+Among numeric stats, there are two fundamental types of stats exported: _counters_ and _gauges_.
+Counters are guaranteed to be monotonically-increasing for the lifetime of a process, while gauges
+may decrease in value.  Aurora uses counters to represent things like the number of times an event
+has occurred, and gauges to capture things like the current length of a queue.  Counters are a
+natural fit for accurate composition into [rate ratios](http://en.wikipedia.org/wiki/Rate_ratio)
+(useful for sample-resistant latency calculation), while gauges are not.
+
+# Alerting
+
+## Quickstart
+If you are looking for just bare-minimum alerting to get something in place quickly, set up alerting
+on `framework_registered` and `task_store_LOST`. These will give you a decent picture of overall
+health.
+
+## A note on thresholds
+One of the most difficult things in monitoring is choosing alert thresholds. With many of these
+stats, there is no value we can offer as a threshold that will be guaranteed to work for you. It
+will depend on the size of your cluster, number of jobs, churn of tasks in the cluster, etc. We
+recommend you start with a strict value after viewing a small amount of collected data, and then
+adjust thresholds as you see fit. Feel free to ask us if you would like to validate that your alerts
+and thresholds make sense.
+
+## Important stats
+
+### `jvm_uptime_secs`
+Type: integer counter
+
+The number of seconds the JVM process has been running. Comes from
+[RuntimeMXBean#getUptime()](http://docs.oracle.com/javase/7/docs/api/java/lang/management/RuntimeMXBean.html#getUptime\(\))
+
+Detecting resets (decreasing values) on this stat will tell you that the scheduler is failing to
+stay alive.
+
+Look at the scheduler logs to identify the reason the scheduler is exiting.
+
+### `system_load_avg`
+Type: double gauge
+
+The current load average of the system for the last minute. Comes from
+[OperatingSystemMXBean#getSystemLoadAverage()](http://docs.oracle.com/javase/7/docs/api/java/lang/management/OperatingSystemMXBean.html?is-external=true#getSystemLoadAverage\(\)).
+
+A high sustained value suggests that the scheduler machine may be over-utilized.
+
+Use standard unix tools like `top` and `ps` to track down the offending process(es).
+
+### `process_cpu_cores_utilized`
+Type: double gauge
+
+The current number of CPU cores in use by the JVM process. This should not exceed the number of
+logical CPU cores on the machine. Derived from
+[OperatingSystemMXBean#getProcessCpuTime()](http://docs.oracle.com/javase/7/docs/jre/api/management/extension/com/sun/management/OperatingSystemMXBean.html)
+
+A high sustained value indicates that the scheduler is overworked. Due to current internal design
+limitations, if this value is sustained at `1`, there is a good chance the scheduler is under water.
+
+There are two main inputs that tend to drive this figure: task scheduling attempts and status
+updates from Mesos.  You may see activity in the scheduler logs to give an indication of where
+time is being spent.  Beyond that, it really takes good familiarity with the code to effectively
+triage this.  We suggest engaging with an Aurora developer.
+
+### `task_store_LOST`
+Type: integer gauge
+
+The number of tasks stored in the scheduler that are in the `LOST` state, and have been rescheduled.
+
+If this value is increasing at a high rate, it is a sign of trouble.
+
+There are many sources of `LOST` tasks in Mesos: the scheduler, master, agent, and executor can all
+trigger this.  The first step is to look in the scheduler logs for `LOST` to identify where the
+state changes are originating.
+
+### `scheduler_resource_offers`
+Type: integer counter
+
+The number of resource offers that the scheduler has received.
+
+For a healthy scheduler, this value must be increasing over time.
+
+Assuming the scheduler is up and otherwise healthy, you will want to check if the master thinks it
+is sending offers. You should also look at the master's web interface to see if it has a large
+number of outstanding offers that it is waiting to be returned.
+
+### `framework_registered`
+Type: binary integer counter
+
+Will be `1` for the leading scheduler that is registered with the Mesos master, `0` for passive
+schedulers,
+
+A sustained period without a `1` (or where `sum() != 1`) warrants investigation.
+
+If there is no leading scheduler, look in the scheduler and master logs for why.  If there are
+multiple schedulers claiming leadership, this suggests a split brain and warrants filing a critical
+bug.
+
+### `rate(scheduler_log_native_append_nanos_total)/rate(scheduler_log_native_append_events)`
+Type: rate ratio of integer counters
+
+This composes two counters to compute a windowed figure for the latency of replicated log writes.
+
+A hike in this value suggests disk bandwidth contention.
+
+Look in scheduler logs for any reported oddness with saving to the replicated log. Also use
+standard tools like `vmstat` and `iotop` to identify whether the disk has become slow or
+over-utilized. We suggest using a dedicated disk for the replicated log to mitigate this.
+
+### `timed_out_tasks`
+Type: integer counter
+
+Tracks the number of times the scheduler has given up while waiting
+(for `-transient_task_state_timeout`) to hear back about a task that is in a transient state
+(e.g. `ASSIGNED`, `KILLING`), and has moved to `LOST` before rescheduling.
+
+This value is currently known to increase occasionally when the scheduler fails over
+([AURORA-740](https://issues.apache.org/jira/browse/AURORA-740)). However, any large spike in this
+value warrants investigation.
+
+The scheduler will log when it times out a task. You should trace the task ID of the timed out
+task into the master, agent, and/or executors to determine where the message was dropped.
+
+### `http_500_responses_events`
+Type: integer counter
+
+The total number of HTTP 500 status responses sent by the scheduler. Includes API and asset serving.
+
+An increase warrants investigation.
+
+Look in scheduler logs to identify why the scheduler returned a 500, there should be a stack trace.

Added: aurora/site/source/documentation/0.20.0/operations/security.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/security.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/security.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/security.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,362 @@
+Securing your Aurora Cluster
+============================
+
+Aurora integrates with [Apache Shiro](http://shiro.apache.org/) to provide security
+controls for its API. In addition to providing some useful features out of the box, Shiro
+also allows Aurora cluster administrators to adapt the security system to their organization’s
+existing infrastructure. The announcer in the Aurora thermos executor also supports security
+controls for talking to ZooKeeper.
+
+
+- [Enabling Security](#enabling-security)
+- [Authentication](#authentication)
+	- [HTTP Basic Authentication](#http-basic-authentication)
+		- [Server Configuration](#server-configuration)
+		- [Client Configuration](#client-configuration)
+	- [HTTP SPNEGO Authentication (Kerberos)](#http-spnego-authentication-kerberos)
+		- [Server Configuration](#server-configuration-1)
+		- [Client Configuration](#client-configuration-1)
+- [Authorization](#authorization)
+	- [Using an INI file to define security controls](#using-an-ini-file-to-define-security-controls)
+		- [Caveats](#caveats)
+- [Implementing a Custom Realm](#implementing-a-custom-realm)
+	- [Packaging a realm module](#packaging-a-realm-module)
+- [Announcer Authentication](#announcer-authentication)
+    - [ZooKeeper authentication configuration](#zookeeper-authentication-configuration)
+    - [Executor settings](#executor-settings)
+- [Scheduler HTTPS](#scheduler-https)
+- [Known Issues](#known-issues)
+
+# Enabling Security
+
+There are two major components of security:
+[authentication and authorization](http://en.wikipedia.org/wiki/Authentication#Authorization).  A
+cluster administrator may choose the approach used for each, and may also implement custom
+mechanisms for either.  Later sections describe the options available. To enable authentication
+ for the announcer, see [Announcer Authentication](#announcer-authentication)
+
+
+# Authentication
+
+The scheduler must be configured with instructions for how to process authentication
+credentials at a minimum.  There are currently two built-in authentication schemes -
+[HTTP Basic Authentication](http://en.wikipedia.org/wiki/Basic_access_authentication), and
+[SPNEGO](http://en.wikipedia.org/wiki/SPNEGO) (Kerberos).
+
+## HTTP Basic Authentication
+
+Basic Authentication is a very quick way to add *some* security.  It is supported
+by all major browsers and HTTP client libraries with minimal work.  However,
+before relying on Basic Authentication you should be aware of the [security
+considerations](http://tools.ietf.org/html/rfc2617#section-4).
+
+### Server Configuration
+
+At a minimum you need to set 4 command-line flags on the scheduler:
+
+```
+-http_authentication_mechanism=BASIC
+-shiro_realm_modules=INI_AUTHNZ
+-shiro_ini_path=path/to/security.ini
+```
+
+And create a security.ini file like so:
+
+```
+[users]
+sally = apple, admin
+
+[roles]
+admin = *
+```
+
+The details of the security.ini file are explained below. Note that this file contains plaintext,
+unhashed passwords.
+
+### Client Configuration
+
+To configure the client for HTTP Basic authentication, add an entry to ~/.netrc with your credentials
+
+```
+% cat ~/.netrc
+# ...
+
+machine aurora.example.com
+login sally
+password apple
+
+# ...
+```
+
+No changes are required to `clusters.json`.
+
+## HTTP SPNEGO Authentication (Kerberos)
+
+### Server Configuration
+At a minimum you need to set 6 command-line flags on the scheduler:
+
+```
+-http_authentication_mechanism=NEGOTIATE
+-shiro_realm_modules=KERBEROS5_AUTHN,INI_AUTHNZ
+-kerberos_server_principal=HTTP/aurora.example.com@EXAMPLE.COM
+-kerberos_server_keytab=path/to/aurora.example.com.keytab
+-shiro_ini_path=path/to/security.ini
+```
+
+And create a security.ini file like so:
+
+```
+% cat path/to/security.ini
+[users]
+sally = _, admin
+
+[roles]
+admin = *
+```
+
+What's going on here? First, Aurora must be configured to request Kerberos credentials when presented with an
+unauthenticated request. This is achieved by setting
+
+```
+-http_authentication_mechanism=NEGOTIATE
+```
+
+Next, a Realm module must be configured to **authenticate** the current request using the Kerberos
+credentials that were requested. Aurora ships with a realm module that can do this
+
+```
+-shiro_realm_modules=KERBEROS5_AUTHN[,...]
+```
+
+The Kerberos5Realm requires a keytab file and a server principal name. The principal name will usually
+be in the form `HTTP/aurora.example.com@EXAMPLE.COM`.
+
+```
+-kerberos_server_principal=HTTP/aurora.example.com@EXAMPLE.COM
+-kerberos_server_keytab=path/to/aurora.example.com.keytab
+```
+
+The Kerberos5 realm module is authentication-only. For scheduler security to work you must also
+enable a realm module that provides an Authorizer implementation. For example, to do this using the
+IniShiroRealmModule:
+
+```
+-shiro_realm_modules=KERBEROS5_AUTHN,INI_AUTHNZ
+```
+
+You can then configure authorization using a security.ini file as described below
+(the password field is ignored). You must configure the realm module with the path to this file:
+
+```
+-shiro_ini_path=path/to/security.ini
+```
+
+### Client Configuration
+To use Kerberos on the client-side you must build Kerberos-enabled client binaries. Do this with
+
+```
+./pants binary src/main/python/apache/aurora/kerberos:kaurora
+./pants binary src/main/python/apache/aurora/kerberos:kaurora_admin
+```
+
+You must also configure each cluster where you've enabled Kerberos on the scheduler
+to use Kerberos authentication. Do this by setting `auth_mechanism` to `KERBEROS`
+in `clusters.json`.
+
+```
+% cat ~/.aurora/clusters.json
+{
+    "devcluser": {
+        "auth_mechanism": "KERBEROS",
+        ...
+    },
+    ...
+}
+```
+
+# Authorization
+Given a means to authenticate the entity a client claims they are, we need to define what privileges they have.
+
+## Using an INI file to define security controls
+
+The simplest security configuration for Aurora is an INI file on the scheduler.  For small
+clusters, or clusters where the users and access controls change relatively infrequently, this is
+likely the preferred approach.  However you may want to avoid this approach if access permissions
+are rapidly changing, or if your access control information already exists in another system.
+
+You can enable INI-based configuration with following scheduler command line arguments:
+
+```
+-http_authentication_mechanism=BASIC
+-shiro_ini_path=path/to/security.ini
+```
+
+*note* As the argument name reveals, this is using Shiro’s
+[IniRealm](http://shiro.apache.org/configuration.html#Configuration-INIConfiguration) behind
+the scenes.
+
+The INI file will contain two sections - users and roles.  Here’s an example for what might
+be in security.ini:
+
+```
+[users]
+sally = apple, admin
+jim = 123456, accounting
+becky = letmein, webapp
+larry = 654321,accounting
+steve = password
+
+[roles]
+admin = *
+accounting = thrift.AuroraAdmin:setQuota
+webapp = thrift.AuroraSchedulerManager:*:webapp
+```
+
+The users section defines user user credentials and the role(s) they are members of.  These lines
+are of the format `<user> = <password>[, <role>...]`.  As you probably noticed, the passwords are
+in plaintext and as a result read access to this file should be restricted.
+
+In this configuration, each user has different privileges for actions in the cluster because
+of the roles they are a part of:
+
+* admin is granted all privileges
+* accounting may adjust the amount of resource quota for any role
+* webapp represents a collection of jobs that represents a service, and its members may create and modify any jobs owned by it
+
+### Caveats
+You might find documentation on the Internet suggesting there are additional sections in `shiro.ini`,
+like `[main]` and `[urls]`. These are not supported by Aurora as it uses a different mechanism to configure
+those parts of Shiro. Think of Aurora's `security.ini` as a subset with only `[users]` and `[roles]` sections.
+
+## Implementing Delegated Authorization
+
+It is possible to leverage Shiro's `runAs` feature by implementing a custom Servlet Filter that provides
+the capability and passing it's fully qualified class name to the command line argument
+`-shiro_after_auth_filter`. The filter is registered in the same filter chain as the Shiro auth filters
+and is placed after the Shiro auth filters in the filter chain. This ensures that the Filter is invoked
+after the Shiro filters have had a chance to authenticate the request.
+
+# Implementing a Custom Realm
+
+Since Aurora’s security is backed by [Apache Shiro](https://shiro.apache.org), you can implement a
+custom [Realm](http://shiro.apache.org/realm.html) to define organization-specific security behavior.
+
+In addition to using Shiro's standard APIs to implement a Realm you can link against Aurora to
+access the type-safe Permissions Aurora uses. See the Javadoc for `org.apache.aurora.scheduler.spi`
+for more information.
+
+## Packaging a realm module
+Package your custom Realm(s) with a Guice module that exposes a `Set<Realm>` multibinding.
+
+```java
+package com.example;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.multibindings.Multibinder;
+import org.apache.shiro.realm.Realm;
+
+public class MyRealmModule extends AbstractModule {
+  @Override
+  public void configure() {
+    Realm myRealm = new MyRealm();
+
+    Multibinder.newSetBinder(binder(), Realm.class).addBinding().toInstance(myRealm);
+  }
+
+  static class MyRealm implements Realm {
+    // Realm implementation.
+  }
+}
+```
+
+To use your module in the scheduler, include it as a realm module based on its fully-qualified
+class name:
+
+```
+-shiro_realm_modules=KERBEROS5_AUTHN,INI_AUTHNZ,com.example.MyRealmModule
+```
+
+
+# Announcer Authentication
+The Thermos executor can be configured to authenticate with ZooKeeper and include
+an [ACL](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl)
+on the nodes it creates, which will specify
+the privileges of clients to perform different actions on these nodes.  This
+feature is enabled by specifying an ACL configuration file to the executor with the
+`--announcer-zookeeper-auth-config` command line argument.
+
+When this feature is _not_ enabled, nodes created by the executor will have 'world/all' permission
+(`ZOO_OPEN_ACL_UNSAFE`).  In most production environments, operators should specify an ACL and
+limit access.
+
+## ZooKeeper Authentication Configuration
+The configuration file must be formatted as JSON with the following schema:
+
+```json
+{
+  "auth": [
+    {
+      "scheme": "<scheme>",
+      "credential": "<plain_credential>"
+    }
+  ],
+  "acl": [
+    {
+      "scheme": "<scheme>",
+      "credential": "<plain_credential>",
+      "permissions": {
+        "read": <bool>,
+        "write": <bool>,
+        "create": <bool>,
+        "delete": <bool>,
+        "admin": <bool>
+      }
+    }
+  ]
+}
+```
+
+The `scheme`
+defines the encoding of the credential field.  Note that these fields are passed directly to
+ZooKeeper (except in the case of _digest_ scheme, where the executor will hash and encode
+the credential appropriately before passing it to ZooKeeper). In addition to `acl`, a list of
+authentication credentials must be provided in `auth` to use for the connection.
+
+All properties of the `permissions` object will default to False if not provided.
+
+## Executor settings
+To enable the executor to authenticate against ZK, `--announcer-zookeeper-auth-config` should be
+set to the configuration file.
+
+
+# Scheduler HTTPS
+
+The Aurora scheduler does not provide native HTTPS support ([AURORA-343](https://issues.apache.org/jira/browse/AURORA-343)).
+It is therefore recommended to deploy it behind an HTTPS capable reverse proxy such as nginx or Apache2.
+
+A simple setup is to launch both the reverse proxy and the Aurora scheduler on the same port, but
+bind the reverse proxy to the public IP of the host and the scheduler to localhost:
+
+    -ip=127.0.0.1
+    -http_port=8081
+
+If your clients connect to the scheduler via [`proxy_url`](../../reference/scheduler-configuration/),
+you can update it to `https`. If you use the ZooKeeper based discovery instead, the scheduler
+needs to be launched via
+
+    -serverset_endpoint_name=https
+
+in order to announce its HTTPS support within ZooKeeper.
+
+
+# Known Issues
+
+While the APIs and SPIs we ship with are stable as of 0.8.0, we are aware of several incremental
+improvements. Please follow, vote, or send patches.
+
+Relevant tickets:
+* [AURORA-1248](https://issues.apache.org/jira/browse/AURORA-1248): Client retries 4xx errors
+* [AURORA-1279](https://issues.apache.org/jira/browse/AURORA-1279): Remove kerberos-specific build targets
+* [AURORA-1293](https://issues.apache.org/jira/browse/AURORA-1291): Consider defining a JSON format in place of INI
+* [AURORA-1179](https://issues.apache.org/jira/browse/AURORA-1179): Supported hashed passwords in security.ini
+* [AURORA-1295](https://issues.apache.org/jira/browse/AURORA-1295): Support security for the ReadOnlyScheduler service

Added: aurora/site/source/documentation/0.20.0/operations/storage.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/storage.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/storage.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/storage.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,96 @@
+# Aurora Scheduler Storage
+
+- [Overview](#overview)
+- [Storage Semantics](#storage-semantics)
+  - [Reads, writes, modifications](#reads-writes-modifications)
+    - [Read lifecycle](#read-lifecycle)
+    - [Write lifecycle](#write-lifecycle)
+  - [Atomicity, consistency and isolation](#atomicity-consistency-and-isolation)
+  - [Population on restart](#population-on-restart)
+
+
+## Overview
+
+Aurora scheduler maintains data that need to be persisted to survive failovers and restarts.
+For example:
+
+* Task configurations and scheduled task instances
+* Job update configurations and update progress
+* Production resource quotas
+* Mesos resource offer host attributes
+
+Aurora solves its persistence needs by leveraging the
+[Mesos implementation of a Paxos replicated log](http://mesos.apache.org/documentation/latest/replicated-log-internals/)
+[[1]](https://ramcloud.stanford.edu/~ongaro/userstudy/paxos.pdf)
+[[2]](http://en.wikipedia.org/wiki/State_machine_replication) with a key-value
+[LevelDB](https://github.com/google/leveldb) storage as persistence media.
+
+Conceptually, it can be represented by the following major components:
+
+* Volatile storage: in-memory cache of all available data. Implemented via in-memory
+[H2 Database](http://www.h2database.com/html/main.html) and accessed via
+[MyBatis](http://mybatis.github.io/mybatis-3/).
+* Log manager: interface between Aurora storage and Mesos replicated log. The default schema format
+is [thrift](https://github.com/apache/thrift). Data is stored in serialized binary form.
+* Snapshot manager: all data is periodically persisted in Mesos replicated log in a single snapshot.
+This helps establishing periodic recovery checkpoints and speeds up volatile storage recovery on
+restart.
+* Backup manager: as a precaution, snapshots are periodically written out into backup files.
+This solves a [disaster recovery problem](../backup-restore/)
+in case of a complete loss or corruption of Mesos log files.
+
+![Storage hierarchy](../images/storage_hierarchy.png)
+
+
+## Storage Semantics
+
+Implementation details of the Aurora storage system. Understanding those can sometimes be useful
+when investigating performance issues.
+
+### Reads, writes, modifications
+
+All services in Aurora access data via a set of predefined store interfaces (aka stores) logically
+grouped by the type of data they serve. Every interface defines a specific set of operations allowed
+on the data thus abstracting out the storage access and the actual persistence implementation. The
+latter is especially important in view of a general immutability of persisted data. With the Mesos
+replicated log as the underlying persistence solution, data can be read and written easily but not
+modified. All modifications are simulated by saving new versions of modified objects. This feature
+and general performance considerations justify the existence of the volatile in-memory store.
+
+#### Read lifecycle
+
+There are two types of reads available in Aurora: consistent and weakly-consistent. The difference
+is explained [below](#atomicity-consistency-and-isolation).
+
+All reads are served from the volatile storage making reads generally cheap storage operations
+from the performance standpoint. The majority of the volatile stores are represented by the
+in-memory H2 database. This allows for rich schema definitions, queries and relationships that
+key-value storage is unable to match.
+
+#### Write lifecycle
+
+Writes are more involved operations since in addition to updating the volatile store data has to be
+appended to the replicated log. Data is not available for reads until fully ack-ed by both
+replicated log and volatile storage.
+
+### Atomicity, consistency and isolation
+
+Aurora uses [write-ahead logging](http://en.wikipedia.org/wiki/Write-ahead_logging) to ensure
+consistency between replicated and volatile storage. In Aurora, data is first written into the
+replicated log and only then updated in the volatile store.
+
+Aurora storage uses read-write locks to serialize data mutations and provide consistent view of the
+available data. The available `Storage` interface exposes 3 major types of operations:
+* `consistentRead` - access is locked using reader's lock and provides consistent view on read
+* `weaklyConsistentRead` - access is lock-less. Delivers best contention performance but may result
+in stale reads
+* `write` - access is fully serialized by using writer's lock. Operation success requires both
+volatile and replicated writes to succeed.
+
+The consistency of the volatile store is enforced via H2 transactional isolation.
+
+### Population on restart
+
+Any time a scheduler restarts, it restores its volatile state from the most recent position recorded
+in the replicated log by restoring the snapshot and replaying individual log entries on top to fully
+recover the state up to the last write.

Added: aurora/site/source/documentation/0.20.0/operations/troubleshooting.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/troubleshooting.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/troubleshooting.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/troubleshooting.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,106 @@
+# Troubleshooting
+
+So you've started your first cluster and are running into some issues? We've collected some common
+stumbling blocks and solutions here to help get you moving.
+
+## Replicated log not initialized
+
+### Symptoms
+- Scheduler RPCs and web interface claim `Storage is not READY`
+- Scheduler log repeatedly prints messages like
+
+  ```
+  I1016 16:12:27.234133 26081 replica.cpp:638] Replica in EMPTY status
+  received a broadcasted recover request
+  I1016 16:12:27.234256 26084 recover.cpp:188] Received a recover response
+  from a replica in EMPTY status
+  ```
+
+### Solution
+When you create a new cluster, you need to inform a quorum of schedulers that they are safe to
+consider their database to be empty by [initializing](../installation/#finalizing) the
+replicated log. This is done to prevent the scheduler from modifying the cluster state in the event
+of multiple simultaneous disk failures or, more likely, misconfiguration of the replicated log path.
+
+
+## No distinct leader elected
+
+### Symptoms
+Either no scheduler or multiple scheduler believe to be leading.
+
+### Solution
+Verify the [network configuration](../configuration/#network-configuration) of the Aurora
+scheduler is correct:
+
+* The `LIBPROCESS_IP:LIBPROCESS_PORT` endpoints must be reachable from all coordinator nodes running
+  a scheduler or a Mesos master.
+* Hostname lookups have to resolve to public ips rather than local ones that cannot be reached
+  from another node.
+
+In addition, double-check the [quota settings](../configuration/#replicated-log-configuration) of the
+replicated log.
+
+
+## Scheduler not registered
+
+### Symptoms
+Scheduler log contains
+
+    Framework has not been registered within the tolerated delay.
+
+### Solution
+Double-check that the scheduler is configured correctly to reach the Mesos master. If you are registering
+the master in ZooKeeper, make sure command line argument to the master:
+
+    --zk=zk://$ZK_HOST:2181/mesos/master
+
+is the same as the one on the scheduler:
+
+    -mesos_master_address=zk://$ZK_HOST:2181/mesos/master
+
+
+## Scheduler not running
+
+### Symptoms
+The scheduler process commits suicide regularly. This happens under error conditions, but
+also on purpose in regular intervals.
+
+### Solution
+Aurora is meant to be run under supervision. You have to configure a supervisor like
+[Monit](http://mmonit.com/monit/), [supervisord](http://supervisord.org/), or systemd to run the
+scheduler and restart it whenever it fails or exists on purpose.
+
+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`.
+
+
+## Executor crashing or hanging
+
+### Symptoms
+Launched task instances never transition to `STARTING` or `RUNNING` but immediately transition
+to `FAILED` or `LOST`.
+
+### Solution
+The executor might be failing due to unknown internal errors such as a missing native dependency
+of the Mesos executor library. Open the Mesos UI and navigate to the failing
+task in question. Inspect the various log files in order to learn about what is going on.
+
+
+## Observer does not discover tasks
+
+### Symptoms
+The observer UI does not list any tasks. When navigating from the scheduler UI to the state of
+a particular task instance the observer returns `Error: 404 Not Found`.
+
+### Solution
+The observer is refreshing its internal state every couple of seconds. If waiting a few seconds
+does not resolve the issue, check that the `--mesos-root` setting of the observer and the
+`--work_dir` option of the Mesos agent are in sync. For details, see our
+[Install instructions](../installation/#worker-configuration).

Added: aurora/site/source/documentation/0.20.0/operations/upgrades.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/operations/upgrades.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/operations/upgrades.md (added)
+++ aurora/site/source/documentation/0.20.0/operations/upgrades.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,41 @@
+# Upgrading Aurora
+
+Aurora can be updated from one version to the next without any downtime or restarts of running
+jobs. The same holds true for Mesos.
+
+Generally speaking, Mesos and Aurora strive for a +1/-1 version compatibility, i.e. all components
+are meant to be forward and backwards compatible for at least one version. This implies it
+does not really matter in which order updates are carried out.
+
+Exceptions to this rule are documented in the [Aurora release-notes](../../../RELEASE-NOTES/)
+and the [Mesos upgrade instructions](https://mesos.apache.org/documentation/latest/upgrades/).
+
+
+## Instructions
+
+To upgrade Aurora, follow these steps:
+
+1. Update the first scheduler instance by updating its software and restarting its process.
+2. Wait until the scheduler is up and its [Replicated Log](../configuration/#replicated-log-configuration)
+   caught up with the other schedulers in the cluster. The log has caught up if `log/recovered` has
+   the value `1`. You can check the metric via `curl LIBPROCESS_IP:LIBPROCESS_PORT/metrics/snapshot`,
+   where ip and port refer to the [libmesos configuration](../configuration/#network-configuration)
+   settings of the scheduler instance.
+3. Proceed with the next scheduler until all instances are updated.
+4. Update the Aurora executor deployed to the compute nodes of your cluster. Jobs will continue
+   running with the old version of the executor, and will only be launched by the new one once
+   they are restarted eventually due to natural cluster churn.
+5. Distribute the new Aurora client to your users.
+
+
+## Best Practices
+
+Even though not absolutely mandatory, we advice to adhere to the following rules:
+
+* Never skip any major or minor releases when updating. If you have to catch up several releases you
+  have to deploy all intermediary versions. Skipping bugfix releases is acceptable though.
+* Verify all updates on a test cluster before touching your production deployments.
+* To minimize the number of failovers during updates, update the currently leading scheduler
+  instance last.
+* Update the Aurora executor on a subset of compute nodes as a canary before deploying the change to
+  the whole fleet.

Added: aurora/site/source/documentation/0.20.0/reference/client-cluster-configuration.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/reference/client-cluster-configuration.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/reference/client-cluster-configuration.md (added)
+++ aurora/site/source/documentation/0.20.0/reference/client-cluster-configuration.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,99 @@
+# Client Cluster Configuration
+
+A cluster configuration file is used by the Aurora client to describe the Aurora clusters with
+which it can communicate. Ultimately this allows client users to reference clusters with short names
+like us-east and eu.
+
+A cluster configuration is formatted as JSON.  The simplest cluster configuration is one that
+communicates with a single (non-leader-elected) scheduler.  For example:
+
+    [{
+      "name": "example",
+      "scheduler_uri": "http://localhost:55555",
+    }]
+
+
+A configuration for a leader-elected scheduler would contain something like:
+
+    [{
+      "name": "example",
+      "zk": "192.168.33.7",
+      "scheduler_zk_path": "/aurora/scheduler"
+    }]
+
+
+The following properties may be set:
+
+  **Property**             | **Type** | **Description**
+  :------------------------| :------- | :--------------
+   **name**                | String   | Cluster name (Required)
+   **slave_root**          | String   | Path to Mesos agent work dir (Required)
+   **slave_run_directory** | String   | Name of Mesos agent run dir (Required)
+   **zk**                  | String   | Hostname of ZooKeeper instance used to resolve Aurora schedulers.
+   **zk_port**             | Integer  | Port of ZooKeeper instance used to locate Aurora schedulers (Default: 2181)
+   **scheduler_zk_path**   | String   | ZooKeeper path under which scheduler instances are registered.
+   **scheduler_uri**       | String   | URI of Aurora scheduler instance.
+   **proxy_url**           | String   | Used by the client to format URLs for display.
+   **auth_mechanism**      | String   | The authentication mechanism to use when communicating with the scheduler. (Default: UNAUTHENTICATED)
+   **docker_registry**     | String   | Used by the client to resolve docker tags.
+
+
+## Details
+
+### `name`
+
+The name of the Aurora cluster represented by this entry. This name will be the `cluster` portion of
+any job keys identifying jobs running within the cluster.
+
+### `slave_root`
+
+The path on the Mesos agents where executing tasks can be found. It is used in combination with the
+`slave_run_directory` property by `aurora task run` and `aurora task ssh` to change into the sandbox
+directory after connecting to the host. This value should match the value passed to `mesos-slave`
+as `-work_dir`.
+
+### `slave_run_directory`
+
+The name of the directory where the task run can be found. This is used in combination with the
+`slave_root` property by `aurora task run` and `aurora task ssh` to change into the sandbox
+directory after connecting to the host. This should almost always be set to `latest`.
+
+### `zk`
+
+The hostname of the ZooKeeper instance used to resolve the Aurora scheduler. Aurora uses ZooKeeper
+to elect a leader. The client will connect to this ZooKeeper instance to determine the current
+leader. This host should match the host passed to the scheduler as `-zk_endpoints`.
+
+### `zk_port`
+
+The port on which the ZooKeeper instance is running. If not set this will default to the standard
+ZooKeeper port of 2181. This port should match the port in the host passed to the scheduler as
+`-zk_endpoints`.
+
+### `scheduler_zk_path`
+
+The path on the ZooKeeper instance under which the Aurora serverset is registered. This value should
+match the value passed to the scheduler as `-serverset_path`.
+
+### `scheduler_uri`
+
+The URI of the scheduler. This would be used in place of the ZooKeeper related configuration above
+in circumstances where direct communication with a single scheduler is needed (e.g. testing
+environments). It is strongly advised to **never** use this property for production deploys.
+
+### `proxy_url`
+
+Instead of using the hostname of the leading scheduler as the base url, if `proxy_url` is set, its
+value will be used instead. In that scenario the value for `proxy_url` would be, for example, the
+URL of your VIP in a loadbalancer or a roundrobin DNS name.
+
+### `auth_mechanism`
+
+The identifier of an authentication mechanism that the client should use when communicating with the
+scheduler. Support for values other than `UNAUTHENTICATED` requires a matching scheduler-side
+[security configuration](../../operations/security/).
+
+### `docker_registry`
+
+The URI of the Docker Registry that will be used by the Aurora client to resolve docker tags to concrete
+image ids, when using the docker binding helper, like `{{docker.image[name][tag]}}`.

Added: aurora/site/source/documentation/0.20.0/reference/client-commands.md
URL: http://svn.apache.org/viewvc/aurora/site/source/documentation/0.20.0/reference/client-commands.md?rev=1828294&view=auto
==============================================================================
--- aurora/site/source/documentation/0.20.0/reference/client-commands.md (added)
+++ aurora/site/source/documentation/0.20.0/reference/client-commands.md Tue Apr  3 23:54:44 2018
@@ -0,0 +1,339 @@
+Aurora Client Commands
+======================
+
+- [Introduction](#introduction)
+- [Cluster Configuration](#cluster-configuration)
+- [Job Keys](#job-keys)
+- [Modifying Aurora Client Commands](#modifying-aurora-client-commands)
+- [Regular Jobs](#regular-jobs)
+    - [Creating and Running a Job](#creating-and-running-a-job)
+    - [Running a Command On a Running Job](#running-a-command-on-a-running-job)
+    - [Killing a Job](#killing-a-job)
+    - [Adding Instances](#adding-instances)
+    - [Updating a Job](#updating-a-job)
+        - [Coordinated job updates](#coordinated-job-updates)
+    - [Renaming a Job](#renaming-a-job)
+    - [Restarting Jobs](#restarting-jobs)
+- [Cron Jobs](#cron-jobs)
+- [Comparing Jobs](#comparing-jobs)
+- [Viewing/Examining Jobs](#viewingexamining-jobs)
+    - [Listing Jobs](#listing-jobs)
+    - [Inspecting a Job](#inspecting-a-job)
+    - [Versions](#versions)
+    - [Checking Your Quota](#checking-your-quota)
+    - [Finding a Job on Web UI](#finding-a-job-on-web-ui)
+    - [Getting Job Status](#getting-job-status)
+    - [Opening the Web UI](#opening-the-web-ui)
+    - [SSHing to a Specific Task Machine](#sshing-to-a-specific-task-machine)
+    - [SCPing with Specific Task Machines](#scping-with-specific-task-machines)
+    - [Templating Command Arguments](#templating-command-arguments)
+
+Introduction
+------------
+
+Once you have written an `.aurora` configuration file that describes
+your Job and its parameters and functionality, you interact with Aurora
+using Aurora Client commands. This document describes all of these commands
+and how and when to use them. All Aurora Client commands start with
+`aurora`, followed by the name of the specific command and its
+arguments.
+
+*Job keys* are a very common argument to Aurora commands, as well as the
+gateway to useful information about a Job. Before using Aurora, you
+should read the next section which describes them in detail. The section
+after that briefly describes how you can modify the behavior of certain
+Aurora Client commands, linking to a detailed document about how to do
+that.
+
+This is followed by the Regular Jobs section, which describes the basic
+Client commands for creating, running, and manipulating Aurora Jobs.
+After that are sections on Comparing Jobs and Viewing/Examining Jobs. In
+other words, various commands for getting information and metadata about
+Aurora Jobs.
+
+Cluster Configuration
+---------------------
+
+The client must be able to find a configuration file that specifies available clusters. This file
+declares shorthand names for clusters, which are in turn referenced by job configuration files
+and client commands.
+
+The client will load at most two configuration files, making both of their defined clusters
+available. The first is intended to be a system-installed cluster, using the path specified in
+the environment variable `AURORA_CONFIG_ROOT`, defaulting to `/etc/aurora/clusters.json` if the
+environment variable is not set. The second is a user-installed file, located at
+`~/.aurora/clusters.json`.
+
+For more details on cluster configuration see the
+[Client Cluster Configuration](../client-cluster-configuration/) documentation.
+
+Job Keys
+--------
+
+A job key is a unique system-wide identifier for an Aurora-managed
+Job, for example `cluster1/web-team/test/experiment204`. It is a 4-tuple
+consisting of, in order, *cluster*, *role*, *environment*, and
+*jobname*, separated by /s. Cluster is the name of an Aurora
+cluster. Role is the Unix service account under which the Job
+runs. Environment is a namespace component like `devel`, `test`,
+`prod`, or `stagingN.` Jobname is the Job's name.
+
+The combination of all four values uniquely specifies the Job. If any
+one value is different from that of another job key, the two job keys
+refer to different Jobs. For example, job key
+`cluster1/tyg/prod/workhorse` is different from
+`cluster1/tyg/prod/workcamel` is different from
+`cluster2/tyg/prod/workhorse` is different from
+`cluster2/foo/prod/workhorse` is different from
+`cluster1/tyg/test/workhorse.`
+
+Role names are user accounts existing on the agent machines. If you don't know what accounts
+are available, contact your sysadmin.
+
+Environment names are namespaces; you can count on `prod`, `devel` and `test` existing.
+
+Modifying Aurora Client Commands
+--------------------------------
+
+For certain Aurora Client commands, you can define hook methods that run
+either before or after an action that takes place during the command's
+execution, as well as based on whether the action finished successfully or failed
+during execution. Basically, a hook is code that lets you extend the
+command's actions. The hook executes on the client side, specifically on
+the machine executing Aurora commands.
+
+Hooks can be associated with these Aurora Client commands.
+
+  - `job create`
+  - `job kill`
+  - `job restart`
+
+The process for writing and activating them is complex enough
+that we explain it in a devoted document, [Hooks for Aurora Client API](../client-hooks/).
+
+Regular Jobs
+------------
+
+This section covers Aurora commands related to running, killing,
+renaming, updating, and restarting a basic Aurora Job.
+
+### Creating and Running a Job
+
+    aurora job create <job key> <configuration file>
+
+Creates and then runs a Job with the specified job key based on a `.aurora` configuration file.
+The configuration file may also contain and activate hook definitions.
+
+### Running a Command On a Running Job
+
+    aurora task run CLUSTER/ROLE/ENV/NAME[/INSTANCES] <cmd>
+
+Runs a shell command on all machines currently hosting shards of a
+single Job.
+
+`run` supports the same command line wildcards used to populate a Job's
+commands; i.e. anything in the `{{mesos.*}}` and `{{thermos.*}}`
+namespaces.
+
+### Killing a Job
+
+    aurora job killall CLUSTER/ROLE/ENV/NAME
+
+Kills all Tasks associated with the specified Job, blocking until all
+are terminated. Defaults to killing all instances in the Job.
+
+The `<configuration file>` argument for `kill` is optional. Use it only
+if it contains hook definitions and activations that affect the
+kill command.
+
+### Adding Instances
+
+    aurora job add CLUSTER/ROLE/ENV/NAME/INSTANCE <count>
+
+Adds `<count>` instances to the existing job. The configuration of the new instances is derived from
+an active job instance pointed by the `/INSTANCE` part of the job specification. This command is
+a simpler way to scale out an existing job when an instance with desired task configuration
+already exists. Use `aurora update start` to add instances with a new (updated) configuration.
+
+### Updating a Job
+
+You can manage job updates using the `aurora update` command.  Please see
+[the Job Update documentation](../../features/job-updates/) for more details.
+
+
+### Renaming a Job
+
+Renaming is a tricky operation as downstream clients must be informed of
+the new name. A conservative approach
+to renaming suitable for production services is:
+
+1.  Modify the Aurora configuration file to change the role,
+    environment, and/or name as appropriate to the standardized naming
+    scheme.
+2.  Check that only these naming components have changed
+    with `aurora diff`.
+
+        aurora job diff CLUSTER/ROLE/ENV/NAME <job_configuration>
+
+3.  Create the (identical) job at the new key. You may need to request a
+    temporary quota increase.
+
+        aurora job create CLUSTER/ROLE/ENV/NEW_NAME <job_configuration>
+
+4.  Migrate all clients over to the new job key. Update all links and
+    dashboards. Ensure that both job keys run identical versions of the
+    code while in this state.
+5.  After verifying that all clients have successfully moved over, kill
+    the old job.
+
+        aurora job killall CLUSTER/ROLE/ENV/NAME
+
+6.  If you received a temporary quota increase, be sure to let the
+    powers that be know you no longer need the additional capacity.
+
+### Restarting Jobs
+
+`restart` restarts all of a job key identified Job's shards:
+
+    aurora job restart CLUSTER/ROLE/ENV/NAME[/INSTANCES]
+
+Restarts are controlled on the client side, so aborting
+the `job restart` command halts the restart operation.
+
+**Note**: `job restart` only applies its command line arguments and does not
+use or is affected by `update.config`. Restarting
+does ***not*** involve a configuration change. To update the
+configuration, use `update.config`.
+
+The `--config` argument for restart is optional. Use it only
+if it contains hook definitions and activations that affect the
+`job restart` command.
+
+Cron Jobs
+---------
+
+You can manage cron jobs using the `aurora cron` command.  Please see
+[the Cron Jobs Feature](../../features/cron-jobs/) for more details.
+
+Comparing Jobs
+--------------
+
+    aurora job diff CLUSTER/ROLE/ENV/NAME <job configuration>
+
+Compares a job configuration against a running job. By default the diff
+is determined using `diff`, though you may choose an alternate
+ diff program by specifying the `DIFF_VIEWER` environment variable.
+
+Viewing/Examining Jobs
+----------------------
+
+Above we discussed creating, killing, and updating Jobs. Here we discuss
+how to view and examine Jobs.
+
+### Listing Jobs
+
+    aurora config list <job configuration>
+
+Lists all Jobs registered with the Aurora scheduler in the named cluster for the named role.
+
+### Inspecting a Job
+
+    aurora job inspect CLUSTER/ROLE/ENV/NAME <job configuration>
+
+`inspect` verifies that its specified job can be parsed from a
+configuration file, and displays the parsed configuration.
+
+### Checking Your Quota
+
+    aurora quota get CLUSTER/ROLE
+
+Prints the production quota allocated to the role's value at the given
+cluster. Only non-[dedicated](../../features/constraints/#dedicated-attribute)
+[production](../configuration/#job-objects) jobs consume quota.
+
+### Finding a Job on Web UI
+
+When you create a job, part of the output response contains a URL that goes
+to the job's scheduler UI page. For example:
+
+    vagrant@precise64:~$ aurora job create devcluster/www-data/prod/hello /vagrant/examples/jobs/hello_world.aurora
+    INFO] Creating job hello
+    INFO] Response from scheduler: OK (message: 1 new tasks pending for job www-data/prod/hello)
+    INFO] Job url: http://precise64:8081/scheduler/www-data/prod/hello
+
+You can go to the scheduler UI page for this job via `http://precise64:8081/scheduler/www-data/prod/hello`
+You can go to the overall scheduler UI page by going to the part of that URL that ends at `scheduler`; `http://precise64:8081/scheduler`
+
+Once you click through to a role page, you see Jobs arranged
+separately by pending jobs, active jobs and finished jobs.
+Jobs are arranged by role, typically a service account for
+production jobs and user accounts for test or development jobs.
+
+### Getting Job Status
+
+    aurora job status <job_key>
+
+Returns the status of recent tasks associated with the
+`job_key` specified Job in its supplied cluster. Typically this includes
+a mix of active tasks (running or assigned) and inactive tasks
+(successful, failed, and lost.)
+
+### Opening the Web UI
+
+Use the Job's web UI scheduler URL or the `aurora status` command to find out on which
+machines individual tasks are scheduled. You can open the web UI via the
+`open` command line command if invoked from your machine:
+
+    aurora job open [<cluster>[/<role>[/<env>/<job_name>]]]
+
+If only the cluster is specified, it goes directly to that cluster's
+scheduler main page. If the role is specified, it goes to the top-level
+role page. If the full job key is specified, it goes directly to the job
+page where you can inspect individual tasks.
+
+### SSHing to a Specific Task Machine
+
+    aurora task ssh <job_key> <shard number>
+
+You can have the Aurora client ssh directly to the machine that has been
+assigned a particular Job/shard number. This may be useful for quickly
+diagnosing issues such as performance issues or abnormal behavior on a
+particular machine.
+
+### SCPing with Specific Task Machines
+
+    aurora task scp [<cluster>/<role>/<env>/<job_name>/<instance_id>]:source [<cluster>/<role>/<env>/<job_name>/<instance_id>]:dest
+
+You can have the Aurora client copy file(s)/folder(s) to, from, and between
+individual tasks. The sandbox folder serves as the relative root and is the
+same folder you see when you browse `chroot` from the Scheduler task UI. You
+can also use absolute paths (like for `/tmp`), but tilde expansion is not
+supported. Currently, this command is only fully supported for Mesos
+containers. Users may use this to copy files from Docker containers but they
+cannot copy files to them.
+
+### Templating Command Arguments
+
+    aurora task run [-e] [-t THREADS] <job_key> -- <<command-line>>
+
+Given a job specification, run the supplied command on all hosts and
+return the output. You may use the standard Mustache templating rules:
+
+- `{{thermos.ports[name]}}` substitutes the specific named port of the
+  task assigned to this machine
+- `{{mesos.instance}}` substitutes the shard id of the job's task
+  assigned to this machine
+- `{{thermos.task_id}}` substitutes the task id of the job's task
+  assigned to this machine
+
+For example, the following type of pattern can be a powerful diagnostic
+tool:
+
+    aurora task run -t5 cluster1/tyg/devel/seizure -- \
+      'curl -s -m1 localhost:{{thermos.ports[http]}}/vars | grep uptime'
+
+By default, the command runs in the Task's sandbox. The `-e` option can
+run the command in the executor's sandbox. This is mostly useful for
+Aurora administrators.
+
+You can parallelize the runs by using the `-t` option.