- Name
- Sandy Ryza
- Handle
- @s_ryz
For a developer-focused product, APIs are the user interface. Users tell us a lot of nice things about the expressivity of Dagster’s APIs, but one of the more persistent pieces of critical feedback that we’ve received is that at times they have a steep learning curve.
In the last two months, we've made a set of changes aimed at making Dagster more approachable - to smooth out Dagster's learning curve and reduce its boilerplate. Many of these changes are small on their own, but together, they constitute a large qualitative change to the feel of developing with Dagster. In this post, I'll run through some of these changes and how they impact users. All of the changes are backwards-compatible - they add new capabilities or enable earlier capabilities to be used in more permissive ways.
Progressive disclosure of complexity
Many of the changes are aimed at improving Dagster’s progressive disclosure of complexity. Dagster includes a large array of concepts that help model and organize the complexity of the data development and deployment lifecycle. Concepts that feel vital when you're building a complex data application can feel heavyweight when you're just trying to get a basic pipeline up and running. You want software that grows with you. Progressive disclosure of complexity is about making it simple to do simple things and having advanced concepts that show up only when you need them.
Many of the changes are aimed at reducing boilerplate. We wanted to make sure that people developing on top of Dagster wouldn't need to replicate the same information in multiple places. A caveat here is that we don’t want to make things too magical. It’s always possible to go further in reducing boilerplate, but at some point, clarity can suffer as well.
The changes covered here include:
- You can now define solids without a context argument, and you can invoke solids directly.
- You no longer need to work around the Dagster Type system to use Python type annotations on your solids.
- By default, solids and resources now accept any config, and you can opt-in to specifying schema.
- You can now provide event metadata to AssetMaterializations, Outputs, and TypeChecks using a less verbose API.
Optional context and direct solid invocation
Our biggest change was making it easier to use solids like vanilla Python functions: solids now no longer require a context
argument and can be directly invoked. This means that you can easily “upgrade” Python functions to Dagster solids by simply adding the @solid
decorator - you don’t need to change those function’s call sites. This makes the ramp-up from prototyping to productionization easier - you might define a set of functions in a .py file, import them into a Jupyter notebook to play around with them in development, then add the solid decorator to execute them with isolation, retries, and output materialization in production.
Optional context
Dagster solids accept a context
argument that allows them to access aspects of their environment, most commonly config and resources.
Many solids don’t need to access anything inside their context, and the context argument ends up as deadweight. Dagster now allows you to omit the context argument when defining solids. I.e. these are now valid solid definitions:
@solid
def my_solid():
return 5
@solid
def add(x, y):
return x + y
Direct invocation
To execute a solid, you used to need to invoke a function called execute_solid
, which would run your solid and give you a result object, which you could pull the output value from. Here’s what it used to look like to write a test for the add
solids above:
def test_add():
assert execute_solid(add, input_values={"x": 5, "y": 6}).output_value() == 11
Solids can now be directly invoked like regular functions. Which means the test can just look like:
def test_add():
assert add(5, 6) == 11
If your solid requires a context object, you can just create a context and pass it in. Here’s what it looks like to write a test for a solid that uses a context argument to access configuration:
@solid(config_schema={"multiplier": int})
def operate(context, x, y):
return (x + y) * context.solid_config["multiplier"]
def test_operate():
context = build_solid_context(config={"multiplier": 2})
assert operate(context, 5, 6) == 22
Direct invocation means fewer APIs to remember, a smaller code footprint, and a smoother ramp from vanilla Python to Dagster.
Dagster Types and Python Types
We built Dagster types to help developers write flexible runtime checks on the data passed between solids. This helps catch errors early - you can notice a problem with data after it's generated but before it's used - and it helps developers build confidence that the data they're working with is the data they expect they're working with.
We’ve historically required users to engage with Dagster Types in some situations where the user is not trying to write a custom runtime check. In particular, users who wanted to use regular PEP 484 Python type annotations on the arguments and return values of their solids needed to interact with the Dagster Type system. With our recent changes, users no longer need to be aware of the Dagster Type system to use Python type annotations on their solids.
Previously, this was the code you needed to write if you wanted an argument to a solid-decorated function to include a Python type annotation for a non-builtin type:
from pandas import Series
from dagster import (
PythonObjectDagsterType,
make_python_type_usable_as_dagster_type,
solid,
)
DagsterSeries = PythonObjectDagsterType(Series)
make_python_type_usable_as_dagster_type(Series, DagsterSeries)
@solid
def my_solid_before(_, input1: Series):
...
You had to write Series
three times - once to define a Dagster Type, once to register that Dagster Type with the Python type, and once to actually annotate the function.
Now, you can just do:
from pandas import Series
from dagster import solid
@solid
def my_solid_before(_, input1: Series):
...
Previously, that code snippet would raise an error, because Dagster would try to find a Dagster Type that corresponded to the Series type annotation and fail. Now, behind the scenes, Dagster creates a Dagster Type to wrap the Python type, which means the Python type can show up in Dagit and be checked at runtime.
In general, if you want to express the Python type of an input or output, just use regular Python type annotations. If you want to define custom checking logic for an input or output, use a Dagster Type.
Gradual config schematization
We’ve made it so that, when you want to supply config to a solid, you only need to supply config schema if you want Dagster to validate the config values.
Historically, if someone wanted to provide configuration to a solid in their pipeline, they would need to provide a value for the config_schema
argument on the solid. Config schemas declare the configuration values and types to the solid, which allow catching errors early and documenting the expected config.
We observed that, for many people, this was a lot to deal with when learning the system. In order to do basic things, they needed to understand how config schemas worked and the type system for config schemas. Even for Dagster experts, it can be annoying to need to provide detailed config schema when just trying to prototype or when passing a blob of data with complex structure.
So we made a small but significant change, which is to make the default config schema type the Any
type. What this essentially means is that now you can provide config to a solid without needing to define a config schema for it.
Here’s an example that defines a pipeline with a single solid that uses config:
@solid(config_schema={"iterations": int})
def my_solid(context):
for _ in range(context.solid_config["iterations"]):
context.log.info("hello")
@pipeline
def my_pipeline():
my_solid()
execute_pipeline(
pipeline,
run_config={"solids": {"my_solid": {"config": {"iterations": 1}}}},
)
You can now simply define the solid as the following, and it will still work:
@solid
def my_solid(context):
for _ in range(context.solid_config["iterations"]):
context.log.info("hello")
The above code snippet is equivalent to the following, which had worked previously (and still works):
@solid(config_schema=Any)
def my_solid(context):
for _ in range(context.solid_config["iterations"]):
context.log.info("hello")
So when should you use config schema? We still believe that it’s wise to use config schema for any solid you’re putting in production, because catching errors early saves time in the long run. But when prototyping in development or when teaching Dagster to someone new, it can make sense to omit it.
Event metadata
Another area we’ve tried to streamline is event metadata. Event metadata is the kind of metadata that you can include on events like AssetMaterializations, which then gets nicely displayed in Dagit. The changes we’ve made here focus on boilerplate elimination.
Here’s an example AssetMaterialization that includes four metadata fields. As you can see, it’s quite a chunk of code to write this out.
yield AssetMaterialization(
asset_key=f"bigquery.{model}",
metadata_entries=[
EventMetadataEntry.float(
node.get("execution_time", -1.0),
label="dbt Model Execution Time",
),
EventMetadataEntry.int(
int(count),
label="dbt Model Number of Rows",
),
EventMetadataEntry.md(
sample_rows.astype("str").to_markdown(),
label="Sample of Rows",
),
EventMetadataEntry.path(
run_results_path,
label="dbt Run Results Artifact",
),
],
)
We now support a condensed format that accepts a dictionary instead of a list and automatically coerces primitive types to their corresponding event metadata types:
yield AssetMaterialization(
asset_key=f"bigquery.{model}",
metadata={
"dbt Model Execution Time": node.get("execution_time", -1.0),
"dbt Model Number of Rows": int(count),
"Sample of Rows": EventMetadata.md(sample_rows.astype("str").to_markdown()),
"dbt Run Results Artifact": EventMetadata.path(run_results_path),
},
)
That’s not all
These are only a sampling of the changes we’ve made to smooth Dagster’s ramp-up and decrease boilerplate. For a full list of all recent Dagster changes, check out the release notes.
Thank you Yuhan Luo, Chris DeCarolis, Phil Rha, Owen Kephart, and Alex Langenfeld for contributing these improvements.
We're always happy to hear your feedback, so please reach out to us! If you have any questions, ask them in the Dagster community Slack (join here!) or start a Github discussion. If you run into any bugs, let us know with a Github issue. And if you're interested in working with us, check out our open roles!
Follow us:
Interactive Debugging With Dagster and Docker
- Name
- Gianfranco Demarco
- Handle
- @gianfranco
AI's Long-Term Impact on Data Engineering Roles
- Name
- Fraser Marlow
- Handle
- @frasermarlow
10 Reasons Why No-Code Solutions Almost Always Fail
- Name
- TéJaun RiChard
- Handle
- @tejaun