Unconnected Inputs#

Relevant APIs#

NameDescription
@dagster_type_loaderThe decorator used to define a Dagster Type Loader.
DagsterTypeLoaderThe base class used to specify how to load inputs that depends on the type.
DagsterTypeLoaderContextThe context object provided to the function decorated by @dagster_type_loader.
@input_managerThe decorator used to define an input manager.
InputManagerThe base class used to specify how to load inputs.

Overview#

Ops in a job may have input definitions that don't correspond to the outputs of upstream ops. You can provide values for these inputs in a few different ways. Dagster checks each, in order, and uses the first that's available:

  • Input Manager - If the input to a job comes from an external source, such as a table in a database, often it makes sense to define a resource that's responsible for loading it. This makes it easy to swap out implementations in different jobs and mock it in tests. A special IOManager, which can be referenced from Ins, can be used to load unconnected inputs.
  • Dagster Type Loader - A DagsterTypeLoader provides a way to specify how to load inputs that depends on a type. A DagsterTypeLoader can be placed on DagsterType, which can be placed on In.
  • Default Values - In accepts a default_value argument.

Examples#

Loading a built-in dagster type from config#

When you have an op at the beginning of a job that operates on a built-in dagster type like string or int, you can provide a value for that input via run config.

Here's a basic job with an unconnected string input:

@op(ins={"input_string": In(String)})
def my_op(context, input_string):
    context.log.info(f"input string: {input_string}")


@job
def my_job():
    my_op()

The String dagster type has a dagster type loader that allows it to load inputs from config:

my_job.execute_in_process(
        run_config={"ops": {"my_op": {"inputs": {"input_string": {"value": "marmot"}}}}}
    )

Loading a custom dagster type from config#

When you have an op at the beginning of your job that operates on a dagster type that you've defined, you can write your own DagsterTypeLoader to define how to load that input via run config.

from typing import Dict, Union

from dagster import (
    DagsterTypeLoaderContext,
    In,
    dagster_type_loader,
    job,
    op,
    usable_as_dagster_type,
)


@dagster_type_loader(
    config_schema={"diameter": float, "juiciness": float, "cultivar": str}
)
def apple_loader(
    _context: DagsterTypeLoaderContext, config: Dict[str, Union[float, str]]
):
    return Apple(
        diameter=config["diameter"],
        juiciness=config["juiciness"],
        cultivar=config["cultivar"],
    )


@usable_as_dagster_type(loader=apple_loader)
class Apple:
    def __init__(self, diameter, juiciness, cultivar):
        self.diameter = diameter
        self.juiciness = juiciness
        self.cultivar = cultivar


@op(ins={"input_apple": In(Apple)})
def my_op(context, input_apple):
    context.log.info(f"input apple diameter: {input_apple.diameter}")


@job
def my_job():
    my_op()

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

my_job.execute_in_process(
        run_config={
            "ops": {
                "my_op": {
                    "inputs": {
                        "input_apple": {
                            "diameter": 2.4,
                            "juiciness": 6.0,
                            "cultivar": "honeycrisp",
                        }
                    }
                }
            }
        },
    )

Providing an input manager for an unconnected input#

When you have an op at the beginning of a job that operates on data from an external source, you might wish to separate that I/O from your op's business logic, in the same way you would with an IO manager if the op were loading from an upstream output.

Dagster provides a few ways to achieve this:

One option is to write a function to load the input and decorate it with @input_manager:

@input_manager
def simple_table_1_manager():
    return read_dataframe_from_table(name="table_1")


@op(ins={"dataframe": In(input_manager_key="simple_load_input_manager")})
def my_op(dataframe):
    """Do some stuff"""
    dataframe.head()


@job(resource_defs={"simple_load_input_manager": simple_table_1_manager})
def simple_load_table_job():
    my_op()

Another option is to define a class that implements the InputManager interface.

class Table1InputManager(InputManager):
    def load_input(self, context):
        return read_dataframe_from_table(name="table_1")


@input_manager
def table_1_manager():
    return Table1InputManager()


@job(resource_defs={"load_input_manager": table_1_manager})
def load_table_job():
    my_op()

If you also want to use Table1InputManager to store outputs, or you want to override the load_input method of an IO Manager used elsewhere in the job, another option is to implement an instance of IOManager:

# in this example, TableIOManager is defined elsewhere and we just want to override load_input
class Table1IOManager(TableIOManager):
    def load_input(self, context):
        return read_dataframe_from_table(name="table_1")


@io_manager
def table_1_io_manager():
    return Table1IOManager()


@job(resource_defs={"load_input_manager": table_1_io_manager})
def io_load_table_job():
    my_op()

In all of these examples, setting the input_manager_key on an In controls how that input is loaded.

Providing per-input config to an input manager#

When launching a run, you might want to parameterize how particular inputs are loaded.

To accomplish this, you can define an input_config_schema on the IO manager or Input manager definition. The load_input function can access this config when storing or loading data, via the InputContext.

class MyConfigurableInputLoader(InputManager):
    def load_input(self, context):
        return read_dataframe_from_table(name=context.config["table"])


@input_manager(input_config_schema={"table": str})
def my_configurable_input_loader():
    return MyConfigurableInputLoader()


# or


@input_manager(input_config_schema={"table": str})
def my_other_configurable_input_loader(context):
    return read_dataframe_from_table(name=context.config["table"])

Then, when executing a job, you can pass in this per-input config.

load_table_job.execute_in_process(
    run_config={"ops": {"my_op": {"inputs": {"dataframe": {"table": "table_1"}}}}},
)

Using a input manager with subselection#

You might want to execute a subset of ops in your job and control how the inputs of those ops are loaded. Custom input managers also help in these situations, because the inputs at the beginning of the subset become unconnected inputs.

For example, you might have op1 that normally produces a table that op2 consumes. To debug op2, you might want to run it on a different table than the one normally produced by op1.

To accomplish this, you can set up the input_manager_key on op2's In to point to an input manager with the desired loading behavior. As in the previous example, setting the input_manager_key on an In controls how that input is loaded and you can write custom loading logic.

class MyIOManager(IOManager):
    def handle_output(self, context, obj):
        table_name = context.name
        write_dataframe_to_table(name=table_name, dataframe=obj)

    def load_input(self, context):
        return read_dataframe_from_table(name=context.upstream_output.name)


@io_manager
def my_io_manager(_):
    return MyIOManager()


@input_manager
def my_subselection_input_manager():
    return read_dataframe_from_table(name="table_1")


@op
def op1():
    """Do stuff"""


@op(ins={"dataframe": In(input_manager_key="my_input_manager")})
def op2(dataframe):
    """Do stuff"""
    dataframe.head()


@job(
    resource_defs={
        "io_manager": my_io_manager,
        "my_input_manager": my_subselection_input_manager,
    }
)
def my_subselection_job():
    op2(op1())

So far, this is set up so that op2 always loads table_1 even if you execute the full job. This would let you debug op2, but if you want to write this so that op2 only loads table_1 when no input is provided from an upstream op, you can rewrite the input manager as a subclass of the IO manager used for the rest of the job as follows:

class MyNewInputLoader(MyIOManager):
    def load_input(self, context):
        if context.upstream_output is None:
            # load input from table since there is no upstream output
            return read_dataframe_from_table(name="table_1")
        else:
            return super().load_input(context)

Now, when running the full job, op2's input will be loaded using the IO manager on the output of op1. When running the job subset, op2's input has no upstream output, so table_1 will be loaded.

my_subselection_job.execute_in_process(
    op_selection=["op2"],
)