def __call__(self, fn: Callable[..., Any]) -> JobDefinition: check.callable_param(fn, "fn") if not self.name: self.name = fn.__name__ from dagster.core.definitions.decorators.composite_solid_decorator import do_composition ( input_mappings, output_mappings, dependencies, solid_defs, config_mapping, positional_inputs, ) = do_composition( decorator_name="@job", graph_name=self.name, fn=fn, provided_input_defs=[], provided_output_defs=[], ignore_output_from_composition_fn=False, config_mapping=None, ) graph_def = GraphDefinition( name=self.name, dependencies=dependencies, node_defs=solid_defs, description=self.description or format_docstring_for_description(fn), input_mappings=input_mappings, output_mappings=output_mappings, config=config_mapping, positional_inputs=positional_inputs, tags=self.tags, ) job_def = graph_def.to_job( description=self.description or format_docstring_for_description(fn), resource_defs=self.resource_defs, config=self.config, tags=self.tags, logger_defs=self.logger_defs, executor_def=self.executor_def, hooks=self.hooks, op_retry_policy=self.op_retry_policy, version_strategy=self.version_strategy, partitions_def=self.partitions_def, ) update_wrapper(job_def, fn) return job_def
def __call__(self, resource_fn: Callable[["InitResourceContext"], Any]): check.callable_param(resource_fn, "resource_fn") any_name = ["*"] if is_context_provided( get_function_params(resource_fn)) else [] params = get_function_params(resource_fn) missing_positional = validate_expected_params(params, any_name) if missing_positional: raise DagsterInvalidDefinitionError( f"@resource decorated function '{resource_fn.__name__}' expects a single " "positional argument.") extras = params[len(any_name):] required_extras = list(filter(is_required_param, extras)) if required_extras: raise DagsterInvalidDefinitionError( f"@resource decorated function '{resource_fn.__name__}' expects only a single positional required argument. " f"Got required extra params {', '.join(positional_arg_name_list(required_extras))}" ) resource_def = ResourceDefinition( resource_fn=resource_fn, config_schema=self.config_schema, description=self.description or format_docstring_for_description(resource_fn), version=self.version, required_resource_keys=self.required_resource_keys, ) update_wrapper(resource_def, wrapped=resource_fn) return resource_def
def __call__(self, fn: Callable[..., Any]) -> PipelineDefinition: check.callable_param(fn, "fn") if not self.name: self.name = fn.__name__ from dagster.core.definitions.decorators.composite_solid import ( do_composition, get_validated_config_mapping, ) config_mapping = get_validated_config_mapping( self.name, self.config_schema, self.config_fn, decorator_name="pipeline") ( input_mappings, output_mappings, dependencies, solid_defs, config_mapping, positional_inputs, ) = do_composition( "@pipeline", self.name, fn, self.input_defs, self.output_defs, config_mapping, ignore_output_from_composition_fn=not self.did_pass_outputs, ) pipeline_def = PipelineDefinition( mode_defs=self.mode_definitions, preset_defs=self.preset_definitions, graph_def=GraphDefinition( name=self.name, description=None, # put desc on the pipeline dependencies=dependencies, node_defs=solid_defs, input_mappings=input_mappings, output_mappings=output_mappings, config=config_mapping, positional_inputs=positional_inputs, ), tags=self.tags, description=self.description or format_docstring_for_description(fn), hook_defs=self.hook_defs, solid_retry_policy=self.solid_retry_policy, version_strategy=self.version_strategy, ) update_wrapper(pipeline_def, fn) return pipeline_def
def test_format_docstring_for_description(): def multiline_indented_docstring(): """ abc 123 """ multiline_indented_docstring_expected = "abc\n123" assert (format_docstring_for_description(multiline_indented_docstring) == multiline_indented_docstring_expected) def no_indentation_at_start(): """abc 123 """ no_indentation_at_start_expected = "abc\n123" assert (format_docstring_for_description(no_indentation_at_start) == no_indentation_at_start_expected)
def __call__(self, fn: Callable[..., Any]) -> GraphDefinition: check.callable_param(fn, "fn") if not self.name: self.name = fn.__name__ if self.ins is not None: input_defs = [inp.to_definition(name) for name, inp in self.ins.items()] else: input_defs = check.opt_list_param( self.input_defs, "input_defs", of_type=InputDefinition ) if self.out is None: output_defs = self.output_defs elif isinstance(self.out, GraphOut): output_defs = [self.out.to_definition(name=None)] else: check.dict_param(self.out, "out", key_type=str, value_type=GraphOut) output_defs = [out.to_definition(name=name) for name, out in self.out.items()] from dagster.core.definitions.decorators.composite_solid_decorator import do_composition ( input_mappings, output_mappings, dependencies, solid_defs, config_mapping, positional_inputs, ) = do_composition( decorator_name="@graph", graph_name=self.name, fn=fn, provided_input_defs=input_defs, provided_output_defs=output_defs, ignore_output_from_composition_fn=False, config_mapping=self.config_mapping, ) graph_def = GraphDefinition( name=self.name, dependencies=dependencies, node_defs=solid_defs, description=self.description or format_docstring_for_description(fn), input_mappings=input_mappings, output_mappings=output_mappings, config=config_mapping, positional_inputs=positional_inputs, tags=self.tags, ) update_wrapper(graph_def, fn) return graph_def
def __call__(self, fn: Callable[..., Any]) -> SolidDefinition: check.callable_param(fn, "fn") if not self.name: self.name = fn.__name__ output_defs: Sequence[OutputDefinition] if self.output_defs is None: output_defs = [ OutputDefinition.create_from_inferred(infer_output_props(fn)) ] elif len(self.output_defs) == 1: output_defs = [ self.output_defs[0].combine_with_inferred( infer_output_props(fn)) ] else: output_defs = self.output_defs compute_fn = (DecoratedSolidFunction( decorated_fn=fn) if self.decorator_takes_context else NoContextDecoratedSolidFunction(decorated_fn=fn)) resolved_input_defs = resolve_checked_solid_fn_inputs( decorator_name="@solid", fn_name=self.name, compute_fn=compute_fn, explicit_input_defs=self.input_defs, exclude_nothing=True, ) solid_def = SolidDefinition( name=self.name, input_defs=resolved_input_defs, output_defs=output_defs, compute_fn=compute_fn, config_schema=self.config_schema, description=self.description or format_docstring_for_description(fn), required_resource_keys=self.required_resource_keys, tags=self.tags, version=self.version, retry_policy=self.retry_policy, ) update_wrapper(solid_def, compute_fn.decorated_fn) return solid_def
def __call__(self, fn: Callable[..., Any]): check.callable_param(fn, "fn") if not self.name: self.name = fn.__name__ config_mapping = get_validated_config_mapping( self.name, self.config_schema, self.config_fn, decorator_name="composite_solid") ( input_mappings, output_mappings, dependencies, solid_defs, config_mapping, positional_inputs, ) = do_composition( "@composite_solid", self.name, fn, self.input_defs, self.output_defs, config_mapping, ignore_output_from_composition_fn=False, ) composite_def = CompositeSolidDefinition( name=self.name, input_mappings=input_mappings, output_mappings=output_mappings, dependencies=dependencies, solid_defs=solid_defs, description=self.description or format_docstring_for_description(fn), config_mapping=config_mapping, positional_inputs=positional_inputs, ) update_wrapper(composite_def, fn) return composite_def
def __call__(self, fn: Callable[..., Any]) -> "OpDefinition": from ..op_definition import OpDefinition if self.input_defs is not None and self.ins is not None: check.failed( "Values cannot be provided for both the 'input_defs' and 'ins' arguments" ) if self.output_defs is not None and self.out is not None: check.failed( "Values cannot be provided for both the 'output_defs' and 'out' arguments" ) inferred_out = infer_output_props(fn) if self.ins is not None: input_defs = [ inp.to_definition(name) for name, inp in self.ins.items() ] else: input_defs = check.opt_list_param(self.input_defs, "input_defs", of_type=InputDefinition) output_defs_from_out = _resolve_output_defs_from_outs( inferred_out=inferred_out, out=self.out) resolved_output_defs = (output_defs_from_out if output_defs_from_out is not None else self.output_defs) if not self.name: self.name = fn.__name__ if resolved_output_defs is None: resolved_output_defs = [ OutputDefinition.create_from_inferred(infer_output_props(fn)) ] elif len(resolved_output_defs) == 1: resolved_output_defs = [ resolved_output_defs[0].combine_with_inferred( infer_output_props(fn)) ] compute_fn = (DecoratedSolidFunction( decorated_fn=fn) if self.decorator_takes_context else NoContextDecoratedSolidFunction(decorated_fn=fn)) resolved_input_defs = resolve_checked_solid_fn_inputs( decorator_name="@op", fn_name=self.name, compute_fn=compute_fn, explicit_input_defs=input_defs, exclude_nothing=True, ) op_def = OpDefinition( name=self.name, input_defs=resolved_input_defs, output_defs=resolved_output_defs, compute_fn=compute_fn, config_schema=self.config_schema, description=self.description or format_docstring_for_description(fn), required_resource_keys=self.required_resource_keys, tags=self.tags, version=self.version, retry_policy=self.retry_policy, ) update_wrapper(op_def, compute_fn.decorated_fn) return op_def