diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml
index 045a49b4f6e77..cf37c187c6ba9 100644
--- a/.github/boring-cyborg.yml
+++ b/.github/boring-cyborg.yml
@@ -43,9 +43,7 @@ labelPRBasedOnFilePath:
- providers/apache/druid/**
provider:apache-flink:
- - providers/src/airflow/providers/apache/flink/**/*
- - docs/apache-airflow-providers-apache-flink/**/*
- - providers/tests/apache/flink/**/*
+ - providers/apache/flink/**
provider:apache-hdfs:
- providers/src/airflow/providers/apache/hdfs/**/*
@@ -59,9 +57,7 @@ labelPRBasedOnFilePath:
- providers/apache/iceberg/**
provider:apache-impala:
- - providers/src/airflow/providers/apache/impala/**/*
- - docs/apache-airflow-providers-apache-impala/**/*
- - providers/tests/apache/impala/**/*
+ - providers/apache/impala/**
provider:apache-kafka:
- providers/apache/kafka/**
@@ -99,19 +95,10 @@ labelPRBasedOnFilePath:
- providers/celery/**
provider:cloudant:
- - providers/src/airflow/providers/cloudant/**/*
- - docs/apache-airflow-providers-cloudant/**/*
- - providers/tests/cloudant/**/*
+ - providers/cloudant/**
provider:cncf-kubernetes:
- - airflow/example_dags/example_kubernetes_executor.py
- - airflow/example_dags/example_local_kubernetes_executor.py
- - providers/src/airflow/providers/cncf/kubernetes/**/*
- - providers/celery/src/airflow/providers/celery/executors/celery_kubernetes_executor.py
- - docs/apache-airflow-providers-cncf-kubernetes/**/*
- - kubernetes_tests/**/*
- - providers/tests/cncf/kubernetes/**/*
- - providers/tests/system/cncf/kubernetes/**/*
+ - providers/cncf/kubernetes/**
provider:cohere:
- providers/cohere/**
@@ -207,10 +194,7 @@ labelPRBasedOnFilePath:
- providers/microsoft/psrp/**
provider:microsoft-winrm:
- - providers/src/airflow/providers/microsoft/winrm/**/*
- - docs/apache-airflow-providers-microsoft-winrm/**/*
- - providers/tests/microsoft/winrm/**/*
- - providers/tests/system/microsoft/winrm/**/*
+ - providers/microsoft/winrm/**
provider:mongo:
- providers/mongo/**
@@ -320,10 +304,7 @@ labelPRBasedOnFilePath:
- providers/weaviate/**
provider:yandex:
- - providers/src/airflow/providers/yandex/**/*
- - docs/apache-airflow-providers-yandex/**/*
- - providers/tests/yandex/**/*
- - providers/tests/system/yandex/**/*
+ - providers/yandex/**
provider:ydb:
- providers/ydb/**
diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml
index b8758146cc1b1..7b9adcf60dba5 100644
--- a/.github/workflows/release_dockerhub_image.yml
+++ b/.github/workflows/release_dockerhub_image.yml
@@ -177,7 +177,7 @@ jobs:
${SKIP_LATEST}
${LIMIT_PLATFORM}
--limit-python ${PYTHON_VERSION}
- --chicken-egg-providers ${CHICKEN_EGG_PROVIDERS}
+ --chicken-egg-providers "${CHICKEN_EGG_PROVIDERS}"
- name: >
Release slim images: ${{ github.event.inputs.airflowVersion }}, ${{ matrix.python-version }}
env:
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index e59a6d1c1f451..e58166f9d383d 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -641,11 +641,11 @@ repos:
^providers/google/src/airflow/providers/google/cloud/operators/dataproc.py$|
^providers/google/src/airflow/providers/google/cloud/operators/mlengine.py$|
^providers/src/airflow/providers/microsoft/azure/hooks/cosmos.py$|
- ^providers/src/airflow/providers/microsoft/winrm/hooks/winrm.py$|
+ ^providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm.py$|
^airflow/www/fab_security/manager.py$|
^docs/.*commits.rst$|
^docs/apache-airflow-providers-apache-cassandra/connections/cassandra.rst$|
- ^providers/src/airflow/providers/microsoft/winrm/operators/winrm.py$|
+ ^providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/operators/winrm.py$|
^providers/opsgenie/src/airflow/providers/opsgenie/hooks/opsgenie.py$|
^providers/redis/src/airflow/providers/redis/provider.yaml$|
^airflow/serialization/serialized_objects.py$|
@@ -665,7 +665,7 @@ repos:
^providers/fab/docs/auth-manager/webserver-authentication.rst$|
^providers/google/docs/operators/cloud/kubernetes_engine.rst$|
^docs/apache-airflow-providers-microsoft-azure/connections/azure_cosmos.rst$|
- ^docs/apache-airflow-providers-cncf-kubernetes/operators.rst$|
+ ^providers/cncf/kubernetes/docs/operators.rst$|
^docs/conf.py$|
^docs/exts/removemarktransform.py$|
^newsfragments/41761.significant.rst$|
@@ -1223,12 +1223,12 @@ repos:
^airflow/serialization/serde.py$ |
^airflow/utils/file.py$ |
^airflow/utils/helpers.py$ |
- ^airflow/utils/log/secrets_masker.py$ |
^providers/ |
^tests/ |
^providers/tests/ |
^providers/.*/tests/ |
^task_sdk/src/airflow/sdk/definitions/dag.py$ |
+ ^task_sdk/src/airflow/sdk/execution_time/secrets_masker.py$ |
^task_sdk/src/airflow/sdk/definitions/_internal/node.py$ |
^dev/.*\.py$ |
^scripts/.*\.py$ |
diff --git a/Dockerfile.ci b/Dockerfile.ci
index 27ec994edd5bf..baf31cab0314e 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -930,17 +930,12 @@ function check_boto_upgrade() {
echo
echo "${COLOR_BLUE}Upgrading boto3, botocore to latest version to run Amazon tests with them${COLOR_RESET}"
echo
- # shellcheck disable=SC2086
- ${PACKAGING_TOOL_CMD} uninstall ${EXTRA_UNINSTALL_FLAGS} aiobotocore s3fs yandexcloud opensearch-py || true
- # We need to include few dependencies to pass pip check with other dependencies:
- # * oss2 as dependency as otherwise jmespath will be bumped (sync with alibaba provider)
- # * cryptography is kept for snowflake-connector-python limitation (sync with snowflake provider)
set -x
# shellcheck disable=SC2086
- ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade boto3 botocore \
- "oss2>=2.14.0" "cryptography<43.0.0" "opensearch-py"
+ ${PACKAGING_TOOL_CMD} uninstall ${EXTRA_UNINSTALL_FLAGS} aiobotocore s3fs || true
+ # shellcheck disable=SC2086
+ ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade boto3 botocore
set +x
- pip check
}
function check_downgrade_sqlalchemy() {
diff --git a/airflow/api_connexion/schemas/connection_schema.py b/airflow/api_connexion/schemas/connection_schema.py
index 4288ce079c554..40085469c2338 100644
--- a/airflow/api_connexion/schemas/connection_schema.py
+++ b/airflow/api_connexion/schemas/connection_schema.py
@@ -53,7 +53,7 @@ class ConnectionSchema(ConnectionCollectionItemSchema):
def serialize_extra(obj: Connection):
if obj.extra is None:
return
- from airflow.utils.log.secrets_masker import redact
+ from airflow.sdk.execution_time.secrets_masker import redact
try:
extra = json.loads(obj.extra)
diff --git a/airflow/api_fastapi/app.py b/airflow/api_fastapi/app.py
index ff74deb2fee4a..4323fcd5017ca 100644
--- a/airflow/api_fastapi/app.py
+++ b/airflow/api_fastapi/app.py
@@ -79,6 +79,7 @@ def create_app(apps: str = "all") -> FastAPI:
if "execution" in apps_list or "all" in apps_list:
task_exec_api_app = create_task_execution_api_app(app)
+ init_error_handlers(task_exec_api_app)
app.mount("/execution", task_exec_api_app)
init_config(app)
diff --git a/airflow/api_fastapi/core_api/datamodels/assets.py b/airflow/api_fastapi/core_api/datamodels/assets.py
index fd8f7cef2d415..e25713f02c1c9 100644
--- a/airflow/api_fastapi/core_api/datamodels/assets.py
+++ b/airflow/api_fastapi/core_api/datamodels/assets.py
@@ -22,7 +22,7 @@
from pydantic import Field, field_validator
from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
-from airflow.utils.log.secrets_masker import redact
+from airflow.sdk.execution_time.secrets_masker import redact
class DagScheduleAssetReference(StrictBaseModel):
diff --git a/airflow/api_fastapi/core_api/datamodels/connections.py b/airflow/api_fastapi/core_api/datamodels/connections.py
index 4650e1354dc1e..19a50eac79bd7 100644
--- a/airflow/api_fastapi/core_api/datamodels/connections.py
+++ b/airflow/api_fastapi/core_api/datamodels/connections.py
@@ -23,7 +23,7 @@
from pydantic_core.core_schema import ValidationInfo
from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
-from airflow.utils.log.secrets_masker import redact
+from airflow.sdk.execution_time.secrets_masker import redact
# Response Models
diff --git a/airflow/api_fastapi/core_api/datamodels/dag_tags.py b/airflow/api_fastapi/core_api/datamodels/dag_tags.py
index 8d5014fdf3488..5b712c086009a 100644
--- a/airflow/api_fastapi/core_api/datamodels/dag_tags.py
+++ b/airflow/api_fastapi/core_api/datamodels/dag_tags.py
@@ -17,16 +17,12 @@
from __future__ import annotations
-from pydantic import ConfigDict
-
from airflow.api_fastapi.core_api.base import BaseModel
class DagTagResponse(BaseModel):
"""DAG Tag serializer for responses."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
name: str
dag_id: str
diff --git a/airflow/api_fastapi/core_api/datamodels/event_logs.py b/airflow/api_fastapi/core_api/datamodels/event_logs.py
index 8ea88f363e947..26a1364a2db98 100644
--- a/airflow/api_fastapi/core_api/datamodels/event_logs.py
+++ b/airflow/api_fastapi/core_api/datamodels/event_logs.py
@@ -19,7 +19,7 @@
from datetime import datetime
-from pydantic import ConfigDict, Field
+from pydantic import Field
from airflow.api_fastapi.core_api.base import BaseModel
@@ -27,8 +27,6 @@
class EventLogResponse(BaseModel):
"""Event Log Response."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
id: int = Field(alias="event_log_id")
dttm: datetime = Field(alias="when")
dag_id: str | None
diff --git a/airflow/api_fastapi/core_api/datamodels/import_error.py b/airflow/api_fastapi/core_api/datamodels/import_error.py
index baf1ffa4fb7f1..ccb72b95d6507 100644
--- a/airflow/api_fastapi/core_api/datamodels/import_error.py
+++ b/airflow/api_fastapi/core_api/datamodels/import_error.py
@@ -18,7 +18,7 @@
from datetime import datetime
-from pydantic import ConfigDict, Field
+from pydantic import Field
from airflow.api_fastapi.core_api.base import BaseModel
@@ -26,8 +26,6 @@
class ImportErrorResponse(BaseModel):
"""Import Error Response."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
id: int = Field(alias="import_error_id")
timestamp: datetime
filename: str
diff --git a/airflow/api_fastapi/core_api/datamodels/pools.py b/airflow/api_fastapi/core_api/datamodels/pools.py
index 096e357dfaf1d..2e7ae13cfcdb2 100644
--- a/airflow/api_fastapi/core_api/datamodels/pools.py
+++ b/airflow/api_fastapi/core_api/datamodels/pools.py
@@ -19,7 +19,7 @@
from typing import Annotated, Callable
-from pydantic import BeforeValidator, ConfigDict, Field
+from pydantic import BeforeValidator, Field
from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
@@ -63,8 +63,6 @@ class PoolCollectionResponse(BaseModel):
class PoolPatchBody(StrictBaseModel):
"""Pool serializer for patch bodies."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
name: str | None = Field(default=None, alias="pool")
slots: int | None = None
description: str | None = None
diff --git a/airflow/api_fastapi/core_api/datamodels/task_instances.py b/airflow/api_fastapi/core_api/datamodels/task_instances.py
index 505b9da207c39..d209d46bd5edc 100644
--- a/airflow/api_fastapi/core_api/datamodels/task_instances.py
+++ b/airflow/api_fastapi/core_api/datamodels/task_instances.py
@@ -23,7 +23,6 @@
AliasPath,
AwareDatetime,
BeforeValidator,
- ConfigDict,
Field,
NonNegativeInt,
StringConstraints,
@@ -42,8 +41,6 @@
class TaskInstanceResponse(BaseModel):
"""TaskInstance serializer for responses."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
id: str
task_id: str
dag_id: str
@@ -126,8 +123,6 @@ class TaskInstancesBatchBody(StrictBaseModel):
class TaskInstanceHistoryResponse(BaseModel):
"""TaskInstanceHistory serializer for responses."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
task_id: str
dag_id: str
@@ -154,6 +149,7 @@ class TaskInstanceHistoryResponse(BaseModel):
pid: int | None
executor: str | None
executor_config: Annotated[str, BeforeValidator(str)]
+ dag_version: DagVersionResponse | None
class TaskInstanceHistoryCollectionResponse(BaseModel):
diff --git a/airflow/api_fastapi/core_api/datamodels/variables.py b/airflow/api_fastapi/core_api/datamodels/variables.py
index 82cfdbb130523..2317d8a168b82 100644
--- a/airflow/api_fastapi/core_api/datamodels/variables.py
+++ b/airflow/api_fastapi/core_api/datamodels/variables.py
@@ -19,19 +19,17 @@
import json
-from pydantic import ConfigDict, Field, model_validator
+from pydantic import Field, model_validator
from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
from airflow.models.base import ID_LEN
+from airflow.sdk.execution_time.secrets_masker import redact
from airflow.typing_compat import Self
-from airflow.utils.log.secrets_masker import redact
class VariableResponse(BaseModel):
"""Variable serializer for responses."""
- model_config = ConfigDict(populate_by_name=True, from_attributes=True)
-
key: str
val: str = Field(alias="value")
description: str | None
diff --git a/airflow/api_fastapi/core_api/datamodels/xcom.py b/airflow/api_fastapi/core_api/datamodels/xcom.py
index f874f8bdeed5a..1acccb702efaf 100644
--- a/airflow/api_fastapi/core_api/datamodels/xcom.py
+++ b/airflow/api_fastapi/core_api/datamodels/xcom.py
@@ -65,3 +65,10 @@ class XComCreateBody(StrictBaseModel):
key: str
value: Any
map_index: int = -1
+
+
+class XComUpdateBody(StrictBaseModel):
+ """Payload serializer for updating an XCom entry."""
+
+ value: Any
+ map_index: int = -1
diff --git a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml
index cab7daffbbe3b..486c371682fdb 100644
--- a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml
+++ b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml
@@ -4380,6 +4380,80 @@ paths:
application/json:
schema:
$ref: '#/components/schemas/HTTPValidationError'
+ patch:
+ tags:
+ - XCom
+ summary: Update Xcom Entry
+ description: Update an existing XCom entry.
+ operationId: update_xcom_entry
+ parameters:
+ - name: dag_id
+ in: path
+ required: true
+ schema:
+ type: string
+ title: Dag Id
+ - name: task_id
+ in: path
+ required: true
+ schema:
+ type: string
+ title: Task Id
+ - name: dag_run_id
+ in: path
+ required: true
+ schema:
+ type: string
+ title: Dag Run Id
+ - name: xcom_key
+ in: path
+ required: true
+ schema:
+ type: string
+ title: Xcom Key
+ requestBody:
+ required: true
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/XComUpdateBody'
+ responses:
+ '200':
+ description: Successful Response
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/XComResponseNative'
+ '401':
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/HTTPExceptionResponse'
+ description: Unauthorized
+ '403':
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/HTTPExceptionResponse'
+ description: Forbidden
+ '400':
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/HTTPExceptionResponse'
+ description: Bad Request
+ '404':
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/HTTPExceptionResponse'
+ description: Not Found
+ '422':
+ description: Validation Error
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/HTTPValidationError'
/public/dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances/{task_id}/xcomEntries:
get:
tags:
@@ -9794,6 +9868,10 @@ components:
executor_config:
type: string
title: Executor Config
+ dag_version:
+ anyOf:
+ - $ref: '#/components/schemas/DagVersionResponse'
+ - type: 'null'
type: object
required:
- task_id
@@ -9819,6 +9897,7 @@ components:
- pid
- executor
- executor_config
+ - dag_version
title: TaskInstanceHistoryResponse
description: TaskInstanceHistory serializer for responses.
TaskInstanceResponse:
@@ -10742,3 +10821,17 @@ components:
- value
title: XComResponseString
description: XCom response serializer with string return type.
+ XComUpdateBody:
+ properties:
+ value:
+ title: Value
+ map_index:
+ type: integer
+ title: Map Index
+ default: -1
+ additionalProperties: false
+ type: object
+ required:
+ - value
+ title: XComUpdateBody
+ description: Payload serializer for updating an XCom entry.
diff --git a/airflow/api_fastapi/core_api/routes/public/job.py b/airflow/api_fastapi/core_api/routes/public/job.py
index b0c99e3ab32b9..71c3857a5e727 100644
--- a/airflow/api_fastapi/core_api/routes/public/job.py
+++ b/airflow/api_fastapi/core_api/routes/public/job.py
@@ -37,7 +37,6 @@
from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.core_api.datamodels.job import (
JobCollectionResponse,
- JobResponse,
)
from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc
from airflow.jobs.job import Job
@@ -124,12 +123,6 @@ def get_jobs(
jobs = [job for job in jobs if job.is_alive()]
return JobCollectionResponse(
- jobs=[
- JobResponse.model_validate(
- job,
- from_attributes=True,
- )
- for job in jobs
- ],
+ jobs=jobs,
total_entries=total_entries,
)
diff --git a/airflow/api_fastapi/core_api/routes/public/task_instances.py b/airflow/api_fastapi/core_api/routes/public/task_instances.py
index d3a67b9804911..927f62028a85d 100644
--- a/airflow/api_fastapi/core_api/routes/public/task_instances.py
+++ b/airflow/api_fastapi/core_api/routes/public/task_instances.py
@@ -272,11 +272,15 @@ def get_task_instance_tries(
"""Get list of task instances history."""
def _query(orm_object: Base) -> Select:
- query = select(orm_object).where(
- orm_object.dag_id == dag_id,
- orm_object.run_id == dag_run_id,
- orm_object.task_id == task_id,
- orm_object.map_index == map_index,
+ query = (
+ select(orm_object)
+ .where(
+ orm_object.dag_id == dag_id,
+ orm_object.run_id == dag_run_id,
+ orm_object.task_id == task_id,
+ orm_object.map_index == map_index,
+ )
+ .options(joinedload(orm_object.dag_version))
)
return query
@@ -291,7 +295,6 @@ def _query(orm_object: Base) -> Select:
status.HTTP_404_NOT_FOUND,
f"The Task Instance with dag_id: `{dag_id}`, run_id: `{dag_run_id}`, task_id: `{task_id}` and map_index: `{map_index}` was not found",
)
-
return TaskInstanceHistoryCollectionResponse(
task_instances=cast(list[TaskInstanceHistoryResponse], task_instances),
total_entries=len(task_instances),
@@ -659,13 +662,7 @@ def post_clear_task_instances(
)
return TaskInstanceCollectionResponse(
- task_instances=[
- TaskInstanceResponse.model_validate(
- ti,
- from_attributes=True,
- )
- for ti in task_instances
- ],
+ task_instances=task_instances,
total_entries=len(task_instances),
)
@@ -775,7 +772,6 @@ def patch_task_instance_dry_run(
task_instances=[
TaskInstanceResponse.model_validate(
ti,
- from_attributes=True,
)
for ti in tis
],
@@ -840,4 +836,4 @@ def patch_task_instance(
ti.task_instance_note.user_id = None
session.commit()
- return TaskInstanceResponse.model_validate(ti, from_attributes=True)
+ return TaskInstanceResponse.model_validate(ti)
diff --git a/airflow/api_fastapi/core_api/routes/public/xcom.py b/airflow/api_fastapi/core_api/routes/public/xcom.py
index e1ef40685d39f..3da163f3e4033 100644
--- a/airflow/api_fastapi/core_api/routes/public/xcom.py
+++ b/airflow/api_fastapi/core_api/routes/public/xcom.py
@@ -30,6 +30,7 @@
XComCreateBody,
XComResponseNative,
XComResponseString,
+ XComUpdateBody,
)
from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc
from airflow.exceptions import TaskNotFound
@@ -222,3 +223,48 @@ def create_xcom_entry(
)
return XComResponseNative.model_validate(xcom)
+
+
+@xcom_router.patch(
+ "/{xcom_key}",
+ status_code=status.HTTP_200_OK,
+ responses=create_openapi_http_exception_doc(
+ [
+ status.HTTP_400_BAD_REQUEST,
+ status.HTTP_404_NOT_FOUND,
+ ]
+ ),
+)
+def update_xcom_entry(
+ dag_id: str,
+ task_id: str,
+ dag_run_id: str,
+ xcom_key: str,
+ patch_body: XComUpdateBody,
+ session: SessionDep,
+) -> XComResponseNative:
+ """Update an existing XCom entry."""
+ # Check if XCom entry exists
+ xcom_new_value = XCom.serialize_value(patch_body.value)
+ xcom_entry = session.scalar(
+ select(XCom)
+ .where(
+ XCom.dag_id == dag_id,
+ XCom.task_id == task_id,
+ XCom.run_id == dag_run_id,
+ XCom.key == xcom_key,
+ XCom.map_index == patch_body.map_index,
+ )
+ .limit(1)
+ )
+
+ if not xcom_entry:
+ raise HTTPException(
+ status.HTTP_404_NOT_FOUND,
+ f"The XCom with key: `{xcom_key}` with mentioned task instance doesn't exist.",
+ )
+
+ # Update XCom entry
+ xcom_entry.value = XCom.serialize_value(xcom_new_value)
+
+ return XComResponseNative.model_validate(xcom_entry)
diff --git a/airflow/api_fastapi/execution_api/app.py b/airflow/api_fastapi/execution_api/app.py
index 61283dc2cf87f..2b85be363f25f 100644
--- a/airflow/api_fastapi/execution_api/app.py
+++ b/airflow/api_fastapi/execution_api/app.py
@@ -77,8 +77,14 @@ def custom_openapi() -> dict:
def get_extra_schemas() -> dict[str, dict]:
"""Get all the extra schemas that are not part of the main FastAPI app."""
- from airflow.api_fastapi.execution_api.datamodels import taskinstance
+ from airflow.api_fastapi.execution_api.datamodels.taskinstance import TaskInstance
+ from airflow.executors.workloads import BundleInfo
+ from airflow.utils.state import TerminalTIState
return {
- "TaskInstance": taskinstance.TaskInstance.model_json_schema(),
+ "TaskInstance": TaskInstance.model_json_schema(),
+ "BundleInfo": BundleInfo.model_json_schema(),
+ # Include the combined state enum too. In the datamodels we separate out SUCCESS from the other states
+ # as that has different payload requirements
+ "TerminalTIState": {"type": "string", "enum": list(TerminalTIState)},
}
diff --git a/airflow/api_fastapi/execution_api/datamodels/asset.py b/airflow/api_fastapi/execution_api/datamodels/asset.py
index 29b260c291c2b..28d352aa23101 100644
--- a/airflow/api_fastapi/execution_api/datamodels/asset.py
+++ b/airflow/api_fastapi/execution_api/datamodels/asset.py
@@ -17,7 +17,7 @@
from __future__ import annotations
-from airflow.api_fastapi.core_api.base import BaseModel
+from airflow.api_fastapi.core_api.base import BaseModel, StrictBaseModel
class AssetResponse(BaseModel):
@@ -36,7 +36,7 @@ class AssetAliasResponse(BaseModel):
group: str
-class AssetProfile(BaseModel):
+class AssetProfile(StrictBaseModel):
"""
Profile of an Asset.
diff --git a/airflow/api_fastapi/execution_api/datamodels/taskinstance.py b/airflow/api_fastapi/execution_api/datamodels/taskinstance.py
index c3ec79c9ddd11..0c8e5eb1b6a3f 100644
--- a/airflow/api_fastapi/execution_api/datamodels/taskinstance.py
+++ b/airflow/api_fastapi/execution_api/datamodels/taskinstance.py
@@ -14,11 +14,11 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
-
from __future__ import annotations
import uuid
from datetime import timedelta
+from enum import Enum
from typing import Annotated, Any, Literal, Union
from pydantic import (
@@ -60,15 +60,20 @@ class TIEnterRunningPayload(StrictBaseModel):
"""When the task started executing"""
+# Create an enum to give a nice name in the generated datamodels
+class TerminalStateNonSuccess(str, Enum):
+ """TaskInstance states that can be reported without extra information."""
+
+ FAILED = TerminalTIState.FAILED
+ SKIPPED = TerminalTIState.SKIPPED
+ REMOVED = TerminalTIState.REMOVED
+ FAIL_WITHOUT_RETRY = TerminalTIState.FAIL_WITHOUT_RETRY
+
+
class TITerminalStatePayload(StrictBaseModel):
"""Schema for updating TaskInstance to a terminal state except SUCCESS state."""
- state: Literal[
- TerminalTIState.FAILED,
- TerminalTIState.SKIPPED,
- TerminalTIState.REMOVED,
- TerminalTIState.FAIL_WITHOUT_RETRY,
- ]
+ state: TerminalStateNonSuccess
end_date: UtcDateTime
"""When the task completed executing"""
@@ -216,7 +221,7 @@ class DagRun(StrictBaseModel):
dag_id: str
run_id: str
- logical_date: UtcDateTime
+ logical_date: UtcDateTime | None
data_interval_start: UtcDateTime | None
data_interval_end: UtcDateTime | None
run_after: UtcDateTime
@@ -242,6 +247,8 @@ class TIRunContext(BaseModel):
connections: Annotated[list[ConnectionResponse], Field(default_factory=list)]
"""Connections that can be accessed by the task instance."""
+ upstream_map_indexes: dict[str, int] | None = None
+
class PrevSuccessfulDagRunResponse(BaseModel):
"""Schema for response with previous successful DagRun information for Task Template Context."""
@@ -252,7 +259,7 @@ class PrevSuccessfulDagRunResponse(BaseModel):
end_date: UtcDateTime | None = None
-class TIRuntimeCheckPayload(BaseModel):
+class TIRuntimeCheckPayload(StrictBaseModel):
"""Payload for performing Runtime checks on the TaskInstance model as requested by the SDK."""
inlets: list[AssetProfile] | None = None
diff --git a/airflow/api_fastapi/execution_api/routes/task_instances.py b/airflow/api_fastapi/execution_api/routes/task_instances.py
index 3cc70dac9c5b5..1c605b5cadcf0 100644
--- a/airflow/api_fastapi/execution_api/routes/task_instances.py
+++ b/airflow/api_fastapi/execution_api/routes/task_instances.py
@@ -30,7 +30,6 @@
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.execution_api.datamodels.taskinstance import (
- DagRun,
PrevSuccessfulDagRunResponse,
TIDeferredStatePayload,
TIEnterRunningPayload,
@@ -66,6 +65,7 @@
status.HTTP_409_CONFLICT: {"description": "The TI is already in the requested state"},
status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload for the state transition"},
},
+ response_model_exclude_unset=True,
)
def ti_run(
task_instance_id: UUID, ti_run_payload: Annotated[TIEnterRunningPayload, Body()], session: SessionDep
@@ -150,6 +150,7 @@ def ti_run(
DR.run_type,
DR.conf,
DR.logical_date,
+ DR.external_trigger,
).filter_by(dag_id=dag_id, run_id=run_id)
).one_or_none()
@@ -171,7 +172,7 @@ def ti_run(
)
return TIRunContext(
- dag_run=DagRun.model_validate(dr, from_attributes=True),
+ dag_run=dr,
max_tries=max_tries,
# TODO: Add variables and connections that are needed (and has perms) for the task
variables=[],
diff --git a/airflow/api_fastapi/execution_api/routes/xcoms.py b/airflow/api_fastapi/execution_api/routes/xcoms.py
index f330744536b33..9a8cef62e07a0 100644
--- a/airflow/api_fastapi/execution_api/routes/xcoms.py
+++ b/airflow/api_fastapi/execution_api/routes/xcoms.py
@@ -21,7 +21,8 @@
import logging
from typing import Annotated
-from fastapi import Body, HTTPException, Query, status
+from fastapi import Body, Depends, HTTPException, Query, Response, status
+from sqlalchemy.sql.selectable import Select
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.common.router import AirflowRouter
@@ -30,6 +31,7 @@
from airflow.api_fastapi.execution_api.datamodels.xcom import XComResponse
from airflow.models.taskmap import TaskMap
from airflow.models.xcom import BaseXCom
+from airflow.utils.db import get_query_count
# TODO: Add dependency on JWT token
router = AirflowRouter(
@@ -42,20 +44,15 @@
log = logging.getLogger(__name__)
-@router.get(
- "/{dag_id}/{run_id}/{task_id}/{key}",
- responses={status.HTTP_404_NOT_FOUND: {"description": "XCom not found"}},
-)
-def get_xcom(
+async def xcom_query(
dag_id: str,
run_id: str,
task_id: str,
key: str,
- token: deps.TokenDep,
session: SessionDep,
- map_index: Annotated[int, Query()] = -1,
-) -> XComResponse:
- """Get an Airflow XCom from database - not other XCom Backends."""
+ token: deps.TokenDep,
+ map_index: Annotated[int | None, Query()] = None,
+) -> Select:
if not has_xcom_access(dag_id, run_id, task_id, key, token):
raise HTTPException(
status_code=status.HTTP_403_FORBIDDEN,
@@ -65,29 +62,87 @@ def get_xcom(
},
)
- # We use `BaseXCom.get_many` to fetch XComs directly from the database, bypassing the XCom Backend.
- # This avoids deserialization via the backend (e.g., from a remote storage like S3) and instead
- # retrieves the raw serialized value from the database. By not relying on `XCom.get_many` or `XCom.get_one`
- # (which automatically deserializes using the backend), we avoid potential
- # performance hits from retrieving large data files into the API server.
query = BaseXCom.get_many(
run_id=run_id,
key=key,
task_ids=task_id,
dag_ids=dag_id,
map_indexes=map_index,
- limit=1,
session=session,
)
+ return query.with_entities(BaseXCom.value)
+
+
+@router.head(
+ "/{dag_id}/{run_id}/{task_id}/{key}",
+ responses={
+ status.HTTP_404_NOT_FOUND: {"description": "XCom not found"},
+ status.HTTP_200_OK: {
+ "description": "Metadata about the number of matching XCom values",
+ "headers": {
+ "Content-Range": {
+ "pattern": r"^map_indexes \d+$",
+ "description": "The number of (mapped) XCom values found for this task.",
+ },
+ },
+ },
+ },
+ description="Return the count of the number of XCom values found via the Content-Range response header",
+)
+def head_xcom(
+ response: Response,
+ token: deps.TokenDep,
+ session: SessionDep,
+ xcom_query: Annotated[Select, Depends(xcom_query)],
+ map_index: Annotated[int | None, Query()] = None,
+) -> None:
+ """Get the count of XComs from database - not other XCom Backends."""
+ if map_index is not None:
+ raise HTTPException(
+ status_code=status.HTTP_400_BAD_REQUEST,
+ detail={"reason": "invalid_request", "message": "Cannot specify map_index in a HEAD request"},
+ )
+
+ count = get_query_count(xcom_query, session=session)
+ # Tell the caller how many items in this query. We define a custom range unit (HTTP spec only defines
+ # "bytes" but we can add our own)
+ response.headers["Content-Range"] = f"map_indexes {count}"
+
+
+@router.get(
+ "/{dag_id}/{run_id}/{task_id}/{key}",
+ responses={status.HTTP_404_NOT_FOUND: {"description": "XCom not found"}},
+ description="Get a single XCom Value",
+)
+def get_xcom(
+ session: SessionDep,
+ dag_id: str,
+ run_id: str,
+ task_id: str,
+ key: str,
+ xcom_query: Annotated[Select, Depends(xcom_query)],
+ map_index: Annotated[int, Query()] = -1,
+) -> XComResponse:
+ """Get an Airflow XCom from database - not other XCom Backends."""
+ # The xcom_query allows no map_index to be passed. This endpoint should always return just a single item,
+ # so we override that query value
+
+ xcom_query = xcom_query.filter_by(map_index=map_index)
+
+ # We use `BaseXCom.get_many` to fetch XComs directly from the database, bypassing the XCom Backend.
+ # This avoids deserialization via the backend (e.g., from a remote storage like S3) and instead
+ # retrieves the raw serialized value from the database. By not relying on `XCom.get_many` or `XCom.get_one`
+ # (which automatically deserializes using the backend), we avoid potential
+ # performance hits from retrieving large data files into the API server.
- result = query.with_entities(BaseXCom.value).first()
+ result = xcom_query.limit(1).first()
if result is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
- "message": f"XCom with key '{key}' not found for task '{task_id}' in DAG '{dag_id}'",
+ "message": f"XCom with {key=} {map_index=} not found for task {task_id!r} in DAG run {run_id!r} of {dag_id!r}",
},
)
diff --git a/airflow/api_fastapi/logging/decorators.py b/airflow/api_fastapi/logging/decorators.py
index b7e1002c180a1..0c709839f3152 100644
--- a/airflow/api_fastapi/logging/decorators.py
+++ b/airflow/api_fastapi/logging/decorators.py
@@ -29,7 +29,7 @@
from airflow.api_fastapi.core_api.security import get_user_with_exception_handling
from airflow.auth.managers.models.base_user import BaseUser
from airflow.models import Log
-from airflow.utils.log import secrets_masker
+from airflow.sdk.execution_time import secrets_masker
logger = logging.getLogger(__name__)
diff --git a/airflow/auth/managers/simple/ui/dev/index.html b/airflow/auth/managers/simple/ui/dev/index.html
new file mode 100644
index 0000000000000..b27bdcd6795bf
--- /dev/null
+++ b/airflow/auth/managers/simple/ui/dev/index.html
@@ -0,0 +1,23 @@
+
+
+
+
+
+
+
+
+
+ Airflow 3.0
+
+
+
+
+
+
diff --git a/airflow/auth/managers/simple/ui/package-lock.json b/airflow/auth/managers/simple/ui/package-lock.json
index a0087cc944ce1..4898896322b97 100644
--- a/airflow/auth/managers/simple/ui/package-lock.json
+++ b/airflow/auth/managers/simple/ui/package-lock.json
@@ -25,7 +25,7 @@
"happy-dom": "^15.10.2",
"vite": "^5.4.14",
"vite-plugin-css-injected-by-js": "^3.5.2",
- "vitest": "^2.1.1"
+ "vitest": "^2.1.9"
}
},
"node_modules/@7nohe/openapi-react-query-codegen": {
@@ -1830,13 +1830,14 @@
}
},
"node_modules/@vitest/expect": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/expect/-/expect-2.1.8.tgz",
- "integrity": "sha512-8ytZ/fFHq2g4PJVAtDX57mayemKgDR6X3Oa2Foro+EygiOJHUXhCqBAAKQYYajZpFoIfvBCF1j6R6IYRSIUFuw==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/expect/-/expect-2.1.9.tgz",
+ "integrity": "sha512-UJCIkTBenHeKT1TTlKMJWy1laZewsRIzYighyYiJKZreqtdxSos/S1t+ktRMQWu2CKqaarrkeszJx1cgC5tGZw==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@vitest/spy": "2.1.8",
- "@vitest/utils": "2.1.8",
+ "@vitest/spy": "2.1.9",
+ "@vitest/utils": "2.1.9",
"chai": "^5.1.2",
"tinyrainbow": "^1.2.0"
},
@@ -1845,12 +1846,13 @@
}
},
"node_modules/@vitest/mocker": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/mocker/-/mocker-2.1.8.tgz",
- "integrity": "sha512-7guJ/47I6uqfttp33mgo6ga5Gr1VnL58rcqYKyShoRK9ebu8T5Rs6HN3s1NABiBeVTdWNrwUMcHH54uXZBN4zA==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/mocker/-/mocker-2.1.9.tgz",
+ "integrity": "sha512-tVL6uJgoUdi6icpxmdrn5YNo3g3Dxv+IHJBr0GXHaEdTcw3F+cPKnsXFhli6nO+f/6SDKPHEK1UN+k+TQv0Ehg==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@vitest/spy": "2.1.8",
+ "@vitest/spy": "2.1.9",
"estree-walker": "^3.0.3",
"magic-string": "^0.30.12"
},
@@ -1871,10 +1873,11 @@
}
},
"node_modules/@vitest/pretty-format": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/pretty-format/-/pretty-format-2.1.8.tgz",
- "integrity": "sha512-9HiSZ9zpqNLKlbIDRWOnAWqgcA7xu+8YxXSekhr0Ykab7PAYFkhkwoqVArPOtJhPmYeE2YHgKZlj3CP36z2AJQ==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/pretty-format/-/pretty-format-2.1.9.tgz",
+ "integrity": "sha512-KhRIdGV2U9HOUzxfiHmY8IFHTdqtOhIzCpd8WRdJiE7D/HUcZVD0EgQCVjm+Q9gkUXWgBvMmTtZgIG48wq7sOQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"tinyrainbow": "^1.2.0"
},
@@ -1883,12 +1886,13 @@
}
},
"node_modules/@vitest/runner": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/runner/-/runner-2.1.8.tgz",
- "integrity": "sha512-17ub8vQstRnRlIU5k50bG+QOMLHRhYPAna5tw8tYbj+jzjcspnwnwtPtiOlkuKC4+ixDPTuLZiqiWWQ2PSXHVg==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/runner/-/runner-2.1.9.tgz",
+ "integrity": "sha512-ZXSSqTFIrzduD63btIfEyOmNcBmQvgOVsPNPe0jYtESiXkhd8u2erDLnMxmGrDCwHCCHE7hxwRDCT3pt0esT4g==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@vitest/utils": "2.1.8",
+ "@vitest/utils": "2.1.9",
"pathe": "^1.1.2"
},
"funding": {
@@ -1896,12 +1900,13 @@
}
},
"node_modules/@vitest/snapshot": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/snapshot/-/snapshot-2.1.8.tgz",
- "integrity": "sha512-20T7xRFbmnkfcmgVEz+z3AU/3b0cEzZOt/zmnvZEctg64/QZbSDJEVm9fLnnlSi74KibmRsO9/Qabi+t0vCRPg==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/snapshot/-/snapshot-2.1.9.tgz",
+ "integrity": "sha512-oBO82rEjsxLNJincVhLhaxxZdEtV0EFHMK5Kmx5sJ6H9L183dHECjiefOAdnqpIgT5eZwT04PoggUnW88vOBNQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@vitest/pretty-format": "2.1.8",
+ "@vitest/pretty-format": "2.1.9",
"magic-string": "^0.30.12",
"pathe": "^1.1.2"
},
@@ -1910,10 +1915,11 @@
}
},
"node_modules/@vitest/spy": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/spy/-/spy-2.1.8.tgz",
- "integrity": "sha512-5swjf2q95gXeYPevtW0BLk6H8+bPlMb4Vw/9Em4hFxDcaOxS+e0LOX4yqNxoHzMR2akEB2xfpnWUzkZokmgWDg==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/spy/-/spy-2.1.9.tgz",
+ "integrity": "sha512-E1B35FwzXXTs9FHNK6bDszs7mtydNi5MIfUWpceJ8Xbfb1gBMscAnwLbEu+B44ed6W3XjL9/ehLPHR1fkf1KLQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"tinyspy": "^3.0.2"
},
@@ -1922,12 +1928,13 @@
}
},
"node_modules/@vitest/utils": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/@vitest/utils/-/utils-2.1.8.tgz",
- "integrity": "sha512-dwSoui6djdwbfFmIgbIjX2ZhIoG7Ex/+xpxyiEgIGzjliY8xGkcpITKTlp6B4MgtGkF2ilvm97cPM96XZaAgcA==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/@vitest/utils/-/utils-2.1.9.tgz",
+ "integrity": "sha512-v0psaMSkNJ3A2NMrUEHFRzJtDPFn+/VWZ5WxImB21T9fjucJRmS7xCS3ppEnARb9y11OAzaD+P2Ps+b+BGX5iQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@vitest/pretty-format": "2.1.8",
+ "@vitest/pretty-format": "2.1.9",
"loupe": "^3.1.2",
"tinyrainbow": "^1.2.0"
},
@@ -2758,6 +2765,7 @@
"resolved": "https://registry.npmjs.org/assertion-error/-/assertion-error-2.0.1.tgz",
"integrity": "sha512-Izi8RQcffqCeNVgFigKli1ssklIbpHnCYc6AknXGYoB6grJqyeby7jv12JUQgmTAnIDnbck1uxksT4dzN3PWBA==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">=12"
}
@@ -2865,6 +2873,7 @@
"resolved": "https://registry.npmjs.org/cac/-/cac-6.7.14.tgz",
"integrity": "sha512-b6Ilus+c3RrdDk+JhLKUAQfzzgLEPy6wcXqS7f/xe1EETvsDP6GORG7SFuOs6cID5YkqchW/LXZbX5bc8j7ZcQ==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">=8"
}
@@ -2894,6 +2903,7 @@
"resolved": "https://registry.npmjs.org/chai/-/chai-5.1.2.tgz",
"integrity": "sha512-aGtmf24DW6MLHHG5gCx4zaI3uBq3KRtxeVs0DjFH6Z0rDNbsvTxFASFvdj79pxjxZ8/5u3PIiN3IwEIQkiiuPw==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"assertion-error": "^2.0.1",
"check-error": "^2.1.1",
@@ -2926,6 +2936,7 @@
"resolved": "https://registry.npmjs.org/check-error/-/check-error-2.1.1.tgz",
"integrity": "sha512-OAlb+T7V4Op9OwdkjmguYRqncdlx5JiofwOAUkmTF+jNdHwzTaTs4sRAGpzLF3oOz5xAyDGrPgeIDFQmDOTiJw==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">= 16"
}
@@ -3118,6 +3129,7 @@
"resolved": "https://registry.npmjs.org/deep-eql/-/deep-eql-5.0.2.tgz",
"integrity": "sha512-h5k/5U50IJJFpzfL6nO9jaaumfjO/f2NjK/oYB2Djzm4p9L+3T9qWpZqZ2hAbLPuuYq9wrU08WQyBTL5GbPk5Q==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">=6"
}
@@ -3215,7 +3227,8 @@
"version": "1.6.0",
"resolved": "https://registry.npmjs.org/es-module-lexer/-/es-module-lexer-1.6.0.tgz",
"integrity": "sha512-qqnD1yMU6tk/jnaMosogGySTZP8YtUgAffA9nMN+E/rjxcfRQ6IEk7IiozUjgxKoFHBGjTLnrHB/YC45r/59EQ==",
- "dev": true
+ "dev": true,
+ "license": "MIT"
},
"node_modules/esbuild": {
"version": "0.21.5",
@@ -3408,6 +3421,7 @@
"resolved": "https://registry.npmjs.org/estree-walker/-/estree-walker-3.0.3.tgz",
"integrity": "sha512-7RUKfXgSMMkzt6ZuXmqapOurLGPPfgj6l9uRZ7lRGolvk0y2yocc35LdcxKC5PQZdn2DMqioAQ2NoWcrTKmm6g==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"@types/estree": "^1.0.0"
}
@@ -4132,10 +4146,11 @@
}
},
"node_modules/loupe": {
- "version": "3.1.2",
- "resolved": "https://registry.npmjs.org/loupe/-/loupe-3.1.2.tgz",
- "integrity": "sha512-23I4pFZHmAemUnz8WZXbYRSKYj801VDaNv9ETuMh7IrMc7VuVVSo+Z9iLE3ni30+U48iDWfi30d3twAXBYmnCg==",
- "dev": true
+ "version": "3.1.3",
+ "resolved": "https://registry.npmjs.org/loupe/-/loupe-3.1.3.tgz",
+ "integrity": "sha512-kkIp7XSkP78ZxJEsSxW3712C6teJVoeHHwgo9zJ380de7IYyJ2ISlxojcH2pC5OFLewESmnRi/+XCDIEEVyoug==",
+ "dev": true,
+ "license": "MIT"
},
"node_modules/lru-cache": {
"version": "10.4.3",
@@ -4159,6 +4174,7 @@
"resolved": "https://registry.npmjs.org/magic-string/-/magic-string-0.30.17.tgz",
"integrity": "sha512-sNPKHvyjVf7gyjwS4xGTaW/mCnF8wnjtifKBEhxfZ7E/S8tQ0rssrwGNn6q8JH/ohItJfSQp9mBtQYuTlH5QnA==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"@jridgewell/sourcemap-codec": "^1.5.0"
}
@@ -4588,6 +4604,7 @@
"resolved": "https://registry.npmjs.org/pathval/-/pathval-2.0.0.tgz",
"integrity": "sha512-vE7JKRyES09KiunauX7nd2Q9/L7lhok4smP9RZTDeD4MVs72Dp2qNFVz39Nz5a0FVEW0BJR6C0DYrq6unoziZA==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">= 14.16"
}
@@ -5264,6 +5281,7 @@
"resolved": "https://registry.npmjs.org/tinyrainbow/-/tinyrainbow-1.2.0.tgz",
"integrity": "sha512-weEDEq7Z5eTHPDh4xjX789+fHfF+P8boiFB+0vbWzpbnbsEr/GRaohi/uMKxg8RZMXnl1ItAi/IUHWMsjDV7kQ==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">=14.0.0"
}
@@ -5273,6 +5291,7 @@
"resolved": "https://registry.npmjs.org/tinyspy/-/tinyspy-3.0.2.tgz",
"integrity": "sha512-n1cw8k1k0x4pgA2+9XrOkFydTerNcJ1zWCO5Nn9scWHTD+5tp8dghT2x1uduQePZTZgd3Tupf+x9BxJjeJi77Q==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">=14.0.0"
}
@@ -5425,10 +5444,11 @@
}
},
"node_modules/vite-node": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/vite-node/-/vite-node-2.1.8.tgz",
- "integrity": "sha512-uPAwSr57kYjAUux+8E2j0q0Fxpn8M9VoyfGiRI8Kfktz9NcYMCenwY5RnZxnF1WTu3TGiYipirIzacLL3VVGFg==",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/vite-node/-/vite-node-2.1.9.tgz",
+ "integrity": "sha512-AM9aQ/IPrW/6ENLQg3AGY4K1N2TGZdR5e4gu/MmmR2xR3Ll1+dib+nook92g4TV3PXVyeyxdWwtaCAiUL0hMxA==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"cac": "^6.7.14",
"debug": "^4.3.7",
@@ -5456,18 +5476,19 @@
}
},
"node_modules/vitest": {
- "version": "2.1.8",
- "resolved": "https://registry.npmjs.org/vitest/-/vitest-2.1.8.tgz",
- "integrity": "sha512-1vBKTZskHw/aosXqQUlVWWlGUxSJR8YtiyZDJAFeW2kPAeX6S3Sool0mjspO+kXLuxVWlEDDowBAeqeAQefqLQ==",
- "dev": true,
- "dependencies": {
- "@vitest/expect": "2.1.8",
- "@vitest/mocker": "2.1.8",
- "@vitest/pretty-format": "^2.1.8",
- "@vitest/runner": "2.1.8",
- "@vitest/snapshot": "2.1.8",
- "@vitest/spy": "2.1.8",
- "@vitest/utils": "2.1.8",
+ "version": "2.1.9",
+ "resolved": "https://registry.npmjs.org/vitest/-/vitest-2.1.9.tgz",
+ "integrity": "sha512-MSmPM9REYqDGBI8439mA4mWhV5sKmDlBKWIYbA3lRb2PTHACE0mgKwA8yQ2xq9vxDTuk4iPrECBAEW2aoFXY0Q==",
+ "dev": true,
+ "license": "MIT",
+ "dependencies": {
+ "@vitest/expect": "2.1.9",
+ "@vitest/mocker": "2.1.9",
+ "@vitest/pretty-format": "^2.1.9",
+ "@vitest/runner": "2.1.9",
+ "@vitest/snapshot": "2.1.9",
+ "@vitest/spy": "2.1.9",
+ "@vitest/utils": "2.1.9",
"chai": "^5.1.2",
"debug": "^4.3.7",
"expect-type": "^1.1.0",
@@ -5479,7 +5500,7 @@
"tinypool": "^1.0.1",
"tinyrainbow": "^1.2.0",
"vite": "^5.0.0",
- "vite-node": "2.1.8",
+ "vite-node": "2.1.9",
"why-is-node-running": "^2.3.0"
},
"bin": {
@@ -5494,8 +5515,8 @@
"peerDependencies": {
"@edge-runtime/vm": "*",
"@types/node": "^18.0.0 || >=20.0.0",
- "@vitest/browser": "2.1.8",
- "@vitest/ui": "2.1.8",
+ "@vitest/browser": "2.1.9",
+ "@vitest/ui": "2.1.9",
"happy-dom": "*",
"jsdom": "*"
},
diff --git a/airflow/auth/managers/simple/ui/package.json b/airflow/auth/managers/simple/ui/package.json
index 2001ef7162626..552d25f84a7b1 100644
--- a/airflow/auth/managers/simple/ui/package.json
+++ b/airflow/auth/managers/simple/ui/package.json
@@ -4,7 +4,7 @@
"version": "0.0.0",
"type": "module",
"scripts": {
- "dev": "vite",
+ "dev": "vite --port 5174",
"build": "vite build",
"preview": "vite preview",
"codegen": "openapi-rq -i \"../openapi/v1-generated.yaml\" -c axios --format prettier -o openapi-gen --operationId",
@@ -29,6 +29,6 @@
"happy-dom": "^15.10.2",
"vite": "^5.4.14",
"vite-plugin-css-injected-by-js": "^3.5.2",
- "vitest": "^2.1.1"
+ "vitest": "^2.1.9"
}
}
diff --git a/airflow/auth/managers/simple/ui/pnpm-lock.yaml b/airflow/auth/managers/simple/ui/pnpm-lock.yaml
index 1676d87998832..dc6fe7218cc16 100644
--- a/airflow/auth/managers/simple/ui/pnpm-lock.yaml
+++ b/airflow/auth/managers/simple/ui/pnpm-lock.yaml
@@ -55,8 +55,8 @@ importers:
specifier: ^3.5.2
version: 3.5.2(vite@5.4.14)
vitest:
- specifier: ^2.1.1
- version: 2.1.8(happy-dom@15.11.7)
+ specifier: ^2.1.9
+ version: 2.1.9(happy-dom@15.11.7)
packages:
@@ -103,8 +103,8 @@ packages:
resolution: {integrity: sha512-Ed61U6XJc3CVRfkERJWDz4dJwKe7iLmmJsbOGu9wSloNSFttHV0I8g6UAgb7qnK5ly5bGLPd4oXZlxCdANBOWQ==}
engines: {node: '>=6.9.0'}
- '@babel/parser@7.26.5':
- resolution: {integrity: sha512-SRJ4jYmXRqV1/Xc+TIVG84WjHBXKlxO9sHQnA2Pf12QQEAp1LOh6kDzNHXcUnbH1QI0FDoPPVOt+vyUDucxpaw==}
+ '@babel/parser@7.26.7':
+ resolution: {integrity: sha512-kEvgGGgEjRUutvdVvZhbn/BxVt+5VSpwXz1j3WYXQbXDo8KzFOPNG2GQbdAiNq8g6wn1yKk7C/qrke03a84V+w==}
engines: {node: '>=6.0.0'}
hasBin: true
@@ -112,16 +112,20 @@ packages:
resolution: {integrity: sha512-FDSOghenHTiToteC/QRlv2q3DhPZ/oOXTBoirfWNx1Cx3TMVcGWQtMMmQcSvb/JjpNeGzx8Pq/b4fKEJuWm1sw==}
engines: {node: '>=6.9.0'}
+ '@babel/runtime@7.26.7':
+ resolution: {integrity: sha512-AOPI3D+a8dXnja+iwsUqGRjr1BbZIe771sXdapOtYI531gSqpi92vXivKcq2asu/DFpdl1ceFAKZyRzK2PCVcQ==}
+ engines: {node: '>=6.9.0'}
+
'@babel/template@7.25.9':
resolution: {integrity: sha512-9DGttpmPvIxBb/2uwpVo3dqJ+O6RooAFOS+lB+xDqoE2PVCE8nfoHMdZLpfCQRLwvohzXISPZcgxt80xLfsuwg==}
engines: {node: '>=6.9.0'}
- '@babel/traverse@7.26.5':
- resolution: {integrity: sha512-rkOSPOw+AXbgtwUga3U4u8RpoK9FEFWBNAlTpcnkLFjL5CT+oyHNuUUC/xx6XefEJ16r38r8Bc/lfp6rYuHeJQ==}
+ '@babel/traverse@7.26.7':
+ resolution: {integrity: sha512-1x1sgeyRLC3r5fQOM0/xtQKsYjyxmFjaOrLJNtZ81inNjyJHGIolTULPiSc/2qe1/qfpFLisLQYFnnZl7QoedA==}
engines: {node: '>=6.9.0'}
- '@babel/types@7.26.5':
- resolution: {integrity: sha512-L6mZmwFDK6Cjh1nRCLXpa6no13ZIioJDz7mdkzHv399pThrTa/k0nUlNaenOeh2kWu/iaOQYElEpKPUswUa9Vg==}
+ '@babel/types@7.26.7':
+ resolution: {integrity: sha512-t8kDRGrKXyp6+tjUh7hw2RLyclsW4TRoRvRHtSyAX9Bb5ldlFh+90YAYY6awRXrlB4G5G2izNeGySpATlFzmOg==}
engines: {node: '>=6.9.0'}
'@chakra-ui/react@3.3.3':
@@ -661,11 +665,11 @@ packages:
peerDependencies:
vite: ^4 || ^5 || ^6
- '@vitest/expect@2.1.8':
- resolution: {integrity: sha512-8ytZ/fFHq2g4PJVAtDX57mayemKgDR6X3Oa2Foro+EygiOJHUXhCqBAAKQYYajZpFoIfvBCF1j6R6IYRSIUFuw==}
+ '@vitest/expect@2.1.9':
+ resolution: {integrity: sha512-UJCIkTBenHeKT1TTlKMJWy1laZewsRIzYighyYiJKZreqtdxSos/S1t+ktRMQWu2CKqaarrkeszJx1cgC5tGZw==}
- '@vitest/mocker@2.1.8':
- resolution: {integrity: sha512-7guJ/47I6uqfttp33mgo6ga5Gr1VnL58rcqYKyShoRK9ebu8T5Rs6HN3s1NABiBeVTdWNrwUMcHH54uXZBN4zA==}
+ '@vitest/mocker@2.1.9':
+ resolution: {integrity: sha512-tVL6uJgoUdi6icpxmdrn5YNo3g3Dxv+IHJBr0GXHaEdTcw3F+cPKnsXFhli6nO+f/6SDKPHEK1UN+k+TQv0Ehg==}
peerDependencies:
msw: ^2.4.9
vite: ^5.0.0
@@ -675,20 +679,20 @@ packages:
vite:
optional: true
- '@vitest/pretty-format@2.1.8':
- resolution: {integrity: sha512-9HiSZ9zpqNLKlbIDRWOnAWqgcA7xu+8YxXSekhr0Ykab7PAYFkhkwoqVArPOtJhPmYeE2YHgKZlj3CP36z2AJQ==}
+ '@vitest/pretty-format@2.1.9':
+ resolution: {integrity: sha512-KhRIdGV2U9HOUzxfiHmY8IFHTdqtOhIzCpd8WRdJiE7D/HUcZVD0EgQCVjm+Q9gkUXWgBvMmTtZgIG48wq7sOQ==}
- '@vitest/runner@2.1.8':
- resolution: {integrity: sha512-17ub8vQstRnRlIU5k50bG+QOMLHRhYPAna5tw8tYbj+jzjcspnwnwtPtiOlkuKC4+ixDPTuLZiqiWWQ2PSXHVg==}
+ '@vitest/runner@2.1.9':
+ resolution: {integrity: sha512-ZXSSqTFIrzduD63btIfEyOmNcBmQvgOVsPNPe0jYtESiXkhd8u2erDLnMxmGrDCwHCCHE7hxwRDCT3pt0esT4g==}
- '@vitest/snapshot@2.1.8':
- resolution: {integrity: sha512-20T7xRFbmnkfcmgVEz+z3AU/3b0cEzZOt/zmnvZEctg64/QZbSDJEVm9fLnnlSi74KibmRsO9/Qabi+t0vCRPg==}
+ '@vitest/snapshot@2.1.9':
+ resolution: {integrity: sha512-oBO82rEjsxLNJincVhLhaxxZdEtV0EFHMK5Kmx5sJ6H9L183dHECjiefOAdnqpIgT5eZwT04PoggUnW88vOBNQ==}
- '@vitest/spy@2.1.8':
- resolution: {integrity: sha512-5swjf2q95gXeYPevtW0BLk6H8+bPlMb4Vw/9Em4hFxDcaOxS+e0LOX4yqNxoHzMR2akEB2xfpnWUzkZokmgWDg==}
+ '@vitest/spy@2.1.9':
+ resolution: {integrity: sha512-E1B35FwzXXTs9FHNK6bDszs7mtydNi5MIfUWpceJ8Xbfb1gBMscAnwLbEu+B44ed6W3XjL9/ehLPHR1fkf1KLQ==}
- '@vitest/utils@2.1.8':
- resolution: {integrity: sha512-dwSoui6djdwbfFmIgbIjX2ZhIoG7Ex/+xpxyiEgIGzjliY8xGkcpITKTlp6B4MgtGkF2ilvm97cPM96XZaAgcA==}
+ '@vitest/utils@2.1.9':
+ resolution: {integrity: sha512-v0psaMSkNJ3A2NMrUEHFRzJtDPFn+/VWZ5WxImB21T9fjucJRmS7xCS3ppEnARb9y11OAzaD+P2Ps+b+BGX5iQ==}
'@zag-js/accordion@0.81.1':
resolution: {integrity: sha512-NMSx9DNz+FigY9E+FtT/3GCjpP4H0VTbBTmqUDxw3FYKgP3txPoIQGrV4Dig4hCtCiPdmlwSZatA29HrTi8+zw==}
@@ -1198,8 +1202,8 @@ packages:
fast-levenshtein@2.0.6:
resolution: {integrity: sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw==}
- fastq@1.18.0:
- resolution: {integrity: sha512-QKHXPW0hD8g4UET03SdOdunzSouc9N4AuHdsX8XNcTsuz+yYFILVNIX4l9yHABMhiEI9Db0JTTIpu0wB+Y1QQw==}
+ fastq@1.19.0:
+ resolution: {integrity: sha512-7SFSRCNjBQIZH/xZR3iy5iQYR8aGBE0h3VG6/cwlbrpdciNYBMotQav8c1XI3HjHH+NikUpP53nPdlZSdWmFzA==}
file-entry-cache@8.0.0:
resolution: {integrity: sha512-XXTUwCvisa5oacNGRP9SfNtYBNAMi+RPwBFmblZEF7N7swHYQS6/Zfk7SRwx4D5j3CH211YNRco1DEMNVfZCnQ==}
@@ -1312,6 +1316,10 @@ packages:
resolution: {integrity: sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw==}
engines: {node: '>=6'}
+ import-fresh@3.3.1:
+ resolution: {integrity: sha512-TR3KfrTZTYLPB6jUjfx6MF9WcWrHL9su5TObK4ZkYgBdWKPOFoSoQIdEuTuR82pmtxH2spWG9h6etwfr1pLBqQ==}
+ engines: {node: '>=6'}
+
imurmurhash@0.1.4:
resolution: {integrity: sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA==}
engines: {node: '>=0.8.19'}
@@ -1409,8 +1417,8 @@ packages:
resolution: {integrity: sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==}
hasBin: true
- loupe@3.1.2:
- resolution: {integrity: sha512-23I4pFZHmAemUnz8WZXbYRSKYj801VDaNv9ETuMh7IrMc7VuVVSo+Z9iLE3ni30+U48iDWfi30d3twAXBYmnCg==}
+ loupe@3.1.3:
+ resolution: {integrity: sha512-kkIp7XSkP78ZxJEsSxW3712C6teJVoeHHwgo9zJ380de7IYyJ2ISlxojcH2pC5OFLewESmnRi/+XCDIEEVyoug==}
lru-cache@10.4.3:
resolution: {integrity: sha512-JNAzZcXrCt42VGLuYz0zfAzDfAvJWW6AfYlDBQyDV5DClI2m5sAmK+OIO7s59XfsRsWHp02jAJrRadPRGTt6SQ==}
@@ -1580,8 +1588,8 @@ packages:
pathe@1.1.2:
resolution: {integrity: sha512-whLdWMYL2TwI08hn8/ZqAbrVemu0LNaNNJZX73O6qaIdCTfXutsLhMkjdENX0qhsQ9uIimo4/aQOmXkoon2nDQ==}
- pathe@2.0.1:
- resolution: {integrity: sha512-6jpjMpOth5S9ITVu5clZ7NOgHNsv5vRQdheL9ztp2vZmM6fRbLvyua1tiBIL4lk8SAe3ARzeXEly6siXCjDHDw==}
+ pathe@2.0.2:
+ resolution: {integrity: sha512-15Ztpk+nov8DR524R4BF7uEuzESgzUEAV4Ah7CUMNGXdE5ELuvxElxGXndBl32vMSsWa1jpNf22Z+Er3sKwq+w==}
pathval@2.0.0:
resolution: {integrity: sha512-vE7JKRyES09KiunauX7nd2Q9/L7lhok4smP9RZTDeD4MVs72Dp2qNFVz39Nz5a0FVEW0BJR6C0DYrq6unoziZA==}
@@ -1830,8 +1838,8 @@ packages:
uri-js@4.4.1:
resolution: {integrity: sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg==}
- vite-node@2.1.8:
- resolution: {integrity: sha512-uPAwSr57kYjAUux+8E2j0q0Fxpn8M9VoyfGiRI8Kfktz9NcYMCenwY5RnZxnF1WTu3TGiYipirIzacLL3VVGFg==}
+ vite-node@2.1.9:
+ resolution: {integrity: sha512-AM9aQ/IPrW/6ENLQg3AGY4K1N2TGZdR5e4gu/MmmR2xR3Ll1+dib+nook92g4TV3PXVyeyxdWwtaCAiUL0hMxA==}
engines: {node: ^18.0.0 || >=20.0.0}
hasBin: true
@@ -1871,15 +1879,15 @@ packages:
terser:
optional: true
- vitest@2.1.8:
- resolution: {integrity: sha512-1vBKTZskHw/aosXqQUlVWWlGUxSJR8YtiyZDJAFeW2kPAeX6S3Sool0mjspO+kXLuxVWlEDDowBAeqeAQefqLQ==}
+ vitest@2.1.9:
+ resolution: {integrity: sha512-MSmPM9REYqDGBI8439mA4mWhV5sKmDlBKWIYbA3lRb2PTHACE0mgKwA8yQ2xq9vxDTuk4iPrECBAEW2aoFXY0Q==}
engines: {node: ^18.0.0 || >=20.0.0}
hasBin: true
peerDependencies:
'@edge-runtime/vm': '*'
'@types/node': ^18.0.0 || >=20.0.0
- '@vitest/browser': 2.1.8
- '@vitest/ui': 2.1.8
+ '@vitest/browser': 2.1.9
+ '@vitest/ui': 2.1.9
happy-dom: '*'
jsdom: '*'
peerDependenciesMeta:
@@ -2025,16 +2033,16 @@ snapshots:
'@babel/generator@7.26.5':
dependencies:
- '@babel/parser': 7.26.5
- '@babel/types': 7.26.5
+ '@babel/parser': 7.26.7
+ '@babel/types': 7.26.7
'@jridgewell/gen-mapping': 0.3.8
'@jridgewell/trace-mapping': 0.3.25
jsesc: 3.1.0
'@babel/helper-module-imports@7.25.9':
dependencies:
- '@babel/traverse': 7.26.5
- '@babel/types': 7.26.5
+ '@babel/traverse': 7.26.7
+ '@babel/types': 7.26.7
transitivePeerDependencies:
- supports-color
@@ -2042,33 +2050,37 @@ snapshots:
'@babel/helper-validator-identifier@7.25.9': {}
- '@babel/parser@7.26.5':
+ '@babel/parser@7.26.7':
dependencies:
- '@babel/types': 7.26.5
+ '@babel/types': 7.26.7
'@babel/runtime@7.26.0':
dependencies:
regenerator-runtime: 0.14.1
+ '@babel/runtime@7.26.7':
+ dependencies:
+ regenerator-runtime: 0.14.1
+
'@babel/template@7.25.9':
dependencies:
'@babel/code-frame': 7.26.2
- '@babel/parser': 7.26.5
- '@babel/types': 7.26.5
+ '@babel/parser': 7.26.7
+ '@babel/types': 7.26.7
- '@babel/traverse@7.26.5':
+ '@babel/traverse@7.26.7':
dependencies:
'@babel/code-frame': 7.26.2
'@babel/generator': 7.26.5
- '@babel/parser': 7.26.5
+ '@babel/parser': 7.26.7
'@babel/template': 7.25.9
- '@babel/types': 7.26.5
+ '@babel/types': 7.26.7
debug: 4.4.0
globals: 11.12.0
transitivePeerDependencies:
- supports-color
- '@babel/types@7.26.5':
+ '@babel/types@7.26.7':
dependencies:
'@babel/helper-string-parser': 7.25.9
'@babel/helper-validator-identifier': 7.25.9
@@ -2089,7 +2101,7 @@ snapshots:
'@emotion/babel-plugin@11.13.5':
dependencies:
'@babel/helper-module-imports': 7.25.9
- '@babel/runtime': 7.26.0
+ '@babel/runtime': 7.26.7
'@emotion/hash': 0.9.2
'@emotion/memoize': 0.9.0
'@emotion/serialize': 1.3.3
@@ -2120,7 +2132,7 @@ snapshots:
'@emotion/react@11.14.0(react@18.3.1)':
dependencies:
- '@babel/runtime': 7.26.0
+ '@babel/runtime': 7.26.7
'@emotion/babel-plugin': 11.13.5
'@emotion/cache': 11.14.0
'@emotion/serialize': 1.3.3
@@ -2344,7 +2356,7 @@ snapshots:
'@nodelib/fs.walk@1.2.8':
dependencies:
'@nodelib/fs.scandir': 2.1.5
- fastq: 1.18.0
+ fastq: 1.19.0
'@pandacss/is-valid-prop@0.41.0': {}
@@ -2477,7 +2489,7 @@ snapshots:
'@testing-library/dom@10.4.0':
dependencies:
'@babel/code-frame': 7.26.2
- '@babel/runtime': 7.26.0
+ '@babel/runtime': 7.26.7
'@types/aria-query': 5.0.4
aria-query: 5.3.0
chalk: 4.1.2
@@ -2524,44 +2536,44 @@ snapshots:
transitivePeerDependencies:
- '@swc/helpers'
- '@vitest/expect@2.1.8':
+ '@vitest/expect@2.1.9':
dependencies:
- '@vitest/spy': 2.1.8
- '@vitest/utils': 2.1.8
+ '@vitest/spy': 2.1.9
+ '@vitest/utils': 2.1.9
chai: 5.1.2
tinyrainbow: 1.2.0
- '@vitest/mocker@2.1.8(vite@5.4.14)':
+ '@vitest/mocker@2.1.9(vite@5.4.14)':
dependencies:
- '@vitest/spy': 2.1.8
+ '@vitest/spy': 2.1.9
estree-walker: 3.0.3
magic-string: 0.30.17
optionalDependencies:
vite: 5.4.14
- '@vitest/pretty-format@2.1.8':
+ '@vitest/pretty-format@2.1.9':
dependencies:
tinyrainbow: 1.2.0
- '@vitest/runner@2.1.8':
+ '@vitest/runner@2.1.9':
dependencies:
- '@vitest/utils': 2.1.8
+ '@vitest/utils': 2.1.9
pathe: 1.1.2
- '@vitest/snapshot@2.1.8':
+ '@vitest/snapshot@2.1.9':
dependencies:
- '@vitest/pretty-format': 2.1.8
+ '@vitest/pretty-format': 2.1.9
magic-string: 0.30.17
pathe: 1.1.2
- '@vitest/spy@2.1.8':
+ '@vitest/spy@2.1.9':
dependencies:
tinyspy: 3.0.2
- '@vitest/utils@2.1.8':
+ '@vitest/utils@2.1.9':
dependencies:
- '@vitest/pretty-format': 2.1.8
- loupe: 3.1.2
+ '@vitest/pretty-format': 2.1.9
+ loupe: 3.1.3
tinyrainbow: 1.2.0
'@zag-js/accordion@0.81.1':
@@ -3087,7 +3099,7 @@ snapshots:
babel-plugin-macros@3.1.0:
dependencies:
- '@babel/runtime': 7.26.0
+ '@babel/runtime': 7.26.7
cosmiconfig: 7.1.0
resolve: 1.22.10
@@ -3136,7 +3148,7 @@ snapshots:
assertion-error: 2.0.1
check-error: 2.1.1
deep-eql: 5.0.2
- loupe: 3.1.2
+ loupe: 3.1.3
pathval: 2.0.0
chalk@3.0.0:
@@ -3194,7 +3206,7 @@ snapshots:
cosmiconfig@7.1.0:
dependencies:
'@types/parse-json': 4.0.2
- import-fresh: 3.3.0
+ import-fresh: 3.3.1
parse-json: 5.2.0
path-type: 4.0.0
yaml: 1.10.2
@@ -3373,7 +3385,7 @@ snapshots:
fast-levenshtein@2.0.6: {}
- fastq@1.18.0:
+ fastq@1.19.0:
dependencies:
reusify: 1.0.4
@@ -3489,6 +3501,11 @@ snapshots:
parent-module: 1.0.1
resolve-from: 4.0.0
+ import-fresh@3.3.1:
+ dependencies:
+ parent-module: 1.0.1
+ resolve-from: 4.0.0
+
imurmurhash@0.1.4: {}
indent-string@4.0.0: {}
@@ -3564,7 +3581,7 @@ snapshots:
dependencies:
js-tokens: 4.0.0
- loupe@3.1.2: {}
+ loupe@3.1.3: {}
lru-cache@10.4.3: {}
@@ -3576,8 +3593,8 @@ snapshots:
magicast@0.3.5:
dependencies:
- '@babel/parser': 7.26.5
- '@babel/types': 7.26.5
+ '@babel/parser': 7.26.7
+ '@babel/types': 7.26.7
source-map-js: 1.2.1
optional: true
@@ -3630,7 +3647,7 @@ snapshots:
mlly@1.7.4:
dependencies:
acorn: 8.14.0
- pathe: 2.0.1
+ pathe: 2.0.2
pkg-types: 1.3.1
ufo: 1.5.4
@@ -3714,7 +3731,7 @@ snapshots:
pathe@1.1.2: {}
- pathe@2.0.1: {}
+ pathe@2.0.2: {}
pathval@2.0.0: {}
@@ -3730,7 +3747,7 @@ snapshots:
dependencies:
confbox: 0.1.8
mlly: 1.7.4
- pathe: 2.0.1
+ pathe: 2.0.2
postcss@8.5.1:
dependencies:
@@ -3950,7 +3967,7 @@ snapshots:
dependencies:
punycode: 2.3.1
- vite-node@2.1.8:
+ vite-node@2.1.9:
dependencies:
cac: 6.7.14
debug: 4.4.0
@@ -3980,15 +3997,15 @@ snapshots:
optionalDependencies:
fsevents: 2.3.3
- vitest@2.1.8(happy-dom@15.11.7):
+ vitest@2.1.9(happy-dom@15.11.7):
dependencies:
- '@vitest/expect': 2.1.8
- '@vitest/mocker': 2.1.8(vite@5.4.14)
- '@vitest/pretty-format': 2.1.8
- '@vitest/runner': 2.1.8
- '@vitest/snapshot': 2.1.8
- '@vitest/spy': 2.1.8
- '@vitest/utils': 2.1.8
+ '@vitest/expect': 2.1.9
+ '@vitest/mocker': 2.1.9(vite@5.4.14)
+ '@vitest/pretty-format': 2.1.9
+ '@vitest/runner': 2.1.9
+ '@vitest/snapshot': 2.1.9
+ '@vitest/spy': 2.1.9
+ '@vitest/utils': 2.1.9
chai: 5.1.2
debug: 4.4.0
expect-type: 1.1.0
@@ -4000,7 +4017,7 @@ snapshots:
tinypool: 1.0.2
tinyrainbow: 1.2.0
vite: 5.4.14
- vite-node: 2.1.8
+ vite-node: 2.1.9
why-is-node-running: 2.3.0
optionalDependencies:
happy-dom: 15.11.7
diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py
index 0186add2bb3bb..78cfbf394b839 100644
--- a/airflow/cli/cli_config.py
+++ b/airflow/cli/cli_config.py
@@ -1073,7 +1073,7 @@ class GroupCommand(NamedTuple):
name="state",
help="Get the status of a dag run",
func=lazy_load_command("airflow.cli.commands.remote_commands.dag_command.dag_state"),
- args=(ARG_DAG_ID, ARG_LOGICAL_DATE, ARG_SUBDIR, ARG_VERBOSE),
+ args=(ARG_DAG_ID, ARG_LOGICAL_DATE_OR_RUN_ID, ARG_SUBDIR, ARG_VERBOSE),
),
ActionCommand(
name="next-execution",
diff --git a/airflow/cli/commands/remote_commands/config_command.py b/airflow/cli/commands/remote_commands/config_command.py
index dee1e89df5682..5a52edbaaa3cc 100644
--- a/airflow/cli/commands/remote_commands/config_command.py
+++ b/airflow/cli/commands/remote_commands/config_command.py
@@ -84,11 +84,13 @@ class ConfigChange:
:param config: The configuration parameter being changed.
:param suggestion: A suggestion for replacing or handling the removed configuration.
:param renamed_to: The new section and option if the configuration is renamed.
+ :param was_deprecated: If the config is removed, whether the old config was deprecated.
"""
config: ConfigParameter
suggestion: str = ""
renamed_to: ConfigParameter | None = None
+ was_deprecated: bool = True
@property
def message(self) -> str:
@@ -96,15 +98,16 @@ def message(self) -> str:
if self.renamed_to:
if self.config.section != self.renamed_to.section:
return (
- f"`{self.config.option}` configuration parameter moved from `{self.config.section}` section to `"
- f"{self.renamed_to.section}` section as `{self.renamed_to.option}`."
+ f"`{self.config.option}` configuration parameter moved from `{self.config.section}` section to "
+ f"`{self.renamed_to.section}` section as `{self.renamed_to.option}`."
)
return (
f"`{self.config.option}` configuration parameter renamed to `{self.renamed_to.option}` "
f"in the `{self.config.section}` section."
)
return (
- f"Removed deprecated `{self.config.option}` configuration parameter from `{self.config.section}` section. "
+ f"Removed{' deprecated' if self.was_deprecated else ''} `{self.config.option}` configuration parameter "
+ f"from `{self.config.section}` section. "
f"{self.suggestion}"
)
@@ -203,6 +206,12 @@ def message(self) -> str:
config=ConfigParameter("core", "dag_file_processor_timeout"),
renamed_to=ConfigParameter("dag_processor", "dag_file_processor_timeout"),
),
+ ConfigChange(
+ config=ConfigParameter("core", "dag_processor_manager_log_location"),
+ ),
+ ConfigChange(
+ config=ConfigParameter("core", "log_processor_filename_template"),
+ ),
# api
ConfigChange(
config=ConfigParameter("api", "access_control_allow_origin"),
@@ -218,6 +227,18 @@ def message(self) -> str:
suggestion="Remove TaskContextLogger: Replaced by the Log table for better handling of task log "
"messages outside the execution context.",
),
+ ConfigChange(
+ config=ConfigParameter("logging", "dag_processor_manager_log_location"),
+ was_deprecated=False,
+ ),
+ ConfigChange(
+ config=ConfigParameter("logging", "dag_processor_manager_log_stdout"),
+ was_deprecated=False,
+ ),
+ ConfigChange(
+ config=ConfigParameter("logging", "log_processor_filename_template"),
+ was_deprecated=False,
+ ),
# metrics
ConfigChange(
config=ConfigParameter("metrics", "metrics_use_pattern_match"),
diff --git a/airflow/cli/commands/remote_commands/dag_command.py b/airflow/cli/commands/remote_commands/dag_command.py
index 6c95451271fbc..bf0841f1250be 100644
--- a/airflow/cli/commands/remote_commands/dag_command.py
+++ b/airflow/cli/commands/remote_commands/dag_command.py
@@ -33,6 +33,7 @@
from airflow.api.client import get_current_api_client
from airflow.api_connexion.schemas.dag_schema import dag_schema
from airflow.cli.simple_table import AirflowConsole
+from airflow.cli.utils import fetch_dag_run_from_run_id_or_logical_date_string
from airflow.exceptions import AirflowException
from airflow.jobs.job import Job
from airflow.models import DagBag, DagModel, DagRun, TaskInstance
@@ -264,12 +265,17 @@ def dag_state(args, session: Session = NEW_SESSION) -> None:
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
- dr = session.scalar(select(DagRun).filter_by(dag_id=args.dag_id, logical_date=args.logical_date))
- out = dr.state if dr else None
- conf_out = ""
- if out and dr.conf:
- conf_out = ", " + json.dumps(dr.conf)
- print(str(out) + conf_out)
+ dr, _ = fetch_dag_run_from_run_id_or_logical_date_string(
+ dag_id=dag.dag_id,
+ value=args.logical_date_or_run_id,
+ session=session,
+ )
+ if not dr:
+ print(None)
+ elif dr.conf:
+ print(f"{dr.state}, {json.dumps(dr.conf)}")
+ else:
+ print(dr.state)
@cli_utils.action_cli
@@ -465,20 +471,20 @@ def dag_list_dag_runs(args, dag: DAG | None = None, session: Session = NEW_SESSI
logical_end_date=args.end_date,
session=session,
)
+ dag_runs.sort(key=operator.attrgetter("run_after"), reverse=True)
- dag_runs.sort(key=lambda x: x.logical_date, reverse=True)
- AirflowConsole().print_as(
- data=dag_runs,
- output=args.output,
- mapper=lambda dr: {
+ def _render_dagrun(dr: DagRun) -> dict[str, str]:
+ return {
"dag_id": dr.dag_id,
"run_id": dr.run_id,
"state": dr.state,
- "logical_date": dr.logical_date.isoformat(),
+ "run_after": dr.run_after.isoformat(),
+ "logical_date": dr.logical_date.isoformat() if dr.logical_date else "",
"start_date": dr.start_date.isoformat() if dr.start_date else "",
"end_date": dr.end_date.isoformat() if dr.end_date else "",
- },
- )
+ }
+
+ AirflowConsole().print_as(data=dag_runs, output=args.output, mapper=_render_dagrun)
@cli_utils.action_cli
@@ -515,7 +521,7 @@ def dag_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> No
tis = session.scalars(
select(TaskInstance).where(
TaskInstance.dag_id == args.dag_id,
- TaskInstance.logical_date == logical_date,
+ TaskInstance.run_id == dr.run_id,
)
).all()
diff --git a/airflow/cli/commands/remote_commands/task_command.py b/airflow/cli/commands/remote_commands/task_command.py
index ccae29ededd09..debe1b814397b 100644
--- a/airflow/cli/commands/remote_commands/task_command.py
+++ b/airflow/cli/commands/remote_commands/task_command.py
@@ -32,11 +32,10 @@
from typing import TYPE_CHECKING, Protocol, cast
import pendulum
-from pendulum.parsing.exceptions import ParserError
-from sqlalchemy import select
from airflow import settings
from airflow.cli.simple_table import AirflowConsole
+from airflow.cli.utils import fetch_dag_run_from_run_id_or_logical_date_string
from airflow.configuration import conf
from airflow.exceptions import AirflowException, DagRunNotFound, TaskDeferred, TaskInstanceNotFound
from airflow.executors.executor_loader import ExecutorLoader
@@ -48,6 +47,7 @@
from airflow.models.dagrun import DagRun
from airflow.models.taskinstance import TaskReturnCode
from airflow.sdk.definitions.param import ParamsDict
+from airflow.sdk.execution_time.secrets_masker import RedactedIO
from airflow.settings import IS_EXECUTOR_CONTAINER, IS_K8S_EXECUTOR_POD
from airflow.ti_deps.dep_context import DepContext
from airflow.ti_deps.dependencies_deps import SCHEDULER_QUEUED_DEPS
@@ -61,7 +61,6 @@
)
from airflow.utils.log.file_task_handler import _set_task_deferred_context_var
from airflow.utils.log.logging_mixin import StreamLogWriter
-from airflow.utils.log.secrets_masker import RedactedIO
from airflow.utils.net import get_hostname
from airflow.utils.providers_configuration_loader import providers_configuration_loaded
from airflow.utils.session import NEW_SESSION, create_session, provide_session
@@ -91,41 +90,6 @@ def _generate_temporary_run_id() -> str:
return f"__airflow_temporary_run_{timezone.utcnow().isoformat()}__"
-def _fetch_dag_run_from_run_id_or_logical_date_string(
- *,
- dag_id: str,
- value: str,
- session: Session,
-) -> tuple[DagRun, pendulum.DateTime | None]:
- """
- Try to find a DAG run with a given string value.
-
- The string value may be a run ID, or a logical date in string form. We first
- try to use it as a run_id; if a run is found, it is returned as-is.
-
- Otherwise, the string value is parsed into a datetime. If that works, it is
- used to find a DAG run.
-
- The return value is a two-tuple. The first item is the found DAG run (or
- *None* if one cannot be found). The second is the parsed logical date. This
- second value can be used to create a new run by the calling function when
- one cannot be found here.
- """
- if dag_run := DAG.fetch_dagrun(dag_id=dag_id, run_id=value, session=session):
- return dag_run, dag_run.logical_date # type: ignore[return-value]
- try:
- logical_date = timezone.parse(value)
- except (ParserError, TypeError):
- return dag_run, None
- dag_run = session.scalar(
- select(DagRun)
- .where(DagRun.dag_id == dag_id, DagRun.logical_date == logical_date)
- .order_by(DagRun.id.desc())
- .limit(1)
- )
- return dag_run, logical_date
-
-
def _get_dag_run(
*,
dag: DAG,
@@ -152,7 +116,7 @@ def _get_dag_run(
logical_date = None
if logical_date_or_run_id:
- dag_run, logical_date = _fetch_dag_run_from_run_id_or_logical_date_string(
+ dag_run, logical_date = fetch_dag_run_from_run_id_or_logical_date_string(
dag_id=dag.dag_id,
value=logical_date_or_run_id,
session=session,
@@ -165,11 +129,7 @@ def _get_dag_run(
f"of {logical_date_or_run_id!r} not found"
)
- if logical_date is not None:
- dag_run_logical_date = logical_date
- else:
- dag_run_logical_date = pendulum.instance(timezone.utcnow())
-
+ dag_run_logical_date = pendulum.instance(logical_date or timezone.utcnow())
if create_if_necessary == "memory":
data_interval = dag.timetable.infer_manual_data_interval(run_after=dag_run_logical_date)
dag_run = DagRun(
@@ -592,17 +552,11 @@ def _guess_debugger() -> _SupportedDebugger:
@provide_session
def task_states_for_dag_run(args, session: Session = NEW_SESSION) -> None:
"""Get the status of all task instances in a DagRun."""
- dag_run = session.scalar(
- select(DagRun).where(DagRun.run_id == args.logical_date_or_run_id, DagRun.dag_id == args.dag_id)
+ dag_run, _ = fetch_dag_run_from_run_id_or_logical_date_string(
+ dag_id=args.dag_id,
+ value=args.logical_date_or_run_id,
+ session=session,
)
- if not dag_run:
- try:
- logical_date = timezone.parse(args.logical_date_or_run_id)
- dag_run = session.scalar(
- select(DagRun).where(DagRun.logical_date == logical_date, DagRun.dag_id == args.dag_id)
- )
- except (ParserError, TypeError) as err:
- raise AirflowException(f"Error parsing the supplied logical_date. Error: {err}")
if dag_run is None:
raise DagRunNotFound(
@@ -615,7 +569,7 @@ def task_states_for_dag_run(args, session: Session = NEW_SESSION) -> None:
def format_task_instance(ti: TaskInstance) -> dict[str, str]:
data = {
"dag_id": ti.dag_id,
- "logical_date": dag_run.logical_date.isoformat(),
+ "logical_date": dag_run.logical_date.isoformat() if dag_run.logical_date else "",
"task_id": ti.task_id,
"state": ti.state,
"start_date": ti.start_date.isoformat() if ti.start_date else "",
diff --git a/airflow/cli/utils.py b/airflow/cli/utils.py
index d132deeb373a9..605244ee71e56 100644
--- a/airflow/cli/utils.py
+++ b/airflow/cli/utils.py
@@ -17,13 +17,17 @@
from __future__ import annotations
-import io
import sys
-from collections.abc import Collection
from typing import TYPE_CHECKING
if TYPE_CHECKING:
- from io import IOBase
+ import datetime
+ from collections.abc import Collection
+ from io import IOBase, TextIOWrapper
+
+ from sqlalchemy.orm import Session
+
+ from airflow.models.dagrun import DagRun
class CliConflictError(Exception):
@@ -45,8 +49,50 @@ def is_stdout(fileio: IOBase) -> bool:
return fileio.fileno() == sys.stdout.fileno()
-def print_export_output(command_type: str, exported_items: Collection, file: io.TextIOWrapper):
+def print_export_output(command_type: str, exported_items: Collection, file: TextIOWrapper):
if not file.closed and is_stdout(file):
print(f"\n{len(exported_items)} {command_type} successfully exported.", file=sys.stderr)
else:
print(f"{len(exported_items)} {command_type} successfully exported to {file.name}.")
+
+
+def fetch_dag_run_from_run_id_or_logical_date_string(
+ *,
+ dag_id: str,
+ value: str,
+ session: Session,
+) -> tuple[DagRun | None, datetime.datetime | None]:
+ """
+ Try to find a DAG run with a given string value.
+
+ The string value may be a run ID, or a logical date in string form. We first
+ try to use it as a run_id; if a run is found, it is returned as-is.
+
+ Otherwise, the string value is parsed into a datetime. If that works, it is
+ used to find a DAG run.
+
+ The return value is a two-tuple. The first item is the found DAG run (or
+ *None* if one cannot be found). The second is the parsed logical date. This
+ second value can be used to create a new run by the calling function when
+ one cannot be found here.
+ """
+ from pendulum.parsing.exceptions import ParserError
+ from sqlalchemy import select
+
+ from airflow.models.dag import DAG
+ from airflow.models.dagrun import DagRun
+ from airflow.utils import timezone
+
+ if dag_run := DAG.fetch_dagrun(dag_id=dag_id, run_id=value, session=session):
+ return dag_run, dag_run.logical_date
+ try:
+ logical_date = timezone.parse(value)
+ except (ParserError, TypeError):
+ return None, None
+ dag_run = session.scalar(
+ select(DagRun)
+ .where(DagRun.dag_id == dag_id, DagRun.logical_date == logical_date)
+ .order_by(DagRun.id.desc())
+ .limit(1)
+ )
+ return dag_run, logical_date
diff --git a/airflow/config_templates/airflow_local_settings.py b/airflow/config_templates/airflow_local_settings.py
index f440261dafc86..5ac7f513d1f87 100644
--- a/airflow/config_templates/airflow_local_settings.py
+++ b/airflow/config_templates/airflow_local_settings.py
@@ -20,7 +20,6 @@
from __future__ import annotations
import os
-from pathlib import Path
from typing import Any
from urllib.parse import urlsplit
@@ -53,17 +52,6 @@
PROCESSOR_LOG_FOLDER: str = conf.get_mandatory_value("scheduler", "CHILD_PROCESS_LOG_DIRECTORY")
-DAG_PROCESSOR_MANAGER_LOG_LOCATION: str = conf.get_mandatory_value(
- "logging", "DAG_PROCESSOR_MANAGER_LOG_LOCATION"
-)
-
-DAG_PROCESSOR_MANAGER_LOG_STDOUT: str = conf.get_mandatory_value(
- "logging", "DAG_PROCESSOR_MANAGER_LOG_STDOUT"
-)
-
-
-PROCESSOR_FILENAME_TEMPLATE: str = conf.get_mandatory_value("logging", "LOG_PROCESSOR_FILENAME_TEMPLATE")
-
DEFAULT_LOGGING_CONFIG: dict[str, Any] = {
"version": 1,
"disable_existing_loggers": False,
@@ -83,7 +71,7 @@
},
"filters": {
"mask_secrets": {
- "()": "airflow.utils.log.secrets_masker.SecretsMasker",
+ "()": "airflow.sdk.execution_time.secrets_masker.SecretsMasker",
},
},
"handlers": {
@@ -99,27 +87,8 @@
"base_log_folder": os.path.expanduser(BASE_LOG_FOLDER),
"filters": ["mask_secrets"],
},
- "processor": {
- "class": "airflow.utils.log.file_processor_handler.FileProcessorHandler",
- "formatter": "airflow",
- "base_log_folder": os.path.expanduser(PROCESSOR_LOG_FOLDER),
- "filename_template": PROCESSOR_FILENAME_TEMPLATE,
- "filters": ["mask_secrets"],
- },
- "processor_to_stdout": {
- "class": "airflow.utils.log.logging_mixin.RedirectStdHandler",
- "formatter": "source_processor",
- "stream": "sys.stdout",
- "filters": ["mask_secrets"],
- },
},
"loggers": {
- "airflow.processor": {
- "handlers": ["processor_to_stdout" if DAG_PROCESSOR_LOG_TARGET == "stdout" else "processor"],
- "level": LOG_LEVEL,
- # Set to true here (and reset via set_context) so that if no file is configured we still get logs!
- "propagate": True,
- },
"airflow.task": {
"handlers": ["task"],
"level": LOG_LEVEL,
@@ -152,54 +121,6 @@
}
DEFAULT_LOGGING_CONFIG["loggers"].update(new_loggers)
-DEFAULT_DAG_PARSING_LOGGING_CONFIG: dict[str, dict[str, dict[str, Any]]] = {
- "handlers": {
- "processor_manager": {
- "class": "airflow.utils.log.non_caching_file_handler.NonCachingRotatingFileHandler",
- "formatter": "airflow",
- "filename": DAG_PROCESSOR_MANAGER_LOG_LOCATION,
- "mode": "a",
- "maxBytes": 104857600, # 100MB
- "backupCount": 5,
- }
- },
- "loggers": {
- "airflow.processor_manager": {
- "handlers": ["processor_manager"],
- "level": LOG_LEVEL,
- "propagate": False,
- }
- },
-}
-
-if DAG_PROCESSOR_MANAGER_LOG_STDOUT == "True":
- DEFAULT_DAG_PARSING_LOGGING_CONFIG["handlers"].update(
- {
- "console": {
- "class": "airflow.utils.log.logging_mixin.RedirectStdHandler",
- "formatter": "airflow",
- "stream": "sys.stdout",
- "filters": ["mask_secrets"],
- }
- }
- )
- DEFAULT_DAG_PARSING_LOGGING_CONFIG["loggers"]["airflow.processor_manager"]["handlers"].append("console")
-
-# Only update the handlers and loggers when CONFIG_PROCESSOR_MANAGER_LOGGER is set.
-# This is to avoid exceptions when initializing RotatingFileHandler multiple times
-# in multiple processes.
-if os.environ.get("CONFIG_PROCESSOR_MANAGER_LOGGER") == "True":
- DEFAULT_LOGGING_CONFIG["handlers"].update(DEFAULT_DAG_PARSING_LOGGING_CONFIG["handlers"])
- DEFAULT_LOGGING_CONFIG["loggers"].update(DEFAULT_DAG_PARSING_LOGGING_CONFIG["loggers"])
-
- # Manually create log directory for processor_manager handler as RotatingFileHandler
- # will only create file but not the directory.
- processor_manager_handler_config: dict[str, Any] = DEFAULT_DAG_PARSING_LOGGING_CONFIG["handlers"][
- "processor_manager"
- ]
- directory: str = os.path.dirname(processor_manager_handler_config["filename"])
- Path(directory).mkdir(parents=True, exist_ok=True, mode=0o755)
-
##################
# Remote logging #
##################
diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index 8e8726bd07e02..201d4e3e6bc83 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -886,10 +886,10 @@ logging:
secret_mask_adapter:
description: |
An import path to a function to add adaptations of each secret added with
- ``airflow.utils.log.secrets_masker.mask_secret`` to be masked in log messages. The given function
- is expected to require a single parameter: the secret to be adapted. It may return a
- single adaptation of the secret or an iterable of adaptations to each be masked as secrets.
- The original secret will be masked as well as any adaptations returned.
+ ``airflow.sdk.execution_time.secrets_masker.mask_secret`` to be masked in log messages.
+ The given function is expected to require a single parameter: the secret to be adapted.
+ It may return a single adaptation of the secret or an iterable of adaptations to each be
+ masked as secrets. The original secret will be masked as well as any adaptations returned.
version_added: 2.6.0
type: string
default: ""
@@ -912,28 +912,6 @@ logging:
default: "dag_id={{ ti.dag_id }}/run_id={{ ti.run_id }}/task_id={{ ti.task_id }}/\
{%% if ti.map_index >= 0 %%}map_index={{ ti.map_index }}/{%% endif %%}\
attempt={{ try_number|default(ti.try_number) }}.log"
- log_processor_filename_template:
- description: |
- Formatting for how airflow generates file names for log
- version_added: 2.0.0
- type: string
- example: ~
- is_template: true
- default: "{{ filename }}.log"
- dag_processor_manager_log_location:
- description: |
- Full path of dag_processor_manager logfile.
- version_added: 2.0.0
- type: string
- example: ~
- default: "{AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log"
- dag_processor_manager_log_stdout:
- description: |
- Whether DAG processor manager will write logs to stdout
- version_added: 2.9.0
- type: boolean
- example: ~
- default: "False"
task_log_reader:
description: |
Name of handler to read task instance logs.
@@ -1595,9 +1573,9 @@ webserver:
default: "120"
worker_refresh_batch_size:
description: |
- Number of workers to refresh at a time. When set to 0, worker refresh is
- disabled. When nonzero, airflow periodically refreshes webserver workers by
- bringing up new ones and killing old ones.
+ Number of workers to refresh at a time through Gunicorn's built-in worker management.
+ When set to 0, worker refresh is disabled. When nonzero, airflow periodically refreshes
+ webserver workers by bringing up new ones and killing old ones.
version_added: ~
type: string
example: ~
@@ -2698,3 +2676,15 @@ dag_processor:
type: integer
example: ~
default: "30"
+fastapi:
+ description: Configuration for the Fastapi webserver.
+ options:
+ base_url:
+ description: |
+ The base url of the Fastapi endpoint. Airflow cannot guess what domain or CNAME you are using.
+ If the Airflow console (the front-end) and the Fastapi apis are on a different domain, this config
+ should contain the Fastapi apis endpoint.
+ version_added: ~
+ type: string
+ example: ~
+ default: "http://localhost:29091"
diff --git a/airflow/configuration.py b/airflow/configuration.py
index 5257da834035c..6ba3fe6006c66 100644
--- a/airflow/configuration.py
+++ b/airflow/configuration.py
@@ -782,7 +782,7 @@ def _create_future_warning(name: str, section: str, current_value: Any, new_valu
)
def mask_secrets(self):
- from airflow.utils.log.secrets_masker import mask_secret
+ from airflow.sdk.execution_time.secrets_masker import mask_secret
for section, key in self.sensitive_config_values:
try:
diff --git a/airflow/dag_processing/bundles/base.py b/airflow/dag_processing/bundles/base.py
index 81bf38d373677..3d86f398c73cb 100644
--- a/airflow/dag_processing/bundles/base.py
+++ b/airflow/dag_processing/bundles/base.py
@@ -17,8 +17,10 @@
from __future__ import annotations
+import fcntl
import tempfile
from abc import ABC, abstractmethod
+from contextlib import contextmanager
from pathlib import Path
from airflow.configuration import conf
@@ -46,6 +48,7 @@ class BaseDagBundle(ABC):
"""
supports_versioning: bool = False
+ _locked: bool = False
def __init__(
self,
@@ -67,6 +70,10 @@ def initialize(self) -> None:
and allows for deferring expensive operations until that point in time. This will
only be called when Airflow needs the bundle files on disk - some uses only need
to call the `view_url` method, which can run without initializing the bundle.
+
+ This method must ultimately be safe to call concurrently from different threads or processes.
+ If it isn't naturally safe, you'll need to make it so with some form of locking.
+ There is a `lock` context manager on this class available for this purpose.
"""
self.is_initialized = True
@@ -101,7 +108,13 @@ def get_current_version(self) -> str | None:
@abstractmethod
def refresh(self) -> None:
- """Retrieve the latest version of the files in the bundle."""
+ """
+ Retrieve the latest version of the files in the bundle.
+
+ This method must ultimately be safe to call concurrently from different threads or processes.
+ If it isn't naturally safe, you'll need to make it so with some form of locking.
+ There is a `lock` context manager on this class available for this purpose.
+ """
def view_url(self, version: str | None = None) -> str | None:
"""
@@ -112,3 +125,27 @@ def view_url(self, version: str | None = None) -> str | None:
:param version: Version to view
:return: URL to view the bundle
"""
+
+ @contextmanager
+ def lock(self):
+ """
+ Ensure only a single bundle can enter this context at a time, by taking an exclusive lock on a lockfile.
+
+ This is useful when a bundle needs to perform operations that are not safe to run concurrently.
+ """
+ if self._locked:
+ yield
+ return
+
+ lock_dir_path = self._dag_bundle_root_storage_path / "_locks"
+ lock_dir_path.mkdir(parents=True, exist_ok=True)
+ lock_file_path = lock_dir_path / f"{self.name}.lock"
+ with open(lock_file_path, "w") as lock_file:
+ # Exclusive lock - blocks until it is available
+ fcntl.flock(lock_file, fcntl.LOCK_EX)
+ try:
+ self._locked = True
+ yield
+ finally:
+ fcntl.flock(lock_file, fcntl.LOCK_UN)
+ self._locked = False
diff --git a/airflow/dag_processing/bundles/git.py b/airflow/dag_processing/bundles/git.py
index abebbb4a33820..ea6833e66bae1 100644
--- a/airflow/dag_processing/bundles/git.py
+++ b/airflow/dag_processing/bundles/git.py
@@ -17,8 +17,10 @@
from __future__ import annotations
+import contextlib
import json
import os
+import tempfile
from typing import TYPE_CHECKING, Any
from urllib.parse import urlparse
@@ -60,6 +62,7 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]:
"extra": json.dumps(
{
"key_file": "optional/path/to/keyfile",
+ "private_key": "optional inline private key",
}
)
},
@@ -70,15 +73,22 @@ def __init__(self, git_conn_id="git_default", *args, **kwargs):
connection = self.get_connection(git_conn_id)
self.repo_url = connection.host
self.auth_token = connection.password
+ self.private_key = connection.extra_dejson.get("private_key")
self.key_file = connection.extra_dejson.get("key_file")
- strict_host_key_checking = connection.extra_dejson.get("strict_host_key_checking", "no")
+ self.strict_host_key_checking = connection.extra_dejson.get("strict_host_key_checking", "no")
self.env: dict[str, str] = {}
- if self.key_file:
- self.env["GIT_SSH_COMMAND"] = (
- f"ssh -i {self.key_file} -o IdentitiesOnly=yes -o StrictHostKeyChecking={strict_host_key_checking}"
- )
+
+ if self.key_file and self.private_key:
+ raise AirflowException("Both 'key_file' and 'private_key' cannot be provided at the same time")
self._process_git_auth_url()
+ def _build_ssh_command(self, key_path: str) -> str:
+ return (
+ f"ssh -i {key_path} "
+ f"-o IdentitiesOnly=yes "
+ f"-o StrictHostKeyChecking={self.strict_host_key_checking}"
+ )
+
def _process_git_auth_url(self):
if not isinstance(self.repo_url, str):
return
@@ -87,6 +97,22 @@ def _process_git_auth_url(self):
elif not self.repo_url.startswith("git@") or not self.repo_url.startswith("https://"):
self.repo_url = os.path.expanduser(self.repo_url)
+ def set_git_env(self, key: str) -> None:
+ self.env["GIT_SSH_COMMAND"] = self._build_ssh_command(key)
+
+ @contextlib.contextmanager
+ def configure_hook_env(self):
+ if self.private_key:
+ with tempfile.NamedTemporaryFile(mode="w", delete=True) as tmp_keyfile:
+ tmp_keyfile.write(self.private_key)
+ tmp_keyfile.flush()
+ os.chmod(tmp_keyfile.name, 0o600)
+ self.set_git_env(tmp_keyfile.name)
+ yield
+ else:
+ self.set_git_env(self.key_file)
+ yield
+
class GitDagBundle(BaseDagBundle, LoggingMixin):
"""
@@ -128,17 +154,20 @@ def __init__(
self.log.warning("Could not create GitHook for connection %s : %s", self.git_conn_id, e)
def _initialize(self):
- self._clone_bare_repo_if_required()
- self._ensure_version_in_bare_repo()
- self._clone_repo_if_required()
- self.repo.git.checkout(self.tracking_ref)
- if self.version:
- if not self._has_version(self.repo, self.version):
- self.repo.remotes.origin.fetch()
- self.repo.head.set_reference(self.repo.commit(self.version))
- self.repo.head.reset(index=True, working_tree=True)
- else:
- self.refresh()
+ with self.lock():
+ with self.hook.configure_hook_env():
+ self._clone_bare_repo_if_required()
+ self._ensure_version_in_bare_repo()
+
+ self._clone_repo_if_required()
+ self.repo.git.checkout(self.tracking_ref)
+ if self.version:
+ if not self._has_version(self.repo, self.version):
+ self.repo.remotes.origin.fetch()
+ self.repo.head.set_reference(self.repo.commit(self.version))
+ self.repo.head.reset(index=True, working_tree=True)
+ else:
+ self.refresh()
def initialize(self) -> None:
if not self.repo_url:
@@ -230,8 +259,11 @@ def _fetch_bare_repo(self):
def refresh(self) -> None:
if self.version:
raise AirflowException("Refreshing a specific version is not supported")
- self._fetch_bare_repo()
- self.repo.remotes.origin.pull()
+
+ with self.lock():
+ with self.hook.configure_hook_env():
+ self._fetch_bare_repo()
+ self.repo.remotes.origin.pull()
@staticmethod
def _convert_git_ssh_url_to_https(url: str) -> str:
diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py
index 815a45b0fd99e..3fb5ac3106d5d 100644
--- a/airflow/dag_processing/manager.py
+++ b/airflow/dag_processing/manager.py
@@ -63,6 +63,7 @@
from airflow.traces.tracer import Trace
from airflow.utils import timezone
from airflow.utils.file import list_py_file_paths, might_contain_dag
+from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.net import get_hostname
from airflow.utils.process_utils import (
kill_child_processes_by_pids,
@@ -97,9 +98,6 @@ class DagFileStat:
last_num_of_db_queries: int = 0
-log = logging.getLogger("airflow.processor_manager")
-
-
@dataclass(frozen=True)
class DagFileInfo:
"""Information about a DAG file."""
@@ -135,7 +133,7 @@ def _resolve_path(instance: Any, attribute: attrs.Attribute, val: str | os.PathL
@attrs.define
-class DagFileProcessorManager:
+class DagFileProcessorManager(LoggingMixin):
"""
Manage processes responsible for parsing DAGs.
@@ -167,8 +165,6 @@ class DagFileProcessorManager:
factory=_config_int_factory("dag_processor", "stale_dag_threshold")
)
- log: logging.Logger = attrs.field(default=log, init=False)
-
_last_deactivate_stale_dags_time: float = attrs.field(default=0, init=False)
print_stats_interval: float = attrs.field(
factory=_config_int_factory("dag_processor", "print_stats_interval")
diff --git a/airflow/decorators/base.py b/airflow/decorators/base.py
index 9db6d058adeb5..b79819bedcc5e 100644
--- a/airflow/decorators/base.py
+++ b/airflow/decorators/base.py
@@ -55,8 +55,6 @@
from airflow.utils.types import NOTSET
if TYPE_CHECKING:
- from sqlalchemy.orm import Session
-
from airflow.models.expandinput import (
ExpandInput,
OperatorExpandArgument,
@@ -184,7 +182,9 @@ def __init__(
kwargs_to_upstream: dict[str, Any] | None = None,
**kwargs,
) -> None:
- task_id = get_unique_task_id(task_id, kwargs.get("dag"), kwargs.get("task_group"))
+ if not getattr(self, "_BaseOperator__from_mapped", False):
+ # If we are being created from calling unmap(), then don't mangle the task id
+ task_id = get_unique_task_id(task_id, kwargs.get("dag"), kwargs.get("task_group"))
self.python_callable = python_callable
kwargs_to_upstream = kwargs_to_upstream or {}
op_args = op_args or []
@@ -218,10 +218,10 @@ def __init__(
The function signature broke while assigning defaults to context key parameters.
The decorator is replacing the signature
- > {python_callable.__name__}({', '.join(str(param) for param in signature.parameters.values())})
+ > {python_callable.__name__}({", ".join(str(param) for param in signature.parameters.values())})
with
- > {python_callable.__name__}({', '.join(str(param) for param in parameters)})
+ > {python_callable.__name__}({", ".join(str(param) for param in parameters)})
which isn't valid: {err}
"""
@@ -568,13 +568,11 @@ def __attrs_post_init__(self):
super(DecoratedMappedOperator, DecoratedMappedOperator).__attrs_post_init__(self)
XComArg.apply_upstream_relationship(self, self.op_kwargs_expand_input.value)
- def _expand_mapped_kwargs(
- self, context: Mapping[str, Any], session: Session, *, include_xcom: bool
- ) -> tuple[Mapping[str, Any], set[int]]:
+ def _expand_mapped_kwargs(self, context: Mapping[str, Any]) -> tuple[Mapping[str, Any], set[int]]:
# We only use op_kwargs_expand_input so this must always be empty.
if self.expand_input is not EXPAND_INPUT_EMPTY:
raise AssertionError(f"unexpected expand_input: {self.expand_input}")
- op_kwargs, resolved_oids = super()._expand_mapped_kwargs(context, session, include_xcom=include_xcom)
+ op_kwargs, resolved_oids = super()._expand_mapped_kwargs(context)
return {"op_kwargs": op_kwargs}, resolved_oids
def _get_unmap_kwargs(self, mapped_kwargs: Mapping[str, Any], *, strict: bool) -> dict[str, Any]:
diff --git a/airflow/migrations/versions/0032_3_0_0_drop_execution_date_unique.py b/airflow/migrations/versions/0032_3_0_0_rename_execution_date_to_logical_date_and_nullable.py
similarity index 84%
rename from airflow/migrations/versions/0032_3_0_0_drop_execution_date_unique.py
rename to airflow/migrations/versions/0032_3_0_0_rename_execution_date_to_logical_date_and_nullable.py
index 399cc8aff91f3..8a63d8112ac28 100644
--- a/airflow/migrations/versions/0032_3_0_0_drop_execution_date_unique.py
+++ b/airflow/migrations/versions/0032_3_0_0_rename_execution_date_to_logical_date_and_nullable.py
@@ -17,9 +17,9 @@
# under the License.
"""
-Drop ``execution_date`` unique constraint on DagRun.
+Make logical_date nullable.
-The column has also been renamed to logical_date, although the Python model is
+The column has been renamed to logical_date, although the Python model is
not changed. This allows us to not need to fix all the Python code at once, but
still do the two changes in one migration instead of two.
@@ -49,10 +49,15 @@ def upgrade():
"execution_date",
new_column_name="logical_date",
existing_type=TIMESTAMP(timezone=True),
- existing_nullable=False,
+ nullable=True,
)
+
with op.batch_alter_table("dag_run", schema=None) as batch_op:
batch_op.drop_constraint("dag_run_dag_id_execution_date_key", type_="unique")
+ batch_op.create_unique_constraint(
+ "dag_run_dag_id_logical_date_key",
+ columns=["dag_id", "logical_date"],
+ )
def downgrade():
@@ -61,9 +66,11 @@ def downgrade():
"logical_date",
new_column_name="execution_date",
existing_type=TIMESTAMP(timezone=True),
- existing_nullable=False,
+ nullable=False,
)
+
with op.batch_alter_table("dag_run", schema=None) as batch_op:
+ batch_op.drop_constraint("dag_run_dag_id_logical_date_key", type_="unique")
batch_op.create_unique_constraint(
"dag_run_dag_id_execution_date_key",
columns=["dag_id", "execution_date"],
diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py
index b8fb54f6966fd..98fd977c59128 100644
--- a/airflow/models/abstractoperator.py
+++ b/airflow/models/abstractoperator.py
@@ -27,7 +27,6 @@
from airflow.configuration import conf
from airflow.exceptions import AirflowException
-from airflow.models.expandinput import NotFullyPopulated
from airflow.sdk.definitions._internal.abstractoperator import (
AbstractOperator as TaskSDKAbstractOperator,
NotMapped as NotMapped, # Re-export this for compat
@@ -237,6 +236,7 @@ def expand_mapped_task(self, run_id: str, *, session: Session) -> tuple[Sequence
)
from airflow.models.baseoperator import BaseOperator as DBBaseOperator
+ from airflow.models.expandinput import NotFullyPopulated
try:
total_length: int | None = DBBaseOperator.get_mapped_ti_count(self, run_id, session=session)
diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py
index 425cbaca68880..3361fe33df6f7 100644
--- a/airflow/models/baseoperator.py
+++ b/airflow/models/baseoperator.py
@@ -848,11 +848,20 @@ def _(cls, task: TaskSDKAbstractOperator, run_id: str, *, session: Session) -> i
@get_mapped_ti_count.register(MappedOperator)
@classmethod
def _(cls, task: MappedOperator, run_id: str, *, session: Session) -> int:
- from airflow.serialization.serialized_objects import _ExpandInputRef
+ from airflow.serialization.serialized_objects import BaseSerialization, _ExpandInputRef
exp_input = task._get_specified_expand_input()
if isinstance(exp_input, _ExpandInputRef):
exp_input = exp_input.deref(task.dag)
+ # TODO: TaskSDK This is only needed to support `dag.test()` etc until we port it over to use the
+ # task sdk runner.
+ if not hasattr(exp_input, "get_total_map_length"):
+ exp_input = _ExpandInputRef(
+ type(exp_input).EXPAND_INPUT_TYPE,
+ BaseSerialization.deserialize(BaseSerialization.serialize(exp_input.value)),
+ )
+ exp_input = exp_input.deref(task.dag)
+
current_count = exp_input.get_total_map_length(run_id, session=session)
group = task.get_closest_mapped_task_group()
@@ -878,18 +887,24 @@ def _(cls, group: TaskGroup, run_id: str, *, session: Session) -> int:
:raise NotFullyPopulated: If upstream tasks are not all complete yet.
:return: Total number of mapped TIs this task should have.
"""
+ from airflow.serialization.serialized_objects import BaseSerialization, _ExpandInputRef
- def iter_mapped_task_groups(group) -> Iterator[MappedTaskGroup]:
+ def iter_mapped_task_group_lengths(group) -> Iterator[int]:
while group is not None:
if isinstance(group, MappedTaskGroup):
- yield group
+ exp_input = group._expand_input
+ # TODO: TaskSDK This is only needed to support `dag.test()` etc until we port it over to use the
+ # task sdk runner.
+ if not hasattr(exp_input, "get_total_map_length"):
+ exp_input = _ExpandInputRef(
+ type(exp_input).EXPAND_INPUT_TYPE,
+ BaseSerialization.deserialize(BaseSerialization.serialize(exp_input.value)),
+ )
+ exp_input = exp_input.deref(group.dag)
+ yield exp_input.get_total_map_length(run_id, session=session)
group = group.parent_group
- groups = iter_mapped_task_groups(group)
- return functools.reduce(
- operator.mul,
- (g._expand_input.get_total_map_length(run_id, session=session) for g in groups),
- )
+ return functools.reduce(operator.mul, iter_mapped_task_group_lengths(group))
def chain(*tasks: DependencyMixin | Sequence[DependencyMixin]) -> None:
diff --git a/airflow/models/connection.py b/airflow/models/connection.py
index 01df1626657da..a8b9bb87985d8 100644
--- a/airflow/models/connection.py
+++ b/airflow/models/connection.py
@@ -32,10 +32,10 @@
from airflow.exceptions import AirflowException, AirflowNotFoundException
from airflow.models.base import ID_LEN, Base
from airflow.models.crypto import get_fernet
+from airflow.sdk.execution_time.secrets_masker import mask_secret
from airflow.secrets.cache import SecretCache
from airflow.utils.helpers import prune_dict
from airflow.utils.log.logging_mixin import LoggingMixin
-from airflow.utils.log.secrets_masker import mask_secret
from airflow.utils.module_loading import import_string
log = logging.getLogger(__name__)
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index 4c54f3dba54b6..e571c016e1914 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -1756,7 +1756,7 @@ def create_dagrun(
self,
*,
run_id: str,
- logical_date: datetime,
+ logical_date: datetime | None,
data_interval: tuple[datetime, datetime],
run_after: datetime,
conf: dict | None = None,
diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py
index 7d46c29285fe0..d9bf14c3f983b 100644
--- a/airflow/models/dagrun.py
+++ b/airflow/models/dagrun.py
@@ -63,7 +63,6 @@
from airflow.models.backfill import Backfill
from airflow.models.base import Base, StringID
from airflow.models.dag_version import DagVersion
-from airflow.models.expandinput import NotFullyPopulated
from airflow.models.taskinstance import TaskInstance as TI
from airflow.models.tasklog import LogTemplate
from airflow.models.taskmap import TaskMap
@@ -135,7 +134,7 @@ class DagRun(Base, LoggingMixin):
id = Column(Integer, primary_key=True)
dag_id = Column(StringID(), nullable=False)
queued_at = Column(UtcDateTime)
- logical_date = Column(UtcDateTime, default=timezone.utcnow, nullable=False)
+ logical_date = Column(UtcDateTime, nullable=True)
start_date = Column(UtcDateTime)
end_date = Column(UtcDateTime)
_state = Column("state", String(50), default=DagRunState.QUEUED)
@@ -186,6 +185,7 @@ class DagRun(Base, LoggingMixin):
__table_args__ = (
Index("dag_id_state", dag_id, _state),
UniqueConstraint("dag_id", "run_id", name="dag_run_dag_id_run_id_key"),
+ UniqueConstraint("dag_id", "logical_date", name="dag_run_dag_id_logical_date_key"),
Index("idx_dag_run_dag_id", dag_id),
Index("idx_dag_run_run_after", run_after),
Index(
@@ -792,6 +792,8 @@ def get_previous_dagrun(
:param session: SQLAlchemy ORM Session
:param state: the dag run state
"""
+ if dag_run.logical_date is None:
+ return None
filters = [
DagRun.dag_id == dag_run.dag_id,
DagRun.logical_date < dag_run.logical_date,
@@ -1321,8 +1323,12 @@ def verify_integrity(self, *, session: Session = NEW_SESSION) -> None:
def task_filter(task: Operator) -> bool:
return task.task_id not in task_ids and (
self.run_type == DagRunType.BACKFILL_JOB
- or (task.start_date is None or task.start_date <= self.logical_date)
- and (task.end_date is None or self.logical_date <= task.end_date)
+ or (
+ task.start_date is None
+ or self.logical_date is None
+ or task.start_date <= self.logical_date
+ )
+ and (task.end_date is None or self.logical_date is None or self.logical_date <= task.end_date)
)
created_counts: dict[str, int] = defaultdict(int)
@@ -1347,6 +1353,7 @@ def _check_for_removed_or_restored_tasks(
"""
from airflow.models.baseoperator import BaseOperator
+ from airflow.models.expandinput import NotFullyPopulated
tis = self.get_task_instances(session=session)
@@ -1484,6 +1491,7 @@ def _create_tasks(
:param task_creator: Function to create task instances
"""
from airflow.models.baseoperator import BaseOperator
+ from airflow.models.expandinput import NotFullyPopulated
map_indexes: Iterable[int]
for task in tasks:
@@ -1555,6 +1563,7 @@ def _revise_map_indexes_if_mapped(self, task: Operator, *, session: Session) ->
for more details.
"""
from airflow.models.baseoperator import BaseOperator
+ from airflow.models.expandinput import NotFullyPopulated
from airflow.settings import task_instance_mutation_hook
try:
diff --git a/airflow/models/expandinput.py b/airflow/models/expandinput.py
index 8fb35f7032965..72f7b0eca22ba 100644
--- a/airflow/models/expandinput.py
+++ b/airflow/models/expandinput.py
@@ -17,108 +17,54 @@
# under the License.
from __future__ import annotations
-import collections.abc
import functools
import operator
-from collections.abc import Iterable, Mapping, Sequence, Sized
-from typing import TYPE_CHECKING, Any, NamedTuple, Union
+from collections.abc import Iterable, Sized
+from typing import TYPE_CHECKING, Any
-import attr
-
-from airflow.sdk.definitions._internal.mixins import ResolveMixin
-from airflow.utils.session import NEW_SESSION, provide_session
+import attrs
if TYPE_CHECKING:
from sqlalchemy.orm import Session
- from airflow.models.xcom_arg import XComArg
- from airflow.sdk.types import Operator
- from airflow.serialization.serialized_objects import _ExpandInputRef
+ from airflow.models.xcom_arg import SchedulerXComArg
from airflow.typing_compat import TypeGuard
-ExpandInput = Union["DictOfListsExpandInput", "ListOfDictsExpandInput"]
-
-# Each keyword argument to expand() can be an XComArg, sequence, or dict (not
-# any mapping since we need the value to be ordered).
-OperatorExpandArgument = Union["MappedArgument", "XComArg", Sequence, dict[str, Any]]
-
-# The single argument of expand_kwargs() can be an XComArg, or a list with each
-# element being either an XComArg or a dict.
-OperatorExpandKwargsArgument = Union["XComArg", Sequence[Union["XComArg", Mapping[str, Any]]]]
-
-
-@attr.define(kw_only=True)
-class MappedArgument(ResolveMixin):
- """
- Stand-in stub for task-group-mapping arguments.
-
- This is very similar to an XComArg, but resolved differently. Declared here
- (instead of in the task group module) to avoid import cycles.
- """
-
- _input: ExpandInput
- _key: str
-
- def iter_references(self) -> Iterable[tuple[Operator, str]]:
- yield from self._input.iter_references()
-
- @provide_session
- def resolve(
- self, context: Mapping[str, Any], *, include_xcom: bool = True, session: Session = NEW_SESSION
- ) -> Any:
- data, _ = self._input.resolve(context, session=session, include_xcom=include_xcom)
- return data[self._key]
-
-
-# To replace tedious isinstance() checks.
-def is_mappable(v: Any) -> TypeGuard[OperatorExpandArgument]:
- from airflow.models.xcom_arg import XComArg
-
- return isinstance(v, (MappedArgument, XComArg, Mapping, Sequence)) and not isinstance(v, str)
-
-
-# To replace tedious isinstance() checks.
-def _is_parse_time_mappable(v: OperatorExpandArgument) -> TypeGuard[Mapping | Sequence]:
- from airflow.models.xcom_arg import XComArg
-
- return not isinstance(v, (MappedArgument, XComArg))
-
-
-# To replace tedious isinstance() checks.
-def _needs_run_time_resolution(v: OperatorExpandArgument) -> TypeGuard[MappedArgument | XComArg]:
- from airflow.models.xcom_arg import XComArg
-
- return isinstance(v, (MappedArgument, XComArg))
-
-
-class NotFullyPopulated(RuntimeError):
- """
- Raise when ``get_map_lengths`` cannot populate all mapping metadata.
-
- This is generally due to not all upstream tasks have finished when the
- function is called.
- """
+from airflow.sdk.definitions._internal.expandinput import (
+ DictOfListsExpandInput,
+ ExpandInput,
+ ListOfDictsExpandInput,
+ MappedArgument,
+ NotFullyPopulated,
+ OperatorExpandArgument,
+ OperatorExpandKwargsArgument,
+ is_mappable,
+)
- def __init__(self, missing: set[str]) -> None:
- self.missing = missing
+__all__ = [
+ "DictOfListsExpandInput",
+ "ListOfDictsExpandInput",
+ "MappedArgument",
+ "NotFullyPopulated",
+ "OperatorExpandArgument",
+ "OperatorExpandKwargsArgument",
+ "is_mappable",
+]
- def __str__(self) -> str:
- keys = ", ".join(repr(k) for k in sorted(self.missing))
- return f"Failed to populate all mapping metadata; missing: {keys}"
+def _needs_run_time_resolution(v: OperatorExpandArgument) -> TypeGuard[MappedArgument | SchedulerXComArg]:
+ from airflow.models.xcom_arg import SchedulerXComArg
-class DictOfListsExpandInput(NamedTuple):
- """
- Storage type of a mapped operator's mapped kwargs.
+ return isinstance(v, (MappedArgument, SchedulerXComArg))
- This is created from ``expand(**kwargs)``.
- """
- value: dict[str, OperatorExpandArgument]
+@attrs.define
+class SchedulerDictOfListsExpandInput:
+ value: dict
def _iter_parse_time_resolved_kwargs(self) -> Iterable[tuple[str, Sized]]:
"""Generate kwargs with values available on parse-time."""
- return ((k, v) for k, v in self.value.items() if _is_parse_time_mappable(v))
+ return ((k, v) for k, v in self.value.items() if not _needs_run_time_resolution(v))
def get_parse_time_mapped_ti_count(self) -> int:
if not self.value:
@@ -136,13 +82,12 @@ def _get_map_lengths(self, run_id: str, *, session: Session) -> dict[str, int]:
If any arguments are not known right now (upstream task not finished),
they will not be present in the dict.
"""
+ from airflow.models.xcom_arg import SchedulerXComArg, get_task_map_length
# TODO: This initiates one database call for each XComArg. Would it be
# more efficient to do one single db call and unpack the value here?
def _get_length(v: OperatorExpandArgument) -> int | None:
- from airflow.models.xcom_arg import get_task_map_length
-
- if _needs_run_time_resolution(v):
+ if isinstance(v, SchedulerXComArg):
return get_task_map_length(v, run_id, session=session)
# Unfortunately a user-defined TypeGuard cannot apply negative type
@@ -164,150 +109,34 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int:
lengths = self._get_map_lengths(run_id, session=session)
return functools.reduce(operator.mul, (lengths[name] for name in self.value), 1)
- def _expand_mapped_field(
- self, key: str, value: Any, context: Mapping[str, Any], *, session: Session, include_xcom: bool
- ) -> Any:
- if _needs_run_time_resolution(value):
- value = (
- value.resolve(context, session=session, include_xcom=include_xcom)
- if include_xcom
- else str(value)
- )
- map_index = context["ti"].map_index
- if map_index < 0:
- raise RuntimeError("can't resolve task-mapping argument without expanding")
- all_lengths = self._get_map_lengths(context["run_id"], session=session)
-
- def _find_index_for_this_field(index: int) -> int:
- # Need to use the original user input to retain argument order.
- for mapped_key in reversed(self.value):
- mapped_length = all_lengths[mapped_key]
- if mapped_length < 1:
- raise RuntimeError(f"cannot expand field mapped to length {mapped_length!r}")
- if mapped_key == key:
- return index % mapped_length
- index //= mapped_length
- return -1
-
- found_index = _find_index_for_this_field(map_index)
- if found_index < 0:
- return value
- if isinstance(value, collections.abc.Sequence):
- return value[found_index]
- if not isinstance(value, dict):
- raise TypeError(f"can't map over value of type {type(value)}")
- for i, (k, v) in enumerate(value.items()):
- if i == found_index:
- return k, v
- raise IndexError(f"index {map_index} is over mapped length")
-
- def iter_references(self) -> Iterable[tuple[Operator, str]]:
- from airflow.models.xcom_arg import XComArg
-
- for x in self.value.values():
- if isinstance(x, XComArg):
- yield from x.iter_references()
-
- def resolve(
- self, context: Mapping[str, Any], session: Session, *, include_xcom: bool = True
- ) -> tuple[Mapping[str, Any], set[int]]:
- data = {
- k: self._expand_mapped_field(k, v, context, session=session, include_xcom=include_xcom)
- for k, v in self.value.items()
- }
- literal_keys = {k for k, _ in self._iter_parse_time_resolved_kwargs()}
- resolved_oids = {id(v) for k, v in data.items() if k not in literal_keys}
- return data, resolved_oids
-
-
-def _describe_type(value: Any) -> str:
- if value is None:
- return "None"
- return type(value).__name__
-
-
-class ListOfDictsExpandInput(NamedTuple):
- """
- Storage type of a mapped operator's mapped kwargs.
-
- This is created from ``expand_kwargs(xcom_arg)``.
- """
- value: OperatorExpandKwargsArgument
+@attrs.define
+class SchedulerListOfDictsExpandInput:
+ value: list
def get_parse_time_mapped_ti_count(self) -> int:
- if isinstance(self.value, collections.abc.Sized):
+ if isinstance(self.value, Sized):
return len(self.value)
raise NotFullyPopulated({"expand_kwargs() argument"})
def get_total_map_length(self, run_id: str, *, session: Session) -> int:
from airflow.models.xcom_arg import get_task_map_length
- if isinstance(self.value, collections.abc.Sized):
+ if isinstance(self.value, Sized):
return len(self.value)
length = get_task_map_length(self.value, run_id, session=session)
if length is None:
raise NotFullyPopulated({"expand_kwargs() argument"})
return length
- def iter_references(self) -> Iterable[tuple[Operator, str]]:
- from airflow.models.xcom_arg import XComArg
-
- if isinstance(self.value, XComArg):
- yield from self.value.iter_references()
- else:
- for x in self.value:
- if isinstance(x, XComArg):
- yield from x.iter_references()
-
- def resolve(
- self, context: Mapping[str, Any], session: Session, *, include_xcom: bool = True
- ) -> tuple[Mapping[str, Any], set[int]]:
- map_index = context["ti"].map_index
- if map_index < 0:
- raise RuntimeError("can't resolve task-mapping argument without expanding")
-
- mapping: Any
- if isinstance(self.value, collections.abc.Sized):
- mapping = self.value[map_index]
- if not isinstance(mapping, collections.abc.Mapping):
- mapping = mapping.resolve(context, session, include_xcom=include_xcom)
- elif include_xcom:
- mappings = self.value.resolve(context, session, include_xcom=include_xcom)
- if not isinstance(mappings, collections.abc.Sequence):
- raise ValueError(f"expand_kwargs() expects a list[dict], not {_describe_type(mappings)}")
- mapping = mappings[map_index]
-
- if not isinstance(mapping, collections.abc.Mapping):
- raise ValueError(f"expand_kwargs() expects a list[dict], not list[{_describe_type(mapping)}]")
-
- for key in mapping:
- if not isinstance(key, str):
- raise ValueError(
- f"expand_kwargs() input dict keys must all be str, "
- f"but {key!r} is of type {_describe_type(key)}"
- )
- # filter out parse time resolved values from the resolved_oids
- resolved_oids = {id(v) for k, v in mapping.items() if not _is_parse_time_mappable(v)}
-
- return mapping, resolved_oids
-
EXPAND_INPUT_EMPTY = DictOfListsExpandInput({}) # Sentinel value.
_EXPAND_INPUT_TYPES = {
- "dict-of-lists": DictOfListsExpandInput,
- "list-of-dicts": ListOfDictsExpandInput,
+ "dict-of-lists": SchedulerDictOfListsExpandInput,
+ "list-of-dicts": SchedulerListOfDictsExpandInput,
}
-def get_map_type_key(expand_input: ExpandInput | _ExpandInputRef) -> str:
- from airflow.serialization.serialized_objects import _ExpandInputRef
-
- if isinstance(expand_input, _ExpandInputRef):
- return expand_input.key
- return next(k for k, v in _EXPAND_INPUT_TYPES.items() if isinstance(expand_input, v))
-
-
def create_expand_input(kind: str, value: Any) -> ExpandInput:
return _EXPAND_INPUT_TYPES[kind](value)
diff --git a/airflow/models/mappedoperator.py b/airflow/models/mappedoperator.py
index e7352ad1323d3..9b0b90b5814f5 100644
--- a/airflow/models/mappedoperator.py
+++ b/airflow/models/mappedoperator.py
@@ -17,10 +17,7 @@
# under the License.
from __future__ import annotations
-import contextlib
-import copy
-from collections.abc import Mapping
-from typing import TYPE_CHECKING, Any
+from typing import TYPE_CHECKING
import attrs
@@ -51,49 +48,6 @@
class MappedOperator(TaskSDKMappedOperator, AbstractOperator): # type: ignore[misc] # It complains about weight_rule being different
"""Object representing a mapped operator in a DAG."""
- def _expand_mapped_kwargs(
- self, context: Mapping[str, Any], session: Session, *, include_xcom: bool
- ) -> tuple[Mapping[str, Any], set[int]]:
- """
- Get the kwargs to create the unmapped operator.
-
- This exists because taskflow operators expand against op_kwargs, not the
- entire operator kwargs dict.
- """
- return self._get_specified_expand_input().resolve(context, session, include_xcom=include_xcom)
-
- def _get_unmap_kwargs(self, mapped_kwargs: Mapping[str, Any], *, strict: bool) -> dict[str, Any]:
- """
- Get init kwargs to unmap the underlying operator class.
-
- :param mapped_kwargs: The dict returned by ``_expand_mapped_kwargs``.
- """
- if strict:
- prevent_duplicates(
- self.partial_kwargs,
- mapped_kwargs,
- fail_reason="unmappable or already specified",
- )
-
- # If params appears in the mapped kwargs, we need to merge it into the
- # partial params, overriding existing keys.
- params = copy.copy(self.params)
- with contextlib.suppress(KeyError):
- params.update(mapped_kwargs["params"])
-
- # Ordering is significant; mapped kwargs should override partial ones,
- # and the specially handled params should be respected.
- return {
- "task_id": self.task_id,
- "dag": self.dag,
- "task_group": self.task_group,
- "start_date": self.start_date,
- "end_date": self.end_date,
- **self.partial_kwargs,
- **mapped_kwargs,
- "params": params,
- }
-
def expand_start_from_trigger(self, *, context: Context, session: Session) -> bool:
"""
Get the start_from_trigger value of the current abstract operator.
@@ -107,7 +61,7 @@ def expand_start_from_trigger(self, *, context: Context, session: Session) -> bo
if not self.start_trigger_args:
return False
- mapped_kwargs, _ = self._expand_mapped_kwargs(context, session, include_xcom=False)
+ mapped_kwargs, _ = self._expand_mapped_kwargs(context)
if self._disallow_kwargs_override:
prevent_duplicates(
self.partial_kwargs,
@@ -129,7 +83,7 @@ def expand_start_trigger_args(self, *, context: Context, session: Session) -> St
if not self.start_trigger_args:
return None
- mapped_kwargs, _ = self._expand_mapped_kwargs(context, session, include_xcom=False)
+ mapped_kwargs, _ = self._expand_mapped_kwargs(context)
if self._disallow_kwargs_override:
prevent_duplicates(
self.partial_kwargs,
diff --git a/airflow/models/renderedtifields.py b/airflow/models/renderedtifields.py
index f2d7d83920fff..ea1c3af2b65b7 100644
--- a/airflow/models/renderedtifields.py
+++ b/airflow/models/renderedtifields.py
@@ -145,7 +145,7 @@ def __repr__(self):
return prefix + ">"
def _redact(self):
- from airflow.utils.log.secrets_masker import redact
+ from airflow.sdk.execution_time.secrets_masker import redact
if self.k8s_pod_yaml:
self.k8s_pod_yaml = redact(self.k8s_pod_yaml)
diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py
index 5da2f24957ff0..2787ce1b82993 100644
--- a/airflow/models/serialized_dag.py
+++ b/airflow/models/serialized_dag.py
@@ -291,7 +291,7 @@ def read_all_dags(cls, session: Session = NEW_SESSION) -> dict[str, SerializedDA
@property
def data(self) -> dict | None:
# use __data_cache to avoid decompress and loads
- if not hasattr(self, "__data_cache") or self.__data_cache is None:
+ if not hasattr(self, "_SerializedDagModel__data_cache") or self.__data_cache is None:
if self._data_compressed:
self.__data_cache = json.loads(zlib.decompress(self._data_compressed))
else:
diff --git a/airflow/models/skipmixin.py b/airflow/models/skipmixin.py
index 3b7d21d7b386b..78e1a631e080d 100644
--- a/airflow/models/skipmixin.py
+++ b/airflow/models/skipmixin.py
@@ -97,7 +97,7 @@ def skip(
dag_id: str,
run_id: str,
tasks: Iterable[DAGNode],
- map_index: int = -1,
+ map_index: int | None = -1,
session: Session = NEW_SESSION,
):
"""
@@ -126,6 +126,9 @@ def skip(
if task_id is not None:
from airflow.models.xcom import XCom
+ if map_index is None:
+ map_index = -1
+
XCom.set(
key=XCOM_SKIPMIXIN_KEY,
value={XCOM_SKIPMIXIN_SKIPPED: task_ids_list},
diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py
index 7106366b67ef8..ceb0acfcf9946 100644
--- a/airflow/models/taskinstance.py
+++ b/airflow/models/taskinstance.py
@@ -573,39 +573,6 @@ def _xcom_pull(
default: Any = None,
run_id: str | None = None,
) -> Any:
- """
- Pull XComs that optionally meet certain criteria.
-
- :param key: A key for the XCom. If provided, only XComs with matching
- keys will be returned. The default key is ``'return_value'``, also
- available as constant ``XCOM_RETURN_KEY``. This key is automatically
- given to XComs returned by tasks (as opposed to being pushed
- manually). To remove the filter, pass *None*.
- :param task_ids: Only XComs from tasks with matching ids will be
- pulled. Pass *None* to remove the filter.
- :param dag_id: If provided, only pulls XComs from this DAG. If *None*
- (default), the DAG of the calling task is used.
- :param map_indexes: If provided, only pull XComs with matching indexes.
- If *None* (default), this is inferred from the task(s) being pulled
- (see below for details).
- :param include_prior_dates: If False, only XComs from the current
- logical_date are returned. If *True*, XComs from previous dates
- are returned as well.
- :param run_id: If provided, only pulls XComs from a DagRun w/a matching run_id.
- If *None* (default), the run_id of the calling task is used.
-
- When pulling one single task (``task_id`` is *None* or a str) without
- specifying ``map_indexes``, the return value is inferred from whether
- the specified task is mapped. If not, value from the one single task
- instance is returned. If the task to pull is mapped, an iterator (not a
- list) yielding XComs from mapped task instances is returned. In either
- case, ``default`` (*None* if not specified) is returned if no matching
- XComs are found.
-
- When pulling multiple tasks (i.e. either ``task_id`` or ``map_index`` is
- a non-str iterable), a list of matching XComs is returned. Elements in
- the list is ordered by item ordering in ``task_id`` and ``map_index``.
- """
if dag_id is None:
dag_id = ti.dag_id
if run_id is None:
@@ -634,11 +601,10 @@ def _xcom_pull(
return default
if map_indexes is not None or first.map_index < 0:
return XCom.deserialize_value(first)
- return LazyXComSelectSequence.from_select(
- query.with_entities(XCom.value).order_by(None).statement,
- order_by=[XCom.map_index],
- session=session,
- )
+
+ # raise RuntimeError("Nothing should hit this anymore")
+
+ # TODO: TaskSDK: We should remove this, but many tests still currently call `ti.run()`. See #45549
# At this point either task_ids or map_indexes is explicitly multi-value.
# Order return values to match task_ids and map_indexes ordering.
@@ -1035,14 +1001,18 @@ def get_triggering_events() -> dict[str, list[AssetEvent]]:
)
context["expanded_ti_count"] = expanded_ti_count
if expanded_ti_count:
- context["_upstream_map_indexes"] = { # type: ignore[typeddict-unknown-key]
- upstream.task_id: task_instance.get_relevant_upstream_map_indexes(
- upstream,
- expanded_ti_count,
- session=session,
- )
- for upstream in task.upstream_list
- }
+ setattr(
+ task_instance,
+ "_upstream_map_indexes",
+ {
+ upstream.task_id: task_instance.get_relevant_upstream_map_indexes(
+ upstream,
+ expanded_ti_count,
+ session=session,
+ )
+ for upstream in task.upstream_list
+ },
+ )
except NotMapped:
pass
@@ -3267,7 +3237,7 @@ def get_rendered_template_fields(self, session: Session = NEW_SESSION) -> None:
try:
# If we get here, either the task hasn't run or the RTIF record was purged.
- from airflow.utils.log.secrets_masker import redact
+ from airflow.sdk.execution_time.secrets_masker import redact
self.render_templates()
for field_name in self.task.template_fields:
@@ -3380,39 +3350,8 @@ def xcom_pull(
default: Any = None,
run_id: str | None = None,
) -> Any:
- """
- Pull XComs that optionally meet certain criteria.
-
- :param key: A key for the XCom. If provided, only XComs with matching
- keys will be returned. The default key is ``'return_value'``, also
- available as constant ``XCOM_RETURN_KEY``. This key is automatically
- given to XComs returned by tasks (as opposed to being pushed
- manually). To remove the filter, pass *None*.
- :param task_ids: Only XComs from tasks with matching ids will be
- pulled. Pass *None* to remove the filter.
- :param dag_id: If provided, only pulls XComs from this DAG. If *None*
- (default), the DAG of the calling task is used.
- :param map_indexes: If provided, only pull XComs with matching indexes.
- If *None* (default), this is inferred from the task(s) being pulled
- (see below for details).
- :param include_prior_dates: If False, only XComs from the current
- logical_date are returned. If *True*, XComs from previous dates
- are returned as well.
- :param run_id: If provided, only pulls XComs from a DagRun w/a matching run_id.
- If *None* (default), the run_id of the calling task is used.
-
- When pulling one single task (``task_id`` is *None* or a str) without
- specifying ``map_indexes``, the return value is inferred from whether
- the specified task is mapped. If not, value from the one single task
- instance is returned. If the task to pull is mapped, an iterator (not a
- list) yielding XComs from mapped task instances is returned. In either
- case, ``default`` (*None* if not specified) is returned if no matching
- XComs are found.
-
- When pulling multiple tasks (i.e. either ``task_id`` or ``map_index`` is
- a non-str iterable), a list of matching XComs is returned. Elements in
- the list is ordered by item ordering in ``task_id`` and ``map_index``.
- """
+ """:meta private:""" # noqa: D400
+ # This is only kept for compatibility in tests for now while AIP-72 is in progress.
return _xcom_pull(
ti=self,
task_ids=task_ids,
diff --git a/airflow/models/taskinstancehistory.py b/airflow/models/taskinstancehistory.py
index e97e6de22ec9a..d99cd34f3b88c 100644
--- a/airflow/models/taskinstancehistory.py
+++ b/airflow/models/taskinstancehistory.py
@@ -33,6 +33,7 @@
text,
)
from sqlalchemy.ext.mutable import MutableDict
+from sqlalchemy.orm import relationship
from sqlalchemy_utils import UUIDType
from airflow.models.base import Base, StringID
@@ -94,6 +95,13 @@ class TaskInstanceHistory(Base):
task_display_name = Column("task_display_name", String(2000), nullable=True)
dag_version_id = Column(UUIDType(binary=False))
+ dag_version = relationship(
+ "DagVersion",
+ primaryjoin="TaskInstanceHistory.dag_version_id == DagVersion.id",
+ viewonly=True,
+ foreign_keys=[dag_version_id],
+ )
+
def __init__(
self,
ti: TaskInstance,
diff --git a/airflow/models/variable.py b/airflow/models/variable.py
index b4cf5560e8421..b4568ad09c489 100644
--- a/airflow/models/variable.py
+++ b/airflow/models/variable.py
@@ -28,10 +28,10 @@
from airflow.configuration import ensure_secrets_loaded
from airflow.models.base import ID_LEN, Base
from airflow.models.crypto import get_fernet
+from airflow.sdk.execution_time.secrets_masker import mask_secret
from airflow.secrets.cache import SecretCache
from airflow.secrets.metastore import MetastoreBackend
from airflow.utils.log.logging_mixin import LoggingMixin
-from airflow.utils.log.secrets_masker import mask_secret
from airflow.utils.session import provide_session
if TYPE_CHECKING:
diff --git a/airflow/models/xcom_arg.py b/airflow/models/xcom_arg.py
index 078a9e6ff5223..1d885fb5bd1b0 100644
--- a/airflow/models/xcom_arg.py
+++ b/airflow/models/xcom_arg.py
@@ -17,46 +17,108 @@
from __future__ import annotations
+from collections.abc import Sequence
from functools import singledispatch
-from typing import TYPE_CHECKING
+from typing import TYPE_CHECKING, Any
+import attrs
from sqlalchemy import func, or_, select
from sqlalchemy.orm import Session
from airflow.sdk.definitions._internal.types import ArgNotSet
from airflow.sdk.definitions.mappedoperator import MappedOperator
from airflow.sdk.definitions.xcom_arg import (
- ConcatXComArg,
- MapXComArg,
- PlainXComArg,
XComArg,
- ZipXComArg,
)
from airflow.utils.db import exists_query
from airflow.utils.state import State
+from airflow.utils.types import NOTSET
from airflow.utils.xcom import XCOM_RETURN_KEY
__all__ = ["XComArg", "get_task_map_length"]
if TYPE_CHECKING:
- from airflow.models.expandinput import OperatorExpandArgument
+ from airflow.models.dag import DAG as SchedulerDAG
+ from airflow.models.operator import Operator
+ from airflow.typing_compat import Self
+
+
+@attrs.define
+class SchedulerXComArg:
+ @classmethod
+ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self:
+ """
+ Deserialize an XComArg.
+
+ The implementation should be the inverse function to ``serialize``,
+ implementing given a data dict converted from this XComArg derivative,
+ how the original XComArg should be created. DAG serialization relies on
+ additional information added in ``serialize_xcom_arg`` to dispatch data
+ dicts to the correct ``_deserialize`` information, so this function does
+ not need to validate whether the incoming data contains correct keys.
+ """
+ raise NotImplementedError()
+
+
+@attrs.define
+class SchedulerPlainXComArg(SchedulerXComArg):
+ operator: Operator
+ key: str
+
+ @classmethod
+ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self:
+ return cls(dag.get_task(data["task_id"]), data["key"])
+
+
+@attrs.define
+class SchedulerMapXComArg(SchedulerXComArg):
+ arg: SchedulerXComArg
+ callables: Sequence[str]
+
+ @classmethod
+ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self:
+ # We are deliberately NOT deserializing the callables. These are shown
+ # in the UI, and displaying a function object is useless.
+ return cls(deserialize_xcom_arg(data["arg"], dag), data["callables"])
+
+
+@attrs.define
+class SchedulerConcatXComArg(SchedulerXComArg):
+ args: Sequence[SchedulerXComArg]
+
+ @classmethod
+ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self:
+ return cls([deserialize_xcom_arg(arg, dag) for arg in data["args"]])
+
+
+@attrs.define
+class SchedulerZipXComArg(SchedulerXComArg):
+ args: Sequence[SchedulerXComArg]
+ fillvalue: Any
+
+ @classmethod
+ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self:
+ return cls(
+ [deserialize_xcom_arg(arg, dag) for arg in data["args"]],
+ fillvalue=data.get("fillvalue", NOTSET),
+ )
@singledispatch
-def get_task_map_length(xcom_arg: OperatorExpandArgument, run_id: str, *, session: Session) -> int | None:
+def get_task_map_length(xcom_arg: SchedulerXComArg, run_id: str, *, session: Session) -> int | None:
# The base implementation -- specific XComArg subclasses have specialised implementations
- raise NotImplementedError()
+ raise NotImplementedError(f"get_task_map_length not implemented for {type(xcom_arg)}")
@get_task_map_length.register
-def _(xcom_arg: PlainXComArg, run_id: str, *, session: Session):
+def _(xcom_arg: SchedulerPlainXComArg, run_id: str, *, session: Session):
from airflow.models.taskinstance import TaskInstance
from airflow.models.taskmap import TaskMap
from airflow.models.xcom import XCom
dag_id = xcom_arg.operator.dag_id
task_id = xcom_arg.operator.task_id
- is_mapped = isinstance(xcom_arg.operator, MappedOperator)
+ is_mapped = xcom_arg.operator.is_mapped or isinstance(xcom_arg.operator, MappedOperator)
if is_mapped:
unfinished_ti_exists = exists_query(
@@ -92,12 +154,12 @@ def _(xcom_arg: PlainXComArg, run_id: str, *, session: Session):
@get_task_map_length.register
-def _(xcom_arg: MapXComArg, run_id: str, *, session: Session):
+def _(xcom_arg: SchedulerMapXComArg, run_id: str, *, session: Session):
return get_task_map_length(xcom_arg.arg, run_id, session=session)
@get_task_map_length.register
-def _(xcom_arg: ZipXComArg, run_id: str, *, session: Session):
+def _(xcom_arg: SchedulerZipXComArg, run_id: str, *, session: Session):
all_lengths = (get_task_map_length(arg, run_id, session=session) for arg in xcom_arg.args)
ready_lengths = [length for length in all_lengths if length is not None]
if len(ready_lengths) != len(xcom_arg.args):
@@ -108,9 +170,23 @@ def _(xcom_arg: ZipXComArg, run_id: str, *, session: Session):
@get_task_map_length.register
-def _(xcom_arg: ConcatXComArg, run_id: str, *, session: Session):
+def _(xcom_arg: SchedulerConcatXComArg, run_id: str, *, session: Session):
all_lengths = (get_task_map_length(arg, run_id, session=session) for arg in xcom_arg.args)
ready_lengths = [length for length in all_lengths if length is not None]
if len(ready_lengths) != len(xcom_arg.args):
return None # If any of the referenced XComs is not ready, we are not ready either.
return sum(ready_lengths)
+
+
+def deserialize_xcom_arg(data: dict[str, Any], dag: SchedulerDAG):
+ """DAG serialization interface."""
+ klass = _XCOM_ARG_TYPES[data.get("type", "")]
+ return klass._deserialize(data, dag)
+
+
+_XCOM_ARG_TYPES: dict[str, type[SchedulerXComArg]] = {
+ "": SchedulerPlainXComArg,
+ "concat": SchedulerConcatXComArg,
+ "map": SchedulerMapXComArg,
+ "zip": SchedulerZipXComArg,
+}
diff --git a/airflow/new_provider.yaml.schema.json b/airflow/new_provider.yaml.schema.json
index 83f98abb038cb..4b5e16cedc0eb 100644
--- a/airflow/new_provider.yaml.schema.json
+++ b/airflow/new_provider.yaml.schema.json
@@ -31,6 +31,13 @@
"removed"
]
},
+ "excluded-python-versions": {
+ "description": "List of python versions excluded for that provider",
+ "type": "array",
+ "items": {
+ "type": "string"
+ }
+ },
"integrations": {
"description": "List of integrations supported by the provider.",
"type": "array",
diff --git a/airflow/serialization/helpers.py b/airflow/serialization/helpers.py
index 2e0e8cba41102..6cea9e1341536 100644
--- a/airflow/serialization/helpers.py
+++ b/airflow/serialization/helpers.py
@@ -21,8 +21,8 @@
from typing import Any
from airflow.configuration import conf
+from airflow.sdk.execution_time.secrets_masker import redact
from airflow.settings import json
-from airflow.utils.log.secrets_masker import redact
def serialize_template_field(template_field: Any, name: str) -> str | dict | list | int | float:
diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py
index 89ea668f02ffe..08d032e873c6a 100644
--- a/airflow/serialization/serialized_objects.py
+++ b/airflow/serialization/serialized_objects.py
@@ -45,10 +45,10 @@
from airflow.models.expandinput import (
EXPAND_INPUT_EMPTY,
create_expand_input,
- get_map_type_key,
)
from airflow.models.taskinstance import SimpleTaskInstance
from airflow.models.taskinstancekey import TaskInstanceKey
+from airflow.models.xcom_arg import SchedulerXComArg, deserialize_xcom_arg
from airflow.providers_manager import ProvidersManager
from airflow.sdk.definitions.asset import (
Asset,
@@ -66,7 +66,7 @@
from airflow.sdk.definitions.mappedoperator import MappedOperator
from airflow.sdk.definitions.param import Param, ParamsDict
from airflow.sdk.definitions.taskgroup import MappedTaskGroup, TaskGroup
-from airflow.sdk.definitions.xcom_arg import XComArg, deserialize_xcom_arg, serialize_xcom_arg
+from airflow.sdk.definitions.xcom_arg import XComArg, serialize_xcom_arg
from airflow.sdk.execution_time.context import OutletEventAccessor, OutletEventAccessors
from airflow.serialization.dag_dependency import DagDependency
from airflow.serialization.enums import DagAttributeTypes as DAT, Encoding
@@ -493,7 +493,7 @@ class _XComRef(NamedTuple):
data: dict
- def deref(self, dag: DAG) -> XComArg:
+ def deref(self, dag: DAG) -> SchedulerXComArg:
return deserialize_xcom_arg(self.data, dag)
@@ -1195,7 +1195,7 @@ def serialize_mapped_operator(cls, op: MappedOperator) -> dict[str, Any]:
if TYPE_CHECKING: # Let Mypy check the input type for us!
_ExpandInputRef.validate_expand_input_value(expansion_kwargs.value)
serialized_op[op._expand_input_attr] = {
- "type": get_map_type_key(expansion_kwargs),
+ "type": type(expansion_kwargs).EXPAND_INPUT_TYPE,
"value": cls.serialize(expansion_kwargs.value),
}
@@ -1792,7 +1792,7 @@ def serialize_task_group(cls, task_group: TaskGroup) -> dict[str, Any] | None:
if isinstance(task_group, MappedTaskGroup):
expand_input = task_group._expand_input
encoded["expand_input"] = {
- "type": get_map_type_key(expand_input),
+ "type": expand_input.EXPAND_INPUT_TYPE,
"value": cls.serialize(expand_input.value),
}
encoded["is_mapped"] = True
diff --git a/airflow/settings.py b/airflow/settings.py
index aae6529c8b129..d8e796db7ce7e 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -328,7 +328,7 @@ def _is_sqlite_db_path_relative(sqla_conn_str: str) -> bool:
def configure_orm(disable_connection_pool=False, pool_class=None):
"""Configure ORM using SQLAlchemy."""
- from airflow.utils.log.secrets_masker import mask_secret
+ from airflow.sdk.execution_time.secrets_masker import mask_secret
if _is_sqlite_db_path_relative(SQL_ALCHEMY_CONN):
from airflow.exceptions import AirflowConfigException
diff --git a/airflow/timetables/trigger.py b/airflow/timetables/trigger.py
index 4488a7fdaf619..bea972f4fb81f 100644
--- a/airflow/timetables/trigger.py
+++ b/airflow/timetables/trigger.py
@@ -17,6 +17,8 @@
from __future__ import annotations
import datetime
+import math
+import operator
from typing import TYPE_CHECKING, Any
from airflow.timetables._cron import CronMixin
@@ -31,6 +33,34 @@
from airflow.timetables.base import TimeRestriction
+def _serialize_interval(interval: datetime.timedelta | relativedelta) -> float | dict:
+ from airflow.serialization.serialized_objects import encode_relativedelta
+
+ if isinstance(interval, datetime.timedelta):
+ return interval.total_seconds()
+ return encode_relativedelta(interval)
+
+
+def _deserialize_interval(value: int | dict) -> datetime.timedelta | relativedelta:
+ from airflow.serialization.serialized_objects import decode_relativedelta
+
+ if isinstance(value, dict):
+ return decode_relativedelta(value)
+ return datetime.timedelta(seconds=value)
+
+
+def _serialize_run_immediately(value: bool | datetime.timedelta) -> bool | float:
+ if isinstance(value, datetime.timedelta):
+ return value.total_seconds()
+ return value
+
+
+def _deserialize_run_immediately(value: bool | float) -> bool | datetime.timedelta:
+ if isinstance(value, float):
+ return datetime.timedelta(seconds=value)
+ return value
+
+
class CronTriggerTimetable(CronMixin, Timetable):
"""
Timetable that triggers DAG runs according to a cron expression.
@@ -77,48 +107,23 @@ def __init__(
@classmethod
def deserialize(cls, data: dict[str, Any]) -> Timetable:
- from airflow.serialization.serialized_objects import decode_relativedelta, decode_timezone
-
- interval: datetime.timedelta | relativedelta
- if isinstance(data["interval"], dict):
- interval = decode_relativedelta(data["interval"])
- else:
- interval = datetime.timedelta(seconds=data["interval"])
-
- immediate: bool | datetime.timedelta
- if "immediate" not in data:
- immediate = False
- elif isinstance(data["immediate"], float):
- immediate = datetime.timedelta(seconds=data["interval"])
- else:
- immediate = data["immediate"]
+ from airflow.serialization.serialized_objects import decode_timezone
return cls(
data["expression"],
timezone=decode_timezone(data["timezone"]),
- interval=interval,
- run_immediately=immediate,
+ interval=_deserialize_interval(data["interval"]),
+ run_immediately=_deserialize_run_immediately(data.get("run_immediately", False)),
)
def serialize(self) -> dict[str, Any]:
- from airflow.serialization.serialized_objects import encode_relativedelta, encode_timezone
+ from airflow.serialization.serialized_objects import encode_timezone
- interval: float | dict[str, Any]
- if isinstance(self._interval, datetime.timedelta):
- interval = self._interval.total_seconds()
- else:
- interval = encode_relativedelta(self._interval)
- timezone = encode_timezone(self._timezone)
- immediate: bool | float
- if isinstance(self.run_immediately, datetime.timedelta):
- immediate = self.run_immediately.total_seconds()
- else:
- immediate = self.run_immediately
return {
"expression": self._expression,
- "timezone": timezone,
- "interval": interval,
- "run_immediately": immediate,
+ "timezone": encode_timezone(self._timezone),
+ "interval": _serialize_interval(self._interval),
+ "run_immediately": _serialize_run_immediately(self.run_immediately),
}
def infer_manual_data_interval(self, *, run_after: DateTime) -> DataInterval:
@@ -184,3 +189,95 @@ def _calc_first_run(self):
return past_run_time
else:
return next_run_time
+
+
+class MultipleCronTriggerTimetable(Timetable):
+ """
+ Timetable that triggers DAG runs according to multiple cron expressions.
+
+ This combines multiple ``CronTriggerTimetable`` instances underneath, and
+ triggers a DAG run whenever one of the timetables want to trigger a run.
+
+ Only at most one run is triggered for any given time, even if more than one
+ timetable fires at the same time.
+ """
+
+ def __init__(
+ self,
+ *crons: str,
+ timezone: str | Timezone | FixedTimezone,
+ interval: datetime.timedelta | relativedelta = datetime.timedelta(),
+ run_immediately: bool | datetime.timedelta = False,
+ ) -> None:
+ if not crons:
+ raise ValueError("cron expression required")
+ self._timetables = [
+ CronTriggerTimetable(cron, timezone=timezone, interval=interval, run_immediately=run_immediately)
+ for cron in crons
+ ]
+ self.description = ", ".join(t.description for t in self._timetables)
+
+ @classmethod
+ def deserialize(cls, data: dict[str, Any]) -> Timetable:
+ from airflow.serialization.serialized_objects import decode_timezone
+
+ return cls(
+ data["expressions"],
+ timezone=decode_timezone(data["timezone"]),
+ interval=_deserialize_interval(data["interval"]),
+ run_immediately=_deserialize_run_immediately(data["run_immediately"]),
+ )
+
+ def serialize(self) -> dict[str, Any]:
+ from airflow.serialization.serialized_objects import encode_timezone
+
+ # All timetables share the same timezone, interval, and run_immediately
+ # values, so we can just use the first to represent them.
+ timetable = self._timetables[0]
+ return {
+ "expressions": [t._expression for t in self._timetables],
+ "timezone": encode_timezone(timetable._timezone),
+ "interval": _serialize_interval(timetable._interval),
+ "run_immediately": _serialize_run_immediately(timetable.run_immediately),
+ }
+
+ @property
+ def summary(self) -> str:
+ return ", ".join(t.summary for t in self._timetables)
+
+ def infer_manual_data_interval(self, *, run_after: DateTime) -> DataInterval:
+ return min(
+ (t.infer_manual_data_interval(run_after=run_after) for t in self._timetables),
+ key=operator.attrgetter("start"),
+ )
+
+ def next_dagrun_info(
+ self,
+ *,
+ last_automated_data_interval: DataInterval | None,
+ restriction: TimeRestriction,
+ ) -> DagRunInfo | None:
+ infos = (
+ timetable.next_dagrun_info(
+ last_automated_data_interval=last_automated_data_interval,
+ restriction=restriction,
+ )
+ for timetable in self._timetables
+ )
+ return min(infos, key=self._dagrun_info_sort_key)
+
+ @staticmethod
+ def _dagrun_info_sort_key(info: DagRunInfo | None) -> float:
+ """
+ Sort key for DagRunInfo values.
+
+ This is passed as the sort key to ``min`` in ``next_dagrun_info`` to
+ find the next closest run, ordered by logical date.
+
+ The sort is done by simply returning the logical date converted to a
+ Unix timestamp. If the input is *None* (no next run), *inf* is returned
+ so it's selected last.
+ """
+ if info is None:
+ return math.inf
+ return info.logical_date.timestamp()
diff --git a/airflow/ui/openapi-gen/queries/common.ts b/airflow/ui/openapi-gen/queries/common.ts
index 7a6d7eb780b24..dcb8ec827d072 100644
--- a/airflow/ui/openapi-gen/queries/common.ts
+++ b/airflow/ui/openapi-gen/queries/common.ts
@@ -1715,6 +1715,9 @@ export type TaskInstanceServicePatchTaskInstanceDryRun1MutationResult = Awaited<
>;
export type PoolServicePatchPoolMutationResult = Awaited>;
export type PoolServiceBulkPoolsMutationResult = Awaited>;
+export type XcomServiceUpdateXcomEntryMutationResult = Awaited<
+ ReturnType
+>;
export type VariableServicePatchVariableMutationResult = Awaited<
ReturnType
>;
diff --git a/airflow/ui/openapi-gen/queries/queries.ts b/airflow/ui/openapi-gen/queries/queries.ts
index d838c9ccf5e5d..5a71695254bfa 100644
--- a/airflow/ui/openapi-gen/queries/queries.ts
+++ b/airflow/ui/openapi-gen/queries/queries.ts
@@ -52,6 +52,7 @@ import {
TriggerDAGRunPostBody,
VariableBody,
XComCreateBody,
+ XComUpdateBody,
} from "../requests/types.gen";
import * as Common from "./common";
@@ -3982,6 +3983,61 @@ export const usePoolServiceBulkPools = <
mutationFn: ({ requestBody }) => PoolService.bulkPools({ requestBody }) as unknown as Promise,
...options,
});
+/**
+ * Update Xcom Entry
+ * Update an existing XCom entry.
+ * @param data The data for the request.
+ * @param data.dagId
+ * @param data.taskId
+ * @param data.dagRunId
+ * @param data.xcomKey
+ * @param data.requestBody
+ * @returns XComResponseNative Successful Response
+ * @throws ApiError
+ */
+export const useXcomServiceUpdateXcomEntry = <
+ TData = Common.XcomServiceUpdateXcomEntryMutationResult,
+ TError = unknown,
+ TContext = unknown,
+>(
+ options?: Omit<
+ UseMutationOptions<
+ TData,
+ TError,
+ {
+ dagId: string;
+ dagRunId: string;
+ requestBody: XComUpdateBody;
+ taskId: string;
+ xcomKey: string;
+ },
+ TContext
+ >,
+ "mutationFn"
+ >,
+) =>
+ useMutation<
+ TData,
+ TError,
+ {
+ dagId: string;
+ dagRunId: string;
+ requestBody: XComUpdateBody;
+ taskId: string;
+ xcomKey: string;
+ },
+ TContext
+ >({
+ mutationFn: ({ dagId, dagRunId, requestBody, taskId, xcomKey }) =>
+ XcomService.updateXcomEntry({
+ dagId,
+ dagRunId,
+ requestBody,
+ taskId,
+ xcomKey,
+ }) as unknown as Promise,
+ ...options,
+ });
/**
* Patch Variable
* Update a variable by key.
diff --git a/airflow/ui/openapi-gen/requests/schemas.gen.ts b/airflow/ui/openapi-gen/requests/schemas.gen.ts
index e5ae3ca34878d..1d688c3442d27 100644
--- a/airflow/ui/openapi-gen/requests/schemas.gen.ts
+++ b/airflow/ui/openapi-gen/requests/schemas.gen.ts
@@ -4746,6 +4746,16 @@ export const $TaskInstanceHistoryResponse = {
type: "string",
title: "Executor Config",
},
+ dag_version: {
+ anyOf: [
+ {
+ $ref: "#/components/schemas/DagVersionResponse",
+ },
+ {
+ type: "null",
+ },
+ ],
+ },
},
type: "object",
required: [
@@ -4772,6 +4782,7 @@ export const $TaskInstanceHistoryResponse = {
"pid",
"executor",
"executor_config",
+ "dag_version",
],
title: "TaskInstanceHistoryResponse",
description: "TaskInstanceHistory serializer for responses.",
@@ -6227,3 +6238,21 @@ export const $XComResponseString = {
title: "XComResponseString",
description: "XCom response serializer with string return type.",
} as const;
+
+export const $XComUpdateBody = {
+ properties: {
+ value: {
+ title: "Value",
+ },
+ map_index: {
+ type: "integer",
+ title: "Map Index",
+ default: -1,
+ },
+ },
+ additionalProperties: false,
+ type: "object",
+ required: ["value"],
+ title: "XComUpdateBody",
+ description: "Payload serializer for updating an XCom entry.",
+} as const;
diff --git a/airflow/ui/openapi-gen/requests/services.gen.ts b/airflow/ui/openapi-gen/requests/services.gen.ts
index 69ab77337a1f9..585796aa2b2db 100644
--- a/airflow/ui/openapi-gen/requests/services.gen.ts
+++ b/airflow/ui/openapi-gen/requests/services.gen.ts
@@ -175,6 +175,8 @@ import type {
GetProvidersResponse,
GetXcomEntryData,
GetXcomEntryResponse,
+ UpdateXcomEntryData,
+ UpdateXcomEntryResponse,
GetXcomEntriesData,
GetXcomEntriesResponse,
CreateXcomEntryData,
@@ -2981,6 +2983,40 @@ export class XcomService {
});
}
+ /**
+ * Update Xcom Entry
+ * Update an existing XCom entry.
+ * @param data The data for the request.
+ * @param data.dagId
+ * @param data.taskId
+ * @param data.dagRunId
+ * @param data.xcomKey
+ * @param data.requestBody
+ * @returns XComResponseNative Successful Response
+ * @throws ApiError
+ */
+ public static updateXcomEntry(data: UpdateXcomEntryData): CancelablePromise {
+ return __request(OpenAPI, {
+ method: "PATCH",
+ url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances/{task_id}/xcomEntries/{xcom_key}",
+ path: {
+ dag_id: data.dagId,
+ task_id: data.taskId,
+ dag_run_id: data.dagRunId,
+ xcom_key: data.xcomKey,
+ },
+ body: data.requestBody,
+ mediaType: "application/json",
+ errors: {
+ 400: "Bad Request",
+ 401: "Unauthorized",
+ 403: "Forbidden",
+ 404: "Not Found",
+ 422: "Validation Error",
+ },
+ });
+ }
+
/**
* Get Xcom Entries
* Get all XCom entries.
diff --git a/airflow/ui/openapi-gen/requests/types.gen.ts b/airflow/ui/openapi-gen/requests/types.gen.ts
index 1283a77fe5548..a63ec17ffadcf 100644
--- a/airflow/ui/openapi-gen/requests/types.gen.ts
+++ b/airflow/ui/openapi-gen/requests/types.gen.ts
@@ -1239,6 +1239,7 @@ export type TaskInstanceHistoryResponse = {
pid: number | null;
executor: string | null;
executor_config: string;
+ dag_version: DagVersionResponse | null;
};
/**
@@ -1541,6 +1542,14 @@ export type XComResponseString = {
value: string | null;
};
+/**
+ * Payload serializer for updating an XCom entry.
+ */
+export type XComUpdateBody = {
+ value: unknown;
+ map_index?: number;
+};
+
export type NextRunAssetsData = {
dagId: string;
};
@@ -2319,6 +2328,16 @@ export type GetXcomEntryData = {
export type GetXcomEntryResponse = XComResponseNative | XComResponseString;
+export type UpdateXcomEntryData = {
+ dagId: string;
+ dagRunId: string;
+ requestBody: XComUpdateBody;
+ taskId: string;
+ xcomKey: string;
+};
+
+export type UpdateXcomEntryResponse = XComResponseNative;
+
export type GetXcomEntriesData = {
dagId: string;
dagRunId: string;
@@ -4720,6 +4739,35 @@ export type $OpenApiTs = {
422: HTTPValidationError;
};
};
+ patch: {
+ req: UpdateXcomEntryData;
+ res: {
+ /**
+ * Successful Response
+ */
+ 200: XComResponseNative;
+ /**
+ * Bad Request
+ */
+ 400: HTTPExceptionResponse;
+ /**
+ * Unauthorized
+ */
+ 401: HTTPExceptionResponse;
+ /**
+ * Forbidden
+ */
+ 403: HTTPExceptionResponse;
+ /**
+ * Not Found
+ */
+ 404: HTTPExceptionResponse;
+ /**
+ * Validation Error
+ */
+ 422: HTTPValidationError;
+ };
+ };
};
"/public/dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances/{task_id}/xcomEntries": {
get: {
diff --git a/airflow/ui/package.json b/airflow/ui/package.json
index 322623e9db68b..4a816fc06da45 100644
--- a/airflow/ui/package.json
+++ b/airflow/ui/package.json
@@ -82,7 +82,7 @@
"typescript-eslint": "^8.5.0",
"vite": "^5.4.12",
"vite-plugin-css-injected-by-js": "^3.5.2",
- "vitest": "^2.1.1",
+ "vitest": "^2.1.9",
"web-worker": "^1.3.0"
}
}
diff --git a/airflow/ui/pnpm-lock.yaml b/airflow/ui/pnpm-lock.yaml
index 20ae58594252c..20ac155c03cad 100644
--- a/airflow/ui/pnpm-lock.yaml
+++ b/airflow/ui/pnpm-lock.yaml
@@ -149,7 +149,7 @@ importers:
version: 3.7.0(@swc/helpers@0.5.13)(vite@5.4.12(@types/node@22.5.4))
'@vitest/coverage-v8':
specifier: ^2.1.1
- version: 2.1.1(vitest@2.1.1(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)))
+ version: 2.1.1(vitest@2.1.9(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)))
eslint:
specifier: ^9.10.0
version: 9.10.0(jiti@1.21.6)
@@ -202,8 +202,8 @@ importers:
specifier: ^3.5.2
version: 3.5.2(vite@5.4.12(@types/node@22.5.4))
vitest:
- specifier: ^2.1.1
- version: 2.1.1(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))
+ specifier: ^2.1.9
+ version: 2.1.9(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))
web-worker:
specifier: ^1.3.0
version: 1.3.0
@@ -1368,14 +1368,13 @@ packages:
'@vitest/browser':
optional: true
- '@vitest/expect@2.1.1':
- resolution: {integrity: sha512-YeueunS0HiHiQxk+KEOnq/QMzlUuOzbU1Go+PgAsHvvv3tUkJPm9xWt+6ITNTlzsMXUjmgm5T+U7KBPK2qQV6w==}
+ '@vitest/expect@2.1.9':
+ resolution: {integrity: sha512-UJCIkTBenHeKT1TTlKMJWy1laZewsRIzYighyYiJKZreqtdxSos/S1t+ktRMQWu2CKqaarrkeszJx1cgC5tGZw==}
- '@vitest/mocker@2.1.1':
- resolution: {integrity: sha512-LNN5VwOEdJqCmJ/2XJBywB11DLlkbY0ooDJW3uRX5cZyYCrc4PI/ePX0iQhE3BiEGiQmK4GE7Q/PqCkkaiPnrA==}
+ '@vitest/mocker@2.1.9':
+ resolution: {integrity: sha512-tVL6uJgoUdi6icpxmdrn5YNo3g3Dxv+IHJBr0GXHaEdTcw3F+cPKnsXFhli6nO+f/6SDKPHEK1UN+k+TQv0Ehg==}
peerDependencies:
- '@vitest/spy': 2.1.1
- msw: ^2.3.5
+ msw: ^2.4.9
vite: ^5.0.0
peerDependenciesMeta:
msw:
@@ -1383,20 +1382,20 @@ packages:
vite:
optional: true
- '@vitest/pretty-format@2.1.1':
- resolution: {integrity: sha512-SjxPFOtuINDUW8/UkElJYQSFtnWX7tMksSGW0vfjxMneFqxVr8YJ979QpMbDW7g+BIiq88RAGDjf7en6rvLPPQ==}
+ '@vitest/pretty-format@2.1.9':
+ resolution: {integrity: sha512-KhRIdGV2U9HOUzxfiHmY8IFHTdqtOhIzCpd8WRdJiE7D/HUcZVD0EgQCVjm+Q9gkUXWgBvMmTtZgIG48wq7sOQ==}
- '@vitest/runner@2.1.1':
- resolution: {integrity: sha512-uTPuY6PWOYitIkLPidaY5L3t0JJITdGTSwBtwMjKzo5O6RCOEncz9PUN+0pDidX8kTHYjO0EwUIvhlGpnGpxmA==}
+ '@vitest/runner@2.1.9':
+ resolution: {integrity: sha512-ZXSSqTFIrzduD63btIfEyOmNcBmQvgOVsPNPe0jYtESiXkhd8u2erDLnMxmGrDCwHCCHE7hxwRDCT3pt0esT4g==}
- '@vitest/snapshot@2.1.1':
- resolution: {integrity: sha512-BnSku1WFy7r4mm96ha2FzN99AZJgpZOWrAhtQfoxjUU5YMRpq1zmHRq7a5K9/NjqonebO7iVDla+VvZS8BOWMw==}
+ '@vitest/snapshot@2.1.9':
+ resolution: {integrity: sha512-oBO82rEjsxLNJincVhLhaxxZdEtV0EFHMK5Kmx5sJ6H9L183dHECjiefOAdnqpIgT5eZwT04PoggUnW88vOBNQ==}
- '@vitest/spy@2.1.1':
- resolution: {integrity: sha512-ZM39BnZ9t/xZ/nF4UwRH5il0Sw93QnZXd9NAZGRpIgj0yvVwPpLd702s/Cx955rGaMlyBQkZJ2Ir7qyY48VZ+g==}
+ '@vitest/spy@2.1.9':
+ resolution: {integrity: sha512-E1B35FwzXXTs9FHNK6bDszs7mtydNi5MIfUWpceJ8Xbfb1gBMscAnwLbEu+B44ed6W3XjL9/ehLPHR1fkf1KLQ==}
- '@vitest/utils@2.1.1':
- resolution: {integrity: sha512-Y6Q9TsI+qJ2CC0ZKj6VBb+T8UPz593N113nnUykqwANqhgf3QkZeHFlusgKLTqrnVHbj/XDKZcDHol+dxVT+rQ==}
+ '@vitest/utils@2.1.9':
+ resolution: {integrity: sha512-v0psaMSkNJ3A2NMrUEHFRzJtDPFn+/VWZ5WxImB21T9fjucJRmS7xCS3ppEnARb9y11OAzaD+P2Ps+b+BGX5iQ==}
'@xyflow/react@12.3.5':
resolution: {integrity: sha512-wAYqpicdrVo1rxCu0X3M9s3YIF45Agqfabw0IBryTGqjWvr2NyfciI8gIP4MB+NKpWWN5kxZ9tiZ9u8lwC7iAg==}
@@ -1798,8 +1797,8 @@ packages:
ccount@2.0.1:
resolution: {integrity: sha512-eyrF0jiFpY+3drT6383f1qhkbGsLSifNAjA61IUjZjmLCWjItY6LB9ft9YhoDgwfmclB2zhu51Lc7+95b8NRAg==}
- chai@5.1.1:
- resolution: {integrity: sha512-pT1ZgP8rPNqUgieVaEY+ryQr6Q4HXNg8Ei9UnLUrjN4IA7dvQC5JB+/kxVcPNDHyBcc/26CXPkbNzq3qwrOEKA==}
+ chai@5.1.2:
+ resolution: {integrity: sha512-aGtmf24DW6MLHHG5gCx4zaI3uBq3KRtxeVs0DjFH6Z0rDNbsvTxFASFvdj79pxjxZ8/5u3PIiN3IwEIQkiiuPw==}
engines: {node: '>=12'}
chakra-react-select@6.0.0-next.2:
@@ -2068,6 +2067,15 @@ packages:
supports-color:
optional: true
+ debug@4.4.0:
+ resolution: {integrity: sha512-6WTZ/IxCY/T6BALoZHaE4ctp9xm+Z5kY/pzYaCHRFeyVhojxlrm+46y68HA6hr0TcwEssoxNiDEUJQjfPZ/RYA==}
+ engines: {node: '>=6.0'}
+ peerDependencies:
+ supports-color: '*'
+ peerDependenciesMeta:
+ supports-color:
+ optional: true
+
decode-named-character-reference@1.0.2:
resolution: {integrity: sha512-O8x12RzrUF8xyVcY0KJowWsmaJxQbmy0/EtnNtHRpsOcT7dFk5W598coHqBVpmWo1oQQfsCqfCmkZN5DJrZVdg==}
@@ -2172,6 +2180,9 @@ packages:
resolution: {integrity: sha512-zoMwbCcH5hwUkKJkT8kDIBZSz9I6mVG//+lDCinLCGov4+r7NIy0ld8o03M0cJxl2spVf6ESYVS6/gpIfq1FFw==}
engines: {node: '>= 0.4'}
+ es-module-lexer@1.6.0:
+ resolution: {integrity: sha512-qqnD1yMU6tk/jnaMosogGySTZP8YtUgAffA9nMN+E/rjxcfRQ6IEk7IiozUjgxKoFHBGjTLnrHB/YC45r/59EQ==}
+
es-object-atoms@1.0.0:
resolution: {integrity: sha512-MZ4iQ6JwHOBQjahnjwaC1ZtIBH+2ohjamzAO3oaHcXYup7qxjF2fixyH+Q71voWHeOkI2q/TnJao/KfXYIZWbw==}
engines: {node: '>= 0.4'}
@@ -2328,6 +2339,10 @@ packages:
resolution: {integrity: sha512-VyhnebXciFV2DESc+p6B+y0LjSm0krU4OgJN44qFAhBY0TJ+1V61tYD2+wHusZ6F9n5K+vl8k0sTy7PEfV4qpg==}
engines: {node: '>=16.17'}
+ expect-type@1.1.0:
+ resolution: {integrity: sha512-bFi65yM+xZgk+u/KRIpekdSYkTB5W1pEf0Lt8Q8Msh7b+eQ7LXVtIB1Bkm4fvclDEL1b2CZkMhv2mOeF8tMdkA==}
+ engines: {node: '>=12.0.0'}
+
extend@3.0.2:
resolution: {integrity: sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==}
@@ -2447,9 +2462,6 @@ packages:
resolution: {integrity: sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==}
engines: {node: 6.* || 8.* || >= 10.*}
- get-func-name@2.0.2:
- resolution: {integrity: sha512-8vXOvuE167CtIc3OyItco7N/dpRtBbYOsPsXCz7X/PMnlGjYjSGuZJgM1Y7mmew7BKf9BqvLX2tnOVy1BBUsxQ==}
-
get-intrinsic@1.2.4:
resolution: {integrity: sha512-5uYhsJH8VJBTv7oslg4BznJYhDoRI6waYCxMmCdnTrcCrHA/fCFKoTFz2JKKE0HdDFUF7/oQuhzumXJK7paBRQ==}
engines: {node: '>= 0.4'}
@@ -2898,8 +2910,8 @@ packages:
resolution: {integrity: sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==}
hasBin: true
- loupe@3.1.1:
- resolution: {integrity: sha512-edNu/8D5MKVfGVFRhFf8aAxiTM6Wumfz5XsaatSxlD3w4R1d/WEKUTydCdPGbl9K7QG/Ca3GnDV2sIKIpXRQcw==}
+ loupe@3.1.3:
+ resolution: {integrity: sha512-kkIp7XSkP78ZxJEsSxW3712C6teJVoeHHwgo9zJ380de7IYyJ2ISlxojcH2pC5OFLewESmnRi/+XCDIEEVyoug==}
lowlight@1.20.0:
resolution: {integrity: sha512-8Ktj+prEb1RoCPkEOrPMYUN/nCggB7qAWe3a7OpMjWQkh3l2RD5wKRQ+o8Q8YuI9RG/xs95waaI/E6ym/7NsTw==}
@@ -2918,6 +2930,9 @@ packages:
magic-string@0.30.11:
resolution: {integrity: sha512-+Wri9p0QHMy+545hKww7YAu5NyzF8iomPL/RQazugQ9+Ez4Ic3mERMd8ZTX5rfK944j+560ZJi8iAwgak1Ac7A==}
+ magic-string@0.30.17:
+ resolution: {integrity: sha512-sNPKHvyjVf7gyjwS4xGTaW/mCnF8wnjtifKBEhxfZ7E/S8tQ0rssrwGNn6q8JH/ohItJfSQp9mBtQYuTlH5QnA==}
+
magicast@0.3.5:
resolution: {integrity: sha512-L0WhttDl+2BOsybvEOLK7fW3UA0OQ0IQ2d6Zl2x/a6vVRs3bAY0ECOSHHeL5jD+SbOpOCUEi0y1DgHEn9Qn1AQ==}
@@ -3698,6 +3713,9 @@ packages:
std-env@3.7.0:
resolution: {integrity: sha512-JPbdCEQLj1w5GilpiHAx3qJvFndqybBysA3qUOnznweH4QbNYUsW/ea8QzSrnh0vNsezMMw5bcVool8lM0gwzg==}
+ std-env@3.8.0:
+ resolution: {integrity: sha512-Bc3YwwCB+OzldMxOXJIIvC6cPRWr/LxOp48CdQTOkPyk/t4JWWJbrilwBd7RJzKV8QW7tJkcgAmeuLLJugl5/w==}
+
stop-iteration-iterator@1.0.0:
resolution: {integrity: sha512-iCGQj+0l0HOdZ2AEeBADlsRC+vsnDsZsbdSiH1yNSjcfKM7fdpCMfqAL/dwF5BLiw/XhRft/Wax6zQbhq2BcjQ==}
engines: {node: '>= 0.4'}
@@ -3799,11 +3817,11 @@ packages:
tinybench@2.9.0:
resolution: {integrity: sha512-0+DUvqWMValLmha6lr4kD8iAMK1HzV0/aKnCtWb9v9641TnP/MFb7Pc2bxoxQjTXAErryXVgUOfv2YqNllqGeg==}
- tinyexec@0.3.0:
- resolution: {integrity: sha512-tVGE0mVJPGb0chKhqmsoosjsS+qUnJVGJpZgsHYQcGoPlG3B51R3PouqTgEGH2Dc9jjFyOqOpix6ZHNMXp1FZg==}
+ tinyexec@0.3.2:
+ resolution: {integrity: sha512-KQQR9yN7R5+OSwaK0XQoj22pwHoTlgYqmUscPYoknOoWCWfj/5/ABTMRi69FrKU5ffPVh5QcFikpWJI/P1ocHA==}
- tinypool@1.0.1:
- resolution: {integrity: sha512-URZYihUbRPcGv95En+sz6MfghfIc2OJ1sv/RmhWZLouPY0/8Vo80viwPvg3dlaS9fuq7fQMEfgRRK7BBZThBEA==}
+ tinypool@1.0.2:
+ resolution: {integrity: sha512-al6n+QEANGFOMf/dmUMsuS5/r9B06uwlyNjZZql/zv8J7ybHCgoihBNORZCY2mzUuAnomQa2JdhyHKzZxPCrFA==}
engines: {node: ^18.0.0 || >=20.0.0}
tinyrainbow@1.2.0:
@@ -4005,8 +4023,8 @@ packages:
vfile@6.0.3:
resolution: {integrity: sha512-KzIbH/9tXat2u30jf+smMwFCsno4wHVdNmzFyL+T/L3UGqqk6JKfVqOFOZEpZSHADH1k40ab6NUIXZq422ov3Q==}
- vite-node@2.1.1:
- resolution: {integrity: sha512-N/mGckI1suG/5wQI35XeR9rsMsPqKXzq1CdUndzVstBj/HvyxxGctwnK6WX43NGt5L3Z5tcRf83g4TITKJhPrA==}
+ vite-node@2.1.9:
+ resolution: {integrity: sha512-AM9aQ/IPrW/6ENLQg3AGY4K1N2TGZdR5e4gu/MmmR2xR3Ll1+dib+nook92g4TV3PXVyeyxdWwtaCAiUL0hMxA==}
engines: {node: ^18.0.0 || >=20.0.0}
hasBin: true
@@ -4046,15 +4064,15 @@ packages:
terser:
optional: true
- vitest@2.1.1:
- resolution: {integrity: sha512-97We7/VC0e9X5zBVkvt7SGQMGrRtn3KtySFQG5fpaMlS+l62eeXRQO633AYhSTC3z7IMebnPPNjGXVGNRFlxBA==}
+ vitest@2.1.9:
+ resolution: {integrity: sha512-MSmPM9REYqDGBI8439mA4mWhV5sKmDlBKWIYbA3lRb2PTHACE0mgKwA8yQ2xq9vxDTuk4iPrECBAEW2aoFXY0Q==}
engines: {node: ^18.0.0 || >=20.0.0}
hasBin: true
peerDependencies:
'@edge-runtime/vm': '*'
'@types/node': ^18.0.0 || >=20.0.0
- '@vitest/browser': 2.1.1
- '@vitest/ui': 2.1.1
+ '@vitest/browser': 2.1.9
+ '@vitest/ui': 2.1.9
happy-dom: '*'
jsdom: '*'
peerDependenciesMeta:
@@ -4374,7 +4392,7 @@ snapshots:
'@babel/helper-split-export-declaration': 7.24.7
'@babel/parser': 7.25.4
'@babel/types': 7.25.4
- debug: 4.3.7
+ debug: 4.4.0
globals: 11.12.0
transitivePeerDependencies:
- supports-color
@@ -4386,7 +4404,7 @@ snapshots:
'@babel/parser': 7.25.6
'@babel/template': 7.25.0
'@babel/types': 7.25.6
- debug: 4.3.7
+ debug: 4.4.0
globals: 11.12.0
transitivePeerDependencies:
- supports-color
@@ -5182,7 +5200,7 @@ snapshots:
'@typescript-eslint/types': 8.5.0
'@typescript-eslint/typescript-estree': 8.5.0(typescript@5.5.4)
'@typescript-eslint/visitor-keys': 8.5.0
- debug: 4.3.7
+ debug: 4.4.0
eslint: 9.10.0(jiti@1.21.6)
optionalDependencies:
typescript: 5.5.4
@@ -5203,7 +5221,7 @@ snapshots:
dependencies:
'@typescript-eslint/typescript-estree': 8.5.0(typescript@5.5.4)
'@typescript-eslint/utils': 8.5.0(eslint@9.10.0(jiti@1.21.6))(typescript@5.5.4)
- debug: 4.3.7
+ debug: 4.4.0
ts-api-utils: 1.3.0(typescript@5.5.4)
optionalDependencies:
typescript: 5.5.4
@@ -5219,7 +5237,7 @@ snapshots:
dependencies:
'@typescript-eslint/types': 8.0.0-alpha.30
'@typescript-eslint/visitor-keys': 8.0.0-alpha.30
- debug: 4.3.6
+ debug: 4.4.0
globby: 11.1.0
is-glob: 4.0.3
minimatch: 9.0.5
@@ -5234,7 +5252,7 @@ snapshots:
dependencies:
'@typescript-eslint/types': 8.5.0
'@typescript-eslint/visitor-keys': 8.5.0
- debug: 4.3.7
+ debug: 4.4.0
fast-glob: 3.3.2
is-glob: 4.0.3
minimatch: 9.0.5
@@ -5696,7 +5714,7 @@ snapshots:
transitivePeerDependencies:
- '@swc/helpers'
- '@vitest/coverage-v8@2.1.1(vitest@2.1.1(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)))':
+ '@vitest/coverage-v8@2.1.1(vitest@2.1.9(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)))':
dependencies:
'@ampproject/remapping': 2.3.0
'@bcoe/v8-coverage': 0.2.3
@@ -5710,49 +5728,49 @@ snapshots:
std-env: 3.7.0
test-exclude: 7.0.1
tinyrainbow: 1.2.0
- vitest: 2.1.1(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))
+ vitest: 2.1.9(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))
transitivePeerDependencies:
- supports-color
- '@vitest/expect@2.1.1':
+ '@vitest/expect@2.1.9':
dependencies:
- '@vitest/spy': 2.1.1
- '@vitest/utils': 2.1.1
- chai: 5.1.1
+ '@vitest/spy': 2.1.9
+ '@vitest/utils': 2.1.9
+ chai: 5.1.2
tinyrainbow: 1.2.0
- '@vitest/mocker@2.1.1(@vitest/spy@2.1.1)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))(vite@5.4.12(@types/node@22.5.4))':
+ '@vitest/mocker@2.1.9(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))(vite@5.4.12(@types/node@22.5.4))':
dependencies:
- '@vitest/spy': 2.1.1
+ '@vitest/spy': 2.1.9
estree-walker: 3.0.3
- magic-string: 0.30.11
+ magic-string: 0.30.17
optionalDependencies:
msw: 2.7.0(@types/node@22.5.4)(typescript@5.5.4)
vite: 5.4.12(@types/node@22.5.4)
- '@vitest/pretty-format@2.1.1':
+ '@vitest/pretty-format@2.1.9':
dependencies:
tinyrainbow: 1.2.0
- '@vitest/runner@2.1.1':
+ '@vitest/runner@2.1.9':
dependencies:
- '@vitest/utils': 2.1.1
+ '@vitest/utils': 2.1.9
pathe: 1.1.2
- '@vitest/snapshot@2.1.1':
+ '@vitest/snapshot@2.1.9':
dependencies:
- '@vitest/pretty-format': 2.1.1
- magic-string: 0.30.11
+ '@vitest/pretty-format': 2.1.9
+ magic-string: 0.30.17
pathe: 1.1.2
- '@vitest/spy@2.1.1':
+ '@vitest/spy@2.1.9':
dependencies:
tinyspy: 3.0.2
- '@vitest/utils@2.1.1':
+ '@vitest/utils@2.1.9':
dependencies:
- '@vitest/pretty-format': 2.1.1
- loupe: 3.1.1
+ '@vitest/pretty-format': 2.1.9
+ loupe: 3.1.3
tinyrainbow: 1.2.0
'@xyflow/react@12.3.5(@types/react@18.3.5)(react-dom@18.3.1(react@18.3.1))(react@18.3.1)':
@@ -6474,12 +6492,12 @@ snapshots:
ccount@2.0.1: {}
- chai@5.1.1:
+ chai@5.1.2:
dependencies:
assertion-error: 2.0.1
check-error: 2.1.1
deep-eql: 5.0.2
- loupe: 3.1.1
+ loupe: 3.1.3
pathval: 2.0.0
chakra-react-select@6.0.0-next.2(@chakra-ui/react@3.1.1(@emotion/react@11.13.3(@types/react@18.3.5)(react@18.3.1))(react-dom@18.3.1(react@18.3.1))(react@18.3.1))(@types/react@18.3.5)(next-themes@0.3.0(react-dom@18.3.1(react@18.3.1))(react@18.3.1))(react-dom@18.3.1(react@18.3.1))(react@18.3.1):
@@ -6745,6 +6763,10 @@ snapshots:
dependencies:
ms: 2.1.3
+ debug@4.4.0:
+ dependencies:
+ ms: 2.1.3
+
decode-named-character-reference@1.0.2:
dependencies:
character-entities: 2.0.2
@@ -6921,6 +6943,8 @@ snapshots:
iterator.prototype: 1.1.2
safe-array-concat: 1.1.2
+ es-module-lexer@1.6.0: {}
+
es-object-atoms@1.0.0:
dependencies:
es-errors: 1.3.0
@@ -7155,6 +7179,8 @@ snapshots:
signal-exit: 4.1.0
strip-final-newline: 3.0.0
+ expect-type@1.1.0: {}
+
extend@3.0.2: {}
fast-deep-equal@3.1.3: {}
@@ -7287,8 +7313,6 @@ snapshots:
get-caller-file@2.0.5: {}
- get-func-name@2.0.2: {}
-
get-intrinsic@1.2.4:
dependencies:
es-errors: 1.3.0
@@ -7734,9 +7758,7 @@ snapshots:
dependencies:
js-tokens: 4.0.0
- loupe@3.1.1:
- dependencies:
- get-func-name: 2.0.2
+ loupe@3.1.3: {}
lowlight@1.20.0:
dependencies:
@@ -7753,6 +7775,10 @@ snapshots:
dependencies:
'@jridgewell/sourcemap-codec': 1.5.0
+ magic-string@0.30.17:
+ dependencies:
+ '@jridgewell/sourcemap-codec': 1.5.0
+
magicast@0.3.5:
dependencies:
'@babel/parser': 7.25.6
@@ -8096,7 +8122,7 @@ snapshots:
micromark@4.0.1:
dependencies:
'@types/debug': 4.1.12
- debug: 4.3.7
+ debug: 4.4.0
decode-named-character-reference: 1.0.2
devlop: 1.1.0
micromark-core-commonmark: 2.0.2
@@ -8817,6 +8843,8 @@ snapshots:
std-env@3.7.0: {}
+ std-env@3.8.0: {}
+
stop-iteration-iterator@1.0.0:
dependencies:
internal-slot: 1.0.7
@@ -8944,9 +8972,9 @@ snapshots:
tinybench@2.9.0: {}
- tinyexec@0.3.0: {}
+ tinyexec@0.3.2: {}
- tinypool@1.0.1: {}
+ tinypool@1.0.2: {}
tinyrainbow@1.2.0: {}
@@ -9154,10 +9182,11 @@ snapshots:
'@types/unist': 3.0.3
vfile-message: 4.0.2
- vite-node@2.1.1(@types/node@22.5.4):
+ vite-node@2.1.9(@types/node@22.5.4):
dependencies:
cac: 6.7.14
- debug: 4.3.7
+ debug: 4.4.0
+ es-module-lexer: 1.6.0
pathe: 1.1.2
vite: 5.4.12(@types/node@22.5.4)
transitivePeerDependencies:
@@ -9184,26 +9213,27 @@ snapshots:
'@types/node': 22.5.4
fsevents: 2.3.3
- vitest@2.1.1(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)):
- dependencies:
- '@vitest/expect': 2.1.1
- '@vitest/mocker': 2.1.1(@vitest/spy@2.1.1)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))(vite@5.4.12(@types/node@22.5.4))
- '@vitest/pretty-format': 2.1.1
- '@vitest/runner': 2.1.1
- '@vitest/snapshot': 2.1.1
- '@vitest/spy': 2.1.1
- '@vitest/utils': 2.1.1
- chai: 5.1.1
- debug: 4.3.7
- magic-string: 0.30.11
+ vitest@2.1.9(@types/node@22.5.4)(happy-dom@15.10.2)(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4)):
+ dependencies:
+ '@vitest/expect': 2.1.9
+ '@vitest/mocker': 2.1.9(msw@2.7.0(@types/node@22.5.4)(typescript@5.5.4))(vite@5.4.12(@types/node@22.5.4))
+ '@vitest/pretty-format': 2.1.9
+ '@vitest/runner': 2.1.9
+ '@vitest/snapshot': 2.1.9
+ '@vitest/spy': 2.1.9
+ '@vitest/utils': 2.1.9
+ chai: 5.1.2
+ debug: 4.4.0
+ expect-type: 1.1.0
+ magic-string: 0.30.17
pathe: 1.1.2
- std-env: 3.7.0
+ std-env: 3.8.0
tinybench: 2.9.0
- tinyexec: 0.3.0
- tinypool: 1.0.1
+ tinyexec: 0.3.2
+ tinypool: 1.0.2
tinyrainbow: 1.2.0
vite: 5.4.12(@types/node@22.5.4)
- vite-node: 2.1.1(@types/node@22.5.4)
+ vite-node: 2.1.9(@types/node@22.5.4)
why-is-node-running: 2.3.0
optionalDependencies:
'@types/node': 22.5.4
diff --git a/airflow/ui/src/components/SearchBar.tsx b/airflow/ui/src/components/SearchBar.tsx
index 349b66ce9e2e6..5a55b6a2f96c0 100644
--- a/airflow/ui/src/components/SearchBar.tsx
+++ b/airflow/ui/src/components/SearchBar.tsx
@@ -16,11 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-import { Button, Input, type ButtonProps } from "@chakra-ui/react";
-import { useState, type ChangeEvent } from "react";
+import { Button, Input, Kbd, type ButtonProps } from "@chakra-ui/react";
+import { useState, useRef, type ChangeEvent } from "react";
+import { useHotkeys } from "react-hotkeys-hook";
import { FiSearch } from "react-icons/fi";
import { useDebouncedCallback } from "use-debounce";
+import { getMetaKey } from "src/utils";
+
import { CloseButton, InputGroup, type InputGroupProps } from "./ui";
const debounceDelay = 200;
@@ -43,14 +46,23 @@ export const SearchBar = ({
placeHolder,
}: Props) => {
const handleSearchChange = useDebouncedCallback((val: string) => onChange(val), debounceDelay);
-
+ const searchRef = useRef(null);
const [value, setValue] = useState(defaultValue);
+ const metaKey = getMetaKey();
const onSearchChange = (event: ChangeEvent) => {
setValue(event.target.value);
handleSearchChange(event.target.value);
};
+ useHotkeys(
+ "mod+k",
+ () => {
+ searchRef.current?.focus();
+ },
+ { preventDefault: true },
+ );
+
return (
)}
+ {metaKey}+K
>
}
startElement={}
@@ -83,6 +96,7 @@ export const SearchBar = ({
onChange={onSearchChange}
placeholder={placeHolder}
pr={150}
+ ref={searchRef}
value={value}
/>
diff --git a/airflow/ui/src/constants/stateOptions.ts b/airflow/ui/src/constants/stateOptions.ts
new file mode 100644
index 0000000000000..cc85cc1115ee9
--- /dev/null
+++ b/airflow/ui/src/constants/stateOptions.ts
@@ -0,0 +1,53 @@
+/*!
+ * 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.
+ */
+import { createListCollection } from "@chakra-ui/react";
+
+import type { TaskInstanceState } from "openapi/requests/types.gen";
+
+export const taskInstanceStateOptions = createListCollection<{
+ label: string;
+ value: TaskInstanceState | "all" | "none";
+}>({
+ items: [
+ { label: "All States", value: "all" },
+ { label: "Scheduled", value: "scheduled" },
+ { label: "Queued", value: "queued" },
+ { label: "Running", value: "running" },
+ { label: "Success", value: "success" },
+ { label: "Restarting", value: "restarting" },
+ { label: "Failed", value: "failed" },
+ { label: "Up For Retry", value: "up_for_retry" },
+ { label: "Up For Reschedule", value: "up_for_reschedule" },
+ { label: "Upstream failed", value: "upstream_failed" },
+ { label: "Skipped", value: "skipped" },
+ { label: "Deferred", value: "deferred" },
+ { label: "Removed", value: "removed" },
+ { label: "No Status", value: "none" },
+ ],
+});
+
+export const dagRunStateOptions = createListCollection({
+ items: [
+ { label: "All States", value: "all" },
+ { label: "Queued", value: "queued" },
+ { label: "Running", value: "running" },
+ { label: "Failed", value: "failed" },
+ { label: "Success", value: "success" },
+ ],
+});
diff --git a/airflow/ui/src/pages/Dag/Runs/Runs.tsx b/airflow/ui/src/pages/Dag/Runs/Runs.tsx
index 29a1e90fd8616..36b6195eeea39 100644
--- a/airflow/ui/src/pages/Dag/Runs/Runs.tsx
+++ b/airflow/ui/src/pages/Dag/Runs/Runs.tsx
@@ -16,15 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-import {
- Box,
- createListCollection,
- Flex,
- HStack,
- Link,
- type SelectValueChangeDetails,
- Text,
-} from "@chakra-ui/react";
+import { Box, Flex, HStack, Link, type SelectValueChangeDetails, Text } from "@chakra-ui/react";
import type { ColumnDef } from "@tanstack/react-table";
import { useCallback } from "react";
import { useParams, Link as RouterLink, useSearchParams } from "react-router-dom";
@@ -40,6 +32,7 @@ import { RunTypeIcon } from "src/components/RunTypeIcon";
import { StateBadge } from "src/components/StateBadge";
import Time from "src/components/Time";
import { Select } from "src/components/ui";
+import { taskInstanceStateOptions as stateOptions } from "src/constants/stateOptions";
import { capitalize, getDuration, useAutoRefresh, isStatePending } from "src/utils";
const columns: Array> = [
@@ -105,16 +98,6 @@ const columns: Array> = [
},
];
-const stateOptions = createListCollection({
- items: [
- { label: "All States", value: "all" },
- { label: "Queued", value: "queued" },
- { label: "Running", value: "running" },
- { label: "Failed", value: "failed" },
- { label: "Success", value: "success" },
- ],
-});
-
const STATE_PARAM = "state";
export const Runs = () => {
diff --git a/airflow/ui/src/pages/Pools/Pools.tsx b/airflow/ui/src/pages/Pools/Pools.tsx
index 329d41484f7e8..163f1039719be 100644
--- a/airflow/ui/src/pages/Pools/Pools.tsx
+++ b/airflow/ui/src/pages/Pools/Pools.tsx
@@ -17,21 +17,50 @@
* under the License.
*/
import { Box, Skeleton } from "@chakra-ui/react";
+import { useState } from "react";
+import { useSearchParams } from "react-router-dom";
import { usePoolServiceGetPools } from "openapi/queries";
import { ErrorAlert } from "src/components/ErrorAlert";
+import { SearchBar } from "src/components/SearchBar";
+import { type SearchParamsKeysType, SearchParamsKeys } from "src/constants/searchParams";
import PoolBar from "./PoolBar";
export const Pools = () => {
- const { data, error, isLoading } = usePoolServiceGetPools();
+ const [searchParams, setSearchParams] = useSearchParams();
+ const { NAME_PATTERN: NAME_PATTERN_PARAM }: SearchParamsKeysType = SearchParamsKeys;
+ const [poolNamePattern, setPoolNamePattern] = useState(searchParams.get(NAME_PATTERN_PARAM) ?? undefined);
+ const { data, error, isLoading } = usePoolServiceGetPools({
+ poolNamePattern: poolNamePattern ?? undefined,
+ });
- return isLoading ? (
-
- ) : (
+ const handleSearchChange = (value: string) => {
+ if (value) {
+ searchParams.set(NAME_PATTERN_PARAM, value);
+ } else {
+ searchParams.delete(NAME_PATTERN_PARAM);
+ }
+ setSearchParams(searchParams);
+ setPoolNamePattern(value);
+ };
+
+ return (
<>
- {data?.pools.map((pool) => )}
+
+
+ {isLoading ? (
+
+ ) : (
+ data?.pools.map((pool) => )
+ )}
+
>
);
};
diff --git a/airflow/ui/src/pages/Run/Details.tsx b/airflow/ui/src/pages/Run/Details.tsx
index e46f7357a8002..a063b3a5e6b64 100644
--- a/airflow/ui/src/pages/Run/Details.tsx
+++ b/airflow/ui/src/pages/Run/Details.tsx
@@ -118,7 +118,7 @@ export const Details = () => {
{dagRun.external_trigger ? (
- Externally Trigger Source
+ External Trigger Source
{dagRun.triggered_by}
) : undefined}
diff --git a/airflow/ui/src/pages/Run/TaskInstances.tsx b/airflow/ui/src/pages/Run/TaskInstances.tsx
index 7ba8be6e5bfe1..f05a83e8c56b7 100644
--- a/airflow/ui/src/pages/Run/TaskInstances.tsx
+++ b/airflow/ui/src/pages/Run/TaskInstances.tsx
@@ -16,14 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-import {
- Box,
- Flex,
- Link,
- createListCollection,
- HStack,
- type SelectValueChangeDetails,
-} from "@chakra-ui/react";
+import { Box, Flex, Link, HStack, type SelectValueChangeDetails } from "@chakra-ui/react";
import type { ColumnDef } from "@tanstack/react-table";
import { useCallback, useState } from "react";
import { Link as RouterLink, useParams, useSearchParams } from "react-router-dom";
@@ -40,6 +33,7 @@ import { StateBadge } from "src/components/StateBadge";
import Time from "src/components/Time";
import { Select } from "src/components/ui";
import { SearchParamsKeys, type SearchParamsKeysType } from "src/constants/searchParams";
+import { taskInstanceStateOptions as stateOptions } from "src/constants/stateOptions";
import { capitalize, getDuration, useAutoRefresh, isStatePending } from "src/utils";
import { getTaskInstanceLink } from "src/utils/links";
@@ -109,25 +103,6 @@ const columns: Array> = [
},
];
-const stateOptions = createListCollection<{ label: string; value: TaskInstanceState | "all" | "none" }>({
- items: [
- { label: "All States", value: "all" },
- { label: "Scheduled", value: "scheduled" },
- { label: "Queued", value: "queued" },
- { label: "Running", value: "running" },
- { label: "Success", value: "success" },
- { label: "Restarting", value: "restarting" },
- { label: "Failed", value: "failed" },
- { label: "Up For Retry", value: "up_for_retry" },
- { label: "Up For Reschedule", value: "up_for_reschedule" },
- { label: "Upstream failed", value: "upstream_failed" },
- { label: "Skipped", value: "skipped" },
- { label: "Deferred", value: "deferred" },
- { label: "Removed", value: "removed" },
- { label: "No Status", value: "none" },
- ],
-});
-
const STATE_PARAM = "state";
export const TaskInstances = () => {
diff --git a/airflow/ui/src/pages/Task/Instances.tsx b/airflow/ui/src/pages/Task/Instances.tsx
index a45c7acebd156..912c588c93193 100644
--- a/airflow/ui/src/pages/Task/Instances.tsx
+++ b/airflow/ui/src/pages/Task/Instances.tsx
@@ -16,18 +16,21 @@
* specific language governing permissions and limitations
* under the License.
*/
-import { Box, Link } from "@chakra-ui/react";
+import { Box, Flex, HStack, Link, type SelectValueChangeDetails } from "@chakra-ui/react";
import type { ColumnDef } from "@tanstack/react-table";
-import { Link as RouterLink, useParams } from "react-router-dom";
+import { useCallback } from "react";
+import { Link as RouterLink, useParams, useSearchParams } from "react-router-dom";
import { useTaskInstanceServiceGetTaskInstances, useTaskServiceGetTask } from "openapi/queries";
-import type { TaskInstanceResponse } from "openapi/requests/types.gen";
+import type { TaskInstanceResponse, TaskInstanceState } from "openapi/requests/types.gen";
import { DataTable } from "src/components/DataTable";
import { useTableURLState } from "src/components/DataTable/useTableUrlState";
import { ErrorAlert } from "src/components/ErrorAlert";
import { StateBadge } from "src/components/StateBadge";
import Time from "src/components/Time";
-import { getDuration } from "src/utils";
+import { Select } from "src/components/ui";
+import { taskInstanceStateOptions as stateOptions } from "src/constants/stateOptions";
+import { capitalize, getDuration } from "src/utils";
import { getTaskInstanceLink } from "src/utils/links";
const columns = (isMapped?: boolean): Array> => [
@@ -79,26 +82,94 @@ const columns = (isMapped?: boolean): Array> =>
},
];
+const STATE_PARAM = "state";
+
export const Instances = () => {
const { dagId = "", taskId } = useParams();
+ const [searchParams, setSearchParams] = useSearchParams();
const { setTableURLState, tableURLState } = useTableURLState();
const { pagination, sorting } = tableURLState;
const [sort] = sorting;
const orderBy = sort ? `${sort.desc ? "-" : ""}${sort.id}` : "-start_date";
+ const filteredState = searchParams.getAll(STATE_PARAM);
+ const hasFilteredState = filteredState.length > 0;
const { data: task, error: taskError, isLoading: isTaskLoading } = useTaskServiceGetTask({ dagId, taskId });
+ const handleStateChange = useCallback(
+ ({ value }: SelectValueChangeDetails) => {
+ const [val, ...rest] = value;
+
+ if ((val === undefined || val === "all") && rest.length === 0) {
+ searchParams.delete(STATE_PARAM);
+ } else {
+ searchParams.delete(STATE_PARAM);
+ value.filter((state) => state !== "all").map((state) => searchParams.append(STATE_PARAM, state));
+ }
+ setTableURLState({
+ pagination: { ...pagination, pageIndex: 0 },
+ sorting,
+ });
+ setSearchParams(searchParams);
+ },
+ [pagination, searchParams, setSearchParams, setTableURLState, sorting],
+ );
+
const { data, error, isFetching, isLoading } = useTaskInstanceServiceGetTaskInstances({
dagId,
dagRunId: "~",
limit: pagination.pageSize,
offset: pagination.pageIndex * pagination.pageSize,
orderBy,
+ state: hasFilteredState ? filteredState : undefined,
taskId,
});
return (
-
+
+
+
+
+
+ {() =>
+ hasFilteredState ? (
+
+ {filteredState.map((state) => (
+
+ {state === "none" ? "No Status" : capitalize(state)}
+
+ ))}
+
+ ) : (
+ "All States"
+ )
+ }
+
+
+
+ {stateOptions.items.map((option) => (
+
+ {option.value === "all" ? (
+ option.label
+ ) : (
+ {option.label}
+ )}
+
+ ))}
+
+
+
+
{
const { dagId = "", runId = "", taskId = "" } = useParams();
const [searchParams, setSearchParams] = useSearchParams();
@@ -82,6 +85,13 @@ export const Details = () => {
taskInstance={taskInstance}
/>
)}
+
+ {taskInstance !== undefined && (taskInstance.trigger ?? taskInstance.triggerer_job) ? (
+
+ ) : undefined}
+
+ Task Instance Info
+
diff --git a/airflow/ui/src/pages/TaskInstance/ExtraLinks.tsx b/airflow/ui/src/pages/TaskInstance/ExtraLinks.tsx
new file mode 100644
index 0000000000000..5990c6fdb5f59
--- /dev/null
+++ b/airflow/ui/src/pages/TaskInstance/ExtraLinks.tsx
@@ -0,0 +1,53 @@
+/*!
+ * 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.
+ */
+import { Box, Button, Heading, HStack } from "@chakra-ui/react";
+import { useParams, useSearchParams } from "react-router-dom";
+
+import { useTaskInstanceServiceGetExtraLinks } from "openapi/queries";
+
+export const ExtraLinks = () => {
+ const { dagId = "", runId = "", taskId = "" } = useParams();
+ const [searchParams] = useSearchParams();
+ const mapIndexParam = searchParams.get("map_index");
+ const mapIndex = parseInt(mapIndexParam ?? "-1", 10);
+
+ const { data } = useTaskInstanceServiceGetExtraLinks({
+ dagId,
+ dagRunId: runId,
+ mapIndex,
+ taskId,
+ });
+
+ return data && Object.keys(data).length > 0 ? (
+
+ Extra Links
+
+ {Object.entries(data).map(([key, value], _) =>
+ value === null ? undefined : (
+
+ ),
+ )}
+
+
+ ) : undefined;
+};
diff --git a/airflow/ui/src/pages/TaskInstance/Header.tsx b/airflow/ui/src/pages/TaskInstance/Header.tsx
index a4a97fcbb1bd0..20e4066c3ad46 100644
--- a/airflow/ui/src/pages/TaskInstance/Header.tsx
+++ b/airflow/ui/src/pages/TaskInstance/Header.tsx
@@ -73,6 +73,9 @@ export const Header = ({
{getDuration(taskInstance.start_date, taskInstance.end_date)}s
+ {taskInstance.dag_version?.version_number !== undefined && (
+ {`v${taskInstance.dag_version.version_number}`}
+ )}
);
diff --git a/airflow/ui/src/pages/TaskInstance/TriggererInfo.tsx b/airflow/ui/src/pages/TaskInstance/TriggererInfo.tsx
new file mode 100644
index 0000000000000..cc718810d58f7
--- /dev/null
+++ b/airflow/ui/src/pages/TaskInstance/TriggererInfo.tsx
@@ -0,0 +1,58 @@
+/*!
+ * 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.
+ */
+import { Box, Table, Heading } from "@chakra-ui/react";
+
+import type { TaskInstanceResponse } from "openapi/requests/types.gen";
+import Time from "src/components/Time";
+
+export const TriggererInfo = ({ taskInstance }: { readonly taskInstance: TaskInstanceResponse }) => (
+
+
+ Triggerer Info
+
+
+
+
+ Trigger class
+ {taskInstance.trigger?.classpath}
+
+
+ Trigger ID
+ {taskInstance.trigger?.id}
+
+
+ Trigger creation time
+
+
+
+
+
+ Assigned triggerer
+ {taskInstance.triggerer_job?.hostname}
+
+
+ Latest triggerer heartbeat
+
+
+
+
+
+
+
+);
diff --git a/airflow/ui/src/queries/usePatchDagRun.ts b/airflow/ui/src/queries/usePatchDagRun.ts
index 1b42c374304b1..0ed16f5ecab33 100644
--- a/airflow/ui/src/queries/usePatchDagRun.ts
+++ b/airflow/ui/src/queries/usePatchDagRun.ts
@@ -22,9 +22,12 @@ import {
UseDagRunServiceGetDagRunKeyFn,
useDagRunServiceGetDagRunsKey,
useDagRunServicePatchDagRun,
+ useTaskInstanceServiceGetTaskInstancesKey,
} from "openapi/queries";
import { toaster } from "src/components/ui";
+import { useClearDagRunDryRunKey } from "./useClearDagRunDryRun";
+
const onError = () => {
toaster.create({
description: "Patch Dag Run request failed",
@@ -45,7 +48,12 @@ export const usePatchDagRun = ({
const queryClient = useQueryClient();
const onSuccessFn = async () => {
- const queryKeys = [UseDagRunServiceGetDagRunKeyFn({ dagId, dagRunId }), [useDagRunServiceGetDagRunsKey]];
+ const queryKeys = [
+ UseDagRunServiceGetDagRunKeyFn({ dagId, dagRunId }),
+ [useDagRunServiceGetDagRunsKey],
+ [useTaskInstanceServiceGetTaskInstancesKey, { dagId, dagRunId }],
+ [useClearDagRunDryRunKey, dagId],
+ ];
await Promise.all(queryKeys.map((key) => queryClient.invalidateQueries({ queryKey: key })));
diff --git a/airflow/ui/src/queries/useTrigger.ts b/airflow/ui/src/queries/useTrigger.ts
index 0a6e6f492abb9..2c56bda669539 100644
--- a/airflow/ui/src/queries/useTrigger.ts
+++ b/airflow/ui/src/queries/useTrigger.ts
@@ -20,15 +20,14 @@ import { useQueryClient } from "@tanstack/react-query";
import { useState } from "react";
import {
- useDagRunServiceGetDagRunsKey,
+ UseDagRunServiceGetDagRunsKeyFn,
useDagRunServiceTriggerDagRun,
useDagServiceGetDagsKey,
useDagsServiceRecentDagRunsKey,
- useTaskInstanceServiceGetTaskInstancesKey,
+ UseTaskInstanceServiceGetTaskInstancesKeyFn,
} from "openapi/queries";
import type { DagRunTriggerParams } from "src/components/TriggerDag/TriggerDAGForm";
import { toaster } from "src/components/ui";
-import { doQueryKeysMatch, type PartialQueryKey } from "src/utils";
export const useTrigger = ({ dagId, onSuccessConfirm }: { dagId: string; onSuccessConfirm: () => void }) => {
const queryClient = useQueryClient();
@@ -37,14 +36,14 @@ export const useTrigger = ({ dagId, onSuccessConfirm }: { dagId: string; onSucce
const [dateValidationError, setDateValidationError] = useState(undefined);
const onSuccess = async () => {
- const queryKeys: Array = [
- { baseKey: useDagServiceGetDagsKey },
- { baseKey: useDagsServiceRecentDagRunsKey },
- { baseKey: useDagRunServiceGetDagRunsKey, options: { dagIds: [dagId] } },
- { baseKey: useTaskInstanceServiceGetTaskInstancesKey, options: { dagId, dagRunId: "~" } },
+ const queryKeys = [
+ [useDagServiceGetDagsKey],
+ [useDagsServiceRecentDagRunsKey],
+ UseDagRunServiceGetDagRunsKeyFn({ dagId }, [{ dagId }]),
+ UseTaskInstanceServiceGetTaskInstancesKeyFn({ dagId, dagRunId: "~" }, [{ dagId, dagRunId: "~" }]),
];
- await queryClient.invalidateQueries({ predicate: (query) => doQueryKeysMatch(query, queryKeys) });
+ await Promise.all(queryKeys.map((key) => queryClient.invalidateQueries({ queryKey: key })));
toaster.create({
description: "DAG run has been successfully triggered.",
diff --git a/airflow/ui/src/utils/query.ts b/airflow/ui/src/utils/query.ts
index 415d1a5c9fa9b..095229d2dac59 100644
--- a/airflow/ui/src/utils/query.ts
+++ b/airflow/ui/src/utils/query.ts
@@ -16,8 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-import type { Query } from "@tanstack/react-query";
-
import { useDagServiceGetDagDetails } from "openapi/queries";
import type { TaskInstanceState } from "openapi/requests/types.gen";
import { useConfig } from "src/queries/useConfig";
@@ -32,26 +30,6 @@ export const isStatePending = (state?: TaskInstanceState | null) =>
state === "restarting" ||
!Boolean(state);
-export type PartialQueryKey = { baseKey: string; options?: Record };
-
-// This allows us to specify what query key values we actually care about and ignore the rest
-// ex: match everything with this dagId and dagRunId but ignore anything related to pagination
-export const doQueryKeysMatch = (query: Query, queryKeysToMatch: Array) => {
- const [baseKey, options] = query.queryKey;
-
- const matchedKey = queryKeysToMatch.find((qk) => qk.baseKey === baseKey);
-
- if (!matchedKey) {
- return false;
- }
-
- return matchedKey.options
- ? Object.entries(matchedKey.options).every(
- ([key, value]) => typeof options === "object" && (options as Record)[key] === value,
- )
- : true;
-};
-
export const useAutoRefresh = ({ dagId, isPaused }: { dagId?: string; isPaused?: boolean }) => {
const autoRefreshInterval = useConfig("auto_refresh_interval") as number | undefined;
const { data: dag } = useDagServiceGetDagDetails(
diff --git a/airflow/utils/cli.py b/airflow/utils/cli.py
index c1aba4da8f580..5c050c50a1dcb 100644
--- a/airflow/utils/cli.py
+++ b/airflow/utils/cli.py
@@ -35,9 +35,9 @@
from airflow import settings
from airflow.exceptions import AirflowException
+from airflow.sdk.execution_time.secrets_masker import should_hide_value_for_key
from airflow.utils import cli_action_loggers, timezone
from airflow.utils.log.non_caching_file_handler import NonCachingFileHandler
-from airflow.utils.log.secrets_masker import should_hide_value_for_key
from airflow.utils.platform import getuser, is_terminal_support_colors
T = TypeVar("T", bound=Callable)
diff --git a/airflow/utils/context.py b/airflow/utils/context.py
index 0415542c6ca8c..62308605dbb37 100644
--- a/airflow/utils/context.py
+++ b/airflow/utils/context.py
@@ -66,7 +66,6 @@
from sqlalchemy.orm import Session
from sqlalchemy.sql.expression import Select, TextClause
- from airflow.sdk.definitions.baseoperator import BaseOperator
from airflow.sdk.types import OutletEventAccessorsProtocol
# NOTE: Please keep this in sync with the following:
@@ -293,24 +292,6 @@ def context_merge(context: Context, *args: Any, **kwargs: Any) -> None:
context.update(*args, **kwargs)
-def context_update_for_unmapped(context: Context, task: BaseOperator) -> None:
- """
- Update context after task unmapping.
-
- Since ``get_template_context()`` is called before unmapping, the context
- contains information about the mapped task. We need to do some in-place
- updates to ensure the template context reflects the unmapped task instead.
-
- :meta private:
- """
- from airflow.sdk.definitions.param import process_params
-
- context["task"] = context["ti"].task = task
- context["params"] = process_params(
- context["dag"], task, context["dag_run"].conf, suppress_exception=False
- )
-
-
def context_copy_partial(source: Context, keys: Container[str]) -> Context:
"""
Create a context by copying items under selected keys in ``source``.
diff --git a/airflow/utils/setup_teardown.py b/airflow/utils/setup_teardown.py
index 3108657d30ac2..32d19d316844c 100644
--- a/airflow/utils/setup_teardown.py
+++ b/airflow/utils/setup_teardown.py
@@ -23,8 +23,8 @@
if TYPE_CHECKING:
from airflow.models.taskmixin import DependencyMixin
- from airflow.models.xcom_arg import PlainXComArg
from airflow.sdk.definitions._internal.abstractoperator import AbstractOperator
+ from airflow.sdk.definitions.xcom_arg import PlainXComArg
class BaseSetupTeardownContext:
@@ -335,7 +335,7 @@ class SetupTeardownContext(BaseSetupTeardownContext):
@staticmethod
def add_task(task: AbstractOperator | PlainXComArg):
"""Add task to context manager."""
- from airflow.models.xcom_arg import PlainXComArg
+ from airflow.sdk.definitions.xcom_arg import PlainXComArg
if not SetupTeardownContext.active:
raise AirflowException("Cannot add task to context outside the context manager.")
diff --git a/airflow/utils/task_group.py b/airflow/utils/task_group.py
index b74921f75d533..2ebb95709913c 100644
--- a/airflow/utils/task_group.py
+++ b/airflow/utils/task_group.py
@@ -19,61 +19,15 @@
from __future__ import annotations
-import functools
-import operator
-from collections.abc import Iterator
from typing import TYPE_CHECKING
import airflow.sdk.definitions.taskgroup
if TYPE_CHECKING:
- from sqlalchemy.orm import Session
-
from airflow.typing_compat import TypeAlias
TaskGroup: TypeAlias = airflow.sdk.definitions.taskgroup.TaskGroup
-
-
-class MappedTaskGroup(airflow.sdk.definitions.taskgroup.MappedTaskGroup): # noqa: D101
- # TODO: Rename this to SerializedMappedTaskGroup perhaps?
-
- def iter_mapped_task_groups(self) -> Iterator[MappedTaskGroup]:
- """
- Return mapped task groups in the hierarchy.
-
- Groups are returned from the closest to the outmost. If *self* is a
- mapped task group, it is returned first.
-
- :meta private:
- """
- group: TaskGroup | None = self
- while group is not None:
- if isinstance(group, MappedTaskGroup):
- yield group
- group = group.parent_group
-
- def get_mapped_ti_count(self, run_id: str, *, session: Session) -> int:
- """
- Return the number of instances a task in this group should be mapped to at run time.
-
- This considers both literal and non-literal mapped arguments, and the
- result is therefore available when all depended tasks have finished. The
- return value should be identical to ``parse_time_mapped_ti_count`` if
- all mapped arguments are literal.
-
- If this group is inside mapped task groups, all the nested counts are
- multiplied and accounted.
-
- :meta private:
-
- :raise NotFullyPopulated: If upstream tasks are not all complete yet.
- :return: Total number of mapped TIs this task should have.
- """
- groups = self.iter_mapped_task_groups()
- return functools.reduce(
- operator.mul,
- (g._expand_input.get_total_map_length(run_id, session=session) for g in groups),
- )
+MappedTaskGroup: TypeAlias = airflow.sdk.definitions.taskgroup.MappedTaskGroup
def task_group_to_dict(task_item_or_group):
diff --git a/airflow/www/decorators.py b/airflow/www/decorators.py
index 9a916da184c13..96a7f32e4c1a7 100644
--- a/airflow/www/decorators.py
+++ b/airflow/www/decorators.py
@@ -31,7 +31,7 @@
from airflow.api_fastapi.app import get_auth_manager
from airflow.models import Log
-from airflow.utils.log import secrets_masker
+from airflow.sdk.execution_time import secrets_masker
from airflow.utils.session import create_session
T = TypeVar("T", bound=Callable)
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 4ce5c6564619f..f99dcd9161f33 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -114,6 +114,7 @@
from airflow.plugins_manager import PLUGINS_ATTRIBUTES_TO_DUMP
from airflow.providers_manager import ProvidersManager
from airflow.sdk.definitions.asset import Asset, AssetAlias
+from airflow.sdk.execution_time import secrets_masker
from airflow.security import permissions
from airflow.ti_deps.dep_context import DepContext
from airflow.ti_deps.dependencies_deps import SCHEDULER_QUEUED_DEPS
@@ -127,7 +128,6 @@
from airflow.utils.db import get_query_count
from airflow.utils.docs import get_doc_url_for_provider, get_docs_url
from airflow.utils.helpers import exactly_one
-from airflow.utils.log import secrets_masker
from airflow.utils.log.log_reader import TaskLogReader
from airflow.utils.net import get_hostname
from airflow.utils.session import NEW_SESSION, create_session, provide_session
diff --git a/chart/templates/_helpers.yaml b/chart/templates/_helpers.yaml
index c76cefa843e34..718674f807441 100644
--- a/chart/templates/_helpers.yaml
+++ b/chart/templates/_helpers.yaml
@@ -620,8 +620,8 @@ server_tls_key_file = /etc/pgbouncer/server.key
{{/* Create the name of the API server service account to use */}}
-{{- define "apiServer.serviceAccountName" -}}
- {{- include "_serviceAccountName" (merge (dict "key" "apiServer" "nameSuffix" "api-server" ) .) -}}
+{{- define "_apiServer.serviceAccountName" -}}
+ {{- include "_serviceAccountName" (merge (dict "key" "_apiServer" "nameSuffix" "api-server" ) .) -}}
{{- end }}
{{/* Create the name of the redis service account to use */}}
diff --git a/chart/templates/api-server/api-server-deployment.yaml b/chart/templates/api-server/api-server-deployment.yaml
index b4cf2cd4461f1..3ce0fb9ac266f 100644
--- a/chart/templates/api-server/api-server-deployment.yaml
+++ b/chart/templates/api-server/api-server-deployment.yaml
@@ -21,15 +21,15 @@
## Airflow API Server Deployment
#################################
{{- if semverCompare ">=3.0.0" .Values.airflowVersion }}
-{{- $nodeSelector := or .Values.apiServer.nodeSelector .Values.nodeSelector }}
-{{- $affinity := or .Values.apiServer.affinity .Values.affinity }}
-{{- $tolerations := or .Values.apiServer.tolerations .Values.tolerations }}
-{{- $topologySpreadConstraints := or .Values.apiServer.topologySpreadConstraints .Values.topologySpreadConstraints }}
-{{- $revisionHistoryLimit := or .Values.apiServer.revisionHistoryLimit .Values.revisionHistoryLimit }}
-{{- $securityContext := include "airflowPodSecurityContext" (list . .Values.apiServer) }}
-{{- $containerSecurityContext := include "containerSecurityContext" (list . .Values.apiServer) }}
-{{- $containerSecurityContextWaitForMigrations := include "containerSecurityContext" (list . .Values.apiServer.waitForMigrations) }}
-{{- $containerLifecycleHooks := or .Values.apiServer.containerLifecycleHooks .Values.containerLifecycleHooks }}
+{{- $nodeSelector := or .Values._apiServer.nodeSelector .Values.nodeSelector }}
+{{- $affinity := or .Values._apiServer.affinity .Values.affinity }}
+{{- $tolerations := or .Values._apiServer.tolerations .Values.tolerations }}
+{{- $topologySpreadConstraints := or .Values._apiServer.topologySpreadConstraints .Values.topologySpreadConstraints }}
+{{- $revisionHistoryLimit := or .Values._apiServer.revisionHistoryLimit .Values.revisionHistoryLimit }}
+{{- $securityContext := include "airflowPodSecurityContext" (list . .Values._apiServer) }}
+{{- $containerSecurityContext := include "containerSecurityContext" (list . .Values._apiServer) }}
+{{- $containerSecurityContextWaitForMigrations := include "containerSecurityContext" (list . .Values._apiServer.waitForMigrations) }}
+{{- $containerLifecycleHooks := or .Values._apiServer.containerLifecycleHooks .Values.containerLifecycleHooks }}
apiVersion: apps/v1
kind: Deployment
metadata:
@@ -43,17 +43,17 @@ metadata:
{{- with .Values.labels }}
{{- toYaml . | nindent 4 }}
{{- end }}
- {{- if .Values.apiServer.annotations }}
- annotations: {{- toYaml .Values.apiServer.annotations | nindent 4 }}
+ {{- if .Values._apiServer.annotations }}
+ annotations: {{- toYaml .Values._apiServer.annotations | nindent 4 }}
{{- end }}
spec:
- replicas: {{ .Values.apiServer.replicas }}
+ replicas: {{ .Values._apiServer.replicas }}
{{- if $revisionHistoryLimit }}
revisionHistoryLimit: {{ $revisionHistoryLimit }}
{{- end }}
strategy:
- {{- if .Values.apiServer.strategy }}
- {{- toYaml .Values.apiServer.strategy | nindent 4 }}
+ {{- if .Values._apiServer.strategy }}
+ {{- toYaml .Values._apiServer.strategy | nindent 4 }}
{{- else }}
# Here we define the rolling update strategy
# - maxSurge define how many pod we can add at a time
@@ -78,8 +78,8 @@ spec:
tier: airflow
component: api-server
release: {{ .Release.Name }}
- {{- if or (.Values.labels) (.Values.apiServer.labels) }}
- {{- mustMerge .Values.apiServer.labels .Values.labels | toYaml | nindent 8 }}
+ {{- if or (.Values.labels) (.Values._apiServer.labels) }}
+ {{- mustMerge .Values._apiServer.labels .Values.labels | toYaml | nindent 8 }}
{{- end }}
annotations:
checksum/metadata-secret: {{ include (print $.Template.BasePath "/secrets/metadata-connection-secret.yaml") . | sha256sum }}
@@ -90,16 +90,16 @@ spec:
{{- if .Values.airflowPodAnnotations }}
{{- toYaml .Values.airflowPodAnnotations | nindent 8 }}
{{- end }}
- {{- if .Values.apiServer.podAnnotations }}
- {{- toYaml .Values.apiServer.podAnnotations | nindent 8 }}
+ {{- if .Values._apiServer.podAnnotations }}
+ {{- toYaml .Values._apiServer.podAnnotations | nindent 8 }}
{{- end }}
spec:
- {{- if .Values.apiServer.hostAliases }}
- hostAliases: {{- toYaml .Values.apiServer.hostAliases | nindent 8 }}
+ {{- if .Values._apiServer.hostAliases }}
+ hostAliases: {{- toYaml .Values._apiServer.hostAliases | nindent 8 }}
{{- end }}
- serviceAccountName: {{ include "apiServer.serviceAccountName" . }}
- {{- if .Values.apiServer.priorityClassName }}
- priorityClassName: {{ .Values.apiServer.priorityClassName }}
+ serviceAccountName: {{ include "_apiServer.serviceAccountName" . }}
+ {{- if .Values._apiServer.priorityClassName }}
+ priorityClassName: {{ .Values._apiServer.priorityClassName }}
{{- end }}
{{- if .Values.schedulerName }}
schedulerName: {{ .Values.schedulerName }}
@@ -127,9 +127,9 @@ spec:
- name: {{ template "registry_secret" . }}
{{- end }}
initContainers:
- {{- if .Values.apiServer.waitForMigrations.enabled }}
+ {{- if .Values._apiServer.waitForMigrations.enabled }}
- name: wait-for-airflow-migrations
- resources: {{- toYaml .Values.apiServer.resources | nindent 12 }}
+ resources: {{- toYaml .Values._apiServer.resources | nindent 12 }}
image: {{ template "airflow_image_for_migrations" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
securityContext: {{ $containerSecurityContextWaitForMigrations | nindent 12 }}
@@ -138,20 +138,20 @@ spec:
{{- if .Values.volumeMounts }}
{{- toYaml .Values.volumeMounts | nindent 12 }}
{{- end }}
- {{- if .Values.apiServer.extraVolumeMounts }}
- {{- tpl (toYaml .Values.apiServer.extraVolumeMounts) . | nindent 12 }}
+ {{- if .Values._apiServer.extraVolumeMounts }}
+ {{- tpl (toYaml .Values._apiServer.extraVolumeMounts) . | nindent 12 }}
{{- end }}
args: {{- include "wait-for-migrations-command" . | indent 10 }}
envFrom: {{- include "custom_airflow_environment_from" . | default "\n []" | indent 10 }}
env:
{{- include "custom_airflow_environment" . | indent 10 }}
{{- include "standard_airflow_environment" . | indent 10 }}
- {{- if .Values.apiServer.waitForMigrations.env }}
- {{- tpl (toYaml .Values.apiServer.waitForMigrations.env) $ | nindent 12 }}
+ {{- if .Values._apiServer.waitForMigrations.env }}
+ {{- tpl (toYaml .Values._apiServer.waitForMigrations.env) $ | nindent 12 }}
{{- end }}
{{- end }}
- {{- if .Values.apiServer.extraInitContainers }}
- {{- toYaml .Values.apiServer.extraInitContainers | nindent 8 }}
+ {{- if .Values._apiServer.extraInitContainers }}
+ {{- toYaml .Values._apiServer.extraInitContainers | nindent 8 }}
{{- end }}
containers:
- name: api-server
@@ -161,13 +161,13 @@ spec:
{{- if $containerLifecycleHooks }}
lifecycle: {{- tpl (toYaml $containerLifecycleHooks) . | nindent 12 }}
{{- end }}
- {{- if .Values.apiServer.command }}
- command: {{ tpl (toYaml .Values.apiServer.command) . | nindent 12 }}
+ {{- if .Values._apiServer.command }}
+ command: {{ tpl (toYaml .Values._apiServer.command) . | nindent 12 }}
{{- end }}
- {{- if .Values.apiServer.args }}
- args: {{- tpl (toYaml .Values.apiServer.args) . | nindent 12 }}
+ {{- if .Values._apiServer.args }}
+ args: {{- tpl (toYaml .Values._apiServer.args) . | nindent 12 }}
{{- end }}
- resources: {{- toYaml .Values.apiServer.resources | nindent 12 }}
+ resources: {{- toYaml .Values._apiServer.resources | nindent 12 }}
volumeMounts:
{{- include "airflow_config_mount" . | nindent 12 }}
{{- if .Values.logs.persistence.enabled }}
@@ -177,48 +177,48 @@ spec:
{{- if .Values.volumeMounts }}
{{- toYaml .Values.volumeMounts | nindent 12 }}
{{- end }}
- {{- if .Values.apiServer.extraVolumeMounts }}
- {{- tpl (toYaml .Values.apiServer.extraVolumeMounts) . | nindent 12 }}
+ {{- if .Values._apiServer.extraVolumeMounts }}
+ {{- tpl (toYaml .Values._apiServer.extraVolumeMounts) . | nindent 12 }}
{{- end }}
ports:
- name: api-server
- containerPort: {{ .Values.ports.apiServer }}
+ containerPort: {{ .Values.ports._apiServer }}
livenessProbe:
httpGet:
path: /public/version
- port: {{ .Values.ports.apiServer }}
- scheme: {{ .Values.apiServer.livenessProbe.scheme | default "http" }}
- initialDelaySeconds: {{ .Values.apiServer.livenessProbe.initialDelaySeconds }}
- timeoutSeconds: {{ .Values.apiServer.livenessProbe.timeoutSeconds }}
- failureThreshold: {{ .Values.apiServer.livenessProbe.failureThreshold }}
- periodSeconds: {{ .Values.apiServer.livenessProbe.periodSeconds }}
+ port: {{ .Values.ports._apiServer }}
+ scheme: {{ .Values._apiServer.livenessProbe.scheme | default "http" }}
+ initialDelaySeconds: {{ .Values._apiServer.livenessProbe.initialDelaySeconds }}
+ timeoutSeconds: {{ .Values._apiServer.livenessProbe.timeoutSeconds }}
+ failureThreshold: {{ .Values._apiServer.livenessProbe.failureThreshold }}
+ periodSeconds: {{ .Values._apiServer.livenessProbe.periodSeconds }}
readinessProbe:
httpGet:
path: /public/version
- port: {{ .Values.ports.apiServer }}
- scheme: {{ .Values.apiServer.readinessProbe.scheme | default "http" }}
- initialDelaySeconds: {{ .Values.apiServer.readinessProbe.initialDelaySeconds }}
- timeoutSeconds: {{ .Values.apiServer.readinessProbe.timeoutSeconds }}
- failureThreshold: {{ .Values.apiServer.readinessProbe.failureThreshold }}
- periodSeconds: {{ .Values.apiServer.readinessProbe.periodSeconds }}
+ port: {{ .Values.ports._apiServer }}
+ scheme: {{ .Values._apiServer.readinessProbe.scheme | default "http" }}
+ initialDelaySeconds: {{ .Values._apiServer.readinessProbe.initialDelaySeconds }}
+ timeoutSeconds: {{ .Values._apiServer.readinessProbe.timeoutSeconds }}
+ failureThreshold: {{ .Values._apiServer.readinessProbe.failureThreshold }}
+ periodSeconds: {{ .Values._apiServer.readinessProbe.periodSeconds }}
startupProbe:
httpGet:
path: /public/version
- port: {{ .Values.ports.apiServer }}
- scheme: {{ .Values.apiServer.startupProbe.scheme | default "http" }}
- timeoutSeconds: {{ .Values.apiServer.startupProbe.timeoutSeconds }}
- failureThreshold: {{ .Values.apiServer.startupProbe.failureThreshold }}
- periodSeconds: {{ .Values.apiServer.startupProbe.periodSeconds }}
+ port: {{ .Values.ports._apiServer }}
+ scheme: {{ .Values._apiServer.startupProbe.scheme | default "http" }}
+ timeoutSeconds: {{ .Values._apiServer.startupProbe.timeoutSeconds }}
+ failureThreshold: {{ .Values._apiServer.startupProbe.failureThreshold }}
+ periodSeconds: {{ .Values._apiServer.startupProbe.periodSeconds }}
envFrom: {{- include "custom_airflow_environment_from" . | default "\n []" | indent 10 }}
env:
{{- include "custom_airflow_environment" . | indent 10 }}
{{- include "standard_airflow_environment" . | indent 10 }}
- {{- include "container_extra_envs" (list . .Values.apiServer.env) | indent 10 }}
+ {{- include "container_extra_envs" (list . .Values._apiServer.env) | indent 10 }}
{{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) (semverCompare "<2.0.0" .Values.airflowVersion) }}
{{- include "git_sync_container" . | nindent 8 }}
{{- end }}
- {{- if .Values.apiServer.extraContainers }}
- {{- tpl (toYaml .Values.apiServer.extraContainers) . | nindent 8 }}
+ {{- if .Values._apiServer.extraContainers }}
+ {{- tpl (toYaml .Values._apiServer.extraContainers) . | nindent 8 }}
{{- end }}
volumes:
- name: config
@@ -234,7 +234,7 @@ spec:
{{- if .Values.volumes }}
{{- toYaml .Values.volumes | nindent 8 }}
{{- end }}
- {{- if .Values.apiServer.extraVolumes }}
- {{- tpl (toYaml .Values.apiServer.extraVolumes) . | nindent 8 }}
+ {{- if .Values._apiServer.extraVolumes }}
+ {{- tpl (toYaml .Values._apiServer.extraVolumes) . | nindent 8 }}
{{- end }}
{{- end }}
diff --git a/chart/templates/api-server/api-server-networkpolicy.yaml b/chart/templates/api-server/api-server-networkpolicy.yaml
index af4601811200e..d648ea4baa25b 100644
--- a/chart/templates/api-server/api-server-networkpolicy.yaml
+++ b/chart/templates/api-server/api-server-networkpolicy.yaml
@@ -32,8 +32,8 @@ metadata:
release: {{ .Release.Name }}
chart: "{{ .Chart.Name }}-{{ .Chart.Version }}"
heritage: {{ .Release.Service }}
- {{- if or (.Values.labels) (.Values.apiServer.labels) }}
- {{- mustMerge .Values.apiServer.labels .Values.labels | toYaml | nindent 4 }}
+ {{- if or (.Values.labels) (.Values._apiServer.labels) }}
+ {{- mustMerge .Values._apiServer.labels .Values.labels | toYaml | nindent 4 }}
{{- end }}
spec:
podSelector:
@@ -43,11 +43,11 @@ spec:
release: {{ .Release.Name }}
policyTypes:
- Ingress
- {{- if .Values.apiServer.networkPolicy.ingress.from }}
+ {{- if .Values._apiServer.networkPolicy.ingress.from }}
ingress:
- - from: {{- toYaml .Values.apiServer.networkPolicy.ingress.from | nindent 6 }}
+ - from: {{- toYaml .Values._apiServer.networkPolicy.ingress.from | nindent 6 }}
ports:
- {{ range .Values.apiServer.networkPolicy.ingress.ports }}
+ {{ range .Values._apiServer.networkPolicy.ingress.ports }}
-
{{- range $key, $val := . }}
{{ $key }}: {{ tpl (toString $val) $ }}
diff --git a/chart/templates/api-server/api-server-poddisruptionbudget.yaml b/chart/templates/api-server/api-server-poddisruptionbudget.yaml
index 7d0b162e41ea9..c8d9249e4acef 100644
--- a/chart/templates/api-server/api-server-poddisruptionbudget.yaml
+++ b/chart/templates/api-server/api-server-poddisruptionbudget.yaml
@@ -21,7 +21,7 @@
## Airflow api-server PodDisruptionBudget
#################################
{{- if semverCompare ">=3.0.0" .Values.airflowVersion }}
-{{- if .Values.apiServer.podDisruptionBudget.enabled }}
+{{- if .Values._apiServer.podDisruptionBudget.enabled }}
apiVersion: policy/v1
kind: PodDisruptionBudget
metadata:
@@ -32,8 +32,8 @@ metadata:
release: {{ .Release.Name }}
chart: {{ .Chart.Name }}
heritage: {{ .Release.Service }}
- {{- if or (.Values.labels) (.Values.apiServer.labels) }}
- {{- mustMerge .Values.apiServer.labels .Values.labels | toYaml | nindent 4 }}
+ {{- if or (.Values.labels) (.Values._apiServer.labels) }}
+ {{- mustMerge .Values._apiServer.labels .Values.labels | toYaml | nindent 4 }}
{{- end }}
spec:
selector:
@@ -41,6 +41,6 @@ spec:
tier: airflow
component: api-server
release: {{ .Release.Name }}
- {{- toYaml .Values.apiServer.podDisruptionBudget.config | nindent 2 }}
+ {{- toYaml .Values._apiServer.podDisruptionBudget.config | nindent 2 }}
{{- end }}
{{- end }}
diff --git a/chart/templates/api-server/api-server-service.yaml b/chart/templates/api-server/api-server-service.yaml
index 0a360aee08539..71ad37f2ff7a9 100644
--- a/chart/templates/api-server/api-server-service.yaml
+++ b/chart/templates/api-server/api-server-service.yaml
@@ -31,29 +31,29 @@ metadata:
release: {{ .Release.Name }}
chart: "{{ .Chart.Name }}-{{ .Chart.Version }}"
heritage: {{ .Release.Service }}
- {{- if or (.Values.labels) (.Values.apiServer.labels) }}
- {{- mustMerge .Values.apiServer.labels .Values.labels | toYaml | nindent 4 }}
+ {{- if or (.Values.labels) (.Values._apiServer.labels) }}
+ {{- mustMerge .Values._apiServer.labels .Values.labels | toYaml | nindent 4 }}
{{- end }}
- {{- with .Values.apiServer.service.annotations }}
+ {{- with .Values._apiServer.service.annotations }}
annotations: {{- toYaml . | nindent 4 }}
{{- end }}
spec:
- type: {{ .Values.apiServer.service.type }}
+ type: {{ .Values._apiServer.service.type }}
selector:
tier: airflow
component: api-server
release: {{ .Release.Name }}
ports:
- {{ range .Values.apiServer.service.ports }}
+ {{ range .Values._apiServer.service.ports }}
-
{{- range $key, $val := . }}
{{ $key }}: {{ tpl (toString $val) $ }}
{{- end }}
{{- end }}
- {{- if .Values.apiServer.service.loadBalancerIP }}
- loadBalancerIP: {{ .Values.apiServer.service.loadBalancerIP }}
+ {{- if .Values._apiServer.service.loadBalancerIP }}
+ loadBalancerIP: {{ .Values._apiServer.service.loadBalancerIP }}
{{- end }}
- {{- if .Values.apiServer.service.loadBalancerSourceRanges }}
- loadBalancerSourceRanges: {{- toYaml .Values.apiServer.service.loadBalancerSourceRanges | nindent 4 }}
+ {{- if .Values._apiServer.service.loadBalancerSourceRanges }}
+ loadBalancerSourceRanges: {{- toYaml .Values._apiServer.service.loadBalancerSourceRanges | nindent 4 }}
{{- end }}
{{- end }}
diff --git a/chart/templates/api-server/api-server-serviceaccount.yaml b/chart/templates/api-server/api-server-serviceaccount.yaml
index 3b864d01602fa..b797a7caadf2c 100644
--- a/chart/templates/api-server/api-server-serviceaccount.yaml
+++ b/chart/templates/api-server/api-server-serviceaccount.yaml
@@ -20,22 +20,22 @@
######################################
## Airflow api-server ServiceAccount
######################################
-{{- if and .Values.apiServer.serviceAccount.create (semverCompare ">=3.0.0" .Values.airflowVersion) }}
+{{- if and .Values._apiServer.serviceAccount.create (semverCompare ">=3.0.0" .Values.airflowVersion) }}
apiVersion: v1
kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.apiServer.serviceAccount.automountServiceAccountToken }}
+automountServiceAccountToken: {{ .Values._apiServer.serviceAccount.automountServiceAccountToken }}
metadata:
- name: {{ include "apiServer.serviceAccountName" . }}
+ name: {{ include "_apiServer.serviceAccountName" . }}
labels:
tier: airflow
component: api-server
release: {{ .Release.Name }}
chart: "{{ .Chart.Name }}-{{ .Chart.Version }}"
heritage: {{ .Release.Service }}
- {{- if or (.Values.labels) (.Values.apiServer.labels) }}
- {{- mustMerge .Values.apiServer.labels .Values.labels | toYaml | nindent 4 }}
+ {{- if or (.Values.labels) (.Values._apiServer.labels) }}
+ {{- mustMerge .Values._apiServer.labels .Values.labels | toYaml | nindent 4 }}
{{- end }}
- {{- with .Values.apiServer.serviceAccount.annotations }}
+ {{- with .Values._apiServer.serviceAccount.annotations }}
annotations: {{- toYaml . | nindent 4 }}
{{- end }}
{{- end }}
diff --git a/chart/templates/configmaps/configmap.yaml b/chart/templates/configmaps/configmap.yaml
index 119e43cce0a5e..ab4f13d918ad4 100644
--- a/chart/templates/configmaps/configmap.yaml
+++ b/chart/templates/configmaps/configmap.yaml
@@ -42,7 +42,7 @@ data:
{{- if semverCompare ">=3.0.0" .Values.airflowVersion -}}
{{- $config := merge .Values.config ( dict "workers" dict )}}
{{- if not (hasKey $config.workers "execution_api_server_url") -}}
- {{- $_ := set $config.workers "execution_api_server_url" (printf "http://%s-api-server:%d/execution/" (include "airflow.fullname" .) (int .Values.ports.apiServer)) -}}
+ {{- $_ := set $config.workers "execution_api_server_url" (printf "http://%s-api-server:%d/execution/" (include "airflow.fullname" .) (int .Values.ports._apiServer)) -}}
{{- end -}}
{{- end -}}
# These are system-specified config overrides.
diff --git a/chart/templates/scheduler/scheduler-serviceaccount.yaml b/chart/templates/scheduler/scheduler-serviceaccount.yaml
index 641fdb82a5e7a..0f4f8cfaa67e0 100644
--- a/chart/templates/scheduler/scheduler-serviceaccount.yaml
+++ b/chart/templates/scheduler/scheduler-serviceaccount.yaml
@@ -23,7 +23,7 @@
{{- if and .Values.scheduler.enabled .Values.scheduler.serviceAccount.create }}
apiVersion: v1
kind: ServiceAccount
-{{- if contains "CeleryExecutor" .Values.executor }}
+{{- if eq .Values.executor "CeleryExecutor" }}
automountServiceAccountToken: {{ .Values.scheduler.serviceAccount.automountServiceAccountToken }}
{{- end }}
metadata:
diff --git a/chart/values.schema.json b/chart/values.schema.json
index 036dc63617a52..4564f43dd26fd 100644
--- a/chart/values.schema.json
+++ b/chart/values.schema.json
@@ -657,7 +657,7 @@
}
},
"airflowLocalSettings": {
- "description": "`airflow_local_settings` file as a string (can be templated).",
+ "description": "`airflow_local_settings` file as a string (templated).",
"type": [
"string",
"null"
@@ -1102,7 +1102,7 @@
}
},
"extraEnv": {
- "description": "Extra env 'items' that will be added to the definition of Airflow containers; a string is expected (can be templated).",
+ "description": "Extra env 'items' that will be added to the definition of Airflow containers; a string is expected (templated).",
"type": [
"null",
"string"
@@ -1114,7 +1114,7 @@
]
},
"extraEnvFrom": {
- "description": "Extra envFrom 'items' that will be added to the definition of Airflow containers; a string is expected (can be templated).",
+ "description": "Extra envFrom 'items' that will be added to the definition of Airflow containers; a string is expected (templated).",
"type": [
"null",
"string"
@@ -1168,7 +1168,7 @@
"x-docsSection": "Kubernetes",
"default": {},
"additionalProperties": {
- "description": "Name of the secret (can be templated).",
+ "description": "Name of the secret (templated).",
"type": "object",
"minProperties": 1,
"additionalProperties": false,
@@ -1199,11 +1199,11 @@
"default": true
},
"data": {
- "description": "Content **as string** for the 'data' item of the secret (can be templated)",
+ "description": "Content **as string** for the 'data' item of the secret (templated)",
"type": "string"
},
"stringData": {
- "description": "Content **as string** for the 'stringData' item of the secret (can be templated)",
+ "description": "Content **as string** for the 'stringData' item of the secret (templated)",
"type": "string"
}
}
@@ -1223,7 +1223,7 @@
"x-docsSection": "Kubernetes",
"default": {},
"additionalProperties": {
- "description": "Name of the configMap (can be templated).",
+ "description": "Name of the configMap (templated).",
"type": "object",
"minProperties": 1,
"additionalProperties": false,
@@ -1250,7 +1250,7 @@
"default": true
},
"data": {
- "description": "Content **as string** for the 'data' item of the configmap (can be templated)",
+ "description": "Content **as string** for the 'data' item of the configmap (templated)",
"type": "string"
}
}
@@ -1790,7 +1790,7 @@
"default": "100Gi"
},
"storageClassName": {
- "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (can be templated).",
+ "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (templated).",
"type": [
"string",
"null"
@@ -3161,7 +3161,7 @@
"description": "PersistentVolumeClaim retention policy to be used in the lifecycle of a StatefulSet"
},
"storageClassName": {
- "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (can be templated).",
+ "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (templated).",
"type": [
"string",
"null"
@@ -4676,8 +4676,8 @@
}
}
},
- "apiServer": {
- "description": "Airflow API server settings.",
+ "_apiServer": {
+ "description": "Airflow API server settings. Experimental / for dev purpose only.",
"type": "object",
"x-docsSection": "API Server",
"additionalProperties": false,
@@ -4949,7 +4949,7 @@
},
"default": [
{
- "port": "{{ .Values.ports.apiServer }}"
+ "port": "{{ .Values.ports._apiServer }}"
}
],
"examples": [
@@ -5176,7 +5176,7 @@
"default": [
{
"name": "api-server",
- "port": "{{ .Values.ports.apiServer }}"
+ "port": "{{ .Values.ports._apiServer }}"
}
],
"examples": [
@@ -5884,7 +5884,7 @@
}
},
"webserverConfig": {
- "description": "This string (can be templated) will be mounted into the Airflow webserver as a custom `webserver_config.py`. You can bake a `webserver_config.py` in to your image instead or specify a configmap containing the webserver_config.py.",
+ "description": "This string (templated) will be mounted into the Airflow webserver as a custom `webserver_config.py`. You can bake a `webserver_config.py` in to your image instead or specify a configmap containing the webserver_config.py.",
"type": [
"string",
"null"
@@ -7736,7 +7736,7 @@
"default": "1Gi"
},
"storageClassName": {
- "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (can be templated).",
+ "description": "If using a custom StorageClass, pass name ref to all StatefulSets here (templated).",
"type": [
"string",
"null"
@@ -8175,7 +8175,7 @@
"type": "integer",
"default": 8080
},
- "apiServer": {
+ "_apiServer": {
"description": "API server port.",
"type": "integer",
"default": 9091
@@ -8668,7 +8668,7 @@
"default": "1Gi"
},
"storageClassName": {
- "description": "If using a custom StorageClass, pass name here (can be templated).",
+ "description": "If using a custom StorageClass, pass name here (templated).",
"type": [
"string",
"null"
@@ -8922,7 +8922,7 @@
]
},
"envFrom": {
- "description": "Extra envFrom 'items' that will be added to the definition of Airflow gitSync containers; a string or array are expected (can be templated).",
+ "description": "Extra envFrom 'items' that will be added to the definition of Airflow gitSync containers; a string or array are expected (templated).",
"type": [
"null",
"string"
@@ -8977,7 +8977,7 @@
"default": "100Gi"
},
"storageClassName": {
- "description": "If using a custom StorageClass, pass name here (can be templated).",
+ "description": "If using a custom StorageClass, pass name here (templated).",
"type": [
"string",
"null"
diff --git a/chart/values.yaml b/chart/values.yaml
index 852d6c2c792a6..89718d2a34e37 100644
--- a/chart/values.yaml
+++ b/chart/values.yaml
@@ -160,7 +160,7 @@ ingress:
# The hostnames or hosts configuration for the web Ingress
hosts: []
- # # The hostname for the web Ingress (can be templated)
+ # # The hostname for the web Ingress (templated)
# - name: ""
# # configs for web Ingress TLS
# tls:
@@ -204,7 +204,7 @@ ingress:
# The hostnames or hosts configuration for the flower Ingress
hosts: []
- # # The hostname for the flower Ingress (can be templated)
+ # # The hostname for the flower Ingress (templated)
# - name: ""
# tls:
# # Enable TLS termination for the flower Ingress
@@ -241,7 +241,7 @@ ingress:
# The hostnames or hosts configuration for the statsd Ingress
hosts: []
- # # The hostname for the statsd Ingress (can be templated)
+ # # The hostname for the statsd Ingress (templated)
# - name: ""
# tls:
# # Enable TLS termination for the statsd Ingress
@@ -271,7 +271,7 @@ ingress:
# The hostnames or hosts configuration for the pgbouncer Ingress
hosts: []
- # # The hostname for the statsd Ingress (can be templated)
+ # # The hostname for the statsd Ingress (templated)
# - name: ""
# tls:
# # Enable TLS termination for the pgbouncer Ingress
@@ -295,7 +295,7 @@ airflowPodAnnotations: {}
# main Airflow configmap
airflowConfigAnnotations: {}
-# `airflow_local_settings` file as a string (can be templated).
+# `airflow_local_settings` file as a string (templated).
airflowLocalSettings: |-
{{- if semverCompare ">=2.2.0" .Values.airflowVersion }}
{{- if not (or .Values.webserverSecretKey .Values.webserverSecretKeySecretName) }}
@@ -386,9 +386,9 @@ priorityClasses: []
# Extra secrets that will be managed by the chart
# (You can use them with extraEnv or extraEnvFrom or some of the extraVolumes values).
# The format for secret data is "key/value" where
-# * key (can be templated) is the name of the secret that will be created
+# * key (templated) is the name of the secret that will be created
# * value: an object with the standard 'data' or 'stringData' key (or both).
-# The value associated with those keys must be a string (can be templated)
+# The value associated with those keys must be a string (templated)
extraSecrets: {}
# eg:
# extraSecrets:
@@ -413,9 +413,9 @@ extraSecrets: {}
# Extra ConfigMaps that will be managed by the chart
# (You can use them with extraEnv or extraEnvFrom or some of the extraVolumes values).
# The format for configmap data is "key/value" where
-# * key (can be templated) is the name of the configmap that will be created
+# * key (templated) is the name of the configmap that will be created
# * value: an object with the standard 'data' key.
-# The value associated with this keys must be a string (can be templated)
+# The value associated with this keys must be a string (templated)
extraConfigMaps: {}
# eg:
# extraConfigMaps:
@@ -427,7 +427,7 @@ extraConfigMaps: {}
# AIRFLOW_VAR_KUBERNETES_NAMESPACE: "{{ .Release.Namespace }}"
# Extra env 'items' that will be added to the definition of airflow containers
-# a string is expected (can be templated).
+# a string is expected (templated).
# TODO: difference from `env`? This is a templated string. Probably should template `env` and remove this.
extraEnv: ~
# eg:
@@ -436,7 +436,7 @@ extraEnv: ~
# value: 'True'
# Extra envFrom 'items' that will be added to the definition of airflow containers
-# A string is expected (can be templated).
+# A string is expected (templated).
extraEnvFrom: ~
# eg:
# extraEnvFrom: |
@@ -1246,7 +1246,7 @@ migrateDatabaseJob:
applyCustomEnv: true
env: []
-apiServer:
+_apiServer:
# Labels specific to workers objects and pods
labels: {}
@@ -1274,7 +1274,7 @@ apiServer:
annotations: {}
ports:
- name: api-server
- port: "{{ .Values.ports.apiServer }}"
+ port: "{{ .Values.ports._apiServer }}"
loadBalancerIP: ~
## Limit load balancer source ips to list of CIDRs
@@ -1313,7 +1313,7 @@ apiServer:
from: []
# Ports for webserver NetworkPolicy ingress (if `from` is set)
ports:
- - port: "{{ .Values.ports.apiServer }}"
+ - port: "{{ .Values.ports._apiServer }}"
resources: {}
# limits:
@@ -1505,7 +1505,7 @@ webserver:
extraVolumes: []
extraVolumeMounts: []
- # This string (can be templated) will be mounted into the Airflow Webserver
+ # This string (templated) will be mounted into the Airflow Webserver
# as a custom webserver_config.py. You can bake a webserver_config.py in to
# your image instead or specify a configmap containing the
# webserver_config.py.
@@ -2529,7 +2529,7 @@ ports:
statsdScrape: 9102
pgbouncer: 6543
pgbouncerScrape: 9127
- apiServer: 9091
+ _apiServer: 9091
# Define any ResourceQuotas for namespace
quotas: {}
diff --git a/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py b/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py
index 82450192422b4..ac6d070a4d4e3 100644
--- a/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py
@@ -42,7 +42,12 @@
option_verbose,
)
from airflow_breeze.commands.production_image_commands import run_build_production_image
-from airflow_breeze.global_constants import ALLOWED_EXECUTORS, ALLOWED_KUBERNETES_VERSIONS
+from airflow_breeze.global_constants import (
+ ALLOWED_EXECUTORS,
+ ALLOWED_KUBERNETES_VERSIONS,
+ CELERY_EXECUTOR,
+ KUBERNETES_EXECUTOR,
+)
from airflow_breeze.params.build_prod_params import BuildProdParams
from airflow_breeze.utils.ci_group import ci_group
from airflow_breeze.utils.click_utils import BreezeGroup
@@ -596,7 +601,7 @@ def _rebuild_k8s_image(
COPY --chown=airflow:0 airflow/example_dags/ /opt/airflow/dags/
-COPY --chown=airflow:0 providers/src/airflow/providers/cncf/kubernetes/kubernetes_executor_templates/ /opt/airflow/pod_templates/
+COPY --chown=airflow:0 providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/kubernetes_executor_templates/ /opt/airflow/pod_templates/
ENV GUNICORN_CMD_ARGS='--preload' AIRFLOW__WEBSERVER__WORKER_REFRESH_INTERVAL=0
"""
@@ -782,6 +787,12 @@ def upload_k8s_image(
if return_code == 0:
get_console().print("\n[warning]NEXT STEP:[/][info] You might now deploy airflow by:\n")
get_console().print("\nbreeze k8s deploy-airflow\n")
+ get_console().print(
+ "\n[warning]Note:[/]\nIf you want to run tests with [info]--executor KubernetesExecutor[/], you should deploy airflow with [info]--multi-namespace-mode --executor KubernetesExecutor[/] flag.\n"
+ )
+ get_console().print(
+ "\nbreeze k8s deploy-airflow --multi-namespace-mode --executor KubernetesExecutor\n"
+ )
sys.exit(return_code)
@@ -1406,6 +1417,31 @@ def _get_parallel_test_args(
return combo_titles, combos, pytest_args, short_combo_titles
+def _is_deployed_with_same_executor(python: str, kubernetes_version: str, executor: str) -> bool:
+ """Check if the current cluster is deployed with the same executor that the current tests are using.
+
+ This is especially useful when running tests with executors like KubernetesExecutor, CeleryExecutor, etc.
+ It verifies by checking the label of the airflow-scheduler deployment.
+ """
+ result = run_command_with_k8s_env(
+ [
+ "kubectl",
+ "get",
+ "deployment",
+ "-n",
+ "airflow",
+ "airflow-scheduler",
+ "-o",
+ "jsonpath='{.metadata.labels.executor}'",
+ ],
+ python=python,
+ kubernetes_version=kubernetes_version,
+ capture_output=True,
+ check=False,
+ )
+ return executor == result.stdout.decode().strip().replace("'", "")
+
+
def _run_tests(
python: str,
kubernetes_version: str,
@@ -1422,7 +1458,17 @@ def _run_tests(
extra_shell_args.append("--no-rcs")
elif shell_binary.endswith("bash"):
extra_shell_args.extend(["--norc", "--noprofile"])
- the_tests: list[str] = ["kubernetes_tests/"]
+ if (
+ executor == KUBERNETES_EXECUTOR or executor == CELERY_EXECUTOR
+ ) and not _is_deployed_with_same_executor(python, kubernetes_version, executor):
+ get_console(output=output).print(
+ f"[warning]{executor} not deployed. Please deploy airflow with {executor} first."
+ )
+ get_console(output=output).print(
+ f"[info]You can deploy airflow with {executor} by running:[/]\nbreeze k8s configure-cluster\nbreeze k8s deploy-airflow --multi-namespace-mode --executor {executor}"
+ )
+ return 1, f"Tests {kubectl_cluster_name}"
+ the_tests: list[str] = ["kubernetes_tests/test_kubernetes_executor.py::TestKubernetesExecutor"]
command_to_run = " ".join([quote(arg) for arg in ["uv", "run", "pytest", *the_tests, *test_args]])
get_console(output).print(f"[info] Command to run:[/] {command_to_run}")
result = run_command(
diff --git a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
index 0f5e95f54b79b..3076b677c6aaa 100644
--- a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
@@ -81,7 +81,11 @@
option_airflow_constraints_location,
option_airflow_constraints_mode_prod,
)
-from airflow_breeze.global_constants import ALLOWED_INSTALLATION_METHODS, DEFAULT_EXTRAS
+from airflow_breeze.global_constants import (
+ ALLOWED_INSTALLATION_METHODS,
+ CONSTRAINTS_SOURCE_PROVIDERS,
+ DEFAULT_EXTRAS,
+)
from airflow_breeze.params.build_prod_params import BuildProdParams
from airflow_breeze.utils.ci_group import ci_group
from airflow_breeze.utils.click_utils import BreezeGroup
@@ -330,6 +334,10 @@ def run_build(prod_image_params: BuildProdParams) -> None:
get_console().print(f"[error]Error when building image! {info}")
sys.exit(return_code)
+ if not install_airflow_version and not airflow_constraints_location:
+ get_console().print(f"[yellow]Using {CONSTRAINTS_SOURCE_PROVIDERS} constraints mode[/]")
+ airflow_constraints_mode = CONSTRAINTS_SOURCE_PROVIDERS
+
perform_environment_checks()
check_remote_ghcr_io_commands()
base_build_params = BuildProdParams(
diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py
index 7df3b551a5b7d..0296c996b9c05 100644
--- a/dev/breeze/src/airflow_breeze/global_constants.py
+++ b/dev/breeze/src/airflow_breeze/global_constants.py
@@ -148,9 +148,13 @@
START_AIRFLOW_DEFAULT_ALLOWED_EXECUTOR = START_AIRFLOW_ALLOWED_EXECUTORS[0]
ALLOWED_CELERY_EXECUTORS = [CELERY_EXECUTOR, CELERY_K8S_EXECUTOR]
+CONSTRAINTS_SOURCE_PROVIDERS = "constraints-source-providers"
+CONSTRAINTS = "constraints"
+CONSTRAINTS_NO_PROVIDERS = "constraints-no-providers"
+
ALLOWED_KIND_OPERATIONS = ["start", "stop", "restart", "status", "deploy", "test", "shell", "k9s"]
-ALLOWED_CONSTRAINTS_MODES_CI = ["constraints-source-providers", "constraints", "constraints-no-providers"]
-ALLOWED_CONSTRAINTS_MODES_PROD = ["constraints", "constraints-no-providers", "constraints-source-providers"]
+ALLOWED_CONSTRAINTS_MODES_CI = [CONSTRAINTS_SOURCE_PROVIDERS, CONSTRAINTS, CONSTRAINTS_NO_PROVIDERS]
+ALLOWED_CONSTRAINTS_MODES_PROD = [CONSTRAINTS, CONSTRAINTS_NO_PROVIDERS, CONSTRAINTS_SOURCE_PROVIDERS]
ALLOWED_CELERY_BROKERS = ["rabbitmq", "redis"]
DEFAULT_CELERY_BROKER = ALLOWED_CELERY_BROKERS[1]
diff --git a/dev/breeze/src/airflow_breeze/prepare_providers/provider_documentation.py b/dev/breeze/src/airflow_breeze/prepare_providers/provider_documentation.py
index 6b705148c94cb..98936310c3c37 100644
--- a/dev/breeze/src/airflow_breeze/prepare_providers/provider_documentation.py
+++ b/dev/breeze/src/airflow_breeze/prepare_providers/provider_documentation.py
@@ -1231,6 +1231,7 @@ def _regenerate_pyproject_toml(context: dict[str, Any], provider_details: Provid
trim_blocks=True,
keep_trailing_newline=True,
)
+
get_pyproject_toml_path.write_text(get_pyproject_toml_content)
get_console().print(
f"[info]Generated {get_pyproject_toml_path} for the {provider_details.provider_id} provider\n"
diff --git a/dev/breeze/src/airflow_breeze/templates/pyproject_TEMPLATE.toml.jinja2 b/dev/breeze/src/airflow_breeze/templates/pyproject_TEMPLATE.toml.jinja2
index 5da149fa0d542..62560249e2331 100644
--- a/dev/breeze/src/airflow_breeze/templates/pyproject_TEMPLATE.toml.jinja2
+++ b/dev/breeze/src/airflow_breeze/templates/pyproject_TEMPLATE.toml.jinja2
@@ -68,7 +68,7 @@ classifiers = [
{% endfor %}
"Topic :: System :: Monitoring",
]
-requires-python = "~=3.9"
+requires-python = "{{ REQUIRES_PYTHON }}"
# The dependencies should be modified in place in the generated file
# Any change in the dependencies is preserved when the file is regenerated
diff --git a/dev/breeze/src/airflow_breeze/utils/packages.py b/dev/breeze/src/airflow_breeze/utils/packages.py
index fec5ed898d697..022f093871e37 100644
--- a/dev/breeze/src/airflow_breeze/utils/packages.py
+++ b/dev/breeze/src/airflow_breeze/utils/packages.py
@@ -32,6 +32,7 @@
from airflow_breeze.global_constants import (
ALLOWED_PYTHON_MAJOR_MINOR_VERSIONS,
+ DEFAULT_PYTHON_MAJOR_MINOR_VERSION,
PROVIDER_DEPENDENCIES,
PROVIDER_RUNTIME_DATA_SCHEMA_PATH,
REGULAR_DOC_PACKAGES,
@@ -788,6 +789,12 @@ def get_provider_jinja_context(
p for p in ALLOWED_PYTHON_MAJOR_MINOR_VERSIONS if p not in provider_details.excluded_python_versions
]
cross_providers_dependencies = get_cross_provider_dependent_packages(provider_package_id=provider_id)
+
+ # Most providers require the same python versions, but some may have exclusions
+ requires_python_version: str = f"~={DEFAULT_PYTHON_MAJOR_MINOR_VERSION}"
+ for excluded_python_version in provider_details.excluded_python_versions:
+ requires_python_version += f",!={excluded_python_version}"
+
context: dict[str, Any] = {
"PROVIDER_ID": provider_details.provider_id,
"PACKAGE_PIP_NAME": get_pip_package_name(provider_details.provider_id),
@@ -825,6 +832,7 @@ def get_provider_jinja_context(
"PIP_REQUIREMENTS_TABLE_RST": convert_pip_requirements_to_table(
get_provider_requirements(provider_id), markdown=False
),
+ "REQUIRES_PYTHON": requires_python_version,
}
return context
diff --git a/dev/moving_providers/move_providers.py b/dev/moving_providers/move_providers.py
index 13c1ef20cb586..da1213e4ffbc1 100755
--- a/dev/moving_providers/move_providers.py
+++ b/dev/moving_providers/move_providers.py
@@ -134,7 +134,11 @@ def _do_stuff(
if from_path.exists():
shutil.move(from_path, to_path)
console.print(f"\n[yellow]Moved {from_path} -> {to_path}\n")
- if remove_empty_parent_dir and len([path for path in from_path.parent.iterdir()]) == 0:
+ if (
+ remove_empty_parent_dir
+ and from_path.exists()
+ and len([path for path in from_path.parent.iterdir()]) == 0
+ ):
console.print(f"\n[yellow]Removed also empty parent dir {from_path.parent}\n")
from_path.parent.rmdir()
return
diff --git a/docs/.gitignore b/docs/.gitignore
index 2125d28bc1c31..ce8d1e7b430fd 100644
--- a/docs/.gitignore
+++ b/docs/.gitignore
@@ -6,8 +6,10 @@ apache-airflow-providers-apache-beam
apache-airflow-providers-apache-cassandra
apache-airflow-providers-apache-drill
apache-airflow-providers-apache-druid
+apache-airflow-providers-apache-flink
apache-airflow-providers-apache-hive
apache-airflow-providers-apache-iceberg
+apache-airflow-providers-apache-impala
apache-airflow-providers-apache-kafka
apache-airflow-providers-apache-kylin
apache-airflow-providers-apache-livy
@@ -18,6 +20,8 @@ apache-airflow-providers-apprise
apache-airflow-providers-asana
apache-airflow-providers-atlassian-jira
apache-airflow-providers-celery
+apache-airflow-providers-cloudant
+apache-airflow-providers-cncf-kubernetes
apache-airflow-providers-cohere
apache-airflow-providers-common-compat
apache-airflow-providers-common-io
@@ -41,6 +45,7 @@ apache-airflow-providers-jdbc
apache-airflow-providers-influxdb
apache-airflow-providers-microsoft-mssql
apache-airflow-providers-microsoft-psrp
+apache-airflow-providers-microsoft-winrm
apache-airflow-providers-mongo
apache-airflow-providers-openlineage
apache-airflow-providers-hashicorp
diff --git a/docs/apache-airflow-providers-apache-flink/changelog.rst b/docs/apache-airflow-providers-apache-flink/changelog.rst
deleted file mode 100644
index 07ffea0939e6f..0000000000000
--- a/docs/apache-airflow-providers-apache-flink/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/apache/flink/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-apache-impala/changelog.rst b/docs/apache-airflow-providers-apache-impala/changelog.rst
deleted file mode 100644
index ad7e0972ce927..0000000000000
--- a/docs/apache-airflow-providers-apache-impala/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/apache/impala/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-cloudant/changelog.rst b/docs/apache-airflow-providers-cloudant/changelog.rst
deleted file mode 100644
index d969e082c17b2..0000000000000
--- a/docs/apache-airflow-providers-cloudant/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/cloudant/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst b/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst
deleted file mode 100644
index 6ad86cec6753c..0000000000000
--- a/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/cncf/kubernetes/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-microsoft-winrm/changelog.rst b/docs/apache-airflow-providers-microsoft-winrm/changelog.rst
deleted file mode 100644
index fb0faf44d10fb..0000000000000
--- a/docs/apache-airflow-providers-microsoft-winrm/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/microsoft/winrm/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-yandex/changelog.rst b/docs/apache-airflow-providers-yandex/changelog.rst
deleted file mode 100644
index 9bcad616eb83d..0000000000000
--- a/docs/apache-airflow-providers-yandex/changelog.rst
+++ /dev/null
@@ -1,25 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
-.. include:: ../../providers/src/airflow/providers/yandex/CHANGELOG.rst
diff --git a/docs/apache-airflow-providers-yandex/commits.rst b/docs/apache-airflow-providers-yandex/commits.rst
deleted file mode 100644
index 78d8068f45cbc..0000000000000
--- a/docs/apache-airflow-providers-yandex/commits.rst
+++ /dev/null
@@ -1,492 +0,0 @@
-
- .. 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.
-
- .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE
- OVERWRITTEN WHEN PREPARING PACKAGES.
-
- .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE
- `PROVIDER_COMMITS_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY
-
- .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
-
-Package apache-airflow-providers-yandex
-------------------------------------------------------
-
-This package is for Yandex, including:
-
- - `Yandex.Cloud `__
-
-
-This is detailed commit list of changes for versions provider package: ``yandex``.
-For high-level changelog, see :doc:`package information including changelog `.
-
-
-
-4.0.0
-.....
-
-Latest change: 2024-12-19
-
-================================================================================================= =========== ========================================================================================
-Commit Committed Subject
-================================================================================================= =========== ========================================================================================
-`35b927fe17 `_ 2024-12-19 ``Update path of example dags in docs (#45069)``
-`4b38bed76c `_ 2024-12-16 ``Bump min version of Providers to 2.9 (#44956)``
-`e786c78f52 `_ 2024-12-07 ``Remove Provider Deprecations in Yandex provider (#44754)``
-`1275fec92f `_ 2024-11-24 ``Use Python 3.9 as target version for Ruff & Black rules (#44298)``
-`4dfae23532 `_ 2024-11-15 ``Update DAG example links in multiple providers documents (#44034)``
-`a53d9f6d25 `_ 2024-11-14 ``Prepare docs for Nov 1st wave of providers (#44011)``
-`857ca4c06c `_ 2024-10-09 ``Split providers out of the main "airflow/" tree into a UV workspace project (#42505)``
-================================================================================================= =========== ========================================================================================
-
-3.12.0
-......
-
-Latest change: 2024-08-19
-
-================================================================================================= =========== =======================================================================
-Commit Committed Subject
-================================================================================================= =========== =======================================================================
-`75fb7acbac `_ 2024-08-19 ``Prepare docs for Aug 2nd wave of providers (#41559)``
-`fcbff15bda `_ 2024-08-12 ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)``
-`2daa5bd01a `_ 2024-08-04 ``providers/yandex: fix typing (#40997)``
-`d23881c648 `_ 2024-08-03 ``Prepare docs for Aug 1st wave of providers (#41230)``
-`09a7bd1d58 `_ 2024-07-09 ``Prepare docs 1st wave July 2024 (#40644)``
-`a62bd83188 `_ 2024-06-27 ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)``
-================================================================================================= =========== =======================================================================
-
-3.11.2
-......
-
-Latest change: 2024-06-22
-
-================================================================================================= =========== =========================================================================
-Commit Committed Subject
-================================================================================================= =========== =========================================================================
-`6e5ae26382 `_ 2024-06-22 ``Prepare docs 2nd wave June 2024 (#40273)``
-`0d5bb60981 `_ 2024-06-17 ``Fix typos in Providers docs and Yandex hook (#40277)``
-`53e6739e67 `_ 2024-06-01 ``Limit yandex provider to avoid mypy errors (#39990)``
-`8173693a70 `_ 2024-05-31 ``Remove upper-binding in yandex after dataproc issue is fixed (#39974)``
-`b8a83b2293 `_ 2024-05-31 ``Workaround new yandexcloud breaking dataproc integration (#39964)``
-================================================================================================= =========== =========================================================================
-
-3.11.1
-......
-
-Latest change: 2024-05-26
-
-================================================================================================= =========== =========================================================================
-Commit Committed Subject
-================================================================================================= =========== =========================================================================
-`34500f3a2f `_ 2024-05-26 ``Prepare docs 3rd wave May 2024 (#39738)``
-`e0dd075d1b `_ 2024-05-15 `` AIP-21: yandexcloud: rename files, emit deprecation warning (#39618)``
-`defe4590e9 `_ 2024-05-11 ``yandex provider: bump version for yq http client package (#39548)``
-`2b1a2f8d56 `_ 2024-05-11 ``Reapply templates for all providers (#39554)``
-`2c05187b07 `_ 2024-05-10 ``Faster 'airflow_version' imports (#39552)``
-`05945a47f3 `_ 2024-05-09 ``add doc about Yandex Query operator (#39445)``
-`73918925ed `_ 2024-05-08 ``Simplify 'airflow_version' imports (#39497)``
-================================================================================================= =========== =========================================================================
-
-3.11.0
-......
-
-Latest change: 2024-05-01
-
-================================================================================================= =========== =======================================================================
-Commit Committed Subject
-================================================================================================= =========== =======================================================================
-`fe4605a10e `_ 2024-05-01 ``Prepare docs 1st wave May 2024 (#39328)``
-`ead9b00f7c `_ 2024-04-25 ``Bump minimum Airflow version in providers to Airflow 2.7.0 (#39240)``
-================================================================================================= =========== =======================================================================
-
-3.10.0
-......
-
-Latest change: 2024-04-13
-
-================================================================================================= =========== ==================================================================
-Commit Committed Subject
-================================================================================================= =========== ==================================================================
-`f9dcc82fb6 `_ 2024-04-13 ``Prepare docs 1st wave (RC2) April 2024 (#38995)``
-`5fa80b6aea `_ 2024-04-10 ``Prepare docs 1st wave (RC1) April 2024 (#38863)``
-`a9a6976dd2 `_ 2024-03-28 ``docs: yandex provider grammatical improvements (#38589)``
-`30817a5c6d `_ 2024-03-22 ``support iam token from metadata, simplify code (#38411)``
-`390bec1c82 `_ 2024-03-20 ``Add Yandex Query support from Yandex.Cloud (#37458)``
-`0a74928894 `_ 2024-03-18 ``Bump ruff to 0.3.3 (#38240)``
-`c0b849ad2b `_ 2024-03-11 ``Avoid use of 'assert' outside of the tests (#37718)``
-`83316b8158 `_ 2024-03-04 ``Prepare docs 1st wave (RC1) March 2024 (#37876)``
-`5a0be392e6 `_ 2024-02-16 ``Add comment about versions updated by release manager (#37488)``
-================================================================================================= =========== ==================================================================
-
-3.9.0
-.....
-
-Latest change: 2024-02-12
-
-================================================================================================= =========== ==================================================================================
-Commit Committed Subject
-================================================================================================= =========== ==================================================================================
-`bfb054e9e8 `_ 2024-02-12 ``Prepare docs 1st wave of Providers February 2024 (#37326)``
-`08036e5df5 `_ 2024-02-08 ``D401 Support in Providers (simple) (#37258)``
-`cea58c1111 `_ 2024-02-02 ``fix: using endpoint from connection if not specified (#37076)``
-`3ec781946a `_ 2024-02-01 ``Add secrets-backends section into the Yandex provider yaml definition (#37065)``
-`0e752383a8 `_ 2024-01-31 ``docs: update description in airflow provider.yaml (#37096)``
-================================================================================================= =========== ==================================================================================
-
-3.8.0
-.....
-
-Latest change: 2024-01-26
-
-================================================================================================= =========== ====================================================================================================================
-Commit Committed Subject
-================================================================================================= =========== ====================================================================================================================
-`cead3da4a6 `_ 2024-01-26 ``Add docs for RC2 wave of providers for 2nd round of Jan 2024 (#37019)``
-`0b680c9492 `_ 2024-01-26 ``Revert "Provide the logger_name param in providers hooks in order to override the logger name (#36675)" (#37015)``
-`12ccb5f0ac `_ 2024-01-25 ``feat: add Yandex Cloud Lockbox secrets backend (#36449)``
-`2b4da0101f `_ 2024-01-22 ``Prepare docs 2nd wave of Providers January 2024 (#36945)``
-`6ff96af480 `_ 2024-01-18 ``Fix stacklevel in warnings.warn into the providers (#36831)``
-`6bd450da1e `_ 2024-01-10 ``Provide the logger_name param in providers hooks in order to override the logger name (#36675)``
-`19ebcac239 `_ 2024-01-07 ``Prepare docs 1st wave of Providers January 2024 (#36640)``
-`6937ae7647 `_ 2023-12-30 ``Speed up autocompletion of Breeze by simplifying provider state (#36499)``
-================================================================================================= =========== ====================================================================================================================
-
-3.7.1
-.....
-
-Latest change: 2023-12-23
-
-================================================================================================= =========== ==================================================================================
-Commit Committed Subject
-================================================================================================= =========== ==================================================================================
-`b15d5578da `_ 2023-12-23 ``Re-apply updated version numbers to 2nd wave of providers in December (#36380)``
-`f5883d6e7b `_ 2023-12-23 ``Prepare 2nd wave of providers in December (#36373)``
-`cd476acd8f `_ 2023-12-11 ``Follow BaseHook connection fields method signature in child classes (#36086)``
-================================================================================================= =========== ==================================================================================
-
-3.7.0
-.....
-
-Latest change: 2023-12-08
-
-================================================================================================= =========== =======================================================================
-Commit Committed Subject
-================================================================================================= =========== =======================================================================
-`999b70178a `_ 2023-12-08 ``Prepare docs 1st wave of Providers December 2023 (#36112)``
-`d0918d77ee `_ 2023-12-07 ``Bump minimum Airflow version in providers to Airflow 2.6.0 (#36017)``
-`0b23d5601c `_ 2023-11-24 ``Prepare docs 2nd wave of Providers November 2023 (#35836)``
-`99534e47f3 `_ 2023-11-19 ``Use reproducible builds for provider packages (#35693)``
-`99df205f42 `_ 2023-11-16 ``Fix and reapply templates for provider documentation (#35686)``
-================================================================================================= =========== =======================================================================
-
-3.6.0
-.....
-
-Latest change: 2023-11-08
-
-================================================================================================= =========== ==================================================================
-Commit Committed Subject
-================================================================================================= =========== ==================================================================
-`1b059c57d6 `_ 2023-11-08 ``Prepare docs 1st wave of Providers November 2023 (#35537)``
-`706878ec35 `_ 2023-11-04 ``Remove empty lines in generated changelog (#35436)``
-`052e26ad47 `_ 2023-11-04 ``Change security.rst to use includes in providers (#35435)``
-`09880741cb `_ 2023-11-03 ``Add configuration files for yandex (#35420)``
-`0b850a97e8 `_ 2023-11-03 ``Yandex dataproc deduce default service account (#35059)``
-`d1c58d86de `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 - FIX (#35233)``
-`3592ff4046 `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 (#35187)``
-`dd7ba3cae1 `_ 2023-10-19 ``Pre-upgrade 'ruff==0.0.292' changes in providers (#35053)``
-================================================================================================= =========== ==================================================================
-
-3.5.0
-.....
-
-Latest change: 2023-10-13
-
-================================================================================================= =========== ===============================================================
-Commit Committed Subject
-================================================================================================= =========== ===============================================================
-`e9987d5059 `_ 2023-10-13 ``Prepare docs 1st wave of Providers in October 2023 (#34916)``
-`0c8e30e43b `_ 2023-10-05 ``Bump min airflow version of providers (#34728)``
-================================================================================================= =========== ===============================================================
-
-3.4.0
-.....
-
-Latest change: 2023-08-26
-
-================================================================================================= =========== ======================================================================
-Commit Committed Subject
-================================================================================================= =========== ======================================================================
-`c077d19060 `_ 2023-08-26 ``Prepare docs for Aug 2023 3rd wave of Providers (#33730)``
-`2ae1c10bfa `_ 2023-08-23 ``add support for Yandex Dataproc cluster labels (#29811)``
-`2b43fa473f `_ 2023-08-22 ``Resume yandex provider (#33574)``
-`73b90c48b1 `_ 2023-07-21 ``Allow configuration to be contributed by providers (#32604)``
-`3878fe6fab `_ 2023-07-05 ``Remove spurious headers for provider changelogs (#32373)``
-`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)``
-`8b146152d6 `_ 2023-06-20 ``Add note about dropping Python 3.7 for providers (#32015)``
-`a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)``
-`abea189022 `_ 2023-05-18 ``Use '__version__' in providers not 'version' (#31393)``
-`0a30706aa7 `_ 2023-05-03 ``Use 'AirflowProviderDeprecationWarning' in providers (#30975)``
-`eef5bc7f16 `_ 2023-05-03 ``Add full automation for min Airflow version for providers (#30994)``
-`a7eb32a5b2 `_ 2023-04-30 ``Bump minimum Airflow version in providers (#30917)``
-`b4d6e83686 `_ 2023-04-19 ``Suspend Yandex provider due to protobuf limitation (#30667)``
-`d23a3bbed8 `_ 2023-04-04 ``Add mechanism to suspend providers (#30422)``
-================================================================================================= =========== ======================================================================
-
-3.3.0
-.....
-
-Latest change: 2023-03-03
-
-================================================================================================= =========== =====================================================================
-Commit Committed Subject
-================================================================================================= =========== =====================================================================
-`fcd3c0149f `_ 2023-03-03 ``Prepare docs for 03/2023 wave of Providers (#29878)``
-`1768872a00 `_ 2023-02-22 ``support Yandex SDK feature "endpoint" (#29635)``
-`2b92c3c74d `_ 2023-01-05 ``Fix providers documentation formatting (#28754)``
-`c8e348dcb0 `_ 2022-12-05 ``Add automated version replacement in example dag indexes (#28090)``
-================================================================================================= =========== =====================================================================
-
-3.2.0
-.....
-
-Latest change: 2022-11-26
-
-================================================================================================= =========== ====================================================================================
-Commit Committed Subject
-================================================================================================= =========== ====================================================================================
-`25bdbc8e67 `_ 2022-11-26 ``Updated docs for RC3 wave of providers (#27937)``
-`2e20e9f7eb `_ 2022-11-24 ``Prepare for follow-up relase for November providers (#27774)``
-`12c3c39d1a `_ 2022-11-15 ``pRepare docs for November 2022 wave of Providers (#27613)``
-`78b8ea2f22 `_ 2022-10-24 ``Move min airflow version to 2.3.0 for all providers (#27196)``
-`2a34dc9e84 `_ 2022-10-23 ``Enable string normalization in python formatting - providers (#27205)``
-`837e463ae8 `_ 2022-10-22 ``Allow no extra prefix in yandex hook (#27040)``
-`f8db64c35c `_ 2022-09-28 ``Update docs for September Provider's release (#26731)``
-`06acf40a43 `_ 2022-09-13 ``Apply PEP-563 (Postponed Evaluation of Annotations) to non-core airflow (#26289)``
-================================================================================================= =========== ====================================================================================
-
-3.1.0
-.....
-
-Latest change: 2022-08-10
-
-================================================================================================= =========== ===============================================================================
-Commit Committed Subject
-================================================================================================= =========== ===============================================================================
-`e5ac6c7cfb `_ 2022-08-10 ``Prepare docs for new providers release (August 2022) (#25618)``
-`a61e0c1df7 `_ 2022-07-29 ``YandexCloud provider: Support new Yandex SDK features for DataProc (#25158)``
-`d2459a241b `_ 2022-07-13 ``Add documentation for July 2022 Provider's release (#25030)``
-`0de31bd73a `_ 2022-06-29 ``Move provider dependencies to inside provider folders (#24672)``
-`510a6bab45 `_ 2022-06-28 ``Remove 'hook-class-names' from provider.yaml (#24702)``
-`08b675cf66 `_ 2022-06-13 ``Fix links to sources for examples (#24386)``
-================================================================================================= =========== ===============================================================================
-
-3.0.0
-.....
-
-Latest change: 2022-06-09
-
-================================================================================================= =========== ==================================================================================
-Commit Committed Subject
-================================================================================================= =========== ==================================================================================
-`dcdcf3a2b8 `_ 2022-06-09 ``Update release notes for RC2 release of Providers for May 2022 (#24307)``
-`717a7588bc `_ 2022-06-07 ``Update package description to remove double min-airflow specification (#24292)``
-`aeabe994b3 `_ 2022-06-07 ``Prepare docs for May 2022 provider's release (#24231)``
-`333e98759b `_ 2022-06-07 ``Fix link broken after #24082 (#24276)``
-`027b707d21 `_ 2022-06-05 ``Add explanatory note for contributors about updating Changelog (#24229)``
-`65ad2aed26 `_ 2022-06-01 ``Migrate Yandex example DAGs to new design AIP-47 (#24082)``
-================================================================================================= =========== ==================================================================================
-
-2.2.3
-.....
-
-Latest change: 2022-03-22
-
-================================================================================================= =========== ==============================================================
-Commit Committed Subject
-================================================================================================= =========== ==============================================================
-`d7dbfb7e26 `_ 2022-03-22 ``Add documentation for bugfix release of Providers (#22383)``
-================================================================================================= =========== ==============================================================
-
-2.2.2
-.....
-
-Latest change: 2022-03-14
-
-================================================================================================= =========== ====================================================================
-Commit Committed Subject
-================================================================================================= =========== ====================================================================
-`16adc035b1 `_ 2022-03-14 ``Add documentation for Classifier release for March 2022 (#22226)``
-================================================================================================= =========== ====================================================================
-
-2.2.1
-.....
-
-Latest change: 2022-03-07
-
-================================================================================================= =========== ==========================================================================
-Commit Committed Subject
-================================================================================================= =========== ==========================================================================
-`f5b96315fe `_ 2022-03-07 ``Add documentation for Feb Providers release (#22056)``
-`6126c4e40f `_ 2022-03-07 ``Fix spelling (#22054)``
-`d94fa37830 `_ 2022-02-08 ``Fixed changelog for January 2022 (delayed) provider's release (#21439)``
-`6c3a67d4fc `_ 2022-02-05 ``Add documentation for January 2021 providers release (#21257)``
-`cb73053211 `_ 2022-01-27 ``Add optional features in providers. (#21074)``
-`602abe8394 `_ 2022-01-20 ``Remove ':type' lines now sphinx-autoapi supports typehints (#20951)``
-================================================================================================= =========== ==========================================================================
-
-2.2.0
-.....
-
-Latest change: 2021-12-31
-
-================================================================================================= =========== ============================================================================================================
-Commit Committed Subject
-================================================================================================= =========== ============================================================================================================
-`f77417eb0d `_ 2021-12-31 ``Fix K8S changelog to be PyPI-compatible (#20614)``
-`97496ba2b4 `_ 2021-12-31 ``Update documentation for provider December 2021 release (#20523)``
-`d56e7b56bb `_ 2021-12-30 ``Fix template_fields type to have MyPy friendly Sequence type (#20571)``
-`a0821235fb `_ 2021-12-30 ``Use typed Context EVERYWHERE (#20565)``
-`6e51608f28 `_ 2021-12-16 ``Fix mypy for providers: elasticsearch, oracle, yandex (#20344)``
-`41c49c7ff6 `_ 2021-12-14 ``YandexCloud provider: Support new Yandex SDK features: log_group_id, user-agent, maven packages (#20103)``
-`9a469d813f `_ 2021-11-30 ``Capitalize names in docs (#19893)``
-`853576d901 `_ 2021-11-30 ``Update documentation for November 2021 provider's release (#19882)``
-`d9567eb106 `_ 2021-10-29 ``Prepare documentation for October Provider's release (#19321)``
-`f5ad26dcdd `_ 2021-10-21 ``Fixup string concatenations (#19099)``
-`840ea3efb9 `_ 2021-09-30 ``Update documentation for September providers release (#18613)``
-`ef037e7021 `_ 2021-09-29 ``Static start_date and default arg cleanup for misc. provider example DAGs (#18597)``
-`e25eea052f `_ 2021-09-19 ``Inclusive Language (#18349)``
-`1cb456cba1 `_ 2021-09-12 ``Add official download page for providers (#18187)``
-`046f02e5a7 `_ 2021-09-09 ``fix misspelling (#18121)``
-================================================================================================= =========== ============================================================================================================
-
-2.1.0
-.....
-
-Latest change: 2021-08-30
-
-================================================================================================= =========== =============================================================================
-Commit Committed Subject
-================================================================================================= =========== =============================================================================
-`0a68588479 `_ 2021-08-30 ``Add August 2021 Provider's documentation (#17890)``
-`be75dcd39c `_ 2021-08-23 ``Update description about the new ''connection-types'' provider meta-data``
-`76ed2a49c6 `_ 2021-08-19 ``Import Hooks lazily individually in providers manager (#17682)``
-`e3089dd5d0 `_ 2021-08-02 ``Add autoscaling subcluster support and remove defaults (#17033)``
-`87f408b1e7 `_ 2021-07-26 ``Prepares docs for Rc2 release of July providers (#17116)``
-`0dbd0f420c `_ 2021-07-26 ``Remove/refactor default_args pattern for miscellaneous providers (#16872)``
-`b916b75079 `_ 2021-07-15 ``Prepare documentation for July release of providers. (#17015)``
-`866a601b76 `_ 2021-06-28 ``Removes pylint from our toolchain (#16682)``
-================================================================================================= =========== =============================================================================
-
-2.0.0
-.....
-
-Latest change: 2021-06-18
-
-================================================================================================= =========== =======================================================================
-Commit Committed Subject
-================================================================================================= =========== =======================================================================
-`bbc627a3da `_ 2021-06-18 ``Prepares documentation for rc2 release of Providers (#16501)``
-`cbf8001d76 `_ 2021-06-16 ``Synchronizes updated changelog after buggfix release (#16464)``
-`1fba5402bb `_ 2021-06-15 ``More documentation update for June providers release (#16405)``
-`9c94b72d44 `_ 2021-06-07 ``Updated documentation for June 2021 provider release (#16294)``
-`1e647029e4 `_ 2021-06-01 ``Rename the main branch of the Airflow repo to be 'main' (#16149)``
-`37681bca00 `_ 2021-05-07 ``Auto-apply apply_default decorator (#15667)``
-`807ad32ce5 `_ 2021-05-01 ``Prepares provider release after PIP 21 compatibility (#15576)``
-`40a2476a5d `_ 2021-04-28 ``Adds interactivity when generating provider documentation. (#15518)``
-`a7ca1b3b0b `_ 2021-03-26 ``Fix Sphinx Issues with Docstrings (#14968)``
-`e172bd0e16 `_ 2021-03-22 ``Update docstrings to adhere to sphinx standards (#14918)``
-`68e4c4dcb0 `_ 2021-03-20 ``Remove Backport Providers (#14886)``
-`6e6526a0f6 `_ 2021-03-13 ``Update documentation for broken package releases (#14734)``
-================================================================================================= =========== =======================================================================
-
-1.0.1
-.....
-
-Latest change: 2021-02-04
-
-================================================================================================= =========== =========================================================
-Commit Committed Subject
-================================================================================================= =========== =========================================================
-`88bdcfa0df `_ 2021-02-04 ``Prepare to release a new wave of providers. (#14013)``
-`ac2f72c98d `_ 2021-02-01 ``Implement provider versioning tools (#13767)``
-`3fd5ef3555 `_ 2021-01-21 ``Add missing logos for integrations (#13717)``
-`295d66f914 `_ 2020-12-30 ``Fix Grammar in PIP warning (#13380)``
-`6cf76d7ac0 `_ 2020-12-18 ``Fix typo in pip upgrade command :( (#13148)``
-`f6448b4e48 `_ 2020-12-15 ``Add link to PyPI Repository to provider docs (#13064)``
-================================================================================================= =========== =========================================================
-
-1.0.0
-.....
-
-Latest change: 2020-12-09
-
-================================================================================================= =========== ======================================================================================================================================================================
-Commit Committed Subject
-================================================================================================= =========== ======================================================================================================================================================================
-`32971a1a2d `_ 2020-12-09 ``Updates providers versions to 1.0.0 (#12955)``
-`b40dffa085 `_ 2020-12-08 ``Rename remaing modules to match AIP-21 (#12917)``
-`9b39f24780 `_ 2020-12-08 ``Add support for dynamic connection form fields per provider (#12558)``
-`bd90136aaf `_ 2020-11-30 ``Move operator guides to provider documentation packages (#12681)``
-`de3b1e687b `_ 2020-11-28 ``Move connection guides to provider documentation packages (#12653)``
-`ef4af21351 `_ 2020-11-22 ``Move providers docs to separate package + Spell-check in a common job with docs-build (#12527)``
-`f2569de7d1 `_ 2020-11-22 ``Add example DAGs to provider docs (#12528)``
-`c34ef853c8 `_ 2020-11-20 ``Separate out documentation building per provider (#12444)``
-`0080354502 `_ 2020-11-18 ``Update provider READMEs for 1.0.0b2 batch release (#12449)``
-`ae7cb4a1e2 `_ 2020-11-17 ``Update wrong commit hash in backport provider changes (#12390)``
-`6889a333cf `_ 2020-11-15 ``Improvements for operators and hooks ref docs (#12366)``
-`7825e8f590 `_ 2020-11-13 ``Docs installation improvements (#12304)``
-`85a18e13d9 `_ 2020-11-09 ``Point at pypi project pages for cross-dependency of provider packages (#12212)``
-`59eb5de78c `_ 2020-11-09 ``Update provider READMEs for up-coming 1.0.0beta1 releases (#12206)``
-`b2a28d1590 `_ 2020-11-09 ``Moves provider packages scripts to dev (#12082)``
-`4e8f9cc8d0 `_ 2020-11-03 ``Enable Black - Python Auto Formmatter (#9550)``
-`8c42cf1b00 `_ 2020-11-03 ``Use PyUpgrade to use Python 3.6 features (#11447)``
-`5a439e84eb `_ 2020-10-26 ``Prepare providers release 0.0.2a1 (#11855)``
-`872b1566a1 `_ 2020-10-25 ``Generated backport providers readmes/setup for 2020.10.29 (#11826)``
-`349b0811c3 `_ 2020-10-20 ``Add D200 pydocstyle check (#11688)``
-`16e7129719 `_ 2020-10-13 ``Added support for provider packages for Airflow 2.0 (#11487)``
-`0a0e1af800 `_ 2020-10-03 ``Fix Broken Markdown links in Providers README TOC (#11249)``
-`ca4238eb4d `_ 2020-10-02 ``Fixed month in backport packages to October (#11242)``
-`5220e4c384 `_ 2020-10-02 ``Prepare Backport release 2020.09.07 (#11238)``
-`5093245d6f `_ 2020-09-30 ``Strict type coverage for Oracle and Yandex provider (#11198)``
-`9549274d11 `_ 2020-09-09 ``Upgrade black to 20.8b1 (#10818)``
-`fdd9b6f65b `_ 2020-08-25 ``Enable Black on Providers Packages (#10543)``
-`3696c34c28 `_ 2020-08-24 ``Fix typo in the word "release" (#10528)``
-`ee7ca128a1 `_ 2020-08-22 ``Fix broken Markdown refernces in Providers README (#10483)``
-`f6734b3b85 `_ 2020-08-12 ``Enable Sphinx spellcheck for doc generation (#10280)``
-`cdec301254 `_ 2020-08-07 ``Add correct signature to all operators and sensors (#10205)``
-`aeea71274d `_ 2020-08-02 ``Remove 'args' parameter from provider operator constructors (#10097)``
-`7d24b088cd `_ 2020-07-25 ``Stop using start_date in default_args in example_dags (2) (#9985)``
-`d0e7db4024 `_ 2020-06-19 ``Fixed release number for fresh release (#9408)``
-`12af6a0800 `_ 2020-06-19 ``Final cleanup for 2020.6.23rc1 release preparation (#9404)``
-`c7e5bce57f `_ 2020-06-19 ``Prepare backport release candidate for 2020.6.23rc1 (#9370)``
-`40bf8f28f9 `_ 2020-06-18 ``Detect automatically the lack of reference to the guide in the operator descriptions (#9290)``
-`f6bd817a3a `_ 2020-06-16 ``Introduce 'transfers' packages (#9320)``
-`0b0e4f7a4c `_ 2020-05-26 ``Preparing for RC3 relase of backports (#9026)``
-`00642a46d0 `_ 2020-05-26 ``Fixed name of 20 remaining wrongly named operators. (#8994)``
-`1d36b0303b `_ 2020-05-23 ``Fix references in docs (#8984)``
-`375d1ca229 `_ 2020-05-19 ``Release candidate 2 for backport packages 2020.05.20 (#8898)``
-`12c5e5d8ae `_ 2020-05-17 ``Prepare release candidate for backport packages (#8891)``
-`f3521fb0e3 `_ 2020-05-16 ``Regenerate readme files for backport package release (#8886)``
-`92585ca4cb `_ 2020-05-15 ``Added automated release notes generation for backport operators (#8807)``
-`59a4f26699 `_ 2020-04-17 ``stop rendering some class docs in wrong place (#8095)``
-`3320e432a1 `_ 2020-02-24 ``[AIRFLOW-6817] Lazy-load 'airflow.DAG' to keep user-facing API untouched (#7517)``
-`4d03e33c11 `_ 2020-02-22 ``[AIRFLOW-6817] remove imports from 'airflow/__init__.py', replaced implicit imports with explicit imports, added entry to 'UPDATING.MD' - squashed/rebased (#7456)``
-`9cbd7de6d1 `_ 2020-02-18 ``[AIRFLOW-6792] Remove _operator/_hook/_sensor in providers package and add tests (#7412)``
-`ee1ab7697c `_ 2020-02-14 ``[AIRFLOW-6531] Initial Yandex.Cloud Dataproc support (#7252)``
-================================================================================================= =========== ======================================================================================================================================================================
diff --git a/docs/apache-airflow-providers-yandex/configurations-ref.rst b/docs/apache-airflow-providers-yandex/configurations-ref.rst
deleted file mode 100644
index 5885c9d91b6e8..0000000000000
--- a/docs/apache-airflow-providers-yandex/configurations-ref.rst
+++ /dev/null
@@ -1,18 +0,0 @@
- .. 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.
-
-.. include:: ../exts/includes/providers-configurations-ref.rst
diff --git a/docs/apache-airflow-providers-yandex/connections/yandexcloud.rst b/docs/apache-airflow-providers-yandex/connections/yandexcloud.rst
deleted file mode 100644
index b1d8b4074c295..0000000000000
--- a/docs/apache-airflow-providers-yandex/connections/yandexcloud.rst
+++ /dev/null
@@ -1,95 +0,0 @@
- .. 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.
-
-.. _yandex_cloud_connection:
-
-Yandex.Cloud Connection
-=======================
-
-The Yandex.Cloud connection type enables the authentication in Yandex.Cloud services.
-
-Configuring the Connection
---------------------------
-
-Service account auth JSON
- JSON object as a string.
-
- Example: ``{"id": "...", "service_account_id": "...", "private_key": "..."}``
-
-Service account auth JSON file path
- Path to the file containing service account auth JSON.
-
- Example: ``/home/airflow/authorized_key.json``
-
-OAuth Token
- User account OAuth token as a string.
-
- Example: ``y3_Vd3eub7w9bIut67GHeL345gfb5GAnd3dZnf08FR1vjeUFve7Yi8hGvc``
-
-SSH public key (optional)
- The key will be placed to all created Compute nodes, allowing you to have a root shell there.
-
-Folder ID (optional)
- A folder is an entity to separate different projects within the cloud.
-
- If specified, this ID will be used by default when creating nodes and clusters.
-
- See `this guide `__ for details.
-
-Endpoint (optional)
- Use this setting to configure your API endpoint.
-
- Leave blank to use default `endpoints `__.
-
-Default Connection IDs
-----------------------
-
-All hooks and operators related to Yandex.Cloud use the ``yandexcloud_default`` connection by default.
-
-Authenticating to Yandex.Cloud
-------------------------------
-
-Using authorized keys to authorize as a service account
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-
-Before you start, make sure you have `created `__
-a Yandex Cloud `service account `__.
-
-First, you need to create an `authorized key `__
-for your service account and save the generated JSON file with both public and private key parts.
-
-Then, you need to specify the key in the ``Service account auth JSON`` field.
-
-Alternatively, you can specify the path to the JSON file in the ``Service account auth JSON file path`` field.
-
-Using an OAuth token to authorize as a user account
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-
-First, you need to create
-an `OAuth token `__ for your user account.
-Your token will look like this: ``y3_Vd3eub7w9bIut67GHeL345gfb5GAnd3dZnf08FR1vjeUFve7Yi8hGvc``.
-
-Then you need to specify your token in the ``OAuth Token`` field.
-
-Using metadata service
-~~~~~~~~~~~~~~~~~~~~~~
-
-If you do not specify any credentials, the connection will attempt to use
-the `metadata service `__ for authentication.
-
-To do this, you need to `link `__
-your service account with your VM.
diff --git a/docs/apache-airflow-providers-yandex/index.rst b/docs/apache-airflow-providers-yandex/index.rst
deleted file mode 100644
index 249542f23edc3..0000000000000
--- a/docs/apache-airflow-providers-yandex/index.rst
+++ /dev/null
@@ -1,110 +0,0 @@
-
- .. 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.
-
-``apache-airflow-providers-yandex``
-===================================
-
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: Basics
-
- Home
- Changelog
- Security
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: Guides
-
- Configuration
- Connection types
- Lockbox Secret Backend
- Operators
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: References
-
- Python API <_api/airflow/providers/yandex/index>
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: System tests
-
- System Tests <_api/tests/system/yandex/index>
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: Resources
-
- Example DAGs
- PyPI Repository
- Installing from sources
-
-.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
-
-
-.. toctree::
- :hidden:
- :maxdepth: 1
- :caption: Commits
-
- Detailed list of commits
-
-
-apache-airflow-providers-yandex package
-------------------------------------------------------
-
-This package is for Yandex, including:
-
- - `Yandex.Cloud `__
-
-
-Release: 4.0.0
-
-Provider package
-----------------
-
-This package is for the ``yandex`` provider.
-All classes for this package are included in the ``airflow.providers.yandex`` python package.
-
-Installation
-------------
-
-You can install this package on top of an existing Airflow 2 installation via
-``pip install apache-airflow-providers-yandex``.
-For the minimum Airflow version supported, see ``Requirements`` below.
-
-Requirements
-------------
-
-The minimum Apache Airflow version supported by this provider package is ``2.9.0``.
-
-======================= ==================
-PIP package Version required
-======================= ==================
-``apache-airflow`` ``>=2.9.0``
-``yandexcloud`` ``>=0.308.0``
-``yandex-query-client`` ``>=0.1.4``
-======================= ==================
diff --git a/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst b/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst
deleted file mode 100644
index b4e730f4ff21a..0000000000000
--- a/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst
+++ /dev/null
@@ -1,18 +0,0 @@
- .. 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.
-
-.. include:: ../exts/includes/installing-providers-from-sources.rst
diff --git a/docs/apache-airflow-providers-yandex/operators/dataproc.rst b/docs/apache-airflow-providers-yandex/operators/dataproc.rst
deleted file mode 100644
index b7188e2ea52f6..0000000000000
--- a/docs/apache-airflow-providers-yandex/operators/dataproc.rst
+++ /dev/null
@@ -1,37 +0,0 @@
- .. 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.
-
-
-Yandex.Cloud Data Proc Operators
-================================
-
-`Yandex Data Proc `__ is a service
-that helps you deploy Apache Hadoop®* and Apache Spark™ clusters in the Yandex Cloud infrastructure.
-
-With Data Proc, you can manage the cluster size and node capacity,
-as well as work with various Apache® services,
-such as Spark, HDFS, YARN, Hive, HBase, Oozie, Sqoop, Flume, Tez, and Zeppelin.
-
-Apache Hadoop is used for storing and analyzing structured and unstructured big data.
-
-Apache Spark is a tool for quick data processing
-that can be integrated with Apache Hadoop and other storage systems.
-
-Using the operators
-^^^^^^^^^^^^^^^^^^^
-To learn how to use Data Proc operators,
-see `example DAGs `_.
diff --git a/docs/apache-airflow-providers-yandex/operators/index.rst b/docs/apache-airflow-providers-yandex/operators/index.rst
deleted file mode 100644
index 12b05418e100f..0000000000000
--- a/docs/apache-airflow-providers-yandex/operators/index.rst
+++ /dev/null
@@ -1,28 +0,0 @@
- .. 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.
-
-
-
-Yandex.Cloud Operators
-======================
-
-
-.. toctree::
- :maxdepth: 1
- :glob:
-
- *
diff --git a/docs/apache-airflow-providers-yandex/operators/yq.rst b/docs/apache-airflow-providers-yandex/operators/yq.rst
deleted file mode 100644
index 23bd4ac336160..0000000000000
--- a/docs/apache-airflow-providers-yandex/operators/yq.rst
+++ /dev/null
@@ -1,28 +0,0 @@
- .. 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.
-
-
-Yandex Query Operators
-======================
-`Yandex Query `__ is a service in the Yandex Cloud to process data from different sources such as
-`Object Storage `__, `MDB ClickHouse `__,
-`MDB PostgreSQL `__, `Yandex DataStreams `__ using SQL scripts.
-
-Using the operators
-^^^^^^^^^^^^^^^^^^^
-To learn how to use Yandex Query operator,
-see `example DAG `__.
diff --git a/docs/apache-airflow-providers-yandex/secrets-backends/yandex-cloud-lockbox-secret-backend.rst b/docs/apache-airflow-providers-yandex/secrets-backends/yandex-cloud-lockbox-secret-backend.rst
deleted file mode 100644
index f30346b24da80..0000000000000
--- a/docs/apache-airflow-providers-yandex/secrets-backends/yandex-cloud-lockbox-secret-backend.rst
+++ /dev/null
@@ -1,298 +0,0 @@
- .. 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.
-
-
-Yandex.Cloud Lockbox Secret Backend
-===================================
-
-This topic describes how to configure Apache Airflow to use `Yandex Lockbox `__
-as a secret backend and how to manage secrets.
-
-Getting started
----------------
-
-Before you start, make sure you have installed the ``yandex`` provider in your Apache Airflow installation:
-
-.. code-block:: bash
-
- pip install apache-airflow-providers-yandex
-
-Enabling the Yandex Lockbox secret backend
-------------------------------------------
-
-To enable Yandex Lockbox as a secret backend,
-specify :py:class:`~airflow.providers.yandex.secrets.lockbox.LockboxSecretBackend`
-as your ``backend`` in the ``[secrets]`` section of the ``airflow.cfg`` file.
-
-Here is a sample configuration:
-
-.. code-block:: ini
-
- [secrets]
- backend = airflow.providers.yandex.secrets.lockbox.LockboxSecretBackend
-
-You can also set this with an environment variable:
-
-.. code-block:: bash
-
- export AIRFLOW__SECRETS__BACKEND=airflow.providers.yandex.secrets.lockbox.LockboxSecretBackend
-
-You can verify whether the configuration options have been set up correctly
-using the ``airflow config get-value`` command:
-
-.. code-block:: console
-
- $ airflow config get-value secrets backend
- airflow.providers.yandex.secrets.lockbox.LockboxSecretBackend
-
-Backend parameters
-------------------
-
-The next step is to configure backend parameters using the ``backend_kwargs`` options
-that allow you to provide the following parameters:
-
-* ``yc_oauth_token``: Specifies the user account OAuth token to connect to Yandex Lockbox. The parameter value should look like ``y3_xx123``.
-* ``yc_sa_key_json``: Specifies the service account key in JSON. The parameter value should look like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
-* ``yc_sa_key_json_path``: Specifies the service account key in JSON file path. The parameter value should look like ``/home/airflow/authorized_key.json``, while the file content should have the following format: ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
-* ``yc_connection_id``: Specifies the connection ID to connect to Yandex Lockbox. The default value is ``yandexcloud_default``.
-* ``folder_id``: Specifies the folder ID to search for Yandex Lockbox secrets in. If set to ``None`` (``null`` in JSON), the requests will use the connection ``folder_id``, if specified.
-* ``connections_prefix``: Specifies the prefix of the secret to read to get connections. If set to ``None`` (``null`` in JSON), the requests for connections will not be sent to Yandex Lockbox. The default value is ``airflow/connections``.
-* ``variables_prefix``: Specifies the prefix of the secret to read to get variables. If set to ``None`` (``null`` in JSON), the requests for variables will not be sent to Yandex Lockbox. The default value is ``airflow/variables``.
-* ``config_prefix``: Specifies the prefix of the secret to read to get configurations. If set to ``None`` (``null`` in JSON), the requests for variables will not be sent to Yandex Lockbox. The default value is ``airflow/config``.
-* ``sep``: Specifies the separator to concatenate ``secret_prefix`` and ``secret_id``. The default value is ``/``.
-* ``endpoint``: Specifies the API endpoint. If set to ``None`` (``null`` in JSON), the requests will use the connection endpoint, if specified; otherwise, they will use the default endpoint.
-
-Make sure to provide all options as a JSON dictionary.
-
-For example, if you want to set ``connections_prefix`` to ``"example-connections-prefix"``
-and ``variables_prefix`` to ``"example-variables-prefix"``,
-your configuration file should look like this:
-
-.. code-block:: ini
-
- [secrets]
- backend = airflow.providers.yandex.secrets.lockbox.LockboxSecretBackend
- backend_kwargs = {"connections_prefix": "example-connections-prefix", "variables_prefix": "example-variables-prefix"}
-
-Setting up credentials
-----------------------
-
-You need to specify credentials or the ID of the ``yandexcloud`` connection to connect to Yandex Lockbox.
-
-The credentials will be used with the following priority:
-
-* OAuth token
-* Service account key in JSON from file
-* Service account key in JSON
-* Yandex Cloud connection
-
-If you do not specify any credentials, the system will use the default connection ID: ``yandexcloud_default``.
-
-Using an OAuth token to authorize as a user account
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-
-First, you need to create
-an `OAuth token