Ask AI

Ops

The foundational unit of computation in Dagster.


Defining ops

@dagster.op(compute_fn=None, *, name=None, description=None, ins=None, out=None, config_schema=None, required_resource_keys=None, tags=None, version=None, retry_policy=None, code_version=None)[source]

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 Output and yielded by the compute function.

  2. Return an Output. This output will be yielded by the compute function.

  3. Yield Output or other event 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 Output 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 Executor implementation that supports running them together.

Parameters:
  • name (Optional[str]) – Name of op. Must be unique within any GraphDefinition using the op.

  • description (Optional[str]) – Human-readable description of this op. If not provided, and the decorated function has docstring, that docstring will be used as the description.

  • ins (Optional[Dict[str, In]]) – Information about the inputs to the op. Information provided here will be combined with what can be inferred from the function signature.

  • out (Optional[Union[Out, Dict[str, Out]]]) – Information about the op outputs. Information provided here will be combined with what can be inferred from the return type signature if the function does not use yield.

  • config_schema (Optional[ConfigSchema) – The schema for the config. If set, Dagster will check that config provided for the op matches this schema and fail if it does not. If not set, Dagster will accept any config provided for the op.

  • 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 expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.

  • code_version (Optional[str]) – (Experimental) Version of the logic encapsulated by the op. If set, this is used as a default version for all outputs.

  • retry_policy (Optional[RetryPolicy]) – 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(compute_fn, name, ins=None, outs=None, description=None, config_schema=None, required_resource_keys=None, tags=None, version=None, retry_policy=None, code_version=None)[source]

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

For more details on what a op is, refer to the Ops Overview .

End users should prefer the @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 GraphDefinition or JobDefinition that contains the op.

  • 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.

    This function will be coerced into a generator or an async generator, which must yield one Output for each of the op’s output_defs, and additionally may yield other types of Dagster events, including AssetMaterialization and ExpectationResult.

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

  • config_schema (Optional[ConfigSchema) – The schema for the config. If set, Dagster will check that the config provided for the op matches this schema and will fail if it does not. If not set, Dagster will accept any config provided for the op.

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

  • tags (Optional[Dict[str, Any]]) – Arbitrary metadata for the op. Frameworks may expect and require certain metadata to be attached to a op. Users should generally not set metadata directly. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.

  • 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, this is used as a default code version for all outputs.

  • retry_policy (Optional[RetryPolicy]) – 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(name)[source]

Creates a copy of this op with the given name.

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]

tag(tags)[source]

Creates a copy of this op with the given tags.

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

with_hooks(hook_defs)[source]

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

with_retry_policy(retry_policy)[source]

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


Ins & outs

class dagster.In(dagster_type=<class 'dagster._core.definitions.utils.NoValueSentinel'>, description=None, default_value=<class 'dagster._core.definitions.utils.NoValueSentinel'>, metadata=None, asset_key=None, asset_partitions=None, input_manager_key=None)[source]

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, DagsterType]]]) – The type of this input. Should only be set if the correct type can not be inferred directly from the type signature of the decorated function.

  • 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[AssetKey, InputContext -> AssetKey]]) – (Experimental) An AssetKey (or function that produces an AssetKey from the InputContext) which should be associated with this In. Used for tracking lineage information through Dagster.

  • asset_partitions (Optional[Union[Set[str], InputContext -> Set[str]]]) – (Experimental) A set of partitions of the given asset_key (or a function that produces this list of partitions from the InputContext) which should be associated with this In.

  • input_manager_key (Optional[str]) – (Experimental) The resource key for the InputManager used for loading this input when it is not connected to an upstream output.

class dagster.Out(dagster_type=<class 'dagster._core.definitions.utils.NoValueSentinel'>, description=None, is_required=True, io_manager_key=None, metadata=None, code_version=None)[source]

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, DagsterType]]]) – The type of this output. Should only be set if the correct type can not be inferred directly from the type signature of the decorated function.

  • 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. (default: “io_manager”).

  • metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the output. For example, users can provide a file path if the data object will be stored in a filesystem, or provide information of a database table when it is going to load the data into the table.

  • code_version (Optional[str]) – (Experimental) Version of the code that generates this output. In general, versions should be set only for code that deterministically produces the same output when given the same inputs.


Execution

