Changeset View
Standalone View
docs/content/concepts/partitions-schedules-sensors/schedules.mdx
Show All 11 Lines | |||||
| Name | Description | | | Name | Description | | ||||
| ------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | | ------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | ||||
| <PyObject object="daily_schedule" decorator /> | Decorator that defines a schedule that executes every day at a fixed time | | | <PyObject object="daily_schedule" decorator /> | Decorator that defines a schedule that executes every day at a fixed time | | ||||
| <PyObject object="hourly_schedule" decorator /> | Decorator that defines a schedule that executes every hour at a fixed time | | | <PyObject object="hourly_schedule" decorator /> | Decorator that defines a schedule that executes every hour at a fixed time | | ||||
| <PyObject object="weekly_schedule" decorator /> | Decorator that defines a schedule that executes every week on a fixed day and time | | | <PyObject object="weekly_schedule" decorator /> | Decorator that defines a schedule that executes every week on a fixed day and time | | ||||
| <PyObject object="monthly_schedule" decorator /> | Decorator that defines a schedule that executes every month on a fixed day and time | | | <PyObject object="monthly_schedule" decorator /> | Decorator that defines a schedule that executes every month on a fixed day and time | | ||||
| <PyObject object="schedule" decorator /> | Decorator that defines a schedule that executes according to a given cron schedule. It's important to note that schedules created using this schedule are [non-partition based schedules](/concepts/partitions-schedules-sensors/schedules#non-partition-based-schedules) | | | <PyObject object="schedule" decorator /> | Decorator that defines a schedule that executes according to a given cron schedule. It's important to note that schedules created using this schedule are [non-partition based schedules](/concepts/partitions-schedules-sensors/schedules#non-partition-based-schedules) | | ||||
| <PyObject object="ScheduleExecutionContext" /> | The context passed to the schedule definition execution function | | | <PyObject object="ScheduleExecutionContext" /> | The context passed to the schedule definition execution function | | ||||
| <PyObject object="build_schedule_context" /> | A function that constructs a `ScheduleExecutionContext` | | |||||
| <PyObject object="ScheduleDefinition" /> | Class for schedules. You almost never want to use initialize this class directly. Instead, you should use one of the many decorators above | | | <PyObject object="ScheduleDefinition" /> | Class for schedules. You almost never want to use initialize this class directly. Instead, you should use one of the many decorators above | | ||||
## Overview | ## Overview | ||||
A _schedule_ is a definition in Dagster that is used to execute a pipeline at a fixed interval. Each time at which a schedule is evaluated is called a _tick_. The schedule definition is responsible for generating run configuration for the pipeline on each tick. | A _schedule_ is a definition in Dagster that is used to execute a pipeline at a fixed interval. Each time at which a schedule is evaluated is called a _tick_. The schedule definition is responsible for generating run configuration for the pipeline on each tick. | ||||
Each schedule: | Each schedule: | ||||
▲ Show 20 Lines • Show All 91 Lines • ▼ Show 20 Lines | |||||
Hourly schedules will be unaffected by daylight savings time transitions - the schedule will continue to run exactly once every hour, even as the timezone changes. In the example above where the hour from 1:00 AM to 2:00 AM repeats, an hourly schedule running at 30 minutes past the hour would run at 12:30 AM, both instances of 1:30 AM, and then proceed normally from 2:30 AM on. | Hourly schedules will be unaffected by daylight savings time transitions - the schedule will continue to run exactly once every hour, even as the timezone changes. In the example above where the hour from 1:00 AM to 2:00 AM repeats, an hourly schedule running at 30 minutes past the hour would run at 12:30 AM, both instances of 1:30 AM, and then proceed normally from 2:30 AM on. | ||||
## Running the scheduler | ## Running the scheduler | ||||
In order for your schedule to run, it must be started. The easiest way to start and stop schedules is in Dagit from the Schedules page. You can also start and stop a schedule with the `dagster schedule start` and `dagster schedule stop` commands. | In order for your schedule to run, it must be started. The easiest way to start and stop schedules is in Dagit from the Schedules page. You can also start and stop a schedule with the `dagster schedule start` and `dagster schedule stop` commands. | ||||
Once your schedule is started, if you're running the [dagster-daemon](/deployment/dagster-daemon) process as part of your deployment, the schedule will begin executing immediately. See the [Troubleshooting](/concepts/partitions-schedules-sensors/schedules#troubleshooting) section below if your schedule has been started but isn't submitting runs. | Once your schedule is started, if you're running the [dagster-daemon](/deployment/dagster-daemon) process as part of your deployment, the schedule will begin executing immediately. See the [Troubleshooting](/concepts/partitions-schedules-sensors/schedules#troubleshooting) section below if your schedule has been started but isn't submitting runs. | ||||
## Testing Schedules | |||||
Schedules can be tested by using `build_schedule_context` to construct a `ScheduleExecutionContext`, and passing this to the schedule's `evaluate_tick` function. The resulting run config from this function can then be validated using the <PyObject object="validate_run_config" /> function. | |||||
```python file=concepts/partitions_schedules_sensors/schedules/schedule_examples.py startafter=start_test_schedule endbefore=end_test_schedule | |||||
@hourly_schedule( | |||||
pipeline_name="test_pipeline", | |||||
start_date=datetime.datetime(2020, 1, 1), | |||||
execution_time=datetime.time(hour=0, minute=25), | |||||
execution_timezone="US/Central", | |||||
) | |||||
def hourly_schedule_to_test(date): | |||||
return { | |||||
"solids": { | |||||
"process_data_for_date": { | |||||
"config": { | |||||
"date": date.strftime("%Y-%m-%d %H"), | |||||
yuhan: similar comment as D8058 - dont think `instance_for_test` is a public api | |||||
} | |||||
Not Done Inline ActionsI know this isn't directly relevant to this diff, but reading this now makes me feel like get_execution_data is a weird name. "execution" and "data" are both extremely general terms, and "execute" normally has a different meaning in Dagster. Also, "get" implies that we're accessing some attribute, rather than triggering some arbitrarily complex computation. Thoughts on renaming this to something like do_tick? @alangenfeld or @dgibson might have opinions. Also, whatever we name it, it's not included in the apidoc, so would be good to add it. sandyryza: I know this isn't directly relevant to this diff, but reading this now makes me feel like… | |||||
Done Inline Actionsthat makes a lot of sense to me as well. get_execution_data is confusing at best, misleading at worst, imo, without really telling you what the function is doing. cdecarolis: that makes a lot of sense to me as well. `get_execution_data` is confusing at best, misleading… | |||||
Done Inline ActionsRelated: ScheduleExecutionContext also feels like a rough name. Execution seems like it doesn't add anything, or fit into our meaning of execution. cdecarolis: Related: `ScheduleExecutionContext` also feels like a rough name. `Execution` seems like it… | |||||
Not Done Inline ActionsAgree. Maybe ScheduleTickContext or ScheduleContext? sandyryza: Agree. Maybe `ScheduleTickContext` or `ScheduleContext`? | |||||
} | |||||
} | |||||
} | |||||
def test_hourly_schedule(): | |||||
schedule_data = hourly_schedule_to_test.evaluate_tick(build_schedule_context()) | |||||
for run_request in schedule_data.run_requests: | |||||
assert validate_run_config(pipeline_for_test, run_request.run_config) | |||||
``` | |||||
## Examples | ## Examples | ||||
### Hourly partition-based schedule | ### Hourly partition-based schedule | ||||
```python file=concepts/partitions_schedules_sensors/schedules/schedule_examples.py startafter=start_hourly_schedule endbefore=end_hourly_schedule | ```python file=concepts/partitions_schedules_sensors/schedules/schedule_examples.py startafter=start_hourly_schedule endbefore=end_hourly_schedule | ||||
@hourly_schedule( | @hourly_schedule( | ||||
pipeline_name="my_pipeline", | pipeline_name="my_pipeline", | ||||
start_date=datetime.datetime(2020, 1, 1), | start_date=datetime.datetime(2020, 1, 1), | ||||
▲ Show 20 Lines • Show All 71 Lines • ▼ Show 20 Lines | @daily_schedule( | ||||
start_date=datetime.datetime(2020, 1, 1), | start_date=datetime.datetime(2020, 1, 1), | ||||
pipeline_name="my_pipeline", | pipeline_name="my_pipeline", | ||||
solid_selection=preset.solid_selection, | solid_selection=preset.solid_selection, | ||||
mode=preset.mode, | mode=preset.mode, | ||||
tags_fn_for_date=lambda _: preset.tags, | tags_fn_for_date=lambda _: preset.tags, | ||||
) | ) | ||||
def my_modified_preset_schedule(date): | def my_modified_preset_schedule(date): | ||||
modified_run_config = preset.run_config.copy() | modified_run_config = preset.run_config.copy() | ||||
modified_run_config["date"] = date.strftime("%Y-%m-%d") | modified_run_config["solids"]["process_data_for_date"]["config"]["date"] = date.strftime( | ||||
"%Y-%m-%d" | |||||
) | |||||
return modified_run_config | return modified_run_config | ||||
``` | ``` | ||||
If you find yourself using presets to generate schedule definitions frequently, you can use a helper function similar to this one to take a preset and return a schedule. | If you find yourself using presets to generate schedule definitions frequently, you can use a helper function similar to this one to take a preset and return a schedule. | ||||
```python file=concepts/partitions_schedules_sensors/schedules/preset_helper.py startafter=start_preset_helper endbefore=end_preset_helper | ```python file=concepts/partitions_schedules_sensors/schedules/preset_helper.py startafter=start_preset_helper endbefore=end_preset_helper | ||||
def daily_schedule_definition_from_pipeline_preset(pipeline, preset_name, start_date): | def daily_schedule_definition_from_pipeline_preset(pipeline, preset_name, start_date): | ||||
preset = pipeline.get_preset(preset_name) | preset = pipeline.get_preset(preset_name) | ||||
▲ Show 20 Lines • Show All 50 Lines • Show Last 20 Lines |
similar comment as D8058 - dont think instance_for_test is a public api