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
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
from dataclasses import dataclass
from enum import Enum

from pydantic import NonNegativeInt


@dataclass
class ConfigurationDetails:
Expand All @@ -46,7 +48,7 @@ class DagDetails:
class BackfillDetails:
"""Represents the details of a backfill."""

id: str | None = None
id: NonNegativeInt | None = None


@dataclass
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

from datetime import datetime

from pydantic import NonNegativeInt

from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
from airflow.models.backfill import ReprocessBehavior

Expand All @@ -38,7 +40,7 @@ class BackfillPostBody(StrictBaseModel):
class BackfillResponse(BaseModel):
"""Base serializer for Backfill."""

id: int
id: NonNegativeInt
dag_id: str
from_date: datetime
to_date: datetime
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,7 @@ components:
properties:
id:
type: integer
minimum: 0.0
title: Id
dag_id:
type: string
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -951,7 +951,8 @@ paths:
in: path
required: true
schema:
type: string
type: integer
minimum: 0
title: Backfill Id
responses:
'200':
Expand Down Expand Up @@ -997,6 +998,8 @@ paths:
in: path
required: true
schema:
type: integer
minimum: 0
title: Backfill Id
responses:
'200':
Expand Down Expand Up @@ -1048,6 +1051,8 @@ paths:
in: path
required: true
schema:
type: integer
minimum: 0
title: Backfill Id
responses:
'200':
Expand Down Expand Up @@ -1099,6 +1104,8 @@ paths:
in: path
required: true
schema:
type: integer
minimum: 0
title: Backfill Id
responses:
'200':
Expand Down Expand Up @@ -7145,6 +7152,7 @@ components:
properties:
id:
type: integer
minimum: 0.0
title: Id
dag_id:
type: string
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

from fastapi import Depends, HTTPException, status
from fastapi.exceptions import RequestValidationError
from pydantic import NonNegativeInt
from sqlalchemy import select, update

from airflow.api_fastapi.auth.managers.models.resource_details import DagAccessEntity
Expand Down Expand Up @@ -98,7 +99,7 @@ def list_backfills(
],
)
def get_backfill(
backfill_id: str,
backfill_id: NonNegativeInt,
session: SessionDep,
) -> BackfillResponse:
backfill = session.get(Backfill, backfill_id)
Expand All @@ -121,7 +122,7 @@ def get_backfill(
Depends(requires_access_dag(method="PUT", access_entity=DagAccessEntity.RUN)),
],
)
def pause_backfill(backfill_id, session: SessionDep) -> BackfillResponse:
def pause_backfill(backfill_id: NonNegativeInt, session: SessionDep) -> BackfillResponse:
b = session.get(Backfill, backfill_id)
if not b:
raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}")
Expand All @@ -147,7 +148,7 @@ def pause_backfill(backfill_id, session: SessionDep) -> BackfillResponse:
Depends(requires_access_dag(method="PUT", access_entity=DagAccessEntity.RUN)),
],
)
def unpause_backfill(backfill_id, session: SessionDep) -> BackfillResponse:
def unpause_backfill(backfill_id: NonNegativeInt, session: SessionDep) -> BackfillResponse:
b = session.get(Backfill, backfill_id)
if not b:
raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}")
Expand All @@ -172,7 +173,7 @@ def unpause_backfill(backfill_id, session: SessionDep) -> BackfillResponse:
Depends(requires_access_backfill(method="PUT")),
],
)
def cancel_backfill(backfill_id, session: SessionDep) -> BackfillResponse:
def cancel_backfill(backfill_id: NonNegativeInt, session: SessionDep) -> BackfillResponse:
b: Backfill = session.get(Backfill, backfill_id)
if not b:
raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}")
Expand Down
3 changes: 2 additions & 1 deletion airflow-core/src/airflow/api_fastapi/core_api/security.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
from fastapi import Depends, HTTPException, Request, status
from fastapi.security import OAuth2PasswordBearer
from jwt import ExpiredSignatureError, InvalidTokenError
from pydantic import NonNegativeInt

from airflow.api_fastapi.app import get_auth_manager
from airflow.api_fastapi.auth.managers.models.base_user import BaseUser
Expand Down Expand Up @@ -198,7 +199,7 @@ def inner(
request: Request,
user: Annotated[BaseUser | None, Depends(get_user)] = None,
) -> None:
backfill_id: str | None = request.path_params.get("backfill_id")
backfill_id: NonNegativeInt | None = request.path_params.get("backfill_id")

