Skip to content

Commit

Permalink
refactor(test): move test to routers/public folder
Browse files Browse the repository at this point in the history
  • Loading branch information
jason810496 committed Oct 18, 2024
1 parent 0bb6154 commit 16f0a6d
Show file tree
Hide file tree
Showing 6 changed files with 153 additions and 506 deletions.
2 changes: 1 addition & 1 deletion airflow/api_fastapi/common/parameters.py
Original file line number Diff line number Diff line change
Expand Up @@ -331,5 +331,5 @@ def _safe_parse_datetime(date_to_check: str) -> datetime:
# DagRun
QueryLastDagRunStateFilter = Annotated[_LastDagRunStateFilter, Depends(_LastDagRunStateFilter().depends)]
# DAGTags
QueryDagTagOrderBy = Annotated[_OrderByParam, Depends(_OrderByParam(DagTag.name).depends)]
QueryDagTagOrderBy = Annotated[_OrderByParam, Depends(_OrderByParam(DagTag.name, skip_none=False).depends)]
QueryDagTagPatternSearch = Annotated[_DagTagNamePatternSearch, Depends(_DagTagNamePatternSearch().depends)]
32 changes: 16 additions & 16 deletions airflow/api_fastapi/core_api/openapi/v1-generated.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -1693,6 +1693,22 @@ components:
- dataset_triggered
title: DAGRunTypes
description: DAG Run Types for responses.
DAGTagCollectionResponse:
properties:
tags:
items:
type: string
type: array
title: Tags
total_entries:
type: integer
title: Total Entries
type: object
required:
- tags
- total_entries
title: DAGTagCollectionResponse
description: DAG Tags Collection serializer for responses.
DagProcessorInfoSchema:
properties:
status:
Expand All @@ -1711,22 +1727,6 @@ components:
- latest_dag_processor_heartbeat
title: DagProcessorInfoSchema
description: Schema for DagProcessor info.
DAGTagCollectionResponse:
properties:
tags:
items:
type: string
type: array
title: Tags
total_entries:
type: integer
title: Total Entries
type: object
required:
- tags
- total_entries
title: DAGTagCollectionResponse
description: DAG Tags Collection serializer for responses.
DagRunState:
type: string
enum:
Expand Down
40 changes: 20 additions & 20 deletions airflow/ui/openapi-gen/requests/schemas.gen.ts
Original file line number Diff line number Diff line change
Expand Up @@ -1013,6 +1013,26 @@ export const $DAGRunTypes = {
description: "DAG Run Types for responses.",
} as const;

export const $DAGTagCollectionResponse = {
properties: {
tags: {
items: {
type: "string",
},
type: "array",
title: "Tags",
},
total_entries: {
type: "integer",
title: "Total Entries",
},
},
type: "object",
required: ["tags", "total_entries"],
title: "DAGTagCollectionResponse",
description: "DAG Tags Collection serializer for responses.",
} as const;

