You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2014/04/02 18:00:21 UTC

git commit: Use a single VM for the dev cluster, manage processes with upstart.

Repository: incubator-aurora
Updated Branches:
  refs/heads/wfarner/vagrant_one_machine [created] 606610443


Use a single VM for the dev cluster, manage processes with upstart.


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

Branch: refs/heads/wfarner/vagrant_one_machine
Commit: 6066104430b241222e8596fa9a98c3d6a1a97e79
Parents: 85ae33b
Author: Bill Farner <wf...@apache.org>
Authored: Wed Apr 2 08:55:07 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Wed Apr 2 08:55:07 2014 -0700

----------------------------------------------------------------------
 Vagrantfile                                     |  49 +------
 docs/clientcommands.md                          |   2 +-
 docs/tutorial.md                                |  61 ++++----
 docs/vagrant.md                                 |   8 +-
 examples/vagrant/provision-aurora-scheduler.sh  | 119 ----------------
 examples/vagrant/provision-dev-cluster.sh       | 138 +++++++++++++++++++
 examples/vagrant/provision-dev-environment.sh   |  68 ---------
 examples/vagrant/provision-mesos-master.sh      |  41 ------
 examples/vagrant/provision-mesos-slave.sh       |  73 ----------
 examples/vagrant/provision-zookeeper.sh         |  26 ----
 examples/vagrant/upstart/aurora-scheduler.conf  |  42 ++++++
 .../upstart/aurora-thermos-observer.conf        |  24 ++++
 examples/vagrant/upstart/mesos-master.conf      |  24 ++++
 examples/vagrant/upstart/mesos-slave.conf       |  29 ++++
 examples/vagrant/upstart/zookeeper.conf         |  23 ++++
 15 files changed, 321 insertions(+), 406 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/Vagrantfile
----------------------------------------------------------------------
diff --git a/Vagrantfile b/Vagrantfile
index 52e2fff..f339368 100644
--- a/Vagrantfile
+++ b/Vagrantfile
@@ -19,7 +19,6 @@
 # Vagrantfile API/syntax version. Don't touch unless you know what you're doing!
 VAGRANTFILE_API_VERSION = "2"
 
-# TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered.
 Vagrant.configure(VAGRANTFILE_API_VERSION) do |config|
   config.vm.box = "precise64"
 
@@ -27,52 +26,12 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config|
   # doesn't already exist on the user's system.
   config.vm.box_url = "http://files.vagrantup.com/precise64.box"
 
-  config.vm.define "devtools" do |devtools|
-    devtools.vm.network :private_network, ip: "192.168.33.7"
-    devtools.vm.provider :virtualbox do |vb|
-      vb.customize ["modifyvm", :id, "--memory", "2048", "--cpus", "8"]
-    end
-    devtools.vm.provision "shell", path: "examples/vagrant/provision-dev-environment.sh"
-  end
-
-  config.vm.define "zookeeper" do |zookeeper|
-    zookeeper.vm.network :private_network, ip: "192.168.33.2"
-    zookeeper.vm.provider :virtualbox do |vb|
-      vb.customize ["modifyvm", :id, "--memory", "512"]
-    end
-    zookeeper.vm.provision "shell", path: "examples/vagrant/provision-zookeeper.sh"
-  end
-
-  config.vm.define "mesos-master" do |master|
-    master.vm.network :private_network, ip: "192.168.33.3"
-    master.vm.provider :virtualbox do |vb|
-      vb.customize ["modifyvm", :id, "--memory", "1024"]
-    end
-    master.vm.provision "shell", path: "examples/vagrant/provision-mesos-master.sh"
-  end
-
-  config.vm.define "mesos-slave1" do |slave|
-    slave.vm.network :private_network, ip: "192.168.33.4"
-    slave.vm.provider :virtualbox do |vb|
-      vb.customize ["modifyvm", :id, "--memory", "512"]
-    end
-    slave.vm.provision "shell", path: "examples/vagrant/provision-mesos-slave.sh"
-  end
-
-  config.vm.define "mesos-slave2" do |slave|
-    slave.vm.network :private_network, ip: "192.168.33.5"
-    slave.vm.provider :virtualbox do |vb|
-      vb.customize ["modifyvm", :id, "--memory", "512"]
-    end
-    slave.vm.provision "shell", path: "examples/vagrant/provision-mesos-slave.sh"
-  end
-
-  config.vm.define "aurora-scheduler" do |scheduler|
-    scheduler.vm.network :private_network, ip: "192.168.33.6"
-    scheduler.vm.provider :virtualbox do |vb|
+  config.vm.define "devcluster" do |dev|
+    dev.vm.network :private_network, ip: "192.168.33.7"
+    dev.vm.provider :virtualbox do |vb|
       vb.customize ["modifyvm", :id, "--memory", "1024"]
     end