class dagster.RetryPolicy(max_retries=1, delay=None, backoff=None, jitter=None)[source]

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 being started. This unit of time can be modulated as a function of attempt number with backoff and randomly with jitter.

  • backoff (Optional[Backoff]) – A modifier for delay as a function of retry attempt number.

  • jitter (Optional[Jitter]) – A randomizing modifier for delay, applied after backoff calculation.

class dagster.Backoff(value)[source]

A modifier for delay as a function of attempt number.

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

class dagster.Jitter(value)[source]

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 Failure and RetryRequested are intended to be raised from ops rather than yielded.)

Event types

class dagster.Output(value, output_name='result', metadata=None, data_version=None)[source]

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 OpDefinition 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 Out or the downstream In is typed.

Parameters:
  • value (Any) – The value returned by the compute function.

  • output_name (Optional[str]) – Name of the corresponding out. (default: “result”)

  • metadata (Optional[Dict[str, Union[str, float, int, MetadataValue]]]) – Arbitrary metadata about the output. Keys are displayed string labels, and values are one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

  • data_version (Optional[DataVersion]) – experimental (This parameter may break in future versions, even between dot releases.) (Experimental) A data version to manually set for 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 Out.

Type:

str

property value

The value returned by the compute function.

Type:

Any

class dagster.AssetMaterialization(asset_key, description=None, metadata=None, partition=None, tags=None)[source]

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], AssetKey]) – A key to identify the materialized asset across job runs

  • description (Optional[str]) – A longer human-readable description of the materialized value.

  • partition (Optional[str]) – The name of the partition that was materialized.

  • tags (Optional[Mapping[str, str]]) – A mapping containing system-populated tags for the materialization. Users should not pass values into this argument.

  • metadata (Optional[Dict[str, RawMetadataValue]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

static file(path, description=None, asset_key=None)[source]

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(success, label=None, description=None, metadata=None)[source]

deprecated This API will be removed in version 1.7.

Please 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 one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

class dagster.TypeCheck(success, description=None, metadata=None)[source]

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() 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 one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

class dagster.Failure(description=None, metadata=None, allow_retries=None)[source]

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 one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

  • allow_retries (Optional[bool]) – Whether this Failure should respect the retry policy or bypass it and immediately fail. Defaults to True, respecting the retry policy and allowing retries.

class dagster.RetryRequested(max_retries=1, seconds_to_wait=None)[source]

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 to the up_for_retry state

Example

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

Event metadata

Dagster uses metadata to communicate arbitrary user-specified metadata about structured events.

class dagster.MetadataValue[source]

Utility class to wrap metadata values passed into Dagster events so that they can be displayed in the Dagster UI and other tooling.

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "my_text_label": "hello",
            "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
            "num_rows": 0,
        },
    )
static asset(asset_key)[source]

Static constructor for a metadata value referencing a Dagster asset, by key.

For example:

@op
def validate_table(context, df):
    yield AssetMaterialization(
        asset_key=AssetKey("my_table"),
        metadata={
            "Related asset": MetadataValue.asset(AssetKey('my_other_table')),
        },
    )
Parameters:

asset_key (AssetKey) – The asset key referencing the asset.

static bool(value)[source]

Static constructor for a metadata value wrapping a bool as BoolMetadataValuye. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "num rows > 1000": MetadataValue.bool(len(df) > 1000),
        },
    )
Parameters:

value (bool) – The bool value for a metadata entry.

static column_lineage(lineage)[source]

Static constructor for a metadata value wrapping a column lineage as TableColumnLineageMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Parameters:

lineage (TableColumnLineage) – The column lineage for a metadata entry.

static dagster_run(run_id)[source]

Static constructor for a metadata value wrapping a reference to a Dagster run.

Parameters:

run_id (str) – The ID of the run.

static float(value)[source]

Static constructor for a metadata value wrapping a float as FloatMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "size (bytes)": MetadataValue.float(calculate_bytes(df)),
        }
    )
Parameters:

value (float) – The float value for a metadata entry.

static int(value)[source]

Static constructor for a metadata value wrapping an int as IntMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "number of rows": MetadataValue.int(len(df)),
        },
    )
Parameters:

value (int) – The int value for a metadata entry.

static job(job_name, location_name, *, repository_name=None)[source]

Static constructor for a metadata value referencing a Dagster job, by name.

For example:

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset"
        metadata={
            "Producing job": MetadataValue.job('my_other_job'),
        },
    )