export const $DagProcessorInfoSchema = {
properties: {
status: {
Expand Down Expand Up @@ -1044,26 +1064,6 @@ export const $DagProcessorInfoSchema = {
description: "Schema for DagProcessor info.",
} as const;

export const $DAGTagCollectionResponse = {
properties: {
tags: {
items: {
type: "string",
},
type: "array",
title: "Tags",
},
total_entries: {
type: "integer",
title: "Total Entries",
},
},
type: "object",
required: ["tags", "total_entries"],
title: "DAGTagCollectionResponse",
description: "DAG Tags Collection serializer for responses.",
} as const;

export const $DagRunState = {
type: "string",
enum: ["queued", "running", "success", "failed"],
Expand Down
16 changes: 8 additions & 8 deletions airflow/ui/openapi-gen/requests/types.gen.ts
Original file line number Diff line number Diff line change
Expand Up @@ -177,14 +177,6 @@ export type DAGRunTypes = {
dataset_triggered: number;
};

/**
* Schema for DagProcessor info.
*/
export type DagProcessorInfoSchema = {
status: string | null;
latest_dag_processor_heartbeat: string | null;
};

/**
* DAG Tags Collection serializer for responses.
*/
Expand All @@ -193,6 +185,14 @@ export type DAGTagCollectionResponse = {
total_entries: number;
};

/**
* Schema for DagProcessor info.
*/
export type DagProcessorInfoSchema = {
status: string | null;
latest_dag_processor_heartbeat: string | null;
};

/**
* All possible states that a DagRun can be in.
*
Expand Down
109 changes: 108 additions & 1 deletion tests/api_fastapi/core_api/routes/public/test_dags.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import pendulum
import pytest

from airflow.models.dag import DagModel
from airflow.models.dag import DagModel, DagTag
from airflow.models.dagrun import DagRun
from airflow.operators.empty import EmptyOperator
from airflow.utils.session import provide_session
Expand Down Expand Up @@ -88,6 +88,11 @@ def _create_deactivated_paused_dag(self, session=None):
session.add(dagrun_failed)
session.add(dagrun_success)

def _create_dag_tags(self, session=None):
session.add(DagTag(dag_id=DAG1_ID, name="tag_2"))
session.add(DagTag(dag_id=DAG2_ID, name="tag_1"))
session.add(DagTag(dag_id=DAG3_ID, name="tag_1"))

@pytest.fixture(autouse=True)
@provide_session
def setup(self, dag_maker, session=None) -> None:
Expand Down Expand Up @@ -118,6 +123,7 @@ def setup(self, dag_maker, session=None) -> None:
EmptyOperator(task_id=TASK_ID)

self._create_deactivated_paused_dag(session)
self._create_dag_tags(session)

dag_maker.dagbag.sync_to_db()
dag_maker.dag_model.has_task_concurrency_limits = True
Expand Down Expand Up @@ -386,6 +392,107 @@ def test_get_dag(self, test_client, query_params, dag_id, expected_status_code,
assert res_json == expected


class TestGetDagTags(TestDagEndpoint):
"""Unit tests for Get DAG Tags."""

@pytest.mark.parametrize(
"query_params, expected_status_code, expected_dag_tags, expected_total_entries",
[
# test with offset, limit, and without any tag_name_pattern
(
{},
200,
[
"example",
"tag_1",
"tag_2",
],
3,
),
(
{"offset": 1},
200,
[
"tag_1",
"tag_2",
],
3,
),
(
{"limit": 2},
200,
[
"example",
"tag_1",
],
3,
),
(
{"offset": 1, "limit": 2},
200,
[
"tag_1",
"tag_2",
],
3,
),
# test with tag_name_pattern
(
{"tag_name_pattern": "invalid"},
200,
[],
0,
),
(
{"tag_name_pattern": "1"},
200,
["tag_1"],
1,
),
(
{"tag_name_pattern": "tag%"},
200,
["tag_1", "tag_2"],
2,
),
# test order_by
(
{"order_by": "desc"},
200,
["tag_2", "tag_1", "example"],
3,
),
# test all query params
(
{"tag_name_pattern": "t%", "order_by": "desc", "offset": 1, "limit": 1},
200,
["tag_1"],
2,
),
(
{"tag_name_pattern": "~", "offset": 1, "limit": 2},
200,
["tag_1", "tag_2"],
3,
),
],
)
def test_get_dag_tags(
self, test_client, query_params, expected_status_code, expected_dag_tags, expected_total_entries
):
response = test_client.get("/public/dags/tags", params=query_params)
assert response.status_code == expected_status_code
if expected_status_code != 200:
return

res_json = response.json()
expected = {
"tags": expected_dag_tags,
"total_entries": expected_total_entries,
}
assert res_json == expected


class TestDeleteDAG(TestDagEndpoint):
"""Unit tests for Delete DAG."""

Expand Down
Loading

0 comments on commit 16f0a6d

Please sign in to comment.