-    scheduler.vm.provision "shell", path: "examples/vagrant/provision-aurora-scheduler.sh"
+    dev.vm.provision "shell", path: "examples/vagrant/provision-dev-cluster.sh"
   end
 end
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/docs/clientcommands.md
----------------------------------------------------------------------
diff --git a/docs/clientcommands.md b/docs/clientcommands.md
index adf378c..098d261 100644
--- a/docs/clientcommands.md
+++ b/docs/clientcommands.md
@@ -82,7 +82,7 @@ A configuration for a leader-elected scheduler would contain something like:
 ```javascript
 [{
   "name": "example",
-  "zk": "192.168.33.2",
+  "zk": "192.168.33.7",
   "scheduler_zk_path": "/aurora/scheduler"
 }]
 ```

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/docs/tutorial.md
----------------------------------------------------------------------
diff --git a/docs/tutorial.md b/docs/tutorial.md
index 3ccc862..6302aa0 100644
--- a/docs/tutorial.md
+++ b/docs/tutorial.md
@@ -34,7 +34,8 @@ To get help, email questions to the Aurora Developer List,
 
 You use the Aurora client and web UI to interact with Aurora jobs. To
 install it locally, see [vagrant.md](vagrant.md). The remainder of this
-Tutorial assumes you are running Aurora using Vagrant.
+Tutorial assumes you are running Aurora using Vagrant.  Unless otherwise stated,
+all commands are to be run from the root of the aurora repository clone.
 
 ## The Script
 
@@ -66,17 +67,22 @@ if __name__ == "__main__":
 
 Once we have our script/program, we need to create a *configuration
 file* that tells Aurora how to manage and launch our Job. Save the below
-code in the file `hello_world.aurora` in the same directory as your
-`hello_world.py` file. (all Aurora configuration files end with `.aurora` and
-are written in a Python variant).
+code in the file `hello_world.aurora`.
 
 ```python
-import os
+pkg_path = '/vagrant/hello_world.py'
+
+# we use a trick here to make the configuration change with
+# the contents of the file, for simplicity.  in a normal setting, packages would be
+# versioned, and the version number would be changed in the configuration.
+import hashlib
+with open(pkg_path, 'rb') as f:
+  pkg_checksum = hashlib.md5(f.read()).hexdigest()
 
 # copy hello_world.py into the local sandbox
 install = Process(
   name = 'fetch_package',
-  cmdline = 'cp /vagrant/hello_world.py . && chmod +x hello_world.py')
+  cmdline = 'cp %s . && echo %s && chmod +x hello_world.py' % (pkg_path, pkg_checksum))
 
 # run the script
 hello_world = Process(
@@ -89,8 +95,11 @@ hello_world_task = SequentialTask(
   resources = Resources(cpu = 1, ram = 1*MB, disk=8*MB))
 
 jobs = [
-  Job(name = 'hello_world', cluster = 'example', role = 'www-data',
-      environment = 'devel', task = hello_world_task)
+  Service(cluster = 'vagrant',
+          environment = 'devel',
+          role = 'www-data',
+          name = 'hello_world',
+          task = hello_world_task)
 ]
 ```
 
@@ -132,7 +141,7 @@ identical, the job keys identify the same job.
 cluster names. For Vagrant, from the top-level of your Aurora repository clone,
 do:
 
-    $ vagrant ssh aurora-scheduler
+    $ vagrant ssh devcluster
 
 Followed by:
 
