Graphs
The core of a job is a graph of ops - connected via data dependencies.
- @dagster.graph [source]
- Create an op graph with the specified parameters from the decorated composition function. - Using this decorator allows you to build up a dependency graph by writing a function that invokes ops (or other graphs) and passes the output to subsequent invocations. - Parameters: - 
name (Optional[str]) – The name of the op graph. Must be unique within any RepositoryDefinitioncontaining the graph.
- 
description (Optional[str]) – A human-readable description of the graph. 
- 
input_defs (Optional[List[InputDefinition]]) – Information about the inputs that this graph maps. Information provided here will be combined with what can be inferred from the function signature, with these explicit InputDefinitions taking precedence. 
- 
output_defs (Optional[List[OutputDefinition]]) – Output definitions for the graph. If not provided explicitly, these will be inferred from typehints. Uses of these outputs in the body of the decorated composition function, as well as the return value of the decorated function, will be used to infer the appropriate set of OutputMappingsfor the underlyingGraphDefinition.
- 
ins (Optional[Dict[str, GraphIn]]) – Information about the inputs that this graph maps. Information provided here will be combined with what can be inferred from the function signature, with these explicit GraphIn taking precedence. 
- 
out – Information about the outputs that this graph maps. Information provided here will be combined with what can be inferred from the return type signature if the function does not use yield. 
 
- 
- classdagster.GraphDefinition [source]
- Defines a Dagster op graph. - An op graph is made up of - Nodes, which can either be an op (the functional unit of computation), or another graph.
- Dependencies, which determine how the values produced by nodes as outputs flow from one node to another. This tells Dagster how to arrange nodes into a directed, acyclic graph (DAG) of compute.
 - End users should prefer the - @graphdecorator. GraphDefinition is generally intended to be used by framework authors or for programatically generated graphs.- Parameters: - name (str) – The name of the graph. Must be unique within any GraphDefinitionorJobDefinitioncontaining the graph.
- description (Optional[str]) – A human-readable description of the job.
- node_defs (Optional[Sequence[NodeDefinition]]) – The set of ops / graphs used in this graph.
- dependencies (Optional[Dict[Union[str, NodeInvocation], Dict[str, DependencyDefinition]]]) – A structure that declares the dependencies of each op’s inputs on the outputs of other ops in the graph. Keys of the top level dict are either the string names of ops in the graph or, in the case of aliased ops, NodeInvocations. Values of the top level dict are themselves dicts, which map input names belonging to the op or aliased op toDependencyDefinitions.
- input_mappings (Optional[Sequence[InputMapping]]) – Defines the inputs to the nested graph, and how they map to the inputs of its constituent ops.
- output_mappings (Optional[Sequence[OutputMapping]]) – Defines the outputs of the nested graph, and how they map from the outputs of its constituent ops.
- config (Optional[ConfigMapping]) – Defines the config of the graph, and how its schema maps to the config of its constituent ops.
- tags (Optional[Dict[str, Any]]) – Arbitrary metadata for any execution of the graph. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value. These tag values may be overwritten by tag values provided at invocation time.
- composition_fn (Optional[Callable]) – The function that defines this graph. Used to generate code references for this graph.
 - Examples: - @op
 def return_one():
 return 1
 @op
 def add_one(num):
 return num + 1
 graph_def = GraphDefinition(
 name='basic',
 node_defs=[return_one, add_one],
 dependencies={'add_one': {'num': DependencyDefinition('return_one')}},
 )- alias [source]
- Aliases the graph with a new name. - Can only be used in the context of a Examples:- @graph,- @job, or- @asset_graphdecorated function.- @job
 def do_it_all():
 my_graph.alias("my_graph_alias")
 - execute_in_process [source]
- Execute this graph in-process, collecting results in-memory. - Parameters: - run_config (Optional[Mapping[str, Any]]) – Run config to provide to execution. The configuration for the underlying graph should exist under the “ops” key.
- instance (Optional[DagsterInstance]) – The instance to execute against, an ephemeral one will be used if none provided.
- resources (Optional[Mapping[str, Any]]) – The resources needed if any are required. Can provide resource instances directly, or resource definitions.
- raise_on_error (Optional[bool]) – Whether or not to raise exceptions when they occur. Defaults to True.
- op_selection (Optional[List[str]]) – A list of op selection queries (including single op names) to execute. For example: * ['some_op']: selectssome_opitself. *['*some_op']: selectsome_opand all its ancestors (upstream dependencies). *['*some_op+++']: selectsome_op, all its ancestors, and its descendants (downstream dependencies) within 3 levels down. *['*some_op', 'other_op_a', 'other_op_b+']: selectsome_opand all its ancestors,other_op_aitself, andother_op_band its direct child ops.
- input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of the graph.
 - Returns: - ExecuteInProcessResult
 - tag [source]
