Skip to main content

Ops

The foundational unit of computation in Dagster.


Defining ops

@dagster.op

Create an op with the specified parameters from the decorated function.

Ins and outs will be inferred from the type signature of the decorated function if not explicitly provided.

The decorated function will be used as the op’s compute function. The signature of the decorated function is more flexible than that of the compute_fn in the core API; it may:

  1. Return a value. This value will be wrapped in an OutputOutput and yielded by the compute function.
  2. Return an OutputOutput. This output will be yielded by the compute function.
  3. Yield OutputOutput or other event objectsevent objects. Same as default compute behavior. Note that options 1) and 2) are incompatible with yielding other events – if you would like to decorate a function that yields events, it must also wrap its eventual output in an OutputOutput and yield it.

@op supports async def functions as well, including async generators when yielding multiple events or outputs. Note that async ops will generally be run on their own unless using a custom ExecutorExecutor implementation that supports running them together.

Parameters:

  • name (Optional[str]) – Name of op. Must be unique within any GraphDefinitionGraphDefinition
  • description (Optional[str]) – Human-readable description of this op. If not provided, and
  • ins (Optional[Dict[str, InIn]]) – Information about the inputs to the op. Information provided here will be combined
  • out (Optional[Union[OutOut, Dict[str, OutOut]]]) – Information about the op outputs. Information provided here will be combined with
  • config_schema (Optional[ConfigSchemaConfigSchema) – The schema for the config. If set, Dagster will check
  • required_resource_keys (Optional[Set[str]]) – Set of resource handles required by this op.
  • tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may
  • code_version (Optional[str]) – (Experimental) Version of the logic encapsulated by the op. If set,
  • retry_policy (Optional[RetryPolicyRetryPolicy]) – The retry policy for this op.

Examples:

@op
def hello_world():
print('hello')

@op
def echo(msg: str) -> str:
return msg

@op(
ins=\{'msg': In(str)},
out=Out(str)
)
def echo_2(msg): # same as above
return msg

@op(
out=\{'word': Out(), 'num': Out()}
)
def multi_out() -> Tuple[str, int]:
return 'cool', 4
class dagster.OpDefinition

Defines an op, the functional unit of user-defined computation.

End users should prefer the @op@op decorator. OpDefinition is generally intended to be used by framework authors or for programatically generated ops.

Parameters:

  • name (str) – Name of the op. Must be unique within any GraphDefinitionGraphDefinition or

  • input_defs (List[InputDefinition]) – Inputs of the op.

  • compute_fn (Callable) –

    The core of the op, the function that performs the actual computation. The signature of this function is determined by input_defs, and optionally, an injected first argument, context, a collection of information provided by the system.

  • output_defs (List[OutputDefinition]) – Outputs of the op.

  • config_schema (Optional[ConfigSchemaConfigSchema) – The schema for the config. If set, Dagster will check

  • description (Optional[str]) – Human-readable description of the op.

  • tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may

  • required_resource_keys (Optional[Set[str]]) – Set of resources handles required by this op.

  • code_version (Optional[str]) – (Experimental) Version of the code encapsulated by the op. If set,

  • retry_policy (Optional[RetryPolicyRetryPolicy]) – The retry policy for this op.

Examples:

def _add_one(_context, inputs):
yield Output(inputs["num"] + 1)

OpDefinition(
name="add_one",
ins=\{"num": In(int)},
outs=\{"result": Out(int)},
compute_fn=_add_one,
)
alias

Creates a copy of this op with the given name.

tag

Creates a copy of this op with the given tags.

with_hooks

Creates a copy of this op with the given hook definitions.

with_retry_policy

Creates a copy of this op with the given retry policy.

property config_schema

The config schema for this op.

Type: IDefinitionConfigSchema

property ins

A mapping from input name to the In object that represents that input.

Type: Mapping[str, In]

property name

The name of this op.

Type: str

property outs

A mapping from output name to the Out object that represents that output.

Type: Mapping[str, Out]

property required_resource_keys

A set of keys for resources that must be provided to this OpDefinition.

Type: AbstractSet[str]

property retry_policy

The RetryPolicy for this op.

Type: Optional[RetryPolicy]

property tags

The tags for this op.

Type: Mapping[str, str]

property version
deprecated

This API will be removed in version 2.0. Use code_version instead..

Version of the code encapsulated by the op. If set, this is used as a default code version for all outputs.

Type: str


Ins & outs

class dagster.In

Defines an argument to an op’s compute function.

Inputs may flow from previous op’s outputs, or be stubbed using config. They may optionally be typed using the Dagster type system.

Parameters:

  • dagster_type (Optional[Union[Type, DagsterTypeDagsterType]]]) – The type of this input. Should only be set if the correct type can not
  • description (Optional[str]) – Human-readable description of the input.
  • default_value (Optional[Any]) – The default value to use if no input is provided.
  • metadata (Optional[Dict[str, RawMetadataValue]]) – A dict of metadata for the input.
  • asset_key (Optional[Union[AssetKeyAssetKey, InputContext -> AssetKey]]) – (Experimental) An AssetKey
  • asset_partitions (Optional[Union[Set[str], InputContext -> Set[str]]]) – (Experimental) A
  • input_manager_key (Optional[str]) – (Experimental) The resource key for the
class dagster.Out

Defines an output from an op’s compute function.

Ops can have multiple outputs, in which case outputs cannot be anonymous.

Many ops have only one output, in which case the user can provide a single output definition that will be given the default name, “result”.

Outs may be typed using the Dagster type system.

Parameters:

  • dagster_type (Optional[Union[Type, DagsterTypeDagsterType]]]) – The type of this output. Should only be set if the correct type can not
  • description (Optional[str]) – Human-readable description of the output.
  • is_required (bool) – Whether the presence of this field is required. (default: True)
  • io_manager_key (Optional[str]) – The resource key of the output manager used for this output.
  • metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the output.
  • code_version (Optional[str]) – (Experimental) Version of the code that generates this output. In

