You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wi...@apache.org on 2014/12/02 01:11:59 UTC

incubator-aurora git commit: Move thermos_runner out of the apache.aurora.executor package.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 88baad6a0 -> 6948dc998


Move thermos_runner out of the apache.aurora.executor package.

The drawback of having it in the apache.aurora.executor package is that if
we want to build a thermos_runner binary from the apache.aurora.executor
distribution, it transitively pulls in mesos.native, making the
thermos_runner pex like 70MB.  thermos_runner.pex really only has
dependencies within apache.thermos and the thermos_runner exit-code contract
is a bit more clear now, so putting it into apache.thermos makes a bit more
sense now.  it also makes thermos_runner.pex 566k instead of 70MB.  it also
fixes AURORA-823.

Testing Done:
build-support/jenkins/build.sh

Bugs closed: AURORA-823

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


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

Branch: refs/heads/master
Commit: 6948dc998ce343f0c628fa958d7ecd97bca040b2
Parents: 88baad6
Author: Brian Wickman <wi...@apache.org>
Authored: Mon Dec 1 16:11:13 2014 -0800
Committer: Brian Wickman <wi...@apache.org>
Committed: Mon Dec 1 16:11:13 2014 -0800

----------------------------------------------------------------------
 examples/vagrant/aurorabuild.sh                 |   2 +-
 src/main/python/apache/aurora/executor/BUILD    |  19 --
 .../python/apache/aurora/executor/bin/BUILD     |  11 --
 .../aurora/executor/bin/thermos_runner_main.py  |  26 ---
 .../apache/aurora/executor/thermos_runner.py    | 188 ------------------
 .../apache/aurora/executor/thermos_statuses.py  |  19 --
 .../aurora/executor/thermos_task_runner.py      |  14 +-
 src/main/python/apache/thermos/BUILD            |   1 +
 src/main/python/apache/thermos/bin/BUILD        |  21 ++
 .../python/apache/thermos/bin/thermos_runner.py | 194 +++++++++++++++++++
 src/main/python/apache/thermos/common/BUILD     |   6 +
 .../python/apache/thermos/common/statuses.py    |  19 ++
 .../aurora/executor/test_thermos_executor.py    |   2 +-
 .../aurora/executor/test_thermos_task_runner.py |   6 +-
 14 files changed, 253 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/examples/vagrant/aurorabuild.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/aurorabuild.sh b/examples/vagrant/aurorabuild.sh