- Attaches the provided tags to the graph immutably. - Can only be used in the context of a Examples:- @graph,- @job, or- @asset_graphdecorated function.- @job
 def do_it_all():
 my_graph.tag({"my_tag": "my_value"})
 - to_job [source]
- Make this graph in to an executable Job by providing remaining components required for execution. - Parameters: - 
name (Optional[str]) – The name for the Job. Defaults to the name of the this graph. 
- 
resource_defs (Optional[Mapping [str, object]]) – Resources that are required by this graph for execution. If not defined, io_manager will default to filesystem. 
- 
config – Describes how the job is parameterized at runtime. If no value is provided, then the schema for the job’s run config is a standard format based on its ops and resources. If a dictionary is provided, then it must conform to the standard config schema, and it will be used as the job’s run config for the job whenever the job is executed. The values provided will be viewable and editable in the Dagster UI, so be careful with secrets. If a ConfigMappingobject is provided, then the schema for the job’s run config is determined by the config mapping, and the ConfigMapping, which should return configuration in the standard format to configure the job.
- 
tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can be used for searching and filtering in the UI. Values that are not already strings will be serialized as JSON. If run_tags is not set, then the content of tags will also be automatically appended to the tags of any runs of this job. 
- 
run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this job. Values that are not already strings will be serialized as JSON. These tag values may be overwritten by tag values provided at invocation time. If run_tags is set, then tags are not automatically appended to the tags of any runs of this job. 
- 
metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary information that will be attached to the JobDefinition and be viewable in the Dagster UI. Keys must be strings, and values must be python primitive types or one of the provided MetadataValue types 
- 
logger_defs (Optional[Mapping[str, LoggerDefinition]]) – A dictionary of string logger identifiers to their implementations. 
- 
executor_def (Optional[ExecutorDefinition]) – How this Job will be executed. Defaults to multi_or_in_process_executor, which can be switched between multi-process and in-process modes of execution. The default mode of execution is multi-process.
- 
op_retry_policy (Optional[RetryPolicy]) – The default retry policy for all ops in this job. Only used if retry policy is not defined on the op definition or op invocation. 
- 
partitions_def (Optional[PartitionsDefinition]) – Defines a discrete set of partition keys that can parameterize the job. If this argument is supplied, the config argument can’t also be supplied. 
- 
asset_layer (Optional[AssetLayer]) – Top level information about the assets this job will produce. Generally should not be set manually. 
- 
input_values (Optional[Mapping[str, Any]]) – A dictionary that maps python objects to the top-level inputs of a job. 
 - Returns: JobDefinition 
- 
 - with_hooks [source]
- Attaches the provided hooks to the graph immutably. - Can only be used in the context of a Examples:- @graph,- @job, or- @asset_graphdecorated function.- @job
 def do_it_all():
 my_graph.with_hooks({my_hook})
 - with_retry_policy [source]
- Attaches the provided retry policy to the graph immutably. - Can only be used in the context of a Examples:- @graph,- @job, or- @asset_graphdecorated function.- @job
 def do_it_all():
 my_graph.with_retry_policy(RetryPolicy(max_retries=5))
 - propertyconfig_mapping [source]
- The config mapping for the graph, if present. - By specifying a config mapping function, you can override the configuration for the child nodes contained within a graph. 
 - propertyinput_mappings [source]
- Input mappings for the graph. - An input mapping is a mapping from an input of the graph to an input of a child node. 
 - propertyname [source]
- The name of the graph. 
 - propertyoutput_mappings [source]
- Output mappings for the graph. - An output mapping is a mapping from an output of the graph to an output of a child node. 
 - propertytags [source]
- The tags associated with the graph. 
 
- classdagster.GraphIn [source]
- Represents information about an input that a graph maps. - Parameters: description (Optional[str]) – Human-readable description of the input. 
- classdagster.GraphOut [source]
- Represents information about the outputs that a graph maps. - Parameters: description (Optional[str]) – Human-readable description of the output. 
Explicit dependencies
- classdagster.DependencyDefinition [source]
- Represents an edge in the DAG of nodes (ops or graphs) forming a job. - This object is used at the leaves of a dictionary structure that represents the complete dependency structure of a job whose keys represent the dependent node and dependent input, so this object only contains information about the dependee. - Concretely, if the input named ‘input’ of op_b depends on the output named ‘result’ of op_a, and the output named ‘other_result’ of graph_a, the structure will look as follows: - from dagster import DependencyDefinition
 dependency_structure = {
 'my_downstream_op': {
 'input': DependencyDefinition('my_upstream_op', 'result')
 },
 'my_other_downstream_op': {
 'input': DependencyDefinition('my_upstream_graph', 'result')
 }
 }- In general, users should prefer not to construct this class directly or use the - JobDefinitionAPI that requires instances of this class. Instead, use the- @jobAPI:- from dagster import job
 @job
 def the_job():
 node_b(node_a())- Parameters: - node (str) – The name of the node (op or graph) that is depended on, that is, from which the value passed between the two nodes originates.