Execution

class dagster.RetryPolicy

A declarative policy for when to request retries when an exception occurs during op execution.

Parameters:

  • max_retries (int) – The maximum number of retries to attempt. Defaults to 1.
  • delay (Optional[Union[int,float]]) – The time in seconds to wait between the retry being requested and the next attempt
  • backoff (Optional[BackoffBackoff]) – A modifier for delay as a function of retry attempt number.
  • jitter (Optional[JitterJitter]) – A randomizing modifier for delay, applied after backoff calculation.
class dagster.Backoff

A modifier for delay as a function of attempt number.

LINEAR: attempt_num * delay EXPONENTIAL: ((2 ^ attempt_num) - 1) * delay

class dagster.Jitter

A randomizing modifier for delay, applied after backoff calculation.

FULL: between 0 and the calculated delay based on backoff: random() * backoff_delay PLUS_MINUS: +/- the delay: backoff_delay + ((2 * (random() * delay)) - delay)


Events

The objects that can be yielded by the body of ops’ compute functions to communicate with the Dagster framework.

(Note that FailureFailure and RetryRequestedRetryRequested are intended to be raised from ops rather than yielded.)

Event types

class dagster.Output

Event corresponding to one of an op’s outputs.

Op compute functions must explicitly yield events of this type when they have more than one output, or when they also yield events of other types, or when defining a op using the OpDefinitionOpDefinition API directly.

Outputs are values produced by ops that will be consumed by downstream ops in a job. They are type-checked at op boundaries when their corresponding OutOut or the downstream InIn is typed.

Parameters:

  • value (Any) – The value returned by the compute function.
  • output_name (str) – Name of the corresponding Out. (default: “result”)
  • metadata (Optional[Dict[str, Union[str, float, int, MetadataValueMetadataValue]]]) – Arbitrary metadata about the output. Keys are displayed string labels, and values are
  • data_version (Optional[DataVersion]) – experimental
  • tags (Optional[Mapping[str, str]]) – (Experimental) Tags that will be attached to the asset
property data_version

A data version that was manually set on the Output.

Type: Optional[DataVersion]

property output_name

Name of the corresponding OutOut.

Type: str

property value

The value returned by the compute function.

Type: Any

class dagster.AssetMaterialization

Event indicating that an op has materialized an asset.

Op compute functions may yield events of this type whenever they wish to indicate to the Dagster framework (and the end user) that they have produced a materialized value as a side effect of computation. Unlike outputs, asset materializations can not be passed to other ops, and their persistence is controlled by op logic, rather than by the Dagster framework.

Op authors should use these events to organize metadata about the side effects of their computations, enabling tooling like the Assets dashboard in the Dagster UI.

Parameters:

  • asset_key (Union[str, List[str], AssetKeyAssetKey]) – A key to identify the materialized asset across
  • description (Optional[str]) – A longer human-readable description of the materialized value.
  • partition (Optional[str]) – The name of the partition
  • tags (Optional[Mapping[str, str]]) – A mapping containing tags for the materialization.
  • metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are
static file

Static constructor for standard materializations corresponding to files on disk.

Parameters:

  • path (str) – The path to the file.
  • description (Optional[str]) – A human-readable description of the materialization.
class dagster.ExpectationResult
deprecated

This API will be removed in version 2.0. If using assets, use AssetCheckResult and @asset_check instead..

Event corresponding to a data quality test.

Op compute functions may yield events of this type whenever they wish to indicate to the Dagster framework (and the end user) that a data quality test has produced a (positive or negative) result.

Parameters:

  • success (bool) – Whether the expectation passed or not.
  • label (Optional[str]) – Short display name for expectation. Defaults to “result”.
  • description (Optional[str]) – A longer human-readable description of the expectation.
  • metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
class dagster.TypeCheck

Event corresponding to a successful typecheck.

Events of this type should be returned by user-defined type checks when they need to encapsulate additional metadata about a type check’s success or failure. (i.e., when using as_dagster_type(), @usable_as_dagster_type, or the underlying PythonObjectDagsterType()PythonObjectDagsterType() API.)

Op compute functions should generally avoid yielding events of this type to avoid confusion.

Parameters:

  • success (bool) – True if the type check succeeded, False otherwise.
  • description (Optional[str]) – A human-readable description of the type check.
  • metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
class dagster.Failure

Event indicating op failure.

Raise events of this type from within op compute functions or custom type checks in order to indicate an unrecoverable failure in user code to the Dagster machinery and return structured metadata about the failure.

Parameters:

  • description (Optional[str]) – A human-readable description of the failure.
  • metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the failure. Keys are displayed string labels, and values are
  • allow_retries (Optional[bool]) – Whether this Failure should respect the retry policy or bypass it and immediately fail.
class dagster.RetryRequested

An exception to raise from an op to indicate that it should be retried.

Parameters:

  • max_retries (Optional[int]) – The max number of retries this step should attempt before failing
  • seconds_to_wait (Optional[Union[float,int]]) – Seconds to wait before restarting the step after putting the step in

Example:

@op
def flakes():
try:
flakey_operation()
except Exception as e:
raise RetryRequested(max_retries=3) from e