Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion .github/boring-cyborg.yml
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,6 @@ labelPRBasedOnFilePath:
- airflow/operators/**/*
- airflow/hooks/**/*
- airflow/sensors/**/*
- tests/operators/**/*
- tests/hooks/**/*
- tests/sensors/**/*
- docs/apache-airflow/operators-and-hooks-ref.rst
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
Expand All @@ -15,3 +14,25 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import annotations

from pydantic import Field

from airflow.api_fastapi.common.types import UtcDateTime
from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
from airflow.utils.state import DagRunState


class TriggerDAGRunPayload(StrictBaseModel):
"""Schema for Trigger DAG Run API request."""

logical_date: UtcDateTime | None = None
conf: dict = Field(default_factory=dict)
reset_dag_run: bool = False


class DagRunStateResponse(BaseModel):
"""Schema for DAG Run State response."""

state: DagRunState
2 changes: 2 additions & 0 deletions airflow/api_fastapi/execution_api/routes/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
asset_events,
assets,
connections,
dag_runs,
health,
task_instances,
variables,
Expand All @@ -38,6 +39,7 @@
authenticated_router.include_router(assets.router, prefix="/assets", tags=["Assets"])
authenticated_router.include_router(asset_events.router, prefix="/asset-events", tags=["Asset Events"])
authenticated_router.include_router(connections.router, prefix="/connections", tags=["Connections"])
authenticated_router.include_router(dag_runs.router, prefix="/dag-runs", tags=["Dag Runs"])
authenticated_router.include_router(task_instances.router, prefix="/task-instances", tags=["Task Instances"])
authenticated_router.include_router(variables.router, prefix="/variables", tags=["Variables"])
authenticated_router.include_router(xcoms.router, prefix="/xcoms", tags=["XComs"])
Expand Down
152 changes: 152 additions & 0 deletions airflow/api_fastapi/execution_api/routes/dag_runs.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import annotations

import logging

from fastapi import HTTPException, status
from sqlalchemy import select

from airflow.api.common.trigger_dag import trigger_dag
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.execution_api.datamodels.dagrun import DagRunStateResponse, TriggerDAGRunPayload
from airflow.exceptions import DagRunAlreadyExists
from airflow.models.dag import DagModel
from airflow.models.dagbag import DagBag
from airflow.models.dagrun import DagRun
from airflow.utils.types import DagRunTriggeredByType

router = AirflowRouter()


log = logging.getLogger(__name__)


@router.post(
"/{dag_id}/{run_id}",
status_code=status.HTTP_204_NO_CONTENT,
responses={
status.HTTP_400_BAD_REQUEST: {"description": "DAG has import errors and cannot be triggered"},
status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"},
status.HTTP_409_CONFLICT: {"description": "DAG Run already exists for the given dag_id"},
status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload"},
},
)
def trigger_dag_run(
dag_id: str,
run_id: str,
payload: TriggerDAGRunPayload,
session: SessionDep,
):
"""Trigger a DAG Run."""
dm = session.scalar(select(DagModel).where(DagModel.is_active, DagModel.dag_id == dag_id).limit(1))
if not dm:
raise HTTPException(
status.HTTP_404_NOT_FOUND,
detail={"reason": "not_found", "message": f"DAG with dag_id: '{dag_id}' not found"},
)

if dm.has_import_errors:
raise HTTPException(
status.HTTP_400_BAD_REQUEST,
detail={
"reason": "import_errors",
"message": f"DAG with dag_id: '{dag_id}' has import errors and cannot be triggered",
},
)

try:
trigger_dag(
dag_id=dag_id,
run_id=run_id,
conf=payload.conf,
logical_date=payload.logical_date,
triggered_by=DagRunTriggeredByType.OPERATOR,
replace_microseconds=False,
session=session,
)
except DagRunAlreadyExists:
raise HTTPException(
status.HTTP_409_CONFLICT,
detail={
"reason": "already_exists",
"message": f"A DAG Run already exists for DAG {dag_id} with run id {run_id}",
},
)


