Source code for airflow.timetables.simple
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations
from typing import TYPE_CHECKING, Any, TypeAlias
from airflow._shared.timezones import timezone
from airflow.serialization.definitions.assets import SerializedAsset, SerializedAssetAll, SerializedAssetBase
from airflow.timetables.base import DagRunInfo, DataInterval, Timetable
try:
from airflow.sdk.definitions.asset import BaseAsset
from airflow.serialization.encoders import ensure_serialized_asset
except ModuleNotFoundError:
def ensure_serialized_asset(o): # type: ignore[misc,no-redef]
return o
if TYPE_CHECKING:
from collections.abc import Collection, Sequence
from pendulum import DateTime
from airflow.partition_mapper.base import PartitionMapper
from airflow.timetables.base import TimeRestriction
from airflow.utils.types import DagRunType
class _TrivialTimetable(Timetable):
"""Some code reuse for "trivial" timetables that has nothing complex."""
periodic = False
run_ordering: Sequence[str] = ("logical_date",)
@classmethod
def deserialize(cls, data: dict[str, Any]) -> Timetable:
return cls()
def __eq__(self, other: object) -> bool:
"""
As long as *other* is of the same type.
This is only for testing purposes and should not be relied on otherwise.
"""
from airflow.serialization.encoders import coerce_to_core_timetable
if not isinstance(other := coerce_to_core_timetable(other), type(self)):
return NotImplemented
return True
def __hash__(self):
return hash(self.__class__.__name__)
def infer_manual_data_interval(self, *, run_after: DateTime) -> DataInterval:
return DataInterval.exact(run_after)
[docs]
class NullTimetable(_TrivialTimetable):
"""
Timetable that never schedules anything.
This corresponds to ``schedule=None``.
"""
@property
[docs]
def next_dagrun_info(
self,
*,
last_automated_data_interval: DataInterval | None,
restriction: TimeRestriction,
) -> DagRunInfo | None:
return None
[docs]
class OnceTimetable(_TrivialTimetable):
"""
Timetable that schedules the execution once as soon as possible.
This corresponds to ``schedule="@once"``.
"""
@property
[docs]
def next_dagrun_info(
self,
*,
last_automated_data_interval: DataInterval | None,
restriction: TimeRestriction,
) -> DagRunInfo | None:
if last_automated_data_interval is not None:
return None # Already run, no more scheduling.
# If the user does not specify an explicit start_date, the dag is ready.
run_after = restriction.earliest or timezone.coerce_datetime(timezone.utcnow())
# "@once" always schedule to the start_date determined by the DAG and
# tasks, regardless of catchup or not. This has been the case since 1.10
# and we're inheriting it.
if restriction.latest is not None and run_after > restriction.latest:
return None
return DagRunInfo.exact(run_after)
[docs]
class ContinuousTimetable(_TrivialTimetable):
"""
Timetable that schedules continually, while still respecting start_date and end_date.
This corresponds to ``schedule="@continuous"``.
"""
# TODO (GH-52141): Find a way to keep this and one in Core in sync.
@property
[docs]
def next_dagrun_info(
self,
*,
last_automated_data_interval: DataInterval | None,
restriction: TimeRestriction,
) -> DagRunInfo | None:
if restriction.earliest is None: # No start date, won't run.
return None
current_time = timezone.coerce_datetime(timezone.utcnow())
if last_automated_data_interval is not None: # has already run once
if last_automated_data_interval.end > current_time: # start date is future
start = restriction.earliest
elapsed = last_automated_data_interval.end - last_automated_data_interval.start
end = start + elapsed.as_timedelta()
else:
start = last_automated_data_interval.end
end = current_time
else: # first run
start = restriction.earliest
end = max(restriction.earliest, current_time)
if restriction.latest is not None and end > restriction.latest:
return None
return DagRunInfo.interval(start, end)
class AssetTriggeredTimetable(_TrivialTimetable):
"""
Timetable that never schedules anything.
This should not be directly used anywhere, but only set if a DAG is triggered by assets.
:meta private:
"""
description: str = "Triggered by assets"
def __init__(self, assets: Collection[SerializedAsset] | SerializedAssetBase) -> None:
super().__init__()
# Compatibility: Handle SDK assets if needed so this class works in dag files.
if isinstance(assets, SerializedAssetBase | BaseAsset):
self.asset_condition = ensure_serialized_asset(assets)
else:
self.asset_condition = SerializedAssetAll([ensure_serialized_asset(a) for a in assets])
@classmethod
def deserialize(cls, data: dict[str, Any]) -> Timetable:
from airflow.serialization.decoders import decode_asset_like
return cls(decode_asset_like(data["asset_condition"]))
@property
def summary(self) -> str:
return "Asset"
def serialize(self) -> dict[str, Any]:
from airflow.serialization.encoders import encode_asset_like
return {"asset_condition": encode_asset_like(self.asset_condition)}
def generate_run_id(
self,
*,
run_type: DagRunType,
data_interval: DataInterval | None,
run_after: DateTime,
**extra,
) -> str:
"""
Generate Run ID based on Run Type, run_after and logical Date.
:param run_type: type of DagRun
:param data_interval: the data interval
:param run_after: the date before which dag run won't start.
"""
from airflow.models.dagrun import DagRun
logical_date = data_interval.start if data_interval is not None else run_after
return DagRun.generate_run_id(run_type=run_type, logical_date=logical_date, run_after=run_after)
def next_dagrun_info(
self,
*,
last_automated_data_interval: DataInterval | None,
restriction: TimeRestriction,
) -> DagRunInfo | None:
return None
[docs]
class PartitionedAssetTimetable(AssetTriggeredTimetable):
"""Asset-driven timetable that listens for partitioned assets."""
@property
def __init__(self, assets: SerializedAssetBase, partition_mapper: PartitionMapper) -> None:
super().__init__(assets=assets)
[docs]
def serialize(self) -> dict[str, Any]:
from airflow.serialization.serialized_objects import encode_asset_like
return {
"asset_condition": encode_asset_like(self.asset_condition),
"partition_mapper": self.partition_mapper.serialize(),
}
@classmethod
[docs]
def deserialize(cls, data: dict[str, Any]) -> Timetable:
from airflow.serialization.decoders import decode_partition_mapper
from airflow.serialization.serialized_objects import decode_asset_like
return cls(
assets=decode_asset_like(data["asset_condition"]),
partition_mapper=decode_partition_mapper(data["partition_mapper"]),
)