Dagster Types

The Dagster type system helps users describe what kind of values their solids accept and produce. Each solid input and output can be annotated with a DagsterType.

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.

The Dagster type system is independent from the PEP 484 Python type system, although we overload the type annotation syntax on functions to make it easier to specify the input and output types of your solids.

Dagster type-checking 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.

DagsterType

The core API for defining Dagster types is DagsterType.

test_type_guide.py
    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.

types_on_definitions.py
    @solid(
        input_defs=[InputDefinition("num", EvenDagsterType)],
        output_defs=[OutputDefinition(EvenDagsterType)],
    )
    def double_even(_, num):
        return num

Using Python 3 and type annotations, the same declaration can be expressed as:

type_annotations.py
    @solid
    def double_even(_, num: EvenDagsterType) -> EvenDagsterType:
        # These type annotations are a shorthand for constructing InputDefinitions
        # and OutputDefinitions, and are not mypy compliant
        return num  # at runtime this is a python int

When raw DagsterTypes are passed, this is not compliant with mypy or other python static type-checking system (we will refer to just mypy throughout the remainder of this document for convenience). We detail how to do that later in this document.

Scalars

Dagster exports top level scalars -- Int, Bool, String, Float -- that can be used as DagsterType instances. For convenience, one can use the corresponding python built-in scalars -- int, bool, str, float -- to specify the corresponding Dagster scalar types. This also makes these arguments mypy-compliant. There is a 1:1 relationship between the python type -- e.g. int -- and its corresponding dagster type -- e.g. dagster.Int and a mapping between them is maintained by the system. This means that those python types can be used in InputDefinitions or OutputDefinitions. If passed as type annotation to solids, the corresponding DagsterType will also be picked up by the system.

As we will see later, the user can also define their own 1:1 relationships between python and Dagster types.

PythonObjectDagsterType

It is common to define types that just do isinstance checks as the Dagster type check. PythonObjectDagsterType does just that.

python_object_dagster_type.py
    class EvenType:
        def __init__(self, num):
            assert num % 2 is 0
            self.num = num

    EvenDagsterType = PythonObjectDagsterType(EvenType, name="EvenDagsterType")

One can now declare the Dagster type as the interface to the solid, and use the corresponding python type in business logic.

dagster_type_annotations.py
    @solid
    def double_even(_, even_num: EvenDagsterType) -> EvenDagsterType:
        # These type annotations are a shorthand for constructing InputDefinitions
        # and OutputDefinitions, and are not mypy compliant
        return EvenType(even_num.num * 2)

The type system truly shines once the type check expresses richer behavior, such as column-level schema on a dataframe. The dagster-pandas library offers that capability through create_dagster_pandas_dataframe_type.

Loading from Config

When executing a pipeline or solid whose data are not produced by a dependency - e.g. for a solid at the beginning of a pipeline or execution subset - the data needs to come from somewhere. This can be specified by config.

The type must declare its interaction with the config system. This means declaring the input schema and a function which the validated config value and produces a valid in-memory value to be passed to the schema. This process is referred to as loading.

    class EvenType:
        def __init__(self, num):
            assert num % 2 is 0
            self.num = num

    @dagster_type_loader(int)
    def load_even_type(_, cfg):
        return EvenType(cfg)

    EvenDagsterType = PythonObjectDagsterType(EvenType, loader=load_even_type)

With this the input can be specified via config as below:

    yaml_doc = """
    solids:
        double_even:
            inputs:
                even_num: 2
    """

Output values can also be materialized to a user-controlled location using config in a similar fashion.

Dagster Type Use Cases

Inputs and Outputs can represent, via the type system:

  1. In-Memory-Data: In-memory data that is either produced a previous computation or specified via configuration. Dagster infrastructure -- the intermediates store -- handles the serialization the transfer between of these data in cases whereas multiple processes or nodes are being used to run the orchestration cluster.
  2. Metadata: Metadata about data that is managed by the user but not the intermediates store. The type system would ensure that the in-memory metadata format is correct and that any preconditions about the in situ data are met. An example of this would be a data lake with a user-defined URL scheme.
  3. Preconditions: Preconditions about external state necessary for computation to succeed. An example of this is a hard-coded database table name.
  4. Execution Ordering: In some pipelines there is no data to pass in between solids and there are no preconditions. This is a common in cases where one is migrating workflows from other systems -- that do not have this concept -- to dagster, or where the upstream solids are purely operational whose effects are difficult or awkward to model. For these cases we offer the Nothing type.

Python Types and Dagster Types

As alluded to earlier, the user can create define their own 1:1 relationships between Dagster types and Python types, making the corresponding python type usable where the Dagster type is expected.

This is convenient when one has business objects that require nothing more than an isinstance check in the Dagster type check and one wishes to use them both in type annotations directly and as business objects. These type definitions reduce boilerplate as well as deliver out-of-the-box mypy compliance.

There are two APIs: usable_as_dagster_type() -- for direct annotations of class declarations -- and 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.

usable_as_dagster_type.py
    @usable_as_dagster_type
    class EvenType:
        def __init__(self, num):
            assert num % 2 is 0
            self.num = num

And make_python_type_usable_as_dagster_type

hello_cereal.py
    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)

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

MyPy Compliance

Users who do not use "usable-as-dagster" python types types by choice or necessity and who desire mypy compliance need additional support.

This is a challenge to do elegantly in light of the fact that, as of mypy 0.761, there is no way to directly annotate a object to treat it as a type during a type checking pass. However there is a way to do this.

type_checking_mypy_compliance.py
    class EvenType:
        def __init__(self, num):
            assert num % 2 is 0
            self.num = num

    if typing.TYPE_CHECKING:
        EvenDagsterType = EvenType
    else:
        EvenDagsterType = PythonObjectDagsterType(EvenType)

    @solid
    def double_even(_, even_num: EvenDagsterType) -> EvenDagsterType:
        return EvenType(even_num.num * 2)

While not particularly elegant, this does work. We recommend centralizing type definitions so that this style of definition is compartmentalized to a single file.