Parameters:
  • job_name (str) – The name of the job.

  • location_name (Optional[str]) – The code location name for the job.

  • repository_name (Optional[str]) – The repository name of the job, if different from the default.

static json(data)[source]

Static constructor for a metadata value wrapping a json-serializable list or dict as JsonMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context):
    yield ExpectationResult(
        success=not missing_things,
        label="is_present",
        metadata={
            "about my dataset": MetadataValue.json({"missing_columns": missing_things})
        },
    )
Parameters:

data (Union[Sequence[Any], Mapping[str, Any]]) – The JSON data for a metadata entry.

static md(data)[source]

Static constructor for a metadata value wrapping markdown data as MarkdownMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, md_str):
    yield AssetMaterialization(
        asset_key="info",
        metadata={
            'Details': MetadataValue.md(md_str)
        },
    )
Parameters:

md_str (str) – The markdown for a metadata entry.

static notebook(path)[source]

Static constructor for a metadata value wrapping a notebook path as NotebookMetadataValue.

Example

@op
def emit_metadata(context):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "notebook_path": MetadataValue.notebook("path/to/notebook.ipynb"),
        }
    )
Parameters:

path (str) – The path to a notebook for a metadata entry.

static null()[source]

Static constructor for a metadata value representing null. Can be used as the value type for the metadata parameter for supported events.

static path(path)[source]

Static constructor for a metadata value wrapping a path as PathMetadataValue.

Example

@op
def emit_metadata(context):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "filepath": MetadataValue.path("path/to/file"),
        }
    )
Parameters:

path (str) – The path for a metadata entry.

static python_artifact(python_artifact)[source]

Static constructor for a metadata value wrapping a python artifact as PythonArtifactMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "class": MetadataValue.python_artifact(MyClass),
            "function": MetadataValue.python_artifact(my_function),
        }
    )
Parameters:

value (Callable) – The python class or function for a metadata entry.

static table(records, schema=None)[source]

experimental This API may break in future versions, even between dot releases.

Static constructor for a metadata value wrapping arbitrary tabular data as TableMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context):
    yield ExpectationResult(
        success=not has_errors,
        label="is_valid",
        metadata={
            "errors": MetadataValue.table(
                records=[
                    TableRecord(code="invalid-data-type", row=2, col="name"),
                ],
                schema=TableSchema(
                    columns=[
                        TableColumn(name="code", type="string"),
                        TableColumn(name="row", type="int"),
                        TableColumn(name="col", type="string"),
                    ]
                )
            ),
        },
    )
static table_schema(schema)[source]

Static constructor for a metadata value wrapping a table schema as TableSchemaMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

schema = TableSchema(
    columns = [
        TableColumn(name="id", type="int"),
        TableColumn(name="status", type="bool"),
    ]
)

DagsterType(
    type_check_fn=some_validation_fn,
    name='MyTable',
    metadata={
        'my_table_schema': MetadataValue.table_schema(schema),
    }
)
Parameters:

schema (TableSchema) – The table schema for a metadata entry.

static text(text)[source]

Static constructor for a metadata value wrapping text as TextMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "my_text_label": MetadataValue.text("hello")
        },
    )
Parameters:

text (str) – The text string for a metadata entry.

static timestamp(value)[source]

Static constructor for a metadata value wrapping a UNIX timestamp as a TimestampMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Parameters:

value (Union[float, datetime]) – The unix timestamp value for a metadata entry. If a datetime is provided, the timestamp will be extracted. datetimes without timezones are not accepted, because their timestamps can be ambiguous.

static url(url)[source]

Static constructor for a metadata value wrapping a URL as UrlMetadataValue. Can be used as the value type for the metadata parameter for supported events.

Example

@op
def emit_metadata(context):
    yield AssetMaterialization(
        asset_key="my_dashboard",
        metadata={
            "dashboard_url": MetadataValue.url("http://mycoolsite.com/my_dashboard"),
        }
    )
Parameters:

url (str) – The URL for a metadata entry.

abstract property value

The wrapped value.

class dagster.MetadataEntry(label, description=None, entry_data=None, value=None)[source]

deprecated This API will be removed in version 2.0.

Please use a dict with MetadataValue values instead..

A structure for describing metadata for Dagster events.

Note

This class is no longer usable in any Dagster API, and will be completely removed in 2.0.

Lists of objects of this type can be passed as arguments to Dagster events and will be displayed in the Dagster UI and other tooling.