_requires_access(
is_authorized_callback=lambda: get_auth_manager().is_authorized_backfill(
Expand Down
5 changes: 3 additions & 2 deletions airflow-core/src/airflow/models/dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@
from airflow.utils.types import DagRunTriggeredByType, DagRunType

if TYPE_CHECKING:
from pydantic import NonNegativeInt
from sqlalchemy.orm.query import Query
from sqlalchemy.orm.session import Session

Expand Down Expand Up @@ -257,7 +258,7 @@ def _create_orm_dagrun(
state: DagRunState | None,
run_type: DagRunType,
creating_job_id: int | None,
backfill_id: int | None,
backfill_id: NonNegativeInt | None,
triggered_by: DagRunTriggeredByType,
session: Session = NEW_SESSION,
) -> DagRun:
Expand Down Expand Up @@ -1795,7 +1796,7 @@ def create_dagrun(
state: DagRunState,
start_date: datetime | None = None,
creating_job_id: int | None = None,
backfill_id: int | None = None,
backfill_id: NonNegativeInt | None = None,
session: Session = NEW_SESSION,
) -> DagRun:
"""
Expand Down
3 changes: 2 additions & 1 deletion airflow-core/src/airflow/models/dagrun.py
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@
from datetime import datetime

from opentelemetry.sdk.trace import Span
from pydantic import NonNegativeInt
from sqlalchemy.orm import Query, Session

from airflow.models.baseoperator import BaseOperator
Expand Down Expand Up @@ -290,7 +291,7 @@ def __init__(
creating_job_id: int | None = None,
data_interval: tuple[datetime, datetime] | None = None,
triggered_by: DagRunTriggeredByType | None = None,
backfill_id: int | None = None,
backfill_id: NonNegativeInt | None = None,
bundle_version: str | None = None,
):
# For manual runs where logical_date is None, ensure no data_interval is set.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ export const UseBackfillServiceGetBackfillKeyFn = (
{
backfillId,
}: {
backfillId: string;
backfillId: number;
},
queryKey?: Array<unknown>,
) => [useBackfillServiceGetBackfillKey, ...(queryKey ?? [{ backfillId }])];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -354,7 +354,7 @@ export const ensureUseBackfillServiceGetBackfillData = (
{
backfillId,
}: {
backfillId: string;
backfillId: number;
},
) =>
queryClient.ensureQueryData({
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -354,7 +354,7 @@ export const prefetchUseBackfillServiceGetBackfill = (
{
backfillId,
}: {
backfillId: string;
backfillId: number;
},
) =>
queryClient.prefetchQuery({
Expand Down
14 changes: 7 additions & 7 deletions airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts
Original file line number Diff line number Diff line change
Expand Up @@ -446,7 +446,7 @@ export const useBackfillServiceGetBackfill = <
{
backfillId,
}: {
backfillId: string;
backfillId: number;
},
queryKey?: TQueryKey,
options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">,
Expand Down Expand Up @@ -3640,7 +3640,7 @@ export const useBackfillServicePauseBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>,
Expand All @@ -3651,7 +3651,7 @@ export const useBackfillServicePauseBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>({
Expand All @@ -3676,7 +3676,7 @@ export const useBackfillServiceUnpauseBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>,
Expand All @@ -3687,7 +3687,7 @@ export const useBackfillServiceUnpauseBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>({
Expand All @@ -3712,7 +3712,7 @@ export const useBackfillServiceCancelBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>,
Expand All @@ -3723,7 +3723,7 @@ export const useBackfillServiceCancelBackfill = <
TData,
TError,
{
backfillId: unknown;
backfillId: number;
},
TContext
>({
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -423,7 +423,7 @@ export const useBackfillServiceGetBackfillSuspense = <
{
backfillId,
}: {
backfillId: string;
backfillId: number;
},
queryKey?: TQueryKey,
options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -427,6 +427,7 @@ export const $BackfillResponse = {
properties: {
id: {
type: "integer",
minimum: 0,
title: "Id",
},
dag_id: {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1882,25 +1882,25 @@ export type CreateBackfillData = {
export type CreateBackfillResponse = BackfillResponse;

export type GetBackfillData = {
backfillId: string;
backfillId: number;
};

export type GetBackfillResponse = BackfillResponse;

export type PauseBackfillData = {
backfillId: unknown;
backfillId: number;
};

export type PauseBackfillResponse = BackfillResponse;

export type UnpauseBackfillData = {
backfillId: unknown;
backfillId: number;
};

export type UnpauseBackfillResponse = BackfillResponse;

export type CancelBackfillData = {
backfillId: unknown;
backfillId: number;
};

export type CancelBackfillResponse = BackfillResponse;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,15 +65,21 @@ const BackfillBanner = ({ dagId }: Props) => {
const { isPending: isStopPending, mutate: stopPending } = useBackfillServiceCancelBackfill({ onSuccess });

const togglePause = () => {
if (backfill?.is_paused) {
if (backfill === undefined) {
return;
}
if (backfill.is_paused) {
unpauseMutate({ backfillId: backfill.id });
} else {
pauseMutate({ backfillId: backfill?.id });
pauseMutate({ backfillId: backfill.id });
}
};

const cancel = () => {
stopPending({ backfillId: backfill?.id });
if (backfill === undefined) {
return;
}
stopPending({ backfillId: backfill.id });
};

if (isLoading || backfill === undefined) {
Expand Down
Loading
Loading