def _type_check_output(output_def: "OutputDefinition", output: Any, context: "BoundSolidExecutionContext") -> Any: """Validates and performs core type check on a provided output. Args: output_def (OutputDefinition): The output definition to validate against. output (Any): The output to validate. context (BoundSolidExecutionContext): Context containing resources to be used for type check. """ from ..execution.plan.execute_step import do_type_check op_label = context.describe_op() if isinstance(output, (Output, DynamicOutput)): dagster_type = output_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, output.value) if not type_check.success: raise DagsterTypeCheckDidNotPass( description= (f'Type check failed for {op_label} output "{output.output_name}" - ' f'expected type "{dagster_type.display_name}". ' f"Description: {type_check.description}"), metadata_entries=type_check.metadata_entries, dagster_type=dagster_type, ) context.observe_output( output.output_name, output.mapping_key if isinstance(output, DynamicOutput) else None) return output else: dagster_type = output_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, output) if not type_check.success: raise DagsterTypeCheckDidNotPass( description= (f'Type check failed for {op_label} output "{output_def.name}" - ' f'expected type "{dagster_type.display_name}". ' f"Description: {type_check.description}"), metadata_entries=type_check.metadata_entries, dagster_type=dagster_type, ) return output
def _execute_and_retrieve_outputs(solid_def: "SolidDefinition", context: "DirectSolidExecutionContext", input_dict: Dict[str, Any]) -> tuple: from dagster.core.execution.plan.execute_step import do_type_check output_values = {} output_defs = { output_def.name: output_def for output_def in solid_def.output_defs } for output in _core_generator(solid_def, context, input_dict): if not isinstance(output, AssetMaterialization): if output.output_name in output_values: raise DagsterInvariantViolationError( f'Solid "{solid_def.name}" returned an output "{output.output_name}" multiple ' "times") elif output.output_name not in output_defs: raise DagsterInvariantViolationError( f'Solid "{solid_def.name}" returned an output "{output.output_name}" that does ' f"not exist. The available outputs are {list(output_defs)}" ) else: dagster_type = output_defs[output.output_name].dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, output.value) if not type_check.success: raise DagsterTypeCheckDidNotPass( description= (f'Type check failed for solid output "{output.output_name}" - ' f'expected type "{dagster_type.display_name}". ' f"Description: {type_check.description}."), metadata_entries=type_check.metadata_entries, dagster_type=dagster_type, ) output_values[output.output_name] = output.value else: context.record_materialization(output) # Check to make sure all non-optional outputs were yielded. for output_def in solid_def.output_defs: if output_def.name not in output_values and output_def.is_required: raise DagsterInvariantViolationError( f'Solid "{solid_def.name}" did not return an output for non-optional ' f'output "{output_def.name}"') # Explicitly preserve the ordering of output defs return tuple([ output_values[output_def.name] for output_def in solid_def.output_defs ])
def _resolve_inputs(solid_def: "SolidDefinition", args, kwargs, context: "BoundSolidExecutionContext"): from dagster.core.execution.plan.execute_step import do_type_check input_defs = solid_def.input_defs # Fail early if too many inputs were provided. if len(input_defs) < len(args) + len(kwargs): raise DagsterInvalidInvocationError( f"Too many input arguments were provided for solid '{context.alias}'. This may be because " "an argument was provided for the context parameter, but no context parameter was defined " "for the solid.") input_dict = { input_def.name: input_val for input_val, input_def in zip(args, input_defs[:len(args)]) } for input_def in input_defs[len(args):]: if not input_def.has_default_value and input_def.name not in kwargs: raise DagsterInvalidInvocationError( f'No value provided for required input "{input_def.name}".') input_dict[input_def.name] = (kwargs[input_def.name] if input_def.name in kwargs else input_def.default_value) # Type check inputs input_defs_by_name = { input_def.name: input_def for input_def in input_defs } for input_name, val in input_dict.items(): input_def = input_defs_by_name[input_name] dagster_type = input_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, val) if not type_check.success: raise DagsterTypeCheckDidNotPass( description=( f'Type check failed for solid input "{input_def.name}" - ' f'expected type "{dagster_type.display_name}". ' f"Description: {type_check.description}."), metadata_entries=type_check.metadata_entries, dagster_type=dagster_type, ) return input_dict
def _resolve_inputs(solid_def: "SolidDefinition", args, kwargs, context: "BoundSolidExecutionContext"): from dagster.core.execution.plan.execute_step import do_type_check nothing_input_defs = [ input_def for input_def in solid_def.input_defs if input_def.dagster_type.is_nothing ] # Check kwargs for nothing inputs, and error if someone provided one. for input_def in nothing_input_defs: if input_def.name in kwargs: node_label = solid_def.node_type_str # string "solid" for solids, "op" for ops raise DagsterInvalidInvocationError( f"Attempted to provide value for nothing input '{input_def.name}'. Nothing " f"dependencies are ignored when directly invoking {node_label}s." ) # Discard nothing dependencies - we ignore them during invocation. input_defs_by_name = { input_def.name: input_def for input_def in solid_def.input_defs if not input_def.dagster_type.is_nothing } # Fail early if too many inputs were provided. if len(input_defs_by_name) < len(args) + len(kwargs): if len(nothing_input_defs) > 0: suggestion = ( "This may be because you attempted to provide a value for a nothing " "dependency. Nothing dependencies are ignored when directly invoking solids." ) else: suggestion = ( "This may be because an argument was provided for the context parameter, " "but no context parameter was defined for the solid.") node_label = solid_def.node_type_str raise DagsterInvalidInvocationError( f"Too many input arguments were provided for {node_label} '{context.alias}'. {suggestion}" ) positional_inputs = cast("DecoratedSolidFunction", solid_def.compute_fn).positional_inputs() input_dict = {} for position, value in enumerate(args): input_dict[positional_inputs[position]] = value for positional_input in positional_inputs[len(args):]: input_def = input_defs_by_name[positional_input] if not input_def.has_default_value and positional_input not in kwargs: raise DagsterInvalidInvocationError( f'No value provided for required input "{positional_input}".') input_dict[positional_input] = (kwargs[positional_input] if positional_input in kwargs else input_def.default_value) # Type check inputs op_label = context.describe_op() for input_name, val in input_dict.items(): input_def = input_defs_by_name[input_name] dagster_type = input_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, val) if not type_check.success: raise DagsterTypeCheckDidNotPass( description= (f'Type check failed for {op_label} input "{input_def.name}" - ' f'expected type "{dagster_type.display_name}". ' f"Description: {type_check.description}"), metadata_entries=type_check.metadata_entries, dagster_type=dagster_type, ) return input_dict