- output (Optional[str]) – The name of the output that is depended on. (default: “result”)
- description (Optional[str]) – Human-readable description of this dependency.
 
- classdagster.MultiDependencyDefinition [source]
- Represents a fan-in edge in the DAG of op instances forming a job. - This object is used only when an input of type - List[T]is assembled by fanning-in multiple upstream outputs of type- T.- This object is used at the leaves of a dictionary structure that represents the complete dependency structure of a job whose keys represent the dependent ops or graphs and dependent input, so this object only contains information about the dependee. - Concretely, if the input named ‘input’ of op_c depends on the outputs named ‘result’ of op_a and op_b, this structure will look as follows: - dependency_structure = {
 'op_c': {
 'input': MultiDependencyDefinition(
 [
 DependencyDefinition('op_a', 'result'),
 DependencyDefinition('op_b', 'result')
 ]
 )
 }
 }- In general, users should prefer not to construct this class directly or use the - JobDefinitionAPI that requires instances of this class. Instead, use the- @jobAPI:- @job
 def the_job():
 op_c(op_a(), op_b())- Parameters: dependencies (List[Union[DependencyDefinition, Type[MappedInputPlaceHolder]]]) – List of upstream dependencies fanned in to this input. - get_dependencies_and_mappings [source]
- Return the combined list of dependencies contained by this object, inculding of - DependencyDefinitionand- MappedInputPlaceholderobjects.
 - get_node_dependencies [source]
- Return the list of - DependencyDefinitioncontained by this object.
 
- classdagster.NodeInvocation [source]
- Identifies an instance of a node in a graph dependency structure. - Parameters: - name (str) – Name of the node of which this is an instance.
- alias (Optional[str]) – Name specific to this instance of the node. Necessary when there are multiple instances of the same node.
- tags (Optional[Dict[str, Any]]) – Optional tags values to extend or override those set on the node definition.
- hook_defs (Optional[AbstractSet[HookDefinition]]) – A set of hook definitions applied to the node instance.
 - Examples: In general, users should prefer not to construct this class directly or use the - JobDefinitionAPI that requires instances of this class. Instead, use the- @jobAPI:- from dagster import job
 @job
 def my_job():
 other_name = some_op.alias('other_name')
 some_graph(other_name(some_op))
- classdagster.OutputMapping [source]
- Defines an output mapping for a graph. - Parameters: - graph_output_name (str) – Name of the output in the graph being mapped to.
- mapped_node_name (str) – Named of the node (op/graph) that the output is being mapped from.
- mapped_node_output_name (str) – Name of the output in the node (op/graph) that is being mapped from.
- graph_output_description (Optional[str]) – A description of the output in the graph being mapped from.
- from_dynamic_mapping (bool) – Set to true if the node being mapped to is a mapped dynamic node.
- dagster_type (Optional[DagsterType]) – deprecated The dagster type of the graph’s output being mapped to.
 - Examples: - from dagster import OutputMapping, GraphDefinition, op, graph, GraphOut
 @op
 def emit_five(x):
 return 5
 # The following two graph definitions are equivalent
 GraphDefinition(
 name="the_graph",
 node_defs=[emit_five],
 output_mappings=[
 OutputMapping(
 graph_output_name="result", # Default output name
 mapped_node_name="emit_five",
 mapped_node_output_name="result"
 )
 ]
 )
 @graph(out=GraphOut())
 def the_graph():
 return emit_five()
- classdagster.InputMapping [source]
- Defines an input mapping for a graph. - Parameters: - graph_input_name (str) – Name of the input in the graph being mapped from.
- mapped_node_name (str) – Named of the node (op/graph) that the input is being mapped to.
- mapped_node_input_name (str) – Name of the input in the node (op/graph) that is being mapped to.
- fan_in_index (Optional[int]) – The index in to a fanned input, otherwise None.
- graph_input_description (Optional[str]) – A description of the input in the graph being mapped from.
- dagster_type (Optional[DagsterType]) – deprecated The dagster type of the graph’s input being mapped from.
 - Examples: - from dagster import InputMapping, GraphDefinition, op, graph
 @op
 def needs_input(x):
 return x + 1
 # The following two graph definitions are equivalent
 GraphDefinition(
 name="the_graph",
 node_defs=[needs_input],
 input_mappings=[
 InputMapping(
 graph_input_name="maps_x", mapped_node_name="needs_input",
 mapped_node_input_name="x"
 )
 ]
 )
 @graph
 def the_graph(maps_x):
 needs_input(maps_x)