index 4d51cee..69983d0 100755
--- a/examples/vagrant/aurorabuild.sh
+++ b/examples/vagrant/aurorabuild.sh
@@ -65,7 +65,7 @@ function build_scheduler {
 function build_executor {
   ./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/bin:thermos_runner
 
   # Package runner within executor.
   python <<EOF

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/BUILD b/src/main/python/apache/aurora/executor/BUILD
index 3a2d4f8..72d1ec5 100644
--- a/src/main/python/apache/aurora/executor/BUILD
+++ b/src/main/python/apache/aurora/executor/BUILD
@@ -15,11 +15,6 @@
 import os
 
 python_library(
-  name = 'thermos_statuses',
-  sources = ['thermos_statuses.py'],
-)
-
-python_library(
   name = 'thermos_task_runner',
   sources = ['thermos_task_runner.py'],
   dependencies = [
@@ -35,7 +30,6 @@ python_library(
     'src/main/python/apache/aurora/executor/common:status_checker',
     'src/main/python/apache/aurora/executor/common:task_info',
     'src/main/python/apache/aurora/executor/common:task_runner',
-    ':thermos_statuses',
   ]
 )
 
@@ -99,18 +93,6 @@ python_library(
   ]
 )
 
-python_library(
-  name = 'thermos_runner',
-  sources = ['thermos_runner.py'],
-  dependencies = [
-    '3rdparty/python:twitter.common.app',
-    '3rdparty/python:twitter.common.log',
-    'src/main/python/apache/thermos/common:planner',
-    'src/main/python/apache/thermos/config:schema',
-    'src/main/python/apache/thermos/core',
-    ':thermos_statuses',
-  ],
-)
 
 python_library(
   name = 'gc_executor',
@@ -155,6 +137,5 @@ python_library(
   ).with_binaries(
     gc_executor = 'src/main/python/apache/aurora/executor/bin:gc_executor',
     thermos_executor = 'src/main/python/apache/aurora/executor/bin:thermos_executor',
-    thermos_runner = 'src/main/python/apache/aurora/executor/bin:thermos_runner',
   )
 )

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/bin/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/bin/BUILD b/src/main/python/apache/aurora/executor/bin/BUILD
index a41cf95..0434c7b 100644
--- a/src/main/python/apache/aurora/executor/bin/BUILD
+++ b/src/main/python/apache/aurora/executor/bin/BUILD
@@ -49,14 +49,3 @@ python_binary(
     'src/main/python/apache/aurora/executor:gc_executor',
   ]
 )
-
-python_binary(
-  name = 'thermos_runner',
-  source = 'thermos_runner_main.py',
-  entry_point = 'apache.aurora.executor.bin.thermos_runner_main:proxy_main',
-  dependencies = [
-    '3rdparty/python:twitter.common.app',
-    '3rdparty/python:twitter.common.log',
-    'src/main/python/apache/aurora/executor:thermos_runner',
-  ],
-)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/bin/thermos_runner_main.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/bin/thermos_runner_main.py b/src/main/python/apache/aurora/executor/bin/thermos_runner_main.py
deleted file mode 100644
index f61caf0..0000000
--- a/src/main/python/apache/aurora/executor/bin/thermos_runner_main.py
+++ /dev/null
@@ -1,26 +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.
-#
-
-from twitter.common import app
-from twitter.common.log.options import LogOptions
-
-from apache.aurora.executor.thermos_runner import proxy_main as runner_proxy_main
-
-LogOptions.set_simple(True)
-
-
-def proxy_main():
-  main = runner_proxy_main  # noqa
-
-  app.main()

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/thermos_runner.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/thermos_runner.py b/src/main/python/apache/aurora/executor/thermos_runner.py
deleted file mode 100644
index e10f438..0000000
--- a/src/main/python/apache/aurora/executor/thermos_runner.py
+++ /dev/null
@@ -1,188 +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.
-#
-
-import functools
-import getpass
-import os
-import pwd
-import signal
-import sys
-import traceback
-
-from twitter.common import app, log
-
-from apache.thermos.common.options import add_port_to
-from apache.thermos.common.planner import TaskPlanner
-from apache.thermos.config.loader import ThermosConfigLoader
-from apache.thermos.core.process import Process
-from apache.thermos.core.runner import TaskRunner
-
-from .thermos_statuses import (
-    INTERNAL_ERROR,
-    INVALID_TASK,
-    TERMINAL_TASK,
-    UNKNOWN_ERROR,
-    UNKNOWN_USER
-)
-
-app.add_option(
-    "--thermos_json",
-    dest="thermos_json",
-    default=None,
-    help="read a thermos Task from a serialized json blob")
-
-
-app.add_option(
-    "--sandbox",
-    dest="sandbox",
-    metavar="PATH",
-    default=None,
-    help="the sandbox in which this task should run")
-
-
-app.add_option(
-     "--checkpoint_root",
-     dest="checkpoint_root",
-     metavar="PATH",
-     default=None,
-     help="the path where we will store checkpoints")
-
-
-app.add_option(
-     "--task_id",
-     dest="task_id",
-     metavar="STRING",
-     default=None,
-     help="The id to which this task should be bound, created if it does not exist.")
-
-
-app.add_option(
-     "--setuid",
-     dest="setuid",
-     metavar="USER",
-     default=None,
-     help="setuid tasks to this user, requires superuser privileges.")
-
-
-app.add_option(
-     "--enable_chroot",
-     dest="chroot",
-     default=False,
-     action='store_true',
-     help="chroot tasks to the sandbox before executing them.")
-
-
-app.add_option(
-     "--port",
-     type='string',
-     nargs=1,
-     action='callback',
-     callback=add_port_to('prebound_ports'),
-     dest='prebound_ports',
-     default={},
-     metavar="NAME:PORT",
-     help="bind a numbered port PORT to name NAME")
-
-
-def get_task_from_options(opts):
-  tasks = ThermosConfigLoader.load_json(opts.thermos_json)
-  if len(tasks.tasks()) == 0:
-    app.error("No tasks specified!")
-  if len(tasks.tasks()) > 1:
-    app.error("Multiple tasks in config but no task name specified!")
-  task = tasks.tasks()[0]
-  if not task.task.check().ok():
-    app.error(task.task.check().message())
-  return task
-
-
-def runner_teardown(runner, sig=signal.SIGUSR1, frame=None):
-  """Destroy runner on SIGUSR1 (kill) or SIGUSR2 (lose)"""
-  op = 'kill' if sig == signal.SIGUSR1 else 'lose'
-  log.info('Thermos runner got signal %s, shutting down.' % sig)
-  log.info('Interrupted frame:')
-  if frame:
-    for line in ''.join(traceback.format_stack(frame)).splitlines():
-      log.info(line)
-  runner.close_ckpt()
-  log.info('Calling runner.%s()' % op)
-  getattr(runner, op)()
-  sys.exit(0)
-
-
-class CappedTaskPlanner(TaskPlanner):
-  TOTAL_RUN_LIMIT = 100
-
-
-def proxy_main(args, opts):
-  assert opts.thermos_json and os.path.exists(opts.thermos_json)
-  assert opts.sandbox
-  assert opts.checkpoint_root
-
-  thermos_task = get_task_from_options(opts)
-  prebound_ports = opts.prebound_ports
-  missing_ports = set(thermos_task.ports()) - set(prebound_ports)
-
-  if missing_ports:
-    log.error('ERROR!  Unbound ports: %s' % ' '.join(port for port in missing_ports))
-    sys.exit(INTERNAL_ERROR)
-
-  if opts.setuid:
-    user = opts.setuid
-  else:
-    user = getpass.getuser()
-
-  # if we cannot get the uid, this is an unknown user and we should fail
-  try:
-    pwd.getpwnam(user).pw_uid
-  except KeyError:
-    log.error('Unknown user: %s' % user)
-    sys.exit(UNKNOWN_USER)
-
-  task_runner = TaskRunner(
-      thermos_task.task,
-      opts.checkpoint_root,
-      opts.sandbox,
-      task_id=opts.task_id,
-      user=opts.setuid,
-      portmap=prebound_ports,
-      chroot=opts.chroot,
-      planner_class=CappedTaskPlanner
-  )
-
-  for sig in (signal.SIGUSR1, signal.SIGUSR2):
-    signal.signal(sig, functools.partial(runner_teardown, task_runner))
-
-  try:
-    task_runner.run()
-  except TaskRunner.InternalError as err:
-    log.error('Internal error: %s' % err)
-    sys.exit(INTERNAL_ERROR)
-  except TaskRunner.InvalidTask as err:
-    log.error('Invalid task: %s' % err)
-    sys.exit(INVALID_TASK)
-  except TaskRunner.StateError as err:
-    log.error('Checkpoint error: %s' % err)
-    sys.exit(TERMINAL_TASK)
-  except Process.UnknownUserError as err:
-    log.error('User ceased to exist: %s' % err)
-    sys.exit(UNKNOWN_USER)
-  except KeyboardInterrupt:
-    log.info('Caught ^C, tearing down runner.')
-    runner_teardown(task_runner)
-  except Exception as e:
-    log.error('Unknown exception: %s' % e)
-    for line in traceback.format_exc().splitlines():
-      log.error(line)
-    sys.exit(UNKNOWN_ERROR)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/thermos_statuses.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/thermos_statuses.py b/src/main/python/apache/aurora/executor/thermos_statuses.py
deleted file mode 100644
index 85675ce..0000000
--- a/src/main/python/apache/aurora/executor/thermos_statuses.py
+++ /dev/null
@@ -1,19 +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.
-#
-
-INTERNAL_ERROR = 101
-INVALID_TASK = 102
-UNKNOWN_ERROR = 103
-UNKNOWN_USER = 104
-TERMINAL_TASK = 105

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/aurora/executor/thermos_task_runner.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/thermos_task_runner.py b/src/main/python/apache/aurora/executor/thermos_task_runner.py
index 9a2faa0..5e4bd65 100644
--- a/src/main/python/apache/aurora/executor/thermos_task_runner.py
+++ b/src/main/python/apache/aurora/executor/thermos_task_runner.py
@@ -30,6 +30,13 @@ from twitter.common.quantity import Amount, Time
 
 from apache.aurora.common.http_signaler import HttpSignaler
 from apache.thermos.common.path import TaskPath
+from apache.thermos.common.statuses import (
+    INTERNAL_ERROR,
+    INVALID_TASK,
+    TERMINAL_TASK,
+    UNKNOWN_ERROR,
+    UNKNOWN_USER
+)
 from apache.thermos.config.loader import ThermosTaskWrapper
 from apache.thermos.core import runner as core
 from apache.thermos.monitoring.monitor import TaskMonitor
@@ -37,13 +44,6 @@ from apache.thermos.monitoring.monitor import TaskMonitor
 from .common.status_checker import StatusResult
 from .common.task_info import mesos_task_instance_from_assigned_task, resolve_ports
 from .common.task_runner import TaskError, TaskRunner, TaskRunnerProvider
-from .thermos_statuses import (
-    INTERNAL_ERROR,
-    INVALID_TASK,
-    TERMINAL_TASK,
-    UNKNOWN_ERROR,
-    UNKNOWN_USER
-)
 
 from gen.apache.thermos.ttypes import TaskState
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/thermos/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/thermos/BUILD b/src/main/python/apache/thermos/BUILD
index 0457464..c62202c 100644
--- a/src/main/python/apache/thermos/BUILD
+++ b/src/main/python/apache/thermos/BUILD
@@ -28,5 +28,6 @@ python_library(
     descripton = 'The Apache Aurora Thermos runtime.',
   ).with_binaries(
     thermos = 'src/main/python/apache/thermos/bin:thermos',
+    thermos_runner = 'src/main/python/apache/thermos/bin:thermos_runner',
   )
 )

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/thermos/bin/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/thermos/bin/BUILD b/src/main/python/apache/thermos/bin/BUILD
index a1d6e52..1a002da 100644
--- a/src/main/python/apache/thermos/bin/BUILD
+++ b/src/main/python/apache/thermos/bin/BUILD
@@ -36,3 +36,24 @@ python_binary(
     'api/src/main/thrift/org/apache/thermos:py-thrift',
   ],
 )
+
+python_library(
+  name = 'thermos_runner_source',
+  sources = ['thermos_runner.py'],
+  dependencies = [
+    'src/main/python/apache/thermos/common:planner',
+    'src/main/python/apache/thermos/common:statuses',
+    'src/main/python/apache/thermos/config:schema',
+    'src/main/python/apache/thermos/core',
+  ],
+)
+
+python_binary(
+  name = 'thermos_runner',
+  entry_point = 'apache.thermos.bin.thermos_runner',
+  dependencies = [
+    '3rdparty/python:twitter.common.app',
+    '3rdparty/python:twitter.common.log',
+    ':thermos_runner_source',
+  ],
+)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/thermos/bin/thermos_runner.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/thermos/bin/thermos_runner.py b/src/main/python/apache/thermos/bin/thermos_runner.py
new file mode 100644
index 0000000..647de27
--- /dev/null
+++ b/src/main/python/apache/thermos/bin/thermos_runner.py
@@ -0,0 +1,194 @@
+#
+# 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.
+#
+
+import functools
+import getpass
+import os
+import pwd
+import signal
+import sys
+import traceback
+
+from twitter.common import app, log
+
+from apache.thermos.common.options import add_port_to
+from apache.thermos.common.planner import TaskPlanner
+from apache.thermos.common.statuses import (
+    INTERNAL_ERROR,
+    INVALID_TASK,
+    TERMINAL_TASK,
+    UNKNOWN_ERROR,
+    UNKNOWN_USER
+)
+from apache.thermos.config.loader import ThermosConfigLoader
+from apache.thermos.core.process import Process
+from apache.thermos.core.runner import TaskRunner
+
+app.add_option(
+    "--thermos_json",
+    dest="thermos_json",
+    default=None,
+    help="read a thermos Task from a serialized json blob")
+
+
+app.add_option(
+    "--sandbox",
+    dest="sandbox",
+    metavar="PATH",
+    default=None,
+    help="the sandbox in which this task should run")
+
+
+app.add_option(
+     "--checkpoint_root",
+     dest="checkpoint_root",
+     metavar="PATH",
+     default=None,
+     help="the path where we will store checkpoints")
+
+
+app.add_option(
+     "--task_id",
+     dest="task_id",
+     metavar="STRING",
+     default=None,
+     help="The id to which this task should be bound, created if it does not exist.")
+
+
+app.add_option(
+     "--setuid",
+     dest="setuid",
+     metavar="USER",
+     default=None,
+     help="setuid tasks to this user, requires superuser privileges.")
+
+
+app.add_option(
+     "--enable_chroot",
+     dest="chroot",
+     default=False,
+     action='store_true',
+     help="chroot tasks to the sandbox before executing them.")
+
+
+app.add_option(
+     "--port",
+     type='string',
+     nargs=1,
+     action='callback',
+     callback=add_port_to('prebound_ports'),
+     dest='prebound_ports',
+     default={},
+     metavar="NAME:PORT",
+     help="bind a numbered port PORT to name NAME")
+
+
+def get_task_from_options(opts):
+  tasks = ThermosConfigLoader.load_json(opts.thermos_json)
+  if len(tasks.tasks()) == 0:
+    app.error("No tasks specified!")
+  if len(tasks.tasks()) > 1:
+    app.error("Multiple tasks in config but no task name specified!")
+  task = tasks.tasks()[0]
+  if not task.task.check().ok():
+    app.error(task.task.check().message())
+  return task
+
+
+def runner_teardown(runner, sig=signal.SIGUSR1, frame=None):
+  """Destroy runner on SIGUSR1 (kill) or SIGUSR2 (lose)"""
+  op = 'kill' if sig == signal.SIGUSR1 else 'lose'
+  log.info('Thermos runner got signal %s, shutting down.' % sig)
+  log.info('Interrupted frame:')
+  if frame:
+    for line in ''.join(traceback.format_stack(frame)).splitlines():
+      log.info(line)
+  runner.close_ckpt()
+  log.info('Calling runner.%s()' % op)
+  getattr(runner, op)()
+  sys.exit(0)
+
+
+class CappedTaskPlanner(TaskPlanner):
+  TOTAL_RUN_LIMIT = 100
+
+
+def proxy_main(args, opts):
+  assert opts.thermos_json and os.path.exists(opts.thermos_json)
+  assert opts.sandbox
+  assert opts.checkpoint_root
+
+  thermos_task = get_task_from_options(opts)
+  prebound_ports = opts.prebound_ports
+  missing_ports = set(thermos_task.ports()) - set(prebound_ports)
+
+  if missing_ports:
+    log.error('ERROR!  Unbound ports: %s' % ' '.join(port for port in missing_ports))
+    sys.exit(INTERNAL_ERROR)
+
+  if opts.setuid:
+    user = opts.setuid
+  else:
+    user = getpass.getuser()
+
+  # if we cannot get the uid, this is an unknown user and we should fail
+  try:
+    pwd.getpwnam(user).pw_uid
+  except KeyError:
+    log.error('Unknown user: %s' % user)
+    sys.exit(UNKNOWN_USER)
+
+  task_runner = TaskRunner(
+      thermos_task.task,
+      opts.checkpoint_root,
+      opts.sandbox,
+      task_id=opts.task_id,
+      user=opts.setuid,
+      portmap=prebound_ports,
+      chroot=opts.chroot,
+      planner_class=CappedTaskPlanner
+  )
+
+  for sig in (signal.SIGUSR1, signal.SIGUSR2):
+    signal.signal(sig, functools.partial(runner_teardown, task_runner))
+
+  try:
+    task_runner.run()
+  except TaskRunner.InternalError as err:
+    log.error('Internal error: %s' % err)
+    sys.exit(INTERNAL_ERROR)
+  except TaskRunner.InvalidTask as err:
+    log.error('Invalid task: %s' % err)
+    sys.exit(INVALID_TASK)
+  except TaskRunner.StateError as err:
+    log.error('Checkpoint error: %s' % err)
+    sys.exit(TERMINAL_TASK)
+  except Process.UnknownUserError as err:
+    log.error('User ceased to exist: %s' % err)
+    sys.exit(UNKNOWN_USER)
+  except KeyboardInterrupt:
+    log.info('Caught ^C, tearing down runner.')
+    runner_teardown(task_runner)
+  except Exception as e:
+    log.error('Unknown exception: %s' % e)
+    for line in traceback.format_exc().splitlines():
+      log.error(line)
+    sys.exit(UNKNOWN_ERROR)
+
+
+def main(args, opts):
+  return proxy_main(args, opts)
+
+
+app.main()

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/thermos/common/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/thermos/common/BUILD b/src/main/python/apache/thermos/common/BUILD
index 24d7b6e..1fb9af6 100644
--- a/src/main/python/apache/thermos/common/BUILD
+++ b/src/main/python/apache/thermos/common/BUILD
@@ -44,12 +44,18 @@ python_library(
 )
 
 python_library(
+  name = 'statuses',
+  sources = ['statuses.py'],
+)
+
+python_library(
   name = 'common',
   dependencies = [
     ':ckpt',
     ':options',
     ':path',
     ':planner',
+    ':statuses',
   ],
   provides = setup_py(
     name = 'apache.thermos.common',

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/main/python/apache/thermos/common/statuses.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/thermos/common/statuses.py b/src/main/python/apache/thermos/common/statuses.py
new file mode 100644
index 0000000..85675ce
--- /dev/null
+++ b/src/main/python/apache/thermos/common/statuses.py
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+INTERNAL_ERROR = 101
+INVALID_TASK = 102
+UNKNOWN_ERROR = 103
+UNKNOWN_USER = 104
+TERMINAL_TASK = 105

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/test/python/apache/aurora/executor/test_thermos_executor.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/test_thermos_executor.py b/src/test/python/apache/aurora/executor/test_thermos_executor.py
index 16a4011..503e62f 100644
--- a/src/test/python/apache/aurora/executor/test_thermos_executor.py
+++ b/src/test/python/apache/aurora/executor/test_thermos_executor.py
@@ -270,7 +270,7 @@ class TestThermosExecutor(object):
     log.init('executor_logger')
     if not cls.PANTS_BUILT and 'SKIP_PANTS_BUILD' not in os.environ:
       assert subprocess.call(["./pants",
-          "src/main/python/apache/aurora/executor/bin:thermos_runner"]) == 0
+          "src/main/python/apache/thermos/bin:thermos_runner"]) == 0
       cls.PANTS_BUILT = True
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/6948dc99/src/test/python/apache/aurora/executor/test_thermos_task_runner.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/test_thermos_task_runner.py b/src/test/python/apache/aurora/executor/test_thermos_task_runner.py
index 4fc9d4b..f8cf9fc 100644
--- a/src/test/python/apache/aurora/executor/test_thermos_task_runner.py
+++ b/src/test/python/apache/aurora/executor/test_thermos_task_runner.py
@@ -30,14 +30,14 @@ from twitter.common.quantity import Amount, Time
 
 from apache.aurora.config.schema.base import MB, MesosTaskInstance, Process, Resources, Task
 from apache.aurora.executor.common.sandbox import DirectorySandbox
-from apache.aurora.executor.thermos_statuses import (
+from apache.aurora.executor.thermos_task_runner import ThermosTaskRunner
+from apache.thermos.common.statuses import (
     INTERNAL_ERROR,
     INVALID_TASK,
     TERMINAL_TASK,
     UNKNOWN_ERROR,
     UNKNOWN_USER
 )
-from apache.aurora.executor.thermos_task_runner import ThermosTaskRunner
 
 from gen.apache.thermos.ttypes import TaskState
 
@@ -66,7 +66,7 @@ class TestThermosTaskRunnerIntegration(object):
     log.init('executor_logger')
     if not cls.PANTS_BUILT and 'SKIP_PANTS_BUILD' not in os.environ:
       assert subprocess.call(["./pants",
-          "src/main/python/apache/aurora/executor/bin:thermos_runner"]) == 0
+          "src/main/python/apache/thermos/bin:thermos_runner"]) == 0
       cls.PANTS_BUILT = True
 
   @classmethod