diff --git a/src/python/pants/backend/python/tasks/pytest_prep.py b/src/python/pants/backend/python/tasks/pytest_prep.py index 1d790743575..780fcfbb04c 100644 --- a/src/python/pants/backend/python/tasks/pytest_prep.py +++ b/src/python/pants/backend/python/tasks/pytest_prep.py @@ -5,6 +5,8 @@ from __future__ import (absolute_import, division, generators, nested_scopes, print_function, unicode_literals, with_statement) +import os + from pex.pex_info import PexInfo from pants.backend.python.subsystems.pytest import PyTest @@ -12,13 +14,33 @@ class PytestPrep(PythonExecutionTaskBase): - """Prepares a pex binary for the current test context with py.test as its entry-point.""" + """Prepares a PEX binary for the current test context with `py.test` as its entry-point.""" + + class PytestBinary(object): + """A `py.test` PEX binary with an embedded default (empty) `pytest.ini` config file.""" + + def __init__(self, pex): + self._pex = pex + + @property + def pex(self): + """Return the loose-source py.test binary PEX. - PYTEST_BINARY = 'pytest_binary' + :rtype: :class:`pex.pex.PEX` + """ + return self._pex + + @property + def config_path(self): + """Return the absolute path of the `pytest.ini` config file in this py.test binary. + + :rtype: str + """ + return os.path.join(self._pex.path(), 'pytest.ini') @classmethod def product_types(cls): - return [cls.PYTEST_BINARY] + return [cls.PytestBinary] @classmethod def subsystem_dependencies(cls): @@ -27,8 +49,11 @@ def subsystem_dependencies(cls): def extra_requirements(self): return PyTest.global_instance().get_requirement_strings() + def extra_files(self): + yield self.ExtraFile.empty('pytest.ini') + def execute(self): pex_info = PexInfo.default() pex_info.entry_point = 'pytest' pytest_binary = self.create_pex(pex_info) - self.context.products.register_data(self.PYTEST_BINARY, pytest_binary) + self.context.products.register_data(self.PytestBinary, self.PytestBinary(pytest_binary)) diff --git a/src/python/pants/backend/python/tasks/pytest_run.py b/src/python/pants/backend/python/tasks/pytest_run.py index a05fe72fe10..d4d67f83fa0 100644 --- a/src/python/pants/backend/python/tasks/pytest_run.py +++ b/src/python/pants/backend/python/tasks/pytest_run.py @@ -135,7 +135,7 @@ def supports_passthru_args(cls): @classmethod def prepare(cls, options, round_manager): super(PytestRun, cls).prepare(options, round_manager) - round_manager.require_data(PytestPrep.PYTEST_BINARY) + round_manager.require_data(PytestPrep.PytestBinary) def _test_target_filter(self): def target_filter(target): @@ -231,7 +231,7 @@ def _maybe_emit_coverage_data(self, workdirs, targets, pex): return def pex_src_root(tgt): - return os.path.relpath(self._source_chroot_path([tgt]), get_buildroot()) + return os.path.relpath(self._source_chroot_path((tgt,)), get_buildroot()) source_mappings = {} for target in targets: @@ -255,7 +255,7 @@ def compute_coverage_sources(tgt): # but also consider supporting configuration of a global scheme whether that be parallel # dirs/packages or some arbitrary function that can be registered that takes a test target # and hands back the source packages or paths under test. - return set(os.path.dirname(s).replace(os.sep, '.') + return set(os.path.dirname(s).replace(os.sep, '.') or pex_src_root(tgt) for s in tgt.sources_relative_to_source_root()) coverage_sources = set(itertools.chain(*[compute_coverage_sources(t) for t in targets])) else: @@ -350,7 +350,7 @@ def is_conftest(itm): except Sharder.InvalidShardSpec as e: raise self.InvalidShardSpecification(e) - def _get_conftest_content(self, sources_map): + def _get_conftest_content(self, sources_map, rootdir_comm_path): # A conftest hook to modify the console output, replacing the chroot-based # source paths with the source-tree based ones, which are more readable to the end user. # Note that python stringifies a dict to its source representation, so we can use sources_map @@ -369,24 +369,48 @@ def _get_conftest_content(self, sources_map): ### GENERATED BY PANTS ### + import os + import pytest - # Map from source path relative to chroot -> source path relative to buildroot. - _SOURCES_MAP = {!r} - - @pytest.hookimpl(hookwrapper=True) - def pytest_runtest_protocol(item, nextitem): - # Temporarily change the nodeid, which pytest uses for display here. - real_nodeid = item.nodeid - real_path = real_nodeid.split('::', 1)[0] - fixed_path = _SOURCES_MAP.get(real_path, real_path) - fixed_nodeid = fixed_path + real_nodeid[len(real_path):] - try: - item._nodeid = fixed_nodeid - yield - finally: - item._nodeid = real_nodeid - """.format(dict(sources_map))) + + class NodeRenamerPlugin(object): + # Map from absolute source chroot path -> buildroot relative path. + _SOURCES_MAP = {sources_map!r} + + def __init__(self, rootdir): + def rootdir_relative(path): + return os.path.relpath(path, rootdir) + + self._sources_map = {{rootdir_relative(k): rootdir_relative(v) + for k, v in self._SOURCES_MAP.items()}} + + @pytest.hookimpl(hookwrapper=True) + def pytest_runtest_protocol(self, item, nextitem): + # Temporarily change the nodeid, which pytest uses for display. + real_nodeid = item.nodeid + real_path = real_nodeid.split('::', 1)[0] + fixed_path = self._sources_map.get(real_path, real_path) + fixed_nodeid = fixed_path + real_nodeid[len(real_path):] + try: + item._nodeid = fixed_nodeid + yield + finally: + item._nodeid = real_nodeid + + + # The path to write out the py.test rootdir to. + _ROOTDIR_COMM_PATH = {rootdir_comm_path!r} + + + def pytest_configure(config): + rootdir = str(config.rootdir) + with open(_ROOTDIR_COMM_PATH, 'w') as fp: + fp.write(rootdir) + + config.pluginmanager.register(NodeRenamerPlugin(rootdir), 'pants_test_renamer') + + """.format(sources_map=dict(sources_map), rootdir_comm_path=rootdir_comm_path)) # Add in the sharding conftest, if any. shard_conftest_content = self._get_shard_conftest_content() return (console_output_conftest_content + shard_conftest_content).encode('utf8') @@ -394,22 +418,30 @@ def pytest_runtest_protocol(item, nextitem): @contextmanager def _conftest(self, sources_map): """Creates a conftest.py to customize our pytest run.""" - conftest_content = self._get_conftest_content(sources_map) # Note that it's important to put the tmpdir under the workdir, because pytest # uses all arguments that look like paths to compute its rootdir, and we want # it to pick the buildroot. with temporary_dir(root_dir=self.workdir) as conftest_dir: + rootdir_comm_path = os.path.join(conftest_dir, 'pytest_rootdir.path') + + def get_pytest_rootdir(): + with open(rootdir_comm_path, 'rb') as fp: + return fp.read() + + conftest_content = self._get_conftest_content(sources_map, + rootdir_comm_path=rootdir_comm_path) + conftest = os.path.join(conftest_dir, 'conftest.py') with open(conftest, 'w') as fp: fp.write(conftest_content) - yield conftest + yield conftest, get_pytest_rootdir @contextmanager def _test_runner(self, workdirs, targets, sources_map): - pex = self.context.products.get_data(PytestPrep.PYTEST_BINARY) - with self._conftest(sources_map) as conftest: - with self._maybe_emit_coverage_data(workdirs, targets, pex) as coverage_args: - yield pex, [conftest] + coverage_args + pytest_binary = self.context.products.get_data(PytestPrep.PytestBinary) + with self._conftest(sources_map) as (conftest, get_pytest_rootdir): + with self._maybe_emit_coverage_data(workdirs, targets, pytest_binary.pex) as coverage_args: + yield pytest_binary, [conftest] + coverage_args, get_pytest_rootdir def _do_run_tests_with_args(self, pex, args): try: @@ -460,8 +492,9 @@ def _map_relsrc_to_targets(self, targets): return relsrc_to_target - def _get_failed_targets_from_junitxml(self, junitxml, targets): + def _get_failed_targets_from_junitxml(self, junitxml, targets, pytest_rootdir): relsrc_to_target = self._map_relsrc_to_targets(targets) + buildroot_relpath = os.path.relpath(pytest_rootdir, get_buildroot()) # Now find the sources that contained failing tests. failed_targets = set() @@ -475,17 +508,22 @@ def _get_failed_targets_from_junitxml(self, junitxml, targets): test_failed = testcase.getElementsByTagName('failure') test_errored = testcase.getElementsByTagName('error') if test_failed or test_errored: - # The 'file' attribute is a relsrc, because that's what we passed in to pytest. - failed_targets.add(relsrc_to_target.get(testcase.getAttribute('file'))) + # The file attribute is always relative to the py.test rootdir. + pytest_relpath = testcase.getAttribute('file') + relsrc = os.path.join(buildroot_relpath, pytest_relpath) + failed_target = relsrc_to_target.get(relsrc) + failed_targets.add(failed_target) except (XmlParser.XmlError, ValueError) as e: raise TaskError('Error parsing xml file at {}: {}'.format(junitxml, e)) return failed_targets - def _get_target_from_test(self, test_info, targets): + def _get_target_from_test(self, test_info, targets, pytest_rootdir): relsrc_to_target = self._map_relsrc_to_targets(targets) - file_info = test_info['file'] - return relsrc_to_target.get(file_info) + buildroot_relpath = os.path.relpath(pytest_rootdir, get_buildroot()) + pytest_relpath = test_info['file'] + relsrc = os.path.join(buildroot_relpath, pytest_relpath) + return relsrc_to_target.get(relsrc) @contextmanager def partitions(self, per_target, all_targets, test_targets): @@ -532,14 +570,15 @@ def fingerprint_strategy(self): def run_tests(self, fail_fast, test_targets, workdirs): try: - return self._run_pytest(fail_fast, test_targets, workdirs) + return self._run_pytest(fail_fast, tuple(test_targets), workdirs) finally: # Unconditionally pluck any results that an end user might need to interact with from the # workdir to the locations they expect. self._expose_results(test_targets, workdirs) - def result_from_error(self, error): - return PytestResult.from_error(error) + @memoized_property + def result_class(self): + return PytestResult def collect_files(self, workdirs): return workdirs.files() @@ -569,21 +608,20 @@ def _run_pytest(self, fail_fast, targets, workdirs): if not targets: return PytestResult.rc(0) - if self.run_tests_in_chroot: - path_func = lambda rel_src: rel_src - else: - source_chroot = os.path.relpath(self._source_chroot_path(targets), get_buildroot()) - path_func = lambda rel_src: os.path.join(source_chroot, rel_src) + source_chroot_path = self._source_chroot_path(targets) - sources_map = OrderedDict() # Path from chroot -> Path from buildroot. + # Absolute path to chrooted source -> Path to original source relative to the buildroot. + sources_map = OrderedDict() for t in targets: for p in t.sources_relative_to_source_root(): - sources_map[path_func(p)] = os.path.join(t.target_base, p) + sources_map[os.path.join(source_chroot_path, p)] = os.path.join(t.target_base, p) if not sources_map: return PytestResult.rc(0) - with self._test_runner(workdirs, targets, sources_map) as (pex, test_args): + with self._test_runner(workdirs, targets, sources_map) as (pytest_binary, + test_args, + get_pytest_rootdir): # Validate that the user didn't provide any passthru args that conflict # with those we must set ourselves. for arg in self.get_passthru_args(): @@ -595,7 +633,9 @@ def _run_pytest(self, fail_fast, targets, workdirs): # N.B. the `--confcutdir` here instructs pytest to stop scanning for conftest.py files at the # top of the buildroot. This prevents conftest.py files from outside (e.g. in users home dirs) # from leaking into pants test runs. See: https://github.com/pantsbuild/pants/issues/2726 - args = ['--junitxml', junitxml_path, '--confcutdir', get_buildroot(), + args = ['-c', pytest_binary.config_path, + '--junitxml', junitxml_path, + '--confcutdir', get_buildroot(), '--continue-on-collection-errors'] if fail_fast: args.extend(['-x']) @@ -614,14 +654,17 @@ def _run_pytest(self, fail_fast, targets, workdirs): os.unlink(junitxml_path) with self._maybe_run_in_chroot(targets): - result = self._do_run_tests_with_args(pex, args) + result = self._do_run_tests_with_args(pytest_binary.pex, args) # There was a problem prior to test execution preventing junit xml file creation so just let # the failure result bubble. if not os.path.exists(junitxml_path): return result - failed_targets = self._get_failed_targets_from_junitxml(junitxml_path, targets) + pytest_rootdir = get_pytest_rootdir() + failed_targets = self._get_failed_targets_from_junitxml(junitxml_path, + targets, + pytest_rootdir) def parse_error_handler(parse_error): # Simple error handler to pass to xml parsing function. @@ -631,11 +674,12 @@ def parse_error_handler(parse_error): all_tests_info = self.parse_test_info(junitxml_path, parse_error_handler, ['file', 'name', 'classname']) for test_name, test_info in all_tests_info.items(): - test_target = self._get_target_from_test(test_info, targets) + test_target = self._get_target_from_test(test_info, targets, pytest_rootdir) self.report_all_info_for_single_test(self.options_scope, test_target, test_name, test_info) return result.with_failed_targets(failed_targets) + @memoized_method def _source_chroot_path(self, targets): if len(targets) > 1: target_bases = {target.target_base for target in targets} diff --git a/src/python/pants/backend/python/tasks/python_execution_task_base.py b/src/python/pants/backend/python/tasks/python_execution_task_base.py index a9f70e8b271..a1fa9cb5efb 100644 --- a/src/python/pants/backend/python/tasks/python_execution_task_base.py +++ b/src/python/pants/backend/python/tasks/python_execution_task_base.py @@ -20,6 +20,8 @@ from pants.backend.python.tasks.wrapped_pex import WrappedPEX from pants.build_graph.files import Files from pants.invalidation.cache_manager import VersionedTargetSet +from pants.util.contextutil import temporary_file +from pants.util.objects import datatype class PythonExecutionTaskBase(ResolveRequirementsTaskBase): @@ -39,9 +41,42 @@ def prepare(cls, options, round_manager): def extra_requirements(self): """Override to provide extra requirements needed for execution. - Must return a list of pip-style requirement strings. + :returns: An iterable of pip-style requirement strings. + :rtype: :class:`collections.Iterable` of str """ - return [] + return () + + class ExtraFile(datatype('ExtraFile', ['path', 'content'])): + """Models an extra file to place in a PEX.""" + + @classmethod + def empty(cls, path): + """Creates an empty file with the given PEX path. + + :param str path: The path this extra file should have when added to a PEX. + :rtype: :class:`ExtraFile` + """ + return cls(path=path, content='') + + def add_to(self, builder): + """Adds this extra file to a PEX builder. + + :param builder: The PEX builder to add this extra file to. + :type builder: :class:`pex.pex_builder.PEXBuilder` + """ + with temporary_file() as fp: + fp.write(self.content) + fp.close() + add = builder.add_source if self.path.endswith('.py') else builder.add_resource + add(fp.name, self.path) + + def extra_files(self): + """Override to provide extra files needed for execution. + + :returns: An iterable of extra files to add to the PEX. + :rtype: :class:`collections.Iterable` of :class:`PythonExecutionTaskBase.ExtraFile` + """ + return () def create_pex(self, pex_info=None): """Returns a wrapped pex that "merges" the other pexes via PEX_PATH.""" @@ -76,6 +111,10 @@ def create_pex(self, pex_info=None): pexes = [extra_requirements_pex] + pexes constraints = {constraint for rt in relevant_targets if is_python_target(rt) for constraint in rt.compatibility} - self.merge_pexes(path, pex_info, interpreter, pexes, constraints) + + with self.merged_pex(path, pex_info, interpreter, pexes, constraints) as builder: + for extra_file in self.extra_files(): + extra_file.add_to(builder) + builder.freeze() return WrappedPEX(PEX(path, interpreter), interpreter) diff --git a/src/python/pants/backend/python/tasks/resolve_requirements_task_base.py b/src/python/pants/backend/python/tasks/resolve_requirements_task_base.py index b6e5161bebe..978b64801a0 100644 --- a/src/python/pants/backend/python/tasks/resolve_requirements_task_base.py +++ b/src/python/pants/backend/python/tasks/resolve_requirements_task_base.py @@ -6,6 +6,7 @@ unicode_literals, with_statement) import os +from contextlib import contextmanager from pex.interpreter import PythonInterpreter from pex.pex import PEX @@ -80,8 +81,12 @@ def resolve_requirement_strings(self, interpreter, requirement_strings): return PEX(path, interpreter=interpreter) @classmethod - def merge_pexes(cls, path, pex_info, interpreter, pexes, interpeter_constraints=None): - """Generates a merged pex at path.""" + @contextmanager + def merged_pex(cls, path, pex_info, interpreter, pexes, interpeter_constraints=None): + """Yields a pex builder at path with the given pexes already merged. + + :rtype: :class:`pex.pex_builder.PEXBuilder` + """ pex_paths = [pex.path() for pex in pexes if pex] if pex_paths: pex_info = pex_info.copy() @@ -92,4 +97,10 @@ def merge_pexes(cls, path, pex_info, interpreter, pexes, interpeter_constraints= if interpeter_constraints: for constraint in interpeter_constraints: builder.add_interpreter_constraint(constraint) + yield builder + + @classmethod + def merge_pexes(cls, path, pex_info, interpreter, pexes, interpeter_constraints=None): + """Generates a merged pex at path.""" + with cls.merged_pex(path, pex_info, interpreter, pexes, interpeter_constraints) as builder: builder.freeze() diff --git a/src/python/pants/task/testrunner_task_mixin.py b/src/python/pants/task/testrunner_task_mixin.py index 28f14f9cec0..d8fd0a8dcbe 100644 --- a/src/python/pants/task/testrunner_task_mixin.py +++ b/src/python/pants/task/testrunner_task_mixin.py @@ -15,11 +15,18 @@ from pants.build_graph.files import Files from pants.invalidation.cache_manager import VersionedTargetSet from pants.task.task import Task +from pants.util.memo import memoized_method, memoized_property from pants.util.process_handler import subprocess class TestResult(object): @classmethod + @memoized_method + def successful(cls): + return cls.rc(0) + + @classmethod + @memoized_method def exception(cls): return cls('EXCEPTION') @@ -446,7 +453,7 @@ def _execute(self, all_targets): try: rv = self._run_partition(fail_fast, partition, *args) except ErrorWhileTesting as e: - rv = self.result_from_error(e) + rv = self.result_class.from_error(e) results[partition] = rv if not rv.success: @@ -456,21 +463,15 @@ def _execute(self, all_targets): for partition in sorted(results): rv = results[partition] - if len(partition) == 1 or rv.success: - log = self.context.log.info if rv.success else self.context.log.error - for target in partition: - log('{0:80}.....{1:>10}'.format(target.address.reference(), rv)) - else: - # There is not much useful we can display in summary for a multi-target partition with - # failures without parsing those failures to link them to individual targets; ie: targets - # 2 and 8 failed in this partition of 10 targets. - # TODO(John Sirois): Punting here works for our 2 common partitionings: - # 1. All targets in singleton partitions - # 2. All targets in 1 partition - # PytestRun supports multiple partitions with multiple targets each when there sre - # multiple python source roots, and so some sort of summary for the multi-target - # partitions is needed: https://github.com/pantsbuild/pants/issues/5415 - pass + failed_targets = set(rv.failed_targets) + for target in partition: + if target in failed_targets: + log = self.context.log.error + result = rv + else: + log = self.context.log.info + result = self.result_class.successful() + log('{0:80}.....{1:>10}'.format(target.address.reference(), result)) msgs = [str(_rv) for _rv in results.values() if not _rv.success] failed_targets = [target @@ -555,15 +556,14 @@ def _run_partition(self, fail_fast, test_targets, *args): return result - def result_from_error(self, error): - """Convert an error into a test result. + @memoized_property + def result_class(self): + """Return the test result type returned by `run_tests`. - :param error: The error to convert into a test result. - :type error: :class:`pants.base.exceptions.TaskError` - :returns: An unsuccessful test result. - :rtype: :class:`TestResult` + :returns: The test result class to use. + :rtype: type that is a subclass of :class:`TestResult` """ - return TestResult.from_error(error) + return TestResult def fingerprint_strategy(self): """Return a fingerprint strategy for target fingerprinting. diff --git a/tests/python/pants_test/backend/python/tasks/test_pytest_run_integration.py b/tests/python/pants_test/backend/python/tasks/test_pytest_run_integration.py index 31846933837..01caee288f5 100644 --- a/tests/python/pants_test/backend/python/tasks/test_pytest_run_integration.py +++ b/tests/python/pants_test/backend/python/tasks/test_pytest_run_integration.py @@ -110,16 +110,19 @@ def test_pants_test_interpreter_selection_with_pexrc(self): if self.has_python_version(py27) and self.has_python_version(py3): print('Found both python {} and python {}. Running test.'.format(py27, py3)) py27_path, py3_path = self.python_interpreter_path(py27), self.python_interpreter_path(py3) - with setup_pexrc_with_pex_python_path(os.path.join(os.path.dirname(sys.argv[0]), '.pexrc'), [py27_path, py3_path]): + with setup_pexrc_with_pex_python_path(os.path.join(os.path.dirname(sys.argv[0]), '.pexrc'), + [py27_path, py3_path]): with temporary_dir() as interpreters_cache: pants_ini_config = {'python-setup': {'interpreter_cache_dir': interpreters_cache}} pants_run_27 = self.run_pants( - command=['test', '{}:test_py2'.format(os.path.join(self.testproject, 'python_3_selection_testing'))], + command=['test', '{}:test_py2'.format(os.path.join(self.testproject, + 'python_3_selection_testing'))], config=pants_ini_config ) self.assert_success(pants_run_27) pants_run_3 = self.run_pants( - command=['test', '{}:test_py3'.format(os.path.join(self.testproject, 'python_3_selection_testing'))], + command=['test', '{}:test_py3'.format(os.path.join(self.testproject, + 'python_3_selection_testing'))], config=pants_ini_config ) self.assert_success(pants_run_3)