The most common kind of partitioned job is a time-partitioned job - each partition is a time window, and each run for a partition processes data within that time window.
Before we dive in, let's look at a non-partitioned job that computes some data for a given date:
from dagster import Config, OpExecutionContext, job, op
classProcessDateConfig(Config):
date:str@opdefprocess_data_for_date(context: OpExecutionContext, config: ProcessDateConfig):
date = config.date
context.log.info(f"processing data for {date}")@jobdefdo_stuff():
process_data_for_date()
It takes, as config, a string date. This piece of config defines which date to compute data for. For example, if you wanted to compute for May 5th, 2020, you would execute the graph with the following config:
With the job above, it's possible to supply any value for the date param. This means if you wanted to launch a backfill, Dagster wouldn't know what values to run it on. You can instead build a partitioned job that operates on a defined set of dates.
First, define the PartitionedConfig. In this case, because each partition is a date, you can use the @daily_partitioned_config decorator. This decorator defines the full set of partitions - every date between the start date and the current date, as well as how to determine the run config for a given partition.
from dagster import daily_partitioned_config
from datetime import datetime
@daily_partitioned_config(start_date=datetime(2020,1,1))defpartitioned_config(start: datetime, _end: datetime):return{"ops":{"process_data_for_date":{"config":{"date": start.strftime("%Y-%m-%d")}}}}
Then you can build a job that uses the PartitionedConfig by supplying it to the config argument when you construct the job:
Running a partitioned job on a schedule is a common use case. For example, if your job has a partition for each date, you likely want to run that job every day, on the partition for that day.
Refer to the Schedule documentation for more info about constructing both schedules for asset and op-based jobs.
In the UI, you can view runs by partition in the Partitions tab of a Job page:
In the Run Matrix, each column corresponds to one of the partitions in the job. The time listed corresponds to the start time of the partition. Each row corresponds to one of the steps in the job. You can click on an individual box to navigate to logs and run information for the step.
You can view and use partitions in the UI Launchpad tab for a job. In the top bar, you can select from the list of all available partitions. Within the config editor, the config for the selected partition will be populated.
In the screenshot below, we select the 2020-01-02 partition, and we can see that the run config for the partition has been populated in the editor.