For jobs partitioned by time, you can use the build_schedule_from_partitioned_job to construct a schedule for the job. The schedule's interval will match the spacing of the partitions in the job. For example, if you have a daily partitioned job that fills in a date partition of a table each time it runs, you likely want to run that job every day.
Op jobs are defined using the @job. In this example, we created a partitioned job named partitioned_op_job and then constructed partitioned_op_schedule using build_schedule_from_partitioned_job:
from dagster import build_schedule_from_partitioned_job, job
@job(config=partitioned_config)defpartitioned_op_job():...
partitioned_op_schedule = build_schedule_from_partitioned_job(
partitioned_op_job,)
The minute_of_hour, hour_of_day, day_of_week, and day_of_month parameters of build_schedule_from_partitioned_job can be used to control the timing of the schedule.
Consider the following job:
from dagster import build_schedule_from_partitioned_job
asset_partitioned_schedule = build_schedule_from_partitioned_job(
partitioned_asset_job, hour_of_day=1, minute_of_hour=30)
On May 20, 2024, the schedule will evaluate at 1:30 AM UTC and then start a run for the partition key of the previous day, 2024-05-19.
Heads up! The examples in this section use daily partitions, but the same logic also applies to other time-based partitions, such as hourly, weekly, and monthly partitions.
Each schedule tick of a partitioned job targets the latest partition in the partition set that exists as of the tick time. For example, consider a schedule that runs a daily-partitioned job. When the schedule runs on 2024-05-20, it will target the most recent partition, which will correspond to the previous day: 2024-05-19.
If a job runs on this date...
It will target this partition
2024-05-20
2024-05-19
2024-05-21
2024-05-20
2024-05-22
2024-05-21
This occurs because each partition is a time window. A time window is a set period of time with a start and an end time. The partition's key is the start of the time window, but the partition isn't included in the partition set until its time window has completed. Kicking off a run after the time window completes allows the run to process data for the entire time window.
Continuing with the daily partition example, the 2024-05-20 partition would have the following start and end times:
Start time - 2024-05-20 00:00:00
End time - 2024-05-20 23:59:59
After 2024-05-20 23:59:59 passes, the time window is complete and Dagster will add a new 2024-05-20 partition to the partition set. At this point, the process will repeat with the next time window of 2024-05-21.
If you need to customize the ending, or most recent partition in a set, use the end_offset parameter in the partition's config:
from dagster import DailyPartitionsDefinition
daily_partition_with_offset = DailyPartitionsDefinition(
start_date="2024-05-20", end_offset=-1)
Setting this parameter changes the partition that will be filled in at each schedule tick. Positive and negative integers are accepted, which will have the following effects:
Positive numbers, like 1, cause the schedule to fill in the partition of the current hour/day/week/month
Negative numbers, like -1, cause the schedule to fill in the partition of an earlier hour/day/week/month
Generally, the calculation for end_offset can be expressed as:
current_date - 1 type_of_partition + end_offset
Let's look at an example schedule that's partitioned by day and how different end_offset values would affect the most recent partition in the set. In this example, we're using a start date of 2024-05-20:
Next, we'll demonstrate how to create a schedule for a job with a static partition. To do this, we'll construct the schedule from scratch using the @schedule decorator, rather than using a helper function like build_schedule_from_partitioned_job. This will allow more flexibility in determining which partitions should be run by the schedule.
In this example, the job is partitioned by continent: