The Dagster type system helps you describe what kind of values your solids accept and produce.
Name | Description |
---|---|
DagsterType | Base class for dagster types. |
PythonObjectDagsterType | A class for defining a Dagster Type whose typecheck is an isinstance check. |
check_dagster_type | The method to test a custom Dagster Type. |
@usable_as_dagster_type | A decorator to define how a Python class is used as a Dagster Type. |
make_python_type_usable_as_dagster_type | A method to map an existing Python type to a Dagster Type. |
Each solid input and output can be annotated with a Dagster Type.
The type system:
Is gradual and optional. Pipelines can run without types specified explicitly, and specifying types in some places doesn't require that types be specified everywhere. Inputs and outputs default to the Any
type.
Happens at solid execution time - each type defines a type_check_fn
that knows how to check whether values match what it expects. When a type is specified for a solid's input, then the type check occurs immediately before the solid is executed. When a type is specified for a solid's output, then the type check occurs immediately after the solid is executed.
Is complimentary to the PEP 484 Python type system. PEP 484 annotations enable static checks that verify variables and return values match particular Python types, while the Dagster type system enables runtime checks that include arbitrary validation logic.
The core API for defining Dagster types is DagsterType
.
EvenDagsterType = DagsterType(
name="EvenDagsterType",
type_check_fn=lambda _, value: isinstance(value, int) and value % 2 is 0,
)
Once created, types can be attached to solid InputDefinitions
and OutputDefinitions
.
@solid(
input_defs=[InputDefinition("num", EvenDagsterType)],
output_defs=[OutputDefinition(EvenDagsterType)],
)
def double_even(_, num):
return num
The type system truly shines once the type check expresses richer behavior, such as column-level schema on a dataframe. For example, check out the Validating Pandas DataFrames with Dagster Types guide.
If a Python input or output has a PEP 484 type annotation, and a DagsterType is not provided on the corresponding input or output definition, then Dagster will automatically generate a DagsterType that corresponds to the annotated Python type.
In this example, the defined solid will end up with a DagsterType named "MyClass" that:
class MyClass:
pass
def my_solid(_) -> MyClass:
return MyClass()
If the solid in the above example returned an object that was not an instance of MyClass, Dagster would raise an error after executing the solid.
Here is a list of Dagster's built-in Dagster types. You can find code examples of each type's usage in its API Reference:
Any
: Use this type for any input, output, or config field whose type is unconstrained.Bool
: Use this type for any boolean input, output, or config_field.Int
: Use this type for any integer input or output.Float
: Use this type for any float input, output, or config value.String
: Use this type for any string input, output, or config value.Optional
: Use this type only for inputs and outputs, if the value can also be None.List
: Use this type for inputs, or outputs.Dict
: Use this type for inputs, or outputs that are dicts.Set
: Use this type for inputs, or outputs that are sets.Tuple
: Use this type for inputs or outputs that are tuples.Nothing
: Use this type only for inputs and outputs, in order to establish an execution dependency without communicating a value.See details in the Nothing dependencies example.
You can use check_dagster_type
to test the type check function of a custom Dagster Type:
from dagster import check_dagster_type, Dict, Any
def test_dagster_type():
assert check_dagster_type(Dict[Any, Any], {"foo": "bar"}).success
Dagster types peacefully coexist with Python type annotations. In this example, the inputs and outputs of the solid compute function are integers, and the type check function for EvenDagsterType
will be invoked at runtime to verify that they are even.
@solid(
input_defs=[InputDefinition("num", EvenDagsterType)],
output_defs=[OutputDefinition(EvenDagsterType)],
)
def double_even_with_annotations(_, num: int) -> int:
return num
As mentioned earlier, any Python type used to annotate an argument or return value of solid-decorated function has a corresponding Dagster type that will be used for the corresponding input or output definition.
By default, Dagster will generate Dagster types for Python types that it's not aware of, but you can also explicitly control the Dagster type that will be used for a particular Python type.
There are two APIs:
@usable_as_dagster_type
for direct annotations of class declarations.make_python_type_usable_as_dagster_type
for mapping existing classes.This is designed for importing python types libraries that cannot be altered and mapping them to Dagster types.
from dagster import solid, usable_as_dagster_type
@usable_as_dagster_type
class EvenType:
def __init__(self, num):
assert num % 2 is 0
self.num = num
@solid
def double_even(_, even_num: EvenType) -> EvenType:
return EvenType(even_num.num * 2)
from dagster import PythonObjectDagsterType, make_python_type_usable_as_dagster_type, solid
class EvenType:
def __init__(self, num):
assert num % 2 is 0
self.num = num
EvenDagsterType = PythonObjectDagsterType(EvenType, name="EvenDagsterType")
make_python_type_usable_as_dagster_type(EvenType, EvenDagsterType)
@solid
def double_even(_, even_num: EvenType) -> EvenType:
return EvenType(even_num.num * 2)
However, this approach does have downsides. Importing these types will causes a global side effect as there is an internal registry which manages these types. This can introduce challenges in terms of testability and also can cause changes in behavior based on import order.
Additionally some of the most useful patterns in the ecosystem are to use type factories to programmatically create Dagster types, such as in our dagster-pandas module. In these cases a 1:1 mapping between dagster type and python type no longer exists. E.g. in dagster-pandas the python representation for all the dataframe variants is simply pandas.DataFrame.
For clearly scoped business objects, the reduction in boilerplate is significant and eases mypy compliance, hence its inclusion in the public Dagster API.
We do not recommend that libraries use this pattern and instead rely on other techniques to achieve mypy compliance.