Skip to content

Commit

Permalink
Migrate public endpoint Get Task to FastAPI
Browse files Browse the repository at this point in the history
  • Loading branch information
omkar-foss committed Nov 5, 2024
1 parent 426dba0 commit 1f75b79
Show file tree
Hide file tree
Showing 12 changed files with 1,322 additions and 0 deletions.
246 changes: 246 additions & 0 deletions airflow/api_fastapi/core_api/openapi/v1-generated.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -2515,6 +2515,62 @@ paths:
application/json:
schema:
$ref: '#/components/schemas/HTTPValidationError'
/public/dags/{dag_id}/tasks/{task_id}:
get:
tags:
- Task
summary: Get Task
description: Get simplified representation of a task.
operationId: get_task
parameters:
- name: dag_id
in: path
required: true
schema:
type: string
title: Dag Id
- name: task_id
in: path
required: true
schema:
title: Task Id
responses:
'200':
description: Successful Response
content:
application/json:
schema:
$ref: '#/components/schemas/TaskResponse'
'400':
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
description: Bad Request
'401':
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
description: Unauthorized
'403':
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
description: Forbidden
'404':
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
description: Not Found
'422':
description: Validation Error
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPValidationError'
components:
schemas:
AppBuilderMenuItemResponse:
Expand Down Expand Up @@ -4276,6 +4332,196 @@ components:
- triggerer_job
title: TaskInstanceResponse
description: TaskInstance serializer for responses.
TaskResponse:
properties:
task_id:
anyOf:
- type: string
- type: 'null'
title: Task Id
task_display_name:
anyOf:
- type: string
- type: 'null'
title: Task Display Name
owner:
anyOf:
- type: string
- type: 'null'
title: Owner
start_date:
anyOf:
- type: string
format: date-time
- type: 'null'
title: Start Date
end_date:
anyOf:
- type: string
format: date-time
- type: 'null'
title: End Date
trigger_rule:
anyOf:
- type: string
- type: 'null'
title: Trigger Rule
depends_on_past:
type: boolean
title: Depends On Past
wait_for_downstream:
type: boolean
title: Wait For Downstream
retries:
anyOf:
- type: number
- type: 'null'
title: Retries
queue:
anyOf:
- type: string
- type: 'null'
title: Queue
pool:
anyOf:
- type: string
- type: 'null'
title: Pool
pool_slots:
anyOf:
- type: number
- type: 'null'
title: Pool Slots
execution_timeout:
anyOf:
- $ref: '#/components/schemas/TimeDeltaResponse'
- type: 'null'
retry_delay:
anyOf:
- $ref: '#/components/schemas/TimeDeltaResponse'
- type: 'null'
retry_exponential_backoff:
type: boolean
title: Retry Exponential Backoff
priority_weight:
anyOf:
- type: number
- type: 'null'
title: Priority Weight
weight_rule:
anyOf:
- type: string
- type: 'null'
title: Weight Rule
ui_color:
anyOf:
- type: string
- type: 'null'
title: Ui Color
ui_fgcolor:
anyOf:
- type: string
- type: 'null'
title: Ui Fgcolor
template_fields:
anyOf:
- items:
type: string
type: array
- type: 'null'
title: Template Fields
downstream_task_ids:
anyOf:
- items:
type: string
type: array
- type: 'null'
title: Downstream Task Ids
doc_md:
anyOf:
- type: string
- type: 'null'
title: Doc Md
operator_name:
anyOf:
- type: string
- type: 'null'
title: Operator Name
params:
anyOf:
- type: object
- type: 'null'
title: Params
class_ref:
anyOf:
- type: object
- type: 'null'
title: Class Ref
is_mapped:
anyOf:
- type: boolean
- type: 'null'
title: Is Mapped
extra_links:
items:
type: string
type: array
title: Extra Links
description: Extract and return extra_links.
readOnly: true
type: object
required:
- task_id
- task_display_name
- owner
- start_date
- end_date
- trigger_rule
- depends_on_past
- wait_for_downstream
- retries
- queue
- pool
- pool_slots
- execution_timeout
- retry_delay
- retry_exponential_backoff
- priority_weight
- weight_rule
- ui_color
- ui_fgcolor
- template_fields
- downstream_task_ids
- doc_md
- operator_name
- params
- class_ref
- is_mapped
- extra_links
title: TaskResponse
description: Task serializer for responses.
TimeDeltaResponse:
properties:
__type:
type: string
title: ' Type'
default: TimeDelta
days:
type: integer
title: Days
seconds:
type: integer
title: Seconds
microseconds:
type: integer
title: Microseconds
type: object
required:
- days
- seconds
- microseconds
title: TimeDeltaResponse
description: Time delta serializer for responses.
TriggerResponse:
properties:
id:
Expand Down
3 changes: 3 additions & 0 deletions airflow/api_fastapi/core_api/routes/public/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
from airflow.api_fastapi.core_api.routes.public.pools import pools_router
from airflow.api_fastapi.core_api.routes.public.providers import providers_router
from airflow.api_fastapi.core_api.routes.public.task_instances import task_instances_router
from airflow.api_fastapi.core_api.routes.public.tasks import tasks_router
from airflow.api_fastapi.core_api.routes.public.variables import variables_router
from airflow.api_fastapi.core_api.routes.public.version import version_router

Expand All @@ -56,3 +57,5 @@
public_router.include_router(variables_router)
public_router.include_router(version_router)
public_router.include_router(dag_stats_router)
public_router.include_router(plugins_router)
public_router.include_router(tasks_router)
54 changes: 54 additions & 0 deletions airflow/api_fastapi/core_api/routes/public/tasks.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import annotations

from fastapi import HTTPException, Request, status

from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc
from airflow.api_fastapi.core_api.serializers.tasks import TaskResponse
from airflow.exceptions import TaskNotFound
from airflow.models import DAG

tasks_router = AirflowRouter(tags=["Task"], prefix="/dags")


@tasks_router.get(
"/{dag_id}/tasks/{task_id}",
responses=create_openapi_http_exception_doc(
[
status.HTTP_400_BAD_REQUEST,
status.HTTP_401_UNAUTHORIZED,
status.HTTP_403_FORBIDDEN,
status.HTTP_404_NOT_FOUND,
]
),
)
async def get_task(dag_id: str, task_id, request: Request) -> TaskResponse:
"""Get simplified representation of a task."""
dag: DAG = request.app.state.dag_bag.get_dag(dag_id)
if not dag:
raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id {dag_id} was not found")
try:
task = dag.get_task(task_id=task_id)
except TaskNotFound:
raise HTTPException(status.HTTP_404_NOT_FOUND, f"Task with id {task_id} was not found")
return TaskResponse.model_validate(
TaskResponse.assign_type_info(task),
from_attributes=True,
)
Loading

0 comments on commit 1f75b79

Please sign in to comment.