@@ -142,8 +151,8 @@ You'll see something like:
 
 ```javascript
 [{
-  "name": "example",
-  "zk": "192.168.33.2",
+  "name": "vagrant",
+  "zk": "192.168.33.7",
   "scheduler_zk_path": "/aurora/scheduler",
   "auth_mechanism": "UNAUTHENTICATED"
 }]
@@ -163,20 +172,17 @@ specified by its job key and configuration file arguments and runs it.
 
 Or for our example:
 
-    aurora create example/www-data/devel/hello_world /vagrant/hello_world.aurora
-
-Note: Remember, the job key's `<jobname>` value is the name of the Job, not the name
-of its code file.
+    aurora create vagrant/www-data/devel/hello_world /vagrant/hello_world.aurora
 
 This returns:
 
-    $ vagrant ssh aurora-scheduler
+    $ vagrant ssh devcluster
     Welcome to Ubuntu 12.04 LTS (GNU/Linux 3.2.0-23-generic x86_64)
 
      * Documentation:  https://help.ubuntu.com/
     Welcome to your Vagrant-built virtual machine.
     Last login: Fri Jan  3 02:18:55 2014 from 10.0.2.2
-    vagrant@precise64:~$ aurora create example/www-data/devel/hello_world \
+    vagrant@precise64:~$ aurora create vagrant/www-data/devel/hello_world \
         /vagrant/hello_world.aurora
      INFO] Creating job hello_world
      INFO] Response from scheduler: OK (message: 1 new tasks pending for job
@@ -187,7 +193,7 @@ This returns:
 
 Now that our job is running, let's see what it's doing. Access the
 scheduler web interface at `http://$scheduler_hostname:$scheduler_port/scheduler`
-Or when using `vagrant`, `http://192.168.33.6:8081/scheduler`
+Or when using `vagrant`, `http://192.168.33.7:8081/scheduler`
 First we see what Jobs are scheduled:
 
 ![Scheduled Jobs](images/ScheduledJobs.png)
@@ -197,15 +203,14 @@ with that role:
 
 ![Role Jobs](images/RoleJobs.png)
 
-Uh oh, that `Unstable` next to our `hello_world` Job doesn't look good. Click the
-`hello_world` Job, and you'll see:
+If you click on your `hello_world` Job, you'll see:
 
 ![hello_world Job](images/HelloWorldJob.png)
 
 Oops, looks like our first job didn't quite work! The task failed, so we have
 to figure out what went wrong.
 
-Access the page for our Task by clicking on its Host.
+Access the page for our Task by clicking on its host.
 
 ![Task page](images/TaskBreakdown.png)
 
@@ -217,12 +222,10 @@ to `stderr` on the failed `hello_world` process, we see what happened.
 ![stderr page](images/stderr.png)
 
 It looks like we made a typo in our Python script. We wanted `xrange`,
-not `xrang`. Edit the `hello_world.py` script, save as `hello_world_v2.py` and change your
-`hello_world.aurora` config file to use `hello_world_v2.py` instead of `hello_world.py`.
-
-Now that we've updated our configuration, let's restart the job:
+not `xrang`. Edit the `hello_world.py` script to use the correct function and
+we will try again.
 
-    aurora update example/www-data/devel/hello_world /vagrant/hello_world.aurora
+    aurora update vagrant/www-data/devel/hello_world /vagrant/hello_world.aurora
 
 This time, the task comes up, we inspect the page, and see that the
 `hello_world` process is running.
@@ -238,13 +241,13 @@ output:
 
 Now that we're done, we kill the job using the Aurora client:
 
-    vagrant@precise64:~$ aurora kill example/www-data/devel/hello_world
-     INFO] Killing tasks for job: example/www-data/devel/hello_world
+    vagrant@precise64:~$ aurora killall vagrant/www-data/devel/hello_world
+     INFO] Killing tasks for job: vagrant/www-data/devel/hello_world
      INFO] Response from scheduler: OK (message: Tasks killed.)
      INFO] Job url: http://precise64:8081/scheduler/www-data/devel/hello_world
     vagrant@precise64:~$
 
-The Task scheduler page now shows the `hello_world` process as `KILLED`.
+The job page now shows the `hello_world` tasks as completed.
 
 ![Killed Task page](images/killedtask.png)
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/docs/vagrant.md
----------------------------------------------------------------------
diff --git a/docs/vagrant.md b/docs/vagrant.md
index 15d72bc..6c0876d 100644
--- a/docs/vagrant.md
+++ b/docs/vagrant.md
@@ -4,11 +4,11 @@ explore Aurora's various components. To get started, install
 then run `vagrant up` somewhere in the repository source tree to create a team of VMs.  This may take some time initially as it builds all
 the components involved in running an aurora cluster.
 
