From 27a13d44db89bbbdcb99b2a92698eef7044e3cc4 Mon Sep 17 00:00:00 2001 From: Danny McClanahan <1305167+cosmicexplorer@users.noreply.github.com> Date: Tue, 5 Mar 2019 22:46:38 -0800 Subject: [PATCH] convert usages of TypeConstraint to TypeId for rule products in the engine (#7114) ### Problem See #4535 and #4005, in particular [this comment on #4535](https://github.com/pantsbuild/pants/issues/4535#issuecomment-455735882). `TypeConstraint` is a pretty general construct that we would like to do more with, for example #6936, and as of [the current discussion in #4535](https://github.com/pantsbuild/pants/issues/4535#issuecomment-455704544) we realize we can emulate union types in `@rule`s without it, matching just against a specific type. ### Solution - Convert `output_constraint` in the `Rule` subclasses in `rules.py` into `output_type`, and add a `SubclassesOf(type)` type check in `datatype` fields in those classes to ensure this. - Remove `satisfied_by()` and `satisfied_by_type()` externs, and add a `product_type()` extern used to intern a python `type` as a `TypeId`. - Convert all `TypeConstraint`s passed to the engine for intrinsics (e.g. `Snapshot`) into `TypeId`s. - Match whether a rule's result matches its declared output type by simply using `==`! - Manually implement `fmt::Debug` for `TypeId` to be the same as `Display` (we may want to do these differently in the future, but it is very useful to see the actual type name when debugging). ### Result Everything is the same, but now we don't have the additional complexity of `TypeConstraint` down in the engine when we can do simple `TypeId` equality. This lets us get more creative with `TypeConstraint` on the python side, while type checking on the rust side is a little less complex (and probably more efficient to some degree). --- src/python/pants/engine/native.py | 123 +++++++-------- src/python/pants/engine/rules.py | 75 +++------ src/python/pants/engine/scheduler.py | 83 +++++----- src/python/pants/engine/selectors.py | 37 ++--- src/python/pants/init/engine_initializer.py | 2 +- src/rust/engine/src/cffi_build.rs | 3 + src/rust/engine/src/core.rs | 47 ++++-- src/rust/engine/src/externs.rs | 75 +++++---- src/rust/engine/src/interning.rs | 31 +++- src/rust/engine/src/lib.rs | 74 ++++----- src/rust/engine/src/nodes.rs | 60 +++----- src/rust/engine/src/rule_graph.rs | 106 ++++--------- src/rust/engine/src/scheduler.rs | 4 +- src/rust/engine/src/selectors.rs | 15 +- src/rust/engine/src/tasks.rs | 38 ++--- src/rust/engine/src/types.rs | 28 ++-- tests/python/pants_test/engine/test_engine.py | 18 +-- tests/python/pants_test/engine/test_rules.py | 142 ++++++++---------- .../pants_test/engine/test_scheduler.py | 8 +- 19 files changed, 428 insertions(+), 541 deletions(-) diff --git a/src/python/pants/engine/native.py b/src/python/pants/engine/native.py index 98bd3884ceb..3060b2999df 100644 --- a/src/python/pants/engine/native.py +++ b/src/python/pants/engine/native.py @@ -19,7 +19,7 @@ from future.utils import PY2, binary_type, text_type from twitter.common.collections.orderedset import OrderedSet -from pants.engine.selectors import Get, constraint_for +from pants.engine.selectors import Get from pants.util.contextutil import temporary_dir from pants.util.dirutil import read_file, safe_mkdir, safe_mkdtemp from pants.util.memo import memoized_classproperty, memoized_property @@ -276,9 +276,33 @@ def extern_log(self, context_handle, level, msg_ptr, msg_len): msg = bytes(self._ffi.buffer(msg_ptr, msg_len)).decode('utf-8') logger.log(level, msg) + @_extern_decl('TypeId', ['ExternContext*', 'Handle*']) + def extern_product_type(self, context_handle, val): + """Return a TypeId for the given Handle, which must be a `type`.""" + c = self._ffi.from_handle(context_handle) + obj = self._ffi.from_handle(val[0]) + # TODO: determine if this assertion has any performance implications. + assert isinstance(obj, type) + tid = c.to_id(obj) + return TypeId(tid) + + @_extern_decl('TypeId', ['ExternContext*', 'Handle*']) + def extern_get_type_for(self, context_handle, val): + """Return a representation of the object's type.""" + c = self._ffi.from_handle(context_handle) + obj = self._ffi.from_handle(val[0]) + type_id = c.to_id(type(obj)) + return TypeId(type_id) + @_extern_decl('Ident', ['ExternContext*', 'Handle*']) def extern_identify(self, context_handle, val): - """Return an Ident containing the __hash__ and TypeId for the given Handle.""" + """Return a representation of the object's identity, including a hash and TypeId. + + `extern_get_type_for()` also returns a TypeId, but doesn't hash the object -- this allows that + method to be used on unhashable objects. `extern_identify()` returns a TypeId as well to avoid + having to make two separate Python calls when interning a Python object in interning.rs, which + requires both the hash and type. + """ c = self._ffi.from_handle(context_handle) obj = self._ffi.from_handle(val[0]) hash_ = hash(obj) @@ -318,19 +342,6 @@ def extern_val_to_str(self, context_handle, val): v_str = '' if v is None else text_type(v) return c.utf8_buf(v_str) - @_extern_decl('_Bool', ['ExternContext*', 'Handle*', 'Handle*']) - def extern_satisfied_by(self, context_handle, constraint_val, val): - """Given a TypeConstraint and a Handle return constraint.satisfied_by(value).""" - constraint = self._ffi.from_handle(constraint_val[0]) - return constraint.satisfied_by(self._ffi.from_handle(val[0])) - - @_extern_decl('_Bool', ['ExternContext*', 'Handle*', 'TypeId*']) - def extern_satisfied_by_type(self, context_handle, constraint_val, cls_id): - """Given a TypeConstraint and a TypeId, return constraint.satisfied_by_type(type_id).""" - c = self._ffi.from_handle(context_handle) - constraint = self._ffi.from_handle(constraint_val[0]) - return constraint.satisfied_by_type(c.from_id(cls_id.tup_0)) - @_extern_decl('Handle', ['ExternContext*', 'Handle**', 'uint64_t']) def extern_store_tuple(self, context_handle, vals_ptr, vals_len): """Given storage and an array of Handles, return a new Handle to represent the list.""" @@ -423,12 +434,12 @@ def extern_generator_send(self, context_handle, func, arg): if isinstance(res, Get): # Get. values = [res.subject] - products = [constraint_for(res.product)] + products = [res.product] tag = 2 elif type(res) in (tuple, list): # GetMulti. values = [g.subject for g in res] - products = [constraint_for(g.product) for g in res] + products = [g.product for g in res] tag = 3 else: # Break. @@ -446,7 +457,7 @@ def extern_generator_send(self, context_handle, func, arg): return ( tag, c.vals_buf([c.to_value(v) for v in values]), - c.vals_buf([c.to_value(v) for v in products]), + c.vals_buf([c.to_value(v) for v in products]) ) @_extern_decl('PyResult', ['ExternContext*', 'Handle*', 'Handle**', 'uint64_t']) @@ -472,10 +483,6 @@ class Function(datatype(['key'])): """Corresponds to the native object of the same name.""" -class TypeConstraint(datatype(['key'])): - """Corresponds to the native object of the same name.""" - - class TypeId(datatype(['tup_0'])): """Corresponds to the native object of the same name.""" @@ -626,14 +633,14 @@ def init_externs(): self.ffi_lib.extern_call, self.ffi_lib.extern_generator_send, self.ffi_lib.extern_eval, + self.ffi_lib.extern_product_type, + self.ffi_lib.extern_get_type_for, self.ffi_lib.extern_identify, self.ffi_lib.extern_equals, self.ffi_lib.extern_clone_val, self.ffi_lib.extern_drop_handles, self.ffi_lib.extern_type_to_str, self.ffi_lib.extern_val_to_str, - self.ffi_lib.extern_satisfied_by, - self.ffi_lib.extern_satisfied_by_type, self.ffi_lib.extern_store_tuple, self.ffi_lib.extern_store_set, self.ffi_lib.extern_store_dict, @@ -644,8 +651,7 @@ def init_externs(): self.ffi_lib.extern_store_bool, self.ffi_lib.extern_project_ignoring_type, self.ffi_lib.extern_project_multi, - self.ffi_lib.extern_create_exception, - TypeId(context.to_id(str))) + self.ffi_lib.extern_create_exception) return context return self.ffi.init_once(init_externs, 'ExternContext singleton') @@ -704,25 +710,25 @@ def new_scheduler(self, construct_file_content, construct_files_content, construct_process_result, - constraint_address, - constraint_path_globs, - constraint_directory_digest, - constraint_snapshot, - constraint_merge_snapshots_request, - constraint_files_content, - constraint_dir, - constraint_file, - constraint_link, - constraint_process_request, - constraint_process_result, - constraint_generator, - constraint_url_to_fetch): + type_address, + type_path_globs, + type_directory_digest, + type_snapshot, + type_merge_snapshots_request, + type_files_content, + type_dir, + type_file, + type_link, + type_process_request, + type_process_result, + type_generator, + type_url_to_fetch): """Create and return an ExternContext and native Scheduler.""" - def func(constraint): - return Function(self.context.to_key(constraint)) - def tc(constraint): - return TypeConstraint(self.context.to_key(constraint)) + def func(fn): + return Function(self.context.to_key(fn)) + def ti(type_obj): + return TypeId(self.context.to_id(type_obj)) scheduler = self.lib.scheduler_create( tasks, @@ -732,23 +738,22 @@ def tc(constraint): func(construct_file_content), func(construct_files_content), func(construct_process_result), - # TypeConstraints. - tc(constraint_address), - tc(constraint_path_globs), - tc(constraint_directory_digest), - tc(constraint_snapshot), - tc(constraint_merge_snapshots_request), - tc(constraint_files_content), - tc(constraint_dir), - tc(constraint_file), - tc(constraint_link), - tc(constraint_process_request), - tc(constraint_process_result), - tc(constraint_generator), - tc(constraint_url_to_fetch), # Types. - TypeId(self.context.to_id(text_type)), - TypeId(self.context.to_id(binary_type)), + ti(type_address), + ti(type_path_globs), + ti(type_directory_digest), + ti(type_snapshot), + ti(type_merge_snapshots_request), + ti(type_files_content), + ti(type_dir), + ti(type_file), + ti(type_link), + ti(type_process_request), + ti(type_process_result), + ti(type_generator), + ti(type_url_to_fetch), + ti(text_type), + ti(binary_type), # Project tree. self.context.utf8_buf(build_root), self.context.utf8_buf(work_dir), diff --git a/src/python/pants/engine/rules.py b/src/python/pants/engine/rules.py index d50351a484a..0a1971905a1 100644 --- a/src/python/pants/engine/rules.py +++ b/src/python/pants/engine/rules.py @@ -17,17 +17,20 @@ from future.utils import PY2 from twitter.common.collections import OrderedSet -from pants.engine.selectors import Get, type_or_constraint_repr +from pants.engine.selectors import Get from pants.util.collections import assert_single_element from pants.util.collections_abc_backport import Iterable, OrderedDict from pants.util.memo import memoized from pants.util.meta import AbstractClass -from pants.util.objects import Exactly, datatype +from pants.util.objects import SubclassesOf, datatype logger = logging.getLogger(__name__) +_type_field = SubclassesOf(type) + + class _RuleVisitor(ast.NodeVisitor): """Pull `Get` calls out of an @rule body and validate `yield` statements.""" @@ -249,17 +252,14 @@ def wrapper(func): def resolve_type(name): resolved = caller_frame.f_globals.get(name) or caller_frame.f_builtins.get(name) - if not isinstance(resolved, (type, Exactly)): - # TODO: should this say "...or Exactly instance;"? - raise ValueError('Expected either a `type` constructor or TypeConstraint instance; ' - 'got: {}'.format(name)) + if not isinstance(resolved, type): + raise ValueError('Expected a `type` constructor, but got: {}'.format(name)) return resolved gets = OrderedSet() rule_func_node = assert_single_element( node for node in ast.iter_child_nodes(module_ast) - if isinstance(node, ast.FunctionDef) and node.name == func.__name__ - ) + if isinstance(node, ast.FunctionDef) and node.name == func.__name__) parents_table = {} for parent in ast.walk(rule_func_node): @@ -300,7 +300,7 @@ def goal_and_return(*args, **kwargs): wrapped_func, input_gets=tuple(gets), goal=for_goal, - cacheable=cacheable + cacheable=cacheable, ) return wrapped_func @@ -368,16 +368,17 @@ class Rule(AbstractClass): """ @abstractproperty - def output_constraint(self): - """An output Constraint type for the rule.""" + def output_type(self): + """An output `type` for the rule.""" - @abstractproperty + @property def dependency_optionables(self): """A tuple of Optionable classes that are known to be necessary to run this rule.""" + return () class TaskRule(datatype([ - 'output_constraint', + ('output_type', _type_field), ('input_selectors', tuple), ('input_gets', tuple), 'func', @@ -400,18 +401,10 @@ def __new__(cls, goal=None, dependency_optionables=None, cacheable=True): - # Validate result type. - if isinstance(output_type, Exactly): - constraint = output_type - elif isinstance(output_type, type): - constraint = Exactly(output_type) - else: - raise TypeError("Expected an output_type for rule `{}`, got: {}".format( - func.__name__, output_type)) return super(TaskRule, cls).__new__( cls, - constraint, + output_type, input_selectors, input_gets, func, @@ -422,41 +415,22 @@ def __new__(cls, def __str__(self): return ('({}, {!r}, {}, gets={}, opts={})' - .format(type_or_constraint_repr(self.output_constraint), + .format(self.output_type.__name__, self.input_selectors, self.func.__name__, self.input_gets, self.dependency_optionables)) -class SingletonRule(datatype(['output_constraint', 'value']), Rule): +class SingletonRule(datatype([('output_type', _type_field), 'value']), Rule): """A default rule for a product, which is thus a singleton for that product.""" @classmethod def from_instance(cls, obj): return cls(type(obj), obj) - def __new__(cls, output_type, value): - # Validate result type. - if isinstance(output_type, Exactly): - constraint = output_type - elif isinstance(output_type, type): - constraint = Exactly(output_type) - else: - raise TypeError("Expected an output_type for rule; got: {}".format(output_type)) - - # Create. - return super(SingletonRule, cls).__new__(cls, constraint, value) - @property - def dependency_optionables(self): - return tuple() - - def __repr__(self): - return '{}({}, {})'.format(type(self).__name__, type_or_constraint_repr(self.output_constraint), self.value) - - -class RootRule(datatype(['output_constraint']), Rule): +class RootRule(datatype([('output_type', _type_field)]), Rule): """Represents a root input to an execution of a rule graph. Roots act roughly like parameters, in that in some cases the only source of a @@ -464,11 +438,8 @@ class RootRule(datatype(['output_constraint']), Rule): of an execution. """ - @property - def dependency_optionables(self): - return tuple() - +# TODO: add typechecking here -- would need to have a TypedCollection for dicts for `union_rules`. class RuleIndex(datatype(['rules', 'roots', 'union_rules'])): """Holds a normalized index of Rules used to instantiate Nodes.""" @@ -480,6 +451,7 @@ def create(cls, rule_entries, union_rules=None): union_rules = OrderedDict(union_rules or ()) def add_task(product_type, rule): + # TODO(#7311): make a defaultdict-like wrapper for OrderedDict if more widely used. if product_type not in serializable_rules: serializable_rules[product_type] = OrderedSet() serializable_rules[product_type].add(rule) @@ -491,12 +463,7 @@ def add_rule(rule): if isinstance(rule, RootRule): add_root_rule(rule) else: - # TODO: Ensure that interior types work by indexing on the list of types in - # the constraint. This heterogenity has some confusing implications: - # see https://github.com/pantsbuild/pants/issues/4005 - for kind in rule.output_constraint.types: - add_task(kind, rule) - add_task(rule.output_constraint, rule) + add_task(rule.output_type, rule) def add_type_transition_rule(union_rule): # NB: This does not require that union bases be supplied to `def rules():`, as the union type diff --git a/src/python/pants/engine/scheduler.py b/src/python/pants/engine/scheduler.py index 808b0c39979..4ee04a1f221 100644 --- a/src/python/pants/engine/scheduler.py +++ b/src/python/pants/engine/scheduler.py @@ -16,11 +16,11 @@ from pants.engine.fs import (Digest, DirectoryToMaterialize, FileContent, FilesContent, MergedDirectories, PathGlobs, PathGlobsAndRoot, Snapshot, UrlToFetch) from pants.engine.isolated_process import ExecuteProcessRequest, FallibleExecuteProcessResult -from pants.engine.native import Function, TypeConstraint, TypeId +from pants.engine.native import Function, TypeId from pants.engine.nodes import Return, Throw from pants.engine.objects import Collection from pants.engine.rules import RuleIndex, SingletonRule, TaskRule -from pants.engine.selectors import Params, Select, constraint_for +from pants.engine.selectors import Params, Select from pants.rules.core.exceptions import GracefulTerminationException from pants.util.contextutil import temporary_file_path from pants.util.dirutil import check_no_overlapping_paths @@ -83,7 +83,7 @@ def __init__( self._visualize_to_dir = visualize_to_dir # Validate and register all provided and intrinsic tasks. rule_index = RuleIndex.create(list(rules), union_rules) - self._root_subject_types = [r.output_constraint for r in rule_index.roots] + self._root_subject_types = [r.output_type for r in rule_index.roots] # Create the native Scheduler and Session. # TODO: This `_tasks` reference could be a local variable, since it is not used @@ -104,19 +104,19 @@ def __init__( construct_file_content=FileContent, construct_files_content=FilesContent, construct_process_result=FallibleExecuteProcessResult, - constraint_address=constraint_for(Address), - constraint_path_globs=constraint_for(PathGlobs), - constraint_directory_digest=constraint_for(Digest), - constraint_snapshot=constraint_for(Snapshot), - constraint_merge_snapshots_request=constraint_for(MergedDirectories), - constraint_files_content=constraint_for(FilesContent), - constraint_dir=constraint_for(Dir), - constraint_file=constraint_for(File), - constraint_link=constraint_for(Link), - constraint_process_request=constraint_for(ExecuteProcessRequest), - constraint_process_result=constraint_for(FallibleExecuteProcessResult), - constraint_generator=constraint_for(GeneratorType), - constraint_url_to_fetch=constraint_for(UrlToFetch), + type_address=Address, + type_path_globs=PathGlobs, + type_directory_digest=Digest, + type_snapshot=Snapshot, + type_merge_snapshots_request=MergedDirectories, + type_files_content=FilesContent, + type_dir=Dir, + type_file=File, + type_link=Link, + type_process_request=ExecuteProcessRequest, + type_process_result=FallibleExecuteProcessResult, + type_generator=GeneratorType, + type_url_to_fetch=UrlToFetch, ) @@ -159,14 +159,11 @@ def _to_id(self, typ): def _to_key(self, obj): return self._native.context.to_key(obj) - def _from_id(self, cdata): - return self._native.context.from_id(cdata) - def _from_key(self, cdata): return self._native.context.from_key(cdata) - def _to_constraint(self, type_or_constraint): - return TypeConstraint(self._to_key(constraint_for(type_or_constraint))) + def _to_type(self, type_obj): + return TypeId(self._to_id(type_obj)) def _to_ids_buf(self, types): return self._native.to_ids_buf(types) @@ -177,59 +174,53 @@ def _to_utf8_buf(self, string): def _register_rules(self, rule_index): """Record the given RuleIndex on `self._tasks`.""" registered = set() - for product_type, rules in rule_index.rules.items(): - # TODO: The rules map has heterogeneous keys, so we normalize them to type constraints - # and dedupe them before registering to the native engine: - # see: https://github.com/pantsbuild/pants/issues/4005 - output_constraint = self._to_constraint(product_type) + for output_type, rules in rule_index.rules.items(): for rule in rules: - key = (output_constraint, rule) + key = (output_type, rule) if key in registered: continue registered.add(key) if type(rule) is SingletonRule: - self._register_singleton(output_constraint, rule) + self._register_singleton(output_type, rule) elif type(rule) is TaskRule: - self._register_task(output_constraint, rule, rule_index.union_rules) + self._register_task(output_type, rule, rule_index.union_rules) else: raise ValueError('Unexpected Rule type: {}'.format(rule)) - def _register_singleton(self, output_constraint, rule): + def _register_singleton(self, output_type, rule): """Register the given SingletonRule. A SingletonRule installed for a type will be the only provider for that type. """ self._native.lib.tasks_singleton_add(self._tasks, self._to_value(rule.value), - output_constraint) + TypeId(self._to_id(output_type))) - def _register_task(self, output_constraint, rule, union_rules): + def _register_task(self, output_type, rule, union_rules): """Register the given TaskRule with the native scheduler.""" func = Function(self._to_key(rule.func)) - self._native.lib.tasks_task_begin(self._tasks, func, output_constraint, rule.cacheable) + self._native.lib.tasks_task_begin(self._tasks, func, self._to_type(output_type), rule.cacheable) for selector in rule.input_selectors: selector_type = type(selector) - product_constraint = self._to_constraint(selector.product) if selector_type is Select: - self._native.lib.tasks_add_select(self._tasks, product_constraint) + self._native.lib.tasks_add_select(self._tasks, self._to_type(selector.product)) else: raise ValueError('Unrecognized Selector type: {}'.format(selector)) def add_get_edge(product, subject): - self._native.lib.tasks_add_get(self._tasks, - self._to_constraint(product), - TypeId(self._to_id(subject))) + self._native.lib.tasks_add_get(self._tasks, self._to_type(product), self._to_type(subject)) - for get in rule.input_gets: - union_members = union_rules.get(get.subject_declared_type, None) + for the_get in rule.input_gets: + union_members = union_rules.get(the_get.subject_declared_type, None) if union_members: - # If the registered subject type is a union, add get edges to all registered union members. + # If the registered subject type is a union, add Get edges to all registered union members. for union_member in union_members: - add_get_edge(get.product, union_member) + add_get_edge(the_get.product, union_member) else: - # Otherwise, the Get subject is a "concrete" type, so add a single get edge. - add_get_edge(get.product, get.subject_declared_type) + # Otherwise, the Get subject is a "concrete" type, so add a single Get edge. + add_get_edge(the_get.product, the_get.subject_declared_type) + self._native.lib.tasks_task_end(self._tasks) def visualize_graph_to_file(self, session, filename): @@ -252,7 +243,7 @@ def rule_graph_visualization(self): def rule_subgraph_visualization(self, root_subject_type, product_type): root_type_id = TypeId(self._to_id(root_subject_type)) - product_type_id = TypeConstraint(self._to_key(constraint_for(product_type))) + product_type_id = TypeId(self._to_id(product_type)) with temporary_file_path() as path: self._native.lib.rule_subgraph_visualize( self._scheduler, @@ -285,7 +276,7 @@ def add_root_selection(self, execution_request, subject_or_params, product): res = self._native.lib.execution_add_root_select(self._scheduler, execution_request, self._to_vals_buf(params), - self._to_constraint(product)) + self._to_type(product)) self._raise_or_return(res) def visualize_to_dir(self): diff --git a/src/python/pants/engine/selectors.py b/src/python/pants/engine/selectors.py index 667d29f803c..99b450c840a 100644 --- a/src/python/pants/engine/selectors.py +++ b/src/python/pants/engine/selectors.py @@ -6,28 +6,17 @@ import ast -from pants.util.objects import Exactly, TypeConstraint, datatype +from pants.util.objects import SubclassesOf, TypeConstraint, datatype -def type_or_constraint_repr(constraint): - """Generate correct repr for types and TypeConstraints""" - if isinstance(constraint, type): - return constraint.__name__ - elif isinstance(constraint, Exactly): - return repr(constraint) +_type_field = SubclassesOf(type) -def constraint_for(type_or_constraint): - """Given a type or an `Exactly` constraint, returns an `Exactly` constraint.""" - if isinstance(type_or_constraint, Exactly): - return type_or_constraint - elif isinstance(type_or_constraint, type): - return Exactly(type_or_constraint) - else: - raise TypeError("Expected a type or constraint: got: {}".format(type_or_constraint)) - - -class Get(datatype(['product', 'subject_declared_type', 'subject'])): +class Get(datatype([ + ('product', _type_field), + ('subject_declared_type', _type_field), + 'subject', +])): """Experimental synchronous generator API. May be called equivalently as either: @@ -115,22 +104,16 @@ def __new__(cls, *args): return super(Params, cls).__new__(cls, tuple(args)) -class Select(datatype(['product', 'optional'])): +class Select(datatype([('product', _type_field), ('optional', bool)])): """Selects the given Product for the Subject provided to the constructor. If optional=True and no matching product can be produced, will return None. """ def __new__(cls, product, optional=False): - obj = super(Select, cls).__new__(cls, product, optional) - return obj - - @property - def type_constraint(self): - """The type constraint for the product type for this selector.""" - return constraint_for(self.product) + return super(Select, cls).__new__(cls, product, optional) def __repr__(self): return '{}({}{})'.format(type(self).__name__, - type_or_constraint_repr(self.product), + self.product.__name__, ', optional=True' if self.optional else '') diff --git a/src/python/pants/init/engine_initializer.py b/src/python/pants/init/engine_initializer.py index 86d8218a1ff..c3535241137 100644 --- a/src/python/pants/init/engine_initializer.py +++ b/src/python/pants/init/engine_initializer.py @@ -248,7 +248,7 @@ def _make_goal_map_from_rules(rules): 'could not map goal `{}` to rule `{}`: already claimed by product `{}`' .format(goal, rule, goal_map[goal]) ) - goal_map[goal] = rule.output_constraint + goal_map[goal] = rule.output_type return goal_map @staticmethod diff --git a/src/rust/engine/src/cffi_build.rs b/src/rust/engine/src/cffi_build.rs index 09e11b77932..88214e8537b 100644 --- a/src/rust/engine/src/cffi_build.rs +++ b/src/rust/engine/src/cffi_build.rs @@ -129,6 +129,9 @@ fn main() -> Result<(), CffiBuildError> { let cffi_bootstrapper = build_root.join("build-support/bin/native/bootstrap_cffi.sh"); mark_for_change_detection(&cffi_bootstrapper); + // TODO: bootstrap_c_source() is used to generate C source code from @_extern_decl methods in + // native.py. It would be very useful to be able to detect when those /declarations/ haven't + // changed and avoid rebuilding the engine crate if we are just iterating on the implementations. mark_for_change_detection(&build_root.join("src/python/pants/engine/native.py")); // N.B. The filename of this source code - at generation time - must line up 1:1 with the diff --git a/src/rust/engine/src/core.rs b/src/rust/engine/src/core.rs index 113f5eb9b4f..b027379f3f1 100644 --- a/src/rust/engine/src/core.rs +++ b/src/rust/engine/src/core.rs @@ -119,39 +119,56 @@ pub type Id = u64; #[derive(Clone, Copy, Eq, Hash, Ord, PartialEq, PartialOrd)] pub struct TypeId(pub Id); -impl fmt::Debug for TypeId { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if *self == ANY_TYPE { +impl TypeId { + fn pretty_print(self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + if self == ANY_TYPE { write!(f, "Any") } else { - write!(f, "{}", externs::type_to_str(*self)) + write!(f, "{}", externs::type_to_str(self)) } } } +impl fmt::Debug for TypeId { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.pretty_print(f) + } +} + impl fmt::Display for TypeId { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if *self == ANY_TYPE { - write!(f, "Any") - } else { - write!(f, "{}", externs::type_to_str(*self)) - } + self.pretty_print(f) } } // On the python side, the 0th type id is used as an anonymous id pub const ANY_TYPE: TypeId = TypeId(0); -// A type constraint, which a TypeId may or may-not satisfy. -#[repr(C)] -#[derive(Clone, Copy, Debug, Eq, Hash, PartialEq)] -pub struct TypeConstraint(pub Key); - // An identifier for a python function. #[repr(C)] -#[derive(Clone, Copy, Debug, Eq, Hash, PartialEq)] +#[derive(Clone, Copy, Eq, Hash, PartialEq)] pub struct Function(pub Key); +impl Function { + fn pretty_print(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let Function(key) = self; + let name = externs::project_str(&externs::val_for(&key), "__name__"); + write!(f, "{}()", name) + } +} + +impl fmt::Display for Function { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.pretty_print(f) + } +} + +impl fmt::Debug for Function { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.pretty_print(f) + } +} + /// /// Wraps a type id for use as a key in HashMaps and sets. /// diff --git a/src/rust/engine/src/externs.rs b/src/rust/engine/src/externs.rs index 3198473ec4f..651ea6ff1f7 100644 --- a/src/rust/engine/src/externs.rs +++ b/src/rust/engine/src/externs.rs @@ -8,7 +8,7 @@ use std::os::raw; use std::os::unix::ffi::{OsStrExt, OsStringExt}; use std::string::FromUtf8Error; -use crate::core::{Failure, Function, Key, TypeConstraint, TypeId, Value}; +use crate::core::{Failure, Function, Key, TypeId, Value}; use crate::handles::{DroppingHandle, Handle}; use crate::interning::Interns; use lazy_static::lazy_static; @@ -20,6 +20,14 @@ pub fn eval(python: &str) -> Result { with_externs(|e| (e.eval)(e.context, python.as_ptr(), python.len() as u64)).into() } +pub fn product_type(val: &Value) -> TypeId { + with_externs(|e| (e.product_type)(e.context, val as &Handle)) +} + +pub fn get_type_for(val: &Value) -> TypeId { + with_externs(|e| (e.get_type_for)(e.context, val as &Handle)) +} + pub fn identify(val: &Value) -> Ident { with_externs(|e| (e.identify)(e.context, val as &Handle)) } @@ -46,22 +54,6 @@ pub fn drop_handles(handles: &[DroppingHandle]) { with_externs(|e| (e.drop_handles)(e.context, handles.as_ptr(), handles.len() as u64)) } -pub fn satisfied_by(constraint: &TypeConstraint, obj: &Value) -> bool { - let interns = INTERNS.read(); - with_externs(|e| { - (e.satisfied_by)( - e.context, - interns.get(&constraint.0) as &Handle, - obj as &Handle, - ) - }) -} - -pub fn satisfied_by_type(constraint: &TypeConstraint, cls: TypeId) -> bool { - let interns = INTERNS.read(); - with_externs(|e| (e.satisfied_by_type)(e.context, interns.get(&constraint.0) as &Handle, &cls)) -} - pub fn store_tuple(values: &[Value]) -> Value { let handles: Vec<_> = values .iter() @@ -201,6 +193,8 @@ pub fn create_exception(msg: &str) -> Value { with_externs(|e| (e.create_exception)(e.context, msg.as_ptr(), msg.len() as u64).into()) } +// TODO: This method is currently unused, but kept as an example of how to call methods on objects. +#[allow(dead_code)] pub fn call_method(value: &Value, method: &str, args: &[Value]) -> Result { call(&project_ignoring_type(&value, method), args) } @@ -226,27 +220,30 @@ pub fn generator_send(generator: &Value, arg: &Value) -> Result Err(PyResult::failure_from(response.values.unwrap_one())), PyGeneratorResponseType::Get => { let mut interns = INTERNS.write(); - let product = TypeConstraint(interns.insert(response.products.unwrap_one())); - let subject = interns.insert(response.values.unwrap_one()); - Ok(GeneratorResponse::Get(Get { product, subject })) + let p = response.products.unwrap_one(); + let v = response.values.unwrap_one(); + let g = Get { + product: *interns.insert_product(p).type_id(), + subject: interns.insert(v), + }; + Ok(GeneratorResponse::Get(g)) } PyGeneratorResponseType::GetMulti => { let mut interns = INTERNS.write(); let PyGeneratorResponse { - values: values_buf, products: products_buf, + values: values_buf, .. } = response; - let values = values_buf.to_vec(); let products = products_buf.to_vec(); - assert_eq!(values.len(), products.len()); - let continues: Vec = values + let values = values_buf.to_vec(); + assert_eq!(products.len(), values.len()); + let continues: Vec = products .into_iter() - .zip(products.into_iter()) - .map(|(val, prod)| { - let subject = interns.insert(val); - let product = TypeConstraint(interns.insert(prod)); - Get { subject, product } + .zip(values.into_iter()) + .map(|(p, v)| Get { + product: *interns.insert_product(p).type_id(), + subject: interns.insert(v), }) .collect(); Ok(GeneratorResponse::GetMulti(continues)) @@ -344,12 +341,12 @@ pub struct Externs { pub call: CallExtern, pub generator_send: GeneratorSendExtern, pub eval: EvalExtern, + pub product_type: ProductTypeExtern, + pub get_type_for: GetTypeForExtern, pub identify: IdentifyExtern, pub equals: EqualsExtern, pub clone_val: CloneValExtern, pub drop_handles: DropHandlesExtern, - pub satisfied_by: SatisfiedByExtern, - pub satisfied_by_type: SatisfiedByTypeExtern, pub store_tuple: StoreTupleExtern, pub store_set: StoreTupleExtern, pub store_dict: StoreTupleExtern, @@ -363,8 +360,6 @@ pub struct Externs { pub type_to_str: TypeToStrExtern, pub val_to_str: ValToStrExtern, pub create_exception: CreateExceptionExtern, - // TODO: This type is also declared on `types::Types`. - pub py_str_type: TypeId, } // The pointer to the context is safe for sharing between threads. @@ -373,11 +368,9 @@ unsafe impl Send for Externs {} pub type LogExtern = extern "C" fn(*const ExternContext, u8, str_ptr: *const u8, str_len: u64); -pub type SatisfiedByExtern = - extern "C" fn(*const ExternContext, *const Handle, *const Handle) -> bool; +pub type ProductTypeExtern = extern "C" fn(*const ExternContext, *const Handle) -> TypeId; -pub type SatisfiedByTypeExtern = - extern "C" fn(*const ExternContext, *const Handle, *const TypeId) -> bool; +pub type GetTypeForExtern = extern "C" fn(*const ExternContext, *const Handle) -> TypeId; pub type IdentifyExtern = extern "C" fn(*const ExternContext, *const Handle) -> Ident; @@ -471,6 +464,8 @@ impl From> for PyResult { } // Only constructed from the python side. +// TODO: map this into a C enum with cbindgen and consume from python instead of using magic numbers +// in extern_generator_send() in native.py! #[allow(dead_code)] #[repr(u8)] pub enum PyGeneratorResponseType { @@ -489,8 +484,7 @@ pub struct PyGeneratorResponse { #[derive(Debug)] pub struct Get { - // TODO(#7114): convert all of these into `TypeId`s! - pub product: TypeConstraint, + pub product: TypeId, pub subject: Key, } @@ -501,7 +495,8 @@ pub enum GeneratorResponse { } /// -/// The result of an `identify` call, including the __hash__ of a Handle and its TypeId. +/// The result of an `identify` call, including the __hash__ of a Handle and a TypeId representing +/// the object's type. /// #[repr(C)] pub struct Ident { diff --git a/src/rust/engine/src/interning.rs b/src/rust/engine/src/interning.rs index 30e2d26f707..01c7118ba5c 100644 --- a/src/rust/engine/src/interning.rs +++ b/src/rust/engine/src/interning.rs @@ -2,10 +2,12 @@ // Licensed under the Apache License, Version 2.0 (see LICENSE). use std::collections::HashMap; -use std::hash; +use std::hash::{self, BuildHasher, Hash, Hasher}; -use crate::core::{Key, Value, FNV}; -use crate::externs; +use lazy_static::lazy_static; + +use crate::core::{Key, TypeId, Value, FNV}; +use crate::externs::{self, Ident}; /// /// A struct that encapsulates interning of python `Value`s as comparable `Key`s. @@ -38,19 +40,21 @@ pub struct Interns { id_generator: u64, } +lazy_static! { + static ref PRODUCT_TYPE_ID_HASH_BUILDER: FNV = FNV::default(); +} + impl Interns { pub fn new() -> Interns { Interns::default() } - pub fn insert(&mut self, v: Value) -> Key { - let ident = externs::identify(&v); - let type_id = ident.type_id; + fn perform_insert(&mut self, v: Value, hash: i64, type_id: TypeId) -> Key { let mut inserted = false; let id_generator = self.id_generator; let key = *self .forward - .entry(InternKey(ident.hash, v.clone())) + .entry(InternKey(hash, v.clone())) .or_insert_with(|| { inserted = true; Key::new(id_generator, type_id) @@ -62,6 +66,19 @@ impl Interns { key } + pub fn insert_product(&mut self, v: Value) -> Key { + let type_id = externs::product_type(&v); + let mut hasher = PRODUCT_TYPE_ID_HASH_BUILDER.build_hasher(); + type_id.hash(&mut hasher); + let hash: i64 = hasher.finish() as i64; + self.perform_insert(v, hash, type_id) + } + + pub fn insert(&mut self, v: Value) -> Key { + let Ident { hash, type_id } = externs::identify(&v); + self.perform_insert(v, hash, type_id) + } + pub fn get(&self, k: &Key) -> &Value { self .reverse diff --git a/src/rust/engine/src/lib.rs b/src/rust/engine/src/lib.rs index a6413b5073b..23822c26a19 100644 --- a/src/rust/engine/src/lib.rs +++ b/src/rust/engine/src/lib.rs @@ -54,13 +54,13 @@ use std::sync::Arc; use std::time::Duration; use crate::context::Core; -use crate::core::{Function, Key, Params, TypeConstraint, TypeId, Value}; +use crate::core::{Function, Key, Params, TypeId, Value}; use crate::externs::{ Buffer, BufferBuffer, CallExtern, CloneValExtern, CreateExceptionExtern, DropHandlesExtern, - EqualsExtern, EvalExtern, ExternContext, Externs, GeneratorSendExtern, HandleBuffer, - IdentifyExtern, LogExtern, ProjectIgnoringTypeExtern, ProjectMultiExtern, PyResult, - SatisfiedByExtern, SatisfiedByTypeExtern, StoreBoolExtern, StoreBytesExtern, StoreF64Extern, - StoreI64Extern, StoreTupleExtern, StoreUtf8Extern, TypeIdBuffer, TypeToStrExtern, ValToStrExtern, + EqualsExtern, EvalExtern, ExternContext, Externs, GeneratorSendExtern, GetTypeForExtern, + HandleBuffer, IdentifyExtern, LogExtern, ProductTypeExtern, ProjectIgnoringTypeExtern, + ProjectMultiExtern, PyResult, StoreBoolExtern, StoreBytesExtern, StoreF64Extern, StoreI64Extern, + StoreTupleExtern, StoreUtf8Extern, TypeIdBuffer, TypeToStrExtern, ValToStrExtern, }; use crate::handles::Handle; use crate::rule_graph::{GraphMaker, RuleGraph}; @@ -103,14 +103,14 @@ pub extern "C" fn externs_set( call: CallExtern, generator_send: GeneratorSendExtern, eval: EvalExtern, + product_type: ProductTypeExtern, + get_type_for: GetTypeForExtern, identify: IdentifyExtern, equals: EqualsExtern, clone_val: CloneValExtern, drop_handles: DropHandlesExtern, type_to_str: TypeToStrExtern, val_to_str: ValToStrExtern, - satisfied_by: SatisfiedByExtern, - satisfied_by_type: SatisfiedByTypeExtern, store_tuple: StoreTupleExtern, store_set: StoreTupleExtern, store_dict: StoreTupleExtern, @@ -122,7 +122,6 @@ pub extern "C" fn externs_set( project_ignoring_type: ProjectIgnoringTypeExtern, project_multi: ProjectMultiExtern, create_exception: CreateExceptionExtern, - py_str_type: TypeId, ) { externs::set_externs(Externs { context, @@ -131,14 +130,14 @@ pub extern "C" fn externs_set( call, generator_send, eval, + product_type, + get_type_for, identify, equals, clone_val, drop_handles, type_to_str, val_to_str, - satisfied_by, - satisfied_by_type, store_tuple, store_set, store_dict, @@ -150,7 +149,6 @@ pub extern "C" fn externs_set( project_ignoring_type, project_multi, create_exception, - py_str_type, }); } @@ -178,19 +176,19 @@ pub extern "C" fn scheduler_create( construct_file_content: Function, construct_files_content: Function, construct_process_result: Function, - type_address: TypeConstraint, - type_path_globs: TypeConstraint, - type_directory_digest: TypeConstraint, - type_snapshot: TypeConstraint, - type_merge_directories_request: TypeConstraint, - type_files_content: TypeConstraint, - type_dir: TypeConstraint, - type_file: TypeConstraint, - type_link: TypeConstraint, - type_process_request: TypeConstraint, - type_process_result: TypeConstraint, - type_generator: TypeConstraint, - type_url_to_fetch: TypeConstraint, + type_address: TypeId, + type_path_globs: TypeId, + type_directory_digest: TypeId, + type_snapshot: TypeId, + type_merge_directories_request: TypeId, + type_files_content: TypeId, + type_dir: TypeId, + type_file: TypeId, + type_link: TypeId, + type_process_request: TypeId, + type_process_result: TypeId, + type_generator: TypeId, + type_url_to_fetch: TypeId, type_string: TypeId, type_bytes: TypeId, build_root_buf: Buffer, @@ -375,7 +373,7 @@ pub extern "C" fn execution_add_root_select( scheduler_ptr: *mut Scheduler, execution_request_ptr: *mut ExecutionRequest, param_vals: HandleBuffer, - product: TypeConstraint, + product: TypeId, ) -> PyResult { with_scheduler(scheduler_ptr, |scheduler| { with_execution_request(execution_request_ptr, |execution_request| { @@ -393,13 +391,9 @@ pub extern "C" fn tasks_create() -> *const Tasks { } #[no_mangle] -pub extern "C" fn tasks_singleton_add( - tasks_ptr: *mut Tasks, - handle: Handle, - output_constraint: TypeConstraint, -) { +pub extern "C" fn tasks_singleton_add(tasks_ptr: *mut Tasks, handle: Handle, output_type: TypeId) { with_tasks(tasks_ptr, |tasks| { - tasks.singleton_add(handle.into(), output_constraint); + tasks.singleton_add(handle.into(), output_type); }) } @@ -407,7 +401,7 @@ pub extern "C" fn tasks_singleton_add( pub extern "C" fn tasks_task_begin( tasks_ptr: *mut Tasks, func: Function, - output_type: TypeConstraint, + output_type: TypeId, cacheable: bool, ) { with_tasks(tasks_ptr, |tasks| { @@ -416,14 +410,14 @@ pub extern "C" fn tasks_task_begin( } #[no_mangle] -pub extern "C" fn tasks_add_get(tasks_ptr: *mut Tasks, product: TypeConstraint, subject: TypeId) { +pub extern "C" fn tasks_add_get(tasks_ptr: *mut Tasks, product: TypeId, subject: TypeId) { with_tasks(tasks_ptr, |tasks| { tasks.add_get(product, subject); }) } #[no_mangle] -pub extern "C" fn tasks_add_select(tasks_ptr: *mut Tasks, product: TypeConstraint) { +pub extern "C" fn tasks_add_select(tasks_ptr: *mut Tasks, product: TypeId) { with_tasks(tasks_ptr, |tasks| { tasks.add_select(product); }) @@ -594,7 +588,7 @@ pub extern "C" fn rule_graph_visualize( pub extern "C" fn rule_subgraph_visualize( scheduler_ptr: *mut Scheduler, subject_type: TypeId, - product_type: TypeConstraint, + product_type: TypeId, path_ptr: *const raw::c_char, ) { with_scheduler(scheduler_ptr, |scheduler| { @@ -609,6 +603,8 @@ pub extern "C" fn rule_subgraph_visualize( }) } +// TODO(#4884): This produces "thread panicked while processing panic. aborting." on my linux +// laptop, requiring me to set RUST_BACKTRACE=1 (or "full") to get the panic message. #[no_mangle] pub extern "C" fn set_panic_handler() { panic::set_hook(Box::new(|panic_info| { @@ -808,13 +804,9 @@ fn graph_full(scheduler: &Scheduler, subject_types: Vec) -> RuleGraph { graph_maker.full_graph() } -fn graph_sub( - scheduler: &Scheduler, - subject_type: TypeId, - product_type: TypeConstraint, -) -> RuleGraph { +fn graph_sub(scheduler: &Scheduler, subject_type: TypeId, product_type: TypeId) -> RuleGraph { let graph_maker = GraphMaker::new(&scheduler.core.tasks, vec![subject_type]); - graph_maker.sub_graph(subject_type, &product_type) + graph_maker.sub_graph(subject_type, product_type) } fn write_to_file(path: &Path, graph: &RuleGraph) -> io::Result<()> { diff --git a/src/rust/engine/src/nodes.rs b/src/rust/engine/src/nodes.rs index 599dda93c10..77d7aebe408 100644 --- a/src/rust/engine/src/nodes.rs +++ b/src/rust/engine/src/nodes.rs @@ -14,7 +14,7 @@ use futures::Stream; use url::Url; use crate::context::{Context, Core}; -use crate::core::{throw, Failure, Key, Params, TypeConstraint, Value}; +use crate::core::{throw, Failure, Key, Params, TypeId, Value}; use crate::externs; use crate::rule_graph; use crate::selectors; @@ -93,12 +93,12 @@ pub trait WrappedNode: Into { #[derive(Clone, Debug, Eq, Hash, PartialEq)] pub struct Select { pub params: Params, - pub product: TypeConstraint, + pub product: TypeId, entry: rule_graph::Entry, } impl Select { - pub fn new(mut params: Params, product: TypeConstraint, entry: rule_graph::Entry) -> Select { + pub fn new(mut params: Params, product: TypeId, entry: rule_graph::Entry) -> Select { params.retain(|k| match &entry { &rule_graph::Entry::Param(ref type_id) => type_id == k.type_id(), &rule_graph::Entry::WithDeps(ref with_deps) => with_deps.params().contains(k.type_id()), @@ -111,11 +111,7 @@ impl Select { } } - pub fn new_from_edges( - params: Params, - product: TypeConstraint, - edges: &rule_graph::RuleEdges, - ) -> Select { + pub fn new_from_edges(params: Params, product: TypeId, edges: &rule_graph::RuleEdges) -> Select { let select_key = rule_graph::SelectKey::JustSelect(selectors::Select::new(product)); // TODO: Is it worth propagating an error here? let entry = edges @@ -128,7 +124,7 @@ impl Select { fn select_product( &self, context: &Context, - product: TypeConstraint, + product: TypeId, caller_description: &str, ) -> NodeFuture { let edges = context @@ -138,8 +134,7 @@ impl Select { .ok_or_else(|| { throw(&format!( "Tried to select product {} for {} but found no edges", - externs::key_to_str(&product.0), - caller_description + product, caller_description )) }); let context = context.clone(); @@ -786,7 +781,7 @@ impl From for NodeKey { #[derive(Clone, Eq, Hash, PartialEq)] pub struct Task { params: Params, - product: TypeConstraint, + product: TypeId, task: tasks::Task, entry: Arc, } @@ -871,10 +866,7 @@ impl fmt::Debug for Task { write!( f, "Task({}, {}, {}, {})", - externs::project_str(&externs::val_for(&self.task.func.0), "__name__"), - self.params, - externs::key_to_str(&self.product.0), - self.task.cacheable, + self.task.func, self.params, self.product, self.task.cacheable, ) } } @@ -909,18 +901,14 @@ impl WrappedNode for Task { Err(failure) => Err(failure), }) .then(move |task_result| match task_result { - Ok(val) => { - if externs::satisfied_by(&context.core.types.generator, &val) { - Self::generate(context, params, entry, val) - } else if externs::satisfied_by(&product, &val) { - ok(val) - } else { - err(throw(&format!( - "{:?} returned a result value that did not satisfy its constraints: {:?}", - func, val - ))) - } - } + Ok(val) => match externs::get_type_for(&val) { + t if t == context.core.types.generator => Self::generate(context, params, entry, val), + t if t == product => ok(val), + _ => err(throw(&format!( + "{:?} returned a result value that did not satisfy its constraints: {:?}", + func, val + ))), + }, Err(failure) => err(failure), }) .to_boxed() @@ -1013,14 +1001,11 @@ pub enum NodeKey { impl NodeKey { fn product_str(&self) -> String { - fn typstr(tc: &TypeConstraint) -> String { - externs::key_to_str(&tc.0) - } match self { &NodeKey::ExecuteProcess(..) => "ProcessResult".to_string(), &NodeKey::DownloadedFile(..) => "DownloadedFile".to_string(), - &NodeKey::Select(ref s) => typstr(&s.product), - &NodeKey::Task(ref s) => typstr(&s.product), + &NodeKey::Select(ref s) => format!("{}", s.product), + &NodeKey::Task(ref s) => format!("{}", s.product), &NodeKey::Snapshot(..) => "Snapshot".to_string(), &NodeKey::DigestFile(..) => "DigestFile".to_string(), &NodeKey::ReadLink(..) => "LinkDest".to_string(), @@ -1095,14 +1080,9 @@ impl Display for NodeKey { &NodeKey::ExecuteProcess(ref s) => write!(f, "ExecuteProcess({:?}", s.0), &NodeKey::ReadLink(ref s) => write!(f, "ReadLink({:?})", s.0), &NodeKey::Scandir(ref s) => write!(f, "Scandir({:?})", s.0), - &NodeKey::Select(ref s) => write!( - f, - "Select({}, {})", - s.params, - externs::key_to_str(&s.product.0) - ), + &NodeKey::Select(ref s) => write!(f, "Select({}, {})", s.params, s.product,), &NodeKey::Task(ref s) => write!(f, "{:?}", s), - &NodeKey::Snapshot(ref s) => write!(f, "Snapshot({})", externs::key_to_str(&s.0)), + &NodeKey::Snapshot(ref s) => write!(f, "Snapshot({})", format!("{}", &s.0)), } } } diff --git a/src/rust/engine/src/rule_graph.rs b/src/rust/engine/src/rule_graph.rs index 80130a197de..30e39981bc8 100644 --- a/src/rust/engine/src/rule_graph.rs +++ b/src/rust/engine/src/rule_graph.rs @@ -6,7 +6,7 @@ use std::io; use itertools::Itertools; -use crate::core::{Function, Key, Params, TypeConstraint, TypeId, Value}; +use crate::core::{Key, Params, TypeId}; use crate::externs; use crate::selectors::{Get, Select}; use crate::tasks::{Intrinsic, Task, Tasks}; @@ -116,7 +116,7 @@ impl EntryWithDeps { pub enum Entry { Param(TypeId), WithDeps(EntryWithDeps), - Singleton(Key, TypeConstraint), + Singleton(Key, TypeId), } #[derive(Eq, Hash, PartialEq, Clone, Debug)] @@ -227,7 +227,7 @@ impl<'t> GraphMaker<'t> { } } - pub fn sub_graph(&self, param_type: TypeId, product_type: &TypeConstraint) -> RuleGraph { + pub fn sub_graph(&self, param_type: TypeId, product_type: TypeId) -> RuleGraph { // TODO: Update to support rendering a subgraph given a set of ParamTypes. let param_types = vec![param_type].into_iter().collect(); @@ -402,7 +402,7 @@ impl<'t> GraphMaker<'t> { let fulfillable_candidates = fulfillable_candidates_by_key .entry(select_key.clone()) .or_insert_with(Vec::new); - for candidate in rhs(&self.tasks, ¶ms, &product) { + for candidate in rhs(&self.tasks, ¶ms, product) { match candidate { Entry::WithDeps(c) => match self.construct_graph_helper( rule_dependency_edges, @@ -455,13 +455,12 @@ impl<'t> GraphMaker<'t> { reason: if params.is_empty() { format!( "No rule was available to compute {}. Maybe declare it as a RootRule({})?", - type_constraint_str(product), - type_constraint_str(product), + product, product, ) } else { format!( "No rule was available to compute {} with parameter type{} {}", - type_constraint_str(product), + product, if params.len() > 1 { "s" } else { "" }, params_str(¶ms), ) @@ -735,25 +734,21 @@ impl<'t> GraphMaker<'t> { }) } - fn gen_root_entries(&self, product_types: &HashSet) -> Vec { + fn gen_root_entries(&self, product_types: &HashSet) -> Vec { product_types .iter() - .filter_map(|product_type| self.gen_root_entry(&self.root_param_types, product_type)) + .filter_map(|product_type| self.gen_root_entry(&self.root_param_types, *product_type)) .collect() } - fn gen_root_entry( - &self, - param_types: &ParamTypes, - product_type: &TypeConstraint, - ) -> Option { + fn gen_root_entry(&self, param_types: &ParamTypes, product_type: TypeId) -> Option { let candidates = rhs(&self.tasks, param_types, product_type); if candidates.is_empty() { None } else { Some(RootEntry { params: param_types.clone(), - clause: vec![Select::new(*product_type)], + clause: vec![Select::new(product_type)], gets: vec![], }) } @@ -783,42 +778,14 @@ pub struct RuleGraph { unreachable_rules: Vec, } -// TODO: Take by reference. -fn type_constraint_str(type_constraint: TypeConstraint) -> String { - let str_val = externs::call_method(&to_val(type_constraint), "graph_str", &[]) - .expect("string from calling repr"); - externs::val_to_str(&str_val) -} - -fn to_val(type_constraint: TypeConstraint) -> Value { - externs::val_for(&type_constraint.0) -} - -fn to_val_from_func(func: &Function) -> Value { - externs::val_for(&func.0) -} - -fn function_str(func: &Function) -> String { - let as_val = to_val_from_func(func); - val_name(&as_val) -} - -pub fn type_str(type_id: TypeId) -> String { - format!("{}", type_id) -} - pub fn params_str(params: &ParamTypes) -> String { let param_names = params .iter() - .map(|type_id| type_str(*type_id)) + .map(|type_id| format!("{}", *type_id)) .collect::>(); Params::display(param_names) } -fn val_name(val: &Value) -> String { - externs::project_str(val, "__name__") -} - pub fn select_key_str(select_key: &SelectKey) -> String { match select_key { &SelectKey::JustSelect(ref s) => select_str(s), @@ -827,15 +794,11 @@ pub fn select_key_str(select_key: &SelectKey) -> String { } pub fn select_str(select: &Select) -> String { - format!("Select({})", type_constraint_str(select.product)).to_string() // TODO variant key + format!("Select({})", select.product).to_string() // TODO variant key } fn get_str(get: &Get) -> String { - format!( - "Get({}, {})", - type_constraint_str(get.product), - type_str(get.subject) - ) + format!("Get({}, {})", get.product, get.subject) } /// @@ -844,12 +807,10 @@ fn get_str(get: &Get) -> String { pub fn entry_str(entry: &Entry) -> String { match entry { &Entry::WithDeps(ref e) => entry_with_deps_str(e), - &Entry::Param(type_id) => format!("Param({})", type_str(type_id)), - &Entry::Singleton(value, product) => format!( - "Singleton({}, {})", - externs::key_to_str(&value), - type_constraint_str(product) - ), + &Entry::Param(type_id) => format!("Param({})", type_id), + &Entry::Singleton(value, product) => { + format!("Singleton({}, {})", externs::key_to_str(&value), product) + } } } @@ -864,8 +825,8 @@ fn entry_with_deps_str(entry: &EntryWithDeps) -> String { ref params, }) => format!( "({}, ({},) for {}", - type_constraint_str(intrinsic.product), - type_constraint_str(intrinsic.input), + intrinsic.product, + intrinsic.input, params_str(params) ), &EntryWithDeps::Root(ref root) => format!( @@ -882,7 +843,7 @@ fn entry_with_deps_str(entry: &EntryWithDeps) -> String { } fn task_display(task: &Task) -> String { - let product = type_constraint_str(task.product); + let product = format!("{}", task.product); let mut clause_portion = task .clause .iter() @@ -901,10 +862,9 @@ fn task_display(task: &Task) -> String { } else { format!("[{}], ", get_portion) }; - let function_name = function_str(&&task.func); format!( "({}, {}, {}{})", - product, clause_portion, get_portion, function_name + product, clause_portion, get_portion, task.func, ) .to_string() } @@ -1060,7 +1020,7 @@ impl RuleGraph { let mut root_subject_type_strs = self .root_param_types .iter() - .map(|&t| type_str(t)) + .map(|&t| format!("{}", t)) .collect::>(); root_subject_type_strs.sort(); writeln!(f, "digraph {{")?; @@ -1151,28 +1111,26 @@ impl RuleEdges { /// /// Select Entries that can provide the given product type with the given parameters. /// -fn rhs(tasks: &Tasks, params: &ParamTypes, product_type: &TypeConstraint) -> Vec { +fn rhs(tasks: &Tasks, params: &ParamTypes, product_type: TypeId) -> Vec { if let Some(&(ref key, _)) = tasks.gen_singleton(product_type) { - return vec![Entry::Singleton(*key, *product_type)]; + return vec![Entry::Singleton(*key, product_type)]; } let mut entries = Vec::new(); - if let Some(type_id) = params - .iter() - .find(|&&type_id| externs::satisfied_by_type(product_type, type_id)) - { - // TODO: We only match the first param type here that satisfies the constraint although it's - // possible that multiple parameters could. Would be nice to be able to remove TypeConstraint. + // If the params can provide the type directly, add that. + if let Some(type_id) = params.get(&product_type) { entries.push(Entry::Param(*type_id)); } + // If there are any intrinsics which can produce the desired type, add them. if let Some(matching_intrinsics) = tasks.gen_intrinsic(product_type) { - for matching_intrinsic in matching_intrinsics { - entries.push(Entry::WithDeps(EntryWithDeps::Inner(InnerEntry { + entries.extend(matching_intrinsics.iter().map(|matching_intrinsic| { + Entry::WithDeps(EntryWithDeps::Inner(InnerEntry { params: params.clone(), rule: Rule::Intrinsic(*matching_intrinsic), - }))); - } + })) + })); } + // If there are any tasks which can produce the desired type, add those. if let Some(matching_tasks) = tasks.gen_tasks(product_type) { entries.extend(matching_tasks.iter().map(|task_rule| { Entry::WithDeps(EntryWithDeps::Inner(InnerEntry { diff --git a/src/rust/engine/src/scheduler.rs b/src/rust/engine/src/scheduler.rs index dff4f06bdb2..8e2fc08c97e 100644 --- a/src/rust/engine/src/scheduler.rs +++ b/src/rust/engine/src/scheduler.rs @@ -10,7 +10,7 @@ use std::time::Duration; use futures::future::{self, Future}; use crate::context::{Context, Core}; -use crate::core::{Failure, Params, TypeConstraint, Value}; +use crate::core::{Failure, Params, TypeId, Value}; use crate::nodes::{NodeKey, Select, Tracer, TryInto, Visualizer}; use crate::selectors; use graph::{EntryId, Graph, InvalidationResult, NodeContext}; @@ -107,7 +107,7 @@ impl Scheduler { &self, request: &mut ExecutionRequest, params: Params, - product: TypeConstraint, + product: TypeId, ) -> Result<(), String> { let edges = self .core diff --git a/src/rust/engine/src/selectors.rs b/src/rust/engine/src/selectors.rs index 66b2881a676..e8255d951bf 100644 --- a/src/rust/engine/src/selectors.rs +++ b/src/rust/engine/src/selectors.rs @@ -1,33 +1,28 @@ // Copyright 2017 Pants project contributors (see CONTRIBUTORS.md). // Licensed under the Apache License, Version 2.0 (see LICENSE). -use crate::core::{TypeConstraint, TypeId}; -use crate::externs; +use crate::core::TypeId; use std::fmt; #[derive(Clone, Copy, Debug, Eq, Hash, PartialEq)] pub struct Get { - pub product: TypeConstraint, + pub product: TypeId, pub subject: TypeId, } #[derive(Clone, Eq, Hash, PartialEq)] pub struct Select { - pub product: TypeConstraint, + pub product: TypeId, } impl Select { - pub fn new(product: TypeConstraint) -> Select { + pub fn new(product: TypeId) -> Select { Select { product: product } } } impl fmt::Debug for Select { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - write!( - f, - "Select {{ product: {} }}", - externs::key_to_str(&self.product.0) - ) + write!(f, "Select {{ product: {} }}", self.product,) } } diff --git a/src/rust/engine/src/tasks.rs b/src/rust/engine/src/tasks.rs index bfe028e3306..91280f81b3e 100644 --- a/src/rust/engine/src/tasks.rs +++ b/src/rust/engine/src/tasks.rs @@ -3,14 +3,14 @@ use std::collections::{HashMap, HashSet}; -use crate::core::{Function, Key, TypeConstraint, TypeId, Value, FNV}; +use crate::core::{Function, Key, TypeId, Value, FNV}; use crate::externs; use crate::selectors::{Get, Select}; use crate::types::Types; #[derive(Clone, Debug, Eq, Hash, PartialEq)] pub struct Task { - pub product: TypeConstraint, + pub product: TypeId, pub clause: Vec