diff --git a/airflow/api/client/local_client.py b/airflow/api/client/local_client.py index 7d828792c426..800afa283021 100644 --- a/airflow/api/client/local_client.py +++ b/airflow/api/client/local_client.py @@ -36,14 +36,14 @@ def __init__(self, auth=None, session: httpx.Client | None = None): self._session.auth = auth def trigger_dag( - self, dag_id, run_id=None, conf=None, execution_date=None, replace_microseconds=True + self, dag_id, run_id=None, conf=None, logical_date=None, replace_microseconds=True ) -> dict | None: dag_run = trigger_dag.trigger_dag( dag_id=dag_id, triggered_by=DagRunTriggeredByType.CLI, run_id=run_id, conf=conf, - execution_date=execution_date, + logical_date=logical_date, replace_microseconds=replace_microseconds, ) if dag_run: diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index 957e82e7de49..a170e6901a50 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -59,19 +59,19 @@ def _create_dagruns( :param dag: The DAG to create runs for. :param infos: List of logical dates and data intervals to evaluate. :param state: The state to set the dag run to - :param run_type: The prefix will be used to construct dag run id: ``{run_id_prefix}__{execution_date}``. - :return: Newly created and existing dag runs for the execution dates supplied. + :param run_type: The prefix will be used to construct dag run id: ``{run_id_prefix}__{logical_date}``. + :return: Newly created and existing dag runs for the logical dates supplied. """ # Find out existing DAG runs that we don't need to create. dag_runs = { run.logical_date: run - for run in DagRun.find(dag_id=dag.dag_id, execution_date=[info.logical_date for info in infos]) + for run in DagRun.find(dag_id=dag.dag_id, logical_date=[info.logical_date for info in infos]) } for info in infos: if info.logical_date not in dag_runs: dag_runs[info.logical_date] = dag.create_dagrun( - execution_date=info.logical_date, + logical_date=info.logical_date, data_interval=info.data_interval, start_date=timezone.utcnow(), external_trigger=False, @@ -87,7 +87,7 @@ def set_state( *, tasks: Collection[Operator | tuple[Operator, int]], run_id: str | None = None, - execution_date: datetime | None = None, + logical_date: datetime | None = None, upstream: bool = False, downstream: bool = False, future: bool = False, @@ -107,11 +107,11 @@ def set_state( :param tasks: the iterable of tasks or (task, map_index) tuples from which to work. ``task.dag`` needs to be set :param run_id: the run_id of the dagrun to start looking from - :param execution_date: the execution date from which to start looking (deprecated) + :param logical_date: the logical date from which to start looking (deprecated) :param upstream: Mark all parents (upstream tasks) :param downstream: Mark all siblings (downstream tasks) of task_id :param future: Mark all future tasks on the interval of the dag up until - last execution date. + last logical date. :param past: Retroactively mark all tasks starting from start_date of the DAG :param state: State to which the tasks need to be set :param commit: Commit tasks to be altered to the database @@ -121,11 +121,11 @@ def set_state( if not tasks: return [] - if not exactly_one(execution_date, run_id): - raise ValueError("Exactly one of dag_run_id and execution_date must be set") + if not exactly_one(logical_date, run_id): + raise ValueError("Exactly one of dag_run_id and logical_date must be set") - if execution_date and not timezone.is_localized(execution_date): - raise ValueError(f"Received non-localized date {execution_date}") + if logical_date and not timezone.is_localized(logical_date): + raise ValueError(f"Received non-localized date {logical_date}") task_dags = {task[0].dag if isinstance(task, tuple) else task.dag for task in tasks} if len(task_dags) > 1: @@ -134,8 +134,8 @@ def set_state( if dag is None: raise ValueError("Received tasks with no DAG") - if execution_date: - run_id = dag.get_dagrun(execution_date=execution_date, session=session).run_id + if logical_date: + run_id = dag.get_dagrun(logical_date=logical_date, session=session).run_id if not run_id: raise ValueError("Received tasks with no run_id") @@ -200,26 +200,26 @@ def find_task_relatives(tasks, downstream, upstream): @provide_session -def get_execution_dates( - dag: DAG, execution_date: datetime, future: bool, past: bool, *, session: SASession = NEW_SESSION +def get_logical_dates( + dag: DAG, logical_date: datetime, future: bool, past: bool, *, session: SASession = NEW_SESSION ) -> list[datetime]: - """Return DAG execution dates.""" - latest_execution_date = dag.get_latest_execution_date(session=session) - if latest_execution_date is None: - raise ValueError(f"Received non-localized date {execution_date}") - execution_date = timezone.coerce_datetime(execution_date) + """Return DAG logical dates.""" + latest_logical_date = dag.get_latest_logical_date(session=session) + if latest_logical_date is None: + raise ValueError(f"Received non-localized date {logical_date}") + logical_date = timezone.coerce_datetime(logical_date) # determine date range of dag runs and tasks to consider - end_date = latest_execution_date if future else execution_date + end_date = latest_logical_date if future else logical_date if dag.start_date: start_date = dag.start_date else: - start_date = execution_date - start_date = execution_date if not past else start_date + start_date = logical_date + start_date = logical_date if not past else start_date if not dag.timetable.can_be_scheduled: # If the DAG never schedules, need to look at existing DagRun if the user wants future or # past runs. dag_runs = dag.get_dagruns_between(start_date=start_date, end_date=end_date) - dates = sorted({d.execution_date for d in dag_runs}) + dates = sorted({d.logical_date for d in dag_runs}) elif not dag.timetable.periodic: dates = [start_date] else: @@ -235,7 +235,7 @@ def get_run_ids(dag: DAG, run_id: str, future: bool, past: bool, session: SASess last_dagrun = dag.get_last_dagrun(include_externally_triggered=True, session=session) current_dagrun = dag.get_dagrun(run_id=run_id, session=session) first_dagrun = session.scalar( - select(DagRun).filter(DagRun.dag_id == dag.dag_id).order_by(DagRun.execution_date.asc()).limit(1) + select(DagRun).filter(DagRun.dag_id == dag.dag_id).order_by(DagRun.logical_date.asc()).limit(1) ) if last_dagrun is None: @@ -255,7 +255,7 @@ def get_run_ids(dag: DAG, run_id: str, future: bool, past: bool, session: SASess dates = [ info.logical_date for info in dag.iter_dagrun_infos_between(start_date, end_date, align=False) ] - run_ids = [dr.run_id for dr in DagRun.find(dag_id=dag.dag_id, execution_date=dates, session=session)] + run_ids = [dr.run_id for dr in DagRun.find(dag_id=dag.dag_id, logical_date=dates, session=session)] return run_ids @@ -279,7 +279,7 @@ def _set_dag_run_state(dag_id: str, run_id: str, state: DagRunState, session: SA def set_dag_run_state_to_success( *, dag: DAG, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, commit: bool = False, session: SASession = NEW_SESSION, @@ -287,29 +287,29 @@ def set_dag_run_state_to_success( """ Set the dag run's state to success. - Set for a specific execution date and its task instances to success. + Set for a specific logical date and its task instances to success. :param dag: the DAG of which to alter state - :param execution_date: the execution date from which to start looking(deprecated) + :param logical_date: the logical date from which to start looking(deprecated) :param run_id: the run_id to start looking from :param commit: commit DAG and tasks to be altered to the database :param session: database session :return: If commit is true, list of tasks that have been updated, otherwise list of tasks that will be updated - :raises: ValueError if dag or execution_date is invalid + :raises: ValueError if dag or logical_date is invalid """ - if not exactly_one(execution_date, run_id): + if not exactly_one(logical_date, run_id): return [] if not dag: return [] - if execution_date: - if not timezone.is_localized(execution_date): - raise ValueError(f"Received non-localized date {execution_date}") - dag_run = dag.get_dagrun(execution_date=execution_date) + if logical_date: + if not timezone.is_localized(logical_date): + raise ValueError(f"Received non-localized date {logical_date}") + dag_run = dag.get_dagrun(logical_date=logical_date) if not dag_run: - raise ValueError(f"DagRun with execution_date: {execution_date} not found") + raise ValueError(f"DagRun with logical_date: {logical_date} not found") run_id = dag_run.run_id if not run_id: raise ValueError(f"Invalid dag_run_id: {run_id}") @@ -333,7 +333,7 @@ def set_dag_run_state_to_success( def set_dag_run_state_to_failed( *, dag: DAG, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, commit: bool = False, session: SASession = NEW_SESSION, @@ -341,28 +341,28 @@ def set_dag_run_state_to_failed( """ Set the dag run's state to failed. - Set for a specific execution date and its task instances to failed. + Set for a specific logical date and its task instances to failed. :param dag: the DAG of which to alter state - :param execution_date: the execution date from which to start looking(deprecated) + :param logical_date: the logical date from which to start looking(deprecated) :param run_id: the DAG run_id to start looking from :param commit: commit DAG and tasks to be altered to the database :param session: database session :return: If commit is true, list of tasks that have been updated, otherwise list of tasks that will be updated - :raises: AssertionError if dag or execution_date is invalid + :raises: AssertionError if dag or logical_date is invalid """ - if not exactly_one(execution_date, run_id): + if not exactly_one(logical_date, run_id): return [] if not dag: return [] - if execution_date: - if not timezone.is_localized(execution_date): - raise ValueError(f"Received non-localized date {execution_date}") - dag_run = dag.get_dagrun(execution_date=execution_date) + if logical_date: + if not timezone.is_localized(logical_date): + raise ValueError(f"Received non-localized date {logical_date}") + dag_run = dag.get_dagrun(logical_date=logical_date) if not dag_run: - raise ValueError(f"DagRun with execution_date: {execution_date} not found") + raise ValueError(f"DagRun with logical_date: {logical_date} not found") run_id = dag_run.run_id if not run_id: @@ -429,16 +429,16 @@ def __set_dag_run_state_to_running_or_queued( *, new_state: DagRunState, dag: DAG, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, commit: bool = False, session: SASession, ) -> list[TaskInstance]: """ - Set the dag run for a specific execution date to running. + Set the dag run for a specific logical date to running. :param dag: the DAG of which to alter state - :param execution_date: the execution date from which to start looking + :param logical_date: the logical date from which to start looking :param run_id: the id of the DagRun :param commit: commit DAG and tasks to be altered to the database :param session: database session @@ -447,18 +447,18 @@ def __set_dag_run_state_to_running_or_queued( """ res: list[TaskInstance] = [] - if not exactly_one(execution_date, run_id): + if not exactly_one(logical_date, run_id): return res if not dag: return res - if execution_date: - if not timezone.is_localized(execution_date): - raise ValueError(f"Received non-localized date {execution_date}") - dag_run = dag.get_dagrun(execution_date=execution_date) + if logical_date: + if not timezone.is_localized(logical_date): + raise ValueError(f"Received non-localized date {logical_date}") + dag_run = dag.get_dagrun(logical_date=logical_date) if not dag_run: - raise ValueError(f"DagRun with execution_date: {execution_date} not found") + raise ValueError(f"DagRun with logical_date: {logical_date} not found") run_id = dag_run.run_id if not run_id: raise ValueError(f"DagRun with run_id: {run_id} not found") @@ -474,7 +474,7 @@ def __set_dag_run_state_to_running_or_queued( def set_dag_run_state_to_running( *, dag: DAG, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, commit: bool = False, session: SASession = NEW_SESSION, @@ -482,12 +482,12 @@ def set_dag_run_state_to_running( """ Set the dag run's state to running. - Set for a specific execution date and its task instances to running. + Set for a specific logical date and its task instances to running. """ return __set_dag_run_state_to_running_or_queued( new_state=DagRunState.RUNNING, dag=dag, - execution_date=execution_date, + logical_date=logical_date, run_id=run_id, commit=commit, session=session, @@ -498,7 +498,7 @@ def set_dag_run_state_to_running( def set_dag_run_state_to_queued( *, dag: DAG, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, commit: bool = False, session: SASession = NEW_SESSION, @@ -506,12 +506,12 @@ def set_dag_run_state_to_queued( """ Set the dag run's state to queued. - Set for a specific execution date and its task instances to queued. + Set for a specific logical date and its task instances to queued. """ return __set_dag_run_state_to_running_or_queued( new_state=DagRunState.QUEUED, dag=dag, - execution_date=execution_date, + logical_date=logical_date, run_id=run_id, commit=commit, session=session, diff --git a/airflow/api/common/trigger_dag.py b/airflow/api/common/trigger_dag.py index 44beae3f1f78..4a94f990191c 100644 --- a/airflow/api/common/trigger_dag.py +++ b/airflow/api/common/trigger_dag.py @@ -44,7 +44,7 @@ def _trigger_dag( triggered_by: DagRunTriggeredByType, run_id: str | None = None, conf: dict | str | None = None, - execution_date: datetime | None = None, + logical_date: datetime | None = None, replace_microseconds: bool = True, ) -> DagRun | None: """ @@ -53,9 +53,9 @@ def _trigger_dag( :param dag_id: DAG ID :param dag_bag: DAG Bag model :param triggered_by: the entity which triggers the dag_run - :param run_id: ID of the dag_run + :param run_id: ID of the run :param conf: configuration - :param execution_date: date of execution + :param logical_date: logical date of the run :param replace_microseconds: whether microseconds should be zeroed :return: list of triggered dags """ @@ -64,31 +64,31 @@ def _trigger_dag( if dag is None or dag_id not in dag_bag.dags: raise DagNotFound(f"Dag id {dag_id} not found") - execution_date = execution_date or timezone.utcnow() + logical_date = logical_date or timezone.utcnow() - if not timezone.is_localized(execution_date): - raise ValueError("The execution_date should be localized") + if not timezone.is_localized(logical_date): + raise ValueError("The logical date should be localized") if replace_microseconds: - execution_date = execution_date.replace(microsecond=0) + logical_date = logical_date.replace(microsecond=0) if dag.default_args and "start_date" in dag.default_args: min_dag_start_date = dag.default_args["start_date"] - if min_dag_start_date and execution_date < min_dag_start_date: + if min_dag_start_date and logical_date < min_dag_start_date: raise ValueError( - f"The execution_date [{execution_date.isoformat()}] should be >= start_date " + f"Logical date [{logical_date.isoformat()}] should be >= start_date " f"[{min_dag_start_date.isoformat()}] from DAG's default_args" ) - logical_date = timezone.coerce_datetime(execution_date) + coerced_logical_date = timezone.coerce_datetime(logical_date) - data_interval = dag.timetable.infer_manual_data_interval(run_after=logical_date) + data_interval = dag.timetable.infer_manual_data_interval(run_after=coerced_logical_date) run_id = run_id or dag.timetable.generate_run_id( - run_type=DagRunType.MANUAL, logical_date=logical_date, data_interval=data_interval + run_type=DagRunType.MANUAL, logical_date=coerced_logical_date, data_interval=data_interval ) - dag_run = DagRun.find_duplicate(dag_id=dag_id, execution_date=execution_date, run_id=run_id) + dag_run = DagRun.find_duplicate(dag_id=dag_id, run_id=run_id, logical_date=logical_date) if dag_run: - raise DagRunAlreadyExists(dag_run=dag_run, execution_date=execution_date, run_id=run_id) + raise DagRunAlreadyExists(dag_run, logical_date=logical_date, run_id=run_id) run_conf = None if conf: @@ -96,7 +96,7 @@ def _trigger_dag( dag_version = DagVersion.get_latest_version(dag.dag_id) dag_run = dag.create_dagrun( run_id=run_id, - execution_date=execution_date, + logical_date=logical_date, state=DagRunState.QUEUED, conf=run_conf, external_trigger=True, @@ -116,7 +116,7 @@ def trigger_dag( triggered_by: DagRunTriggeredByType, run_id: str | None = None, conf: dict | str | None = None, - execution_date: datetime | None = None, + logical_date: datetime | None = None, replace_microseconds: bool = True, session: Session = NEW_SESSION, ) -> DagRun | None: @@ -126,7 +126,7 @@ def trigger_dag( :param dag_id: DAG ID :param run_id: ID of the dag_run :param conf: configuration - :param execution_date: date of execution + :param logical_date: date of execution :param replace_microseconds: whether microseconds should be zeroed :param session: Unused. Only added in compatibility with database isolation mode :param triggered_by: the entity which triggers the dag_run @@ -142,7 +142,7 @@ def trigger_dag( dag_bag=dagbag, run_id=run_id, conf=conf, - execution_date=execution_date, + logical_date=logical_date, replace_microseconds=replace_microseconds, triggered_by=triggered_by, ) diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py index 6a38eb27ff45..4ee621158b66 100644 --- a/airflow/api_connexion/endpoints/dag_run_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_run_endpoint.py @@ -157,11 +157,11 @@ def _fetch_dag_runs( query = query.where(DagRun.start_date >= start_date_gte) if start_date_lte: query = query.where(DagRun.start_date <= start_date_lte) - # filter execution date + # filter logical date if execution_date_gte: - query = query.where(DagRun.execution_date >= execution_date_gte) + query = query.where(DagRun.logical_date >= execution_date_gte) if execution_date_lte: - query = query.where(DagRun.execution_date <= execution_date_lte) + query = query.where(DagRun.logical_date <= execution_date_lte) # filter end date if end_date_gte: query = query.where(DagRun.end_date >= end_date_gte) @@ -174,12 +174,12 @@ def _fetch_dag_runs( query = query.where(DagRun.updated_at <= updated_at_lte) total_entries = get_query_count(query, session=session) - to_replace = {"dag_run_id": "run_id", "execution_date": "logical_date"} + to_replace = {"dag_run_id": "run_id", "logical_date": "logical_date"} allowed_sort_attrs = [ "id", "state", "dag_id", - "execution_date", + "logical_date", "dag_run_id", "start_date", "end_date", @@ -319,13 +319,13 @@ def post_dag_run(*, dag_id: str, session: Session = NEW_SESSION) -> APIResponse: except ValidationError as err: raise BadRequest(detail=str(err)) - logical_date = pendulum.instance(post_body["execution_date"]) + logical_date = pendulum.instance(post_body["logical_date"]) run_id = post_body["run_id"] dagrun_instance = session.scalar( select(DagRun) .where( DagRun.dag_id == dag_id, - or_(DagRun.run_id == run_id, DagRun.execution_date == logical_date), + or_(DagRun.run_id == run_id, DagRun.logical_date == logical_date), ) .limit(1) ) @@ -346,7 +346,7 @@ def post_dag_run(*, dag_id: str, session: Session = NEW_SESSION) -> APIResponse: dag_run = dag.create_dagrun( run_type=DagRunType.MANUAL, run_id=run_id, - execution_date=logical_date, + logical_date=logical_date, data_interval=data_interval, state=DagRunState.QUEUED, conf=post_body.get("conf"), @@ -363,7 +363,7 @@ def post_dag_run(*, dag_id: str, session: Session = NEW_SESSION) -> APIResponse: except (ValueError, ParamValidationError) as ve: raise BadRequest(detail=str(ve)) - if dagrun_instance.execution_date == logical_date: + if dagrun_instance.logical_date == logical_date: raise AlreadyExists( detail=( f"DAGRun with DAG ID: '{dag_id}' and " diff --git a/airflow/api_connexion/endpoints/task_instance_endpoint.py b/airflow/api_connexion/endpoints/task_instance_endpoint.py index 9f9461910082..68267317af3b 100644 --- a/airflow/api_connexion/endpoints/task_instance_endpoint.py +++ b/airflow/api_connexion/endpoints/task_instance_endpoint.py @@ -198,7 +198,7 @@ def get_mapped_task_instances( # Other search criteria base_query = _apply_range_filter( base_query, - key=DR.execution_date, + key=DR.logical_date, value_range=(execution_date_gte, execution_date_lte), ) base_query = _apply_range_filter( @@ -336,7 +336,7 @@ def get_task_instances( base_query = base_query.where(TI.run_id == dag_run_id) base_query = _apply_range_filter( base_query, - key=DR.execution_date, + key=DR.logical_date, value_range=(execution_date_gte, execution_date_lte), ) base_query = _apply_range_filter( @@ -398,7 +398,7 @@ def get_task_instances_batch(session: Session = NEW_SESSION) -> APIResponse: base_query = _apply_array_filter(base_query, key=TI.task_id, values=data["task_ids"]) base_query = _apply_range_filter( base_query, - key=DR.execution_date, + key=DR.logical_date, value_range=(data["execution_date_gte"], data["execution_date_lte"]), ) base_query = _apply_range_filter( @@ -522,22 +522,18 @@ def post_set_task_instances_state(*, dag_id: str, session: Session = NEW_SESSION error_message = f"Task ID {task_id} not found" raise NotFound(error_message) - execution_date = data.get("execution_date") + logical_date = data.get("logical_date") run_id = data.get("dag_run_id") if ( - execution_date + logical_date and ( session.scalars( - select(TI).where( - TI.task_id == task_id, TI.dag_id == dag_id, TI.execution_date == execution_date - ) + select(TI).where(TI.task_id == task_id, TI.dag_id == dag_id, TI.logical_date == logical_date) ).one_or_none() ) is None ): - raise NotFound( - detail=f"Task instance not found for task {task_id!r} on execution_date {execution_date}" - ) + raise NotFound(detail=f"Task instance not found for task {task_id!r} on logical_date {logical_date}") select_stmt = select(TI).where( TI.dag_id == dag_id, TI.task_id == task_id, TI.run_id == run_id, TI.map_index == -1 @@ -550,7 +546,7 @@ def post_set_task_instances_state(*, dag_id: str, session: Session = NEW_SESSION tis = dag.set_task_instance_state( task_id=task_id, run_id=run_id, - execution_date=execution_date, + logical_date=logical_date, state=data["new_state"], upstream=data["include_upstream"], downstream=data["include_downstream"], diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index af6a3055670d..10cee57d95fc 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3080,7 +3080,7 @@ components: The value of this field can be set only when creating the object. If you try to modify the field of an existing object, the request fails with an BAD_REQUEST error. - If not provided, a value will be generated based on execution_date. + If not provided, a value will be generated based on logical_date. If the specified dag_run_id is in use, the creation request fails with an ALREADY_EXISTS error. @@ -3102,19 +3102,6 @@ components: *New in version 2.2.0* format: date-time - execution_date: - type: string - nullable: true - description: | - The execution date. This is the same as logical_date, kept for backwards compatibility. - If both this field and logical_date are provided but with different values, the request - will fail with an BAD_REQUEST error. - - *Changed in version 2.2.0*: Field becomes nullable. - - *Deprecated since version 2.2.0*: Use 'logical_date' instead. - format: date-time - deprecated: true start_date: type: string format: date-time @@ -3677,7 +3664,7 @@ components: The DagRun ID for this task instance *New in version 2.3.0* - execution_date: + logical_date: type: string format: datetime start_date: @@ -3788,7 +3775,7 @@ components: type: string readOnly: true description: The DAG ID. - execution_date: + logical_date: type: string format: datetime readOnly: true @@ -3860,7 +3847,7 @@ components: timestamp: type: string format: datetime - execution_date: + logical_date: type: string format: datetime map_index: @@ -4734,14 +4721,14 @@ components: description: The task ID. type: string - execution_date: - description: The execution date. Either set this or dag_run_id but not both. + logical_date: + description: The logical date. Either set this or dag_run_id but not both. type: string format: datetime dag_run_id: description: | - The task instance's DAG run ID. Either set this or execution_date but not both. + The task instance's DAG run ID. Either set this or logical_date but not both. *New in version 2.3.0* type: string diff --git a/airflow/api_connexion/schemas/asset_schema.py b/airflow/api_connexion/schemas/asset_schema.py index 7f84b799d1a7..e83c4f1b4279 100644 --- a/airflow/api_connexion/schemas/asset_schema.py +++ b/airflow/api_connexion/schemas/asset_schema.py @@ -119,7 +119,7 @@ class Meta: run_id = auto_field(data_key="dag_run_id") dag_id = auto_field(dump_only=True) - execution_date = auto_field(data_key="logical_date", dump_only=True) + logical_date = auto_field(data_key="logical_date", dump_only=True) start_date = auto_field(dump_only=True) end_date = auto_field(dump_only=True) state = auto_field(dump_only=True) diff --git a/airflow/api_connexion/schemas/dag_run_schema.py b/airflow/api_connexion/schemas/dag_run_schema.py index e829dd956a4c..c2560613def7 100644 --- a/airflow/api_connexion/schemas/dag_run_schema.py +++ b/airflow/api_connexion/schemas/dag_run_schema.py @@ -63,7 +63,7 @@ class Meta: run_id = auto_field(data_key="dag_run_id") dag_id = auto_field(dump_only=True) - execution_date = auto_field(data_key="logical_date", validate=validate_istimezone) + logical_date = auto_field(data_key="logical_date", validate=validate_istimezone) start_date = auto_field(dump_only=True) end_date = auto_field(dump_only=True) state = DagStateField(dump_only=True) @@ -78,25 +78,12 @@ class Meta: @pre_load def autogenerate(self, data, **kwargs): - """ - Auto generate run_id and logical_date if they are not provided. - - For compatibility, if `execution_date` is submitted, it is converted - to `logical_date`. - """ + """Auto generate run_id and logical_date if they are not provided.""" logical_date = data.get("logical_date", _MISSING) - execution_date = data.pop("execution_date", _MISSING) - if logical_date is execution_date is _MISSING: # Both missing. + + # Auto-generate logical_date if missing + if logical_date is _MISSING: data["logical_date"] = str(timezone.utcnow()) - elif logical_date is _MISSING: # Only logical_date missing. - data["logical_date"] = execution_date - elif execution_date is _MISSING: # Only execution_date missing. - pass - elif logical_date != execution_date: # Both provided but don't match. - raise BadRequest( - "logical_date conflicts with execution_date", - detail=f"{logical_date!r} != {execution_date!r}", - ) if "dag_run_id" not in data: try: @@ -109,9 +96,8 @@ def autogenerate(self, data, **kwargs): @post_dump def autofill(self, data, **kwargs): - """Populate execution_date from logical_date for compatibility.""" + """Ensure that only requested fields are returned if 'fields' context is set.""" ret_data = {} - data["execution_date"] = data["logical_date"] if self.context.get("fields"): ret_fields = self.context.get("fields") for ret_field in ret_fields: diff --git a/airflow/api_connexion/schemas/event_log_schema.py b/airflow/api_connexion/schemas/event_log_schema.py index bf88d1a9ce88..33ecf4ca947a 100644 --- a/airflow/api_connexion/schemas/event_log_schema.py +++ b/airflow/api_connexion/schemas/event_log_schema.py @@ -40,7 +40,7 @@ class Meta: map_index = auto_field(dump_only=True) try_number = auto_field(dump_only=True) event = auto_field(dump_only=True) - execution_date = auto_field(dump_only=True) + logical_date = auto_field(dump_only=True) owner = auto_field(dump_only=True) extra = auto_field(dump_only=True) diff --git a/airflow/api_connexion/schemas/task_instance_schema.py b/airflow/api_connexion/schemas/task_instance_schema.py index f0b8285fdfa0..3e864f18652c 100644 --- a/airflow/api_connexion/schemas/task_instance_schema.py +++ b/airflow/api_connexion/schemas/task_instance_schema.py @@ -45,7 +45,7 @@ class Meta: dag_id = auto_field() run_id = auto_field(data_key="dag_run_id") map_index = auto_field() - execution_date = auto_field() + logical_date = auto_field() start_date = auto_field() end_date = auto_field() duration = auto_field() @@ -196,7 +196,7 @@ class SetTaskInstanceStateFormSchema(Schema): dry_run = fields.Boolean(load_default=True) task_id = fields.Str(required=True) - execution_date = fields.DateTime(validate=validate_istimezone) + logical_date = fields.DateTime(validate=validate_istimezone) dag_run_id = fields.Str() include_upstream = fields.Boolean(required=True) include_downstream = fields.Boolean(required=True) @@ -212,8 +212,8 @@ class SetTaskInstanceStateFormSchema(Schema): @validates_schema def validate_form(self, data, **kwargs): """Validate set task instance state form.""" - if not exactly_one(data.get("execution_date"), data.get("dag_run_id")): - raise ValidationError("Exactly one of execution_date or dag_run_id must be provided") + if not exactly_one(data.get("logical_date"), data.get("dag_run_id")): + raise ValidationError("Exactly one of logical_date or dag_run_id must be provided") class SetSingleTaskInstanceStateFormSchema(Schema): @@ -234,7 +234,7 @@ class TaskInstanceReferenceSchema(Schema): task_id = fields.Str() run_id = fields.Str(data_key="dag_run_id") dag_id = fields.Str() - execution_date = fields.DateTime() + logical_date = fields.DateTime() class TaskInstanceReferenceCollection(NamedTuple): diff --git a/airflow/api_connexion/schemas/xcom_schema.py b/airflow/api_connexion/schemas/xcom_schema.py index 625f05bd1459..a56adf7551d7 100644 --- a/airflow/api_connexion/schemas/xcom_schema.py +++ b/airflow/api_connexion/schemas/xcom_schema.py @@ -34,7 +34,7 @@ class Meta: key = auto_field() timestamp = auto_field() - execution_date = auto_field() + logical_date = auto_field() map_index = auto_field() task_id = auto_field() dag_id = auto_field() diff --git a/airflow/api_fastapi/core_api/datamodels/event_logs.py b/airflow/api_fastapi/core_api/datamodels/event_logs.py index f70e5bd15834..5b65ec85ba7b 100644 --- a/airflow/api_fastapi/core_api/datamodels/event_logs.py +++ b/airflow/api_fastapi/core_api/datamodels/event_logs.py @@ -33,7 +33,7 @@ class EventLogResponse(BaseModel): map_index: int | None try_number: int | None event: str - execution_date: datetime | None = Field(alias="logical_date") + logical_date: datetime | None owner: str | None extra: str | None diff --git a/airflow/api_fastapi/core_api/datamodels/task_instances.py b/airflow/api_fastapi/core_api/datamodels/task_instances.py index 00910bce1c8f..a54b85e58ddf 100644 --- a/airflow/api_fastapi/core_api/datamodels/task_instances.py +++ b/airflow/api_fastapi/core_api/datamodels/task_instances.py @@ -36,7 +36,7 @@ class TaskInstanceResponse(BaseModel): dag_id: str run_id: str = Field(alias="dag_run_id") map_index: int - execution_date: datetime = Field(alias="logical_date") + logical_date: datetime start_date: datetime | None end_date: datetime | None duration: float | None diff --git a/airflow/api_fastapi/core_api/datamodels/xcom.py b/airflow/api_fastapi/core_api/datamodels/xcom.py index dadbc51290e0..186b5aad77f0 100644 --- a/airflow/api_fastapi/core_api/datamodels/xcom.py +++ b/airflow/api_fastapi/core_api/datamodels/xcom.py @@ -27,7 +27,7 @@ class XComResponse(BaseModel): key: str timestamp: datetime - execution_date: datetime + logical_date: datetime map_index: int task_id: str dag_id: str diff --git a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml index 890fb6b6c8d0..741b6d360532 100644 --- a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml +++ b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml @@ -6119,10 +6119,10 @@ components: type: string format: date-time title: Timestamp - execution_date: + logical_date: type: string format: date-time - title: Execution Date + title: Logical Date map_index: type: integer title: Map Index @@ -6138,7 +6138,7 @@ components: required: - key - timestamp - - execution_date + - logical_date - map_index - task_id - dag_id @@ -6154,10 +6154,10 @@ components: type: string format: date-time title: Timestamp - execution_date: + logical_date: type: string format: date-time - title: Execution Date + title: Logical Date map_index: type: integer title: Map Index @@ -6176,7 +6176,7 @@ components: required: - key - timestamp - - execution_date + - logical_date - map_index - task_id - dag_id diff --git a/airflow/api_fastapi/core_api/routes/public/event_logs.py b/airflow/api_fastapi/core_api/routes/public/event_logs.py index 4047bc5a2749..6405c177a390 100644 --- a/airflow/api_fastapi/core_api/routes/public/event_logs.py +++ b/airflow/api_fastapi/core_api/routes/public/event_logs.py @@ -79,7 +79,7 @@ def get_event_logs( "task_id", "run_id", "event", - "execution_date", # logical_date + "logical_date", "owner", "extra", ], diff --git a/airflow/api_fastapi/core_api/routes/public/task_instances.py b/airflow/api_fastapi/core_api/routes/public/task_instances.py index 56fa57d3ebeb..516e113fc24f 100644 --- a/airflow/api_fastapi/core_api/routes/public/task_instances.py +++ b/airflow/api_fastapi/core_api/routes/public/task_instances.py @@ -95,9 +95,7 @@ def get_mapped_task_instances( dag_run_id: str, task_id: str, request: Request, - logical_date_range: Annotated[ - RangeFilter, Depends(datetime_range_filter_factory("logical_date", TI, "execution_date")) - ], + logical_date_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("logical_date", TI))], start_date_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("start_date", TI))], end_date_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("end_date", TI))], update_at_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("updated_at", TI))], @@ -270,9 +268,7 @@ def get_task_instances( dag_id: str, dag_run_id: str, request: Request, - logical_date: Annotated[ - RangeFilter, Depends(datetime_range_filter_factory("logical_date", TI, "execution_date")) - ], + logical_date: Annotated[RangeFilter, Depends(datetime_range_filter_factory("logical_date", TI))], start_date_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("start_date", TI))], end_date_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("end_date", TI))], update_at_range: Annotated[RangeFilter, Depends(datetime_range_filter_factory("updated_at", TI))], diff --git a/airflow/api_fastapi/core_api/routes/ui/assets.py b/airflow/api_fastapi/core_api/routes/ui/assets.py index 1615f97c2bd2..ca2aae1bb6c0 100644 --- a/airflow/api_fastapi/core_api/routes/ui/assets.py +++ b/airflow/api_fastapi/core_api/routes/ui/assets.py @@ -71,8 +71,8 @@ def next_run_assets( and_( AssetEvent.asset_id == AssetModel.id, ( - AssetEvent.timestamp >= latest_run.execution_date - if latest_run and latest_run.execution_date + AssetEvent.timestamp >= latest_run.logical_date + if latest_run and latest_run.logical_date else True ), ), diff --git a/airflow/api_fastapi/core_api/routes/ui/dags.py b/airflow/api_fastapi/core_api/routes/ui/dags.py index da906bbbd3c9..fad736ced379 100644 --- a/airflow/api_fastapi/core_api/routes/ui/dags.py +++ b/airflow/api_fastapi/core_api/routes/ui/dags.py @@ -68,39 +68,39 @@ def recent_dag_runs( recent_runs_subquery = ( select( DagRun.dag_id, - DagRun.execution_date, + DagRun.logical_date, func.rank() .over( partition_by=DagRun.dag_id, - order_by=DagRun.execution_date.desc(), + order_by=DagRun.logical_date.desc(), ) .label("rank"), ) - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .subquery() ) dags_with_recent_dag_runs_select = ( select( DagRun, DagModel, - recent_runs_subquery.c.execution_date, + recent_runs_subquery.c.logical_date, ) .join(DagModel, DagModel.dag_id == recent_runs_subquery.c.dag_id) .join( DagRun, and_( DagRun.dag_id == DagModel.dag_id, - DagRun.execution_date == recent_runs_subquery.c.execution_date, + DagRun.logical_date == recent_runs_subquery.c.logical_date, ), ) .where(recent_runs_subquery.c.rank <= dag_runs_limit) .group_by( DagModel.dag_id, - recent_runs_subquery.c.execution_date, - DagRun.execution_date, + recent_runs_subquery.c.logical_date, + DagRun.logical_date, DagRun.id, ) - .order_by(recent_runs_subquery.c.execution_date.desc()) + .order_by(recent_runs_subquery.c.logical_date.desc()) ) dags_with_recent_dag_runs_select_filter, _ = paginated_select( dags_with_recent_dag_runs_select, diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index e93d5e25c631..00dd68041cbd 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -144,17 +144,17 @@ def string_lower_type(val): # Shared ARG_DAG_ID = Arg(("dag_id",), help="The id of the dag") ARG_TASK_ID = Arg(("task_id",), help="The id of the task") -ARG_EXECUTION_DATE = Arg(("execution_date",), help="The execution date of the DAG", type=parsedate) -ARG_EXECUTION_DATE_OPTIONAL = Arg( - ("execution_date",), nargs="?", help="The execution date of the DAG (optional)", type=parsedate +ARG_LOGICAL_DATE = Arg(("logical_date",), help="The logical date of the DAG", type=parsedate) +ARG_LOGICAL_DATE_OPTIONAL = Arg( + ("logical_date",), nargs="?", help="The logical date of the DAG (optional)", type=parsedate ) -ARG_EXECUTION_DATE_OR_RUN_ID = Arg( - ("execution_date_or_run_id",), help="The execution_date of the DAG or run_id of the DAGRun" +ARG_LOGICAL_DATE_OR_RUN_ID = Arg( + ("logical_date_or_run_id",), help="The logical date of the DAG or run_id of the DAGRun" ) -ARG_EXECUTION_DATE_OR_RUN_ID_OPTIONAL = Arg( - ("execution_date_or_run_id",), +ARG_LOGICAL_DATE_OR_RUN_ID_OPTIONAL = Arg( + ("logical_date_or_run_id",), nargs="?", - help="The execution_date of the DAG or run_id of the DAGRun (optional)", + help="The logical date of the DAG or run_id of the DAGRun (optional)", ) ARG_TASK_REGEX = Arg(("-t", "--task-regex"), help="The regex to filter specific task_ids (optional)") ARG_SUBDIR = Arg( @@ -261,7 +261,7 @@ def string_lower_type(val): ("-n", "--num-executions"), default=1, type=positive_int(allow_zero=False), - help="The number of next execution datetimes to show", + help="The number of next logical date times to show", ) # misc @@ -410,7 +410,7 @@ def string_lower_type(val): # trigger_dag ARG_RUN_ID = Arg(("-r", "--run-id"), help="Helps to identify this run") ARG_CONF = Arg(("-c", "--conf"), help="JSON string that gets pickled into the DagRun's conf attribute") -ARG_EXEC_DATE = Arg(("-e", "--exec-date"), help="The execution date of the DAG", type=parsedate) +ARG_EXEC_DATE = Arg(("-e", "--exec-date"), help="The logical date of the DAG", type=parsedate) ARG_REPLACE_MICRO = Arg( ("--no-replace-microseconds",), help="whether microseconds should be zeroed", @@ -1089,13 +1089,13 @@ class GroupCommand(NamedTuple): name="state", help="Get the status of a dag run", func=lazy_load_command("airflow.cli.commands.dag_command.dag_state"), - args=(ARG_DAG_ID, ARG_EXECUTION_DATE, ARG_SUBDIR, ARG_VERBOSE), + args=(ARG_DAG_ID, ARG_LOGICAL_DATE, ARG_SUBDIR, ARG_VERBOSE), ), ActionCommand( name="next-execution", - help="Get the next execution datetimes of a DAG", + help="Get the next logical datetimes of a DAG", description=( - "Get the next execution datetimes of a DAG. It returns one execution unless the " + "Get the next logical datetimes of a DAG. It returns one execution unless the " "num-executions option is given" ), func=lazy_load_command("airflow.cli.commands.dag_command.dag_next_execution"), @@ -1208,7 +1208,7 @@ class GroupCommand(NamedTuple): name="test", help="Execute one single DagRun", description=( - "Execute one single DagRun for a given DAG and execution date.\n" + "Execute one single DagRun for a given DAG and logical date.\n" "\n" "The --imgcat-dagrun option only works in iTerm.\n" "\n" @@ -1221,15 +1221,15 @@ class GroupCommand(NamedTuple): "see: https://www.graphviz.org/doc/info/output.html\n" "\n" "If you want to create a PNG file then you should execute the following command:\n" - "airflow dags test --save-dagrun output.png\n" + "airflow dags test --save-dagrun output.png\n" "\n" "If you want to create a DOT file then you should execute the following command:\n" - "airflow dags test --save-dagrun output.dot\n" + "airflow dags test --save-dagrun output.dot\n" ), func=lazy_load_command("airflow.cli.commands.dag_command.dag_test"), args=( ARG_DAG_ID, - ARG_EXECUTION_DATE_OPTIONAL, + ARG_LOGICAL_DATE_OPTIONAL, ARG_CONF, ARG_SUBDIR, ARG_SHOW_DAGRUN, @@ -1289,7 +1289,7 @@ class GroupCommand(NamedTuple): args=( ARG_DAG_ID, ARG_TASK_ID, - ARG_EXECUTION_DATE_OR_RUN_ID, + ARG_LOGICAL_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_VERBOSE, ARG_MAP_INDEX, @@ -1304,7 +1304,7 @@ class GroupCommand(NamedTuple): "and then run by an executor." ), func=lazy_load_command("airflow.cli.commands.task_command.task_failed_deps"), - args=(ARG_DAG_ID, ARG_TASK_ID, ARG_EXECUTION_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_MAP_INDEX, ARG_VERBOSE), + args=(ARG_DAG_ID, ARG_TASK_ID, ARG_LOGICAL_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_MAP_INDEX, ARG_VERBOSE), ), ActionCommand( name="render", @@ -1313,7 +1313,7 @@ class GroupCommand(NamedTuple): args=( ARG_DAG_ID, ARG_TASK_ID, - ARG_EXECUTION_DATE_OR_RUN_ID, + ARG_LOGICAL_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_VERBOSE, ARG_MAP_INDEX, @@ -1326,7 +1326,7 @@ class GroupCommand(NamedTuple): args=( ARG_DAG_ID, ARG_TASK_ID, - ARG_EXECUTION_DATE_OR_RUN_ID, + ARG_LOGICAL_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_MARK_SUCCESS, ARG_FORCE, @@ -1355,7 +1355,7 @@ class GroupCommand(NamedTuple): args=( ARG_DAG_ID, ARG_TASK_ID, - ARG_EXECUTION_DATE_OR_RUN_ID_OPTIONAL, + ARG_LOGICAL_DATE_OR_RUN_ID_OPTIONAL, ARG_SUBDIR, ARG_DRY_RUN, ARG_TASK_PARAMS, @@ -1369,7 +1369,7 @@ class GroupCommand(NamedTuple): name="states-for-dag-run", help="Get the status of all task instances in a dag run", func=lazy_load_command("airflow.cli.commands.task_command.task_states_for_dag_run"), - args=(ARG_DAG_ID, ARG_EXECUTION_DATE_OR_RUN_ID, ARG_OUTPUT, ARG_VERBOSE), + args=(ARG_DAG_ID, ARG_LOGICAL_DATE_OR_RUN_ID, ARG_OUTPUT, ARG_VERBOSE), ), ) POOLS_COMMANDS = ( @@ -1817,7 +1817,7 @@ class GroupCommand(NamedTuple): help="Generate YAML files for all tasks in DAG. Useful for debugging tasks without " "launching into a cluster", func=lazy_load_command("airflow.providers.cncf.kubernetes.cli.kubernetes_command.generate_pod_yaml"), - args=(ARG_DAG_ID, ARG_EXECUTION_DATE, ARG_SUBDIR, ARG_OUTPUT_PATH, ARG_VERBOSE), + args=(ARG_DAG_ID, ARG_LOGICAL_DATE, ARG_SUBDIR, ARG_OUTPUT_PATH, ARG_VERBOSE), ), ) diff --git a/airflow/cli/commands/dag_command.py b/airflow/cli/commands/dag_command.py index dfff75ee2d6c..0b4d3192d6ed 100644 --- a/airflow/cli/commands/dag_command.py +++ b/airflow/cli/commands/dag_command.py @@ -65,7 +65,7 @@ def dag_trigger(args) -> None: dag_id=args.dag_id, run_id=args.run_id, conf=args.conf, - execution_date=args.exec_date, + logical_date=args.exec_date, replace_microseconds=args.replace_microseconds, ) AirflowConsole().print_as( @@ -264,7 +264,7 @@ def dag_state(args, session: Session = NEW_SESSION) -> None: if not dag: raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table") - dr = session.scalar(select(DagRun).filter_by(dag_id=args.dag_id, execution_date=args.execution_date)) + dr = session.scalar(select(DagRun).filter_by(dag_id=args.dag_id, logical_date=args.logical_date)) out = dr.state if dr else None conf_out = "" if out and dr.conf: @@ -276,7 +276,7 @@ def dag_state(args, session: Session = NEW_SESSION) -> None: @providers_configuration_loaded def dag_next_execution(args) -> None: """ - Return the next execution datetime of a DAG at the command line. + Return the next logical datetime of a DAG at the command line. >>> airflow dags next-execution tutorial 2018-08-31 10:38:00 @@ -461,12 +461,12 @@ def dag_list_dag_runs(args, dag: DAG | None = None, session: Session = NEW_SESSI dag_id=args.dag_id, state=state, no_backfills=args.no_backfill, - execution_start_date=args.start_date, - execution_end_date=args.end_date, + logical_start_date=args.start_date, + logical_end_date=args.end_date, session=session, ) - dag_runs.sort(key=lambda x: x.execution_date, reverse=True) + dag_runs.sort(key=lambda x: x.logical_date, reverse=True) AirflowConsole().print_as( data=dag_runs, output=args.output, @@ -474,7 +474,7 @@ def dag_list_dag_runs(args, dag: DAG | None = None, session: Session = NEW_SESSI "dag_id": dr.dag_id, "run_id": dr.run_id, "state": dr.state, - "execution_date": dr.execution_date.isoformat(), + "logical_date": dr.logical_date.isoformat(), "start_date": dr.start_date.isoformat() if dr.start_date else "", "end_date": dr.end_date.isoformat() if dr.end_date else "", }, @@ -485,14 +485,14 @@ def dag_list_dag_runs(args, dag: DAG | None = None, session: Session = NEW_SESSI @providers_configuration_loaded @provide_session def dag_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> None: - """Execute one single DagRun for a given DAG and execution date.""" + """Execute one single DagRun for a given DAG and logical date.""" run_conf = None if args.conf: try: run_conf = json.loads(args.conf) except ValueError as e: raise SystemExit(f"Configuration {args.conf!r} is not valid JSON. Error: {e}") - execution_date = args.execution_date or timezone.utcnow() + logical_date = args.logical_date or timezone.utcnow() use_executor = args.use_executor mark_success_pattern = ( @@ -502,7 +502,7 @@ def dag_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> No with _airflow_parsing_context_manager(dag_id=args.dag_id): dag = dag or get_dag(subdir=args.subdir, dag_id=args.dag_id) dr: DagRun = dag.test( - execution_date=execution_date, + logical_date=logical_date, run_conf=run_conf, use_executor=use_executor, mark_success_pattern=mark_success_pattern, @@ -515,7 +515,7 @@ def dag_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> No tis = session.scalars( select(TaskInstance).where( TaskInstance.dag_id == args.dag_id, - TaskInstance.execution_date == execution_date, + TaskInstance.logical_date == logical_date, ) ).all() diff --git a/airflow/cli/commands/kubernetes_command.py b/airflow/cli/commands/kubernetes_command.py index eab11133c9dd..8f5f7333b7f4 100644 --- a/airflow/cli/commands/kubernetes_command.py +++ b/airflow/cli/commands/kubernetes_command.py @@ -48,10 +48,10 @@ @providers_configuration_loaded def generate_pod_yaml(args): """Generate yaml files for each task in the DAG. Used for testing output of KubernetesExecutor.""" - execution_date = args.execution_date + logical_date = args.logical_date dag = get_dag(subdir=args.subdir, dag_id=args.dag_id) yaml_output_path = args.output_path - dr = DagRun(dag.dag_id, execution_date=execution_date) + dr = DagRun(dag.dag_id, logical_date=logical_date) kube_config = KubeConfig() for task in dag.tasks: ti = TaskInstance(task, None) @@ -62,7 +62,7 @@ def generate_pod_yaml(args): pod_id=create_unique_id(args.dag_id, ti.task_id), try_number=ti.try_number, kube_image=kube_config.kube_image, - date=ti.execution_date, + date=ti.logical_date, args=ti.command_as_list(), pod_override_object=PodGenerator.from_obj(ti.executor_config), scheduler_job_id="worker-config", @@ -71,7 +71,7 @@ def generate_pod_yaml(args): with_mutation_hook=True, ) api_client = ApiClient() - date_string = pod_generator.datetime_to_label_safe_datestring(execution_date) + date_string = pod_generator.datetime_to_label_safe_datestring(logical_date) yaml_file_name = f"{args.dag_id}_{ti.task_id}_{date_string}.yml" os.makedirs(os.path.dirname(yaml_output_path + "/airflow_yaml_output/"), exist_ok=True) with open(yaml_output_path + "/airflow_yaml_output/" + yaml_file_name, "w") as output: diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index e14c18399555..396186bf14d2 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -114,43 +114,43 @@ def _get_dag_run( """ if not exec_date_or_run_id and not create_if_necessary: raise ValueError("Must provide `exec_date_or_run_id` if not `create_if_necessary`.") - execution_date: pendulum.DateTime | None = None + logical_date: pendulum.DateTime | None = None if exec_date_or_run_id: dag_run = DAG.fetch_dagrun(dag_id=dag.dag_id, run_id=exec_date_or_run_id, session=session) if dag_run: return dag_run, False with suppress(ParserError, TypeError): - execution_date = timezone.parse(exec_date_or_run_id) - if execution_date: - dag_run = DAG.fetch_dagrun(dag_id=dag.dag_id, execution_date=execution_date, session=session) + logical_date = timezone.parse(exec_date_or_run_id) + if logical_date: + dag_run = DAG.fetch_dagrun(dag_id=dag.dag_id, logical_date=logical_date, session=session) if dag_run: return dag_run, False elif not create_if_necessary: raise DagRunNotFound( - f"DagRun for {dag.dag_id} with run_id or execution_date " + f"DagRun for {dag.dag_id} with run_id or logical_date " f"of {exec_date_or_run_id!r} not found" ) - if execution_date is not None: - dag_run_execution_date = execution_date + if logical_date is not None: + dag_run_logical_date = logical_date else: - dag_run_execution_date = pendulum.instance(timezone.utcnow()) + dag_run_logical_date = pendulum.instance(timezone.utcnow()) if create_if_necessary == "memory": dag_run = DagRun( dag_id=dag.dag_id, run_id=exec_date_or_run_id, - execution_date=dag_run_execution_date, - data_interval=dag.timetable.infer_manual_data_interval(run_after=dag_run_execution_date), + logical_date=dag_run_logical_date, + data_interval=dag.timetable.infer_manual_data_interval(run_after=dag_run_logical_date), triggered_by=DagRunTriggeredByType.CLI, ) return dag_run, True elif create_if_necessary == "db": dag_run = dag.create_dagrun( state=DagRunState.QUEUED, - execution_date=dag_run_execution_date, + logical_date=dag_run_logical_date, run_id=_generate_temporary_run_id(), - data_interval=dag.timetable.infer_manual_data_interval(run_after=dag_run_execution_date), + data_interval=dag.timetable.infer_manual_data_interval(run_after=dag_run_logical_date), session=session, triggered_by=DagRunTriggeredByType.CLI, ) @@ -165,7 +165,7 @@ def _get_ti_db_access( task: Operator, map_index: int, *, - exec_date_or_run_id: str | None = None, + logical_date_or_run_id: str | None = None, pool: str | None = None, create_if_necessary: CreateIfNecessary = False, session: Session = NEW_SESSION, @@ -177,7 +177,7 @@ def _get_ti_db_access( if task.task_id not in dag.task_dict: raise ValueError(f"Provided task {task.task_id} is not in dag '{dag.dag_id}.") - if not exec_date_or_run_id and not create_if_necessary: + if not logical_date_or_run_id and not create_if_necessary: raise ValueError("Must provide `exec_date_or_run_id` if not `create_if_necessary`.") if task.get_needs_expansion(): if map_index < 0: @@ -186,7 +186,7 @@ def _get_ti_db_access( raise RuntimeError("map_index passed to non-mapped task") dag_run, dr_created = _get_dag_run( dag=dag, - exec_date_or_run_id=exec_date_or_run_id, + exec_date_or_run_id=logical_date_or_run_id, create_if_necessary=create_if_necessary, session=session, ) @@ -197,7 +197,7 @@ def _get_ti_db_access( if not create_if_necessary: raise TaskInstanceNotFound( f"TaskInstance for {dag.dag_id}, {task.task_id}, map={map_index} with " - f"run_id or execution_date of {exec_date_or_run_id!r} not found" + f"run_id or logical_date of {logical_date_or_run_id!r} not found" ) # TODO: Validate map_index is in range? ti = TaskInstance(task, run_id=dag_run.run_id, map_index=map_index) @@ -214,7 +214,7 @@ def _get_ti( task: Operator, map_index: int, *, - exec_date_or_run_id: str | None = None, + logical_date_or_run_id: str | None = None, pool: str | None = None, create_if_necessary: CreateIfNecessary = False, ): @@ -226,7 +226,7 @@ def _get_ti( dag=dag, task=task, map_index=map_index, - exec_date_or_run_id=exec_date_or_run_id, + logical_date_or_run_id=logical_date_or_run_id, pool=pool, create_if_necessary=create_if_necessary, ) @@ -438,7 +438,7 @@ def task_run(args, dag: DAG | None = None) -> TaskReturnCode | None: else: _dag = dag task = _dag.get_task(task_id=args.task_id) - ti, _ = _get_ti(task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id, pool=args.pool) + ti, _ = _get_ti(task, args.map_index, logical_date_or_run_id=args.logical_date_or_run_id, pool=args.pool) ti.init_run_context(raw=args.raw) hostname = get_hostname() @@ -487,7 +487,7 @@ def task_failed_deps(args) -> None: """ dag = get_dag(args.subdir, args.dag_id) task = dag.get_task(task_id=args.task_id) - ti, _ = _get_ti(task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id) + ti, _ = _get_ti(task, args.map_index, logical_date_or_run_id=args.logical_date_or_run_id) # tasks_failed-deps is executed with access to the database. if isinstance(ti, TaskInstancePydantic): raise ValueError("not a TaskInstance") @@ -514,7 +514,7 @@ def task_state(args) -> None: """ dag = get_dag(args.subdir, args.dag_id) task = dag.get_task(task_id=args.task_id) - ti, _ = _get_ti(task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id) + ti, _ = _get_ti(task, args.map_index, logical_date_or_run_id=args.logical_date_or_run_id) # task_state is executed with access to the database. if isinstance(ti, TaskInstancePydantic): raise ValueError("not a TaskInstance") @@ -572,20 +572,20 @@ def _guess_debugger() -> _SupportedDebugger: def task_states_for_dag_run(args, session: Session = NEW_SESSION) -> None: """Get the status of all task instances in a DagRun.""" dag_run = session.scalar( - select(DagRun).where(DagRun.run_id == args.execution_date_or_run_id, DagRun.dag_id == args.dag_id) + select(DagRun).where(DagRun.run_id == args.logical_date_or_run_id, DagRun.dag_id == args.dag_id) ) if not dag_run: try: - execution_date = timezone.parse(args.execution_date_or_run_id) + logical_date = timezone.parse(args.logical_date_or_run_id) dag_run = session.scalar( - select(DagRun).where(DagRun.execution_date == execution_date, DagRun.dag_id == args.dag_id) + select(DagRun).where(DagRun.logical_date == logical_date, DagRun.dag_id == args.dag_id) ) except (ParserError, TypeError) as err: - raise AirflowException(f"Error parsing the supplied execution_date. Error: {err}") + raise AirflowException(f"Error parsing the supplied logical_date. Error: {err}") if dag_run is None: raise DagRunNotFound( - f"DagRun for {args.dag_id} with run_id or execution_date of {args.execution_date_or_run_id!r} " + f"DagRun for {args.dag_id} with run_id or logical_date of {args.logical_date_or_run_id!r} " "not found" ) @@ -594,7 +594,7 @@ def task_states_for_dag_run(args, session: Session = NEW_SESSION) -> None: def format_task_instance(ti: TaskInstance) -> dict[str, str]: data = { "dag_id": ti.dag_id, - "execution_date": dag_run.execution_date.isoformat(), + "logical_date": dag_run.logical_date.isoformat(), "task_id": ti.task_id, "state": ti.state, "start_date": ti.start_date.isoformat() if ti.start_date else "", @@ -643,7 +643,7 @@ def task_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> N task.params.validate() ti, dr_created = _get_ti( - task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id, create_if_necessary="db" + task, args.map_index, logical_date_or_run_id=args.logical_date_or_run_id, create_if_necessary="db" ) # task_test is executed with access to the database. if isinstance(ti, TaskInstancePydantic): @@ -694,7 +694,7 @@ def task_render(args, dag: DAG | None = None) -> None: dag = get_dag(args.subdir, args.dag_id) task = dag.get_task(task_id=args.task_id) ti, _ = _get_ti( - task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id, create_if_necessary="memory" + task, args.map_index, logical_date_or_run_id=args.logical_date_or_run_id, create_if_necessary="memory" ) # task_render is executed with access to the database. if isinstance(ti, TaskInstancePydantic): diff --git a/airflow/configuration.py b/airflow/configuration.py index 82718325865f..cd218657133a 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -376,7 +376,7 @@ def inversed_deprecated_sections(self): }, "elasticsearch": { "log_id_template": ( - re2.compile("^" + re2.escape("{dag_id}-{task_id}-{execution_date}-{try_number}") + "$"), + re2.compile("^" + re2.escape("{dag_id}-{task_id}-{logical_date}-{try_number}") + "$"), "{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}", "3.0", ) diff --git a/airflow/dag_processing/collection.py b/airflow/dag_processing/collection.py index f608900ee76e..0ca121c56185 100644 --- a/airflow/dag_processing/collection.py +++ b/airflow/dag_processing/collection.py @@ -89,7 +89,7 @@ def _get_latest_runs_stmt(dag_ids: Collection[str]) -> Select: if len(dag_ids) == 1: # Index optimized fast path to avoid more complicated & slower groupby queryplan. (dag_id,) = dag_ids last_automated_runs_subq = ( - select(func.max(DagRun.execution_date).label("max_execution_date")) + select(func.max(DagRun.logical_date).label("max_execution_date")) .where( DagRun.dag_id == dag_id, DagRun.run_type.in_((DagRunType.BACKFILL_JOB, DagRunType.SCHEDULED)), @@ -98,11 +98,11 @@ def _get_latest_runs_stmt(dag_ids: Collection[str]) -> Select: ) query = select(DagRun).where( DagRun.dag_id == dag_id, - DagRun.execution_date == last_automated_runs_subq, + DagRun.logical_date == last_automated_runs_subq, ) else: last_automated_runs_subq = ( - select(DagRun.dag_id, func.max(DagRun.execution_date).label("max_execution_date")) + select(DagRun.dag_id, func.max(DagRun.logical_date).label("max_execution_date")) .where( DagRun.dag_id.in_(dag_ids), DagRun.run_type.in_((DagRunType.BACKFILL_JOB, DagRunType.SCHEDULED)), @@ -112,12 +112,12 @@ def _get_latest_runs_stmt(dag_ids: Collection[str]) -> Select: ) query = select(DagRun).where( DagRun.dag_id == last_automated_runs_subq.c.dag_id, - DagRun.execution_date == last_automated_runs_subq.c.max_execution_date, + DagRun.logical_date == last_automated_runs_subq.c.max_execution_date, ) return query.options( load_only( DagRun.dag_id, - DagRun.execution_date, + DagRun.logical_date, DagRun.data_interval_start, DagRun.data_interval_end, ) diff --git a/airflow/dag_processing/processor.py b/airflow/dag_processing/processor.py index 394e09245127..219c7aa9776a 100644 --- a/airflow/dag_processing/processor.py +++ b/airflow/dag_processing/processor.py @@ -608,7 +608,7 @@ def _execute_dag_callbacks(cls, dagbag: DagBag, request: DagCallbackRequest, ses dag = dagbag.dags[request.dag_id] callbacks, context = DAG.fetch_callback( dag=dag, - dag_run_id=request.run_id, + run_id=request.run_id, success=not request.is_failure_callback, reason=request.msg, session=session, diff --git a/airflow/example_dags/example_branch_python_dop_operator_3.py b/airflow/example_dags/example_branch_python_dop_operator_3.py index a7031961ab55..4bb940ca36dd 100644 --- a/airflow/example_dags/example_branch_python_dop_operator_3.py +++ b/airflow/example_dags/example_branch_python_dop_operator_3.py @@ -32,15 +32,13 @@ @task.branch() def should_run(**kwargs) -> str: """ - Determine which empty_task should be run based on if the execution date minute is even or odd. + Determine which empty_task should be run based on if the logical date minute is even or odd. :param dict kwargs: Context :return: Id of the task to run """ - print( - f"------------- exec dttm = {kwargs['execution_date']} and minute = {kwargs['execution_date'].minute}" - ) - if kwargs["execution_date"].minute % 2 == 0: + print(f"------------- exec dttm = {kwargs['logical_date']} and minute = {kwargs['logical_date'].minute}") + if kwargs["logical_date"].minute % 2 == 0: return "empty_task_1" else: return "empty_task_2" diff --git a/airflow/example_dags/tutorial_objectstorage.py b/airflow/example_dags/tutorial_objectstorage.py index 4660aa3c8e8c..d03ec6ec8003 100644 --- a/airflow/example_dags/tutorial_objectstorage.py +++ b/airflow/example_dags/tutorial_objectstorage.py @@ -72,7 +72,7 @@ def get_air_quality_data(**kwargs) -> ObjectStoragePath: """ import pandas as pd - execution_date = kwargs["logical_date"] + logical_date = kwargs["logical_date"] start_time = kwargs["data_interval_start"] params = { @@ -83,7 +83,7 @@ def get_air_quality_data(**kwargs) -> ObjectStoragePath: "area": "Uusimaa", "param": ",".join(aq_fields.keys()), "starttime": start_time.isoformat(timespec="seconds"), - "endtime": execution_date.isoformat(timespec="seconds"), + "endtime": logical_date.isoformat(timespec="seconds"), "tz": "UTC", } @@ -93,7 +93,7 @@ def get_air_quality_data(**kwargs) -> ObjectStoragePath: # ensure the bucket exists base.mkdir(exist_ok=True) - formatted_date = execution_date.format("YYYYMMDD") + formatted_date = logical_date.format("YYYYMMDD") path = base / f"air_quality_{formatted_date}.parquet" df = pd.DataFrame(response.json()).astype(aq_fields) diff --git a/airflow/exceptions.py b/airflow/exceptions.py index 316fe880b66b..3b07b9a6fda9 100644 --- a/airflow/exceptions.py +++ b/airflow/exceptions.py @@ -230,12 +230,12 @@ class DagRunNotFound(AirflowNotFoundException): class DagRunAlreadyExists(AirflowBadRequest): """Raise when creating a DAG run for DAG which already has DAG run entry.""" - def __init__(self, dag_run: DagRun, execution_date: datetime.datetime, run_id: str) -> None: + def __init__(self, dag_run: DagRun, logical_date: datetime.datetime, run_id: str) -> None: super().__init__( - f"A DAG Run already exists for DAG {dag_run.dag_id} at {execution_date} with run id {run_id}" + f"A DAG Run already exists for DAG {dag_run.dag_id} at {logical_date} with run id {run_id}" ) self.dag_run = dag_run - self.execution_date = execution_date + self.logical_date = logical_date self.run_id = run_id def serialize(self): @@ -249,13 +249,13 @@ def serialize(self): run_id=self.dag_run.run_id, external_trigger=self.dag_run.external_trigger, run_type=self.dag_run.run_type, - execution_date=self.dag_run.execution_date, + logical_date=self.dag_run.logical_date, ) dag_run.id = self.dag_run.id return ( f"{cls.__module__}.{cls.__name__}", (), - {"dag_run": dag_run, "execution_date": self.execution_date, "run_id": self.run_id}, + {"dag_run": dag_run, "logical_date": self.logical_date, "run_id": self.run_id}, ) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 5369f425c9c4..6c7887c643cb 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -340,7 +340,7 @@ def _executable_task_instances_to_queued(self, max_tis: int, session: Session) - .where(not_(DM.is_paused)) .where(TI.state == TaskInstanceState.SCHEDULED) .options(selectinload(TI.dag_model)) - .order_by(-TI.priority_weight, DR.execution_date, TI.map_index) + .order_by(-TI.priority_weight, DR.logical_date, TI.map_index) ) if starved_pools: @@ -735,7 +735,7 @@ def process_executor_events( # Report execution for ti_key, (state, _) in event_buffer.items(): - # We create map (dag_id, task_id, execution_date) -> in-memory try_number + # We create map (dag_id, task_id, logical_date) -> in-memory try_number ti_primary_key_to_try_number_map[ti_key.primary] = ti_key.try_number cls.logger().info("Received executor event with state %s for task instance %s", state, ti_key) @@ -871,7 +871,7 @@ def _set_span_attrs__process_executor_events(cls, span, state, ti): "end_date": str(ti.end_date), "duration": ti.duration, "executor_config": str(ti.executor_config), - "execution_date": str(ti.execution_date), + "logical_date": str(ti.logical_date), "hostname": ti.hostname, "log_url": ti.log_url, "operator": str(ti.operator), @@ -1288,15 +1288,15 @@ def _mark_backfills_complete(self, session: Session = NEW_SESSION) -> None: @add_span def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) -> None: """Create a DAG run and update the dag_model to control if/when the next DAGRun should be created.""" - # Bulk Fetch DagRuns with dag_id and execution_date same + # Bulk Fetch DagRuns with dag_id and logical_date same # as DagModel.dag_id and DagModel.next_dagrun # This list is used to verify if the DagRun already exist so that we don't attempt to create # duplicate dag runs existing_dagruns = ( session.execute( - select(DagRun.dag_id, DagRun.execution_date).where( + select(DagRun.dag_id, DagRun.logical_date).where( tuple_in_condition( - (DagRun.dag_id, DagRun.execution_date), + (DagRun.dag_id, DagRun.logical_date), ((dm.dag_id, dm.next_dagrun) for dm in dag_models), ), ) @@ -1337,7 +1337,7 @@ def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) - try: dag.create_dagrun( run_type=DagRunType.SCHEDULED, - execution_date=dag_model.next_dagrun, + logical_date=dag_model.next_dagrun, state=DagRunState.QUEUED, data_interval=data_interval, external_trigger=False, @@ -1372,18 +1372,18 @@ def _create_dag_runs_asset_triggered( session: Session, ) -> None: """For DAGs that are triggered by assets, create dag runs.""" - # Bulk Fetch DagRuns with dag_id and execution_date same + # Bulk Fetch DagRuns with dag_id and logical_date same # as DagModel.dag_id and DagModel.next_dagrun # This list is used to verify if the DagRun already exist so that we don't attempt to create # duplicate dag runs - exec_dates = { + logical_dates = { dag_id: timezone.coerce_datetime(last_time) for dag_id, (_, last_time) in asset_triggered_dag_info.items() } existing_dagruns: set[tuple[str, timezone.DateTime]] = set( session.execute( - select(DagRun.dag_id, DagRun.execution_date).where( - tuple_in_condition((DagRun.dag_id, DagRun.execution_date), exec_dates.items()) + select(DagRun.dag_id, DagRun.logical_date).where( + tuple_in_condition((DagRun.dag_id, DagRun.logical_date), logical_dates.items()) ) ) ) @@ -1411,24 +1411,24 @@ def _create_dag_runs_asset_triggered( # we need to set dag.next_dagrun_info if the Dag Run already exists or if we # create a new one. This is so that in the next Scheduling loop we try to create new runs # instead of falling in a loop of Integrity Error. - exec_date = exec_dates[dag.dag_id] - if (dag.dag_id, exec_date) not in existing_dagruns: + logical_date = logical_dates[dag.dag_id] + if (dag.dag_id, logical_date) not in existing_dagruns: previous_dag_run = session.scalar( select(DagRun) .where( DagRun.dag_id == dag.dag_id, - DagRun.execution_date < exec_date, + DagRun.logical_date < logical_date, DagRun.run_type == DagRunType.ASSET_TRIGGERED, ) - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .limit(1) ) asset_event_filters = [ DagScheduleAssetReference.dag_id == dag.dag_id, - AssetEvent.timestamp <= exec_date, + AssetEvent.timestamp <= logical_date, ] if previous_dag_run: - asset_event_filters.append(AssetEvent.timestamp > previous_dag_run.execution_date) + asset_event_filters.append(AssetEvent.timestamp > previous_dag_run.logical_date) asset_events = session.scalars( select(AssetEvent) @@ -1439,10 +1439,10 @@ def _create_dag_runs_asset_triggered( .where(*asset_event_filters) ).all() - data_interval = dag.timetable.data_interval_for_events(exec_date, asset_events) + data_interval = dag.timetable.data_interval_for_events(logical_date, asset_events) run_id = dag.timetable.generate_run_id( run_type=DagRunType.ASSET_TRIGGERED, - logical_date=exec_date, + logical_date=logical_date, data_interval=data_interval, session=session, events=asset_events, @@ -1451,7 +1451,7 @@ def _create_dag_runs_asset_triggered( dag_run = dag.create_dagrun( run_id=run_id, run_type=DagRunType.ASSET_TRIGGERED, - execution_date=exec_date, + logical_date=logical_date, data_interval=data_interval, state=DagRunState.QUEUED, external_trigger=False, @@ -1706,8 +1706,8 @@ def _schedule_dag_run( ) return callback_to_execute - if dag_run.execution_date > timezone.utcnow() and not dag.allow_future_exec_dates: - self.log.error("Execution date is in future: %s", dag_run.execution_date) + if dag_run.logical_date > timezone.utcnow() and not dag.allow_future_exec_dates: + self.log.error("Logical date is in future: %s", dag_run.logical_date) return callback if not self._verify_integrity_if_dag_changed(dag_run=dag_run, session=session): @@ -1721,7 +1721,7 @@ def _schedule_dag_run( if self._should_update_dag_next_dagruns(dag, dag_model, last_dag_run=dag_run, session=session): dag_model.calculate_dagrun_date_fields(dag, dag.get_run_data_interval(dag_run)) # This will do one query per dag run. We "could" build up a complex - # query to update all the TIs across all the execution dates and dag + # query to update all the TIs across all the logical dates and dag # IDs in a single query, but it turns out that can be _very very slow_ # see #11147/commit ee90807ac for more details if span.is_recording(): diff --git a/airflow/models/backfill.py b/airflow/models/backfill.py index 11d677542fc0..0e88fa15bb04 100644 --- a/airflow/models/backfill.py +++ b/airflow/models/backfill.py @@ -171,7 +171,7 @@ def _create_backfill_dag_run( dr = session.scalar( with_row_locks( select(DagRun) - .where(DagRun.execution_date == info.logical_date) + .where(DagRun.logical_date == info.logical_date) .order_by(nulls_first(desc(DagRun.start_date), session=session)) .limit(1), session=session, @@ -204,7 +204,7 @@ def _create_backfill_dag_run( dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) dr = dag.create_dagrun( triggered_by=DagRunTriggeredByType.BACKFILL, - execution_date=info.logical_date, + logical_date=info.logical_date, data_interval=info.data_interval, start_date=timezone.utcnow(), state=DagRunState.QUEUED, diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index fa26a2026f5d..520060b8b8f7 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -438,16 +438,16 @@ class derived from this one results in the creation of a task object, :param max_retry_delay: maximum delay interval between retries, can be set as ``timedelta`` or ``float`` seconds, which will be converted into ``timedelta``. :param start_date: The ``start_date`` for the task, determines - the ``execution_date`` for the first task instance. The best practice + the ``logical_date`` for the first task instance. The best practice is to have the start_date rounded to your DAG's schedule. Daily jobs have their start_date some day at 00:00:00, hourly jobs have their start_date at 00:00 of a specific hour. Note that Airflow simply looks at the latest - ``execution_date`` and adds the schedule to determine - the next ``execution_date``. It is also very important + ``logical_date`` and adds the schedule to determine + the next ``logical_date``. It is also very important to note that different tasks' dependencies need to line up in time. If task A depends on task B and their - start_date are offset in a way that their execution_date don't line + start_date are offset in a way that their logical_date don't line up, A's dependencies will never be met. If you are looking to delay a task, for example running a daily task at 2AM, look into the ``TimeSensor`` and ``TimeDeltaSensor``. We advise against using @@ -552,7 +552,7 @@ class derived from this one results in the creation of a task object, Resources constructor) to their values. :param run_as_user: unix username to impersonate while running the task :param max_active_tis_per_dag: When set, a task will be able to limit the concurrent - runs across execution_dates. + runs across logical_dates. :param max_active_tis_per_dagrun: When set, a task will be able to limit the concurrent task instances per DAG run. :param executor: Which executor to target when running this task. NOT YET SUPPORTED @@ -772,9 +772,9 @@ def clear( qry = select(TaskInstance).where(TaskInstance.dag_id == self.dag_id) if start_date: - qry = qry.where(TaskInstance.execution_date >= start_date) + qry = qry.where(TaskInstance.logical_date >= start_date) if end_date: - qry = qry.where(TaskInstance.execution_date <= end_date) + qry = qry.where(TaskInstance.logical_date <= end_date) tasks = [self.task_id] @@ -814,10 +814,10 @@ def get_task_instances( .where(TaskInstance.task_id == self.task_id) ) if start_date: - query = query.where(DagRun.execution_date >= start_date) + query = query.where(DagRun.logical_date >= start_date) if end_date: - query = query.where(DagRun.execution_date <= end_date) - return session.scalars(query.order_by(DagRun.execution_date)).all() + query = query.where(DagRun.logical_date <= end_date) + return session.scalars(query.order_by(DagRun.logical_date)).all() @provide_session def run( @@ -853,7 +853,7 @@ def run( dag_run = session.scalars( select(DagRun).where( DagRun.dag_id == self.dag_id, - DagRun.execution_date == info.logical_date, + DagRun.logical_date == info.logical_date, ) ).one() ti = TaskInstance(self, run_id=dag_run.run_id) @@ -863,7 +863,7 @@ def run( dag_id=self.dag_id, run_id=DagRun.generate_run_id(DagRunType.MANUAL, info.logical_date), run_type=DagRunType.MANUAL, - execution_date=info.logical_date, + logical_date=info.logical_date, data_interval=info.data_interval, triggered_by=DagRunTriggeredByType.TEST, ) @@ -953,7 +953,7 @@ def xcom_pull( :param dag_id: If provided, only pulls XComs from this DAG. If None (default), the DAG of the calling task is used. :param include_prior_dates: If False, only XComs from the current - execution_date are returned. If True, XComs from previous dates + logical_date are returned. If True, XComs from previous dates are returned as well. """ return context["ti"].xcom_pull( diff --git a/airflow/models/dag.py b/airflow/models/dag.py index e48ec0a9a9c5..7ab252b88b02 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -194,7 +194,7 @@ def get_last_dagrun(dag_id, session, include_externally_triggered=False): query = select(DR).where(DR.dag_id == dag_id) if not include_externally_triggered: query = query.where(DR.external_trigger == expression.false()) - query = query.order_by(DR.execution_date.desc()) + query = query.order_by(DR.logical_date.desc()) return session.scalar(query.limit(1)) @@ -267,7 +267,7 @@ def _create_orm_dagrun( run = DagRun( dag_id=dag_id, run_id=run_id, - execution_date=logical_date, + logical_date=logical_date, start_date=start_date, external_trigger=external_trigger, conf=conf, @@ -538,7 +538,7 @@ def get_run_data_interval(self, run: DagRun | DagRunPydantic) -> DataInterval: return data_interval # Compatibility: runs created before AIP-39 implementation don't have an # explicit data interval. Try to infer from the logical date. - return self.infer_automated_data_interval(run.execution_date) + return self.infer_automated_data_interval(run.logical_date) def infer_automated_data_interval(self, logical_date: datetime) -> DataInterval: """ @@ -581,13 +581,13 @@ def next_dagrun_info( Get information about the next DagRun of this dag after ``date_last_automated_dagrun``. This calculates what time interval the next DagRun should operate on - (its execution date) and when it can be scheduled, according to the + (its logical date) and when it can be scheduled, according to the dag's timetable, start_date, end_date, etc. This doesn't check max active run or any other "max_active_tasks" type limits, but only performs calculations based on the various date and interval fields of this dag and its tasks. - :param last_automated_dagrun: The ``max(execution_date)`` of + :param last_automated_dagrun: The ``max(logical_date)`` of existing "automated" DagRuns for this dag (scheduled or backfill, but not manual). :param restricted: If set to *False* (default is *True*), ignore @@ -784,7 +784,7 @@ def get_serialized_fields(cls): @provide_session def fetch_callback( dag: DAG, - dag_run_id: str, + run_id: str, success: bool = True, reason: str | None = None, *, @@ -797,14 +797,14 @@ def fetch_callback( the list of callbacks. :param dag: DAG object - :param dag_run_id: The DAG run ID + :param run_id: The DAG run ID :param success: Flag to specify if failure or success callback should be called :param reason: Completion reason :param session: Database session """ callbacks = dag.on_success_callback if success else dag.on_failure_callback if callbacks: - dagrun = DAG.fetch_dagrun(dag_id=dag.dag_id, run_id=dag_run_id, session=session) + dagrun = DAG.fetch_dagrun(dag_id=dag.dag_id, run_id=run_id, session=session) callbacks = callbacks if isinstance(callbacks, list) else [callbacks] tis = dagrun.get_task_instances(session=session) # tis from a dagrun may not be a part of dag.partial_subset, @@ -840,7 +840,7 @@ def handle_callback(self, dagrun: DagRun, success=True, reason=None, session=NEW :param session: Database session """ callbacks, context = DAG.fetch_callback( - dag=self, dag_run_id=dagrun.run_id, success=success, reason=reason, session=session + dag=self, run_id=dagrun.run_id, success=success, reason=reason, session=session ) or (None, None) DAG.execute_callback(callbacks, context, self.dag_id) @@ -865,15 +865,15 @@ def execute_callback(cls, callbacks: list[Callable] | None, context: Context | N def get_active_runs(self): """ - Return a list of dag run execution dates currently running. + Return a list of dag run logical dates currently running. - :return: List of execution dates + :return: List of logical dates """ runs = DagRun.find(dag_id=self.dag_id, state=DagRunState.RUNNING) active_dates = [] for run in runs: - active_dates.append(run.execution_date) + active_dates.append(run.logical_date) return active_dates @@ -882,24 +882,24 @@ def get_active_runs(self): @provide_session def fetch_dagrun( dag_id: str, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, session: Session = NEW_SESSION, ) -> DagRun | DagRunPydantic: """ - Return the dag run for a given execution date or run_id if it exists, otherwise none. + Return the dag run for a given logical date or run_id if it exists, otherwise none. :param dag_id: The dag_id of the DAG to find. - :param execution_date: The execution date of the DagRun to find. + :param logical_date: The logical date of the DagRun to find. :param run_id: The run_id of the DagRun to find. :param session: :return: The DagRun if found, otherwise None. """ - if not (execution_date or run_id): - raise TypeError("You must provide either the execution_date or the run_id") + if not (logical_date or run_id): + raise TypeError("You must provide either the logical_date or the run_id") query = select(DagRun) - if execution_date: - query = query.where(DagRun.dag_id == dag_id, DagRun.execution_date == execution_date) + if logical_date: + query = query.where(DagRun.dag_id == dag_id, DagRun.logical_date == logical_date) if run_id: query = query.where(DagRun.dag_id == dag_id, DagRun.run_id == run_id) return session.scalar(query) @@ -907,38 +907,36 @@ def fetch_dagrun( @provide_session def get_dagrun( self, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, session: Session = NEW_SESSION, ) -> DagRun | DagRunPydantic: - return DAG.fetch_dagrun( - dag_id=self.dag_id, execution_date=execution_date, run_id=run_id, session=session - ) + return DAG.fetch_dagrun(dag_id=self.dag_id, logical_date=logical_date, run_id=run_id, session=session) @provide_session def get_dagruns_between(self, start_date, end_date, session=NEW_SESSION): """ Return the list of dag runs between start_date (inclusive) and end_date (inclusive). - :param start_date: The starting execution date of the DagRun to find. - :param end_date: The ending execution date of the DagRun to find. + :param start_date: The starting logical date of the DagRun to find. + :param end_date: The ending logical date of the DagRun to find. :param session: :return: The list of DagRuns found. """ dagruns = session.scalars( select(DagRun).where( DagRun.dag_id == self.dag_id, - DagRun.execution_date >= start_date, - DagRun.execution_date <= end_date, + DagRun.logical_date >= start_date, + DagRun.logical_date <= end_date, ) ).all() return dagruns @provide_session - def get_latest_execution_date(self, session: Session = NEW_SESSION) -> pendulum.DateTime | None: + def get_latest_logical_date(self, session: Session = NEW_SESSION) -> pendulum.DateTime | None: """Return the latest date for which at least one dag run exists.""" - return session.scalar(select(func.max(DagRun.execution_date)).where(DagRun.dag_id == self.dag_id)) + return session.scalar(select(func.max(DagRun.logical_date)).where(DagRun.dag_id == self.dag_id)) @provide_session def get_task_instances_before( @@ -955,21 +953,21 @@ def get_task_instances_before( corresponding to any DagRunType. It can have less if there are less than ``num`` scheduled DAG runs before ``base_date``. """ - execution_dates: list[Any] = session.execute( - select(DagRun.execution_date) + logical_dates: list[Any] = session.execute( + select(DagRun.logical_date) .where( DagRun.dag_id == self.dag_id, - DagRun.execution_date <= base_date, + DagRun.logical_date <= base_date, ) - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .limit(num) ).all() - if not execution_dates: + if not logical_dates: return self.get_task_instances(start_date=base_date, end_date=base_date, session=session) - min_date: datetime | None = execution_dates[-1]._mapping.get( - "execution_date" + min_date: datetime | None = logical_dates[-1]._mapping.get( + "logical_date" ) # getting the last value from the list return self.get_task_instances(start_date=min_date, end_date=base_date, session=session) @@ -997,7 +995,7 @@ def get_task_instances( exclude_task_ids=(), session=session, ) - return session.scalars(cast(Select, query).order_by(DagRun.execution_date)).all() + return session.scalars(cast(Select, query).order_by(DagRun.logical_date)).all() @overload def _get_task_instances( @@ -1074,14 +1072,14 @@ def _get_task_instances( if run_id: tis = tis.where(TaskInstance.run_id == run_id) if start_date: - tis = tis.where(DagRun.execution_date >= start_date) + tis = tis.where(DagRun.logical_date >= start_date) if task_ids is not None: tis = tis.where(TaskInstance.ti_selector_condition(task_ids)) # This allows allow_trigger_in_future config to take affect, rather than mandating exec_date <= UTC if end_date or not self.allow_future_exec_dates: end_date = end_date or timezone.utcnow() - tis = tis.where(DagRun.execution_date <= end_date) + tis = tis.where(DagRun.logical_date <= end_date) if state: if isinstance(state, (str, TaskInstanceState)): @@ -1146,7 +1144,7 @@ def _get_task_instances( .where( TI.dag_id == task.external_dag_id, TI.task_id == task.external_task_id, - DagRun.execution_date == pendulum.parse(task.execution_date), + DagRun.logical_date == pendulum.parse(task.logical_date), ) ) @@ -1219,7 +1217,7 @@ def set_task_instance_state( *, task_id: str, map_indexes: Collection[int] | None = None, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, state: TaskInstanceState, upstream: bool = False, @@ -1235,7 +1233,7 @@ def set_task_instance_state( :param task_id: Task ID of the TaskInstance :param map_indexes: Only set TaskInstance if its map_index matches. If None (default), all mapped TaskInstances of the task are set. - :param execution_date: Execution date of the TaskInstance + :param logical_date: Logical date of the TaskInstance :param run_id: The run_id of the TaskInstance :param state: State to set the TaskInstance to :param upstream: Include all upstream tasks of the given task_id @@ -1246,8 +1244,8 @@ def set_task_instance_state( """ from airflow.api.common.mark_tasks import set_state - if not exactly_one(execution_date, run_id): - raise ValueError("Exactly one of execution_date or run_id must be provided") + if not exactly_one(logical_date, run_id): + raise ValueError("Exactly one of logical_date or run_id must be provided") task = self.get_task(task_id) task.dag = self @@ -1260,7 +1258,7 @@ def set_task_instance_state( altered = set_state( tasks=tasks_to_set_state, - execution_date=execution_date, + logical_date=logical_date, run_id=run_id, upstream=upstream, downstream=downstream, @@ -1283,16 +1281,16 @@ def set_task_instance_state( include_upstream=False, ) - if execution_date is None: + if logical_date is None: dag_run = session.scalars( select(DagRun).where(DagRun.run_id == run_id, DagRun.dag_id == self.dag_id) ).one() # Raises an error if not found - resolve_execution_date = dag_run.execution_date + resolve_logical_date = dag_run.logical_date else: - resolve_execution_date = execution_date + resolve_logical_date = logical_date - end_date = resolve_execution_date if not future else None - start_date = resolve_execution_date if not past else None + end_date = resolve_logical_date if not future else None + start_date = resolve_logical_date if not past else None subdag.clear( start_date=start_date, @@ -1310,7 +1308,7 @@ def set_task_group_state( self, *, group_id: str, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_id: str | None = None, state: TaskInstanceState, upstream: bool = False, @@ -1324,7 +1322,7 @@ def set_task_group_state( Set TaskGroup to the given state and clear downstream tasks in failed or upstream_failed state. :param group_id: The group_id of the TaskGroup - :param execution_date: Execution date of the TaskInstance + :param logical_date: Logical date of the TaskInstance :param run_id: The run_id of the TaskInstance :param state: State to set the TaskInstance to :param upstream: Include all upstream tasks of the given task_id @@ -1336,22 +1334,22 @@ def set_task_group_state( """ from airflow.api.common.mark_tasks import set_state - if not exactly_one(execution_date, run_id): - raise ValueError("Exactly one of execution_date or run_id must be provided") + if not exactly_one(logical_date, run_id): + raise ValueError("Exactly one of logical_date or run_id must be provided") tasks_to_set_state: list[BaseOperator | tuple[BaseOperator, int]] = [] task_ids: list[str] = [] - if execution_date is None: + if logical_date is None: dag_run = session.scalars( select(DagRun).where(DagRun.run_id == run_id, DagRun.dag_id == self.dag_id) ).one() # Raises an error if not found - resolve_execution_date = dag_run.execution_date + resolve_logical_date = dag_run.logical_date else: - resolve_execution_date = execution_date + resolve_logical_date = logical_date - end_date = resolve_execution_date if not future else None - start_date = resolve_execution_date if not past else None + end_date = resolve_logical_date if not future else None + start_date = resolve_logical_date if not past else None task_group_dict = self.task_group.get_task_group_dict() task_group = task_group_dict.get(group_id) @@ -1361,17 +1359,17 @@ def set_task_group_state( task_ids = [task.task_id for task in task_group.iter_tasks()] dag_runs_query = select(DagRun.id).where(DagRun.dag_id == self.dag_id) if start_date is None and end_date is None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date == start_date) + dag_runs_query = dag_runs_query.where(DagRun.logical_date == start_date) else: if start_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date >= start_date) + dag_runs_query = dag_runs_query.where(DagRun.logical_date >= start_date) if end_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date <= end_date) + dag_runs_query = dag_runs_query.where(DagRun.logical_date <= end_date) with lock_rows(dag_runs_query, session): altered = set_state( tasks=tasks_to_set_state, - execution_date=execution_date, + logical_date=logical_date, run_id=run_id, upstream=upstream, downstream=downstream, @@ -1423,8 +1421,8 @@ def clear( Clear a set of task instances associated with the current dag for a specified date range. :param task_ids: List of task ids or (``task_id``, ``map_index``) tuples to clear - :param start_date: The minimum execution_date to clear - :param end_date: The maximum execution_date to clear + :param start_date: The minimum logical_date to clear + :param end_date: The maximum logical_date to clear :param only_failed: Only clear failed tasks :param only_running: Only clear running tasks. :param confirm_prompt: Ask for confirmation @@ -1550,7 +1548,7 @@ def cli(self): @provide_session def test( self, - execution_date: datetime | None = None, + logical_date: datetime | None = None, run_conf: dict[str, Any] | None = None, conn_file_path: str | None = None, variable_file_path: str | None = None, @@ -1559,9 +1557,9 @@ def test( session: Session = NEW_SESSION, ) -> DagRun: """ - Execute one single DagRun for a given DAG and execution date. + Execute one single DagRun for a given DAG and logical date. - :param execution_date: execution date for the DAG run + :param logical_date: logical date for the DAG run :param run_conf: configuration to pass to newly created dagrun :param conn_file_path: file path to a connection file in either yaml or json :param variable_file_path: file path to a variable file in either yaml or json @@ -1599,23 +1597,23 @@ def add_logger_if_needed(ti: TaskInstance): exit_stack.callback(lambda: secrets_backend_list.pop(0)) with exit_stack: - execution_date = execution_date or timezone.utcnow() + logical_date = logical_date or timezone.utcnow() self.validate() - self.log.debug("Clearing existing task instances for execution date %s", execution_date) + self.log.debug("Clearing existing task instances for logical date %s", logical_date) self.clear( - start_date=execution_date, - end_date=execution_date, + start_date=logical_date, + end_date=logical_date, dag_run_state=False, # type: ignore session=session, ) self.log.debug("Getting dagrun for dag %s", self.dag_id) - logical_date = timezone.coerce_datetime(execution_date) + logical_date = timezone.coerce_datetime(logical_date) data_interval = self.timetable.infer_manual_data_interval(run_after=logical_date) dr: DagRun = _get_or_create_dagrun( dag=self, - start_date=execution_date, - execution_date=execution_date, - run_id=DagRun.generate_run_id(DagRunType.MANUAL, execution_date), + start_date=logical_date, + logical_date=logical_date, + run_id=DagRun.generate_run_id(DagRunType.MANUAL, logical_date), session=session, conf=run_conf, triggered_by=DagRunTriggeredByType.TEST, @@ -1699,8 +1697,8 @@ def create_dagrun( self, state: DagRunState, *, - triggered_by: DagRunTriggeredByType, - execution_date: datetime | None = None, + triggered_by: DagRunTriggeredByType | None, + logical_date: datetime | None = None, run_id: str | None = None, start_date: datetime | None = None, external_trigger: bool | None = False, @@ -1721,7 +1719,7 @@ def create_dagrun( :param triggered_by: The entity which triggers the DagRun :param run_id: defines the run id for this dag run :param run_type: type of DagRun - :param execution_date: the execution date of this dag run + :param logical_date: the logical date of this dag run :param start_date: the date this dag run should be evaluated :param external_trigger: whether this dag run is externally triggered :param conf: Dict containing configuration/parameters to pass to the DAG @@ -1731,7 +1729,7 @@ def create_dagrun( :param data_interval: Data interval of the DagRun :param backfill_id: id of the backfill run if one exists """ - logical_date = timezone.coerce_datetime(execution_date) + logical_date = timezone.coerce_datetime(logical_date) if data_interval and not isinstance(data_interval, DataInterval): data_interval = DataInterval(*map(timezone.coerce_datetime, data_interval)) @@ -1761,13 +1759,13 @@ def create_dagrun( f"A {run_type.value} DAG run cannot use ID {run_id!r} since it " f"is reserved for {inferred_run_type.value} runs" ) - elif run_type and logical_date is not None: # Generate run_id from run_type and execution_date. + elif run_type and logical_date is not None: # Generate run_id from run_type and logical_date. run_id = self.timetable.generate_run_id( run_type=run_type, logical_date=logical_date, data_interval=data_interval ) else: raise AirflowException( - "Creating DagRun needs either `run_id` or both `run_type` and `execution_date`" + "Creating DagRun needs either `run_id` or both `run_type` and `logical_date`" ) regex = airflow_conf.get("scheduler", "allowed_run_id_pattern") @@ -2427,7 +2425,7 @@ def _get_or_create_dagrun( dag: DAG, conf: dict[Any, Any] | None, start_date: datetime, - execution_date: datetime, + logical_date: datetime, run_id: str, session: Session, triggered_by: DagRunTriggeredByType, @@ -2441,7 +2439,7 @@ def _get_or_create_dagrun( :param dag: DAG to be used to find run. :param conf: Configuration to pass to newly created run. :param start_date: Start date of new run. - :param execution_date: Logical date for finding an existing run. + :param logical_date: Logical date for finding an existing run. :param run_id: Run ID for the new DAG run. :param triggered_by: the entity which triggers the dag_run @@ -2449,7 +2447,7 @@ def _get_or_create_dagrun( """ log.info("dagrun id: %s", dag.dag_id) dr: DagRun = session.scalar( - select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.execution_date == execution_date) + select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.logical_date == logical_date) ) if dr: session.delete(dr) @@ -2457,9 +2455,9 @@ def _get_or_create_dagrun( dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) dr = dag.create_dagrun( state=DagRunState.RUNNING, - execution_date=execution_date, + logical_date=logical_date, run_id=run_id, - start_date=start_date or execution_date, + start_date=start_date or logical_date, session=session, conf=conf, data_interval=data_interval, diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index ccc4832f5fb6..a2327221ad5d 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -129,7 +129,7 @@ class DagRun(Base, LoggingMixin): id = Column(Integer, primary_key=True) dag_id = Column(StringID(), nullable=False) queued_at = Column(UtcDateTime) - execution_date = Column("logical_date", UtcDateTime, default=timezone.utcnow, nullable=False) + logical_date = Column(UtcDateTime, default=timezone.utcnow, nullable=False) start_date = Column(UtcDateTime) end_date = Column(UtcDateTime) _state = Column("state", String(50), default=DagRunState.QUEUED) @@ -228,7 +228,7 @@ def __init__( dag_id: str | None = None, run_id: str | None = None, queued_at: datetime | None | ArgNotSet = NOTSET, - execution_date: datetime | None = None, + logical_date: datetime | None = None, start_date: datetime | None = None, external_trigger: bool | None = None, conf: Any | None = None, @@ -248,7 +248,7 @@ def __init__( self.dag_id = dag_id self.run_id = run_id - self.execution_date = execution_date + self.logical_date = logical_date self.start_date = start_date self.external_trigger = external_trigger self.conf = conf or {} @@ -268,7 +268,7 @@ def __init__( def __repr__(self): return ( - f"" ) @@ -287,10 +287,6 @@ def validate_run_id(self, key: str, run_id: str) -> str | None: def stats_tags(self) -> dict[str, str]: return prune_dict({"dag_id": self.dag_id, "run_type": self.run_type}) - @property - def logical_date(self) -> datetime: - return self.execution_date - def get_state(self): return self._state @@ -437,13 +433,13 @@ def get_running_dag_runs_to_examine(cls, session: Session) -> Query: .order_by( nulls_first(BackfillDagRun.sort_ordinal, session=session), nulls_first(cls.last_scheduling_decision, session=session), - cls.execution_date, + cls.logical_date, ) .limit(cls.DEFAULT_DAGRUNS_TO_EXAMINE) ) - if not settings.ALLOW_FUTURE_EXEC_DATES: - query = query.where(DagRun.execution_date <= func.now()) + if not settings.ALLOW_FUTURE_LOGICAL_DATES: + query = query.where(DagRun.logical_date <= func.now()) return session.scalars(with_row_locks(query, of=cls, session=session, skip_locked=True)) @@ -523,13 +519,13 @@ def get_queued_dag_runs_to_set_running(cls, session: Session) -> Query: nulls_first(BackfillDagRun.sort_ordinal, session=session), nulls_first(cls.last_scheduling_decision, session=session), nulls_first(running_drs.c.num_running, session=session), # many running -> lower priority - cls.execution_date, + cls.logical_date, ) .limit(cls.DEFAULT_DAGRUNS_TO_EXAMINE) ) - if not settings.ALLOW_FUTURE_EXEC_DATES: - query = query.where(DagRun.execution_date <= func.now()) + if not settings.ALLOW_FUTURE_LOGICAL_DATES: + query = query.where(DagRun.logical_date <= func.now()) return session.scalars(with_row_locks(query, of=cls, session=session, skip_locked=True)) @@ -539,14 +535,14 @@ def find( cls, dag_id: str | list[str] | None = None, run_id: Iterable[str] | None = None, - execution_date: datetime | Iterable[datetime] | None = None, + logical_date: datetime | Iterable[datetime] | None = None, state: DagRunState | None = None, external_trigger: bool | None = None, no_backfills: bool = False, run_type: DagRunType | None = None, session: Session = NEW_SESSION, - execution_start_date: datetime | None = None, - execution_end_date: datetime | None = None, + logical_start_date: datetime | None = None, + logical_end_date: datetime | None = None, ) -> list[DagRun]: """ Return a set of dag runs for the given search criteria. @@ -554,14 +550,14 @@ def find( :param dag_id: the dag_id or list of dag_id to find dag runs for :param run_id: defines the run id for this dag run :param run_type: type of DagRun - :param execution_date: the execution date + :param logical_date: the logical date :param state: the state of the dag run :param external_trigger: whether this dag run is externally triggered :param no_backfills: return no backfills (True), return all (False). Defaults to False :param session: database session - :param execution_start_date: dag run that was executed from this date - :param execution_end_date: dag run that was executed until this date + :param logical_start_date: dag run that was executed from this date + :param logical_end_date: dag run that was executed until this date """ qry = select(cls) dag_ids = [dag_id] if isinstance(dag_id, str) else dag_id @@ -572,16 +568,16 @@ def find( qry = qry.where(cls.run_id.in_(run_id)) elif run_id is not None: qry = qry.where(cls.run_id == run_id) - if is_container(execution_date): - qry = qry.where(cls.execution_date.in_(execution_date)) - elif execution_date is not None: - qry = qry.where(cls.execution_date == execution_date) - if execution_start_date and execution_end_date: - qry = qry.where(cls.execution_date.between(execution_start_date, execution_end_date)) - elif execution_start_date: - qry = qry.where(cls.execution_date >= execution_start_date) - elif execution_end_date: - qry = qry.where(cls.execution_date <= execution_end_date) + if is_container(logical_date): + qry = qry.where(cls.logical_date.in_(logical_date)) + elif logical_date is not None: + qry = qry.where(cls.logical_date == logical_date) + if logical_start_date and logical_end_date: + qry = qry.where(cls.logical_date.between(logical_start_date, logical_end_date)) + elif logical_start_date: + qry = qry.where(cls.logical_date >= logical_start_date) + elif logical_end_date: + qry = qry.where(cls.logical_date <= logical_end_date) if state: qry = qry.where(cls.state == state) if external_trigger is not None: @@ -591,7 +587,7 @@ def find( if no_backfills: qry = qry.where(cls.run_type != DagRunType.BACKFILL_JOB) - return session.scalars(qry.order_by(cls.execution_date)).all() + return session.scalars(qry.order_by(cls.logical_date)).all() @classmethod @provide_session @@ -599,31 +595,29 @@ def find_duplicate( cls, dag_id: str, run_id: str, - execution_date: datetime, + logical_date: datetime, session: Session = NEW_SESSION, ) -> DagRun | None: """ - Return an existing run for the DAG with a specific run_id or execution_date. - - *None* is returned if no such DAG run is found. + Return an existing run for the DAG with a specific run_id or logical date. :param dag_id: the dag_id to find duplicates for :param run_id: defines the run id for this dag run - :param execution_date: the execution date + :param logical_date: the logical date :param session: database session """ return session.scalars( select(cls).where( cls.dag_id == dag_id, - or_(cls.run_id == run_id, cls.execution_date == execution_date), + or_(cls.run_id == run_id, cls.logical_date == logical_date), ) ).one_or_none() @staticmethod - def generate_run_id(run_type: DagRunType, execution_date: datetime) -> str: - """Generate Run ID based on Run Type and Execution Date.""" + def generate_run_id(run_type: DagRunType, logical_date: datetime) -> str: + """Generate Run ID based on Run Type and Logical Date.""" # _Ensure_ run_type is a DagRunType, not just a string from user code - return DagRunType(run_type).generate_run_id(execution_date) + return DagRunType(run_type).generate_run_id(logical_date) @staticmethod @internal_api_call @@ -669,7 +663,7 @@ def _check_last_n_dagruns_failed(self, dag_id, max_consecutive_failed_dag_runs, dag_runs = ( session.query(DagRun) .filter(DagRun.dag_id == dag_id) - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .limit(max_consecutive_failed_dag_runs) .all() ) @@ -797,11 +791,11 @@ def get_previous_dagrun( """ filters = [ DagRun.dag_id == dag_run.dag_id, - DagRun.execution_date < dag_run.execution_date, + DagRun.logical_date < dag_run.logical_date, ] if state is not None: filters.append(DagRun.state == state) - return session.scalar(select(DagRun).where(*filters).order_by(DagRun.execution_date.desc()).limit(1)) + return session.scalar(select(DagRun).where(*filters).order_by(DagRun.logical_date.desc()).limit(1)) @staticmethod @internal_api_call @@ -821,10 +815,10 @@ def get_previous_scheduled_dagrun( select(DagRun) .where( DagRun.dag_id == dag_run.dag_id, - DagRun.execution_date < dag_run.execution_date, + DagRun.logical_date < dag_run.logical_date, DagRun.run_type != DagRunType.MANUAL, ) - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .limit(1) ) @@ -994,7 +988,7 @@ def recalculate(self) -> _UnfinishedStates: if self._state == DagRunState.FAILED or self._state == DagRunState.SUCCESS: msg = ( - "DagRun Finished: dag_id=%s, execution_date=%s, run_id=%s, " + "DagRun Finished: dag_id=%s, logical_date=%s, run_id=%s, " "run_start_date=%s, run_end_date=%s, run_duration=%s, " "state=%s, external_trigger=%s, run_type=%s, " "data_interval_start=%s, data_interval_end=%s, dag_version_name=%s" @@ -1003,7 +997,7 @@ def recalculate(self) -> _UnfinishedStates: self.log.info( msg, self.dag_id, - self.execution_date, + self.logical_date, self.run_id, self.start_date, self.end_date, @@ -1039,7 +1033,7 @@ def _trace_dagrun(self, dagv) -> None: attributes = { "category": "DAG runs", "dag_id": self.dag_id, - "execution_date": str(self.execution_date), + "logical_date": str(self.logical_date), "run_id": self.run_id, "queued_at": str(self.queued_at), "run_start_date": str(self.start_date), @@ -1330,8 +1324,8 @@ def verify_integrity(self, *, session: Session = NEW_SESSION) -> None: def task_filter(task: Operator) -> bool: return task.task_id not in task_ids and ( self.run_type == DagRunType.BACKFILL_JOB - or (task.start_date is None or task.start_date <= self.execution_date) - and (task.end_date is None or self.execution_date <= task.end_date) + or (task.start_date is None or task.start_date <= self.logical_date) + and (task.end_date is None or self.logical_date <= task.end_date) ) created_counts: dict[str, int] = defaultdict(int) @@ -1607,14 +1601,14 @@ def _revise_map_indexes_if_mapped(self, task: Operator, *, session: Session) -> def get_latest_runs(cls, session: Session = NEW_SESSION) -> list[DagRun]: """Return the latest DagRun for each DAG.""" subquery = ( - select(cls.dag_id, func.max(cls.execution_date).label("execution_date")) + select(cls.dag_id, func.max(cls.logical_date).label("logical_date")) .group_by(cls.dag_id) .subquery() ) return session.scalars( select(cls).join( subquery, - and_(cls.dag_id == subquery.c.dag_id, cls.execution_date == subquery.c.execution_date), + and_(cls.dag_id == subquery.c.dag_id, cls.logical_date == subquery.c.logical_date), ) ).all() diff --git a/airflow/models/log.py b/airflow/models/log.py index e01cc29b45e0..8669d228b250 100644 --- a/airflow/models/log.py +++ b/airflow/models/log.py @@ -41,7 +41,7 @@ class Log(Base): task_id = Column(StringID()) map_index = Column(Integer) event = Column(String(60)) - execution_date = Column(UtcDateTime) + logical_date = Column(UtcDateTime) run_id = Column(StringID()) owner = Column(String(500)) owner_display_name = Column(String(500)) @@ -69,13 +69,13 @@ def __init__( task_owner = None - self.execution_date = None + self.logical_date = None if task_instance: self.dag_id = task_instance.dag_id self.task_id = task_instance.task_id - if execution_date := getattr(task_instance, "execution_date", None): - self.execution_date = execution_date self.run_id = task_instance.run_id + if logical_date := getattr(task_instance, "logical_date", None): + self.logical_date = logical_date self.try_number = task_instance.try_number self.map_index = task_instance.map_index if task := getattr(task_instance, "task", None): @@ -85,8 +85,8 @@ def __init__( self.task_id = kwargs["task_id"] if "dag_id" in kwargs: self.dag_id = kwargs["dag_id"] - if kwargs.get("execution_date"): - self.execution_date = kwargs["execution_date"] + if kwargs.get("logical_date"): + self.logical_date = kwargs["logical_date"] if kwargs.get("run_id"): self.run_id = kwargs["run_id"] if "map_index" in kwargs: diff --git a/airflow/models/renderedtifields.py b/airflow/models/renderedtifields.py index 5b0b2bef52d3..b8ab93cf41a7 100644 --- a/airflow/models/renderedtifields.py +++ b/airflow/models/renderedtifields.py @@ -104,7 +104,7 @@ class RenderedTaskInstanceFields(TaskInstanceDependencies): ) # We don't need a DB level FK here, as we already have that to TI (which has one to DR) but by defining - # the relationship we can more easily find the execution date for these rows + # the relationship we can more easily find the logical date for these rows dag_run = relationship( "DagRun", primaryjoin="""and_( @@ -114,7 +114,7 @@ class RenderedTaskInstanceFields(TaskInstanceDependencies): viewonly=True, ) - execution_date = association_proxy("dag_run", "execution_date") + logical_date = association_proxy("dag_run", "logical_date") def __init__(self, ti: TaskInstance, render_templates=True, rendered_fields=None): self.dag_id = ti.dag_id @@ -254,11 +254,11 @@ def delete_old_records( from airflow.models.dagrun import DagRun tis_to_keep_query = ( - select(cls.dag_id, cls.task_id, cls.run_id, DagRun.execution_date) + select(cls.dag_id, cls.task_id, cls.run_id, DagRun.logical_date) .where(cls.dag_id == dag_id, cls.task_id == task_id) .join(cls.dag_run) .distinct() - .order_by(DagRun.execution_date.desc()) + .order_by(DagRun.logical_date.desc()) .limit(num_to_keep) ) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 30ef941ceea5..7a2f451727a7 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -292,11 +292,11 @@ def _run_raw_task( raise ti.defer_task(exception=defer, session=session) ti.log.info( - "Pausing task as DEFERRED. dag_id=%s, task_id=%s, run_id=%s, execution_date=%s, start_date=%s", + "Pausing task as DEFERRED. dag_id=%s, task_id=%s, run_id=%s, logical_date=%s, start_date=%s", ti.dag_id, ti.task_id, ti.run_id, - _date_or_empty(task_instance=ti, attr="execution_date"), + _date_or_empty(task_instance=ti, attr="logical_date"), _date_or_empty(task_instance=ti, attr="start_date"), ) return TaskReturnCode.DEFERRED @@ -577,7 +577,7 @@ def _xcom_pull( If *None* (default), this is inferred from the task(s) being pulled (see below for details). :param include_prior_dates: If False, only XComs from the current - execution_date are returned. If *True*, XComs from previous dates + logical_date are returned. If *True*, XComs from previous dates are returned as well. :param run_id: If provided, only pulls XComs from a DagRun w/a matching run_id. If *None* (default), the run_id of the calling task is used. @@ -821,11 +821,11 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.dag_version_id = source.dag_version_id if include_dag_run: - target.execution_date = source.execution_date + target.logical_date = source.logical_date target.dag_run.id = source.dag_run.id target.dag_run.dag_id = source.dag_run.dag_id target.dag_run.queued_at = source.dag_run.queued_at - target.dag_run.execution_date = source.dag_run.execution_date + target.dag_run.logical_date = source.dag_run.logical_date target.dag_run.start_date = source.dag_run.start_date target.dag_run.end_date = source.dag_run.end_date target.dag_run.state = source.dag_run.state @@ -967,7 +967,7 @@ def _get_template_context( validated_params = process_params(dag, task, dag_run, suppress_exception=ignore_param_exceptions) - logical_date: DateTime = timezone.coerce_datetime(task_instance.execution_date) + logical_date: DateTime = timezone.coerce_datetime(task_instance.logical_date) ds = logical_date.strftime("%Y-%m-%d") ds_nodash = ds.replace("-", "") ts = logical_date.isoformat() @@ -1008,76 +1008,6 @@ def get_prev_end_date_success() -> pendulum.DateTime | None: return None return timezone.coerce_datetime(dagrun.end_date) - @cache - def get_yesterday_ds() -> str: - return (logical_date - timedelta(1)).strftime("%Y-%m-%d") - - def get_yesterday_ds_nodash() -> str: - return get_yesterday_ds().replace("-", "") - - @cache - def get_tomorrow_ds() -> str: - return (logical_date + timedelta(1)).strftime("%Y-%m-%d") - - def get_tomorrow_ds_nodash() -> str: - return get_tomorrow_ds().replace("-", "") - - @cache - def get_next_execution_date() -> pendulum.DateTime | None: - # For manually triggered dagruns that aren't run on a schedule, - # the "next" execution date doesn't make sense, and should be set - # to execution date for consistency with how execution_date is set - # for manually triggered tasks, i.e. triggered_date == execution_date. - if dag_run.external_trigger: - return logical_date - if dag is None: - return None - next_info = dag.next_dagrun_info(data_interval, restricted=False) - if next_info is None: - return None - return timezone.coerce_datetime(next_info.logical_date) - - def get_next_ds() -> str | None: - execution_date = get_next_execution_date() - if execution_date is None: - return None - return execution_date.strftime("%Y-%m-%d") - - def get_next_ds_nodash() -> str | None: - ds = get_next_ds() - if ds is None: - return ds - return ds.replace("-", "") - - @cache - def get_prev_execution_date(): - # For manually triggered dagruns that aren't run on a schedule, - # the "previous" execution date doesn't make sense, and should be set - # to execution date for consistency with how execution_date is set - # for manually triggered tasks, i.e. triggered_date == execution_date. - if dag_run.external_trigger: - return logical_date - - # Workaround code copy until deprecated context fields are removed in Airflow 3 - from airflow.timetables.interval import _DataIntervalTimetable - - if not isinstance(dag.timetable, _DataIntervalTimetable): - return None - return dag.timetable._get_prev(timezone.coerce_datetime(logical_date)) - - @cache - def get_prev_ds() -> str | None: - execution_date = get_prev_execution_date() - if execution_date is None: - return None - return execution_date.strftime("%Y-%m-%d") - - def get_prev_ds_nodash() -> str | None: - prev_ds = get_prev_ds() - if prev_ds is None: - return None - return prev_ds.replace("-", "") - def get_triggering_events() -> dict[str, list[AssetEvent | AssetEventPydantic]]: if TYPE_CHECKING: assert session is not None @@ -1114,27 +1044,16 @@ def get_triggering_events() -> dict[str, list[AssetEvent | AssetEventPydantic]]: "outlet_events": OutletEventAccessors(), "ds": ds, "ds_nodash": ds_nodash, - "execution_date": logical_date, "expanded_ti_count": expanded_ti_count, "inlets": task.inlets, "inlet_events": InletEventsAccessors(task.inlets, session=session), "logical_date": logical_date, "macros": macros, "map_index_template": task.map_index_template, - "next_ds": get_next_ds(), - "next_ds_nodash": get_next_ds_nodash(), - "next_execution_date": get_next_execution_date(), "outlets": task.outlets, "params": validated_params, "prev_data_interval_start_success": get_prev_data_interval_start_success(), "prev_data_interval_end_success": get_prev_data_interval_end_success(), - "prev_ds": get_prev_ds(), - "prev_ds_nodash": get_prev_ds_nodash(), - "prev_execution_date": get_prev_execution_date(), - "prev_execution_date_success": task_instance.get_previous_execution_date( - state=DagRunState.SUCCESS, - session=session, - ), "prev_start_date_success": get_prev_start_date_success(), "prev_end_date_success": get_prev_end_date_success(), "run_id": task_instance.run_id, @@ -1143,8 +1062,6 @@ def get_triggering_events() -> dict[str, list[AssetEvent | AssetEventPydantic]]: "task_instance_key_str": f"{task.dag_id}__{task.task_id}__{ds_nodash}", "test_mode": task_instance.test_mode, "ti": task_instance, - "tomorrow_ds": get_tomorrow_ds(), - "tomorrow_ds_nodash": get_tomorrow_ds_nodash(), "triggering_asset_events": lazy_object_proxy.Proxy(get_triggering_events), "ts": ts, "ts_nodash": ts_nodash, @@ -1154,8 +1071,6 @@ def get_triggering_events() -> dict[str, list[AssetEvent | AssetEventPydantic]]: "value": VariableAccessor(deserialize_json=False), }, "conn": ConnectionAccessor(), - "yesterday_ds": get_yesterday_ds(), - "yesterday_ds_nodash": get_yesterday_ds_nodash(), } # Mypy doesn't like turning existing dicts in to a TypeDict -- and we "lie" in the type stub to say it # is one, but in practice it isn't. See https://github.com/python/mypy/issues/8890 @@ -1257,12 +1172,11 @@ def _handle_failure( "end_date": str(task_instance.end_date), "duration": task_instance.duration, "executor_config": str(task_instance.executor_config), - "execution_date": str(task_instance.execution_date), + "logical_date": str(task_instance.logical_date), "hostname": task_instance.hostname, "operator": str(task_instance.operator), } ) - if isinstance(task_instance, TaskInstance): span.set_attribute("log_url", task_instance.log_url) @@ -1389,14 +1303,14 @@ def _get_previous_dagrun( return None -def _get_previous_execution_date( +def _get_previous_logical_date( *, task_instance: TaskInstance | TaskInstancePydantic, state: DagRunState | None, session: Session, ) -> pendulum.DateTime | None: """ - Get execution date from property previous_ti_success. + Get logical date from property previous_ti_success. :param task_instance: the task instance :param session: SQLAlchemy ORM Session @@ -1404,9 +1318,9 @@ def _get_previous_execution_date( :meta private: """ - log.debug("previous_execution_date was called") + log.debug("previous_logical_date was called") prev_ti = task_instance.get_previous_ti(state=state, session=session) - return pendulum.instance(prev_ti.execution_date) if prev_ti and prev_ti.execution_date else None + return pendulum.instance(prev_ti.logical_date) if prev_ti and prev_ti.logical_date else None def _get_previous_start_date( @@ -1592,11 +1506,11 @@ def _log_state(*, task_instance: TaskInstance | TaskInstancePydantic, lead_msg: if task_instance.map_index >= 0: params.append(task_instance.map_index) message += "map_index=%d, " - message += "execution_date=%s, start_date=%s, end_date=%s" + message += "logical_date=%s, start_date=%s, end_date=%s" log.info( message, *params, - _date_or_empty(task_instance=task_instance, attr="execution_date"), + _date_or_empty(task_instance=task_instance, attr="logical_date"), _date_or_empty(task_instance=task_instance, attr="start_date"), _date_or_empty(task_instance=task_instance, attr="end_date"), stacklevel=2, @@ -1920,7 +1834,7 @@ class TaskInstance(Base, LoggingMixin): triggerer_job = association_proxy("trigger", "triggerer_job") dag_run = relationship("DagRun", back_populates="task_instances", lazy="joined", innerjoin=True) rendered_task_instance_fields = relationship("RenderedTaskInstanceFields", lazy="noload", uselist=False) - execution_date = association_proxy("dag_run", "execution_date") + logical_date = association_proxy("dag_run", "logical_date") task_instance_note = relationship( "TaskInstanceNote", back_populates="task_instance", @@ -2183,7 +2097,7 @@ def generate_command( def log_url(self) -> str: """Log URL for TaskInstance.""" run_id = quote(self.run_id) - base_date = quote(self.execution_date.strftime("%Y-%m-%dT%H:%M:%S%z")) + base_date = quote(self.logical_date.strftime("%Y-%m-%dT%H:%M:%S%z")) base_url = conf.get_mandatory_value("webserver", "BASE_URL") map_index = f"&map_index={self.map_index}" if self.map_index >= 0 else "" return ( @@ -2432,18 +2346,18 @@ def get_previous_ti( return _get_previous_ti(task_instance=self, state=state, session=session) @provide_session - def get_previous_execution_date( + def get_previous_logical_date( self, state: DagRunState | None = None, session: Session = NEW_SESSION, ) -> pendulum.DateTime | None: """ - Return the execution date from property previous_ti_success. + Return the logical date from property previous_ti_success. :param state: If passed, it only take into account instances of a specific state. :param session: SQLAlchemy ORM Session """ - return _get_previous_execution_date(task_instance=self, state=state, session=session) + return _get_previous_logical_date(task_instance=self, state=state, session=session) @provide_session def get_previous_start_date( @@ -2542,7 +2456,7 @@ def next_retry_datetime(self): # deterministic per task instance ti_hash = int( hashlib.sha1( - f"{self.dag_id}#{self.task_id}#{self.execution_date}#{self.try_number}".encode() + f"{self.dag_id}#{self.task_id}#{self.logical_date}#{self.try_number}".encode() ).hexdigest(), 16, ) @@ -2592,7 +2506,7 @@ def get_dagrun(self, session: Session = NEW_SESSION) -> DagRun: if TYPE_CHECKING: assert self.task dr.dag = self.task.dag - # Record it in the instance for next time. This means that `self.execution_date` will work correctly + # Record it in the instance for next time. This means that `self.logical_date` will work correctly set_committed_value(self, "dag_run", dr) return dr @@ -2757,9 +2671,9 @@ def _check_and_change_state_before_execution( settings.engine.dispose() # type: ignore if verbose: if mark_success: - cls.logger().info("Marking success for %s on %s", ti.task, ti.execution_date) + cls.logger().info("Marking success for %s on %s", ti.task, ti.logical_date) else: - cls.logger().info("Executing %s on %s", ti.task, ti.execution_date) + cls.logger().info("Executing %s on %s", ti.task, ti.logical_date) return True @provide_session @@ -3483,7 +3397,7 @@ def xcom_pull( If *None* (default), this is inferred from the task(s) being pulled (see below for details). :param include_prior_dates: If False, only XComs from the current - execution_date are returned. If *True*, XComs from previous dates + logical_date are returned. If *True*, XComs from previous dates are returned as well. :param run_id: If provided, only pulls XComs from a DagRun w/a matching run_id. If *None* (default), the run_id of the calling task is used. diff --git a/airflow/models/taskreschedule.py b/airflow/models/taskreschedule.py index 2fc06b798a67..aa987294e8ee 100644 --- a/airflow/models/taskreschedule.py +++ b/airflow/models/taskreschedule.py @@ -75,7 +75,7 @@ class TaskReschedule(TaskInstanceDependencies): ), ) dag_run = relationship("DagRun") - execution_date = association_proxy("dag_run", "execution_date") + logical_date = association_proxy("dag_run", "logical_date") def __init__( self, diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py index 87c72d5bf7f5..69843db5ab5e 100644 --- a/airflow/models/xcom.py +++ b/airflow/models/xcom.py @@ -111,7 +111,7 @@ class BaseXCom(TaskInstanceDependencies, LoggingMixin): lazy="joined", passive_deletes="all", ) - execution_date = association_proxy("dag_run", "execution_date") + logical_date = association_proxy("dag_run", "logical_date") @reconstructor def init_on_load(self): @@ -374,12 +374,12 @@ def get_many( query = query.filter(BaseXCom.map_index == map_indexes) if include_prior_dates: - dr = session.query(DagRun.execution_date).filter(DagRun.run_id == run_id).subquery() - query = query.filter(BaseXCom.execution_date <= dr.c.execution_date) + dr = session.query(DagRun.logical_date).filter(DagRun.run_id == run_id).subquery() + query = query.filter(BaseXCom.logical_date <= dr.c.logical_date) else: query = query.filter(BaseXCom.run_id == run_id) - query = query.order_by(DagRun.execution_date.desc(), BaseXCom.timestamp.desc()) + query = query.order_by(DagRun.logical_date.desc(), BaseXCom.timestamp.desc()) if limit: return query.limit(limit) return query @@ -424,7 +424,7 @@ def clear( :param session: Database session. If not given, a new session will be created for this function. """ - # Given the historic order of this function (execution_date was first argument) to add a new optional + # Given the historic order of this function (logical_date was first argument) to add a new optional # param we need to add default values for everything :( if dag_id is None: raise TypeError("clear() missing required argument: dag_id") diff --git a/airflow/operators/trigger_dagrun.py b/airflow/operators/trigger_dagrun.py index bb1eac7c6963..e65a74ef2b9a 100644 --- a/airflow/operators/trigger_dagrun.py +++ b/airflow/operators/trigger_dagrun.py @@ -195,7 +195,7 @@ def execute(self, context: Context): dag_id=self.trigger_dag_id, run_id=run_id, conf=self.conf, - execution_date=parsed_logical_date, + logical_date=parsed_logical_date, replace_microseconds=False, triggered_by=DagRunTriggeredByType.OPERATOR, ) @@ -234,7 +234,7 @@ def execute(self, context: Context): trigger=DagStateTrigger( dag_id=self.trigger_dag_id, states=self.allowed_states + self.failed_states, - execution_dates=[dag_run.logical_date], + logical_dates=[dag_run.logical_date], poll_interval=self.poke_interval, ), method_name="execute_complete", @@ -261,12 +261,12 @@ def execute(self, context: Context): @provide_session def execute_complete(self, context: Context, session: Session, event: tuple[str, dict[str, Any]]): # This logical_date is parsed from the return trigger event - provided_logical_date = event[1]["execution_dates"][0] + provided_logical_date = event[1]["logical_dates"][0] try: # Note: here execution fails on database isolation mode. Needs structural changes for AIP-72 dag_run = session.execute( select(DagRun).where( - DagRun.dag_id == self.trigger_dag_id, DagRun.execution_date == provided_logical_date + DagRun.dag_id == self.trigger_dag_id, DagRun.logical_date == provided_logical_date ) ).scalar_one() except NoResultFound: diff --git a/airflow/policies.py b/airflow/policies.py index 49691b393866..e0c48931a286 100644 --- a/airflow/policies.py +++ b/airflow/policies.py @@ -99,7 +99,7 @@ def get_airflow_context_vars(context) -> dict[str, str]: # type: ignore[empty-b This setting allows getting the airflow context vars, which are key value pairs. They are then injected to default airflow context vars, which in the end are available as environment variables when running - tasks dag_id, task_id, execution_date, dag_run_id, try_number are reserved keys. + tasks dag_id, task_id, logical_date, dag_run_id, try_number are reserved keys. :param context: The context for the task_instance of interest. """ diff --git a/airflow/sensors/external_task.py b/airflow/sensors/external_task.py index 331e17168bab..d5504ab2e9fb 100644 --- a/airflow/sensors/external_task.py +++ b/airflow/sensors/external_task.py @@ -70,7 +70,7 @@ def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey) -> str: ) query = { "dag_id": external_dag_id, - "execution_date": ti.execution_date.isoformat(), # type: ignore[union-attr] + "logical_date": ti.logical_date.isoformat(), # type: ignore[union-attr] } return build_airflow_url_with_query(query) @@ -351,7 +351,7 @@ def execute(self, context: Context) -> None: external_dag_id=self.external_dag_id, external_task_group_id=self.external_task_group_id, external_task_ids=self.external_task_ids, - execution_dates=self._get_dttm_filter(context), + logical_dates=self._get_dttm_filter(context), allowed_states=self.allowed_states, poke_interval=self.poll_interval, soft_fail=self.soft_fail, @@ -406,7 +406,7 @@ def get_count(self, dttm_filter, session, states) -> int: """ Get the count of records against dttm filter and states. - :param dttm_filter: date time filter for execution date + :param dttm_filter: date time filter for logical date :param session: airflow session object :param states: task or dag states :return: count of record against the filters @@ -436,7 +436,7 @@ def _handle_execution_date_fn(self, context) -> Any: Handle backward compatibility. This function is to handle backwards compatibility with how this operator was - previously where it only passes the execution date, but also allow for the newer + previously where it only passes the logical date, but also allow for the newer implementation to pass all context variables as keyword arguments, to allow for more sophisticated returns of dates to return. """ @@ -464,14 +464,14 @@ class ExternalTaskMarker(EmptyOperator): :param external_dag_id: The dag_id that contains the dependent task that needs to be cleared. :param external_task_id: The task_id of the dependent task that needs to be cleared. - :param execution_date: The logical date of the dependent task execution that needs to be cleared. + :param logical_date: The logical date of the dependent task execution that needs to be cleared. :param recursion_depth: The maximum level of transitive dependencies allowed. Default is 10. This is mostly used for preventing cyclic dependencies. It is fine to increase this number if necessary. However, too many levels of transitive dependencies will make it slower to clear tasks in the web UI. """ - template_fields = ["external_dag_id", "external_task_id", "execution_date"] + template_fields = ["external_dag_id", "external_task_id", "logical_date"] ui_color = "#4db7db" operator_extra_links = [ExternalDagLink()] @@ -483,20 +483,20 @@ def __init__( *, external_dag_id: str, external_task_id: str, - execution_date: str | datetime.datetime | None = "{{ logical_date.isoformat() }}", + logical_date: str | datetime.datetime | None = "{{ logical_date.isoformat() }}", recursion_depth: int = 10, **kwargs, ): super().__init__(**kwargs) self.external_dag_id = external_dag_id self.external_task_id = external_task_id - if isinstance(execution_date, datetime.datetime): - self.execution_date = execution_date.isoformat() - elif isinstance(execution_date, str): - self.execution_date = execution_date + if isinstance(logical_date, datetime.datetime): + self.logical_date = logical_date.isoformat() + elif isinstance(logical_date, str): + self.logical_date = logical_date else: raise TypeError( - f"Expected str or datetime.datetime type for execution_date. Got {type(execution_date)}" + f"Expected str or datetime.datetime type for logical_date. Got {type(logical_date)}" ) if recursion_depth <= 0: diff --git a/airflow/sentry.py b/airflow/sentry.py index d5fbf3c04d21..22261bd99fd2 100644 --- a/airflow/sentry.py +++ b/airflow/sentry.py @@ -62,7 +62,7 @@ def flush(self): class ConfiguredSentry(DummySentry): """Configure Sentry SDK.""" - SCOPE_DAG_RUN_TAGS = frozenset(("data_interval_end", "data_interval_start", "execution_date")) + SCOPE_DAG_RUN_TAGS = frozenset(("data_interval_end", "data_interval_start", "logical_date")) SCOPE_TASK_INSTANCE_TAGS = frozenset(("task_id", "dag_id", "try_number")) SCOPE_CRUMBS = frozenset(("task_id", "state", "operator", "duration")) diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py index a0175e3749d9..b3b5ab6d70d3 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -40,7 +40,7 @@ class DagRunPydantic(BaseModelPydantic): id: int dag_id: str queued_at: Optional[datetime] - execution_date: datetime + logical_date: datetime start_date: Optional[datetime] end_date: Optional[datetime] state: str @@ -61,10 +61,6 @@ class DagRunPydantic(BaseModelPydantic): model_config = ConfigDict(from_attributes=True, arbitrary_types_allowed=True) - @property - def logical_date(self) -> datetime: - return self.execution_date - def get_task_instances( self, state: Iterable[TaskInstanceState | None] | None = None, diff --git a/airflow/serialization/pydantic/taskinstance.py b/airflow/serialization/pydantic/taskinstance.py index e8a1cfaf79a7..eea493758d17 100644 --- a/airflow/serialization/pydantic/taskinstance.py +++ b/airflow/serialization/pydantic/taskinstance.py @@ -90,7 +90,7 @@ class TaskInstancePydantic(BaseModelPydantic, LoggingMixin): map_index: int start_date: Optional[datetime] end_date: Optional[datetime] - execution_date: Optional[datetime] + logical_date: Optional[datetime] duration: Optional[float] state: Optional[str] try_number: int @@ -180,7 +180,7 @@ def xcom_pull( :param task_ids: task id or list of task ids, if None, the task_id of the current task is used :param dag_id: dag id, if None, the dag_id of the current task is used :param key: the key to identify the XCom value - :param include_prior_dates: whether to include prior execution dates + :param include_prior_dates: whether to include prior logical dates :param session: the sqlalchemy session :param map_indexes: map index or list of map indexes, if None, the map_index of the current task is used @@ -367,20 +367,20 @@ def get_previous_dagrun( return _get_previous_dagrun(task_instance=self, state=state, session=session) - def get_previous_execution_date( + def get_previous_logical_date( self, state: DagRunState | None = None, session: Session | None = None, ) -> pendulum.DateTime | None: """ - Return the execution date from property previous_ti_success. + Return the logical date from property previous_ti_success. :param state: If passed, it only take into account instances of a specific state. :param session: SQLAlchemy ORM Session """ - from airflow.models.taskinstance import _get_previous_execution_date + from airflow.models.taskinstance import _get_previous_logical_date - return _get_previous_execution_date(task_instance=self, state=state, session=session) + return _get_previous_logical_date(task_instance=self, state=state, session=session) def get_previous_start_date( self, @@ -388,7 +388,7 @@ def get_previous_start_date( session: Session | None = None, ) -> pendulum.DateTime | None: """ - Return the execution date from property previous_ti_success. + Return the logical date from property previous_ti_success. :param state: If passed, it only take into account instances of a specific state. :param session: SQLAlchemy ORM Session diff --git a/airflow/settings.py b/airflow/settings.py index c3f32fa59d98..db8ee4f41117 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -829,7 +829,7 @@ def is_usage_data_collection_enabled() -> bool: fallback=False, ) -ALLOW_FUTURE_EXEC_DATES = conf.getboolean("scheduler", "allow_trigger_in_future", fallback=False) +ALLOW_FUTURE_LOGICAL_DATES = conf.getboolean("scheduler", "allow_trigger_in_future", fallback=False) USE_JOB_SCHEDULE = conf.getboolean("scheduler", "use_job_schedule", fallback=True) diff --git a/airflow/task/standard_task_runner.py b/airflow/task/standard_task_runner.py index bc846574f024..c446bccac960 100644 --- a/airflow/task/standard_task_runner.py +++ b/airflow/task/standard_task_runner.py @@ -160,7 +160,7 @@ def _start_by_fork(self): self.log.info("Running: %s", self._command) self.log.info("Subtask %s", self._task_instance.task_id) - proc_title = "airflow task runner: {0.dag_id} {0.task_id} {0.execution_date_or_run_id}" + proc_title = "airflow task runner: {0.dag_id} {0.task_id} {0.logical_date_or_run_id}" setproctitle(proc_title.format(args)) return_code = 0 try: diff --git a/airflow/ti_deps/dep_context.py b/airflow/ti_deps/dep_context.py index 988cefdfcf94..bc2d476d8018 100644 --- a/airflow/ti_deps/dep_context.py +++ b/airflow/ti_deps/dep_context.py @@ -89,7 +89,7 @@ def ensure_finished_tis(self, dag_run: DagRun, session: Session) -> list[TaskIns Ensure finished_tis is populated if it's currently None, which allows running tasks without dag_run. :param dag_run: The DagRun for which to find finished tasks - :return: A list of all the finished tasks of this DAG and execution_date + :return: A list of all the finished tasks of this DAG and logical_date """ if self.finished_tis is None: finished_tis = dag_run.get_task_instances(state=State.finished, session=session) diff --git a/airflow/ti_deps/deps/exec_date_after_start_date_dep.py b/airflow/ti_deps/deps/exec_date_after_start_date_dep.py index 09e0d8c22961..4832ea2806be 100644 --- a/airflow/ti_deps/deps/exec_date_after_start_date_dep.py +++ b/airflow/ti_deps/deps/exec_date_after_start_date_dep.py @@ -22,25 +22,25 @@ class ExecDateAfterStartDateDep(BaseTIDep): - """Determines whether a task's execution date is after start date.""" + """Determines whether a task's logical date is after start date.""" - NAME = "Execution Date" + NAME = "Logical Date" IGNORABLE = True @provide_session def _get_dep_statuses(self, ti, session, dep_context): - if ti.task.start_date and ti.execution_date < ti.task.start_date: + if ti.task.start_date and ti.logical_date < ti.task.start_date: yield self._failing_status( reason=( - f"The execution date is {ti.execution_date.isoformat()} but this is before " + f"The logical date is {ti.logical_date.isoformat()} but this is before " f"the task's start date {ti.task.start_date.isoformat()}." ) ) - if ti.task.dag and ti.task.dag.start_date and ti.execution_date < ti.task.dag.start_date: + if ti.task.dag and ti.task.dag.start_date and ti.logical_date < ti.task.dag.start_date: yield self._failing_status( reason=( - f"The execution date is {ti.execution_date.isoformat()} but this is " + f"The logical date is {ti.logical_date.isoformat()} but this is " f"before the task's DAG's start date {ti.task.dag.start_date.isoformat()}." ) ) diff --git a/airflow/ti_deps/deps/prev_dagrun_dep.py b/airflow/ti_deps/deps/prev_dagrun_dep.py index 7265727f6d53..c756e6ec1c64 100644 --- a/airflow/ti_deps/deps/prev_dagrun_dep.py +++ b/airflow/ti_deps/deps/prev_dagrun_dep.py @@ -78,7 +78,7 @@ def _has_any_prior_tis(ti: TI, *, session: Session) -> bool: query = exists_query( TI.dag_id == ti.dag_id, TI.task_id == ti.task_id, - TI.execution_date < ti.execution_date, + TI.logical_date < ti.logical_date, session=session, ) return query @@ -169,7 +169,7 @@ def _get_dep_statuses(self, ti: TI, session: Session, dep_context): return # There was a DAG run, but the task wasn't active back then. - if catchup and last_dagrun.execution_date < ti.task.start_date: + if catchup and last_dagrun.logical_date < ti.task.start_date: self._push_past_deps_met_xcom_if_needed(ti, dep_context) yield self._passing_status(reason="This task instance was the first task instance for its task.") return diff --git a/airflow/ti_deps/deps/runnable_exec_date_dep.py b/airflow/ti_deps/deps/runnable_exec_date_dep.py index da9d88a2ec06..990264563a2f 100644 --- a/airflow/ti_deps/deps/runnable_exec_date_dep.py +++ b/airflow/ti_deps/deps/runnable_exec_date_dep.py @@ -23,7 +23,7 @@ class RunnableExecDateDep(BaseTIDep): - """Determines whether a task's execution date is valid.""" + """Determines whether a task's logical date is valid.""" NAME = "Execution Date" IGNORABLE = True @@ -34,11 +34,11 @@ def _get_dep_statuses(self, ti, session, dep_context): # don't consider runs that are executed in the future unless # specified by config and schedule is None - logical_date = ti.get_dagrun(session).execution_date + logical_date = ti.get_dagrun(session).logical_date if logical_date > cur_date and not ti.task.dag.allow_future_exec_dates: yield self._failing_status( reason=( - f"Execution date {logical_date.isoformat()} is in the future " + f"Logical date {logical_date.isoformat()} is in the future " f"(the current date is {cur_date.isoformat()})." ) ) @@ -46,7 +46,7 @@ def _get_dep_statuses(self, ti, session, dep_context): if ti.task.end_date and logical_date > ti.task.end_date: yield self._failing_status( reason=( - f"The execution date is {logical_date.isoformat()} but this is " + f"The logical date is {logical_date.isoformat()} but this is " f"after the task's end date {ti.task.end_date.isoformat()}." ) ) @@ -54,7 +54,7 @@ def _get_dep_statuses(self, ti, session, dep_context): if ti.task.dag and ti.task.dag.end_date and logical_date > ti.task.dag.end_date: yield self._failing_status( reason=( - f"The execution date is {logical_date.isoformat()} but this is after " + f"The logical date is {logical_date.isoformat()} but this is after " f"the task's DAG's end date {ti.task.dag.end_date.isoformat()}." ) ) diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index 64a261202651..f8aa4279ebba 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -174,7 +174,7 @@ class Timetable(Protocol): ``NullTimetable`` sets this to *False*. """ - run_ordering: Sequence[str] = ("data_interval_end", "execution_date") + run_ordering: Sequence[str] = ("data_interval_end", "logical_date") """How runs triggered from this timetable should be ordered in UI. This should be a list of field names on the DAG run object. diff --git a/airflow/timetables/simple.py b/airflow/timetables/simple.py index 5a931b40dd11..3457c52a08aa 100644 --- a/airflow/timetables/simple.py +++ b/airflow/timetables/simple.py @@ -36,7 +36,7 @@ class _TrivialTimetable(Timetable): """Some code reuse for "trivial" timetables that has nothing complex.""" periodic = False - run_ordering: Sequence[str] = ("execution_date",) + run_ordering: Sequence[str] = ("logical_date",) @classmethod def deserialize(cls, data: dict[str, Any]) -> Timetable: diff --git a/airflow/triggers/external_task.py b/airflow/triggers/external_task.py index cd43d59876e9..159a6df90950 100644 --- a/airflow/triggers/external_task.py +++ b/airflow/triggers/external_task.py @@ -41,7 +41,7 @@ class WorkflowTrigger(BaseTrigger): A trigger to monitor tasks, task group and dag execution in Apache Airflow. :param external_dag_id: The ID of the external DAG. - :param execution_dates: A list of execution dates for the external DAG. + :param logical_dates: A list of logical dates for the external DAG. :param external_task_ids: A collection of external task IDs to wait for. :param external_task_group_id: The ID of the external task group to wait for. :param failed_states: States considered as failed for external tasks. @@ -54,7 +54,7 @@ class WorkflowTrigger(BaseTrigger): def __init__( self, external_dag_id: str, - execution_dates: list, + logical_dates: list, external_task_ids: typing.Collection[str] | None = None, external_task_group_id: str | None = None, failed_states: typing.Iterable[str] | None = None, @@ -70,7 +70,7 @@ def __init__( self.failed_states = failed_states self.skipped_states = skipped_states self.allowed_states = allowed_states - self.execution_dates = execution_dates + self.logical_dates = logical_dates self.poke_interval = poke_interval self.soft_fail = soft_fail super().__init__(**kwargs) @@ -86,7 +86,7 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "failed_states": self.failed_states, "skipped_states": self.skipped_states, "allowed_states": self.allowed_states, - "execution_dates": self.execution_dates, + "logical_dates": self.logical_dates, "poke_interval": self.poke_interval, "soft_fail": self.soft_fail, }, @@ -109,7 +109,7 @@ async def run(self) -> typing.AsyncIterator[TriggerEvent]: yield TriggerEvent({"status": "skipped"}) return allowed_count = await self._get_count(self.allowed_states) - if allowed_count == len(self.execution_dates): + if allowed_count == len(self.logical_dates): yield TriggerEvent({"status": "success"}) return self.log.info("Sleeping for %s seconds", self.poke_interval) @@ -124,7 +124,7 @@ def _get_count(self, states: typing.Iterable[str] | None) -> int: :return The count of records. """ return _get_count( - dttm_filter=self.execution_dates, + dttm_filter=self.logical_dates, external_task_ids=self.external_task_ids, external_task_group_id=self.external_task_group_id, external_dag_id=self.external_dag_id, @@ -138,7 +138,7 @@ class DagStateTrigger(BaseTrigger): :param dag_id: The dag_id that contains the task you want to wait for :param states: allowed states, default is ``['success']`` - :param execution_dates: The logical date at which DAG run. + :param logical_dates: The logical date at which DAG run. :param poll_interval: The time interval in seconds to check the state. The default value is 5.0 sec. """ @@ -147,13 +147,13 @@ def __init__( self, dag_id: str, states: list[DagRunState], - execution_dates: list[datetime], + logical_dates: list[datetime], poll_interval: float = 5.0, ): super().__init__() self.dag_id = dag_id self.states = states - self.execution_dates = execution_dates + self.logical_dates = logical_dates self.poll_interval = poll_interval def serialize(self) -> tuple[str, dict[str, typing.Any]]: @@ -163,7 +163,7 @@ def serialize(self) -> tuple[str, dict[str, typing.Any]]: { "dag_id": self.dag_id, "states": self.states, - "execution_dates": self.execution_dates, + "logical_dates": self.logical_dates, "poll_interval": self.poll_interval, }, ) @@ -173,7 +173,7 @@ async def run(self) -> typing.AsyncIterator[TriggerEvent]: while True: # mypy confuses typing here num_dags = await self.count_dags() # type: ignore[call-arg] - if num_dags == len(self.execution_dates): + if num_dags == len(self.logical_dates): yield TriggerEvent(self.serialize()) return await asyncio.sleep(self.poll_interval) @@ -187,7 +187,7 @@ def count_dags(self, *, session: Session = NEW_SESSION) -> int | None: .filter( DagRun.dag_id == self.dag_id, DagRun.state.in_(self.states), - DagRun.execution_date.in_(self.execution_dates), + DagRun.logical_date.in_(self.logical_dates), ) .scalar() ) diff --git a/airflow/ui/openapi-gen/requests/schemas.gen.ts b/airflow/ui/openapi-gen/requests/schemas.gen.ts index 8a8a50bb7437..d6a5a299ec40 100644 --- a/airflow/ui/openapi-gen/requests/schemas.gen.ts +++ b/airflow/ui/openapi-gen/requests/schemas.gen.ts @@ -3895,10 +3895,10 @@ export const $XComResponseNative = { format: "date-time", title: "Timestamp", }, - execution_date: { + logical_date: { type: "string", format: "date-time", - title: "Execution Date", + title: "Logical Date", }, map_index: { type: "integer", @@ -3920,7 +3920,7 @@ export const $XComResponseNative = { required: [ "key", "timestamp", - "execution_date", + "logical_date", "map_index", "task_id", "dag_id", @@ -3941,10 +3941,10 @@ export const $XComResponseString = { format: "date-time", title: "Timestamp", }, - execution_date: { + logical_date: { type: "string", format: "date-time", - title: "Execution Date", + title: "Logical Date", }, map_index: { type: "integer", @@ -3974,7 +3974,7 @@ export const $XComResponseString = { required: [ "key", "timestamp", - "execution_date", + "logical_date", "map_index", "task_id", "dag_id", diff --git a/airflow/ui/openapi-gen/requests/types.gen.ts b/airflow/ui/openapi-gen/requests/types.gen.ts index 35848f3a88f9..80fa416445c2 100644 --- a/airflow/ui/openapi-gen/requests/types.gen.ts +++ b/airflow/ui/openapi-gen/requests/types.gen.ts @@ -933,7 +933,7 @@ export type VersionInfo = { export type XComResponseNative = { key: string; timestamp: string; - execution_date: string; + logical_date: string; map_index: number; task_id: string; dag_id: string; @@ -946,7 +946,7 @@ export type XComResponseNative = { export type XComResponseString = { key: string; timestamp: string; - execution_date: string; + logical_date: string; map_index: number; task_id: string; dag_id: string; diff --git a/airflow/utils/cli.py b/airflow/utils/cli.py index 81b09f9d1104..c3cd1faf6b26 100644 --- a/airflow/utils/cli.py +++ b/airflow/utils/cli.py @@ -75,7 +75,7 @@ def action_logging(f: T) -> T: log : airflow.models.log.Log ORM instance dag_id : dag id (optional) task_id : task_id (optional) - execution_date : execution date (optional) + logical_date : logical date (optional) error : exception instance if there's an exception :param f: function instance @@ -130,7 +130,7 @@ def _build_metrics(func_name, namespace): It assumes that function arguments is from airflow.bin.cli module's function and has Namespace instance where it optionally contains "dag_id", "task_id", - and "execution_date". + and "logical_date". :param func_name: name of function :param namespace: Namespace instance from argparse @@ -172,7 +172,7 @@ def _build_metrics(func_name, namespace): tmp_dic = vars(namespace) metrics["dag_id"] = tmp_dic.get("dag_id") metrics["task_id"] = tmp_dic.get("task_id") - metrics["execution_date"] = tmp_dic.get("execution_date") + metrics["logical_date"] = tmp_dic.get("logical_date") metrics["host_name"] = socket.gethostname() return metrics diff --git a/airflow/utils/cli_action_loggers.py b/airflow/utils/cli_action_loggers.py index dfb8e41de856..568286c11179 100644 --- a/airflow/utils/cli_action_loggers.py +++ b/airflow/utils/cli_action_loggers.py @@ -103,7 +103,7 @@ def on_post_execution(**kwargs): logger.exception("Failed on post-execution callback using %s", callback) -def default_action_log(sub_command, user, task_id, dag_id, execution_date, host_name, full_command, **_): +def default_action_log(sub_command, user, task_id, dag_id, logical_date, host_name, full_command, **_): """ Behave similar to ``action_logging``; default action logger callback. @@ -115,7 +115,7 @@ def default_action_log(sub_command, user, task_id, dag_id, execution_date, host_ user=user, task_id=task_id, dag_id=dag_id, - execution_date=execution_date, + logical_date=logical_date, host_name=host_name, full_command=full_command, ) @@ -129,7 +129,7 @@ def _default_action_log_internal( user, task_id, dag_id, - execution_date, + logical_date, host_name, full_command, session: Session = NEW_SESSION, @@ -160,7 +160,7 @@ def _default_action_log_internal( "extra": json.dumps({"host_name": host_name, "full_command": full_command}), "task_id": task_id, "dag_id": dag_id, - "execution_date": execution_date, + "logical_date": logical_date, "dttm": timezone.utcnow(), } ], diff --git a/airflow/utils/context.py b/airflow/utils/context.py index b28559999e50..3e217e748d02 100644 --- a/airflow/utils/context.py +++ b/airflow/utils/context.py @@ -71,7 +71,6 @@ "data_interval_start", "ds", "ds_nodash", - "execution_date", "expanded_ti_count", "exception", "inlets", @@ -79,18 +78,11 @@ "logical_date", "macros", "map_index_template", - "next_ds", - "next_ds_nodash", - "next_execution_date", "outlets", "outlet_events", "params", "prev_data_interval_start_success", "prev_data_interval_end_success", - "prev_ds", - "prev_ds_nodash", - "prev_execution_date", - "prev_execution_date_success", "prev_start_date_success", "prev_end_date_success", "reason", @@ -101,16 +93,12 @@ "test_mode", "templates_dict", "ti", - "tomorrow_ds", - "tomorrow_ds_nodash", "triggering_asset_events", "ts", "ts_nodash", "ts_nodash_with_tz", "try_number", "var", - "yesterday_ds", - "yesterday_ds_nodash", } @@ -330,20 +318,7 @@ class Context(MutableMapping[str, Any]): (and only when) deprecated context keys are accessed. """ - _DEPRECATION_REPLACEMENTS: dict[str, list[str]] = { - "execution_date": ["data_interval_start", "logical_date"], - "next_ds": ["{{ data_interval_end | ds }}"], - "next_ds_nodash": ["{{ data_interval_end | ds_nodash }}"], - "next_execution_date": ["data_interval_end"], - "prev_ds": [], - "prev_ds_nodash": [], - "prev_execution_date": [], - "prev_execution_date_success": ["prev_data_interval_start_success"], - "tomorrow_ds": [], - "tomorrow_ds_nodash": [], - "yesterday_ds": [], - "yesterday_ds_nodash": [], - } + _DEPRECATION_REPLACEMENTS: dict[str, list[str]] = {} def __init__(self, context: MutableMapping[str, Any] | None = None, **kwargs: Any) -> None: self._context: MutableMapping[str, Any] = context or {} diff --git a/airflow/utils/context.pyi b/airflow/utils/context.pyi index 4dc4659548ac..f4ed77537ff1 100644 --- a/airflow/utils/context.pyi +++ b/airflow/utils/context.pyi @@ -102,24 +102,16 @@ class Context(TypedDict, total=False): ds: str ds_nodash: str exception: BaseException | str | None - execution_date: DateTime expanded_ti_count: int | None inlets: list inlet_events: InletEventsAccessors logical_date: DateTime macros: Any map_index_template: str - next_ds: str | None - next_ds_nodash: str | None - next_execution_date: DateTime | None outlets: list params: ParamsDict prev_data_interval_start_success: DateTime | None prev_data_interval_end_success: DateTime | None - prev_ds: str | None - prev_ds_nodash: str | None - prev_execution_date: DateTime | None - prev_execution_date_success: DateTime | None prev_start_date_success: DateTime | None prev_end_date_success: DateTime | None reason: str | None @@ -130,16 +122,12 @@ class Context(TypedDict, total=False): test_mode: bool templates_dict: Mapping[str, Any] | None ti: TaskInstance | TaskInstancePydantic - tomorrow_ds: str - tomorrow_ds_nodash: str triggering_asset_events: Mapping[str, Collection[AssetEvent | AssetEventPydantic]] ts: str ts_nodash: str ts_nodash_with_tz: str try_number: int | None var: _VariableAccessors - yesterday_ds: str - yesterday_ds_nodash: str class AirflowContextDeprecationWarning(DeprecationWarning): ... diff --git a/airflow/utils/db.py b/airflow/utils/db.py index a45b14ae666e..7995c1a802d6 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -973,7 +973,7 @@ def synchronize_log_template(*, session: Session = NEW_SESSION) -> None: session.add( LogTemplate( filename="{{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log", - elasticsearch_id="{dag_id}-{task_id}-{execution_date}-{try_number}", + elasticsearch_id="{dag_id}-{task_id}-{logical_date}-{try_number}", ) ) diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index 8b83e3b13496..1a6b7396e322 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -198,7 +198,7 @@ def render_log_filename(ti: TaskInstance, try_number, filename_template) -> str: return filename_template.format( dag_id=ti.dag_id, task_id=ti.task_id, - execution_date=ti.execution_date.isoformat(), + logical_date=ti.logical_date.isoformat(), try_number=try_number, ) @@ -234,7 +234,7 @@ def build_airflow_url_with_query(query: dict[str, Any]) -> str: Build airflow url using base_url and default_view and provided query. For example: - http://0.0.0.0:8000/base/graph?dag_id=my-task&root=&execution_date=2020-10-27T10%3A59%3A25.615587 + http://0.0.0.0:8000/base/graph?dag_id=my-task&root=&logical_date=2020-10-27T10%3A59%3A25.615587 """ import flask diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 0192892603f2..622effe60ee3 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -315,7 +315,7 @@ def _render_filename(self, ti: TaskInstance | TaskInstancePydantic, try_number: run_id=ti.run_id, data_interval_start=data_interval_start, data_interval_end=data_interval_end, - execution_date=ti.get_dagrun().logical_date.isoformat(), + logical_date=ti.get_dagrun().logical_date.isoformat(), try_number=try_number, ) else: diff --git a/airflow/utils/operator_helpers.py b/airflow/utils/operator_helpers.py index a259b455776b..a06cdf42b50c 100644 --- a/airflow/utils/operator_helpers.py +++ b/airflow/utils/operator_helpers.py @@ -46,9 +46,9 @@ "default": f"{DEFAULT_FORMAT_PREFIX}task_id", "env_var_format": f"{ENV_VAR_FORMAT_PREFIX}TASK_ID", }, - "AIRFLOW_CONTEXT_EXECUTION_DATE": { - "default": f"{DEFAULT_FORMAT_PREFIX}execution_date", - "env_var_format": f"{ENV_VAR_FORMAT_PREFIX}EXECUTION_DATE", + "AIRFLOW_CONTEXT_LOGICAL_DATE": { + "default": f"{DEFAULT_FORMAT_PREFIX}logical_date", + "env_var_format": f"{ENV_VAR_FORMAT_PREFIX}LOGICAL_DATE", }, "AIRFLOW_CONTEXT_TRY_NUMBER": { "default": f"{DEFAULT_FORMAT_PREFIX}try_number", @@ -97,7 +97,7 @@ def context_to_airflow_vars(context: Mapping[str, Any], in_env_var_format: bool (task, "owner", "AIRFLOW_CONTEXT_DAG_OWNER"), (task_instance, "dag_id", "AIRFLOW_CONTEXT_DAG_ID"), (task_instance, "task_id", "AIRFLOW_CONTEXT_TASK_ID"), - (task_instance, "execution_date", "AIRFLOW_CONTEXT_EXECUTION_DATE"), + (task_instance, "logical_date", "AIRFLOW_CONTEXT_LOGICAL_DATE"), (task_instance, "try_number", "AIRFLOW_CONTEXT_TRY_NUMBER"), (dag_run, "run_id", "AIRFLOW_CONTEXT_DAG_RUN_ID"), ] diff --git a/airflow/utils/sensor_helper.py b/airflow/utils/sensor_helper.py index fe72a7004180..57d906da671c 100644 --- a/airflow/utils/sensor_helper.py +++ b/airflow/utils/sensor_helper.py @@ -40,7 +40,7 @@ def _get_count( """ Get the count of records against dttm filter and states. - :param dttm_filter: date time filter for execution date + :param dttm_filter: date time filter for logical date :param external_task_ids: The list of task_ids :param external_task_group_id: The ID of the external task group :param external_dag_id: The ID of the external DAG. @@ -85,12 +85,12 @@ def _count_query(model, states, dttm_filter, external_dag_id, session: Session) :param model: The SQLAlchemy model representing the relevant table. :param states: task or dag states - :param dttm_filter: date time filter for execution date + :param dttm_filter: date time filter for logical date :param external_dag_id: The ID of the external DAG. :param session: airflow session object """ query = select(func.count()).filter( - model.dag_id == external_dag_id, model.state.in_(states), model.execution_date.in_(dttm_filter) + model.dag_id == external_dag_id, model.state.in_(states), model.logical_date.in_(dttm_filter) ) return query @@ -99,7 +99,7 @@ def _get_external_task_group_task_ids(dttm_filter, external_task_group_id, exter """ Get the count of records against dttm filter and states. - :param dttm_filter: date time filter for execution date + :param dttm_filter: date time filter for logical date :param external_task_group_id: The ID of the external task group :param external_dag_id: The ID of the external DAG. :param session: airflow session object @@ -112,7 +112,7 @@ def _get_external_task_group_task_ids(dttm_filter, external_task_group_id, exter select(TaskInstance).filter( TaskInstance.dag_id == external_dag_id, TaskInstance.task_id.in_(task.task_id for task in task_group), - TaskInstance.execution_date.in_(dttm_filter), + TaskInstance.logical_date.in_(dttm_filter), ) ) diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py index 476cfe2df28b..aca3810c548d 100644 --- a/airflow/utils/timezone.py +++ b/airflow/utils/timezone.py @@ -38,7 +38,7 @@ utc = pendulum.UTC -def is_localized(value): +def is_localized(value: dt.datetime) -> bool: """ Determine if a given datetime.datetime is aware. diff --git a/airflow/www/decorators.py b/airflow/www/decorators.py index 3eae5f623918..fc0ad893369f 100644 --- a/airflow/www/decorators.py +++ b/airflow/www/decorators.py @@ -105,7 +105,7 @@ def wrapper(*args, **kwargs): "task_id", "dag_run_id", "run_id", - "execution_date", + "logical_date", } extra_fields = { k: secrets_masker.redact(v, k) @@ -145,13 +145,13 @@ def wrapper(*args, **kwargs): run_id=params.get("run_id") or params.get("dag_run_id"), ) - if "execution_date" in request.values: - execution_date_value = request.values.get("execution_date") + if "logical_date" in request.values: + logical_date_value = request.values.get("logical_date") try: - log.execution_date = pendulum.parse(execution_date_value, strict=False) + log.logical_date = pendulum.parse(logical_date_value, strict=False) except ParserError: logger.exception( - "Failed to parse execution_date from the request: %s", execution_date_value + "Failed to parse logical_date from the request: %s", logical_date_value ) session.add(log) diff --git a/airflow/www/forms.py b/airflow/www/forms.py index a3b1d5262db2..57b4fde3a6e6 100644 --- a/airflow/www/forms.py +++ b/airflow/www/forms.py @@ -95,7 +95,7 @@ def _get_default_timezone(self): class DateTimeForm(FlaskForm): """Date filter form needed for task views.""" - execution_date = DateTimeWithTimezoneField("Logical date", widget=AirflowDateTimePickerWidget()) + logical_date = DateTimeWithTimezoneField("Logical date", widget=AirflowDateTimePickerWidget()) class DagRunEditForm(DynamicForm): @@ -114,7 +114,7 @@ class DagRunEditForm(DynamicForm): ) run_id = StringField(lazy_gettext("Run Id"), validators=[ReadOnly()], widget=BS3TextFieldROWidget()) state = StringField(lazy_gettext("State"), validators=[ReadOnly()], widget=BS3TextFieldROWidget()) - execution_date = DateTimeWithTimezoneField( + logical_date = DateTimeWithTimezoneField( lazy_gettext("Logical Date"), validators=[ReadOnly()], widget=AirflowDateTimePickerROWidget(), @@ -162,7 +162,7 @@ class TaskInstanceEditForm(DynamicForm): widget=Select2Widget(), validators=[InputRequired()], ) - execution_date = DateTimeWithTimezoneField( + logical_date = DateTimeWithTimezoneField( lazy_gettext("Logical Date"), widget=AirflowDateTimePickerROWidget(), validators=[InputRequired(), ReadOnly()], diff --git a/airflow/www/static/js/api/useClearTask.ts b/airflow/www/static/js/api/useClearTask.ts index b4f80e5a7bf2..56f7d7b8601f 100644 --- a/airflow/www/static/js/api/useClearTask.ts +++ b/airflow/www/static/js/api/useClearTask.ts @@ -31,13 +31,13 @@ export default function useClearTask({ dagId, runId, taskId, - executionDate, + logicalDate, isGroup, }: { dagId: string; runId: string; taskId: string; - executionDate: string; + logicalDate: string; isGroup: boolean; }) { const queryClient = useQueryClient(); @@ -70,7 +70,7 @@ export default function useClearTask({ dag_id: dagId, dag_run_id: runId, confirmed, - execution_date: executionDate, + logical_date: logicalDate, past, future, upstream, diff --git a/airflow/www/static/js/api/useClearTaskDryRun.ts b/airflow/www/static/js/api/useClearTaskDryRun.ts index 33986a9b4868..ba0cbf6bb163 100644 --- a/airflow/www/static/js/api/useClearTaskDryRun.ts +++ b/airflow/www/static/js/api/useClearTaskDryRun.ts @@ -30,7 +30,7 @@ const useClearTaskDryRun = ({ dagId, runId, taskId, - executionDate, + logicalDate, isGroup, past, future, @@ -44,7 +44,7 @@ const useClearTaskDryRun = ({ dagId: string; runId: string; taskId: string; - executionDate: string; + logicalDate: string; isGroup: boolean; past: boolean; future: boolean; @@ -75,7 +75,7 @@ const useClearTaskDryRun = ({ dag_id: dagId, dag_run_id: runId, confirmed: false, - execution_date: executionDate, + logical_date: logicalDate, past, future, upstream, diff --git a/airflow/www/static/js/api/useExtraLinks.ts b/airflow/www/static/js/api/useExtraLinks.ts index e4140d7d4c88..a61f0f3aa69d 100644 --- a/airflow/www/static/js/api/useExtraLinks.ts +++ b/airflow/www/static/js/api/useExtraLinks.ts @@ -31,20 +31,20 @@ interface LinkData { export default function useExtraLinks({ dagId, taskId, - executionDate, + logicalDate, mapIndex, extraLinks, tryNumber, }: { dagId: string; taskId: string; - executionDate: string; + logicalDate: string; mapIndex?: number | undefined; extraLinks: string[]; tryNumber?: number | undefined; }) { return useQuery( - ["extraLinks", dagId, taskId, executionDate, mapIndex, tryNumber], + ["extraLinks", dagId, taskId, logicalDate, mapIndex, tryNumber], async () => { const data = await Promise.all( extraLinks.map(async (link) => { @@ -55,8 +55,8 @@ export default function useExtraLinks({ taskId )}&dag_id=${encodeURIComponent( dagId - )}&execution_date=${encodeURIComponent( - executionDate + )}&logical_date=${encodeURIComponent( + logicalDate )}&link_name=${encodeURIComponent( link )}&map_index=${definedMapIndex}${tryNumberParam}`; diff --git a/airflow/www/static/js/api/useGridData.test.ts b/airflow/www/static/js/api/useGridData.test.ts index 3e83e29ee56a..efbfe5d9820b 100644 --- a/airflow/www/static/js/api/useGridData.test.ts +++ b/airflow/www/static/js/api/useGridData.test.ts @@ -24,7 +24,7 @@ import { areActiveRuns } from "./useGridData"; const commonDagRunParams = { runId: "runId", - executionDate: "2022-01-01T10:00+00:00", + logicalDate: "2022-01-01T10:00+00:00", dataIntervalStart: "2022-01-01T05:00+00:00", dataIntervalEnd: "2022-01-01T10:00+00:00", runType: "scheduled" as DagRun["runType"], diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts index 6bed9dfd71c7..795a5db1d5b8 100644 --- a/airflow/www/static/js/api/useGridData.ts +++ b/airflow/www/static/js/api/useGridData.ts @@ -124,8 +124,8 @@ const useGridData = () => { const selectedRun = await axios.get( dagRunUrl ); - if (selectedRun?.executionDate) { - onBaseDateChange(selectedRun.executionDate); + if (selectedRun?.logicalDate) { + onBaseDateChange(selectedRun.logicalDate); } // otherwise the run_id isn't valid and we should unselect it } catch (e) { diff --git a/airflow/www/static/js/dag/details/index.tsx b/airflow/www/static/js/dag/details/index.tsx index 0046d339972a..b26311c464eb 100644 --- a/airflow/www/static/js/dag/details/index.tsx +++ b/airflow/www/static/js/dag/details/index.tsx @@ -272,7 +272,7 @@ const Details = ({ { tryNumber={tryNumber} dagId="dummyDagId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" isInternal /> ); @@ -42,7 +42,7 @@ describe("Test LogLink Component.", () => { expect(linkElement).not.toHaveAttribute("target"); expect( linkElement?.href.includes( - `?dag_id=dummyDagId&task_id=dummyTaskId&execution_date=2020%3A01%3A01T01%3A00%2B00%3A00&map_index=-1&try_number=${tryNumber}&format=file` + `?dag_id=dummyDagId&task_id=dummyTaskId&logical_date=2020%3A01%3A01T01%3A00%2B00%3A00&map_index=-1&try_number=${tryNumber}&format=file` ) ).toBeTruthy(); }); @@ -55,7 +55,7 @@ describe("Test LogLink Component.", () => { tryNumber={tryNumber} dagId="dummyDagId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" mapIndex={mapIndex} /> ); @@ -66,7 +66,7 @@ describe("Test LogLink Component.", () => { expect(linkElement).toHaveAttribute("target", "_blank"); expect( linkElement?.href.includes( - `?dag_id=dummyDagId&task_id=dummyTaskId&execution_date=2020%3A01%3A01T01%3A00%2B00%3A00&map_index=${mapIndex}&try_number=${tryNumber}` + `?dag_id=dummyDagId&task_id=dummyTaskId&logical_date=2020%3A01%3A01T01%3A00%2B00%3A00&map_index=${mapIndex}&try_number=${tryNumber}` ) ).toBeTruthy(); }); diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/LogLink.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/LogLink.tsx index 72a203c7bf84..828f09236176 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/LogLink.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/LogLink.tsx @@ -29,7 +29,7 @@ const externalLogUrl = getMetaValue("external_log_url"); interface Props { dagId: Dag["id"]; taskId: TaskInstance["taskId"]; - executionDate: DagRun["executionDate"]; + logicalDate: DagRun["logicalDate"]; isInternal?: boolean; tryNumber: TaskInstance["tryNumber"]; mapIndex?: TaskInstance["mapIndex"]; @@ -38,7 +38,7 @@ interface Props { const LogLink = ({ dagId, taskId, - executionDate, + logicalDate, isInternal, tryNumber, mapIndex, @@ -47,8 +47,8 @@ const LogLink = ({ isInternal ? logsWithMetadataUrl : externalLogUrl }?dag_id=${encodeURIComponent(dagId)}&task_id=${encodeURIComponent( taskId - )}&execution_date=${encodeURIComponent( - executionDate + )}&logical_date=${encodeURIComponent( + logicalDate )}&map_index=${encodeURIComponent( mapIndex?.toString() ?? "-1" )}&try_number=${tryNumber}`; diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/index.test.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/index.test.tsx index 74db45a7d2f3..31bfbe36c738 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/index.test.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/index.test.tsx @@ -108,7 +108,7 @@ describe("Test Logs Component.", () => { dagId="dummyDagId" dagRunId="dummyDagRunId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" tryNumber={tryNumber} /> ); @@ -153,7 +153,7 @@ describe("Test Logs Component.", () => { dagId="dummyDagId" dagRunId="dummyDagRunId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" mapIndex={1} tryNumber={tryNumber} /> @@ -175,7 +175,7 @@ describe("Test Logs Component.", () => { dagId="dummyDagId" dagRunId="dummyDagRunId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" mapIndex={1} tryNumber={tryNumber} /> @@ -211,7 +211,7 @@ describe("Test Logs Component.", () => { dagId="dummyDagId" dagRunId="dummyDagRunId" taskId="dummyTaskId" - executionDate="2020:01:01T01:00+00:00" + logicalDate="2020:01:01T01:00+00:00" tryNumber={tryNumber} /> ); diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx index 2647e590d84d..3c861f396b41 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx @@ -64,7 +64,7 @@ interface Props { dagRunId: DagRun["runId"]; taskId: TaskInstance["taskId"]; mapIndex?: TaskInstance["mapIndex"]; - executionDate: DagRun["executionDate"]; + logicalDate: DagRun["logicalDate"]; tryNumber: TaskInstance["tryNumber"]; state?: TaskInstance["state"]; } @@ -74,7 +74,7 @@ const Logs = ({ dagRunId, taskId, mapIndex, - executionDate, + logicalDate, tryNumber: finalTryNumber, state, }: Props) => { @@ -109,7 +109,7 @@ const Logs = ({ const params = new URLSearchParamsWrapper({ task_id: taskId, - execution_date: executionDate, + logical_date: logicalDate, }); if (mapIndex !== undefined) { @@ -164,7 +164,7 @@ const Logs = ({ key={tryNumber} dagId={dagId} taskId={taskId} - executionDate={executionDate} + logicalDate={logicalDate} tryNumber={tryNumber} mapIndex={mapIndex} /> @@ -231,7 +231,7 @@ const Logs = ({ ( - ({ taskId, executionDate, isMapped = false, mapIndex }, ref) => { + ({ taskId, logicalDate, isMapped = false, mapIndex }, ref) => { if (!taskId) return null; const params = new URLSearchParamsWrapper({ task_id: taskId, - execution_date: executionDate, + logical_date: logicalDate, map_index: mapIndex ?? -1, }); const detailsLink = `${taskUrl}&${params}`; const listParams = new URLSearchParamsWrapper({ _flt_3_dag_id: dagId, _flt_3_task_id: taskId, - _oc_TaskInstanceModelView: "dag_run.execution_date", + _oc_TaskInstanceModelView: "dag_run.logical_date", }); if (mapIndex !== undefined && mapIndex >= 0) diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 22617dcbd65a..dda550b5bd4b 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -83,12 +83,12 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { ref={taskInstanceRef} overflowY="auto" > - {!isGroup && run?.executionDate && ( + {!isGroup && run?.logicalDate && ( )} {!isGroupOrMappedTaskSummary && } @@ -105,12 +105,12 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { )} {!!group?.extraLinks?.length && !isGroupOrMappedTaskSummary && - run?.executionDate && ( + run?.logicalDate && ( diff --git a/airflow/www/static/js/dag/details/taskInstance/taskActions/ClearInstance.tsx b/airflow/www/static/js/dag/details/taskInstance/taskActions/ClearInstance.tsx index f2a63fcb8927..e4853cecdc69 100644 --- a/airflow/www/static/js/dag/details/taskInstance/taskActions/ClearInstance.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/taskActions/ClearInstance.tsx @@ -45,7 +45,7 @@ const dagId = getMetaValue("dag_id"); interface Props { runId: string; taskId: string; - executionDate: string; + logicalDate: string; isGroup?: boolean; isMapped?: boolean; mapIndex?: number; @@ -60,7 +60,7 @@ const ClearModal = ({ runId, taskId, mapIndex, - executionDate, + logicalDate, isGroup, isMapped, isOpen, @@ -94,7 +94,7 @@ const ClearModal = ({ dagId, runId, taskId, - executionDate, + logicalDate, isGroup: !!isGroup, past, future, @@ -110,7 +110,7 @@ const ClearModal = ({ dagId, runId, taskId, - executionDate, + logicalDate, isGroup: !!isGroup, }); @@ -233,7 +233,7 @@ const ClearInstance = ({ runId, taskId, mapIndex, - executionDate, + logicalDate, isGroup, isMapped, ...otherProps @@ -262,7 +262,7 @@ const ClearInstance = ({ runId={runId} taskId={taskId} mapIndex={mapIndex} - executionDate={executionDate} + logicalDate={logicalDate} isGroup={isGroup} isMapped={isMapped} isOpen={isOpen} diff --git a/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx b/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx index ebbb1f86e0cb..10cb7aee50e9 100644 --- a/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx +++ b/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx @@ -40,7 +40,7 @@ const generateRuns = (length: number): DagRun[] => startDate: "2021-11-08T21:14:19.704433+00:00", endDate: "2021-11-08T21:17:13.206426+00:00", lastSchedulingDecision: datestring, - executionDate: datestring, + logicalDate: datestring, externalTrigger: false, conf: null, note: "someRandomValue", @@ -59,7 +59,7 @@ describe("Test DagRuns", () => { endDate: "2021-11-08T21:17:13.206426+00:00", state: "failed", runType: "scheduled", - executionDate: "2021-11-08T21:14:19.704433+00:00", + logicalDate: "2021-11-08T21:14:19.704433+00:00", lastSchedulingDecision: datestring, externalTrigger: false, conf: null, @@ -74,7 +74,7 @@ describe("Test DagRuns", () => { queuedAt: "2021-11-08T21:14:18.21521+00:00", startDate: "2021-11-09T00:19:43.023200+00:00", endDate: "2021-11-09T00:22:18.607167+00:00", - executionDate: "2021-11-08T21:14:19.704433+00:00", + logicalDate: "2021-11-08T21:14:19.704433+00:00", lastSchedulingDecision: datestring, externalTrigger: false, conf: null, @@ -102,7 +102,7 @@ describe("Test DagRuns", () => { expect(getByText("00:01:26")).toBeInTheDocument(); expect( // @ts-ignore - queryByText(moment.utc(dagRuns[0].executionDate).format("MMM DD, HH:mm")) + queryByText(moment.utc(dagRuns[0].logicalDate).format("MMM DD, HH:mm")) ).toBeNull(); spy.mockRestore(); diff --git a/airflow/www/static/js/dag/grid/index.test.tsx b/airflow/www/static/js/dag/grid/index.test.tsx index b6eb000b768c..7daf444a94b7 100644 --- a/airflow/www/static/js/dag/grid/index.test.tsx +++ b/airflow/www/static/js/dag/grid/index.test.tsx @@ -148,7 +148,7 @@ const mockGridData = { endDate: "2021-11-08T21:17:13.206426+00:00", state: "failed", runType: "scheduled", - executionDate: "2021-11-08T21:14:19.704433+00:00", + logicalDate: "2021-11-08T21:14:19.704433+00:00", lastSchedulingDecision: "2021-11-08T21:14:19.704433+00:00", note: "myCoolDagRun", externalTrigger: false, diff --git a/airflow/www/static/js/dag/useFilters.tsx b/airflow/www/static/js/dag/useFilters.tsx index 5a1a49b68655..8f82598230de 100644 --- a/airflow/www/static/js/dag/useFilters.tsx +++ b/airflow/www/static/js/dag/useFilters.tsx @@ -69,7 +69,7 @@ export interface FilterHookReturn extends UtilFunctions { // Params names export const BASE_DATE_PARAM = "base_date"; -export const EXECUTION_DATE_PARAM = "execution_date"; +export const LOGICAL_DATE_PARAM = "logical_date"; export const NUM_RUNS_PARAM = "num_runs"; export const RUN_TYPE_PARAM = "run_type"; export const RUN_STATE_PARAM = "run_state"; @@ -96,7 +96,7 @@ const useFilters = (): FilterHookReturn => { const baseDate = searchParams.get(BASE_DATE_PARAM) || - searchParams.get(EXECUTION_DATE_PARAM) || + searchParams.get(LOGICAL_DATE_PARAM) || now; const numRuns = searchParams.get(NUM_RUNS_PARAM) || defaultDagRunDisplayNumber.toString(); @@ -182,7 +182,7 @@ const useFilters = (): FilterHookReturn => { searchParams.delete(BASE_DATE_PARAM); searchParams.delete(RUN_TYPE_PARAM); searchParams.delete(RUN_STATE_PARAM); - searchParams.delete(EXECUTION_DATE_PARAM); + searchParams.delete(LOGICAL_DATE_PARAM); setSearchParams(searchParams); }; diff --git a/airflow/www/static/js/dags.js b/airflow/www/static/js/dags.js index 66ea1244879c..9c69edd80618 100644 --- a/airflow/www/static/js/dags.js +++ b/airflow/www/static/js/dags.js @@ -198,7 +198,7 @@ function lastDagRunsHandler(error, json) { $(".js-loading-last-run").remove(); Object.keys(json).forEach((safeDagId) => { const dagId = json[safeDagId].dag_id; - const executionDate = json[safeDagId].execution_date; + const logicalDate = json[safeDagId].logical_date; const g = d3.select(`#last-run-${safeDagId}`); // Show last run as a link to the graph view @@ -207,10 +207,10 @@ function lastDagRunsHandler(error, json) { "href", `${graphUrl}?dag_id=${encodeURIComponent( dagId - )}&execution_date=${encodeURIComponent(executionDate)}` + )}&logical_date=${encodeURIComponent(logicalDate)}` ) .html("") - .insert(isoDateToTimeEl.bind(null, executionDate, { title: false })); + .insert(isoDateToTimeEl.bind(null, logicalDate, { title: false })); // Only show the tooltip when we have a last run and add the json to a custom data- attribute g.selectAll("span") diff --git a/airflow/www/static/js/task_instances.js b/airflow/www/static/js/task_instances.js index 47a5dccb6f3b..211bf1dd2ddf 100644 --- a/airflow/www/static/js/task_instances.js +++ b/airflow/www/static/js/task_instances.js @@ -99,8 +99,8 @@ export default function tiTooltip(ti, task, { includeTryNumber = false } = {}) { if (ti.task_id !== undefined) { tt += `Task_id: ${escapeHtml(ti.task_id)}
`; } - if (ti.execution_date !== undefined) { - tt += `Run: ${formatDateTime(ti.execution_date)}
`; + if (ti.logical_date !== undefined) { + tt += `Run: ${formatDateTime(ti.logical_date)}
`; } if (ti.run_id !== undefined) { tt += `Run Id: ${escapeHtml(ti.run_id)}
`; diff --git a/airflow/www/static/js/ti_log.js b/airflow/www/static/js/ti_log.js index 1fd66acee6dd..02064a419d7c 100644 --- a/airflow/www/static/js/ti_log.js +++ b/airflow/www/static/js/ti_log.js @@ -22,7 +22,7 @@ import { AnsiUp } from "ansi_up"; import { getMetaValue } from "./utils"; import { formatDateTime } from "./datetime_utils"; -const executionDate = getMetaValue("execution_date"); +const logicalDate = getMetaValue("logical_date"); const dagId = getMetaValue("dag_id"); const taskId = getMetaValue("task_id"); const mapIndex = getMetaValue("map_index"); @@ -70,7 +70,7 @@ window.scrollBottomLogs = scrollBottom; function autoTailingLog(tryNumber, metadata = null, autoTailing = false) { console.debug( `Auto-tailing log for dag_id: ${dagId}, task_id: ${taskId}, ` + - `execution_date: ${executionDate}, map_index: ${mapIndex}, try_number: ${tryNumber}, ` + + `logical_date: ${logicalDate}, map_index: ${mapIndex}, try_number: ${tryNumber}, ` + `metadata: ${JSON.stringify(metadata)}` ); @@ -81,7 +81,7 @@ function autoTailingLog(tryNumber, metadata = null, autoTailing = false) { dag_id: dagId, task_id: taskId, map_index: mapIndex, - execution_date: executionDate, + logical_date: logicalDate, try_number: tryNumber, metadata: JSON.stringify(metadata), }, @@ -214,7 +214,7 @@ function setDownloadUrl(tryNumber) { const query = new URLSearchParams({ dag_id: dagId, task_id: taskId, - execution_date: executionDate, + logical_date: logicalDate, try_number: tryNumberData, metadata: "null", format: "file", diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index c94c20f1d8fe..0f91a2329dd2 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1116,7 +1116,7 @@ export interface components { * The value of this field can be set only when creating the object. If you try to modify the * field of an existing object, the request fails with an BAD_REQUEST error. * - * If not provided, a value will be generated based on execution_date. + * If not provided, a value will be generated based on logical_date. * * If the specified dag_run_id is in use, the creation request fails with an ALREADY_EXISTS error. * @@ -1137,18 +1137,6 @@ export interface components { * *New in version 2.2.0* */ logical_date?: string | null; - /** - * Format: date-time - * @deprecated - * @description The execution date. This is the same as logical_date, kept for backwards compatibility. - * If both this field and logical_date are provided but with different values, the request - * will fail with an BAD_REQUEST error. - * - * *Changed in version 2.2.0*: Field becomes nullable. - * - * *Deprecated since version 2.2.0*: Use 'logical_date' instead. - */ - execution_date?: string | null; /** * Format: date-time * @description The start time. The time when DAG run was actually created. @@ -1480,7 +1468,7 @@ export interface components { */ dag_run_id?: string; /** Format: datetime */ - execution_date?: string; + logical_date?: string; /** Format: datetime */ start_date?: string | null; /** Format: datetime */ @@ -1544,7 +1532,7 @@ export interface components { /** @description The DAG ID. */ dag_id?: string; /** Format: datetime */ - execution_date?: string; + logical_date?: string; /** @description The DAG run ID. */ dag_run_id?: string; }; @@ -1586,7 +1574,7 @@ export interface components { /** Format: datetime */ timestamp?: string; /** Format: datetime */ - execution_date?: string; + logical_date?: string; map_index?: number; task_id?: string; dag_id?: string; @@ -2099,11 +2087,11 @@ export interface components { task_id?: string; /** * Format: datetime - * @description The execution date. Either set this or dag_run_id but not both. + * @description The logical date. Either set this or dag_run_id but not both. */ - execution_date?: string; + logical_date?: string; /** - * @description The task instance's DAG run ID. Either set this or execution_date but not both. + * @description The task instance's DAG run ID. Either set this or logical_date but not both. * * *New in version 2.3.0* */ diff --git a/airflow/www/static/js/types/index.ts b/airflow/www/static/js/types/index.ts index 13ee62966214..618706a9e350 100644 --- a/airflow/www/static/js/types/index.ts +++ b/airflow/www/static/js/types/index.ts @@ -56,7 +56,7 @@ interface DagRun { runId: string; runType: "manual" | "backfill" | "scheduled" | "asset_triggered"; state: RunState; - executionDate: string; + logicalDate: string; dataIntervalStart: string; dataIntervalEnd: string; queuedAt: string | null; @@ -109,11 +109,7 @@ interface Task { setupTeardownType?: "setup" | "teardown"; } -type RunOrdering = ( - | "dataIntervalStart" - | "executionDate" - | "dataIntervalEnd" -)[]; +type RunOrdering = ("dataIntervalStart" | "logicalDate" | "dataIntervalEnd")[]; export interface MidEdge { id: string; diff --git a/airflow/www/static/js/utils/index.test.ts b/airflow/www/static/js/utils/index.test.ts index 569d3af98b53..4e367dc08f8c 100644 --- a/airflow/www/static/js/utils/index.test.ts +++ b/airflow/www/static/js/utils/index.test.ts @@ -139,7 +139,7 @@ describe("Test getDagRunLabel", () => { endDate: "2021-11-08T21:17:13.206426+00:00", state: "failed", runType: "scheduled", - executionDate: "2021-12-09T21:14:19.704433+00:00", + logicalDate: "2021-12-09T21:14:19.704433+00:00", lastSchedulingDecision: "2021-11-08T21:14:19.704433+00:00", externalTrigger: false, conf: null, @@ -148,7 +148,7 @@ describe("Test getDagRunLabel", () => { test("Defaults to executionDate", async () => { const runLabel = getDagRunLabel({ dagRun }); - expect(runLabel).toBe(dagRun.executionDate); + expect(runLabel).toBe(dagRun.logicalDate); }); test("Passing an order overrides default", async () => { diff --git a/airflow/www/static/js/utils/index.ts b/airflow/www/static/js/utils/index.ts index 8bef31a8582a..cbebc09c98c1 100644 --- a/airflow/www/static/js/utils/index.ts +++ b/airflow/www/static/js/utils/index.ts @@ -169,7 +169,7 @@ interface RunLabelProps { const getDagRunLabel = ({ dagRun, - ordering = ["executionDate"], + ordering = ["logicalDate"], }: RunLabelProps) => dagRun[ordering[0]]; const getStatusBackgroundColor = (color: string, hasNote: boolean) => diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index cd212d6620aa..6969993e9199 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -50,10 +50,10 @@ {% set dag_is_paused = dag.get_is_paused() %} {% set base_date_arg = request.args.get('base_date') %} {% set num_runs_arg = request.args.get('num_runs') %} -{% if execution_date is defined %} - {% set execution_date_arg = execution_date %} +{% if logical_date is defined %} + {% set execution_date_arg = logical_date %} {% else %} - {% set execution_date_arg = request.args.get('execution_date') %} + {% set execution_date_arg = request.args.get('logical_date') %} {% endif %} {% block head_meta %} diff --git a/airflow/www/templates/airflow/task_instance.html b/airflow/www/templates/airflow/task_instance.html index 9c5953ae28e8..90e468cc37c5 100644 --- a/airflow/www/templates/airflow/task_instance.html +++ b/airflow/www/templates/airflow/task_instance.html @@ -26,7 +26,7 @@

Task Instance: {{ task_display_name }} - at + at {% if map_index is defined and map_index >= 0 %} Map Index: {{ map_index }} {% endif %} @@ -34,9 +34,9 @@