Should be yielded from within an IO manager to append metadata for a given input/output event. For other event types, passing a dict with MetadataValue values to the metadata argument is preferred.

Parameters:
  • label (str) – Short display label for this metadata entry.

  • description (Optional[str]) – A human-readable description of this metadata entry.

  • value (MetadataValue) – Typed metadata entry data. The different types allow for customized display in tools like the Dagster UI.

Metadata types

All metadata types inherit from MetadataValue. The following types are defined:

class dagster.DagsterAssetMetadataValue(asset_key)[source]

Representation of a dagster asset.

Parameters:

asset_key (AssetKey) – The dagster asset key

property value

The wrapped AssetKey.

Type:

AssetKey

class dagster.DagsterRunMetadataValue(run_id)[source]

Representation of a dagster run.

Parameters:

run_id (str) – The run id

property value

The wrapped run id.

Type:

str

class dagster.FloatMetadataValue(value)[source]

Container class for float metadata entry data.

Parameters:

value (Optional[float]) – The float value.

class dagster.IntMetadataValue(value)[source]

Container class for int metadata entry data.

Parameters:

value (Optional[int]) – The int value.

class dagster.JsonMetadataValue(data)[source]

Container class for JSON metadata entry data.

Parameters:

data (Union[Sequence[Any], Dict[str, Any]]) – The JSON data.

property value

The wrapped JSON data.

Type:

Optional[Union[Sequence[Any], Dict[str, Any]]]

class dagster.MarkdownMetadataValue(md_str)[source]

Container class for markdown metadata entry data.

Parameters:

md_str (Optional[str]) – The markdown as a string.

property value

The wrapped markdown as a string.

Type:

Optional[str]

class dagster.PathMetadataValue(path)[source]

Container class for path metadata entry data.

Parameters:

path (Optional[str]) – The path as a string or conforming to os.PathLike.

property value

The wrapped path.

Type:

Optional[str]

class dagster.NotebookMetadataValue(path)[source]

Container class for notebook metadata entry data.

Parameters:

path (Optional[str]) – The path to the notebook as a string or conforming to os.PathLike.

property value

The wrapped path to the notebook as a string.

Type:

Optional[str]

class dagster.PythonArtifactMetadataValue(module, name)[source]

Container class for python artifact metadata entry data.

Parameters:
  • module (str) – The module where the python artifact can be found

  • name (str) – The name of the python artifact

property value

Identity function.

Type:

PythonArtifactMetadataValue

class dagster.TableColumnLineageMetadataValue(column_lineage)[source]

Representation of the lineage of column inputs to column outputs of arbitrary tabular data.

Parameters:

column_lineage (TableColumnLineage) – The lineage of column inputs to column outputs for the table.

property value

The wrapped TableSpec.

Type:

TableSpec

class dagster.TableMetadataValue(records, schema)[source]

experimental This API may break in future versions, even between dot releases.

Container class for table metadata entry data.

Parameters:
  • records (TableRecord) – The data as a list of records (i.e. rows).

  • schema (Optional[TableSchema]) – A schema for the table.

Example

from dagster import TableMetadataValue, TableRecord

TableMetadataValue(
    schema=None,
    records=[
        TableRecord({"column1": 5, "column2": "x"}),
        TableRecord({"column1": 7, "column2": "y"}),
    ]
)
static infer_column_type(value)[source]

str: Infer the TableSchema column type that will be used for a value.

property value

Identity function.

Type:

TableMetadataValue

class dagster.TableSchemaMetadataValue(schema)[source]

Representation of a schema for arbitrary tabular data.

Parameters:

schema (TableSchema) – The dictionary containing the schema representation.

property value

The wrapped TableSchema.

Type:

TableSchema

class dagster.TextMetadataValue(text)[source]

Container class for text metadata entry data.

Parameters:

text (Optional[str]) – The text data.

property value

The wrapped text data.

Type:

Optional[str]

class dagster.TimestampMetadataValue(value)[source]

Container class for metadata value that’s a unix timestamp.

Parameters:

value (float) – Seconds since the unix epoch.

class dagster.UrlMetadataValue(url)[source]

Container class for URL metadata entry data.

Parameters:

url (Optional[str]) – The URL as a string.

property value

The wrapped URL.

Type:

Optional[str]

Tables

