You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by se...@apache.org on 2016/09/09 08:07:52 UTC

aurora git commit: Document the Mesos containerizer

Repository: aurora
Updated Branches:
  refs/heads/master 87ae96802 -> c7f710a56


Document the Mesos containerizer

Included changes:

* consistent example jobs for both containerizers
* short enduser and operator  documentation
* shuffled the reference documentation so that it is clear certain limitations apply only to the Docker containerizer

Bugs closed: AURORA-1640

Reviewed at https://reviews.apache.org/r/51664/


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

Branch: refs/heads/master
Commit: c7f710a567466e43a434a0fa8a169b5348a5cdef
Parents: 87ae968
Author: Stephan Erb <se...@apache.org>
Authored: Fri Sep 9 10:06:49 2016 +0200
Committer: Stephan Erb <se...@apache.org>
Committed: Fri Sep 9 10:06:49 2016 +0200

----------------------------------------------------------------------
 docs/features/containers.md              | 108 ++++++++++++++++-------
 docs/operations/configuration.md         | 120 +++++++++++++++-----------
 docs/reference/configuration.md          |  53 ++++++------
 examples/jobs/docker/hello_docker.aurora |  35 --------
 examples/jobs/hello_docker_engine.aurora |  39 +++++++++
 examples/jobs/hello_docker_image.aurora  |  40 +++++++++
 examples/vagrant/aurorabuild.sh          |   2 -
 7 files changed, 250 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/docs/features/containers.md
----------------------------------------------------------------------
diff --git a/docs/features/containers.md b/docs/features/containers.md
index 6b9f717..8af38e3 100644
--- a/docs/features/containers.md
+++ b/docs/features/containers.md
@@ -1,16 +1,27 @@
 Containers
 ==========
 
-Docker
-------
+Aurora supports several containerizers, notably the Mesos containerizer and the Docker
+containerizer. The Mesos containerizer uses native OS features directly to provide isolation between
+containers, while the Docker containerizer delegates container management to the Docker engine.
 