@router.post(
"/{dag_id}/{run_id}/clear",
status_code=status.HTTP_204_NO_CONTENT,
responses={
status.HTTP_400_BAD_REQUEST: {"description": "DAG has import errors and cannot be triggered"},
status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"},
status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload"},
},
)
def clear_dag_run(
dag_id: str,
run_id: str,
session: SessionDep,
):
"""Clear a DAG Run."""
dm = session.scalar(select(DagModel).where(DagModel.is_active, DagModel.dag_id == dag_id).limit(1))
if not dm:
raise HTTPException(
status.HTTP_404_NOT_FOUND,
detail={"reason": "not_found", "message": f"DAG with dag_id: '{dag_id}' not found"},
)

if dm.has_import_errors:
raise HTTPException(
status.HTTP_400_BAD_REQUEST,
detail={
"reason": "import_errors",
"message": f"DAG with dag_id: '{dag_id}' has import errors and cannot be triggered",
},
)

dag_bag = DagBag(dag_folder=dm.fileloc, read_dags_from_db=True)
dag = dag_bag.get_dag(dag_id)
dag.clear(run_id=run_id)


@router.get(
"/{dag_id}/{run_id}/state",
responses={
status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"},
},
)
def get_dagrun_state(
dag_id: str,
run_id: str,
session: SessionDep,
) -> DagRunStateResponse:
"""Get a DAG Run State."""
dag_run = session.scalar(select(DagRun).where(DagRun.dag_id == dag_id, DagRun.run_id == run_id))
if dag_run is None:
raise HTTPException(
status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
"message": f"The DagRun with dag_id: `{dag_id}` and run_id: `{run_id}` was not found",
},
)

return DagRunStateResponse(state=dag_run.state)
38 changes: 37 additions & 1 deletion airflow/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@

import warnings
from collections.abc import Collection, Sequence
from datetime import timedelta
from datetime import datetime, timedelta
from http import HTTPStatus
from typing import TYPE_CHECKING, Any, NamedTuple

Expand All @@ -34,6 +34,7 @@

from airflow.models import DagRun
from airflow.sdk.definitions.asset import AssetNameRef, AssetUniqueKey, AssetUriRef
from airflow.utils.state import DagRunState


class AirflowException(Exception):
Expand Down Expand Up @@ -418,6 +419,41 @@ def __init__(self, *, tasks: Sequence[str | tuple[str, int]]):
self.tasks = tasks


class DagRunTriggerException(AirflowException):
"""
Signal by an operator to trigger a specific Dag Run of a dag.

Special exception raised to signal that the operator it was raised from wishes to trigger
a specific Dag Run of a dag. This is used in the ``TriggerDagRunOperator``.
"""

def __init__(
self,
*,
trigger_dag_id: str,
dag_run_id: str,
conf: dict | None,
logical_date: datetime | None,
reset_dag_run: bool,
skip_when_already_exists: bool,
wait_for_completion: bool,
allowed_states: list[str | DagRunState],
failed_states: list[str | DagRunState],
poke_interval: int,
):
super().__init__()
self.trigger_dag_id = trigger_dag_id
self.dag_run_id = dag_run_id
self.conf = conf
self.logical_date = logical_date
self.reset_dag_run = reset_dag_run
self.skip_when_already_exists = skip_when_already_exists
self.wait_for_completion = wait_for_completion
self.allowed_states = allowed_states
self.failed_states = failed_states
self.poke_interval = poke_interval


class TaskDeferred(BaseException):
"""
Signal an operator moving to deferred state.
Expand Down
2 changes: 1 addition & 1 deletion dev/breeze/doc/ci/04_selective_checks.md
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ together using `pytest-xdist` (pytest-xdist distributes the tests among parallel
* if there are any changes to "common" provider code not belonging to any provider (usually system tests
or tests), then tests for all Providers are run
* The specific unit test type is enabled only if changed files match the expected patterns for each type
(`API`, `CLI`, `WWW`, `Providers`, `Operators` etc.). The `Always` test type is added always if any unit
(`API`, `CLI`, `WWW`, `Providers` etc.). The `Always` test type is added always if any unit
tests are run. `Providers` tests are removed if current branch is different than `main`
* If there are no files left in sources after matching the test types and Kubernetes files,
then apparently some Core/Other files have been changed. This automatically adds all test
Expand Down
4 changes: 2 additions & 2 deletions dev/breeze/doc/images/output-commands.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading