Return-Path: X-Original-To: apmail-aurora-commits-archive@minotaur.apache.org Delivered-To: apmail-aurora-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 37BD818091 for ; Mon, 22 Jun 2015 18:50:42 +0000 (UTC) Received: (qmail 59469 invoked by uid 500); 22 Jun 2015 18:50:42 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 59432 invoked by uid 500); 22 Jun 2015 18:50:42 -0000 Mailing-List: contact commits-help@aurora.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.apache.org Delivered-To: mailing list commits@aurora.apache.org Received: (qmail 59423 invoked by uid 99); 22 Jun 2015 18:50:42 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 22 Jun 2015 18:50:42 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0351DE10A4; Mon, 22 Jun 2015 18:50:42 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wickman@apache.org To: commits@aurora.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: aurora git commit: Do better sanitation on the client side when encountering unbound pystachio refs Date: Mon, 22 Jun 2015 18:50:42 +0000 (UTC) Repository: aurora Updated Branches: refs/heads/master cbd8a1427 -> 155947135 Do better sanitation on the client side when encountering unbound pystachio refs It's possible to define nested refs that can cause the executor to stack trace, e.g. {{derp[{{thermos.ports[http]}}]}} is perfectly valid but crashes the executor. Testing Done: Added some regression tests. Bugs closed: AURORA-739 Reviewed at https://reviews.apache.org/r/34300/ Project: http://git-wip-us.apache.org/repos/asf/aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/15594713 Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/15594713 Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/15594713 Branch: refs/heads/master Commit: 1559471350d8c7feb5e72b8d685cc82d1a68f39e Parents: cbd8a14 Author: Brian Wickman Authored: Mon Jun 22 11:50:33 2015 -0700 Committer: Brian Wickman Committed: Mon Jun 22 11:50:33 2015 -0700 ---------------------------------------------------------------------- .../python/apache/aurora/config/__init__.py | 41 +-------- .../python/apache/aurora/config/schema/base.py | 2 +- src/main/python/apache/aurora/config/thrift.py | 5 +- .../apache/aurora/executor/common/task_info.py | 58 ++++++------- src/main/python/apache/thermos/config/dsl.py | 13 --- src/main/python/apache/thermos/config/loader.py | 23 ++++- .../python/apache/aurora/client/test_config.py | 88 +++++++++++++------- .../python/apache/aurora/config/test_thrift.py | 23 ----- .../aurora/executor/common/test_task_info.py | 42 +++++++++- 9 files changed, 146 insertions(+), 149 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/aurora/config/__init__.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/config/__init__.py b/src/main/python/apache/aurora/config/__init__.py index dd2f890..d63177c 100644 --- a/src/main/python/apache/aurora/config/__init__.py +++ b/src/main/python/apache/aurora/config/__init__.py @@ -20,8 +20,7 @@ from pystachio import Empty, Environment, Ref from apache.aurora.common.aurora_job_key import AuroraJobKey from apache.aurora.config.schema.base import MesosContext -from apache.thermos.config.loader import PortExtractor, ThermosTaskWrapper -from apache.thermos.config.schema import ThermosContext +from apache.thermos.config.loader import ThermosTaskWrapper from .loader import AuroraConfigLoader from .port_resolver import PortResolver @@ -159,7 +158,7 @@ class AuroraConfig(object): }) def __init__(self, job): - self.validate_job(job) + self.validate_job(job) # first-pass validation that required fields are present self._job = self.standard_bindings(job) self._metadata = [] self.binding_dicts = defaultdict(dict) @@ -172,25 +171,6 @@ class AuroraConfig(object): def job(self): interpolated_job = self._job % self.context() - - # TODO(wickman) Once thermos is onto thrift instead of pystachio, use - # %%replacements%% instead. - # - # Typecheck against the Job, with the following free variables unwrapped at the Task level: - # - a dummy {{mesos.instance}} - # - dummy values for the {{thermos.ports}} context, to allow for their use in task_links - env = dict(mesos=Environment(instance=0)) - if interpolated_job.task_links() is not Empty: - try: - dummy_ports = dict( - (port, 31337) for port in PortExtractor.extract(interpolated_job.task_links())) - except PortExtractor.InvalidPorts as err: - raise self.InvalidConfig('Invalid port references in task_links! %s' % err) - env.update(thermos=ThermosContext(ports=dummy_ports)) - typecheck = interpolated_job.bind(Environment(env)).check() - if not typecheck.ok(): - raise self.InvalidConfig(typecheck.message()) - interpolated_job = interpolated_job(task_links=self.task_links()) try: return convert_thrift(interpolated_job, self._metadata, self.ports()) except InvalidThriftConfig as e: @@ -252,23 +232,6 @@ class AuroraConfig(object): def has_health_port(self): return "health" in ThermosTaskWrapper(self._job.task(), strict=False).ports() - def task_links(self): - # {{mesos.instance}} --> %shard_id% - # {{thermos.ports[foo]}} --> %port:foo% - task_links = self._job.task_links() - if task_links is Empty: - return task_links - _, uninterp = task_links.interpolate() - substitutions = { - Ref.from_address('mesos.instance'): '%shard_id%' - } - port_scope = Ref.from_address('thermos.ports') - for ref in uninterp: - subscope = port_scope.scoped_to(ref) - if subscope: - substitutions[ref] = '%%port:%s%%' % subscope.action().value - return task_links.bind(substitutions) - def update_config(self): return self._job.update_config() http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/aurora/config/schema/base.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/config/schema/base.py b/src/main/python/apache/aurora/config/schema/base.py index ec9f983..9a6f8a1 100644 --- a/src/main/python/apache/aurora/config/schema/base.py +++ b/src/main/python/apache/aurora/config/schema/base.py @@ -98,7 +98,7 @@ class MesosJob(Struct): production = Default(Boolean, False) priority = Default(Integer, 0) health_check_config = Default(HealthCheckConfig, HealthCheckConfig()) - task_links = Map(String, String) + task_links = Map(String, String) # Unsupported. See AURORA-739 enable_hooks = Default(Boolean, False) # enable client API hooks; from env python-list 'hooks' http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/aurora/config/thrift.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/config/thrift.py b/src/main/python/apache/aurora/config/thrift.py index 810febb..0a3e910 100644 --- a/src/main/python/apache/aurora/config/thrift.py +++ b/src/main/python/apache/aurora/config/thrift.py @@ -82,7 +82,6 @@ def task_instance_from_job(job, instance): health_check_config = HealthCheckConfig() if job.has_health_check_config(): health_check_config = job.health_check_config() - ti = MesosTaskInstance(task=job.task(), role=job.role(), health_check_config=health_check_config, @@ -91,7 +90,7 @@ def task_instance_from_job(job, instance): ti = ti(announce=job.announce()) if job.has_environment(): ti = ti(environment=job.environment()) - return ti.bind(mesos=instance_context).interpolate() + return ti.bind(mesos=instance_context) def fully_interpolated(pystachio_object, coerce_fn=lambda i: i): @@ -215,7 +214,7 @@ def convert(job, metadata=frozenset(), ports=frozenset()): task.job = key task.owner = owner task.requestedPorts = ports - task.taskLinks = not_empty_or(job.task_links(), {}) + task.taskLinks = {} # See AURORA-739 task.constraints = constraints_to_thrift(not_empty_or(job.constraints(), {})) task.container = create_container_config(job.container()) http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/aurora/executor/common/task_info.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/executor/common/task_info.py b/src/main/python/apache/aurora/executor/common/task_info.py index d110faf..0829475 100644 --- a/src/main/python/apache/aurora/executor/common/task_info.py +++ b/src/main/python/apache/aurora/executor/common/task_info.py @@ -14,7 +14,6 @@ import json -from pystachio import Ref from thrift.Thrift import TException from thrift.TSerialization import deserialize as thrift_deserialize from twitter.common import log @@ -22,16 +21,25 @@ from twitter.common import log from apache.aurora.config.port_resolver import PortResolver from apache.aurora.config.schema.base import MesosJob, MesosTaskInstance from apache.aurora.config.thrift import task_instance_from_job +from apache.thermos.config.loader import ThermosTaskValidator from gen.apache.aurora.api.ttypes import AssignedTask +class TaskInfoError(ValueError): + pass + + +class UnexpectedUnboundRefsError(TaskInfoError): + pass + + def assigned_task_from_mesos_task(task): """Deserialize AssignedTask from a launchTask task protocol buffer.""" try: assigned_task = thrift_deserialize(AssignedTask(), task.data) except (EOFError, TException) as e: - raise ValueError('Could not deserialize task! %s' % e) + raise TaskInfoError('Could not deserialize task! %s' % e) return assigned_task @@ -53,46 +61,30 @@ def mesos_task_instance_from_assigned_task(assigned_task): thermos_task = assigned_task.task.executorConfig.data if not thermos_task: - raise ValueError('Task did not have a thermos config!') + raise TaskInfoError('Task did not have a thermos config!') try: json_blob = json.loads(thermos_task) except (TypeError, ValueError) as e: - raise ValueError('Could not deserialize thermos config: %s' % e) + raise TaskInfoError('Could not deserialize thermos config: %s' % e) - # As part of the transition for MESOS-2133, we can send either a MesosTaskInstance - # or we can be sending a MesosJob. So handle both possible cases. Once everyone - # is using MesosJob, then we can begin to leverage additional information that - # becomes available such as cluster. + # TODO(wickman) Determine if there are any serialized MesosTaskInstances in the wild; + # kill this code if not. if 'instance' in json_blob: return MesosTaskInstance.json_loads(thermos_task) # This is a MesosJob - mti, refs = task_instance_from_job(MesosJob.json_loads(thermos_task), assigned_task.instanceId) - unbound_refs = [] - for ref in refs: - # If the ref is {{thermos.task_id}} or a subscope of - # {{thermos.ports}}, it currently gets bound by the Thermos Runner, - # so we must leave them unbound. - # - # {{thermos.user}} is a legacy binding which we can safely ignore. - # - # TODO(wickman) These should be rewritten by the mesos client to use - # %%style%% replacements in order to allow us to better type-check configs - # client-side. - if ref == Ref.from_address('thermos.task_id'): - continue - if Ref.subscope(Ref.from_address('thermos.ports'), ref): - continue - if ref == Ref.from_address('thermos.user'): - continue - else: - unbound_refs.append(ref) - - if len(unbound_refs) != 0: - raise ValueError('Unexpected unbound refs: %s' % ' '.join(map(str, unbound_refs))) - - return mti + task_instance = task_instance_from_job( + MesosJob.json_loads(thermos_task), assigned_task.instanceId) + + try: + ThermosTaskValidator.assert_valid_task(task_instance.task()) + ThermosTaskValidator.assert_all_refs_bound(task_instance) + except ThermosTaskValidator.InvalidTaskError as e: + raise UnexpectedUnboundRefsError('Got invalid task: %s' % e) + + task_instance, _ = task_instance.interpolate() + return task_instance def resolve_ports(mesos_task, portmap): http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/thermos/config/dsl.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/thermos/config/dsl.py b/src/main/python/apache/thermos/config/dsl.py deleted file mode 100644 index 0663a9a..0000000 --- a/src/main/python/apache/thermos/config/dsl.py +++ /dev/null @@ -1,13 +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. -# http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/main/python/apache/thermos/config/loader.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/thermos/config/loader.py b/src/main/python/apache/thermos/config/loader.py index d77ab9a..14a8759 100644 --- a/src/main/python/apache/thermos/config/loader.py +++ b/src/main/python/apache/thermos/config/loader.py @@ -14,6 +14,7 @@ import json import os +import random import re import textwrap @@ -22,7 +23,7 @@ from pystachio.config import Config from twitter.common.dirutil import safe_open from apache.thermos.common.planner import TaskPlanner -from apache.thermos.config.schema import Task +from apache.thermos.config.schema import Task, ThermosContext class PortExtractor(object): @@ -115,6 +116,7 @@ class ThermosTaskValidator(object): cls.assert_valid_names(task) cls.assert_typecheck(task) cls.assert_valid_plan(task) + cls.assert_all_refs_bound(task) @classmethod def assert_valid_plan(cls, task): @@ -155,6 +157,25 @@ class ThermosTaskValidator(object): raise cls.InvalidTaskError('Task differs from on disk copy: %r vs %r' % ( task_on_disk.task if task_on_disk else None, task)) + @classmethod + def assert_all_refs_bound(cls, task): + port_names = PortExtractor.extract(task) + + # Create fake bindings and make sure that there are no unbound refs afterwards. If + # there are unbound refs that could indicate improper scoping e.g. + # {{array[{{mesos.instance}}]}} which is disallowed. + thermos_bindings = ThermosContext( + task_id='dummy_task_id', + user='dummy_user', + ports=dict((name, random.randrange(30000, 40000)) for name in port_names), + ) + task_instance, unbindable_refs = (task % dict(thermos=thermos_bindings)).interpolate() + + if len(unbindable_refs) != 0: + raise cls.InvalidTaskError( + 'Unexpected unbound refs: %s. Make sure you are not nesting template variables.' + % ' '.join(map(str, unbindable_refs))) + class ThermosConfigLoader(object): SCHEMA = textwrap.dedent(""" http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/test/python/apache/aurora/client/test_config.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/test_config.py b/src/test/python/apache/aurora/client/test_config.py index c567797..14a7435 100644 --- a/src/test/python/apache/aurora/client/test_config.py +++ b/src/test/python/apache/aurora/client/test_config.py @@ -13,11 +13,13 @@ # import os +from io import BytesIO import pytest -from twitter.common.contextutil import temporary_dir, temporary_file +from twitter.common.contextutil import temporary_dir from apache.aurora.client import config +from apache.aurora.client.config import get_config as get_aurora_config from apache.aurora.config import AuroraConfig from apache.aurora.config.loader import AuroraConfigLoader from apache.aurora.config.schema.base import ( @@ -36,10 +38,10 @@ HELLO_WORLD = Job( role = 'john_doe', cluster = 'test-cluster', environment = 'test', - %s + %(announce)s task = Task( name = 'main', - processes = [Process(name = 'hello_world', cmdline = 'echo {{thermos.ports[http]}}')], + processes = [Process(name = 'hello_world', cmdline = '%(cmdline)s')], resources = Resources(cpu = 0.1, ram = 64 * MB, disk = 64 * MB), ) ) @@ -53,44 +55,66 @@ include(%s) """ -MESOS_CONFIG_WITH_ANNOUNCE_1 = MESOS_CONFIG_BASE % 'announce = Announcer(primary_port="http"),' -MESOS_CONFIG_WITH_ANNOUNCE_2 = MESOS_CONFIG_BASE % ( - '''announce = Announcer( +MESOS_CONFIG_WITH_ANNOUNCE_1 = MESOS_CONFIG_BASE % { + 'cmdline': 'echo {{thermos.ports[http]}}', + 'announce': 'announce = Announcer(primary_port="http"),'} +MESOS_CONFIG_WITH_ANNOUNCE_2 = MESOS_CONFIG_BASE % { + 'cmdline': 'echo {{thermos.ports[http]}}', + 'announce': '''announce = Announcer( primary_port = "http", portmap = {"aurora": "http"}), - ''') -MESOS_CONFIG_WITH_INVALID_STATS = MESOS_CONFIG_BASE % ( - 'announce = Announcer(primary_port="http", stats_port="blah"),') -MESOS_CONFIG_WITHOUT_ANNOUNCE = MESOS_CONFIG_BASE % '' +'''} +MESOS_CONFIG_WITH_INVALID_STATS = MESOS_CONFIG_BASE % { + 'cmdline': 'echo {{thermos.ports[http]}}', + 'announce': 'announce = Announcer(primary_port="http", stats_port="blah"),'} +MESOS_CONFIG_WITHOUT_ANNOUNCE = MESOS_CONFIG_BASE % { + 'cmdline': 'echo {{thermos.ports[http]}}', + 'announce': '' +} def test_get_config_announces(): - for good_config in (MESOS_CONFIG_WITH_ANNOUNCE_1, MESOS_CONFIG_WITH_ANNOUNCE_2, - MESOS_CONFIG_WITHOUT_ANNOUNCE): - with temporary_file() as fp: - fp.write(good_config) - fp.flush() + for good_config in ( + MESOS_CONFIG_WITH_ANNOUNCE_1, + MESOS_CONFIG_WITH_ANNOUNCE_2, + MESOS_CONFIG_WITHOUT_ANNOUNCE): - fp.seek(0) - config.get_config('hello_world', fp) + bio = BytesIO(good_config) + get_aurora_config('hello_world', bio).job() + + +def test_get_config_with_broken_subscopes(): + bad_config = MESOS_CONFIG_BASE % { + 'cmdline': 'echo {{hello[{{thermos.ports[http]}}]}}', + 'announce': '', + } + bio = BytesIO(bad_config) + with pytest.raises(AuroraConfig.InvalidConfig) as cm: + get_aurora_config('hello_world', bio).job() + assert 'Unexpected unbound refs' in str(cm.value.message) def test_get_config_select(): - with temporary_file() as fp: - fp.write(MESOS_CONFIG_BASE % '') - fp.flush() + bio = BytesIO(MESOS_CONFIG_WITHOUT_ANNOUNCE) + + get_aurora_config( + 'hello_world', + bio, + select_env='test', + select_role='john_doe', + select_cluster='test-cluster').job() + + bio.seek(0) - fp.seek(0) - config.get_config( - 'hello_world', fp, select_env='test', - select_role='john_doe', select_cluster='test-cluster') + with pytest.raises(ValueError) as cm: + get_aurora_config( + 'hello_world', + bio, + select_env='staging42', + select_role='moua', + select_cluster='test-cluster').job() - fp.seek(0) - with pytest.raises(ValueError) as cm: - config.get_config( - 'hello_world', fp, select_env='staging42', - select_role='moua', select_cluster='test-cluster') - assert 'test-cluster/john_doe/test/hello_world' in str(cm.value.message) + assert 'test-cluster/john_doe/test/hello_world' in str(cm.value.message) def test_include(): @@ -107,11 +131,11 @@ def test_include(): ("", """'%s'""" % hello_mesos_fname)) hello_include_fname_fp.flush() - config.get_config('hello_world', hello_include_fname_path) + get_aurora_config('hello_world', hello_include_fname_path) hello_include_fname_fp.seek(0) with pytest.raises(AuroraConfigLoader.InvalidConfigError): - config.get_config('hello_world', hello_include_fname_fp) + get_aurora_config('hello_world', hello_include_fname_fp) def test_environment_names(): http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/test/python/apache/aurora/config/test_thrift.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/config/test_thrift.py b/src/test/python/apache/aurora/config/test_thrift.py index 654c0b5..f48ac88 100644 --- a/src/test/python/apache/aurora/config/test_thrift.py +++ b/src/test/python/apache/aurora/config/test_thrift.py @@ -16,8 +16,6 @@ import getpass import re import pytest -from pystachio import Map, String -from pystachio.naming import frozendict from apache.aurora.config import AuroraConfig from apache.aurora.config.schema.base import HealthCheckConfig, Job, SimpleTask @@ -133,27 +131,6 @@ def test_config_with_bad_resources(): convert_pystachio_to_thrift(HELLO_WORLD(task=hwtask(resources=resource))) -def test_config_with_task_links(): - tl = Map(String, String) - unresolved_tl = { - 'foo': 'http://%host%:{{thermos.ports[foo]}}', - 'bar': 'http://%host%:{{thermos.ports[bar]}}/{{mesos.instance}}', - } - resolved_tl = { - 'foo': 'http://%host%:%port:foo%', - 'bar': 'http://%host%:%port:bar%/%shard_id%' - } - aurora_config = AuroraConfig(HELLO_WORLD(task_links=tl(unresolved_tl))) - assert aurora_config.task_links() == tl(resolved_tl) - assert aurora_config.job().taskConfig.taskLinks == frozendict(resolved_tl) - - bad_tl = { - 'foo': '{{thermos.ports.bad}}' - } - with pytest.raises(AuroraConfig.InvalidConfig): - AuroraConfig(HELLO_WORLD(task_links=tl(bad_tl))).job() - - def test_unbound_references(): def job_command(cmdline): return AuroraConfig(HELLO_WORLD(task=SimpleTask('hello_world', cmdline))).raw() http://git-wip-us.apache.org/repos/asf/aurora/blob/15594713/src/test/python/apache/aurora/executor/common/test_task_info.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/executor/common/test_task_info.py b/src/test/python/apache/aurora/executor/common/test_task_info.py index 102ba53..29da828 100644 --- a/src/test/python/apache/aurora/executor/common/test_task_info.py +++ b/src/test/python/apache/aurora/executor/common/test_task_info.py @@ -14,9 +14,21 @@ import pytest -from apache.aurora.executor.common.task_info import mesos_task_instance_from_assigned_task +from apache.aurora.config.schema.base import Process +from apache.aurora.executor.common.task_info import ( + mesos_task_instance_from_assigned_task, + TaskInfoError, + UnexpectedUnboundRefsError +) -from .fixtures import BASE_MTI, HELLO_WORLD, HELLO_WORLD_MTI, HELLO_WORLD_UNBOUND, MESOS_JOB +from .fixtures import ( + BASE_MTI, + BASE_TASK, + HELLO_WORLD, + HELLO_WORLD_MTI, + HELLO_WORLD_UNBOUND, + MESOS_JOB +) from gen.apache.aurora.api.ttypes import AssignedTask, ExecutorConfig, TaskConfig @@ -34,11 +46,33 @@ def test_deserialize_thermos_task(): def test_deserialize_thermos_task_unbound_refs(): + # test unbound {{standard}} refs task_config = TaskConfig( executorConfig=ExecutorConfig( name='thermos', data=MESOS_JOB(task=HELLO_WORLD_UNBOUND).json_dumps())) assigned_task = AssignedTask(task=task_config, instanceId=0) - with pytest.raises(ValueError) as execinfo: + with pytest.raises(TaskInfoError) as execinfo: mesos_task_instance_from_assigned_task(assigned_task) - assert execinfo.value.message == "Unexpected unbound refs: {{unbound_cmd}} {{unbound}}" + assert "Unexpected unbound refs: {{unbound_cmd}} {{unbound}}" in execinfo.value.message + + # test bound unscoped refs, valid case. + task = BASE_TASK( + name='task_name', + processes=[Process(name='process_name', cmdline='echo {{thermos.ports[health]}}')]) + task_config = TaskConfig( + executorConfig=ExecutorConfig(name='thermos', data=MESOS_JOB(task=task).json_dumps())) + assigned_task = AssignedTask(task=task_config, instanceId=0) + assert mesos_task_instance_from_assigned_task(assigned_task) is not None + + # test unbound unscoped refs + for cmdline in ( + 'echo {{hello_{{thermos.ports[health]}}}}', + 'echo {{hello_{{thermos.user_id}}}}'): + task = BASE_TASK(name='task_name', processes=[Process(name='process_name', cmdline=cmdline)]) + task_config = TaskConfig( + executorConfig=ExecutorConfig(name='thermos', data=MESOS_JOB(task=task).json_dumps())) + assigned_task = AssignedTask(task=task_config, instanceId=0) + + with pytest.raises(UnexpectedUnboundRefsError): + mesos_task_instance_from_assigned_task(assigned_task)