-Aurora has optional support for launching Docker containers, if correctly [configured by an Operator](../operations/configuration.md#docker-containers).
+The support for launching container images via both containerizers has to be
+[enabled by a cluster operator](../operations/configuration.md#containers).
 
-Example (available in the [Vagrant environment](../getting-started/vagrant.md)):
+Mesos Containerizer
+-------------------
+
+The Mesos containerizer is the native Mesos containerizer solution. It allows tasks to be
+run with an array of [pluggable isolators](resource-isolation.md) and can launch tasks using
+[Docker](https://github.com/docker/docker/blob/master/image/spec/v1.md) images,
+[AppC](https://github.com/appc/spec/blob/master/SPEC.md) images, or directly on the agent host
+filesystem.
 
+The following example (available in our [Vagrant environment](../getting-started/vagrant.md))
+launches a hello world example within a `debian/jessie` Docker image:
 
-    $ cat /vagrant/examples/jobs/docker/hello_docker.aurora
-    hello_world_proc = Process(
+    $ cat /vagrant/examples/jobs/hello_docker_image.aurora
+    hello_loop = Process(
       name = 'hello',
       cmdline = """
         while true; do
@@ -19,42 +30,77 @@ Example (available in the [Vagrant environment](../getting-started/vagrant.md)):
         done
       """)
 
-    hello_world_docker = Task(
-      name = 'hello docker',
-      processes = [hello_world_proc],
-      resources = Resources(cpu = 1, ram = 1*MB, disk=8*MB)
+    task = Task(
+      processes = [hello_loop],
+      resources = Resources(cpu=1, ram=1*MB, disk=8*MB)
     )
 
     jobs = [
       Service(
         cluster = 'devcluster',
         environment = 'devel',
-        role = 'docker-test',
-        name = 'hello_docker',
-        task = hello_world_docker,
-        container = Docker(image = 'python:2.7')
+        role = 'www-data',
+        name = 'hello_docker_image',
+        task = task,
+        container = Mesos(image=DockerImage(name='debian', tag='jessie'))
       )
     ]
 
+Docker and Appc images are designated using an appropriate `image` property of the `Mesos`
+configuration object. If either `container` or `image` is left unspecified, the host filesystem
+will be used. Further details of how to specify images can be found in the
+[Reference Documentation](../reference/configuration.md#mesos-object).
+
+By default, Aurora launches processes as the Linux user named like the used role (e.g. `www-data`
+in the example above). This user has to exist on the host filesystem. If it does not exist within
+the container image, it will be created automatically. Otherwise, this user and its primary group
+has to exist in the image with matching uid/gid.
+
+For more information on the Mesos containerizer filesystem, namespace, and isolator features, visit
+[Mesos Containerizer](http://mesos.apache.org/documentation/latest/mesos-containerizer/) and
+[Mesos Container Images](http://mesos.apache.org/documentation/latest/container-image/).
+
 
-In order to correctly execute processes inside a job, the docker container must have Python 2.7
-installed. Further details of how to use Docker can be found in the
-[Reference Documentation](../reference/configuration.md#docker-object).
+Docker Containerizer
+--------------------
 
-Mesos
------
+The Docker containerizer launches container images using the Docker engine. It may often provide
+more advanced features than the native Mesos containerizer, but has to be installed separately to
+Mesos on each agent host,
 
-*Note: In order to use filesystem images with Aurora, you must be running at least Mesos 0.28.x*
+Example (available in the [Vagrant environment](../getting-started/vagrant.md)):
+
+    $ cat /vagrant/examples/jobs/hello_docker_engine.aurora
+    hello_loop = Process(
+      name = 'hello',
+      cmdline = """
+        while true; do
+          echo hello world
+          sleep 10
+        done
+      """)
+
+    task = Task(
+      processes = [hello_loop],
+      resources = Resources(cpu=1, ram=1*MB, disk=8*MB)
+    )
+
+    jobs = [
+      Service(
+        cluster = 'devcluster',
+        environment = 'devel',
+        role = 'www-data',
+        name = 'hello_docker',
+        task = task,
+        container = Docker(image = 'python:2.7')
+      )
+    ]
 
-Aurora supports specifying a task filesystem image to use with the [Mesos containerizer](http://mesos.apache.org/documentation/latest/container-image/).
-This is done by setting the ```container``` property of the Job to a ```Mesos``` container object
-that includes the image to use. Both [AppC](https://github.com/appc/spec/blob/master/SPEC.md) and 
-[Docker](https://github.com/docker/docker/blob/master/image/spec/v1.md) images are supported.
+Details of how to use Docker via the Docker engine can be found in the
+[Reference Documentation](../reference/configuration.md#docker-object). Please note that in order to
+correctly execute processes inside a job, the Docker container must have Python 2.7 and potentitally
+further Mesos dependencies installed. This limitation does not hold for Docker containers used via
+the Mesos containerizer.
 
-```
-job = Job(
-   ...
-   container = Mesos(image=DockerImage(name='my-image', tag='my-tag'))
-   ...
-)
-```
+For more information on launching Docker containers through the Docker containerizer, visit
+[Docker Containerizer](http://mesos.apache.org/documentation/latest/docker-containerizer/)

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/docs/operations/configuration.md
----------------------------------------------------------------------
diff --git a/docs/operations/configuration.md b/docs/operations/configuration.md
index 85787b0..90dde57 100644
--- a/docs/operations/configuration.md
+++ b/docs/operations/configuration.md
@@ -29,8 +29,9 @@ Like Mesos, Aurora uses command-line flags for runtime configuration. As such th
     # Environment variables controlling libmesos
     export JAVA_HOME=...
     export GLOG_v=1
-    # Port used to communicate with the Mesos master and for the replicated log
+    # Port and public ip used to communicate with the Mesos master and for the replicated log
     export LIBPROCESS_PORT=8083
+    export LIBPROCESS_IP=192.168.33.7
 
     JAVA_OPTS="${JAVA_OPTS[*]}" exec "$AURORA_HOME/bin/aurora-scheduler" "${AURORA_FLAGS[@]}"
 
@@ -90,8 +91,8 @@ or truncating of the replicated log used by Aurora. In that case, see the docume
 
 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.
+* `-backup_interval`: The interval on which the scheduler writes local storage backups.  The default is every hour.
+* `-backup_dir`: Directory to write backups to.
 * `-max_saved_backups`: Maximum number of backups to retain before deleting the oldest backup(s).
 
 
@@ -131,6 +132,62 @@ tier configuration, you will also have to specify a file path:
     -tier_config=path/to/tiers/config.json
 
 
+## 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
@@ -150,7 +207,7 @@ By default, Thermos will not rotate the stdout/stderr logs from child processes
 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):
+`-thermos_executor_flags` argument to the Aurora scheduler):
 
     --runner-logger-mode=rotate
     --runner-rotate-log-size-mb=100
@@ -163,12 +220,13 @@ 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).
+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`
@@ -182,47 +240,7 @@ For more information on this feature please see the custom executors [documentat
 
 ## A note on increasing executor overhead
 
-Increasing executor overhead on an existing cluster, whether it be for custom executors or for thermos,
+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.
 
-## Docker containers
-In order for Aurora to launch jobs using docker containers, a few extra configuration options
-must be set.  The [docker containerizer](http://mesos.apache.org/documentation/latest/docker-containerizer/)
-must be enabled on the Mesos agents by launching them with the `--containerizers=docker,mesos` option.
-
-By default, Aurora will configure Mesos to copy the file specified in `-thermos_executor_path`
-into the container's sandbox.  If using a wrapper script to launch the thermos executor,
-specify the path to the wrapper in that argument. In addition, the path to the executor pex itself
-must be included in the `-thermos_executor_resources` option. Doing so will ensure that both the
-wrapper script and executor are correctly copied into the sandbox. Finally, 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 will not exist.
-
-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`.
-
-If you would like to run a container with a read-only filesystem, it may also be necessary to
-pass 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"

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/docs/reference/configuration.md
----------------------------------------------------------------------
diff --git a/docs/reference/configuration.md b/docs/reference/configuration.md
index ff40262..f2a0b18 100644
--- a/docs/reference/configuration.md
+++ b/docs/reference/configuration.md
@@ -451,37 +451,16 @@ tasks with the same static port allocations from being co-scheduled.
 External constraints such as agent attributes should be used to enforce such
 guarantees should they be needed.
 
-### Container Objects
-
-*Note: Both Docker and Mesos unified-container support are currently EXPERIMENTAL.*
-*Note: In order to correctly execute processes inside a job, the Docker container must have python 2.7 installed.*
 
-*Note: For private docker registry, mesos mandates the docker credential file to be named as `.dockercfg`, even though docker may create a credential file with a different name on various platforms. Also, the `.dockercfg` file needs to be copied into the sandbox using the `-thermos_executor_resources` flag, specified while starting Aurora.*
+### Container Objects
 
 Describes the container the job's processes will run inside. If not using Docker or the Mesos
 unified-container, the container can be omitted from your job config.
 
   param          | type           | description
   -----          | :----:         | -----------
-  ```docker```   | Docker         | A docker container to use.
-  ```mesos```    | Mesos          | A mesos container to use.
-
-### Docker Object
-
-  param            | type            | description
-  -----            | :----:          | -----------
-  ```image```      | String          | The name of the docker image to execute.  If the image does not exist locally it will be pulled with ```docker pull```.
-  ```parameters``` | List(Parameter) | Additional parameters to pass to the docker containerizer.
-
-### Docker Parameter Object
-
-Docker CLI parameters. This needs to be enabled by the scheduler `allow_docker_parameters` option.
-See [Docker Command Line Reference](https://docs.docker.com/reference/commandline/run/) for valid parameters.
-
-  param            | type            | description
-  -----            | :----:          | -----------
-  ```name```       | String          | The name of the docker parameter. E.g. volume
-  ```value```      | String          | The value of the parameter. E.g. /usr/local/bin:/usr/bin:rw
+  ```mesos```    | Mesos          | A native Mesos container to use.
+  ```docker```   | Docker         | A Docker container to use (via Docker engine)
 
 ### Mesos Object
 
@@ -491,8 +470,6 @@ See [Docker Command Line Reference](https://docs.docker.com/reference/commandlin
 
 ### AppcImage
 
-*Note: In order to correctly execute processes inside a job, the filesystem image must include python 2.7.*
-
 Describes an AppC filesystem image.
 
   param          | type   | description
@@ -502,8 +479,6 @@ Describes an AppC filesystem image.
 
 ### DockerImage
 
-*Note: In order to correctly execute processes inside a job, the filesystem image must include python 2.7.*
-
 Describes a Docker filesystem image.
 
   param      | type   | description
@@ -511,6 +486,28 @@ Describes a Docker filesystem image.
   ```name``` | String | The name of the docker image.
   ```tag```  | String | The tag that identifies the docker image.
 
+
+### Docker Object
+
+*Note: In order to correctly execute processes inside a job, the Docker container must have Python 2.7 installed.*
+*Note: For private docker registry, mesos mandates the docker credential file to be named as `.dockercfg`, even though docker may create a credential file with a different name on various platforms. Also, the `.dockercfg` file needs to be copied into the sandbox using the `-thermos_executor_resources` flag, specified while starting Aurora.*
+
+  param            | type            | description
+  -----            | :----:          | -----------
+  ```image```      | String          | The name of the docker image to execute.  If the image does not exist locally it will be pulled with ```docker pull```.
+  ```parameters``` | List(Parameter) | Additional parameters to pass to the Docker engine.
+
+### Docker Parameter Object
+
+Docker CLI parameters. This needs to be enabled by the scheduler `-allow_docker_parameters` option.
+See [Docker Command Line Reference](https://docs.docker.com/reference/commandline/run/) for valid parameters.
+
+  param            | type            | description
+  -----            | :----:          | -----------
+  ```name```       | String          | The name of the docker parameter. E.g. volume
+  ```value```      | String          | The value of the parameter. E.g. /usr/local/bin:/usr/bin:rw
+
+
 ### LifecycleConfig Objects
 
 *Note: The only lifecycle configuration supported is the HTTP lifecycle via the HttpLifecycleConfig.*

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/examples/jobs/docker/hello_docker.aurora
----------------------------------------------------------------------
diff --git a/examples/jobs/docker/hello_docker.aurora b/examples/jobs/docker/hello_docker.aurora
deleted file mode 100644
index d5611e6..0000000
--- a/examples/jobs/docker/hello_docker.aurora
+++ /dev/null
@@ -1,35 +0,0 @@
-#
-# 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.
-#
-
-hello_world_proc = Process(
-    name="hello_process",
-    cmdline="""
-while true; do
-    echo -n "Hello world! The time is now: " && date
-    sleep 10
-done
-""")
-
-hello_world_docker = Task(
-  name = 'hello docker',
-  processes = [hello_world_proc],
-  resources = Resources(cpu = 1, ram = 1*MB, disk=8*MB))
-
-jobs = [
-  Service(cluster = 'devcluster',
-          environment = 'devel',
-          role = 'docker-test',
-          name = 'hello_docker',
-          task = hello_world_docker,
-          container = Docker(image = 'python:2.7'))]

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/examples/jobs/hello_docker_engine.aurora
----------------------------------------------------------------------
diff --git a/examples/jobs/hello_docker_engine.aurora b/examples/jobs/hello_docker_engine.aurora
new file mode 100644
index 0000000..3c830e8
--- /dev/null
+++ b/examples/jobs/hello_docker_engine.aurora
@@ -0,0 +1,39 @@
+#
+# 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.
+#
+
+
+hello_loop = Process(
+  name = 'hello',
+  cmdline = """
+    while true; do
+      echo hello world
+      sleep 10
+    done
+  """)
+
+task = Task(
+  processes = [hello_loop],
+  resources = Resources(cpu=1, ram=1*MB, disk=8*MB)
+)
+
+jobs = [
+  Service(
+    cluster = 'devcluster',
+    environment = 'devel',
+    role = 'www-data',
+    name = 'hello_docker_engine',
+    task = task,
+    container = Docker(image = 'python:2.7')
+  )
+]

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/examples/jobs/hello_docker_image.aurora
----------------------------------------------------------------------
diff --git a/examples/jobs/hello_docker_image.aurora b/examples/jobs/hello_docker_image.aurora
new file mode 100644
index 0000000..9cd1e4a
--- /dev/null
+++ b/examples/jobs/hello_docker_image.aurora
@@ -0,0 +1,40 @@
+#
+# 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.
+#
+
+
+hello_loop = Process(
+  name = 'hello',
+  cmdline = """
+    while true; do
+      echo hello world
+      sleep 10
+    done
+  """)
+
+task = Task(
+  processes = [hello_loop],
+  resources = Resources(cpu=1, ram=1*MB, disk=8*MB)
+)
+
+jobs = [
+  Service(
+    cluster = 'devcluster',
+    environment = 'devel',
+    role = 'www-data',
+    name = 'hello_docker_image',
+    task = task,
+    container = Mesos(image=DockerImage(name='debian', tag='jessie'))
+  )
+]
+

http://git-wip-us.apache.org/repos/asf/aurora/blob/c7f710a5/examples/vagrant/aurorabuild.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/aurorabuild.sh b/examples/vagrant/aurorabuild.sh
index 9daca06..dbec54d 100755
--- a/examples/vagrant/aurorabuild.sh
+++ b/examples/vagrant/aurorabuild.sh
@@ -56,7 +56,6 @@ function build_scheduler {
   fi
   CLASSPATH_PREFIX=$hot_resources_dir ./gradlew installDist
 
-  export LD_LIBRARY_PATH=/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64/server
   sudo mkdir -p /var/db/aurora
   if sudo mesos-log initialize --path="/var/db/aurora"
   then
@@ -64,7 +63,6 @@ function build_scheduler {
   else
     echo "Replicated log initialization failed with code $? (likely already initialized)."
   fi
-  unset LD_LIBRARY_PATH
   upstart_update aurora-scheduler
 }