These APIs provide the ability to express table schemas (TableSchema) and table rows/records (TableRecord) in Dagster. Currently the only use case for TableSchemas and TableRecords is to wrap them in their corresponding metadata classes TableMetadataValue and TableSchemaMetadataValue for attachment to events or Dagster types.

class dagster.TableRecord(data)[source]

experimental This API may break in future versions, even between dot releases.

Represents one record in a table. Field keys are arbitrary strings– field values must be strings, integers, floats, or bools.

class dagster.TableSchema(columns, constraints=None)[source]

Representation of a schema for tabular data.

Schema is composed of two parts:

  • A required list of columns (TableColumn). Each column specifies a name, type, set of constraints, and (optional) description. type defaults to string if unspecified. Column constraints (TableColumnConstraints) consist of boolean properties unique and nullable, as well as a list of strings other containing string descriptions of all additional constraints (e.g. “<= 5”).

  • An optional list of table-level constraints (TableConstraints). A table-level constraint cannot be expressed in terms of a single column, e.g. col a > col b. Presently, all table-level constraints must be expressed as strings under the other attribute of a TableConstraints object.

# example schema
TableSchema(
    constraints = TableConstraints(
        other = [
            "foo > bar",
        ],
    ),
    columns = [
        TableColumn(
            name = "foo",
            type = "string",
            description = "Foo description",
            constraints = TableColumnConstraints(
                required = True,
                other = [
                    "starts with the letter 'a'",
                ],
            ),
        ),
        TableColumn(
            name = "bar",
            type = "string",
        ),
        TableColumn(
            name = "baz",
            type = "custom_type",
            constraints = TableColumnConstraints(
                unique = True,
            )
        ),
    ],
)
Parameters:
static from_name_type_dict(name_type_dict)[source]

Constructs a TableSchema from a dictionary whose keys are column names and values are the names of data types of those columns.

class dagster.TableConstraints(other)[source]

Descriptor for “table-level” constraints. Presently only one property, other is supported. This contains strings describing arbitrary table-level constraints. A table-level constraint is a constraint defined in terms of multiple columns (e.g. col_A > col_B) or in terms of rows.

Parameters:

other (List[str]) – Descriptions of arbitrary table-level constraints.

class dagster.TableColumn(name, type='string', description=None, constraints=None)[source]

Descriptor for a table column. The only property that must be specified by the user is name. If no type is specified, string is assumed. If no constraints are specified, the column is assumed to be nullable (i.e. required = False) and have no other constraints beyond the data type.

Parameters:
  • name (List[str]) – Descriptions of arbitrary table-level constraints.

  • type (Optional[str]) – The type of the column. Can be an arbitrary string. Defaults to “string”.

  • description (Optional[str]) – Description of this column. Defaults to None.

  • constraints (Optional[TableColumnConstraints]) – Column-level constraints. If unspecified, column is nullable with no constraints.

class dagster.TableColumnConstraints(nullable=True, unique=False, other=None)[source]

Descriptor for a table column’s constraints. Nullability and uniqueness are specified with boolean properties. All other constraints are described using arbitrary strings under the other property.

Parameters:
  • nullable (Optional[bool]) – If true, this column can hold null values.

  • unique (Optional[bool]) – If true, all values in this column must be unique.

  • other (List[str]) – Descriptions of arbitrary column-level constraints not expressible by the predefined properties.


Asset key

Dagster uses AssetKey to build an index on Materialization events. Assets materialized with an AssetKey are highlighted in the Dagster UI on the Assets dashboard.

class dagster.AssetKey(path)[source]

Object representing the structure of an asset key. Takes in a sanitized string, list of strings, or tuple of strings.

Example usage:

from dagster import op

@op
def emit_metadata(context, df):
    yield AssetMaterialization(
        asset_key=AssetKey('flat_asset_key'),
        metadata={"text_metadata": "Text-based metadata for this event"},
    )

@op
def structured_asset_key(context, df):
    yield AssetMaterialization(
        asset_key=AssetKey(['parent', 'child', 'grandchild']),
        metadata={"text_metadata": "Text-based metadata for this event"},
    )

@op
def structured_asset_key_2(context, df):
    yield AssetMaterialization(
        asset_key=AssetKey(('parent', 'child', 'grandchild')),
        metadata={"text_metadata": "Text-based metadata for this event"},
    )
Parameters:

path (Sequence[str]) – String, list of strings, or tuple of strings. A list of strings represent the hierarchical structure of the asset_key.