-The scheduler is listening on http://192.168.33.5:8081/scheduler
-The observer is listening on http://192.168.33.4:1338/
-The master is listening on http://192.168.33.3:5050/
+The scheduler is listening on http://192.168.33.7:8081/scheduler
+The observer is listening on http://192.168.33.7:1338
+The master is listening on http://192.168.33.7:5050
 
-Once everything is up, you can `vagrant ssh aurora-scheduler` and execute aurora client commands using the `aurora` client.
+Once everything is up, you can `vagrant ssh devcluster` and execute aurora client commands using the `aurora` client.
 
 Troubleshooting
 ---------------

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-aurora-scheduler.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-aurora-scheduler.sh b/examples/vagrant/provision-aurora-scheduler.sh
deleted file mode 100755
index 87c369e..0000000
--- a/examples/vagrant/provision-aurora-scheduler.sh
+++ /dev/null
@@ -1,119 +0,0 @@
-#!/bin/bash -ex
-#
-# Copyright 2014 Apache Software Foundation
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# TODO(ksweeney): Use public and versioned URLs instead of local artifacts.
-AURORA_VERSION=$(cat /vagrant/.auroraversion | tr '[a-z]' '[A-Z]')
-
-tar xvf /vagrant/dist/distributions/aurora-scheduler-$AURORA_VERSION.tar -C /usr/local
-install -m 755 /vagrant/dist/aurora_client.pex /usr/local/bin/aurora
-install -m 755 /vagrant/dist/aurora_admin.pex /usr/local/bin/aurora_admin
-
-apt-get update
-apt-get -y install java7-runtime-headless curl
-wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.deb
-dpkg --install mesos_0.17.0_amd64.deb
-
-# TODO(ksweeney): Make this a be part of the Aurora distribution tarball.
-# Location where aurora-scheduler.zip was unpacked.
-AURORA_SCHEDULER_HOME=/usr/local/aurora-scheduler-$AURORA_VERSION
-cat > /usr/local/sbin/aurora-scheduler.sh <<EOF
-#!/usr/bin/env bash
-# An example scheduler launch script that works with the included Vagrantfile.
-
-
-# Flags that control the behavior of the JVM.
-JAVA_OPTS=(
-  -server
-  -Xmx1g
-  -Xms1g
-
-  # Location of libmesos-0.17.0.so / libmesos-0.17.0.dylib
-  -Djava.library.path=/usr/local/lib
-)
-
-# Flags control the behavior of the Aurora scheduler.
-# For a full list of available flags, run bin/aurora-scheduler -help
-AURORA_FLAGS=(
-  -cluster_name=example
-
-  # Ports to listen on.
-  -http_port=8081
-  -thrift_port=8082
-
-  -native_log_quorum_size=1
-
-  -zk_endpoints=192.168.33.2:2181
-  -mesos_master_address=zk://192.168.33.2:2181/mesos/master
-
-  -serverset_path=/aurora/scheduler
-
-  -native_log_zk_group_path=/aurora/replicated-log
-
-  -native_log_file_path="$AURORA_SCHEDULER_HOME/db"
-  -backup_dir="$AURORA_SCHEDULER_HOME/backups"
-
-  -thermos_executor_path=/usr/local/bin/thermos_executor
-  -gc_executor_path=/usr/local/bin/gc_executor
-
-  -vlog=INFO
-  -logtostderr
-)
-
-# Environment variables control the behavior of the Mesos scheduler driver (libmesos).
-export GLOG_v=0
-export LIBPROCESS_PORT=8083
-export LIBPROCESS_IP=192.168.33.6
-
-(
-  while true
-  do
-    JAVA_OPTS="\${JAVA_OPTS[*]}" exec "$AURORA_SCHEDULER_HOME/bin/aurora-scheduler" \\
-      "\${AURORA_FLAGS[@]}"
-  done
-) &
-EOF
-chmod +x /usr/local/sbin/aurora-scheduler.sh
-
-mkdir -p /etc/aurora
-cat > /etc/aurora/clusters.json <<EOF
-[{
-  "name": "example",
-  "zk": "192.168.33.2",
-  "scheduler_zk_path": "/aurora/scheduler",
-  "auth_mechanism": "UNAUTHENTICATED",
-  "slave_run_directory": "latest",
-  "slave_root": "/var/lib/mesos"
-}]
-EOF
-
-cat > /etc/rc.local <<EOF
-#!/bin/sh -e
-/usr/local/sbin/aurora-scheduler.sh \
-  1> /var/log/aurora-scheduler-stdout.log \
-  2> /var/log/aurora-scheduler-stderr.log
-EOF
-chmod +x /etc/rc.local
-
-export LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
-if mesos-log initialize --path="/usr/local/aurora-scheduler-$AURORA_VERSION/db"; then
-  echo "Replicated log initialized."
-else
-  echo "Replicated log initialization failed with code $? (likely already initialized)."
-fi
-unset LD_LIBRARY_PATH
-
-/etc/rc.local

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-dev-cluster.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-dev-cluster.sh b/examples/vagrant/provision-dev-cluster.sh
new file mode 100755
index 0000000..d381943
--- /dev/null
+++ b/examples/vagrant/provision-dev-cluster.sh
@@ -0,0 +1,138 @@
+#!/bin/bash -x
+#
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+apt-get update
+apt-get -y install \
+    automake \
+    curl \
+    git \
+    g++ \
+    libcurl4-openssl-dev \
+    libsasl2-dev \
+    libtool \
+    make \
+    openjdk-7-jdk \
+    python-dev \
+    zookeeper
+
+# Ensure java 7 is the default java.
+update-alternatives --set java /usr/lib/jvm/java-7-openjdk-amd64/jre/bin/java
+
+# Set the hostname to the IP address.  This simplifies things for components
+# that want to advertise the hostname to the user, or other components.
+hostname 192.168.33.7
+
+AURORA_VERSION=$(cat /vagrant/.auroraversion | tr '[a-z]' '[A-Z]')
+
+function build_all() {
+  if [ ! -d aurora ]; then
+    echo Cloning aurora repo
+    git clone /vagrant aurora
+  fi
+
+  pushd aurora
+    mkdir -p third_party
+    pushd third_party
+      wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.egg \
+        -O mesos-0.17.0-py2.7-linux-x86_64.egg
+    popd
+    git pull
+
+    # build scheduler
+    ./gradlew distTar
+
+    # build clients
+    ./pants src/main/python/apache/aurora/client/bin:aurora_admin
+    ./pants src/main/python/apache/aurora/client/bin:aurora_client
+
+    # build executors/observers
+    ./pants src/main/python/apache/aurora/executor/bin:gc_executor
+    ./pants src/main/python/apache/aurora/executor/bin:thermos_executor
+    ./pants src/main/python/apache/aurora/executor/bin:thermos_runner
+    ./pants src/main/python/apache/thermos/observer/bin:thermos_observer
+
+    # package runner w/in executor
+    python <<EOF
+import contextlib
+import zipfile
+with contextlib.closing(zipfile.ZipFile('dist/thermos_executor.pex', 'a')) as zf:
+  zf.writestr('apache/aurora/executor/resources/__init__.py', '')
+  zf.write('dist/thermos_runner.pex', 'apache/aurora/executor/resources/thermos_runner.pex')
+EOF
+  popd
+
+  sudo chown -R vagrant:vagrant aurora
+}
+
+function install_mesos() {
+  wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.deb
+  dpkg --install mesos_0.17.0_amd64.deb
+}
+
+function install_aurora_scheduler() {
+  # The bulk of the 'install' was done by the gradle build, the result of which we access
+  # through /vagrant.  All that's left is to initialize the log replica.
+  tar xvf ~/aurora/dist/distributions/aurora-scheduler-$AURORA_VERSION.tar -C /usr/local
+
+  export LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
+  if mesos-log initialize --path="/usr/local/aurora-scheduler-$AURORA_VERSION/db"; then
+    echo "Replicated log initialized."
+  else
+    echo "Replicated log initialization failed with code $? (likely already initialized)."
+  fi
+  unset LD_LIBRARY_PATH
+}
+
+function install_aurora_executors() {
+  install -m 755 ~/aurora/dist/gc_executor.pex /usr/local/bin/gc_executor
+  install -m 755 ~/aurora/dist/thermos_executor.pex /usr/local/bin/thermos_executor
+  install -m 755 ~/aurora/dist/thermos_observer.pex /usr/local/bin/thermos_observer
+}
+
+function install_aurora_client() {
+  install -m 755 /vagrant/dist/aurora_client.pex /usr/local/bin/aurora
+  install -m 755 /vagrant/dist/aurora_admin.pex /usr/local/bin/aurora_admin
+
+  mkdir -p /etc/aurora
+  cat > /etc/aurora/clusters.json <<EOF
+[{
+  "name": "vagrant",
+  "zk": "192.168.33.7",
+  "scheduler_zk_path": "/aurora/scheduler",
+  "auth_mechanism": "UNAUTHENTICATED",
+  "slave_run_directory": "latest",
+  "slave_root": "/var/lib/mesos"
+}]
+EOF
+}
+
+function start_services() {
+  cp /vagrant/examples/vagrant/upstart/*.conf /etc/init
+
+  start zookeeper
+  start mesos-master
+  start mesos-slave
+  start aurora-thermos-observer
+  start aurora-scheduler
+}
+
+build_all
+install_mesos
+install_aurora_scheduler
+install_aurora_executors
+install_aurora_client
+start_services

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-dev-environment.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-dev-environment.sh b/examples/vagrant/provision-dev-environment.sh
deleted file mode 100755
index 2306911..0000000
--- a/examples/vagrant/provision-dev-environment.sh
+++ /dev/null
@@ -1,68 +0,0 @@
-#!/bin/bash -x
-#
-# Copyright 2014 Apache Software Foundation
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-apt-get update
-apt-get -y install \
-    git automake libtool g++ java7-runtime-headless curl \
-    openjdk-7-jdk python-dev libsasl2-dev libcurl4-openssl-dev \
-    make
-
-if [ ! -d aurora ]; then
-  echo Cloning aurora repo
-  git clone /vagrant aurora
-fi
-
-pushd aurora
-  AURORA_VERSION=$(cat .auroraversion | tr '[a-z]' '[A-Z]')
-  mkdir -p third_party
-  pushd third_party
-    wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.egg \
-      -O mesos-0.17.0-py2.7-linux-x86_64.egg
-  popd
-  git pull
-
-  # build scheduler
-  ./gradlew distTar
-
-  # build clients
-  ./pants src/main/python/apache/aurora/client/bin:aurora_admin
-  ./pants src/main/python/apache/aurora/client/bin:aurora_client
-
-  # build executors/observers
-  ./pants src/main/python/apache/aurora/executor/bin:gc_executor
-  ./pants src/main/python/apache/aurora/executor/bin:thermos_executor
-  ./pants src/main/python/apache/aurora/executor/bin:thermos_runner
-  ./pants src/main/python/apache/thermos/observer/bin:thermos_observer
-
-  # package runner w/in executor
-  python <<EOF
-import contextlib
-import zipfile
-with contextlib.closing(zipfile.ZipFile('dist/thermos_executor.pex', 'a')) as zf:
-  zf.writestr('apache/aurora/executor/resources/__init__.py', '')
-  zf.write('dist/thermos_runner.pex', 'apache/aurora/executor/resources/thermos_runner.pex')
-EOF
-
-  mkdir -p /vagrant/dist/distributions
-  cp dist/distributions/aurora-scheduler-$AURORA_VERSION.tar /vagrant/dist/distributions
-
-  for pex in aurora_admin aurora_client gc_executor thermos_executor thermos_observer; do
-    cp dist/$pex.pex /vagrant/dist
-  done
-popd
-
-sudo chown -R vagrant:vagrant aurora

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-mesos-master.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-mesos-master.sh b/examples/vagrant/provision-mesos-master.sh
deleted file mode 100755
index 075d23d..0000000
--- a/examples/vagrant/provision-mesos-master.sh
+++ /dev/null
@@ -1,41 +0,0 @@
-#!/bin/bash -x
-#
-# Copyright 2014 Apache Software Foundation
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-apt-get update
-apt-get -y install java7-runtime-headless curl
-wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.deb
-dpkg --install mesos_0.17.0_amd64.deb
-cat > /usr/local/sbin/mesos-master.sh <<EOF
-#!/bin/bash
-export LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
-(
-  while true
-  do
-    /usr/local/sbin/mesos-master --zk=zk://192.168.33.2:2181/mesos/master --ip=192.168.33.3
-    echo "Master exited with $?, restarting."
-  done
-) & disown
-EOF
-chmod +x /usr/local/sbin/mesos-master.sh
-
-cat > /etc/rc.local <<EOF
-#!/bin/sh -e
-/usr/local/sbin/mesos-master.sh >/var/log/mesos-master-stdout.log 2>/var/log/mesos-master-stderr.log
-EOF
-chmod +x /etc/rc.local
-
-/etc/rc.local

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-mesos-slave.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-mesos-slave.sh b/examples/vagrant/provision-mesos-slave.sh
deleted file mode 100755
index f42b5c3..0000000
--- a/examples/vagrant/provision-mesos-slave.sh
+++ /dev/null
@@ -1,73 +0,0 @@
-#!/bin/bash -x
-#
-# Copyright 2014 Apache Software Foundation
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-apt-get update
-apt-get -y install java7-runtime-headless curl
-wget -c http://downloads.mesosphere.io/master/ubuntu/12.04/mesos_0.17.0_amd64.deb
-dpkg --install mesos_0.17.0_amd64.deb
-
-# Get the IP address; eth1 is the net interface we care about.
-export SLAVE_IP_ADDRESS=$(ip addr show eth1 | grep 'inet ' | cut -d'/' -f 1 | cut -c '10-')
-cat> /usr/local/sbin/mesos-slave.sh <<EOF
-#!/bin/bash
-export LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
-(
-  while true
-  do
-    # TODO(ksweeney): Scheduler assumes 'rack' and 'host' are present. Make them optional.
-    /usr/local/sbin/mesos-slave --master=zk://192.168.33.2:2181/mesos/master --ip=$SLAVE_IP_ADDRESS \
-      --attributes='host:$SLAVE_IP_ADDRESS;rack:a' --resources="cpus:4;mem:1024;disk:20000"
-    echo "Master exited with \$?, restarting."
-  done
-) & disown
-EOF
-chmod +x /usr/local/sbin/mesos-slave.sh
-
-cat > /usr/local/bin/thermos_observer.sh <<"EOF"
-#!/bin/bash
-(
-  while true
-  do
-    /usr/local/bin/thermos_observer \
-         --root=/var/run/thermos \
-         --port=1338 \
-         --log_to_disk=NONE \
-         --log_to_stderr=google:INFO
-    echo "Observer exited with $?, restarting."
-    sleep 10
-  done
-) & disown
-EOF
-chmod +x /usr/local/bin/thermos_observer.sh
-
-# TODO(ksweeney): Hack until the --hostname change for mesos-slave lands.
-echo $SLAVE_IP_ADDRESS > /etc/hostname
-hostname $SLAVE_IP_ADDRESS
-
-# TODO(ksweeney): Replace with public and versioned URLs.
-install -m 755 /vagrant/dist/gc_executor.pex /usr/local/bin/gc_executor
-install -m 755 /vagrant/dist/thermos_executor.pex /usr/local/bin/thermos_executor
-install -m 755 /vagrant/dist/thermos_observer.pex /usr/local/bin/thermos_observer
-
-cat > /etc/rc.local <<EOF
-#!/bin/sh -e
-/usr/local/sbin/mesos-slave.sh >/var/log/mesos-slave-stdout.log 2>/var/log/mesos-slave-stderr.log
-/usr/local/bin/thermos_observer.sh >/var/log/thermos-observer.log 2>&1
-EOF
-chmod +x /etc/rc.local
-
-/etc/rc.local

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/provision-zookeeper.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-zookeeper.sh b/examples/vagrant/provision-zookeeper.sh
deleted file mode 100755
index 64db4a1..0000000
--- a/examples/vagrant/provision-zookeeper.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash -x
-#
-# Copyright 2014 Apache Software Foundation
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-apt-get update
-apt-get -y install zookeeper
-echo "JVMFLAGS=\"-Djava.net.preferIPv4Stack=true\"" >> /etc/zookeeper/conf/environment
-cat > /etc/rc.local <<EOF
-#!/bin/sh -e
-/usr/share/zookeeper/bin/zkServer.sh start
-EOF
-chmod +x /etc/rc.local
-/etc/rc.local

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/upstart/aurora-scheduler.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/aurora-scheduler.conf b/examples/vagrant/upstart/aurora-scheduler.conf
new file mode 100644
index 0000000..6db8e50
--- /dev/null
+++ b/examples/vagrant/upstart/aurora-scheduler.conf
@@ -0,0 +1,42 @@
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+description "aurora scheduler"
+start on stopped rc RUNLEVEL=[2345]
+respawn
+post-stop exec sleep 5
+
+# Environment variables control the behavior of the Mesos scheduler driver (libmesos).
+env GLOG_v=0
+env LIBPROCESS_PORT=8083
+env LIBPROCESS_IP=192.168.33.7
+
+# Flags that control the behavior of the JVM.
+env JAVA_OPTS='-Djava.library.path=/usr/local/lib'
+
+exec /usr/local/aurora-scheduler-0.5.0-SNAPSHOT/bin/aurora-scheduler \
+  -cluster_name=example \
+  -http_port=8081 \
+  -thrift_port=8082 \
+  -native_log_quorum_size=1 \
+  -zk_endpoints=localhost:2181 \
+  -mesos_master_address=zk://localhost:2181/mesos/master \
+  -serverset_path=/aurora/scheduler \
+  -native_log_zk_group_path=/aurora/replicated-log \
+  -native_log_file_path="/usr/local/aurora-scheduler-0.5.0-SNAPSHOT/db" \
+  -backup_dir="/usr/local/aurora-scheduler-0.5.0-SNAPSHOT/backups" \
+  -thermos_executor_path=/usr/local/bin/thermos_executor \
+  -gc_executor_path=/usr/local/bin/gc_executor \
+  -vlog=INFO \
+  -logtostderr
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/upstart/aurora-thermos-observer.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/aurora-thermos-observer.conf b/examples/vagrant/upstart/aurora-thermos-observer.conf
new file mode 100644
index 0000000..84d58f1
--- /dev/null
+++ b/examples/vagrant/upstart/aurora-thermos-observer.conf
@@ -0,0 +1,24 @@
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+description "aurora thermos observer"
+start on stopped rc RUNLEVEL=[2345]
+respawn
+post-stop exec sleep 5
+
+exec /usr/local/bin/thermos_observer \
+  --root=/var/run/thermos \
+  --port=1338 \
+  --log_to_disk=NONE \
+  --log_to_stderr=google:INFO
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/upstart/mesos-master.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/mesos-master.conf b/examples/vagrant/upstart/mesos-master.conf
new file mode 100644
index 0000000..fbec196
--- /dev/null
+++ b/examples/vagrant/upstart/mesos-master.conf
@@ -0,0 +1,24 @@
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+description "mesos master"
+start on stopped rc RUNLEVEL=[2345]
+respawn
+post-stop exec sleep 5
+
+env LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
+env MY_HOST=192.168.33.7
+env ZK_HOST=192.168.33.7
+
+exec /usr/local/sbin/mesos-master --zk=zk://$ZK_HOST:2181/mesos/master --ip=$MY_HOST
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/upstart/mesos-slave.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/mesos-slave.conf b/examples/vagrant/upstart/mesos-slave.conf
new file mode 100644
index 0000000..6f766e8
--- /dev/null
+++ b/examples/vagrant/upstart/mesos-slave.conf
@@ -0,0 +1,29 @@
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+description "mesos slave"
+start on stopped rc RUNLEVEL=[2345]
+respawn
+post-stop exec sleep 5
+
+env LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
+env MY_HOST=192.168.33.7
+env ZK_HOST=192.168.33.7
+
+# TODO(ksweeney): Scheduler assumes 'rack' and 'host' are present. Make them optional.
+exec /usr/local/sbin/mesos-slave --master=zk://$ZK_HOST:2181/mesos/master \
+  --ip=$MY_HOST \
+  --hostname=$MY_HOST \
+  --attributes="host:$MY_HOST;rack:a" \
+  --resources="cpus:4;mem:1024;disk:20000"

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/60661044/examples/vagrant/upstart/zookeeper.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/zookeeper.conf b/examples/vagrant/upstart/zookeeper.conf
new file mode 100644
index 0000000..0843060
--- /dev/null
+++ b/examples/vagrant/upstart/zookeeper.conf
@@ -0,0 +1,23 @@
+# Copyright 2014 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+description "zookeeper"
+start on stopped rc RUNLEVEL=[2345]
+respawn
+post-stop exec sleep 5
+
+env JVMFLAGS='-Djava.net.preferIPv4Stack=true'
+
+exec /usr/share/zookeeper/bin/zkServer.sh start-foreground
+