Skip to content

Commit 7467a22

Browse files
committed
Migrate public endpoint Get Task to FastAPI, with main resynced
1 parent f57db71 commit 7467a22

File tree

14 files changed

+1316
-1
lines changed

14 files changed

+1316
-1
lines changed

airflow/api_connexion/endpoints/task_endpoint.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,12 +25,14 @@
2525
from airflow.auth.managers.models.resource_details import DagAccessEntity
2626
from airflow.exceptions import TaskNotFound
2727
from airflow.utils.airflow_flask_app import get_airflow_app
28+
from airflow.utils.api_migration import mark_fastapi_migration_done
2829

2930
if TYPE_CHECKING:
3031
from airflow import DAG
3132
from airflow.api_connexion.types import APIResponse
3233

3334

35+
@mark_fastapi_migration_done
3436
@security.requires_access_dag("GET", DagAccessEntity.TASK)
3537
def get_task(*, dag_id: str, task_id: str) -> APIResponse:
3638
"""Get simplified representation of a task."""

airflow/api_fastapi/common/types.py

Lines changed: 63 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,10 +16,72 @@
1616
# under the License.
1717
from __future__ import annotations
1818

19-
from pydantic import AfterValidator, AwareDatetime
19+
import inspect
20+
from datetime import timedelta
21+
22+
from pydantic import AfterValidator, AliasGenerator, AwareDatetime, BaseModel, BeforeValidator, ConfigDict
2023
from typing_extensions import Annotated
2124

25+
from airflow.models.mappedoperator import MappedOperator
26+
from airflow.models.operator import Operator
27+
from airflow.serialization.serialized_objects import SerializedBaseOperator
2228
from airflow.utils import timezone
2329

2430
UtcDateTime = Annotated[AwareDatetime, AfterValidator(lambda d: d.astimezone(timezone.utc))]
2531
"""UTCDateTime is a datetime with timezone information"""
32+
33+
34+
def _validate_timedelta_field(td: timedelta | None) -> TimeDelta | None:
35+
"""Validate the execution_timeout property."""
36+
if td is None:
37+
return None
38+
return TimeDelta(
39+
days=td.days,
40+
seconds=td.seconds,
41+
microseconds=td.microseconds,
42+
)
43+
44+
45+
class TimeDelta(BaseModel):
46+
"""TimeDelta can be used to interact with datetime.timedelta objects."""
47+
48+
object_type: str = "TimeDelta"
49+
days: int
50+
seconds: int
51+
microseconds: int
52+
53+
model_config = ConfigDict(
54+
alias_generator=AliasGenerator(
55+
serialization_alias=lambda field_name: {
56+
"object_type": "__type",
57+
}.get(field_name, field_name),
58+
)
59+
)
60+
61+
62+
TimeDeltaWithValidation = Annotated[TimeDelta, BeforeValidator(_validate_timedelta_field)]
63+
64+
65+
def get_class_ref(obj: Operator) -> dict[str, str | None]:
66+
"""Return the class_ref dict for obj."""
67+
is_mapped_or_serialized = isinstance(obj, (MappedOperator, SerializedBaseOperator))
68+
69+
module_path = None
70+
if is_mapped_or_serialized:
71+
module_path = obj._task_module
72+
else:
73+
module_type = inspect.getmodule(obj)
74+
module_path = module_type.__name__ if module_type else None
75+
76+
class_name = None
77+
if is_mapped_or_serialized:
78+
class_name = obj._task_type
79+
elif obj.__class__ is type:
80+
class_name = obj.__name__
81+
else:
82+
class_name = type(obj).__name__
83+
84+
return {
85+
"module_path": module_path,
86+
"class_name": class_name,
87+
}

airflow/api_fastapi/core_api/openapi/v1-generated.yaml

Lines changed: 246 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3104,6 +3104,62 @@ paths:
31043104
application/json:
31053105
schema:
31063106
$ref: '#/components/schemas/HTTPValidationError'
3107+
/public/dags/{dag_id}/tasks/{task_id}:
3108+
get:
3109+
tags:
3110+
- Task
3111+
summary: Get Task
3112+
description: Get simplified representation of a task.
3113+
operationId: get_task
3114+
parameters:
3115+
- name: dag_id
3116+
in: path
3117+
required: true
3118+
schema:
3119+
type: string
3120+
title: Dag Id
3121+
- name: task_id
3122+
in: path
3123+
required: true
3124+
schema:
3125+
title: Task Id
3126+
responses:
3127+
'200':
3128+
description: Successful Response
3129+
content:
3130+
application/json:
3131+
schema:
3132+
$ref: '#/components/schemas/TaskResponse'
3133+
'400':
3134+
content:
3135+
application/json:
3136+
schema:
3137+
$ref: '#/components/schemas/HTTPExceptionResponse'
3138+
description: Bad Request
3139+
'401':
3140+
content:
3141+
application/json:
3142+
schema:
3143+
$ref: '#/components/schemas/HTTPExceptionResponse'
3144+
description: Unauthorized
3145+
'403':
3146+
content:
3147+
application/json:
3148+
schema:
3149+
$ref: '#/components/schemas/HTTPExceptionResponse'
3150+
description: Forbidden
3151+
'404':
3152+
content:
3153+
application/json:
3154+
schema:
3155+
$ref: '#/components/schemas/HTTPExceptionResponse'
3156+
description: Not Found
3157+
'422':
3158+
description: Validation Error
3159+
content:
3160+
application/json:
3161+
schema:
3162+
$ref: '#/components/schemas/HTTPValidationError'
31073163
components:
31083164
schemas:
31093165
AppBuilderMenuItemResponse:
@@ -4913,6 +4969,196 @@ components:
49134969
- triggerer_job
49144970
title: TaskInstanceResponse
49154971
description: TaskInstance serializer for responses.
4972+
TaskResponse:
4973+
properties:
4974+
task_id:
4975+
anyOf:
4976+
- type: string
4977+
- type: 'null'
4978+
title: Task Id
4979+
task_display_name:
4980+
anyOf:
4981+
- type: string
4982+
- type: 'null'
4983+
title: Task Display Name
4984+
owner:
4985+
anyOf:
4986+
- type: string
4987+
- type: 'null'
4988+
title: Owner
4989+
start_date:
4990+
anyOf:
4991+
- type: string
4992+
format: date-time
4993+
- type: 'null'
4994+
title: Start Date
4995+
end_date:
4996+
anyOf:
4997+
- type: string
4998+
format: date-time
4999+
- type: 'null'
5000+
title: End Date
5001+
trigger_rule:
5002+
anyOf:
5003+
- type: string
5004+
- type: 'null'
5005+
title: Trigger Rule
5006+
depends_on_past:
5007+
type: boolean
5008+
title: Depends On Past
5009+
wait_for_downstream:
5010+
type: boolean
5011+
title: Wait For Downstream
5012+
retries:
5013+
anyOf:
5014+
- type: number
5015+
- type: 'null'
5016+
title: Retries
5017+
queue:
5018+
anyOf:
5019+
- type: string
5020+
- type: 'null'
5021+
title: Queue
5022+
pool:
5023+
anyOf:
5024+
- type: string
5025+
- type: 'null'
5026+
title: Pool
5027+
pool_slots:
5028+
anyOf:
5029+
- type: number
5030+
- type: 'null'
5031+
title: Pool Slots
5032+
execution_timeout:
5033+
anyOf:
5034+
- $ref: '#/components/schemas/TimeDelta'
5035+
- type: 'null'
5036+
retry_delay:
5037+
anyOf:
5038+
- $ref: '#/components/schemas/TimeDelta'
5039+
- type: 'null'
5040+
retry_exponential_backoff:
5041+
type: boolean
5042+
title: Retry Exponential Backoff
5043+
priority_weight:
5044+
anyOf:
5045+
- type: number
5046+
- type: 'null'
5047+
title: Priority Weight
5048+
weight_rule:
5049+
anyOf:
5050+
- type: string
5051+
- type: 'null'
5052+
title: Weight Rule
5053+
ui_color:
5054+
anyOf:
5055+
- type: string
5056+
- type: 'null'
5057+
title: Ui Color
5058+
ui_fgcolor:
5059+
anyOf:
5060+
- type: string
5061+
- type: 'null'
5062+
title: Ui Fgcolor
5063+
template_fields:
5064+
anyOf:
5065+
- items:
5066+
type: string
5067+
type: array
5068+
- type: 'null'
5069+
title: Template Fields
5070+
downstream_task_ids:
5071+
anyOf:
5072+
- items:
5073+
type: string
5074+
type: array
5075+
- type: 'null'
5076+
title: Downstream Task Ids
5077+
doc_md:
5078+
anyOf:
5079+
- type: string
5080+
- type: 'null'
5081+
title: Doc Md
5082+
operator_name:
5083+
anyOf:
5084+
- type: string
5085+
- type: 'null'
5086+
title: Operator Name
5087+
params:
5088+
anyOf:
5089+
- type: object
5090+
- type: 'null'
5091+
title: Params
5092+
class_ref:
5093+
anyOf:
5094+
- type: object
5095+
- type: 'null'
5096+
title: Class Ref
5097+
is_mapped:
5098+
anyOf:
5099+
- type: boolean
5100+
- type: 'null'
5101+
title: Is Mapped
5102+
extra_links:
5103+
items:
5104+
type: string
5105+
type: array
5106+
title: Extra Links
5107+
description: Extract and return extra_links.
5108+
readOnly: true
5109+
type: object
5110+
required:
5111+
- task_id
5112+
- task_display_name
5113+
- owner
5114+
- start_date
5115+
- end_date
5116+
- trigger_rule
5117+
- depends_on_past
5118+
- wait_for_downstream
5119+
- retries
5120+
- queue
5121+
- pool
5122+
- pool_slots
5123+
- execution_timeout
5124+
- retry_delay
5125+
- retry_exponential_backoff
5126+
- priority_weight
5127+
- weight_rule
5128+
- ui_color
5129+
- ui_fgcolor
5130+
- template_fields
5131+
- downstream_task_ids
5132+
- doc_md
5133+
- operator_name
5134+
- params
5135+
- class_ref
5136+
- is_mapped
5137+
- extra_links
5138+
title: TaskResponse
5139+
description: Task serializer for responses.
5140+
TimeDelta:
5141+
properties:
5142+
__type:
5143+
type: string
5144+
title: ' Type'
5145+
default: TimeDelta
5146+
days:
5147+
type: integer
5148+
title: Days
5149+
seconds:
5150+
type: integer
5151+
title: Seconds
5152+
microseconds:
5153+
type: integer
5154+
title: Microseconds
5155+
type: object
5156+
required:
5157+
- days
5158+
- seconds
5159+
- microseconds
5160+
title: TimeDelta
5161+
description: TimeDelta can be used to interact with datetime.timedelta objects.
49165162
TriggerResponse:
49175163
properties:
49185164
id:

airflow/api_fastapi/core_api/routes/public/__init__.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
from airflow.api_fastapi.core_api.routes.public.pools import pools_router
3333
from airflow.api_fastapi.core_api.routes.public.providers import providers_router
3434
from airflow.api_fastapi.core_api.routes.public.task_instances import task_instances_router
35+
from airflow.api_fastapi.core_api.routes.public.tasks import tasks_router
3536
from airflow.api_fastapi.core_api.routes.public.variables import variables_router
3637
from airflow.api_fastapi.core_api.routes.public.version import version_router
3738

@@ -56,3 +57,5 @@
5657
public_router.include_router(variables_router)
5758
public_router.include_router(version_router)
5859
public_router.include_router(dag_stats_router)
60+
public_router.include_router(plugins_router)
61+
public_router.include_router(tasks_router)

0 commit comments

Comments
 (0)