This is an automated email from the ASF dual-hosted git repository.

pierrejeambrun pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new 90e638b341b Add DAG calendar view endpoint (#52748)
90e638b341b is described below

commit 90e638b341b72472dc36c49854c7891d79d93094
Author: Ivan <99082173+savingfr...@users.noreply.github.com>
AuthorDate: Mon Jul 7 15:33:58 2025 +0200

    Add DAG calendar view endpoint (#52748)
    
    * Add DAG calendar view endpoint
    
    Implement calendar view functionality that provides DAG scheduling
    information in a calendar format.
    The endpoint allows users to visualize DAG schedules and execution
    patterns over time, improving workflow management and scheduling
    oversight.
    In case if you want get get data for several days you can use daily
    granularity.
    
    * restructure calendar response format
    
    Change calendar endpoint to return structured response with total count
    and dag runs array instead of flat list for better API consistency.
    
    * Update swagger documentation
    
    * standardize calendar datetime format to ISO 8601 with UTC
    
    Update calendar API to return consistent ISO 8601 datetime format with
    UTC timezone suffix across all database dialects and remove timezone
    stripping.
    
    * Update 
airflow-core/src/airflow/api_fastapi/core_api/datamodels/ui/calendar.py
    
    Co-authored-by: Pierre Jeambrun <pierrejb...@gmail.com>
    
    * Update 
airflow-core/src/airflow/api_fastapi/core_api/datamodels/ui/calendar.py
    
    Co-authored-by: Pierre Jeambrun <pierrejb...@gmail.com>
    
    * Rename pydantic models
    
    ---------
    
    Co-authored-by: Pierre Jeambrun <pierrejb...@gmail.com>
---
 .../api_fastapi/core_api/datamodels/ui/calendar.py |  45 +++
 .../api_fastapi/core_api/openapi/_private_ui.yaml  |  99 +++++++
 .../api_fastapi/core_api/routes/ui/__init__.py     |   2 +
 .../api_fastapi/core_api/routes/ui/calendar.py     |  70 +++++
 .../api_fastapi/core_api/services/ui/calendar.py   | 325 +++++++++++++++++++++
 .../src/airflow/ui/openapi-gen/queries/common.ts   |  11 +-
 .../ui/openapi-gen/queries/ensureQueryData.ts      |  19 +-
 .../src/airflow/ui/openapi-gen/queries/prefetch.ts |  19 +-
 .../src/airflow/ui/openapi-gen/queries/queries.ts  |  19 +-
 .../src/airflow/ui/openapi-gen/queries/suspense.ts |  19 +-
 .../airflow/ui/openapi-gen/requests/schemas.gen.ts |  43 +++
 .../ui/openapi-gen/requests/services.gen.ts        |  34 ++-
 .../airflow/ui/openapi-gen/requests/types.gen.ts   |  43 +++
 .../core_api/routes/ui/test_calendar.py            | 151 ++++++++++
 14 files changed, 893 insertions(+), 6 deletions(-)

diff --git 
a/airflow-core/src/airflow/api_fastapi/core_api/datamodels/ui/calendar.py 
b/airflow-core/src/airflow/api_fastapi/core_api/datamodels/ui/calendar.py
new file mode 100644
index 00000000000..c3cc5a53d8c
--- /dev/null
+++ b/airflow-core/src/airflow/api_fastapi/core_api/datamodels/ui/calendar.py
@@ -0,0 +1,45 @@
+# 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 datetime import datetime
+from typing import Literal
+
+from pydantic import BaseModel
+
+from airflow.utils.state import DagRunState
+
+
+class CalendarTimeRangeResponse(BaseModel):
+    """Represents a summary of DAG runs for a specific calendar time range."""
+
+    date: datetime
+    state: Literal[
+        DagRunState.QUEUED,
+        DagRunState.RUNNING,
+        DagRunState.SUCCESS,
+        DagRunState.FAILED,
+        "planned",
+    ]
+    count: int
+
+
+class CalendarTimeRangeCollectionResponse(BaseModel):
+    """Response model for calendar time range results."""
+
+    total_entries: int
+    dag_runs: list[CalendarTimeRangeResponse]
diff --git 
a/airflow-core/src/airflow/api_fastapi/core_api/openapi/_private_ui.yaml 
b/airflow-core/src/airflow/api_fastapi/core_api/openapi/_private_ui.yaml
index 8b7bded7866..79c00723946 100644
--- a/airflow-core/src/airflow/api_fastapi/core_api/openapi/_private_ui.yaml
+++ b/airflow-core/src/airflow/api_fastapi/core_api/openapi/_private_ui.yaml
@@ -891,6 +891,64 @@ paths:
             application/json:
               schema:
                 $ref: '#/components/schemas/HTTPValidationError'
+  /ui/calendar/{dag_id}:
+    get:
+      tags:
+      - Calendar
+      summary: Get Calendar
+      description: Get calendar data for a DAG including historical and 
planned DAG
+        runs.
+      operationId: get_calendar
+      security:
+      - OAuth2PasswordBearer: []
+      parameters:
+      - name: dag_id
+        in: path
+        required: true
+        schema:
+          type: string
+          title: Dag Id
+      - name: granularity
+        in: query
+        required: false
+        schema:
+          enum:
+          - hourly
+          - daily
+          type: string
+          default: daily
+          title: Granularity
+      - name: logical_date_gte
+        in: query
+        required: false
+        schema:
+          anyOf:
+          - type: string
+            format: date-time
+          - type: 'null'
+          title: Logical Date Gte
+      - name: logical_date_lte
+        in: query
+        required: false
+        schema:
+          anyOf:
+          - type: string
+            format: date-time
+          - type: 'null'
+          title: Logical Date Lte
+      responses:
+        '200':
+          description: Successful Response
+          content:
+            application/json:
+              schema:
+                $ref: 
'#/components/schemas/CalendarTimeRangeCollectionResponse'
+        '422':
+          description: Validation Error
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/HTTPValidationError'
 components:
   schemas:
     BackfillCollectionResponse:
@@ -1032,6 +1090,47 @@ components:
       - type
       title: BaseNodeResponse
       description: Base Node serializer for responses.
+    CalendarTimeRangeCollectionResponse:
+      properties:
+        total_entries:
+          type: integer
+          title: Total Entries
+        dag_runs:
+          items:
+            $ref: '#/components/schemas/CalendarTimeRangeResponse'
+          type: array
+          title: Dag Runs
+      type: object
+      required:
+      - total_entries
+      - dag_runs
+      title: CalendarTimeRangeCollectionResponse
+      description: Response model for calendar time range results.
+    CalendarTimeRangeResponse:
+      properties:
+        date:
+          type: string
+          format: date-time
+          title: Date
+        state:
+          type: string
+          enum:
+          - queued
+          - running
+          - success
+          - failed
+          - planned
+          title: State
+        count:
+          type: integer
+          title: Count
+      type: object
+      required:
+      - date
+      - state
+      - count
+      title: CalendarTimeRangeResponse
+      description: Represents a summary of DAG runs for a specific calendar 
time range.
     ConfigResponse:
       properties:
         page_size:
diff --git 
a/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/__init__.py 
b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/__init__.py
index f7b19c53ce2..677776574e8 100644
--- a/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/__init__.py
+++ b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/__init__.py
@@ -20,6 +20,7 @@ from airflow.api_fastapi.common.router import AirflowRouter
 from airflow.api_fastapi.core_api.routes.ui.assets import assets_router
 from airflow.api_fastapi.core_api.routes.ui.auth import auth_router
 from airflow.api_fastapi.core_api.routes.ui.backfills import backfills_router
+from airflow.api_fastapi.core_api.routes.ui.calendar import calendar_router
 from airflow.api_fastapi.core_api.routes.ui.config import config_router
 from airflow.api_fastapi.core_api.routes.ui.connections import 
connections_router
 from airflow.api_fastapi.core_api.routes.ui.dags import dags_router
@@ -40,3 +41,4 @@ ui_router.include_router(dashboard_router)
 ui_router.include_router(structure_router)
 ui_router.include_router(backfills_router)
 ui_router.include_router(grid_router)
+ui_router.include_router(calendar_router)
diff --git 
a/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/calendar.py 
b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/calendar.py
new file mode 100644
index 00000000000..cdb1902904b
--- /dev/null
+++ b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/calendar.py
@@ -0,0 +1,70 @@
+# 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 typing import Annotated, Literal
+
+from fastapi import Depends
+
+from airflow.api_fastapi.auth.managers.models.resource_details import 
DagAccessEntity
+from airflow.api_fastapi.common.dagbag import DagBagDep
+from airflow.api_fastapi.common.db.common import SessionDep
+from airflow.api_fastapi.common.parameters import RangeFilter, 
datetime_range_filter_factory
+from airflow.api_fastapi.common.router import AirflowRouter
+from airflow.api_fastapi.core_api.datamodels.ui.calendar import 
CalendarTimeRangeCollectionResponse
+from airflow.api_fastapi.core_api.security import requires_access_dag
+from airflow.api_fastapi.core_api.services.ui.calendar import CalendarService
+from airflow.models.dagrun import DagRun
+
+calendar_router = AirflowRouter(prefix="/calendar", tags=["Calendar"])
+
+
+@calendar_router.get(
+    "/{dag_id}",
+    dependencies=[
+        Depends(
+            requires_access_dag(
+                method="GET",
+                access_entity=DagAccessEntity.TASK_INSTANCE,
+            )
+        ),
+        Depends(
+            requires_access_dag(
+                method="GET",
+                access_entity=DagAccessEntity.RUN,
+            )
+        ),
+    ],
+)
+def get_calendar(
+    dag_id: str,
+    session: SessionDep,
+    dag_bag: DagBagDep,
+    logical_date: Annotated[RangeFilter, 
Depends(datetime_range_filter_factory("logical_date", DagRun))],
+    granularity: Literal["hourly", "daily"] = "daily",
+) -> CalendarTimeRangeCollectionResponse:
+    """Get calendar data for a DAG including historical and planned DAG 
runs."""
+    dag = dag_bag.get_dag(dag_id)
+    calendar_service = CalendarService()
+
+    return calendar_service.get_calendar_data(
+        dag_id=dag_id,
+        session=session,
+        dag=dag,
+        logical_date=logical_date,
+        granularity=granularity,
+    )
diff --git 
a/airflow-core/src/airflow/api_fastapi/core_api/services/ui/calendar.py 
b/airflow-core/src/airflow/api_fastapi/core_api/services/ui/calendar.py
new file mode 100644
index 00000000000..de51a1dd27f
--- /dev/null
+++ b/airflow-core/src/airflow/api_fastapi/core_api/services/ui/calendar.py
@@ -0,0 +1,325 @@
+# 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 collections
+import datetime
+from collections.abc import Iterator
+from typing import Literal, cast
+
+import sqlalchemy as sa
+import structlog
+from croniter.croniter import croniter
+from pendulum import DateTime
+from sqlalchemy.engine import Row
+from sqlalchemy.orm import Session
+
+from airflow.api_fastapi.common.parameters import RangeFilter
+from airflow.api_fastapi.core_api.datamodels.ui.calendar import (
+    CalendarTimeRangeCollectionResponse,
+    CalendarTimeRangeResponse,
+)
+from airflow.models.dag import DAG
+from airflow.models.dagrun import DagRun
+from airflow.timetables._cron import CronMixin
+from airflow.timetables.base import DataInterval, TimeRestriction
+from airflow.timetables.simple import ContinuousTimetable
+from airflow.utils import timezone
+
+log = structlog.get_logger(logger_name=__name__)
+
+
+class CalendarService:
+    """Service class for calendar-related operations."""
+
+    MAX_PLANNED_RUNS: int = 2000
+
+    def get_calendar_data(
+        self,
+        dag_id: str,
+        session: Session,
+        dag: DAG,
+        logical_date: RangeFilter,
+        granularity: Literal["hourly", "daily"] = "daily",
+    ) -> CalendarTimeRangeCollectionResponse:
+        """
+        Get calendar data for a DAG including historical and planned runs.
+
+        Args:
+            dag_id: The DAG ID
+            session: Database session
+            dag: The DAG object
+            logical_date: Date range filter
+            granularity: Time granularity ("hourly" or "daily")
+
+        Returns:
+            List of calendar time range results
+        """
+        historical_data, raw_dag_states = self._get_historical_dag_runs(
+            dag_id,
+            session,
+            logical_date,
+            granularity,
+        )
+
+        planned_data = self._get_planned_dag_runs(dag, raw_dag_states, 
logical_date, granularity)
+
+        all_data = historical_data + planned_data
+        return CalendarTimeRangeCollectionResponse(
+            total_entries=len(all_data),
+            dag_runs=all_data,
+        )
+
+    def _get_historical_dag_runs(
+        self,
+        dag_id: str,
+        session: Session,
+        logical_date: RangeFilter,
+        granularity: Literal["hourly", "daily"],
+    ) -> tuple[list[CalendarTimeRangeResponse], list[Row]]:
+        """Get historical DAG runs from the database."""
+        dialect = session.bind.dialect.name
+
+        time_expression = 
self._get_time_truncation_expression(DagRun.logical_date, granularity, dialect)
+
+        select_stmt = (
+            sa.select(
+                time_expression.label("datetime"),
+                DagRun.state,
+                
sa.func.max(DagRun.data_interval_start).label("data_interval_start"),
+                
sa.func.max(DagRun.data_interval_end).label("data_interval_end"),
+                sa.func.count("*").label("count"),
+            )
+            .where(DagRun.dag_id == dag_id)
+            .group_by(time_expression, DagRun.state)
+            .order_by(time_expression.asc())
+        )
+
+        select_stmt = logical_date.to_orm(select_stmt)
+        dag_states = session.execute(select_stmt).all()
+
+        calendar_results = [
+            CalendarTimeRangeResponse(
+                # ds.datetime in sqlite and mysql is a string, in postgresql 
it is a datetime
+                date=ds.datetime,
+                state=ds.state,
+                count=ds.count,
+            )
+            for ds in dag_states
+        ]
+
+        return calendar_results, dag_states
+
+    def _get_planned_dag_runs(
+        self,
+        dag: DAG,
+        raw_dag_states: list[Row],
+        logical_date: RangeFilter,
+        granularity: Literal["hourly", "daily"],
+    ) -> list[CalendarTimeRangeResponse]:
+        """Get planned DAG runs based on the DAG's timetable."""
+        if not self._should_calculate_planned_runs(dag, raw_dag_states):
+            return []
+
+        last_data_interval = self._get_last_data_interval(raw_dag_states)
+        if not last_data_interval:
+            return []
+
+        year = last_data_interval.end.year
+        restriction = TimeRestriction(
+            timezone.coerce_datetime(dag.start_date) if dag.start_date else 
None,
+            timezone.coerce_datetime(dag.end_date) if dag.end_date else None,
+            False,
+        )
+
+        if isinstance(dag.timetable, CronMixin):
+            return self._calculate_cron_planned_runs(dag, last_data_interval, 
year, logical_date, granularity)
+        return self._calculate_timetable_planned_runs(
+            dag, last_data_interval, year, restriction, logical_date, 
granularity
+        )
+
+    def _should_calculate_planned_runs(self, dag: DAG, raw_dag_states: 
list[Row]) -> bool:
+        """Check if we should calculate planned runs."""
+        return (
+            bool(raw_dag_states)
+            and bool(raw_dag_states[-1].data_interval_start)
+            and bool(raw_dag_states[-1].data_interval_end)
+            and not isinstance(dag.timetable, ContinuousTimetable)
+        )
+
+    def _get_last_data_interval(self, raw_dag_states: list[Row]) -> 
DataInterval | None:
+        """Extract the last data interval from raw database results."""
+        if not raw_dag_states:
+            return None
+
+        last_state = raw_dag_states[-1]
+        if not (last_state.data_interval_start and 
last_state.data_interval_end):
+            return None
+
+        return DataInterval(
+            timezone.coerce_datetime(last_state.data_interval_start),
+            timezone.coerce_datetime(last_state.data_interval_end),
+        )
+
+    def _calculate_cron_planned_runs(
+        self,
+        dag: DAG,
+        last_data_interval: DataInterval,
+        year: int,
+        logical_date: RangeFilter,
+        granularity: Literal["hourly", "daily"],
+    ) -> list[CalendarTimeRangeResponse]:
+        """Calculate planned runs for cron-based timetables."""
+        dates: dict[datetime.datetime, int] = collections.Counter()
+
+        dates_iter: Iterator[datetime.datetime | None] = croniter(
+            cast("CronMixin", dag.timetable)._expression,
+            start_time=last_data_interval.end,
+            ret_type=datetime.datetime,
+        )
+
+        for dt in dates_iter:
+            if dt is None or dt.year != year:
+                break
+            if dag.end_date and dt > dag.end_date:
+                break
+            if not self._is_date_in_range(dt, logical_date):
+                continue
+
+            dates[self._truncate_datetime_for_granularity(dt, granularity)] += 
1
+
+        return [
+            CalendarTimeRangeResponse(date=dt, state="planned", count=count) 
for dt, count in dates.items()
+        ]
+
+    def _calculate_timetable_planned_runs(
+        self,
+        dag: DAG,
+        last_data_interval: DataInterval,
+        year: int,
+        restriction: TimeRestriction,
+        logical_date: RangeFilter,
+        granularity: Literal["hourly", "daily"],
+    ) -> list[CalendarTimeRangeResponse]:
+        """Calculate planned runs for generic timetables."""
+        dates: dict[datetime.datetime, int] = collections.Counter()
+        prev_logical_date = DateTime.min
+        total_planned = 0
+
+        while total_planned < self.MAX_PLANNED_RUNS:
+            curr_info = dag.timetable.next_dagrun_info(
+                last_automated_data_interval=last_data_interval,
+                restriction=restriction,
+            )
+
+            if curr_info is None:  # No more DAG runs to schedule
+                break
+            if curr_info.logical_date <= prev_logical_date:  # Timetable not 
progressing, stopping
+                break
+            if curr_info.logical_date.year != year:  # Crossed year boundary
+                break
+
+            if not self._is_date_in_range(curr_info.logical_date, 
logical_date):
+                last_data_interval = curr_info.data_interval
+                prev_logical_date = curr_info.logical_date
+                total_planned += 1
+                continue
+
+            last_data_interval = curr_info.data_interval
+            dt = 
self._truncate_datetime_for_granularity(curr_info.logical_date, granularity)
+            dates[dt] += 1
+            prev_logical_date = curr_info.logical_date
+            total_planned += 1
+
+        return [
+            CalendarTimeRangeResponse(date=dt, state="planned", count=count) 
for dt, count in dates.items()
+        ]
+
+    def _get_time_truncation_expression(
+        self,
+        column: sa.Column,
+        granularity: Literal["hourly", "daily"],
+        dialect: str,
+    ) -> sa.Column:
+        """
+        Get database-specific time truncation expression for SQLAlchemy.
+
+        We want to return always timestamp for both hourly and daily 
truncation.
+        Unfortunately different databases have different functions for 
truncating datetime, so we need to handle
+        them separately.
+
+        Args:
+            column: The datetime column to truncate
+            granularity: Either "hourly" or "daily"
+            dialect: Database dialect ("postgresql", "mysql", "sqlite")
+
+        Returns:
+            SQLAlchemy expression for time truncation
+
+        Raises:
+            ValueError: If the dialect is not supported
+        """
+        if granularity == "hourly":
+            if dialect == "postgresql":
+                expression = sa.func.date_trunc("hour", column)
+            elif dialect == "mysql":
+                expression = sa.func.date_format(column, "%Y-%m-%dT%H:00:00Z")
+            elif dialect == "sqlite":
+                expression = sa.func.strftime("%Y-%m-%dT%H:00:00Z", column)
+            else:
+                raise ValueError(f"Unsupported dialect: {dialect}")
+        else:
+            if dialect == "postgresql":
+                expression = sa.func.timezone("UTC", 
sa.func.cast(sa.func.cast(column, sa.Date), sa.DateTime))
+            elif dialect == "mysql":
+                expression = sa.func.date_format(column, "%Y-%m-%dT%00:00:00Z")
+            elif dialect == "sqlite":
+                expression = sa.func.strftime("%Y-%m-%dT00:00:00Z", column)
+            else:
+                raise ValueError(f"Unsupported dialect: {dialect}")
+        return expression
+
+    def _truncate_datetime_for_granularity(
+        self,
+        dt: datetime.datetime,
+        granularity: Literal["hourly", "daily"],
+    ) -> datetime.datetime:
+        """
+        Truncate datetime based on granularity for planned tasks grouping.
+
+        Args:
+            dt: The datetime to truncate
+            granularity: Either "hourly" or "daily"
+
+        Returns:
+            Truncated datetime
+        """
+        if granularity == "hourly":
+            return dt.replace(minute=0, second=0, microsecond=0)
+        return dt.replace(hour=0, minute=0, second=0, microsecond=0)
+
+    def _is_date_in_range(self, dt: datetime.datetime, logical_date: 
RangeFilter) -> bool:
+        """Check if a date is within the specified range filter."""
+        if not logical_date.value:
+            return True
+
+        if logical_date.value.lower_bound and dt < 
logical_date.value.lower_bound:
+            return False
+        if logical_date.value.upper_bound and dt > 
logical_date.value.upper_bound:
+            return False
+
+        return True
diff --git a/airflow-core/src/airflow/ui/openapi-gen/queries/common.ts 
b/airflow-core/src/airflow/ui/openapi-gen/queries/common.ts
index d538e3e6175..2eb6e29ef57 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/queries/common.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/queries/common.ts
@@ -1,7 +1,7 @@
 // generated with @7nohe/openapi-react-query-codegen@1.6.2 
 
 import { UseQueryResult } from "@tanstack/react-query";
-import { AssetService, AuthLinksService, BackfillService, ConfigService, 
ConnectionService, DagParsingService, DagReportService, DagRunService, 
DagService, DagSourceService, DagStatsService, DagVersionService, 
DagWarningService, DashboardService, DependenciesService, EventLogService, 
ExtraLinksService, GridService, ImportErrorService, JobService, LoginService, 
MonitorService, PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, [...]
+import { AssetService, AuthLinksService, BackfillService, CalendarService, 
ConfigService, ConnectionService, DagParsingService, DagReportService, 
DagRunService, DagService, DagSourceService, DagStatsService, 
DagVersionService, DagWarningService, DashboardService, DependenciesService, 
EventLogService, ExtraLinksService, GridService, ImportErrorService, 
JobService, LoginService, MonitorService, PluginService, PoolService, 
ProviderService, StructureService, TaskInstanceService, TaskService, [...]
 import { DagRunState, DagWarningType } from "../requests/types.gen";
 export type AssetServiceGetAssetsDefaultResponse = Awaited<ReturnType<typeof 
AssetService.getAssets>>;
 export type AssetServiceGetAssetsQueryResult<TData = 
AssetServiceGetAssetsDefaultResponse, TError = unknown> = UseQueryResult<TData, 
TError>;
@@ -704,6 +704,15 @@ export const useGridServiceGetLatestRunKey = 
"GridServiceGetLatestRun";
 export const UseGridServiceGetLatestRunKeyFn = ({ dagId }: {
   dagId: string;
 }, queryKey?: Array<unknown>) => [useGridServiceGetLatestRunKey, ...(queryKey 
?? [{ dagId }])];
+export type CalendarServiceGetCalendarDefaultResponse = 
Awaited<ReturnType<typeof CalendarService.getCalendar>>;
+export type CalendarServiceGetCalendarQueryResult<TData = 
CalendarServiceGetCalendarDefaultResponse, TError = unknown> = 
UseQueryResult<TData, TError>;
+export const useCalendarServiceGetCalendarKey = "CalendarServiceGetCalendar";
+export const UseCalendarServiceGetCalendarKeyFn = ({ dagId, granularity, 
logicalDateGte, logicalDateLte }: {
+  dagId: string;
+  granularity?: "hourly" | "daily";
+  logicalDateGte?: string;
+  logicalDateLte?: string;
+}, queryKey?: Array<unknown>) => [useCalendarServiceGetCalendarKey, 
...(queryKey ?? [{ dagId, granularity, logicalDateGte, logicalDateLte }])];
 export type AssetServiceCreateAssetEventMutationResult = 
Awaited<ReturnType<typeof AssetService.createAssetEvent>>;
 export type AssetServiceMaterializeAssetMutationResult = 
Awaited<ReturnType<typeof AssetService.materializeAsset>>;
 export type BackfillServiceCreateBackfillMutationResult = 
Awaited<ReturnType<typeof BackfillService.createBackfill>>;
diff --git a/airflow-core/src/airflow/ui/openapi-gen/queries/ensureQueryData.ts 
b/airflow-core/src/airflow/ui/openapi-gen/queries/ensureQueryData.ts
index 5ba5f5f3620..bea49000ad7 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/queries/ensureQueryData.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/queries/ensureQueryData.ts
@@ -1,7 +1,7 @@
 // generated with @7nohe/openapi-react-query-codegen@1.6.2 
 
 import { type QueryClient } from "@tanstack/react-query";
-import { AssetService, AuthLinksService, BackfillService, ConfigService, 
ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, VersionService, Xc [...]
+import { AssetService, AuthLinksService, BackfillService, CalendarService, 
ConfigService, ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, V [...]
 import { DagRunState, DagWarningType } from "../requests/types.gen";
 import * as Common from "./common";
 /**
@@ -1338,3 +1338,20 @@ export const ensureUseGridServiceGetGridTiSummariesData 
= (queryClient: QueryCli
 export const ensureUseGridServiceGetLatestRunData = (queryClient: QueryClient, 
{ dagId }: {
   dagId: string;
 }) => queryClient.ensureQueryData({ queryKey: 
Common.UseGridServiceGetLatestRunKeyFn({ dagId }), queryFn: () => 
GridService.getLatestRun({ dagId }) });
+/**
+* Get Calendar
+* Get calendar data for a DAG including historical and planned DAG runs.
+* @param data The data for the request.
+* @param data.dagId
+* @param data.granularity
+* @param data.logicalDateGte
+* @param data.logicalDateLte
+* @returns CalendarTimeRangeCollectionResponse Successful Response
+* @throws ApiError
+*/
+export const ensureUseCalendarServiceGetCalendarData = (queryClient: 
QueryClient, { dagId, granularity, logicalDateGte, logicalDateLte }: {
+  dagId: string;
+  granularity?: "hourly" | "daily";
+  logicalDateGte?: string;
+  logicalDateLte?: string;
+}) => queryClient.ensureQueryData({ queryKey: 
Common.UseCalendarServiceGetCalendarKeyFn({ dagId, granularity, logicalDateGte, 
logicalDateLte }), queryFn: () => CalendarService.getCalendar({ dagId, 
granularity, logicalDateGte, logicalDateLte }) });
diff --git a/airflow-core/src/airflow/ui/openapi-gen/queries/prefetch.ts 
b/airflow-core/src/airflow/ui/openapi-gen/queries/prefetch.ts
index f878624e22e..67dcf913da5 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/queries/prefetch.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/queries/prefetch.ts
@@ -1,7 +1,7 @@
 // generated with @7nohe/openapi-react-query-codegen@1.6.2 
 
 import { type QueryClient } from "@tanstack/react-query";
-import { AssetService, AuthLinksService, BackfillService, ConfigService, 
ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, VersionService, Xc [...]
+import { AssetService, AuthLinksService, BackfillService, CalendarService, 
ConfigService, ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, V [...]
 import { DagRunState, DagWarningType } from "../requests/types.gen";
 import * as Common from "./common";
 /**
@@ -1338,3 +1338,20 @@ export const prefetchUseGridServiceGetGridTiSummaries = 
(queryClient: QueryClien
 export const prefetchUseGridServiceGetLatestRun = (queryClient: QueryClient, { 
dagId }: {
   dagId: string;
 }) => queryClient.prefetchQuery({ queryKey: 
Common.UseGridServiceGetLatestRunKeyFn({ dagId }), queryFn: () => 
GridService.getLatestRun({ dagId }) });
+/**
+* Get Calendar
+* Get calendar data for a DAG including historical and planned DAG runs.
+* @param data The data for the request.
+* @param data.dagId
+* @param data.granularity
+* @param data.logicalDateGte
+* @param data.logicalDateLte
+* @returns CalendarTimeRangeCollectionResponse Successful Response
+* @throws ApiError
+*/
+export const prefetchUseCalendarServiceGetCalendar = (queryClient: 
QueryClient, { dagId, granularity, logicalDateGte, logicalDateLte }: {
+  dagId: string;
+  granularity?: "hourly" | "daily";
+  logicalDateGte?: string;
+  logicalDateLte?: string;
+}) => queryClient.prefetchQuery({ queryKey: 
Common.UseCalendarServiceGetCalendarKeyFn({ dagId, granularity, logicalDateGte, 
logicalDateLte }), queryFn: () => CalendarService.getCalendar({ dagId, 
granularity, logicalDateGte, logicalDateLte }) });
diff --git a/airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts 
b/airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts
index 3e3ba37467f..627188e6746 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts
@@ -1,7 +1,7 @@
 // generated with @7nohe/openapi-react-query-codegen@1.6.2 
 
 import { UseMutationOptions, UseQueryOptions, useMutation, useQuery } from 
"@tanstack/react-query";
-import { AssetService, AuthLinksService, BackfillService, ConfigService, 
ConnectionService, DagParsingService, DagReportService, DagRunService, 
DagService, DagSourceService, DagStatsService, DagVersionService, 
DagWarningService, DashboardService, DependenciesService, EventLogService, 
ExtraLinksService, GridService, ImportErrorService, JobService, LoginService, 
MonitorService, PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, [...]
+import { AssetService, AuthLinksService, BackfillService, CalendarService, 
ConfigService, ConnectionService, DagParsingService, DagReportService, 
DagRunService, DagService, DagSourceService, DagStatsService, 
DagVersionService, DagWarningService, DashboardService, DependenciesService, 
EventLogService, ExtraLinksService, GridService, ImportErrorService, 
JobService, LoginService, MonitorService, PluginService, PoolService, 
ProviderService, StructureService, TaskInstanceService, TaskService, [...]
 import { BackfillPostBody, BulkBody_BulkTaskInstanceBody_, 
BulkBody_ConnectionBody_, BulkBody_PoolBody_, BulkBody_VariableBody_, 
ClearTaskInstancesBody, ConnectionBody, CreateAssetEventsBody, DAGPatchBody, 
DAGRunClearBody, DAGRunPatchBody, DAGRunsBatchBody, DagRunState, 
DagWarningType, PatchTaskInstanceBody, PoolBody, PoolPatchBody, 
TaskInstancesBatchBody, TriggerDAGRunPostBody, VariableBody, XComCreateBody, 
XComUpdateBody } from "../requests/types.gen";
 import * as Common from "./common";
 /**
@@ -1339,6 +1339,23 @@ export const useGridServiceGetLatestRun = <TData = 
Common.GridServiceGetLatestRu
   dagId: string;
 }, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, 
"queryKey" | "queryFn">) => useQuery<TData, TError>({ queryKey: 
Common.UseGridServiceGetLatestRunKeyFn({ dagId }, queryKey), queryFn: () => 
GridService.getLatestRun({ dagId }) as TData, ...options });
 /**
+* Get Calendar
+* Get calendar data for a DAG including historical and planned DAG runs.
+* @param data The data for the request.
+* @param data.dagId
+* @param data.granularity
+* @param data.logicalDateGte
+* @param data.logicalDateLte
+* @returns CalendarTimeRangeCollectionResponse Successful Response
+* @throws ApiError
+*/
+export const useCalendarServiceGetCalendar = <TData = 
Common.CalendarServiceGetCalendarDefaultResponse, TError = unknown, TQueryKey 
extends Array<unknown> = unknown[]>({ dagId, granularity, logicalDateGte, 
logicalDateLte }: {
+  dagId: string;
+  granularity?: "hourly" | "daily";
+  logicalDateGte?: string;
+  logicalDateLte?: string;
+}, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, 
"queryKey" | "queryFn">) => useQuery<TData, TError>({ queryKey: 
Common.UseCalendarServiceGetCalendarKeyFn({ dagId, granularity, logicalDateGte, 
logicalDateLte }, queryKey), queryFn: () => CalendarService.getCalendar({ 
dagId, granularity, logicalDateGte, logicalDateLte }) as TData, ...options });
+/**
 * Create Asset Event
 * Create asset events.
 * @param data The data for the request.
diff --git a/airflow-core/src/airflow/ui/openapi-gen/queries/suspense.ts 
b/airflow-core/src/airflow/ui/openapi-gen/queries/suspense.ts
index 2ebe843edb3..beb1362751f 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/queries/suspense.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/queries/suspense.ts
@@ -1,7 +1,7 @@
 // generated with @7nohe/openapi-react-query-codegen@1.6.2 
 
 import { UseQueryOptions, useSuspenseQuery } from "@tanstack/react-query";
-import { AssetService, AuthLinksService, BackfillService, ConfigService, 
ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, VersionService, Xc [...]
+import { AssetService, AuthLinksService, BackfillService, CalendarService, 
ConfigService, ConnectionService, DagReportService, DagRunService, DagService, 
DagSourceService, DagStatsService, DagVersionService, DagWarningService, 
DashboardService, DependenciesService, EventLogService, ExtraLinksService, 
GridService, ImportErrorService, JobService, LoginService, MonitorService, 
PluginService, PoolService, ProviderService, StructureService, 
TaskInstanceService, TaskService, VariableService, V [...]
 import { DagRunState, DagWarningType } from "../requests/types.gen";
 import * as Common from "./common";
 /**
@@ -1338,3 +1338,20 @@ export const useGridServiceGetGridTiSummariesSuspense = 
<TData = Common.GridServ
 export const useGridServiceGetLatestRunSuspense = <TData = 
Common.GridServiceGetLatestRunDefaultResponse, TError = unknown, TQueryKey 
extends Array<unknown> = unknown[]>({ dagId }: {
   dagId: string;
 }, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, 
"queryKey" | "queryFn">) => useSuspenseQuery<TData, TError>({ queryKey: 
Common.UseGridServiceGetLatestRunKeyFn({ dagId }, queryKey), queryFn: () => 
GridService.getLatestRun({ dagId }) as TData, ...options });
+/**
+* Get Calendar
+* Get calendar data for a DAG including historical and planned DAG runs.
+* @param data The data for the request.
+* @param data.dagId
+* @param data.granularity
+* @param data.logicalDateGte
+* @param data.logicalDateLte
+* @returns CalendarTimeRangeCollectionResponse Successful Response
+* @throws ApiError
+*/
+export const useCalendarServiceGetCalendarSuspense = <TData = 
Common.CalendarServiceGetCalendarDefaultResponse, TError = unknown, TQueryKey 
extends Array<unknown> = unknown[]>({ dagId, granularity, logicalDateGte, 
logicalDateLte }: {
+  dagId: string;
+  granularity?: "hourly" | "daily";
+  logicalDateGte?: string;
+  logicalDateLte?: string;
+}, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, 
"queryKey" | "queryFn">) => useSuspenseQuery<TData, TError>({ queryKey: 
Common.UseCalendarServiceGetCalendarKeyFn({ dagId, granularity, logicalDateGte, 
logicalDateLte }, queryKey), queryFn: () => CalendarService.getCalendar({ 
dagId, granularity, logicalDateGte, logicalDateLte }) as TData, ...options });
diff --git a/airflow-core/src/airflow/ui/openapi-gen/requests/schemas.gen.ts 
b/airflow-core/src/airflow/ui/openapi-gen/requests/schemas.gen.ts
index a6202ec4e4c..8efd3e111ef 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/requests/schemas.gen.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/requests/schemas.gen.ts
@@ -6124,6 +6124,49 @@ export const $BaseNodeResponse = {
     description: 'Base Node serializer for responses.'
 } as const;
 
+export const $CalendarTimeRangeCollectionResponse = {
+    properties: {
+        total_entries: {
+            type: 'integer',
+            title: 'Total Entries'
+        },
+        dag_runs: {
+            items: {
+                '$ref': '#/components/schemas/CalendarTimeRangeResponse'
+            },
+            type: 'array',
+            title: 'Dag Runs'
+        }
+    },
+    type: 'object',
+    required: ['total_entries', 'dag_runs'],
+    title: 'CalendarTimeRangeCollectionResponse',
+    description: 'Response model for calendar time range results.'
+} as const;
+
+export const $CalendarTimeRangeResponse = {
+    properties: {
+        date: {
+            type: 'string',
+            format: 'date-time',
+            title: 'Date'
+        },
+        state: {
+            type: 'string',
+            enum: ['queued', 'running', 'success', 'failed', 'planned'],
+            title: 'State'
+        },
+        count: {
+            type: 'integer',
+            title: 'Count'
+        }
+    },
+    type: 'object',
+    required: ['date', 'state', 'count'],
+    title: 'CalendarTimeRangeResponse',
+    description: 'Represents a summary of DAG runs for a specific calendar 
time range.'
+} as const;
+
 export const $ConfigResponse = {
     properties: {
         page_size: {
diff --git a/airflow-core/src/airflow/ui/openapi-gen/requests/services.gen.ts 
b/airflow-core/src/airflow/ui/openapi-gen/requests/services.gen.ts
index 02acbbca343..a12039ac322 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/requests/services.gen.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/requests/services.gen.ts
@@ -3,7 +3,7 @@
 import type { CancelablePromise } from './core/CancelablePromise';
 import { OpenAPI } from './core/OpenAPI';
 import { request as __request } from './core/request';
-import type { GetAssetsData, GetAssetsResponse, GetAssetAliasesData, 
GetAssetAliasesResponse, GetAssetAliasData, GetAssetAliasResponse, 
GetAssetEventsData, GetAssetEventsResponse, CreateAssetEventData, 
CreateAssetEventResponse, MaterializeAssetData, MaterializeAssetResponse, 
GetAssetQueuedEventsData, GetAssetQueuedEventsResponse, 
DeleteAssetQueuedEventsData, DeleteAssetQueuedEventsResponse, GetAssetData, 
GetAssetResponse, GetDagAssetQueuedEventsData, GetDagAssetQueuedEventsResponse, 
Dele [...]
+import type { GetAssetsData, GetAssetsResponse, GetAssetAliasesData, 
GetAssetAliasesResponse, GetAssetAliasData, GetAssetAliasResponse, 
GetAssetEventsData, GetAssetEventsResponse, CreateAssetEventData, 
CreateAssetEventResponse, MaterializeAssetData, MaterializeAssetResponse, 
GetAssetQueuedEventsData, GetAssetQueuedEventsResponse, 
DeleteAssetQueuedEventsData, DeleteAssetQueuedEventsResponse, GetAssetData, 
GetAssetResponse, GetDagAssetQueuedEventsData, GetDagAssetQueuedEventsResponse, 
Dele [...]
 
 export class AssetService {
     /**
@@ -3612,4 +3612,36 @@ export class GridService {
         });
     }
     
+}
+
+export class CalendarService {
+    /**
+     * Get Calendar
+     * Get calendar data for a DAG including historical and planned DAG runs.
+     * @param data The data for the request.
+     * @param data.dagId
+     * @param data.granularity
+     * @param data.logicalDateGte
+     * @param data.logicalDateLte
+     * @returns CalendarTimeRangeCollectionResponse Successful Response
+     * @throws ApiError
+     */
+    public static getCalendar(data: GetCalendarData): 
CancelablePromise<GetCalendarResponse> {
+        return __request(OpenAPI, {
+            method: 'GET',
+            url: '/ui/calendar/{dag_id}',
+            path: {
+                dag_id: data.dagId
+            },
+            query: {
+                granularity: data.granularity,
+                logical_date_gte: data.logicalDateGte,
+                logical_date_lte: data.logicalDateLte
+            },
+            errors: {
+                422: 'Validation Error'
+            }
+        });
+    }
+    
 }
\ No newline at end of file
diff --git a/airflow-core/src/airflow/ui/openapi-gen/requests/types.gen.ts 
b/airflow-core/src/airflow/ui/openapi-gen/requests/types.gen.ts
index f28b965b25a..e09579e651c 100644
--- a/airflow-core/src/airflow/ui/openapi-gen/requests/types.gen.ts
+++ b/airflow-core/src/airflow/ui/openapi-gen/requests/types.gen.ts
@@ -1566,6 +1566,25 @@ export type BaseNodeResponse = {
 
 export type type = 'join' | 'task' | 'asset-condition' | 'asset' | 
'asset-alias' | 'asset-name-ref' | 'asset-uri-ref' | 'dag' | 'sensor' | 
'trigger';
 
+/**
+ * Response model for calendar time range results.
+ */
+export type CalendarTimeRangeCollectionResponse = {
+    total_entries: number;
+    dag_runs: Array<CalendarTimeRangeResponse>;
+};
+
+/**
+ * Represents a summary of DAG runs for a specific calendar time range.
+ */
+export type CalendarTimeRangeResponse = {
+    date: string;
+    state: 'queued' | 'running' | 'success' | 'failed' | 'planned';
+    count: number;
+};
+
+export type state = 'queued' | 'running' | 'success' | 'failed' | 'planned';
+
 /**
  * configuration serializer.
  */
@@ -2934,6 +2953,15 @@ export type GetLatestRunData = {
 
 export type GetLatestRunResponse = LatestRunResponse | null;
 
+export type GetCalendarData = {
+    dagId: string;
+    granularity?: 'hourly' | 'daily';
+    logicalDateGte?: string | null;
+    logicalDateLte?: string | null;
+};
+
+export type GetCalendarResponse = CalendarTimeRangeCollectionResponse;
+
 export type $OpenApiTs = {
     '/api/v2/assets': {
         get: {
@@ -5957,4 +5985,19 @@ export type $OpenApiTs = {
             };
         };
     };
+    '/ui/calendar/{dag_id}': {
+        get: {
+            req: GetCalendarData;
+            res: {
+                /**
+                 * Successful Response
+                 */
+                200: CalendarTimeRangeCollectionResponse;
+                /**
+                 * Validation Error
+                 */
+                422: HTTPValidationError;
+            };
+        };
+    };
 };
\ No newline at end of file
diff --git 
a/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_calendar.py 
b/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_calendar.py
new file mode 100644
index 00000000000..e052b677f8f
--- /dev/null
+++ b/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_calendar.py
@@ -0,0 +1,151 @@
+# 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 datetime import datetime
+
+import pytest
+
+from airflow.providers.standard.operators.empty import EmptyOperator
+from airflow.utils.session import provide_session
+from airflow.utils.state import DagRunState
+
+from tests_common.test_utils.db import clear_db_dags, clear_db_runs
+
+pytestmark = pytest.mark.db_test
+
+
+class TestCalendar:
+    DAG_NAME = "test_dag1"
+
+    @pytest.fixture(autouse=True)
+    @provide_session
+    def setup_dag_runs(self, dag_maker, session=None) -> None:
+        clear_db_runs()
+        clear_db_dags()
+        with dag_maker(
+            self.DAG_NAME,
+            schedule="0 0,1 * * *",
+            start_date=datetime(2025, 1, 1),
+            end_date=datetime(2025, 1, 3, 2),
+            catchup=True,
+            serialized=True,
+            session=session,
+        ):
+            EmptyOperator(task_id="test_task1")
+        dag_maker.create_dagrun(run_id="run_1", state=DagRunState.FAILED, 
logical_date=datetime(2025, 1, 1))
+        dag_maker.create_dagrun(
+            run_id="run_2",
+            state=DagRunState.SUCCESS,
+            logical_date=datetime(2025, 1, 1, 1),
+        )
+        dag_maker.create_dagrun(run_id="run_3", state=DagRunState.RUNNING, 
logical_date=datetime(2025, 1, 2))
+
+        dag_maker.sync_dagbag_to_db()
+
+        session.commit()
+
+    def teardown_method(self) -> None:
+        clear_db_runs()
+        clear_db_dags()
+
+    @pytest.mark.parametrize(
+        "query_params, result",
+        [
+            (
+                {},
+                {
+                    "total_entries": 5,
+                    "dag_runs": [
+                        {"date": "2025-01-01T00:00:00Z", "state": "failed", 
"count": 1},
+                        {"date": "2025-01-01T00:00:00Z", "state": "success", 
"count": 1},
+                        {"date": "2025-01-02T00:00:00Z", "state": "running", 
"count": 1},
+                        {"date": "2025-01-02T00:00:00Z", "state": "planned", 
"count": 1},
+                        {"date": "2025-01-03T00:00:00Z", "state": "planned", 
"count": 2},
+                    ],
+                },
+            ),
+            (
+                {"logical_date_gte": "2025-01-01T00:00:00Z", 
"logical_date_lte": "2025-01-01T23:23:59Z"},
+                {
+                    "total_entries": 2,
+                    "dag_runs": [
+                        {"date": "2025-01-01T00:00:00Z", "state": "failed", 
"count": 1},
+                        {"date": "2025-01-01T00:00:00Z", "state": "success", 
"count": 1},
+                    ],
+                },
+            ),
+            (
+                {"logical_date_gte": "2025-01-02T00:00:00Z", 
"logical_date_lte": "2025-01-02T23:23:59Z"},
+                {
+                    "total_entries": 2,
+                    "dag_runs": [
+                        {"date": "2025-01-02T00:00:00Z", "state": "running", 
"count": 1},
+                        {"date": "2025-01-02T00:00:00Z", "state": "planned", 
"count": 1},
+                    ],
+                },
+            ),
+        ],
+    )
+    def test_daily_calendar(self, test_client, query_params, result):
+        response = test_client.get(f"/calendar/{self.DAG_NAME}", 
params=query_params)
+        assert response.status_code == 200
+        body = response.json()
+        print(body)
+
+        assert body == result
+
+    @pytest.mark.parametrize(
+        "query_params, result",
+        [
+            (
+                {"granularity": "hourly"},
+                {
+                    "total_entries": 6,
+                    "dag_runs": [
+                        {"date": "2025-01-01T00:00:00Z", "state": "failed", 
"count": 1},
+                        {"date": "2025-01-01T01:00:00Z", "state": "success", 
"count": 1},
+                        {"date": "2025-01-02T00:00:00Z", "state": "running", 
"count": 1},
+                        {"date": "2025-01-02T01:00:00Z", "state": "planned", 
"count": 1},
+                        {"date": "2025-01-03T00:00:00Z", "state": "planned", 
"count": 1},
+                        {"date": "2025-01-03T01:00:00Z", "state": "planned", 
"count": 1},
+                    ],
+                },
+            ),
+            (
+                {
+                    "granularity": "hourly",
+                    "logical_date_gte": "2025-01-02T00:00:00Z",
+                    "logical_date_lte": "2025-01-02T23:23:59Z",
+                },
+                {
+                    "total_entries": 2,
+                    "dag_runs": [
+                        {"date": "2025-01-02T00:00:00Z", "state": "running", 
"count": 1},
+                        {"date": "2025-01-02T01:00:00Z", "state": "planned", 
"count": 1},
+                    ],
+                },
+            ),
+        ],
+    )
+    def test_hourly_calendar(self, setup_dag_runs, test_client, query_params, 
result):
+        response = test_client.get(f"/calendar/{self.DAG_NAME}", 
params=query_params)
+        assert response.status_code == 200
+        body = response.json()
+
+        assert body == result

Reply via email to