diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml index 6bb8a9d2193058..6b1160d6f17dbf 100644 --- a/.github/workflows/build-images.yml +++ b/.github/workflows/build-images.yml @@ -204,7 +204,7 @@ jobs: pull-request-target: "true" is-committer-build: ${{ needs.build-info.outputs.is-committer-build }} push-image: "true" - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} image-tag: ${{ needs.build-info.outputs.image-tag }} platform: "linux/amd64" python-versions: ${{ needs.build-info.outputs.python-versions }} @@ -249,7 +249,7 @@ jobs: pull-request-target: "true" is-committer-build: ${{ needs.build-info.outputs.is-committer-build }} push-image: "true" - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} image-tag: ${{ needs.build-info.outputs.image-tag }} platform: linux/amd64 python-versions: ${{ needs.build-info.outputs.python-versions }} diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index acf711c4a2384d..5a0f2ca6106f8f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -206,7 +206,7 @@ jobs: platform: "linux/amd64" python-versions: ${{ needs.build-info.outputs.python-versions }} branch: ${{ needs.build-info.outputs.default-branch }} - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} upgrade-to-newer-dependencies: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }} constraints-branch: ${{ needs.build-info.outputs.default-constraints-branch }} docker-cache: ${{ needs.build-info.outputs.docker-cache }} @@ -272,7 +272,7 @@ jobs: latest-versions-only: ${{ needs.build-info.outputs.latest-versions-only }} include-success-outputs: ${{ needs.build-info.outputs.include-success-outputs }} debug-resources: ${{ needs.build-info.outputs.debug-resources }} - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} generate-constraints: @@ -559,7 +559,7 @@ jobs: default-python-version: ${{ needs.build-info.outputs.default-python-version }} branch: ${{ needs.build-info.outputs.default-branch }} push-image: "true" - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} build-provider-packages: ${{ needs.build-info.outputs.default-branch == 'main' }} upgrade-to-newer-dependencies: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }} chicken-egg-providers: ${{ needs.build-info.outputs.chicken-egg-providers }} @@ -642,7 +642,7 @@ jobs: kubernetes-versions-list-as-string: ${{ needs.build-info.outputs.kubernetes-versions-list-as-string }} kubernetes-combos-list-as-string: ${{ needs.build-info.outputs.kubernetes-combos-list-as-string }} include-success-outputs: ${{ needs.build-info.outputs.include-success-outputs }} - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} debug-resources: ${{ needs.build-info.outputs.debug-resources }} if: > ( needs.build-info.outputs.run-kubernetes-tests == 'true' || diff --git a/.github/workflows/finalize-tests.yml b/.github/workflows/finalize-tests.yml index c948984ee10c56..6f9bc74168b429 100644 --- a/.github/workflows/finalize-tests.yml +++ b/.github/workflows/finalize-tests.yml @@ -149,7 +149,7 @@ jobs: python-versions: ${{ inputs.python-versions }} branch: ${{ inputs.branch }} constraints-branch: ${{ inputs.constraints-branch }} - use-uv: ${{ needs.build-info.outputs.force-pip && 'false' || 'true' }} + use-uv: ${{ needs.build-info.outputs.force-pip == 'true' && 'false' || 'true' }} include-success-outputs: ${{ inputs.include-success-outputs }} docker-cache: ${{ inputs.docker-cache }} disable-airflow-repo-cache: ${{ inputs.disable-airflow-repo-cache }} diff --git a/.github/workflows/k8s-tests.yml b/.github/workflows/k8s-tests.yml index 9a764e88c4e99f..3b3e067038db9b 100644 --- a/.github/workflows/k8s-tests.yml +++ b/.github/workflows/k8s-tests.yml @@ -101,7 +101,7 @@ jobs: k8s-env-${{ steps.breeze.outputs.host-python-version }}-\ ${{ hashFiles('scripts/ci/kubernetes/k8s_requirements.txt','hatch_build.py') }}" - name: "Switch breeze to use uv" - run: breeze setup-config --use-uv + run: breeze setup config --use-uv if: inputs.use-uv == 'true' - name: Run complete K8S tests ${{ inputs.kubernetes-combos-list-as-string }} run: breeze k8s run-complete-tests --run-in-parallel --upgrade --no-copy-local-sources diff --git a/Dockerfile.ci b/Dockerfile.ci index 666344e285257a..6ddf2f4e1ac4d8 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1017,6 +1017,13 @@ function determine_airflow_to_use() { --constraint https://raw.githubusercontent.com/apache/airflow/constraints-main/constraints-${PYTHON_MAJOR_MINOR_VERSION}.txt # Some packages might leave legacy typing module which causes test issues pip uninstall -y typing || true + if [[ ${LINK_PROVIDERS_TO_AIRFLOW_PACKAGE=} == "true" ]]; then + echo + echo "${COLOR_BLUE}Linking providers to airflow package as we are using them from mounted sources.${COLOR_RESET}" + echo + rm -rf /usr/local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/providers + ln -s "${AIRFLOW_SOURCES}/providers/src/airflow/providers" "/usr/local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/providers" + fi fi if [[ "${USE_AIRFLOW_VERSION}" =~ ^2\.2\..*|^2\.1\..*|^2\.0\..* && "${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN=}" != "" ]]; then diff --git a/airflow/api_connexion/endpoints/backfill_endpoint.py b/airflow/api_connexion/endpoints/backfill_endpoint.py deleted file mode 100644 index b53d461dc5d1e1..00000000000000 --- a/airflow/api_connexion/endpoints/backfill_endpoint.py +++ /dev/null @@ -1,183 +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. - -from __future__ import annotations - -import logging -from functools import wraps -from typing import TYPE_CHECKING - -from flask import request -from marshmallow import ValidationError -from sqlalchemy import select - -from airflow.api_connexion import security -from airflow.api_connexion.exceptions import BadRequest, Conflict, NotFound -from airflow.api_connexion.schemas.backfill_schema import ( - BackfillCollection, - backfill_collection_schema, - backfill_schema, -) -from airflow.models.backfill import ( - AlreadyRunningBackfill, - Backfill, - ReprocessBehavior, - _cancel_backfill, - _create_backfill, -) -from airflow.utils.session import NEW_SESSION, provide_session -from airflow.www.decorators import action_logging - -if TYPE_CHECKING: - from datetime import datetime - - from sqlalchemy.orm import Session - - from airflow.api_connexion.types import APIResponse - -log = logging.getLogger(__name__) - -RESOURCE_EVENT_PREFIX = "dag" - - -def backfill_to_dag(func): - """ - Enrich the request with dag_id. - - :meta private: - """ - - @wraps(func) - def wrapper(*, backfill_id, session, **kwargs): - backfill = session.get(Backfill, backfill_id) - if not backfill: - raise NotFound("Backfill not found") - return func(dag_id=backfill.dag_id, backfill_id=backfill_id, session=session, **kwargs) - - return wrapper - - -@security.requires_access_dag("GET") -@action_logging -@provide_session -def list_backfills(dag_id, session): - backfills = session.scalars(select(Backfill).where(Backfill.dag_id == dag_id)).all() - obj = BackfillCollection( - backfills=backfills, - total_entries=len(backfills), - ) - return backfill_collection_schema.dump(obj) - - -@provide_session -@backfill_to_dag -@security.requires_access_dag("PUT") -@action_logging -def pause_backfill(*, backfill_id, session, **kwargs): - br = session.get(Backfill, backfill_id) - if br.completed_at: - raise Conflict("Backfill is already completed.") - if br.is_paused is False: - br.is_paused = True - session.commit() - return backfill_schema.dump(br) - - -@provide_session -@backfill_to_dag -@security.requires_access_dag("PUT") -@action_logging -def unpause_backfill(*, backfill_id, session, **kwargs): - br = session.get(Backfill, backfill_id) - if br.completed_at: - raise Conflict("Backfill is already completed.") - if br.is_paused: - br.is_paused = False - session.commit() - return backfill_schema.dump(br) - - -@provide_session -@backfill_to_dag -@security.requires_access_dag("GET") -@action_logging -def get_backfill(*, backfill_id: int, session: Session = NEW_SESSION, **kwargs): - backfill = session.get(Backfill, backfill_id) - if backfill: - return backfill_schema.dump(backfill) - raise NotFound("Backfill not found") - - -def backfill_obj_to_kwargs(f): - """ - Convert the request body (containing backfill object json) to kwargs. - - The main point here is to be compatible with the ``requires_access_dag`` decorator, - which takes dag_id kwarg and doesn't support json request body. - """ - - @wraps(f) - def inner(): - body = request.json - try: - obj = backfill_schema.load(body) - except ValidationError as err: - raise BadRequest(detail=str(err.messages)) - return f(**obj) - - return inner - - -@backfill_obj_to_kwargs -@security.requires_access_dag("PUT") -@action_logging -def create_backfill( - dag_id: str, - from_date: datetime, - to_date: datetime, - max_active_runs: int = 10, - reverse: bool = False, - dag_run_conf: dict | None = None, - reprocess_behavior: ReprocessBehavior | None = None, -) -> APIResponse: - try: - backfill_obj = _create_backfill( - dag_id=dag_id, - from_date=from_date, - to_date=to_date, - max_active_runs=max_active_runs, - reverse=reverse, - dag_run_conf=dag_run_conf, - reprocess_behavior=reprocess_behavior, - ) - return backfill_schema.dump(backfill_obj) - except AlreadyRunningBackfill: - raise Conflict(f"There is already a running backfill for dag {dag_id}") - - -@provide_session -@backfill_to_dag -@security.requires_access_dag("PUT") -@action_logging -def cancel_backfill( - *, - backfill_id, - session: Session = NEW_SESSION, # used by backfill_to_dag decorator - **kwargs, -): - br = _cancel_backfill(backfill_id=backfill_id) - return backfill_schema.dump(br) diff --git a/airflow/api_connexion/endpoints/dag_warning_endpoint.py b/airflow/api_connexion/endpoints/dag_warning_endpoint.py index 8a15a30cece8ff..a158c3f443c87c 100644 --- a/airflow/api_connexion/endpoints/dag_warning_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_warning_endpoint.py @@ -29,6 +29,7 @@ from airflow.api_connexion.security import get_readable_dags from airflow.auth.managers.models.resource_details import DagAccessEntity from airflow.models.dagwarning import DagWarning as DagWarningModel +from airflow.utils.api_migration import mark_fastapi_migration_done from airflow.utils.db import get_query_count from airflow.utils.session import NEW_SESSION, provide_session @@ -38,6 +39,7 @@ from airflow.api_connexion.types import APIResponse +@mark_fastapi_migration_done @security.requires_access_dag("GET", DagAccessEntity.WARNING) @format_parameters({"limit": check_limit}) @provide_session diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 0ec0a3cdcb68ea..c884c15954116b 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -245,155 +245,6 @@ servers: description: Apache Airflow Stable API. paths: - # Database entities - /backfills: - get: - summary: List backfills - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: list_backfills - tags: [Backfill] - parameters: - - name: dag_id - in: query - schema: - type: string - required: true - description: | - List backfills for this dag. - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/BackfillCollection" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - - post: - summary: Create a backfill job. - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: create_backfill - tags: [Backfill] - requestBody: - required: true - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - "400": - $ref: "#/components/responses/BadRequest" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - - /backfills/{backfill_id}: - parameters: - - $ref: "#/components/parameters/BackfillIdPath" - get: - summary: Get a backfill - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: get_backfill - tags: [Backfill] - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - "404": - $ref: "#/components/responses/NotFound" - - /backfills/{backfill_id}/pause: - parameters: - - $ref: "#/components/parameters/BackfillIdPath" - post: - summary: Pause a backfill - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: pause_backfill - tags: [Backfill] - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - "404": - $ref: "#/components/responses/NotFound" - "409": - $ref: "#/components/responses/Conflict" - - /backfills/{backfill_id}/unpause: - parameters: - - $ref: "#/components/parameters/BackfillIdPath" - post: - summary: Pause a backfill - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: unpause_backfill - tags: [Backfill] - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - "404": - $ref: "#/components/responses/NotFound" - "409": - $ref: "#/components/responses/Conflict" - - /backfills/{backfill_id}/cancel: - parameters: - - $ref: "#/components/parameters/BackfillIdPath" - post: - summary: Cancel a backfill - description: | - When a backfill is cancelled, all queued dag runs will be marked as failed. - Running dag runs will be allowed to continue. - x-openapi-router-controller: airflow.api_connexion.endpoints.backfill_endpoint - operationId: cancel_backfill - tags: [Backfill] - responses: - "200": - description: Success. - content: - application/json: - schema: - $ref: "#/components/schemas/Backfill" - "401": - $ref: "#/components/responses/Unauthenticated" - "403": - $ref: "#/components/responses/PermissionDenied" - "404": - $ref: "#/components/responses/NotFound" - "409": - $ref: "#/components/responses/Conflict" - # Database entities /connections: get: diff --git a/airflow/api_connexion/schemas/backfill_schema.py b/airflow/api_connexion/schemas/backfill_schema.py deleted file mode 100644 index 9f71984691574e..00000000000000 --- a/airflow/api_connexion/schemas/backfill_schema.py +++ /dev/null @@ -1,100 +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. -from __future__ import annotations - -import typing -from typing import NamedTuple - -from marshmallow import Schema, fields, utils, validate -from marshmallow_sqlalchemy import SQLAlchemySchema, auto_field - -from airflow.models.backfill import Backfill, BackfillDagRun, ReprocessBehavior - - -class ReprocessBehaviorField(fields.String): - """Schema for ReprocessBehavior enum.""" - - def __init__(self, **metadata): - super().__init__(**metadata) - self.validators = [validate.OneOf(ReprocessBehavior), *self.validators] - - def _serialize(self, value, attr, obj, **kwargs) -> str | None: - if value is None: - return None - return utils.ensure_text_type(ReprocessBehavior(value).value) - - def _deserialize(self, value, attr, data, **kwargs) -> typing.Any: - deser = super()._deserialize(value, attr, data, **kwargs) - if not deser: - return None - return ReprocessBehavior(deser) - - -class BackfillSchema(SQLAlchemySchema): - """Backfill Schema.""" - - class Meta: - """Meta.""" - - model = Backfill - - id = auto_field(dump_only=True) - dag_id = auto_field() - from_date = auto_field() - to_date = auto_field() - dag_run_conf = fields.Dict(allow_none=True) - reverse = fields.Boolean() - is_paused = auto_field() - reprocess_behavior = ReprocessBehaviorField() - max_active_runs = auto_field() - created_at = auto_field() - completed_at = auto_field() - updated_at = auto_field() - - -class BackfillDagRunSchema(SQLAlchemySchema): - """Trigger Schema.""" - - class Meta: - """Meta.""" - - model = BackfillDagRun - - id = auto_field(dump_only=True) - backfill_id = auto_field(dump_only=True) - dag_run_id = auto_field(dump_only=True) - sort_ordinal = auto_field(dump_only=True) - - -class BackfillCollection(NamedTuple): - """List of Backfills with meta.""" - - backfills: list[Backfill] - total_entries: int - - -class BackfillCollectionSchema(Schema): - """Backfill Collection Schema.""" - - backfills = fields.List(fields.Nested(BackfillSchema)) - total_entries = fields.Int() - - -backfill_schema = BackfillSchema() -backfill_dag_run_schema = BackfillDagRunSchema() -backfill_collection_schema = BackfillCollectionSchema() diff --git a/airflow/api_fastapi/common/parameters.py b/airflow/api_fastapi/common/parameters.py index 7137a0a124847a..bd650176372278 100644 --- a/airflow/api_fastapi/common/parameters.py +++ b/airflow/api_fastapi/common/parameters.py @@ -31,6 +31,7 @@ from airflow.models import Base, Connection from airflow.models.dag import DagModel, DagTag from airflow.models.dagrun import DagRun +from airflow.models.dagwarning import DagWarning, DagWarningType from airflow.utils import timezone from airflow.utils.state import DagRunState @@ -292,6 +293,34 @@ def _safe_parse_datetime(date_to_check: str) -> datetime: ) +class _WarningTypeFilter(BaseParam[str]): + """Filter on warning type.""" + + def to_orm(self, select: Select) -> Select: + if self.value is None and self.skip_none: + return select + return select.where(DagWarning.warning_type == self.value) + + def depends(self, warning_type: DagWarningType | None = None) -> _WarningTypeFilter: + return self.set_value(warning_type) + + +class _DagIdFilter(BaseParam[str]): + """Filter on dag_id.""" + + def __init__(self, attribute: ColumnElement, skip_none: bool = True) -> None: + super().__init__(skip_none) + self.attribute = attribute + + def to_orm(self, select: Select) -> Select: + if self.value is None and self.skip_none: + return select + return select.where(self.attribute == self.value) + + def depends(self, dag_id: str | None = None) -> _DagIdFilter: + return self.set_value(dag_id) + + # Common Safe DateTime DateTimeQuery = Annotated[str, AfterValidator(_safe_parse_datetime)] # DAG @@ -310,5 +339,8 @@ def _safe_parse_datetime(date_to_check: str) -> datetime: QueryOwnersFilter = Annotated[_OwnersFilter, Depends(_OwnersFilter().depends)] # DagRun QueryLastDagRunStateFilter = Annotated[_LastDagRunStateFilter, Depends(_LastDagRunStateFilter().depends)] +# DAGWarning +QueryDagIdInDagWarningFilter = Annotated[_DagIdFilter, Depends(_DagIdFilter(DagWarning.dag_id).depends)] +QueryWarningTypeFilter = Annotated[_WarningTypeFilter, Depends(_WarningTypeFilter().depends)] # DAGTags QueryDagTagPatternSearch = Annotated[_DagTagNamePatternSearch, Depends(_DagTagNamePatternSearch().depends)] diff --git a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml index b7e7f626937191..abd7f0baf5edfc 100644 --- a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml +++ b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml @@ -170,67 +170,46 @@ paths: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - /public/connections/{connection_id}: - delete: + /public/backfills/: + get: tags: - - Connection - summary: Delete Connection - description: Delete a connection entry. - operationId: delete_connection + - Backfill + summary: List Backfills + operationId: list_backfills parameters: - - name: connection_id - in: path + - name: dag_id + in: query required: true schema: type: string - title: Connection Id - responses: - '204': - description: Successful Response - '401': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Unauthorized - '403': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Forbidden - '404': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Not Found - '422': - description: Validation Error - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPValidationError' - get: - tags: - - Connection - summary: Get Connection - description: Get a connection entry. - operationId: get_connection - parameters: - - name: connection_id - in: path - required: true + title: Dag Id + - name: limit + in: query + required: false + schema: + type: integer + default: 100 + title: Limit + - name: offset + in: query + required: false + schema: + type: integer + default: 0 + title: Offset + - name: order_by + in: query + required: false schema: type: string - title: Connection Id + default: id + title: Order By responses: '200': description: Successful Response content: application/json: - schema: - $ref: '#/components/schemas/ConnectionResponse' + schema: {} '401': content: application/json: @@ -243,54 +222,29 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Forbidden - '404': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Not Found '422': description: Validation Error content: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - /public/connections/: - get: + post: tags: - - Connection - summary: Get Connections - description: Get all connection entries. - operationId: get_connections - parameters: - - name: limit - in: query - required: false - schema: - type: integer - default: 100 - title: Limit - - name: offset - in: query - required: false - schema: - type: integer - default: 0 - title: Offset - - name: order_by - in: query - required: false - schema: - type: string - default: id - title: Order By + - Backfill + summary: Create Backfill + operationId: create_backfill + requestBody: + required: true + content: + application/json: + schema: + $ref: '#/components/schemas/BackfillPostBody' responses: '200': description: Successful Response content: application/json: - schema: - $ref: '#/components/schemas/ConnectionCollectionResponse' + schema: {} '401': content: application/json: @@ -309,38 +263,37 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found + '409': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Conflict '422': description: Validation Error content: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - /public/dags/{dag_id}/dagRuns/{dag_run_id}: + /public/backfills/{backfill_id}: get: tags: - - DagRun - summary: Get Dag Run - operationId: get_dag_run + - Backfill + summary: Get Backfill + operationId: get_backfill parameters: - - name: dag_id - in: path - required: true - schema: - type: string - title: Dag Id - - name: dag_run_id + - name: backfill_id in: path required: true schema: type: string - title: Dag Run Id + title: Backfill Id responses: '200': description: Successful Response content: application/json: - schema: - $ref: '#/components/schemas/DAGRunResponse' + schema: {} '401': content: application/json: @@ -365,34 +318,24 @@ paths: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - delete: + /public/backfills/{backfill_id}/pause: + put: tags: - - DagRun - summary: Delete Dag Run - description: Delete a DAG Run entry. - operationId: delete_dag_run + - Backfill + summary: Pause Backfill + operationId: pause_backfill parameters: - - name: dag_id - in: path - required: true - schema: - type: string - title: Dag Id - - name: dag_run_id + - name: backfill_id in: path required: true schema: - type: string - title: Dag Run Id + title: Backfill Id responses: - '204': + '200': description: Successful Response - '400': content: application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Bad Request + schema: {} '401': content: application/json: @@ -411,60 +354,36 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found + '409': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Conflict '422': description: Validation Error content: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - patch: + /public/backfills/{backfill_id}/unpause: + put: tags: - - DagRun - summary: Patch Dag Run State - description: Modify a DAG Run. - operationId: patch_dag_run_state + - Backfill + summary: Unpause Backfill + operationId: unpause_backfill parameters: - - name: dag_id - in: path - required: true - schema: - type: string - title: Dag Id - - name: dag_run_id + - name: backfill_id in: path required: true schema: - type: string - title: Dag Run Id - - name: update_mask - in: query - required: false - schema: - anyOf: - - type: array - items: - type: string - - type: 'null' - title: Update Mask - requestBody: - required: true - content: - application/json: - schema: - $ref: '#/components/schemas/DAGRunPatchBody' + title: Backfill Id responses: '200': description: Successful Response content: application/json: - schema: - $ref: '#/components/schemas/DAGRunResponse' - '400': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Bad Request + schema: {} '401': content: application/json: @@ -483,50 +402,36 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found + '409': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Conflict '422': description: Validation Error content: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - /public/dagSources/{file_token}: - get: + /public/backfills/{backfill_id}/cancel: + put: tags: - - DagSource - summary: Get Dag Source - description: Get source code using file token. - operationId: get_dag_source + - Backfill + summary: Cancel Backfill + operationId: cancel_backfill parameters: - - name: file_token + - name: backfill_id in: path required: true schema: - type: string - title: File Token - - name: accept - in: header - required: false - schema: - type: string - default: '*/*' - title: Accept + title: Backfill Id responses: '200': description: Successful Response content: application/json: - schema: - $ref: '#/components/schemas/DAGSourceResponse' - text/plain: - schema: - type: string - example: dag code - '400': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Bad Request + schema: {} '401': content: application/json: @@ -545,12 +450,12 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found - '406': + '409': content: application/json: schema: $ref: '#/components/schemas/HTTPExceptionResponse' - description: Not Acceptable + description: Conflict '422': description: Validation Error content: @@ -830,20 +735,381 @@ paths: application/json: schema: $ref: '#/components/schemas/HTTPExceptionResponse' - description: Forbidden + description: Forbidden + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' + /public/dags/{dag_id}: + get: + tags: + - DAG + summary: Get Dag + description: Get basic information about a DAG. + operationId: get_dag + parameters: + - name: dag_id + in: path + required: true + schema: + type: string + title: Dag Id + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/DAGResponse' + '400': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Bad Request + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unprocessable Entity + patch: + tags: + - DAG + summary: Patch Dag + description: Patch the specific DAG. + operationId: patch_dag + parameters: + - name: dag_id + in: path + required: true + schema: + type: string + title: Dag Id + - name: update_mask + in: query + required: false + schema: + anyOf: + - type: array + items: + type: string + - type: 'null' + title: Update Mask + requestBody: + required: true + content: + application/json: + schema: + $ref: '#/components/schemas/DAGPatchBody' + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/DAGResponse' + '400': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Bad Request + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' + delete: + tags: + - DAG + summary: Delete Dag + description: Delete the specific DAG. + operationId: delete_dag + parameters: + - name: dag_id + in: path + required: true + schema: + type: string + title: Dag Id + responses: + '200': + description: Successful Response + content: + application/json: + schema: {} + '400': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Bad Request + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unprocessable Entity + /public/dags/{dag_id}/details: + get: + tags: + - DAG + summary: Get Dag Details + description: Get details of DAG. + operationId: get_dag_details + parameters: + - name: dag_id + in: path + required: true + schema: + type: string + title: Dag Id + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/DAGDetailsResponse' + '400': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Bad Request + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unprocessable Entity + /public/connections/{connection_id}: + delete: + tags: + - Connection + summary: Delete Connection + description: Delete a connection entry. + operationId: delete_connection + parameters: + - name: connection_id + in: path + required: true + schema: + type: string + title: Connection Id + responses: + '204': + description: Successful Response + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' + get: + tags: + - Connection + summary: Get Connection + description: Get a connection entry. + operationId: get_connection + parameters: + - name: connection_id + in: path + required: true + schema: + type: string + title: Connection Id + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/ConnectionResponse' + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' + /public/connections/: + get: + tags: + - Connection + summary: Get Connections + description: Get all connection entries. + operationId: get_connections + parameters: + - name: limit + in: query + required: false + schema: + type: integer + default: 100 + title: Limit + - name: offset + in: query + required: false + schema: + type: integer + default: 0 + title: Offset + - name: order_by + in: query + required: false + schema: + type: string + default: id + title: Order By + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/ConnectionCollectionResponse' + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found '422': description: Validation Error content: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - /public/dags/{dag_id}: + /public/dags/{dag_id}/dagRuns/{dag_run_id}: get: tags: - - DAG - summary: Get Dag - description: Get basic information about a DAG. - operationId: get_dag + - DagRun + summary: Get Dag Run + operationId: get_dag_run parameters: - name: dag_id in: path @@ -851,19 +1117,19 @@ paths: schema: type: string title: Dag Id + - name: dag_run_id + in: path + required: true + schema: + type: string + title: Dag Run Id responses: '200': description: Successful Response content: application/json: schema: - $ref: '#/components/schemas/DAGResponse' - '400': - content: - application/json: - schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Bad Request + $ref: '#/components/schemas/DAGRunResponse' '401': content: application/json: @@ -883,17 +1149,17 @@ paths: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found '422': + description: Validation Error content: application/json: schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Unprocessable Entity - patch: + $ref: '#/components/schemas/HTTPValidationError' + delete: tags: - - DAG - summary: Patch Dag - description: Patch the specific DAG. - operationId: patch_dag + - DagRun + summary: Delete Dag Run + description: Delete a DAG Run entry. + operationId: delete_dag_run parameters: - name: dag_id in: path @@ -901,29 +1167,15 @@ paths: schema: type: string title: Dag Id - - name: update_mask - in: query - required: false - schema: - anyOf: - - type: array - items: - type: string - - type: 'null' - title: Update Mask - requestBody: + - name: dag_run_id + in: path required: true - content: - application/json: - schema: - $ref: '#/components/schemas/DAGPatchBody' + schema: + type: string + title: Dag Run Id responses: - '200': + '204': description: Successful Response - content: - application/json: - schema: - $ref: '#/components/schemas/DAGResponse' '400': content: application/json: @@ -954,12 +1206,12 @@ paths: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' - delete: + patch: tags: - - DAG - summary: Delete Dag - description: Delete the specific DAG. - operationId: delete_dag + - DagRun + summary: Patch Dag Run State + description: Modify a DAG Run. + operationId: patch_dag_run_state parameters: - name: dag_id in: path @@ -967,12 +1219,35 @@ paths: schema: type: string title: Dag Id + - name: dag_run_id + in: path + required: true + schema: + type: string + title: Dag Run Id + - name: update_mask + in: query + required: false + schema: + anyOf: + - type: array + items: + type: string + - type: 'null' + title: Update Mask + requestBody: + required: true + content: + application/json: + schema: + $ref: '#/components/schemas/DAGRunPatchBody' responses: '200': description: Successful Response content: application/json: - schema: {} + schema: + $ref: '#/components/schemas/DAGRunResponse' '400': content: application/json: @@ -998,32 +1273,43 @@ paths: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found '422': + description: Validation Error content: application/json: schema: - $ref: '#/components/schemas/HTTPExceptionResponse' - description: Unprocessable Entity - /public/dags/{dag_id}/details: + $ref: '#/components/schemas/HTTPValidationError' + /public/dagSources/{file_token}: get: tags: - - DAG - summary: Get Dag Details - description: Get details of DAG. - operationId: get_dag_details + - DagSource + summary: Get Dag Source + description: Get source code using file token. + operationId: get_dag_source parameters: - - name: dag_id + - name: file_token in: path required: true schema: type: string - title: Dag Id + title: File Token + - name: accept + in: header + required: false + schema: + type: string + default: '*/*' + title: Accept responses: '200': description: Successful Response content: application/json: schema: - $ref: '#/components/schemas/DAGDetailsResponse' + $ref: '#/components/schemas/DAGSourceResponse' + text/plain: + schema: + type: string + example: dag code '400': content: application/json: @@ -1048,12 +1334,18 @@ paths: schema: $ref: '#/components/schemas/HTTPExceptionResponse' description: Not Found - '422': + '406': content: application/json: schema: $ref: '#/components/schemas/HTTPExceptionResponse' - description: Unprocessable Entity + description: Not Acceptable + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' /public/eventLogs/{event_log_id}: get: tags: @@ -1111,6 +1403,76 @@ paths: application/json: schema: $ref: '#/components/schemas/HealthInfoSchema' + /public/dagWarnings: + get: + tags: + - DagWarning + summary: List Dag Warnings + description: Get a list of DAG warnings. + operationId: list_dag_warnings + parameters: + - name: dag_id + in: query + required: false + schema: + anyOf: + - type: string + - type: 'null' + title: Dag Id + - name: warning_type + in: query + required: false + schema: + anyOf: + - $ref: '#/components/schemas/DagWarningType' + - type: 'null' + title: Warning Type + - name: limit + in: query + required: false + schema: + type: integer + default: 100 + title: Limit + - name: offset + in: query + required: false + schema: + type: integer + default: 0 + title: Offset + - name: order_by + in: query + required: false + schema: + type: string + default: dag_id + title: Order By + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/DAGWarningCollectionResponse' + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' /public/plugins/: get: tags: @@ -1859,6 +2221,41 @@ components: type: object title: AppBuilderViewResponse description: Serializer for AppBuilder View responses. + BackfillPostBody: + properties: + dag_id: + type: string + title: Dag Id + from_date: + type: string + format: date-time + title: From Date + to_date: + type: string + format: date-time + title: To Date + run_backwards: + type: boolean + title: Run Backwards + default: false + dag_run_conf: + type: object + title: Dag Run Conf + default: {} + reprocess_behavior: + $ref: '#/components/schemas/ReprocessBehavior' + default: none + max_active_runs: + type: integer + title: Max Active Runs + default: 10 + type: object + required: + - dag_id + - from_date + - to_date + title: BackfillPostBody + description: Object used for create backfill request. BaseInfoSchema: properties: status: @@ -2505,6 +2902,44 @@ components: - total_entries title: DAGTagCollectionResponse description: DAG Tags Collection serializer for responses. + DAGWarningCollectionResponse: + properties: + dag_warnings: + items: + $ref: '#/components/schemas/DAGWarningResponse' + type: array + title: Dag Warnings + total_entries: + type: integer + title: Total Entries + type: object + required: + - dag_warnings + - total_entries + title: DAGWarningCollectionResponse + description: DAG warning collection serializer for responses. + DAGWarningResponse: + properties: + dag_id: + type: string + title: Dag Id + warning_type: + $ref: '#/components/schemas/DagWarningType' + message: + type: string + title: Message + timestamp: + type: string + format: date-time + title: Timestamp + type: object + required: + - dag_id + - warning_type + - message + - timestamp + title: DAGWarningResponse + description: DAG Warning serializer for responses. DAGWithLatestDagRunsCollectionResponse: properties: total_entries: @@ -2744,6 +3179,18 @@ components: title: DagTagPydantic description: Serializable representation of the DagTag ORM SqlAlchemyModel used by internal API. + DagWarningType: + type: string + enum: + - asset conflict + - non-existent pool + title: DagWarningType + description: 'Enum for DAG warning types. + + + This is the set of allowable values for the ``warning_type`` field + + in the DagWarning model.' EventLogResponse: properties: event_log_id: @@ -3189,6 +3636,17 @@ components: - version title: ProviderResponse description: Provider serializer for responses. + ReprocessBehavior: + type: string + enum: + - failed + - completed + - none + title: ReprocessBehavior + description: 'Internal enum for setting reprocess behavior in a backfill. + + + :meta private:' SchedulerInfoSchema: properties: status: @@ -3209,6 +3667,9 @@ components: description: Schema for Scheduler info. TaskInstanceResponse: properties: + id: + type: string + title: Id task_id: type: string title: Task Id @@ -3329,6 +3790,7 @@ components: - type: 'null' type: object required: + - id - task_id - dag_id - dag_run_id diff --git a/airflow/api_fastapi/core_api/routes/public/__init__.py b/airflow/api_fastapi/core_api/routes/public/__init__.py index cc9dd9c5e1ba1b..a443f5a28ae8cd 100644 --- a/airflow/api_fastapi/core_api/routes/public/__init__.py +++ b/airflow/api_fastapi/core_api/routes/public/__init__.py @@ -18,9 +18,11 @@ from __future__ import annotations from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.core_api.routes.public.backfills import backfills_router from airflow.api_fastapi.core_api.routes.public.connections import connections_router from airflow.api_fastapi.core_api.routes.public.dag_run import dag_run_router from airflow.api_fastapi.core_api.routes.public.dag_sources import dag_sources_router +from airflow.api_fastapi.core_api.routes.public.dag_warning import dag_warning_router from airflow.api_fastapi.core_api.routes.public.dags import dags_router from airflow.api_fastapi.core_api.routes.public.event_logs import event_logs_router from airflow.api_fastapi.core_api.routes.public.monitor import monitor_router @@ -34,12 +36,15 @@ public_router = AirflowRouter(prefix="/public") +public_router.include_router(backfills_router) +public_router.include_router(dags_router) public_router.include_router(connections_router) public_router.include_router(dag_run_router) public_router.include_router(dag_sources_router) public_router.include_router(dags_router) public_router.include_router(event_logs_router) public_router.include_router(monitor_router) +public_router.include_router(dag_warning_router) public_router.include_router(plugins_router) public_router.include_router(pools_router) public_router.include_router(providers_router) diff --git a/airflow/api_fastapi/core_api/routes/public/backfills.py b/airflow/api_fastapi/core_api/routes/public/backfills.py new file mode 100644 index 00000000000000..e9e93673b8af91 --- /dev/null +++ b/airflow/api_fastapi/core_api/routes/public/backfills.py @@ -0,0 +1,217 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +from fastapi import Depends, HTTPException, status +from sqlalchemy import select, update +from sqlalchemy.orm import Session +from typing_extensions import Annotated + +from airflow.api_fastapi.common.db.common import get_session, paginated_select +from airflow.api_fastapi.common.parameters import QueryLimit, QueryOffset, SortParam +from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.core_api.openapi.exceptions import ( + create_openapi_http_exception_doc, +) +from airflow.api_fastapi.core_api.serializers.backfills import ( + BackfillCollectionResponse, + BackfillPostBody, + BackfillResponse, +) +from airflow.models import DagRun +from airflow.models.backfill import ( + AlreadyRunningBackfill, + Backfill, + BackfillDagRun, + _create_backfill, +) +from airflow.utils import timezone +from airflow.utils.state import DagRunState + +backfills_router = AirflowRouter(tags=["Backfill"], prefix="/backfills") + + +@backfills_router.get( + path="/", + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), +) +async def list_backfills( + dag_id: str, + limit: QueryLimit, + offset: QueryOffset, + order_by: Annotated[ + SortParam, + Depends(SortParam(["id"], Backfill).dynamic_depends()), + ], + session: Annotated[Session, Depends(get_session)], +): + select_stmt, total_entries = paginated_select( + select(Backfill).where(Backfill.dag_id == dag_id), + [], + order_by=order_by, + offset=offset, + limit=limit, + session=session, + ) + backfills = session.scalars(select_stmt).all() + + return BackfillCollectionResponse( + backfills=[BackfillResponse.model_validate(x, from_attributes=True) for x in backfills], + total_entries=len(backfills), + ) + + +@backfills_router.get( + path="/{backfill_id}", + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), +) +async def get_backfill( + backfill_id: str, + session: Annotated[Session, Depends(get_session)], +): + backfill = session.get(Backfill, backfill_id) + if backfill: + return BackfillResponse.model_validate(backfill, from_attributes=True) + raise HTTPException(status.HTTP_404_NOT_FOUND, "Backfill not found") + + +@backfills_router.put( + path="/{backfill_id}/pause", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_409_CONFLICT, + ] + ), +) +async def pause_backfill(*, backfill_id, session: Annotated[Session, Depends(get_session)]): + b = session.get(Backfill, backfill_id) + if not b: + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}") + if b.completed_at: + raise HTTPException(status.HTTP_409_CONFLICT, "Backfill is already completed.") + if b.is_paused is False: + b.is_paused = True + session.commit() + return BackfillResponse.model_validate(b, from_attributes=True) + + +@backfills_router.put( + path="/{backfill_id}/unpause", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_409_CONFLICT, + ] + ), +) +async def unpause_backfill(*, backfill_id, session: Annotated[Session, Depends(get_session)]): + b = session.get(Backfill, backfill_id) + if not b: + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}") + if b.completed_at: + raise HTTPException(status.HTTP_409_CONFLICT, "Backfill is already completed.") + if b.is_paused: + b.is_paused = False + return BackfillResponse.model_validate(b, from_attributes=True) + + +@backfills_router.put( + path="/{backfill_id}/cancel", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_409_CONFLICT, + ] + ), +) +async def cancel_backfill(*, backfill_id, session: Annotated[Session, Depends(get_session)]): + b: Backfill = session.get(Backfill, backfill_id) + if not b: + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Could not find backfill with id {backfill_id}") + if b.completed_at is not None: + raise HTTPException(status.HTTP_409_CONFLICT, "Backfill is already completed.") + + # first, pause, and commit immediately to ensure no other dag runs are started + if not b.is_paused: + b.is_paused = True + session.commit() # ensure no new runs started + + query = ( + update(DagRun) + .where( + DagRun.id.in_( + select( + BackfillDagRun.dag_run_id, + ).where( + BackfillDagRun.backfill_id == b.id, + ), + ), + DagRun.state == DagRunState.QUEUED, + ) + .values(state=DagRunState.FAILED) + .execution_options(synchronize_session=False) + ) + session.execute(query) + session.commit() # this will fail all the queued dag runs in this backfill + + # this is in separate transaction just to avoid potential conflicts + session.refresh(b) + b.completed_at = timezone.utcnow() + return BackfillResponse.model_validate(b, from_attributes=True) + + +@backfills_router.post( + path="/", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_409_CONFLICT, + ] + ), +) +async def create_backfill( + backfill_request: BackfillPostBody, +): + from_date = timezone.coerce_datetime(backfill_request.from_date) + to_date = timezone.coerce_datetime(backfill_request.to_date) + try: + backfill_obj = _create_backfill( + dag_id=backfill_request.dag_id, + from_date=from_date, + to_date=to_date, + max_active_runs=backfill_request.max_active_runs, + reverse=backfill_request.run_backwards, + dag_run_conf=backfill_request.dag_run_conf, + reprocess_behavior=backfill_request.reprocess_behavior, + ) + return BackfillResponse.model_validate(backfill_obj, from_attributes=True) + except AlreadyRunningBackfill: + raise HTTPException( + status_code=status.HTTP_409_CONFLICT, + detail=f"There is already a running backfill for dag {backfill_request.dag_id}", + ) diff --git a/airflow/api_fastapi/core_api/routes/public/connections.py b/airflow/api_fastapi/core_api/routes/public/connections.py index 60898d2a63a6f6..8d9f9ddb8ebfce 100644 --- a/airflow/api_fastapi/core_api/routes/public/connections.py +++ b/airflow/api_fastapi/core_api/routes/public/connections.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from fastapi import Depends, HTTPException +from fastapi import Depends, HTTPException, status from sqlalchemy import select from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -37,7 +37,9 @@ @connections_router.delete( "/{connection_id}", status_code=204, - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def delete_connection( connection_id: str, @@ -47,14 +49,18 @@ async def delete_connection( connection = session.scalar(select(Connection).filter_by(conn_id=connection_id)) if connection is None: - raise HTTPException(404, f"The Connection with connection_id: `{connection_id}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, f"The Connection with connection_id: `{connection_id}` was not found" + ) session.delete(connection) @connections_router.get( "/{connection_id}", - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_connection( connection_id: str, @@ -64,14 +70,18 @@ async def get_connection( connection = session.scalar(select(Connection).filter_by(conn_id=connection_id)) if connection is None: - raise HTTPException(404, f"The Connection with connection_id: `{connection_id}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, f"The Connection with connection_id: `{connection_id}` was not found" + ) return ConnectionResponse.model_validate(connection, from_attributes=True) @connections_router.get( "/", - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_connections( limit: QueryLimit, diff --git a/airflow/api_fastapi/core_api/routes/public/dag_run.py b/airflow/api_fastapi/core_api/routes/public/dag_run.py index 02780d6088e943..7f41573b1dba21 100644 --- a/airflow/api_fastapi/core_api/routes/public/dag_run.py +++ b/airflow/api_fastapi/core_api/routes/public/dag_run.py @@ -17,7 +17,7 @@ from __future__ import annotations -from fastapi import Depends, HTTPException, Query, Request +from fastapi import Depends, HTTPException, Query, Request, status from sqlalchemy import select from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -40,21 +40,40 @@ dag_run_router = AirflowRouter(tags=["DagRun"], prefix="/dags/{dag_id}/dagRuns") -@dag_run_router.get("/{dag_run_id}", responses=create_openapi_http_exception_doc([401, 403, 404])) +@dag_run_router.get( + "/{dag_run_id}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def get_dag_run( dag_id: str, dag_run_id: str, session: Annotated[Session, Depends(get_session)] ) -> DAGRunResponse: dag_run = session.scalar(select(DagRun).filter_by(dag_id=dag_id, run_id=dag_run_id)) if dag_run is None: raise HTTPException( - 404, f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found" + status.HTTP_404_NOT_FOUND, + f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found", ) return DAGRunResponse.model_validate(dag_run, from_attributes=True) @dag_run_router.delete( - "/{dag_run_id}", status_code=204, responses=create_openapi_http_exception_doc([400, 401, 403, 404]) + "/{dag_run_id}", + status_code=status.HTTP_204_NO_CONTENT, + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), ) async def delete_dag_run(dag_id: str, dag_run_id: str, session: Annotated[Session, Depends(get_session)]): """Delete a DAG Run entry.""" @@ -62,13 +81,24 @@ async def delete_dag_run(dag_id: str, dag_run_id: str, session: Annotated[Sessio if dag_run is None: raise HTTPException( - 404, f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found" + status.HTTP_404_NOT_FOUND, + f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found", ) session.delete(dag_run) -@dag_run_router.patch("/{dag_run_id}", responses=create_openapi_http_exception_doc([400, 401, 403, 404])) +@dag_run_router.patch( + "/{dag_run_id}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def patch_dag_run_state( dag_id: str, dag_run_id: str, @@ -81,17 +111,20 @@ async def patch_dag_run_state( dag_run = session.scalar(select(DagRun).filter_by(dag_id=dag_id, run_id=dag_run_id)) if dag_run is None: raise HTTPException( - 404, f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found" + status.HTTP_404_NOT_FOUND, + f"The DagRun with dag_id: `{dag_id}` and run_id: `{dag_run_id}` was not found", ) dag: DAG = request.app.state.dag_bag.get_dag(dag_id) if not dag: - raise HTTPException(404, f"Dag with id {dag_id} was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id {dag_id} was not found") if update_mask: if update_mask != ["state"]: - raise HTTPException(400, "Only `state` field can be updated through the REST API") + raise HTTPException( + status.HTTP_400_BAD_REQUEST, "Only `state` field can be updated through the REST API" + ) else: update_mask = ["state"] diff --git a/airflow/api_fastapi/core_api/routes/public/dag_sources.py b/airflow/api_fastapi/core_api/routes/public/dag_sources.py index 2a660a7d0264ed..3cf046f5b7573c 100644 --- a/airflow/api_fastapi/core_api/routes/public/dag_sources.py +++ b/airflow/api_fastapi/core_api/routes/public/dag_sources.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from fastapi import Depends, Header, HTTPException, Request, Response +from fastapi import Depends, Header, HTTPException, Request, Response, status from itsdangerous import BadSignature, URLSafeSerializer from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -37,7 +37,15 @@ @dag_sources_router.get( "/{file_token}", responses={ - **create_openapi_http_exception_doc([400, 401, 403, 404, 406]), + **create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_406_NOT_ACCEPTABLE, + ] + ), "200": { "description": "Successful Response", "content": { @@ -62,10 +70,10 @@ async def get_dag_source( content=DagCode.code(path, session=session), ) except (BadSignature, FileNotFoundError): - raise HTTPException(404, "DAG source not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, "DAG source not found") if accept.startswith(mime_type_text): return Response(dag_source_model.content, media_type=mime_type_text) if accept.startswith(mime_type_json) or accept.startswith(mime_type_any): return dag_source_model - raise HTTPException(406, "Content not available for Accept header") + raise HTTPException(status.HTTP_406_NOT_ACCEPTABLE, "Content not available for Accept header") diff --git a/airflow/api_fastapi/core_api/routes/public/dag_warning.py b/airflow/api_fastapi/core_api/routes/public/dag_warning.py new file mode 100644 index 00000000000000..f445fb0afcf5ba --- /dev/null +++ b/airflow/api_fastapi/core_api/routes/public/dag_warning.py @@ -0,0 +1,75 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from fastapi import Depends, status +from sqlalchemy import select +from sqlalchemy.orm import Session +from typing_extensions import Annotated + +from airflow.api_fastapi.common.db.common import ( + get_session, + paginated_select, +) +from airflow.api_fastapi.common.parameters import ( + QueryDagIdInDagWarningFilter, + QueryLimit, + QueryOffset, + QueryWarningTypeFilter, + SortParam, +) +from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc +from airflow.api_fastapi.core_api.serializers.dag_warning import ( + DAGWarningCollectionResponse, + DAGWarningResponse, +) +from airflow.models import DagWarning + +dag_warning_router = AirflowRouter(tags=["DagWarning"]) + + +@dag_warning_router.get( + "/dagWarnings", + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), +) +async def list_dag_warnings( + dag_id: QueryDagIdInDagWarningFilter, + warning_type: QueryWarningTypeFilter, + limit: QueryLimit, + offset: QueryOffset, + order_by: Annotated[ + SortParam, + Depends(SortParam(["dag_id", "warning_type", "message", "timestamp"], DagWarning).dynamic_depends()), + ], + session: Annotated[Session, Depends(get_session)], +) -> DAGWarningCollectionResponse: + """Get a list of DAG warnings.""" + dag_warnings_select, total_entries = paginated_select( + select(DagWarning), [warning_type, dag_id], order_by, offset, limit, session + ) + + dag_warnings = session.scalars(dag_warnings_select).all() + + return DAGWarningCollectionResponse( + dag_warnings=[ + DAGWarningResponse.model_validate(dag_warning, from_attributes=True) + for dag_warning in dag_warnings + ], + total_entries=total_entries, + ) diff --git a/airflow/api_fastapi/core_api/routes/public/dags.py b/airflow/api_fastapi/core_api/routes/public/dags.py index c7b753b5cdbd9b..7d2ee68fa144bd 100644 --- a/airflow/api_fastapi/core_api/routes/public/dags.py +++ b/airflow/api_fastapi/core_api/routes/public/dags.py @@ -17,7 +17,7 @@ from __future__ import annotations -from fastapi import Depends, HTTPException, Query, Request, Response +from fastapi import Depends, HTTPException, Query, Request, Response, status from sqlalchemy import select, update from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -99,7 +99,7 @@ async def get_dags( @dags_router.get( "/tags", - responses=create_openapi_http_exception_doc([401, 403]), + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), ) async def get_dag_tags( limit: QueryLimit, @@ -130,18 +130,29 @@ async def get_dag_tags( return DAGTagCollectionResponse(tags=[dag_tag for dag_tag in dag_tags], total_entries=total_entries) -@dags_router.get("/{dag_id}", responses=create_openapi_http_exception_doc([400, 401, 403, 404, 422])) +@dags_router.get( + "/{dag_id}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_422_UNPROCESSABLE_ENTITY, + ] + ), +) async def get_dag( dag_id: str, session: Annotated[Session, Depends(get_session)], request: Request ) -> DAGResponse: """Get basic information about a DAG.""" dag: DAG = request.app.state.dag_bag.get_dag(dag_id) if not dag: - raise HTTPException(404, f"Dag with id {dag_id} was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id {dag_id} was not found") dag_model: DagModel = session.get(DagModel, dag_id) if not dag_model: - raise HTTPException(404, f"Unable to obtain dag with id {dag_id} from session") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Unable to obtain dag with id {dag_id} from session") for key, value in dag.__dict__.items(): if not key.startswith("_") and not hasattr(dag_model, key): @@ -157,11 +168,11 @@ async def get_dag_details( """Get details of DAG.""" dag: DAG = request.app.state.dag_bag.get_dag(dag_id) if not dag: - raise HTTPException(404, f"Dag with id {dag_id} was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id {dag_id} was not found") dag_model: DagModel = session.get(DagModel, dag_id) if not dag_model: - raise HTTPException(404, f"Unable to obtain dag with id {dag_id} from session") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Unable to obtain dag with id {dag_id} from session") for key, value in dag.__dict__.items(): if not key.startswith("_") and not hasattr(dag_model, key): @@ -170,7 +181,17 @@ async def get_dag_details( return DAGDetailsResponse.model_validate(dag_model, from_attributes=True) -@dags_router.patch("/{dag_id}", responses=create_openapi_http_exception_doc([400, 401, 403, 404])) +@dags_router.patch( + "/{dag_id}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def patch_dag( dag_id: str, patch_body: DAGPatchBody, @@ -181,11 +202,13 @@ async def patch_dag( dag = session.get(DagModel, dag_id) if dag is None: - raise HTTPException(404, f"Dag with id: {dag_id} was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id: {dag_id} was not found") if update_mask: if update_mask != ["is_paused"]: - raise HTTPException(400, "Only `is_paused` field can be updated through the REST API") + raise HTTPException( + status.HTTP_400_BAD_REQUEST, "Only `is_paused` field can be updated through the REST API" + ) else: update_mask = ["is_paused"] @@ -197,7 +220,17 @@ async def patch_dag( return DAGResponse.model_validate(dag, from_attributes=True) -@dags_router.patch("/", responses=create_openapi_http_exception_doc([400, 401, 403, 404])) +@dags_router.patch( + "/", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def patch_dags( patch_body: DAGPatchBody, limit: QueryLimit, @@ -214,7 +247,9 @@ async def patch_dags( """Patch multiple DAGs.""" if update_mask: if update_mask != ["is_paused"]: - raise HTTPException(400, "Only `is_paused` field can be updated through the REST API") + raise HTTPException( + status.HTTP_400_BAD_REQUEST, "Only `is_paused` field can be updated through the REST API" + ) else: update_mask = ["is_paused"] @@ -244,7 +279,18 @@ async def patch_dags( ) -@dags_router.delete("/{dag_id}", responses=create_openapi_http_exception_doc([400, 401, 403, 404, 422])) +@dags_router.delete( + "/{dag_id}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_422_UNPROCESSABLE_ENTITY, + ] + ), +) async def delete_dag( dag_id: str, session: Annotated[Session, Depends(get_session)], @@ -253,7 +299,9 @@ async def delete_dag( try: delete_dag_module.delete_dag(dag_id, session=session) except DagNotFound: - raise HTTPException(404, f"Dag with id: {dag_id} was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"Dag with id: {dag_id} was not found") except AirflowException: - raise HTTPException(409, f"Task instances of dag with id: '{dag_id}' are still running") - return Response(status_code=204) + raise HTTPException( + status.HTTP_409_CONFLICT, f"Task instances of dag with id: '{dag_id}' are still running" + ) + return Response(status_code=status.HTTP_204_NO_CONTENT) diff --git a/airflow/api_fastapi/core_api/routes/public/event_logs.py b/airflow/api_fastapi/core_api/routes/public/event_logs.py index 75f12cbefb03b2..537bb5ffe4df95 100644 --- a/airflow/api_fastapi/core_api/routes/public/event_logs.py +++ b/airflow/api_fastapi/core_api/routes/public/event_logs.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from fastapi import Depends, HTTPException +from fastapi import Depends, HTTPException, status from sqlalchemy import select from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -36,7 +36,9 @@ @event_logs_router.get( "/{event_log_id}", - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_event_log( event_log_id: int, @@ -44,7 +46,7 @@ async def get_event_log( ) -> EventLogResponse: event_log = session.scalar(select(Log).where(Log.id == event_log_id)) if event_log is None: - raise HTTPException(404, f"The Event Log with id: `{event_log_id}` not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"The Event Log with id: `{event_log_id}` not found") return EventLogResponse.model_validate( event_log, from_attributes=True, diff --git a/airflow/api_fastapi/core_api/routes/public/pools.py b/airflow/api_fastapi/core_api/routes/public/pools.py index 5690196e850a51..99389e0bd6e924 100644 --- a/airflow/api_fastapi/core_api/routes/public/pools.py +++ b/airflow/api_fastapi/core_api/routes/public/pools.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from fastapi import Depends, HTTPException, Query +from fastapi import Depends, HTTPException, Query, status from fastapi.exceptions import RequestValidationError from pydantic import ValidationError from sqlalchemy import delete, select @@ -41,8 +41,15 @@ @pools_router.delete( "/{pool_name}", - status_code=204, - responses=create_openapi_http_exception_doc([400, 401, 403, 404]), + status_code=status.HTTP_204_NO_CONTENT, + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), ) async def delete_pool( pool_name: str, @@ -50,17 +57,19 @@ async def delete_pool( ): """Delete a pool entry.""" if pool_name == "default_pool": - raise HTTPException(400, "Default Pool can't be deleted") + raise HTTPException(status.HTTP_400_BAD_REQUEST, "Default Pool can't be deleted") affected_count = session.execute(delete(Pool).where(Pool.pool == pool_name)).rowcount if affected_count == 0: - raise HTTPException(404, f"The Pool with name: `{pool_name}` was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"The Pool with name: `{pool_name}` was not found") @pools_router.get( "/{pool_name}", - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_pool( pool_name: str, @@ -69,14 +78,16 @@ async def get_pool( """Get a pool.""" pool = session.scalar(select(Pool).where(Pool.pool == pool_name)) if pool is None: - raise HTTPException(404, f"The Pool with name: `{pool_name}` was not found") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"The Pool with name: `{pool_name}` was not found") return PoolResponse.model_validate(pool, from_attributes=True) @pools_router.get( "/", - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_pools( limit: QueryLimit, @@ -105,7 +116,17 @@ async def get_pools( ) -@pools_router.patch("/{pool_name}", responses=create_openapi_http_exception_doc([400, 401, 403, 404])) +@pools_router.patch( + "/{pool_name}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def patch_pool( pool_name: str, patch_body: PoolPatchBody, @@ -118,11 +139,16 @@ async def patch_pool( if update_mask and all(mask.strip() in {"slots", "include_deferred"} for mask in update_mask): pass else: - raise HTTPException(400, "Only slots and included_deferred can be modified on Default Pool") + raise HTTPException( + status.HTTP_400_BAD_REQUEST, + "Only slots and included_deferred can be modified on Default Pool", + ) pool = session.scalar(select(Pool).where(Pool.pool == pool_name).limit(1)) if not pool: - raise HTTPException(404, detail=f"The Pool with name: `{pool_name}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, detail=f"The Pool with name: `{pool_name}` was not found" + ) if update_mask: data = patch_body.model_dump(include=set(update_mask), by_alias=True) @@ -139,7 +165,11 @@ async def patch_pool( return PoolResponse.model_validate(pool, from_attributes=True) -@pools_router.post("/", status_code=201, responses=create_openapi_http_exception_doc([401, 403])) +@pools_router.post( + "/", + status_code=status.HTTP_201_CREATED, + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), +) async def post_pool( post_body: PoolPostBody, session: Annotated[Session, Depends(get_session)], 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 c9458e843afeea..df16c0bc450d42 100644 --- a/airflow/api_fastapi/core_api/routes/public/task_instances.py +++ b/airflow/api_fastapi/core_api/routes/public/task_instances.py @@ -17,7 +17,7 @@ from __future__ import annotations -from fastapi import Depends, HTTPException +from fastapi import Depends, HTTPException, status from sqlalchemy.orm import Session, joinedload from sqlalchemy.sql import select from typing_extensions import Annotated @@ -33,7 +33,12 @@ ) -@task_instances_router.get("/{task_id}", responses=create_openapi_http_exception_doc([401, 403, 404])) +@task_instances_router.get( + "/{task_id}", + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), +) async def get_task_instance( dag_id: str, dag_run_id: str, task_id: str, session: Annotated[Session, Depends(get_session)] ) -> TaskInstanceResponse: @@ -48,17 +53,22 @@ async def get_task_instance( if task_instance is None: raise HTTPException( - 404, + status.HTTP_404_NOT_FOUND, f"The Task Instance with dag_id: `{dag_id}`, run_id: `{dag_run_id}` and task_id: `{task_id}` was not found", ) if task_instance.map_index != -1: - raise HTTPException(404, "Task instance is mapped, add the map_index value to the URL") + raise HTTPException( + status.HTTP_404_NOT_FOUND, "Task instance is mapped, add the map_index value to the URL" + ) return TaskInstanceResponse.model_validate(task_instance, from_attributes=True) @task_instances_router.get( - "/{task_id}/{map_index}", responses=create_openapi_http_exception_doc([401, 403, 404]) + "/{task_id}/{map_index}", + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def get_mapped_task_instance( dag_id: str, @@ -78,7 +88,7 @@ async def get_mapped_task_instance( if task_instance is None: raise HTTPException( - 404, + status.HTTP_404_NOT_FOUND, f"The Mapped Task Instance with dag_id: `{dag_id}`, run_id: `{dag_run_id}`, task_id: `{task_id}`, and map_index: `{map_index}` was not found", ) diff --git a/airflow/api_fastapi/core_api/routes/public/variables.py b/airflow/api_fastapi/core_api/routes/public/variables.py index 6b834a6de75817..5d2bf5a899d8a9 100644 --- a/airflow/api_fastapi/core_api/routes/public/variables.py +++ b/airflow/api_fastapi/core_api/routes/public/variables.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from fastapi import Depends, HTTPException, Query +from fastapi import Depends, HTTPException, Query, status from sqlalchemy import select from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -38,7 +38,9 @@ @variables_router.delete( "/{variable_key}", status_code=204, - responses=create_openapi_http_exception_doc([401, 403, 404]), + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), ) async def delete_variable( variable_key: str, @@ -46,10 +48,17 @@ async def delete_variable( ): """Delete a variable entry.""" if Variable.delete(variable_key, session) == 0: - raise HTTPException(404, f"The Variable with key: `{variable_key}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, f"The Variable with key: `{variable_key}` was not found" + ) -@variables_router.get("/{variable_key}", responses=create_openapi_http_exception_doc([401, 403, 404])) +@variables_router.get( + "/{variable_key}", + responses=create_openapi_http_exception_doc( + [status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN, status.HTTP_404_NOT_FOUND] + ), +) async def get_variable( variable_key: str, session: Annotated[Session, Depends(get_session)], @@ -58,14 +67,16 @@ async def get_variable( variable = session.scalar(select(Variable).where(Variable.key == variable_key).limit(1)) if variable is None: - raise HTTPException(404, f"The Variable with key: `{variable_key}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, f"The Variable with key: `{variable_key}` was not found" + ) return VariableResponse.model_validate(variable, from_attributes=True) @variables_router.get( "/", - responses=create_openapi_http_exception_doc([401, 403]), + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), ) async def get_variables( limit: QueryLimit, @@ -99,7 +110,17 @@ async def get_variables( ) -@variables_router.patch("/{variable_key}", responses=create_openapi_http_exception_doc([400, 401, 403, 404])) +@variables_router.patch( + "/{variable_key}", + responses=create_openapi_http_exception_doc( + [ + status.HTTP_400_BAD_REQUEST, + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + ] + ), +) async def patch_variable( variable_key: str, patch_body: VariableBody, @@ -108,11 +129,15 @@ async def patch_variable( ) -> VariableResponse: """Update a variable by key.""" if patch_body.key != variable_key: - raise HTTPException(400, "Invalid body, key from request body doesn't match uri parameter") + raise HTTPException( + status.HTTP_400_BAD_REQUEST, "Invalid body, key from request body doesn't match uri parameter" + ) non_update_fields = {"key"} variable = session.scalar(select(Variable).filter_by(key=variable_key).limit(1)) if not variable: - raise HTTPException(404, f"The Variable with key: `{variable_key}` was not found") + raise HTTPException( + status.HTTP_404_NOT_FOUND, f"The Variable with key: `{variable_key}` was not found" + ) if update_mask: data = patch_body.model_dump(include=set(update_mask) - non_update_fields) else: @@ -122,7 +147,11 @@ async def patch_variable( return variable -@variables_router.post("/", status_code=201, responses=create_openapi_http_exception_doc([401, 403])) +@variables_router.post( + "/", + status_code=status.HTTP_201_CREATED, + responses=create_openapi_http_exception_doc([status.HTTP_401_UNAUTHORIZED, status.HTTP_403_FORBIDDEN]), +) async def post_variable( post_body: VariableBody, session: Annotated[Session, Depends(get_session)], diff --git a/airflow/api_fastapi/core_api/routes/ui/assets.py b/airflow/api_fastapi/core_api/routes/ui/assets.py index 6786bc30ae6800..3b98e4f59a3cd0 100644 --- a/airflow/api_fastapi/core_api/routes/ui/assets.py +++ b/airflow/api_fastapi/core_api/routes/ui/assets.py @@ -17,7 +17,7 @@ from __future__ import annotations -from fastapi import Depends, HTTPException, Request +from fastapi import Depends, HTTPException, Request, status from sqlalchemy import and_, func, select from sqlalchemy.orm import Session from typing_extensions import Annotated @@ -39,12 +39,12 @@ async def next_run_assets( dag = request.app.state.dag_bag.get_dag(dag_id) if not dag: - raise HTTPException(404, f"can't find dag {dag_id}") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"can't find dag {dag_id}") dag_model = DagModel.get_dagmodel(dag_id, session=session) if dag_model is None: - raise HTTPException(404, f"can't find associated dag_model {dag_id}") + raise HTTPException(status.HTTP_404_NOT_FOUND, f"can't find associated dag_model {dag_id}") latest_run = dag_model.get_last_dagrun(session=session) diff --git a/airflow/api_fastapi/core_api/serializers/backfills.py b/airflow/api_fastapi/core_api/serializers/backfills.py new file mode 100644 index 00000000000000..69d6a98ccfd1af --- /dev/null +++ b/airflow/api_fastapi/core_api/serializers/backfills.py @@ -0,0 +1,59 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from datetime import datetime + +from pydantic import BaseModel + +from airflow.models.backfill import ReprocessBehavior + + +class BackfillPostBody(BaseModel): + """Object used for create backfill request.""" + + dag_id: str + from_date: datetime + to_date: datetime + run_backwards: bool = False + dag_run_conf: dict = {} + reprocess_behavior: ReprocessBehavior = ReprocessBehavior.NONE + max_active_runs: int = 10 + + +class BackfillResponse(BaseModel): + """Base serializer for Backfill.""" + + id: int + dag_id: str + from_date: datetime + to_date: datetime + dag_run_conf: dict + is_paused: bool + reprocess_behavior: ReprocessBehavior + max_active_runs: int + created_at: datetime + completed_at: datetime | None + updated_at: datetime + + +class BackfillCollectionResponse(BaseModel): + """Backfill Collection serializer for responses.""" + + backfills: list[BackfillResponse] + total_entries: int diff --git a/airflow/api_fastapi/core_api/serializers/dag_warning.py b/airflow/api_fastapi/core_api/serializers/dag_warning.py new file mode 100644 index 00000000000000..f38a3a8d093f74 --- /dev/null +++ b/airflow/api_fastapi/core_api/serializers/dag_warning.py @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from datetime import datetime + +from pydantic import BaseModel + +from airflow.models.dagwarning import DagWarningType + + +class DAGWarningResponse(BaseModel): + """DAG Warning serializer for responses.""" + + dag_id: str + warning_type: DagWarningType + message: str + timestamp: datetime + + +class DAGWarningCollectionResponse(BaseModel): + """DAG warning collection serializer for responses.""" + + dag_warnings: list[DAGWarningResponse] + total_entries: int diff --git a/airflow/api_fastapi/core_api/serializers/task_instances.py b/airflow/api_fastapi/core_api/serializers/task_instances.py index b8a10e8fb86149..07ef42d8e24b11 100644 --- a/airflow/api_fastapi/core_api/serializers/task_instances.py +++ b/airflow/api_fastapi/core_api/serializers/task_instances.py @@ -31,6 +31,7 @@ class TaskInstanceResponse(BaseModel): model_config = ConfigDict(populate_by_name=True) + id: str task_id: str dag_id: str run_id: str = Field(alias="dag_run_id") diff --git a/airflow/example_dags/example_branch_operator_decorator.py b/airflow/example_dags/example_branch_operator_decorator.py index e9b3bea97a72f8..95bfba4cdc605c 100644 --- a/airflow/example_dags/example_branch_operator_decorator.py +++ b/airflow/example_dags/example_branch_operator_decorator.py @@ -112,7 +112,7 @@ def some_ext_py_task(): # Run the example a second time and see that it re-uses it and is faster. VENV_CACHE_PATH = tempfile.gettempdir() - @task.branch_virtualenv(requirements=["numpy~=1.24.4"], venv_cache_path=VENV_CACHE_PATH) + @task.branch_virtualenv(requirements=["numpy~=1.26.0"], venv_cache_path=VENV_CACHE_PATH) def branching_virtualenv(choices) -> str: import random @@ -132,7 +132,7 @@ def branching_virtualenv(choices) -> str: for option in options: @task.virtualenv( - task_id=f"venv_{option}", requirements=["numpy~=1.24.4"], venv_cache_path=VENV_CACHE_PATH + task_id=f"venv_{option}", requirements=["numpy~=1.26.0"], venv_cache_path=VENV_CACHE_PATH ) def some_venv_task(): import numpy as np diff --git a/airflow/models/backfill.py b/airflow/models/backfill.py index 72df6a1e0eb7f8..648b35c5bdebe1 100644 --- a/airflow/models/backfill.py +++ b/airflow/models/backfill.py @@ -36,12 +36,11 @@ desc, func, select, - update, ) from sqlalchemy.orm import relationship, validates from sqlalchemy_jsonfield import JSONField -from airflow.api_connexion.exceptions import Conflict, NotFound +from airflow.api_connexion.exceptions import NotFound from airflow.exceptions import AirflowException from airflow.models.base import Base, StringID from airflow.settings import json @@ -87,7 +86,7 @@ class Backfill(Base): dag_id = Column(StringID(), nullable=False) from_date = Column(UtcDateTime, nullable=False) to_date = Column(UtcDateTime, nullable=False) - dag_run_conf = Column(JSONField(json=json), nullable=True) + dag_run_conf = Column(JSONField(json=json), nullable=False, default={}) is_paused = Column(Boolean, default=False) """ Controls whether new dag runs will be created for this backfill. @@ -333,33 +332,3 @@ def _create_backfill( info, ) return br - - -def _cancel_backfill(backfill_id) -> Backfill: - with create_session() as session: - b: Backfill = session.get(Backfill, backfill_id) - if b.completed_at is not None: - raise Conflict("Backfill is already completed.") - - b.completed_at = timezone.utcnow() - - # first, pause - if not b.is_paused: - b.is_paused = True - - session.commit() - - from airflow.models import DagRun - - # now, let's mark all queued dag runs as failed - query = ( - update(DagRun) - .where( - DagRun.id.in_(select(BackfillDagRun.dag_run_id).where(BackfillDagRun.backfill_id == b.id)), - DagRun.state == DagRunState.QUEUED, - ) - .values(state=DagRunState.FAILED) - .execution_options(synchronize_session=False) - ) - session.execute(query) - return b diff --git a/airflow/ui/openapi-gen/queries/common.ts b/airflow/ui/openapi-gen/queries/common.ts index 07edb67a993040..75a343ce74c7cb 100644 --- a/airflow/ui/openapi-gen/queries/common.ts +++ b/airflow/ui/openapi-gen/queries/common.ts @@ -3,10 +3,12 @@ import { UseQueryResult } from "@tanstack/react-query"; import { AssetService, + BackfillService, ConnectionService, DagRunService, DagService, DagSourceService, + DagWarningService, DagsService, DashboardService, EventLogService, @@ -18,7 +20,7 @@ import { VariableService, VersionService, } from "../requests/services.gen"; -import { DagRunState } from "../requests/types.gen"; +import { DagRunState, DagWarningType } from "../requests/types.gen"; export type AssetServiceNextRunAssetsDefaultResponse = Awaited< ReturnType @@ -108,90 +110,48 @@ export const UseDagsServiceRecentDagRunsKeyFn = ( }, ]), ]; -export type ConnectionServiceGetConnectionDefaultResponse = Awaited< - ReturnType +export type BackfillServiceListBackfillsDefaultResponse = Awaited< + ReturnType >; -export type ConnectionServiceGetConnectionQueryResult< - TData = ConnectionServiceGetConnectionDefaultResponse, +export type BackfillServiceListBackfillsQueryResult< + TData = BackfillServiceListBackfillsDefaultResponse, TError = unknown, > = UseQueryResult; -export const useConnectionServiceGetConnectionKey = - "ConnectionServiceGetConnection"; -export const UseConnectionServiceGetConnectionKeyFn = ( - { - connectionId, - }: { - connectionId: string; - }, - queryKey?: Array, -) => [ - useConnectionServiceGetConnectionKey, - ...(queryKey ?? [{ connectionId }]), -]; -export type ConnectionServiceGetConnectionsDefaultResponse = Awaited< - ReturnType ->; -export type ConnectionServiceGetConnectionsQueryResult< - TData = ConnectionServiceGetConnectionsDefaultResponse, - TError = unknown, -> = UseQueryResult; -export const useConnectionServiceGetConnectionsKey = - "ConnectionServiceGetConnections"; -export const UseConnectionServiceGetConnectionsKeyFn = ( +export const useBackfillServiceListBackfillsKey = + "BackfillServiceListBackfills"; +export const UseBackfillServiceListBackfillsKeyFn = ( { + dagId, limit, offset, orderBy, }: { + dagId: string; limit?: number; offset?: number; orderBy?: string; - } = {}, + }, queryKey?: Array, ) => [ - useConnectionServiceGetConnectionsKey, - ...(queryKey ?? [{ limit, offset, orderBy }]), + useBackfillServiceListBackfillsKey, + ...(queryKey ?? [{ dagId, limit, offset, orderBy }]), ]; -export type DagRunServiceGetDagRunDefaultResponse = Awaited< - ReturnType ->; -export type DagRunServiceGetDagRunQueryResult< - TData = DagRunServiceGetDagRunDefaultResponse, - TError = unknown, -> = UseQueryResult; -export const useDagRunServiceGetDagRunKey = "DagRunServiceGetDagRun"; -export const UseDagRunServiceGetDagRunKeyFn = ( - { - dagId, - dagRunId, - }: { - dagId: string; - dagRunId: string; - }, - queryKey?: Array, -) => [useDagRunServiceGetDagRunKey, ...(queryKey ?? [{ dagId, dagRunId }])]; -export type DagSourceServiceGetDagSourceDefaultResponse = Awaited< - ReturnType +export type BackfillServiceGetBackfillDefaultResponse = Awaited< + ReturnType >; -export type DagSourceServiceGetDagSourceQueryResult< - TData = DagSourceServiceGetDagSourceDefaultResponse, +export type BackfillServiceGetBackfillQueryResult< + TData = BackfillServiceGetBackfillDefaultResponse, TError = unknown, > = UseQueryResult; -export const useDagSourceServiceGetDagSourceKey = - "DagSourceServiceGetDagSource"; -export const UseDagSourceServiceGetDagSourceKeyFn = ( +export const useBackfillServiceGetBackfillKey = "BackfillServiceGetBackfill"; +export const UseBackfillServiceGetBackfillKeyFn = ( { - accept, - fileToken, + backfillId, }: { - accept?: string; - fileToken: string; + backfillId: string; }, queryKey?: Array, -) => [ - useDagSourceServiceGetDagSourceKey, - ...(queryKey ?? [{ accept, fileToken }]), -]; +) => [useBackfillServiceGetBackfillKey, ...(queryKey ?? [{ backfillId }])]; export type DagServiceGetDagsDefaultResponse = Awaited< ReturnType >; @@ -299,6 +259,90 @@ export const UseDagServiceGetDagDetailsKeyFn = ( }, queryKey?: Array, ) => [useDagServiceGetDagDetailsKey, ...(queryKey ?? [{ dagId }])]; +export type ConnectionServiceGetConnectionDefaultResponse = Awaited< + ReturnType +>; +export type ConnectionServiceGetConnectionQueryResult< + TData = ConnectionServiceGetConnectionDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useConnectionServiceGetConnectionKey = + "ConnectionServiceGetConnection"; +export const UseConnectionServiceGetConnectionKeyFn = ( + { + connectionId, + }: { + connectionId: string; + }, + queryKey?: Array, +) => [ + useConnectionServiceGetConnectionKey, + ...(queryKey ?? [{ connectionId }]), +]; +export type ConnectionServiceGetConnectionsDefaultResponse = Awaited< + ReturnType +>; +export type ConnectionServiceGetConnectionsQueryResult< + TData = ConnectionServiceGetConnectionsDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useConnectionServiceGetConnectionsKey = + "ConnectionServiceGetConnections"; +export const UseConnectionServiceGetConnectionsKeyFn = ( + { + limit, + offset, + orderBy, + }: { + limit?: number; + offset?: number; + orderBy?: string; + } = {}, + queryKey?: Array, +) => [ + useConnectionServiceGetConnectionsKey, + ...(queryKey ?? [{ limit, offset, orderBy }]), +]; +export type DagRunServiceGetDagRunDefaultResponse = Awaited< + ReturnType +>; +export type DagRunServiceGetDagRunQueryResult< + TData = DagRunServiceGetDagRunDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useDagRunServiceGetDagRunKey = "DagRunServiceGetDagRun"; +export const UseDagRunServiceGetDagRunKeyFn = ( + { + dagId, + dagRunId, + }: { + dagId: string; + dagRunId: string; + }, + queryKey?: Array, +) => [useDagRunServiceGetDagRunKey, ...(queryKey ?? [{ dagId, dagRunId }])]; +export type DagSourceServiceGetDagSourceDefaultResponse = Awaited< + ReturnType +>; +export type DagSourceServiceGetDagSourceQueryResult< + TData = DagSourceServiceGetDagSourceDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useDagSourceServiceGetDagSourceKey = + "DagSourceServiceGetDagSource"; +export const UseDagSourceServiceGetDagSourceKeyFn = ( + { + accept, + fileToken, + }: { + accept?: string; + fileToken: string; + }, + queryKey?: Array, +) => [ + useDagSourceServiceGetDagSourceKey, + ...(queryKey ?? [{ accept, fileToken }]), +]; export type EventLogServiceGetEventLogDefaultResponse = Awaited< ReturnType >; @@ -327,6 +371,34 @@ export const UseMonitorServiceGetHealthKeyFn = (queryKey?: Array) => [ useMonitorServiceGetHealthKey, ...(queryKey ?? []), ]; +export type DagWarningServiceListDagWarningsDefaultResponse = Awaited< + ReturnType +>; +export type DagWarningServiceListDagWarningsQueryResult< + TData = DagWarningServiceListDagWarningsDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useDagWarningServiceListDagWarningsKey = + "DagWarningServiceListDagWarnings"; +export const UseDagWarningServiceListDagWarningsKeyFn = ( + { + dagId, + limit, + offset, + orderBy, + warningType, + }: { + dagId?: string; + limit?: number; + offset?: number; + orderBy?: string; + warningType?: DagWarningType; + } = {}, + queryKey?: Array, +) => [ + useDagWarningServiceListDagWarningsKey, + ...(queryKey ?? [{ dagId, limit, offset, orderBy, warningType }]), +]; export type PluginServiceGetPluginsDefaultResponse = Awaited< ReturnType >; @@ -500,14 +572,23 @@ export const UseVersionServiceGetVersionKeyFn = (queryKey?: Array) => [ useVersionServiceGetVersionKey, ...(queryKey ?? []), ]; +export type BackfillServiceCreateBackfillMutationResult = Awaited< + ReturnType +>; export type PoolServicePostPoolMutationResult = Awaited< ReturnType >; export type VariableServicePostVariableMutationResult = Awaited< ReturnType >; -export type DagRunServicePatchDagRunStateMutationResult = Awaited< - ReturnType +export type BackfillServicePauseBackfillMutationResult = Awaited< + ReturnType +>; +export type BackfillServiceUnpauseBackfillMutationResult = Awaited< + ReturnType +>; +export type BackfillServiceCancelBackfillMutationResult = Awaited< + ReturnType >; export type DagServicePatchDagsMutationResult = Awaited< ReturnType @@ -515,21 +596,24 @@ export type DagServicePatchDagsMutationResult = Awaited< export type DagServicePatchDagMutationResult = Awaited< ReturnType >; +export type DagRunServicePatchDagRunStateMutationResult = Awaited< + ReturnType +>; export type PoolServicePatchPoolMutationResult = Awaited< ReturnType >; export type VariableServicePatchVariableMutationResult = Awaited< ReturnType >; +export type DagServiceDeleteDagMutationResult = Awaited< + ReturnType +>; export type ConnectionServiceDeleteConnectionMutationResult = Awaited< ReturnType >; export type DagRunServiceDeleteDagRunMutationResult = Awaited< ReturnType >; -export type DagServiceDeleteDagMutationResult = Awaited< - ReturnType ->; export type PoolServiceDeletePoolMutationResult = Awaited< ReturnType >; diff --git a/airflow/ui/openapi-gen/queries/prefetch.ts b/airflow/ui/openapi-gen/queries/prefetch.ts index db61369e19ff73..63e8d4b43132a1 100644 --- a/airflow/ui/openapi-gen/queries/prefetch.ts +++ b/airflow/ui/openapi-gen/queries/prefetch.ts @@ -3,10 +3,12 @@ import { type QueryClient } from "@tanstack/react-query"; import { AssetService, + BackfillService, ConnectionService, DagRunService, DagService, DagSourceService, + DagWarningService, DagsService, DashboardService, EventLogService, @@ -18,7 +20,7 @@ import { VariableService, VersionService, } from "../requests/services.gen"; -import { DagRunState } from "../requests/types.gen"; +import { DagRunState, DagWarningType } from "../requests/types.gen"; import * as Common from "./common"; /** @@ -137,102 +139,57 @@ export const prefetchUseDagsServiceRecentDagRuns = ( }), }); /** - * Get Connection - * Get a connection entry. - * @param data The data for the request. - * @param data.connectionId - * @returns ConnectionResponse Successful Response - * @throws ApiError - */ -export const prefetchUseConnectionServiceGetConnection = ( - queryClient: QueryClient, - { - connectionId, - }: { - connectionId: string; - }, -) => - queryClient.prefetchQuery({ - queryKey: Common.UseConnectionServiceGetConnectionKeyFn({ connectionId }), - queryFn: () => ConnectionService.getConnection({ connectionId }), - }); -/** - * Get Connections - * Get all connection entries. + * List Backfills * @param data The data for the request. + * @param data.dagId * @param data.limit * @param data.offset * @param data.orderBy - * @returns ConnectionCollectionResponse Successful Response + * @returns unknown Successful Response * @throws ApiError */ -export const prefetchUseConnectionServiceGetConnections = ( +export const prefetchUseBackfillServiceListBackfills = ( queryClient: QueryClient, { + dagId, limit, offset, orderBy, }: { + dagId: string; limit?: number; offset?: number; orderBy?: string; - } = {}, + }, ) => queryClient.prefetchQuery({ - queryKey: Common.UseConnectionServiceGetConnectionsKeyFn({ + queryKey: Common.UseBackfillServiceListBackfillsKeyFn({ + dagId, limit, offset, orderBy, }), - queryFn: () => ConnectionService.getConnections({ limit, offset, orderBy }), - }); -/** - * Get Dag Run - * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @returns DAGRunResponse Successful Response - * @throws ApiError - */ -export const prefetchUseDagRunServiceGetDagRun = ( - queryClient: QueryClient, - { - dagId, - dagRunId, - }: { - dagId: string; - dagRunId: string; - }, -) => - queryClient.prefetchQuery({ - queryKey: Common.UseDagRunServiceGetDagRunKeyFn({ dagId, dagRunId }), - queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }), + queryFn: () => + BackfillService.listBackfills({ dagId, limit, offset, orderBy }), }); /** - * Get Dag Source - * Get source code using file token. + * Get Backfill * @param data The data for the request. - * @param data.fileToken - * @param data.accept - * @returns DAGSourceResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ -export const prefetchUseDagSourceServiceGetDagSource = ( +export const prefetchUseBackfillServiceGetBackfill = ( queryClient: QueryClient, { - accept, - fileToken, + backfillId, }: { - accept?: string; - fileToken: string; + backfillId: string; }, ) => queryClient.prefetchQuery({ - queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn({ - accept, - fileToken, - }), - queryFn: () => DagSourceService.getDagSource({ accept, fileToken }), + queryKey: Common.UseBackfillServiceGetBackfillKeyFn({ backfillId }), + queryFn: () => BackfillService.getBackfill({ backfillId }), }); /** * Get Dags @@ -379,6 +336,104 @@ export const prefetchUseDagServiceGetDagDetails = ( queryKey: Common.UseDagServiceGetDagDetailsKeyFn({ dagId }), queryFn: () => DagService.getDagDetails({ dagId }), }); +/** + * Get Connection + * Get a connection entry. + * @param data The data for the request. + * @param data.connectionId + * @returns ConnectionResponse Successful Response + * @throws ApiError + */ +export const prefetchUseConnectionServiceGetConnection = ( + queryClient: QueryClient, + { + connectionId, + }: { + connectionId: string; + }, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseConnectionServiceGetConnectionKeyFn({ connectionId }), + queryFn: () => ConnectionService.getConnection({ connectionId }), + }); +/** + * Get Connections + * Get all connection entries. + * @param data The data for the request. + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns ConnectionCollectionResponse Successful Response + * @throws ApiError + */ +export const prefetchUseConnectionServiceGetConnections = ( + queryClient: QueryClient, + { + limit, + offset, + orderBy, + }: { + limit?: number; + offset?: number; + orderBy?: string; + } = {}, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseConnectionServiceGetConnectionsKeyFn({ + limit, + offset, + orderBy, + }), + queryFn: () => ConnectionService.getConnections({ limit, offset, orderBy }), + }); +/** + * Get Dag Run + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ +export const prefetchUseDagRunServiceGetDagRun = ( + queryClient: QueryClient, + { + dagId, + dagRunId, + }: { + dagId: string; + dagRunId: string; + }, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseDagRunServiceGetDagRunKeyFn({ dagId, dagRunId }), + queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }), + }); +/** + * Get Dag Source + * Get source code using file token. + * @param data The data for the request. + * @param data.fileToken + * @param data.accept + * @returns DAGSourceResponse Successful Response + * @throws ApiError + */ +export const prefetchUseDagSourceServiceGetDagSource = ( + queryClient: QueryClient, + { + accept, + fileToken, + }: { + accept?: string; + fileToken: string; + }, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn({ + accept, + fileToken, + }), + queryFn: () => DagSourceService.getDagSource({ accept, fileToken }), + }); /** * Get Event Log * @param data The data for the request. @@ -408,6 +463,51 @@ export const prefetchUseMonitorServiceGetHealth = (queryClient: QueryClient) => queryKey: Common.UseMonitorServiceGetHealthKeyFn(), queryFn: () => MonitorService.getHealth(), }); +/** + * List Dag Warnings + * Get a list of DAG warnings. + * @param data The data for the request. + * @param data.dagId + * @param data.warningType + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns DAGWarningCollectionResponse Successful Response + * @throws ApiError + */ +export const prefetchUseDagWarningServiceListDagWarnings = ( + queryClient: QueryClient, + { + dagId, + limit, + offset, + orderBy, + warningType, + }: { + dagId?: string; + limit?: number; + offset?: number; + orderBy?: string; + warningType?: DagWarningType; + } = {}, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseDagWarningServiceListDagWarningsKeyFn({ + dagId, + limit, + offset, + orderBy, + warningType, + }), + queryFn: () => + DagWarningService.listDagWarnings({ + dagId, + limit, + offset, + orderBy, + warningType, + }), + }); /** * Get Plugins * @param data The data for the request. diff --git a/airflow/ui/openapi-gen/queries/queries.ts b/airflow/ui/openapi-gen/queries/queries.ts index 7820656799e5bf..5f8f649372db69 100644 --- a/airflow/ui/openapi-gen/queries/queries.ts +++ b/airflow/ui/openapi-gen/queries/queries.ts @@ -8,10 +8,12 @@ import { import { AssetService, + BackfillService, ConnectionService, DagRunService, DagService, DagSourceService, + DagWarningService, DagsService, DashboardService, EventLogService, @@ -24,9 +26,11 @@ import { VersionService, } from "../requests/services.gen"; import { + BackfillPostBody, DAGPatchBody, DAGRunPatchBody, DagRunState, + DagWarningType, PoolPatchBody, PoolPostBody, VariableBody, @@ -171,132 +175,69 @@ export const useDagsServiceRecentDagRuns = < ...options, }); /** - * Get Connection - * Get a connection entry. - * @param data The data for the request. - * @param data.connectionId - * @returns ConnectionResponse Successful Response - * @throws ApiError - */ -export const useConnectionServiceGetConnection = < - TData = Common.ConnectionServiceGetConnectionDefaultResponse, - TError = unknown, - TQueryKey extends Array = unknown[], ->( - { - connectionId, - }: { - connectionId: string; - }, - queryKey?: TQueryKey, - options?: Omit, "queryKey" | "queryFn">, -) => - useQuery({ - queryKey: Common.UseConnectionServiceGetConnectionKeyFn( - { connectionId }, - queryKey, - ), - queryFn: () => ConnectionService.getConnection({ connectionId }) as TData, - ...options, - }); -/** - * Get Connections - * Get all connection entries. + * List Backfills * @param data The data for the request. + * @param data.dagId * @param data.limit * @param data.offset * @param data.orderBy - * @returns ConnectionCollectionResponse Successful Response + * @returns unknown Successful Response * @throws ApiError */ -export const useConnectionServiceGetConnections = < - TData = Common.ConnectionServiceGetConnectionsDefaultResponse, +export const useBackfillServiceListBackfills = < + TData = Common.BackfillServiceListBackfillsDefaultResponse, TError = unknown, TQueryKey extends Array = unknown[], >( { + dagId, limit, offset, orderBy, }: { + dagId: string; limit?: number; offset?: number; orderBy?: string; - } = {}, - queryKey?: TQueryKey, - options?: Omit, "queryKey" | "queryFn">, -) => - useQuery({ - queryKey: Common.UseConnectionServiceGetConnectionsKeyFn( - { limit, offset, orderBy }, - queryKey, - ), - queryFn: () => - ConnectionService.getConnections({ limit, offset, orderBy }) as TData, - ...options, - }); -/** - * Get Dag Run - * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @returns DAGRunResponse Successful Response - * @throws ApiError - */ -export const useDagRunServiceGetDagRun = < - TData = Common.DagRunServiceGetDagRunDefaultResponse, - TError = unknown, - TQueryKey extends Array = unknown[], ->( - { - dagId, - dagRunId, - }: { - dagId: string; - dagRunId: string; }, queryKey?: TQueryKey, options?: Omit, "queryKey" | "queryFn">, ) => useQuery({ - queryKey: Common.UseDagRunServiceGetDagRunKeyFn( - { dagId, dagRunId }, + queryKey: Common.UseBackfillServiceListBackfillsKeyFn( + { dagId, limit, offset, orderBy }, queryKey, ), - queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }) as TData, + queryFn: () => + BackfillService.listBackfills({ dagId, limit, offset, orderBy }) as TData, ...options, }); /** - * Get Dag Source - * Get source code using file token. + * Get Backfill * @param data The data for the request. - * @param data.fileToken - * @param data.accept - * @returns DAGSourceResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ -export const useDagSourceServiceGetDagSource = < - TData = Common.DagSourceServiceGetDagSourceDefaultResponse, +export const useBackfillServiceGetBackfill = < + TData = Common.BackfillServiceGetBackfillDefaultResponse, TError = unknown, TQueryKey extends Array = unknown[], >( { - accept, - fileToken, + backfillId, }: { - accept?: string; - fileToken: string; + backfillId: string; }, queryKey?: TQueryKey, options?: Omit, "queryKey" | "queryFn">, ) => useQuery({ - queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn( - { accept, fileToken }, + queryKey: Common.UseBackfillServiceGetBackfillKeyFn( + { backfillId }, queryKey, ), - queryFn: () => - DagSourceService.getDagSource({ accept, fileToken }) as TData, + queryFn: () => BackfillService.getBackfill({ backfillId }) as TData, ...options, }); /** @@ -474,6 +415,135 @@ export const useDagServiceGetDagDetails = < queryFn: () => DagService.getDagDetails({ dagId }) as TData, ...options, }); +/** + * Get Connection + * Get a connection entry. + * @param data The data for the request. + * @param data.connectionId + * @returns ConnectionResponse Successful Response + * @throws ApiError + */ +export const useConnectionServiceGetConnection = < + TData = Common.ConnectionServiceGetConnectionDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + connectionId, + }: { + connectionId: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseConnectionServiceGetConnectionKeyFn( + { connectionId }, + queryKey, + ), + queryFn: () => ConnectionService.getConnection({ connectionId }) as TData, + ...options, + }); +/** + * Get Connections + * Get all connection entries. + * @param data The data for the request. + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns ConnectionCollectionResponse Successful Response + * @throws ApiError + */ +export const useConnectionServiceGetConnections = < + TData = Common.ConnectionServiceGetConnectionsDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + limit, + offset, + orderBy, + }: { + limit?: number; + offset?: number; + orderBy?: string; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseConnectionServiceGetConnectionsKeyFn( + { limit, offset, orderBy }, + queryKey, + ), + queryFn: () => + ConnectionService.getConnections({ limit, offset, orderBy }) as TData, + ...options, + }); +/** + * Get Dag Run + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ +export const useDagRunServiceGetDagRun = < + TData = Common.DagRunServiceGetDagRunDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + dagId, + dagRunId, + }: { + dagId: string; + dagRunId: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseDagRunServiceGetDagRunKeyFn( + { dagId, dagRunId }, + queryKey, + ), + queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }) as TData, + ...options, + }); +/** + * Get Dag Source + * Get source code using file token. + * @param data The data for the request. + * @param data.fileToken + * @param data.accept + * @returns DAGSourceResponse Successful Response + * @throws ApiError + */ +export const useDagSourceServiceGetDagSource = < + TData = Common.DagSourceServiceGetDagSourceDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + fileToken, + }: { + accept?: string; + fileToken: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn( + { accept, fileToken }, + queryKey, + ), + queryFn: () => + DagSourceService.getDagSource({ accept, fileToken }) as TData, + ...options, + }); /** * Get Event Log * @param data The data for the request. @@ -520,6 +590,54 @@ export const useMonitorServiceGetHealth = < queryFn: () => MonitorService.getHealth() as TData, ...options, }); +/** + * List Dag Warnings + * Get a list of DAG warnings. + * @param data The data for the request. + * @param data.dagId + * @param data.warningType + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns DAGWarningCollectionResponse Successful Response + * @throws ApiError + */ +export const useDagWarningServiceListDagWarnings = < + TData = Common.DagWarningServiceListDagWarningsDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + dagId, + limit, + offset, + orderBy, + warningType, + }: { + dagId?: string; + limit?: number; + offset?: number; + orderBy?: string; + warningType?: DagWarningType; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseDagWarningServiceListDagWarningsKeyFn( + { dagId, limit, offset, orderBy, warningType }, + queryKey, + ), + queryFn: () => + DagWarningService.listDagWarnings({ + dagId, + limit, + offset, + orderBy, + warningType, + }) as TData, + ...options, + }); /** * Get Plugins * @param data The data for the request. @@ -808,6 +926,44 @@ export const useVersionServiceGetVersion = < queryFn: () => VersionService.getVersion() as TData, ...options, }); +/** + * Create Backfill + * @param data The data for the request. + * @param data.requestBody + * @returns unknown Successful Response + * @throws ApiError + */ +export const useBackfillServiceCreateBackfill = < + TData = Common.BackfillServiceCreateBackfillMutationResult, + TError = unknown, + TContext = unknown, +>( + options?: Omit< + UseMutationOptions< + TData, + TError, + { + requestBody: BackfillPostBody; + }, + TContext + >, + "mutationFn" + >, +) => + useMutation< + TData, + TError, + { + requestBody: BackfillPostBody; + }, + TContext + >({ + mutationFn: ({ requestBody }) => + BackfillService.createBackfill({ + requestBody, + }) as unknown as Promise, + ...options, + }); /** * Post Pool * Create a Pool. @@ -885,18 +1041,14 @@ export const useVariableServicePostVariable = < ...options, }); /** - * Patch Dag Run State - * Modify a DAG Run. + * Pause Backfill * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @param data.requestBody - * @param data.updateMask - * @returns DAGRunResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ -export const useDagRunServicePatchDagRunState = < - TData = Common.DagRunServicePatchDagRunStateMutationResult, +export const useBackfillServicePauseBackfill = < + TData = Common.BackfillServicePauseBackfillMutationResult, TError = unknown, TContext = unknown, >( @@ -905,10 +1057,7 @@ export const useDagRunServicePatchDagRunState = < TData, TError, { - dagId: string; - dagRunId: string; - requestBody: DAGRunPatchBody; - updateMask?: string[]; + backfillId: unknown; }, TContext >, @@ -919,19 +1068,89 @@ export const useDagRunServicePatchDagRunState = < TData, TError, { - dagId: string; - dagRunId: string; - requestBody: DAGRunPatchBody; - updateMask?: string[]; + backfillId: unknown; }, TContext >({ - mutationFn: ({ dagId, dagRunId, requestBody, updateMask }) => - DagRunService.patchDagRunState({ - dagId, - dagRunId, - requestBody, - updateMask, + mutationFn: ({ backfillId }) => + BackfillService.pauseBackfill({ + backfillId, + }) as unknown as Promise, + ...options, + }); +/** + * Unpause Backfill + * @param data The data for the request. + * @param data.backfillId + * @returns unknown Successful Response + * @throws ApiError + */ +export const useBackfillServiceUnpauseBackfill = < + TData = Common.BackfillServiceUnpauseBackfillMutationResult, + TError = unknown, + TContext = unknown, +>( + options?: Omit< + UseMutationOptions< + TData, + TError, + { + backfillId: unknown; + }, + TContext + >, + "mutationFn" + >, +) => + useMutation< + TData, + TError, + { + backfillId: unknown; + }, + TContext + >({ + mutationFn: ({ backfillId }) => + BackfillService.unpauseBackfill({ + backfillId, + }) as unknown as Promise, + ...options, + }); +/** + * Cancel Backfill + * @param data The data for the request. + * @param data.backfillId + * @returns unknown Successful Response + * @throws ApiError + */ +export const useBackfillServiceCancelBackfill = < + TData = Common.BackfillServiceCancelBackfillMutationResult, + TError = unknown, + TContext = unknown, +>( + options?: Omit< + UseMutationOptions< + TData, + TError, + { + backfillId: unknown; + }, + TContext + >, + "mutationFn" + >, +) => + useMutation< + TData, + TError, + { + backfillId: unknown; + }, + TContext + >({ + mutationFn: ({ backfillId }) => + BackfillService.cancelBackfill({ + backfillId, }) as unknown as Promise, ...options, }); @@ -1068,6 +1287,57 @@ export const useDagServicePatchDag = < }) as unknown as Promise, ...options, }); +/** + * Patch Dag Run State + * Modify a DAG Run. + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @param data.requestBody + * @param data.updateMask + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ +export const useDagRunServicePatchDagRunState = < + TData = Common.DagRunServicePatchDagRunStateMutationResult, + TError = unknown, + TContext = unknown, +>( + options?: Omit< + UseMutationOptions< + TData, + TError, + { + dagId: string; + dagRunId: string; + requestBody: DAGRunPatchBody; + updateMask?: string[]; + }, + TContext + >, + "mutationFn" + >, +) => + useMutation< + TData, + TError, + { + dagId: string; + dagRunId: string; + requestBody: DAGRunPatchBody; + updateMask?: string[]; + }, + TContext + >({ + mutationFn: ({ dagId, dagRunId, requestBody, updateMask }) => + DagRunService.patchDagRunState({ + dagId, + dagRunId, + requestBody, + updateMask, + }) as unknown as Promise, + ...options, + }); /** * Patch Pool * Update a Pool. @@ -1163,15 +1433,15 @@ export const useVariableServicePatchVariable = < ...options, }); /** - * Delete Connection - * Delete a connection entry. + * Delete Dag + * Delete the specific DAG. * @param data The data for the request. - * @param data.connectionId - * @returns void Successful Response + * @param data.dagId + * @returns unknown Successful Response * @throws ApiError */ -export const useConnectionServiceDeleteConnection = < - TData = Common.ConnectionServiceDeleteConnectionMutationResult, +export const useDagServiceDeleteDag = < + TData = Common.DagServiceDeleteDagMutationResult, TError = unknown, TContext = unknown, >( @@ -1180,7 +1450,7 @@ export const useConnectionServiceDeleteConnection = < TData, TError, { - connectionId: string; + dagId: string; }, TContext >, @@ -1191,27 +1461,24 @@ export const useConnectionServiceDeleteConnection = < TData, TError, { - connectionId: string; + dagId: string; }, TContext >({ - mutationFn: ({ connectionId }) => - ConnectionService.deleteConnection({ - connectionId, - }) as unknown as Promise, + mutationFn: ({ dagId }) => + DagService.deleteDag({ dagId }) as unknown as Promise, ...options, }); /** - * Delete Dag Run - * Delete a DAG Run entry. + * Delete Connection + * Delete a connection entry. * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId + * @param data.connectionId * @returns void Successful Response * @throws ApiError */ -export const useDagRunServiceDeleteDagRun = < - TData = Common.DagRunServiceDeleteDagRunMutationResult, +export const useConnectionServiceDeleteConnection = < + TData = Common.ConnectionServiceDeleteConnectionMutationResult, TError = unknown, TContext = unknown, >( @@ -1220,8 +1487,7 @@ export const useDagRunServiceDeleteDagRun = < TData, TError, { - dagId: string; - dagRunId: string; + connectionId: string; }, TContext >, @@ -1232,28 +1498,27 @@ export const useDagRunServiceDeleteDagRun = < TData, TError, { - dagId: string; - dagRunId: string; + connectionId: string; }, TContext >({ - mutationFn: ({ dagId, dagRunId }) => - DagRunService.deleteDagRun({ - dagId, - dagRunId, + mutationFn: ({ connectionId }) => + ConnectionService.deleteConnection({ + connectionId, }) as unknown as Promise, ...options, }); /** - * Delete Dag - * Delete the specific DAG. + * Delete Dag Run + * Delete a DAG Run entry. * @param data The data for the request. * @param data.dagId - * @returns unknown Successful Response + * @param data.dagRunId + * @returns void Successful Response * @throws ApiError */ -export const useDagServiceDeleteDag = < - TData = Common.DagServiceDeleteDagMutationResult, +export const useDagRunServiceDeleteDagRun = < + TData = Common.DagRunServiceDeleteDagRunMutationResult, TError = unknown, TContext = unknown, >( @@ -1263,6 +1528,7 @@ export const useDagServiceDeleteDag = < TError, { dagId: string; + dagRunId: string; }, TContext >, @@ -1274,11 +1540,15 @@ export const useDagServiceDeleteDag = < TError, { dagId: string; + dagRunId: string; }, TContext >({ - mutationFn: ({ dagId }) => - DagService.deleteDag({ dagId }) as unknown as Promise, + mutationFn: ({ dagId, dagRunId }) => + DagRunService.deleteDagRun({ + dagId, + dagRunId, + }) as unknown as Promise, ...options, }); /** diff --git a/airflow/ui/openapi-gen/queries/suspense.ts b/airflow/ui/openapi-gen/queries/suspense.ts index 2cb0841d71f288..1222b7f5536c26 100644 --- a/airflow/ui/openapi-gen/queries/suspense.ts +++ b/airflow/ui/openapi-gen/queries/suspense.ts @@ -3,10 +3,12 @@ import { UseQueryOptions, useSuspenseQuery } from "@tanstack/react-query"; import { AssetService, + BackfillService, ConnectionService, DagRunService, DagService, DagSourceService, + DagWarningService, DagsService, DashboardService, EventLogService, @@ -18,7 +20,7 @@ import { VariableService, VersionService, } from "../requests/services.gen"; -import { DagRunState } from "../requests/types.gen"; +import { DagRunState, DagWarningType } from "../requests/types.gen"; import * as Common from "./common"; /** @@ -159,132 +161,69 @@ export const useDagsServiceRecentDagRunsSuspense = < ...options, }); /** - * Get Connection - * Get a connection entry. - * @param data The data for the request. - * @param data.connectionId - * @returns ConnectionResponse Successful Response - * @throws ApiError - */ -export const useConnectionServiceGetConnectionSuspense = < - TData = Common.ConnectionServiceGetConnectionDefaultResponse, - TError = unknown, - TQueryKey extends Array = unknown[], ->( - { - connectionId, - }: { - connectionId: string; - }, - queryKey?: TQueryKey, - options?: Omit, "queryKey" | "queryFn">, -) => - useSuspenseQuery({ - queryKey: Common.UseConnectionServiceGetConnectionKeyFn( - { connectionId }, - queryKey, - ), - queryFn: () => ConnectionService.getConnection({ connectionId }) as TData, - ...options, - }); -/** - * Get Connections - * Get all connection entries. + * List Backfills * @param data The data for the request. + * @param data.dagId * @param data.limit * @param data.offset * @param data.orderBy - * @returns ConnectionCollectionResponse Successful Response + * @returns unknown Successful Response * @throws ApiError */ -export const useConnectionServiceGetConnectionsSuspense = < - TData = Common.ConnectionServiceGetConnectionsDefaultResponse, +export const useBackfillServiceListBackfillsSuspense = < + TData = Common.BackfillServiceListBackfillsDefaultResponse, TError = unknown, TQueryKey extends Array = unknown[], >( { + dagId, limit, offset, orderBy, }: { + dagId: string; limit?: number; offset?: number; orderBy?: string; - } = {}, - queryKey?: TQueryKey, - options?: Omit, "queryKey" | "queryFn">, -) => - useSuspenseQuery({ - queryKey: Common.UseConnectionServiceGetConnectionsKeyFn( - { limit, offset, orderBy }, - queryKey, - ), - queryFn: () => - ConnectionService.getConnections({ limit, offset, orderBy }) as TData, - ...options, - }); -/** - * Get Dag Run - * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @returns DAGRunResponse Successful Response - * @throws ApiError - */ -export const useDagRunServiceGetDagRunSuspense = < - TData = Common.DagRunServiceGetDagRunDefaultResponse, - TError = unknown, - TQueryKey extends Array = unknown[], ->( - { - dagId, - dagRunId, - }: { - dagId: string; - dagRunId: string; }, queryKey?: TQueryKey, options?: Omit, "queryKey" | "queryFn">, ) => useSuspenseQuery({ - queryKey: Common.UseDagRunServiceGetDagRunKeyFn( - { dagId, dagRunId }, + queryKey: Common.UseBackfillServiceListBackfillsKeyFn( + { dagId, limit, offset, orderBy }, queryKey, ), - queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }) as TData, + queryFn: () => + BackfillService.listBackfills({ dagId, limit, offset, orderBy }) as TData, ...options, }); /** - * Get Dag Source - * Get source code using file token. + * Get Backfill * @param data The data for the request. - * @param data.fileToken - * @param data.accept - * @returns DAGSourceResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ -export const useDagSourceServiceGetDagSourceSuspense = < - TData = Common.DagSourceServiceGetDagSourceDefaultResponse, +export const useBackfillServiceGetBackfillSuspense = < + TData = Common.BackfillServiceGetBackfillDefaultResponse, TError = unknown, TQueryKey extends Array = unknown[], >( { - accept, - fileToken, + backfillId, }: { - accept?: string; - fileToken: string; + backfillId: string; }, queryKey?: TQueryKey, options?: Omit, "queryKey" | "queryFn">, ) => useSuspenseQuery({ - queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn( - { accept, fileToken }, + queryKey: Common.UseBackfillServiceGetBackfillKeyFn( + { backfillId }, queryKey, ), - queryFn: () => - DagSourceService.getDagSource({ accept, fileToken }) as TData, + queryFn: () => BackfillService.getBackfill({ backfillId }) as TData, ...options, }); /** @@ -462,6 +401,135 @@ export const useDagServiceGetDagDetailsSuspense = < queryFn: () => DagService.getDagDetails({ dagId }) as TData, ...options, }); +/** + * Get Connection + * Get a connection entry. + * @param data The data for the request. + * @param data.connectionId + * @returns ConnectionResponse Successful Response + * @throws ApiError + */ +export const useConnectionServiceGetConnectionSuspense = < + TData = Common.ConnectionServiceGetConnectionDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + connectionId, + }: { + connectionId: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseConnectionServiceGetConnectionKeyFn( + { connectionId }, + queryKey, + ), + queryFn: () => ConnectionService.getConnection({ connectionId }) as TData, + ...options, + }); +/** + * Get Connections + * Get all connection entries. + * @param data The data for the request. + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns ConnectionCollectionResponse Successful Response + * @throws ApiError + */ +export const useConnectionServiceGetConnectionsSuspense = < + TData = Common.ConnectionServiceGetConnectionsDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + limit, + offset, + orderBy, + }: { + limit?: number; + offset?: number; + orderBy?: string; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseConnectionServiceGetConnectionsKeyFn( + { limit, offset, orderBy }, + queryKey, + ), + queryFn: () => + ConnectionService.getConnections({ limit, offset, orderBy }) as TData, + ...options, + }); +/** + * Get Dag Run + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ +export const useDagRunServiceGetDagRunSuspense = < + TData = Common.DagRunServiceGetDagRunDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + dagId, + dagRunId, + }: { + dagId: string; + dagRunId: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseDagRunServiceGetDagRunKeyFn( + { dagId, dagRunId }, + queryKey, + ), + queryFn: () => DagRunService.getDagRun({ dagId, dagRunId }) as TData, + ...options, + }); +/** + * Get Dag Source + * Get source code using file token. + * @param data The data for the request. + * @param data.fileToken + * @param data.accept + * @returns DAGSourceResponse Successful Response + * @throws ApiError + */ +export const useDagSourceServiceGetDagSourceSuspense = < + TData = Common.DagSourceServiceGetDagSourceDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + fileToken, + }: { + accept?: string; + fileToken: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseDagSourceServiceGetDagSourceKeyFn( + { accept, fileToken }, + queryKey, + ), + queryFn: () => + DagSourceService.getDagSource({ accept, fileToken }) as TData, + ...options, + }); /** * Get Event Log * @param data The data for the request. @@ -508,6 +576,54 @@ export const useMonitorServiceGetHealthSuspense = < queryFn: () => MonitorService.getHealth() as TData, ...options, }); +/** + * List Dag Warnings + * Get a list of DAG warnings. + * @param data The data for the request. + * @param data.dagId + * @param data.warningType + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns DAGWarningCollectionResponse Successful Response + * @throws ApiError + */ +export const useDagWarningServiceListDagWarningsSuspense = < + TData = Common.DagWarningServiceListDagWarningsDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + dagId, + limit, + offset, + orderBy, + warningType, + }: { + dagId?: string; + limit?: number; + offset?: number; + orderBy?: string; + warningType?: DagWarningType; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseDagWarningServiceListDagWarningsKeyFn( + { dagId, limit, offset, orderBy, warningType }, + queryKey, + ), + queryFn: () => + DagWarningService.listDagWarnings({ + dagId, + limit, + offset, + orderBy, + warningType, + }) as TData, + ...options, + }); /** * Get Plugins * @param data The data for the request. diff --git a/airflow/ui/openapi-gen/requests/schemas.gen.ts b/airflow/ui/openapi-gen/requests/schemas.gen.ts index 712cc8cae984e1..cf51451c98bdcf 100644 --- a/airflow/ui/openapi-gen/requests/schemas.gen.ts +++ b/airflow/ui/openapi-gen/requests/schemas.gen.ts @@ -89,6 +89,48 @@ export const $AppBuilderViewResponse = { description: "Serializer for AppBuilder View responses.", } as const; +export const $BackfillPostBody = { + properties: { + dag_id: { + type: "string", + title: "Dag Id", + }, + from_date: { + type: "string", + format: "date-time", + title: "From Date", + }, + to_date: { + type: "string", + format: "date-time", + title: "To Date", + }, + run_backwards: { + type: "boolean", + title: "Run Backwards", + default: false, + }, + dag_run_conf: { + type: "object", + title: "Dag Run Conf", + default: {}, + }, + reprocess_behavior: { + $ref: "#/components/schemas/ReprocessBehavior", + default: "none", + }, + max_active_runs: { + type: "integer", + title: "Max Active Runs", + default: 10, + }, + }, + type: "object", + required: ["dag_id", "from_date", "to_date"], + title: "BackfillPostBody", + description: "Object used for create backfill request.", +} as const; + export const $BaseInfoSchema = { properties: { status: { @@ -1130,6 +1172,51 @@ export const $DAGTagCollectionResponse = { description: "DAG Tags Collection serializer for responses.", } as const; +export const $DAGWarningCollectionResponse = { + properties: { + dag_warnings: { + items: { + $ref: "#/components/schemas/DAGWarningResponse", + }, + type: "array", + title: "Dag Warnings", + }, + total_entries: { + type: "integer", + title: "Total Entries", + }, + }, + type: "object", + required: ["dag_warnings", "total_entries"], + title: "DAGWarningCollectionResponse", + description: "DAG warning collection serializer for responses.", +} as const; + +export const $DAGWarningResponse = { + properties: { + dag_id: { + type: "string", + title: "Dag Id", + }, + warning_type: { + $ref: "#/components/schemas/DagWarningType", + }, + message: { + type: "string", + title: "Message", + }, + timestamp: { + type: "string", + format: "date-time", + title: "Timestamp", + }, + }, + type: "object", + required: ["dag_id", "warning_type", "message", "timestamp"], + title: "DAGWarningResponse", + description: "DAG Warning serializer for responses.", +} as const; + export const $DAGWithLatestDagRunsCollectionResponse = { properties: { total_entries: { @@ -1483,6 +1570,16 @@ export const $DagTagPydantic = { "Serializable representation of the DagTag ORM SqlAlchemyModel used by internal API.", } as const; +export const $DagWarningType = { + type: "string", + enum: ["asset conflict", "non-existent pool"], + title: "DagWarningType", + description: `Enum for DAG warning types. + +This is the set of allowable values for the \`\`warning_type\`\` field +in the DagWarning model.`, +} as const; + export const $EventLogResponse = { properties: { event_log_id: { @@ -2161,6 +2258,15 @@ export const $ProviderResponse = { description: "Provider serializer for responses.", } as const; +export const $ReprocessBehavior = { + type: "string", + enum: ["failed", "completed", "none"], + title: "ReprocessBehavior", + description: `Internal enum for setting reprocess behavior in a backfill. + +:meta private:`, +} as const; + export const $SchedulerInfoSchema = { properties: { status: { @@ -2194,6 +2300,10 @@ export const $SchedulerInfoSchema = { export const $TaskInstanceResponse = { properties: { + id: { + type: "string", + title: "Id", + }, task_id: { type: "string", title: "Task Id", @@ -2423,6 +2533,7 @@ export const $TaskInstanceResponse = { }, type: "object", required: [ + "id", "task_id", "dag_id", "dag_run_id", diff --git a/airflow/ui/openapi-gen/requests/services.gen.ts b/airflow/ui/openapi-gen/requests/services.gen.ts index 486e04b056f8a9..e82d0c5146fa85 100644 --- a/airflow/ui/openapi-gen/requests/services.gen.ts +++ b/airflow/ui/openapi-gen/requests/services.gen.ts @@ -9,20 +9,18 @@ import type { HistoricalMetricsResponse, RecentDagRunsData, RecentDagRunsResponse, - DeleteConnectionData, - DeleteConnectionResponse, - GetConnectionData, - GetConnectionResponse, - GetConnectionsData, - GetConnectionsResponse, - GetDagRunData, - GetDagRunResponse, - DeleteDagRunData, - DeleteDagRunResponse, - PatchDagRunStateData, - PatchDagRunStateResponse, - GetDagSourceData, - GetDagSourceResponse, + ListBackfillsData, + ListBackfillsResponse, + CreateBackfillData, + CreateBackfillResponse, + GetBackfillData, + GetBackfillResponse, + PauseBackfillData, + PauseBackfillResponse, + UnpauseBackfillData, + UnpauseBackfillResponse, + CancelBackfillData, + CancelBackfillResponse, GetDagsData, GetDagsResponse, PatchDagsData, @@ -37,9 +35,25 @@ import type { DeleteDagResponse, GetDagDetailsData, GetDagDetailsResponse, + DeleteConnectionData, + DeleteConnectionResponse, + GetConnectionData, + GetConnectionResponse, + GetConnectionsData, + GetConnectionsResponse, + GetDagRunData, + GetDagRunResponse, + DeleteDagRunData, + DeleteDagRunResponse, + PatchDagRunStateData, + PatchDagRunStateResponse, + GetDagSourceData, + GetDagSourceResponse, GetEventLogData, GetEventLogResponse, GetHealthResponse, + ListDagWarningsData, + ListDagWarningsResponse, GetPluginsData, GetPluginsResponse, DeletePoolData, @@ -166,108 +180,77 @@ export class DagsService { } } -export class ConnectionService { +export class BackfillService { /** - * Delete Connection - * Delete a connection entry. + * List Backfills * @param data The data for the request. - * @param data.connectionId - * @returns void Successful Response - * @throws ApiError - */ - public static deleteConnection( - data: DeleteConnectionData, - ): CancelablePromise { - return __request(OpenAPI, { - method: "DELETE", - url: "/public/connections/{connection_id}", - path: { - connection_id: data.connectionId, - }, - errors: { - 401: "Unauthorized", - 403: "Forbidden", - 404: "Not Found", - 422: "Validation Error", - }, - }); - } - - /** - * Get Connection - * Get a connection entry. - * @param data The data for the request. - * @param data.connectionId - * @returns ConnectionResponse Successful Response + * @param data.dagId + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns unknown Successful Response * @throws ApiError */ - public static getConnection( - data: GetConnectionData, - ): CancelablePromise { + public static listBackfills( + data: ListBackfillsData, + ): CancelablePromise { return __request(OpenAPI, { method: "GET", - url: "/public/connections/{connection_id}", - path: { - connection_id: data.connectionId, + url: "/public/backfills/", + query: { + dag_id: data.dagId, + limit: data.limit, + offset: data.offset, + order_by: data.orderBy, }, errors: { 401: "Unauthorized", 403: "Forbidden", - 404: "Not Found", 422: "Validation Error", }, }); } /** - * Get Connections - * Get all connection entries. + * Create Backfill * @param data The data for the request. - * @param data.limit - * @param data.offset - * @param data.orderBy - * @returns ConnectionCollectionResponse Successful Response + * @param data.requestBody + * @returns unknown Successful Response * @throws ApiError */ - public static getConnections( - data: GetConnectionsData = {}, - ): CancelablePromise { + public static createBackfill( + data: CreateBackfillData, + ): CancelablePromise { return __request(OpenAPI, { - method: "GET", - url: "/public/connections/", - query: { - limit: data.limit, - offset: data.offset, - order_by: data.orderBy, - }, + method: "POST", + url: "/public/backfills/", + body: data.requestBody, + mediaType: "application/json", errors: { 401: "Unauthorized", 403: "Forbidden", 404: "Not Found", + 409: "Conflict", 422: "Validation Error", }, }); } -} -export class DagRunService { /** - * Get Dag Run + * Get Backfill * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @returns DAGRunResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ - public static getDagRun( - data: GetDagRunData, - ): CancelablePromise { + public static getBackfill( + data: GetBackfillData, + ): CancelablePromise { return __request(OpenAPI, { method: "GET", - url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + url: "/public/backfills/{backfill_id}", path: { - dag_id: data.dagId, - dag_run_id: data.dagRunId, + backfill_id: data.backfillId, }, errors: { 401: "Unauthorized", @@ -279,99 +262,78 @@ export class DagRunService { } /** - * Delete Dag Run - * Delete a DAG Run entry. + * Pause Backfill * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @returns void Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ - public static deleteDagRun( - data: DeleteDagRunData, - ): CancelablePromise { + public static pauseBackfill( + data: PauseBackfillData, + ): CancelablePromise { return __request(OpenAPI, { - method: "DELETE", - url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + method: "PUT", + url: "/public/backfills/{backfill_id}/pause", path: { - dag_id: data.dagId, - dag_run_id: data.dagRunId, + backfill_id: data.backfillId, }, errors: { - 400: "Bad Request", 401: "Unauthorized", 403: "Forbidden", 404: "Not Found", + 409: "Conflict", 422: "Validation Error", }, }); } /** - * Patch Dag Run State - * Modify a DAG Run. + * Unpause Backfill * @param data The data for the request. - * @param data.dagId - * @param data.dagRunId - * @param data.requestBody - * @param data.updateMask - * @returns DAGRunResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ - public static patchDagRunState( - data: PatchDagRunStateData, - ): CancelablePromise { + public static unpauseBackfill( + data: UnpauseBackfillData, + ): CancelablePromise { return __request(OpenAPI, { - method: "PATCH", - url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + method: "PUT", + url: "/public/backfills/{backfill_id}/unpause", path: { - dag_id: data.dagId, - dag_run_id: data.dagRunId, + backfill_id: data.backfillId, }, - query: { - update_mask: data.updateMask, - }, - body: data.requestBody, - mediaType: "application/json", errors: { - 400: "Bad Request", 401: "Unauthorized", 403: "Forbidden", 404: "Not Found", + 409: "Conflict", 422: "Validation Error", }, }); } -} -export class DagSourceService { /** - * Get Dag Source - * Get source code using file token. + * Cancel Backfill * @param data The data for the request. - * @param data.fileToken - * @param data.accept - * @returns DAGSourceResponse Successful Response + * @param data.backfillId + * @returns unknown Successful Response * @throws ApiError */ - public static getDagSource( - data: GetDagSourceData, - ): CancelablePromise { + public static cancelBackfill( + data: CancelBackfillData, + ): CancelablePromise { return __request(OpenAPI, { - method: "GET", - url: "/public/dagSources/{file_token}", + method: "PUT", + url: "/public/backfills/{backfill_id}/cancel", path: { - file_token: data.fileToken, - }, - headers: { - accept: data.accept, + backfill_id: data.backfillId, }, errors: { - 400: "Bad Request", 401: "Unauthorized", 403: "Forbidden", 404: "Not Found", - 406: "Not Acceptable", + 409: "Conflict", 422: "Validation Error", }, }); @@ -611,6 +573,218 @@ export class DagService { } } +export class ConnectionService { + /** + * Delete Connection + * Delete a connection entry. + * @param data The data for the request. + * @param data.connectionId + * @returns void Successful Response + * @throws ApiError + */ + public static deleteConnection( + data: DeleteConnectionData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "DELETE", + url: "/public/connections/{connection_id}", + path: { + connection_id: data.connectionId, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } + + /** + * Get Connection + * Get a connection entry. + * @param data The data for the request. + * @param data.connectionId + * @returns ConnectionResponse Successful Response + * @throws ApiError + */ + public static getConnection( + data: GetConnectionData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/connections/{connection_id}", + path: { + connection_id: data.connectionId, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } + + /** + * Get Connections + * Get all connection entries. + * @param data The data for the request. + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns ConnectionCollectionResponse Successful Response + * @throws ApiError + */ + public static getConnections( + data: GetConnectionsData = {}, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/connections/", + query: { + limit: data.limit, + offset: data.offset, + order_by: data.orderBy, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } +} + +export class DagRunService { + /** + * Get Dag Run + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ + public static getDagRun( + data: GetDagRunData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + path: { + dag_id: data.dagId, + dag_run_id: data.dagRunId, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } + + /** + * Delete Dag Run + * Delete a DAG Run entry. + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @returns void Successful Response + * @throws ApiError + */ + public static deleteDagRun( + data: DeleteDagRunData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "DELETE", + url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + path: { + dag_id: data.dagId, + dag_run_id: data.dagRunId, + }, + errors: { + 400: "Bad Request", + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } + + /** + * Patch Dag Run State + * Modify a DAG Run. + * @param data The data for the request. + * @param data.dagId + * @param data.dagRunId + * @param data.requestBody + * @param data.updateMask + * @returns DAGRunResponse Successful Response + * @throws ApiError + */ + public static patchDagRunState( + data: PatchDagRunStateData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "PATCH", + url: "/public/dags/{dag_id}/dagRuns/{dag_run_id}", + path: { + dag_id: data.dagId, + dag_run_id: data.dagRunId, + }, + query: { + update_mask: data.updateMask, + }, + body: data.requestBody, + mediaType: "application/json", + errors: { + 400: "Bad Request", + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 422: "Validation Error", + }, + }); + } +} + +export class DagSourceService { + /** + * Get Dag Source + * Get source code using file token. + * @param data The data for the request. + * @param data.fileToken + * @param data.accept + * @returns DAGSourceResponse Successful Response + * @throws ApiError + */ + public static getDagSource( + data: GetDagSourceData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/dagSources/{file_token}", + path: { + file_token: data.fileToken, + }, + headers: { + accept: data.accept, + }, + errors: { + 400: "Bad Request", + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 406: "Not Acceptable", + 422: "Validation Error", + }, + }); + } +} + export class EventLogService { /** * Get Event Log @@ -652,6 +826,41 @@ export class MonitorService { } } +export class DagWarningService { + /** + * List Dag Warnings + * Get a list of DAG warnings. + * @param data The data for the request. + * @param data.dagId + * @param data.warningType + * @param data.limit + * @param data.offset + * @param data.orderBy + * @returns DAGWarningCollectionResponse Successful Response + * @throws ApiError + */ + public static listDagWarnings( + data: ListDagWarningsData = {}, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/dagWarnings", + query: { + dag_id: data.dagId, + warning_type: data.warningType, + limit: data.limit, + offset: data.offset, + order_by: data.orderBy, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 422: "Validation Error", + }, + }); + } +} + export class PluginService { /** * Get Plugins diff --git a/airflow/ui/openapi-gen/requests/types.gen.ts b/airflow/ui/openapi-gen/requests/types.gen.ts index 0580694ba78f0b..18d5bc296eb286 100644 --- a/airflow/ui/openapi-gen/requests/types.gen.ts +++ b/airflow/ui/openapi-gen/requests/types.gen.ts @@ -21,6 +21,21 @@ export type AppBuilderViewResponse = { [key: string]: unknown; }; +/** + * Object used for create backfill request. + */ +export type BackfillPostBody = { + dag_id: string; + from_date: string; + to_date: string; + run_backwards?: boolean; + dag_run_conf?: { + [key: string]: unknown; + }; + reprocess_behavior?: ReprocessBehavior; + max_active_runs?: number; +}; + /** * Base status field for metadatabase and scheduler. */ @@ -222,6 +237,24 @@ export type DAGTagCollectionResponse = { total_entries: number; }; +/** + * DAG warning collection serializer for responses. + */ +export type DAGWarningCollectionResponse = { + dag_warnings: Array; + total_entries: number; +}; + +/** + * DAG Warning serializer for responses. + */ +export type DAGWarningResponse = { + dag_id: string; + warning_type: DagWarningType; + message: string; + timestamp: string; +}; + /** * DAG with latest dag runs collection response serializer. */ @@ -312,6 +345,14 @@ export type DagTagPydantic = { dag_id: string; }; +/** + * Enum for DAG warning types. + * + * This is the set of allowable values for the ``warning_type`` field + * in the DagWarning model. + */ +export type DagWarningType = "asset conflict" | "non-existent pool"; + /** * Event Log Response. */ @@ -476,6 +517,13 @@ export type ProviderResponse = { version: string; }; +/** + * Internal enum for setting reprocess behavior in a backfill. + * + * :meta private: + */ +export type ReprocessBehavior = "failed" | "completed" | "none"; + /** * Schema for Scheduler info. */ @@ -488,6 +536,7 @@ export type SchedulerInfoSchema = { * TaskInstance serializer for responses. */ export type TaskInstanceResponse = { + id: string; task_id: string; dag_id: string; dag_run_id: string; @@ -648,55 +697,44 @@ export type RecentDagRunsData = { export type RecentDagRunsResponse = DAGWithLatestDagRunsCollectionResponse; -export type DeleteConnectionData = { - connectionId: string; -}; - -export type DeleteConnectionResponse = void; - -export type GetConnectionData = { - connectionId: string; -}; - -export type GetConnectionResponse = ConnectionResponse; - -export type GetConnectionsData = { +export type ListBackfillsData = { + dagId: string; limit?: number; offset?: number; orderBy?: string; }; -export type GetConnectionsResponse = ConnectionCollectionResponse; +export type ListBackfillsResponse = unknown; -export type GetDagRunData = { - dagId: string; - dagRunId: string; +export type CreateBackfillData = { + requestBody: BackfillPostBody; }; -export type GetDagRunResponse = DAGRunResponse; +export type CreateBackfillResponse = unknown; -export type DeleteDagRunData = { - dagId: string; - dagRunId: string; +export type GetBackfillData = { + backfillId: string; }; -export type DeleteDagRunResponse = void; +export type GetBackfillResponse = unknown; -export type PatchDagRunStateData = { - dagId: string; - dagRunId: string; - requestBody: DAGRunPatchBody; - updateMask?: Array | null; +export type PauseBackfillData = { + backfillId: unknown; }; -export type PatchDagRunStateResponse = DAGRunResponse; +export type PauseBackfillResponse = unknown; -export type GetDagSourceData = { - accept?: string; - fileToken: string; +export type UnpauseBackfillData = { + backfillId: unknown; }; -export type GetDagSourceResponse = DAGSourceResponse; +export type UnpauseBackfillResponse = unknown; + +export type CancelBackfillData = { + backfillId: unknown; +}; + +export type CancelBackfillResponse = unknown; export type GetDagsData = { dagDisplayNamePattern?: string | null; @@ -763,6 +801,56 @@ export type GetDagDetailsData = { export type GetDagDetailsResponse = DAGDetailsResponse; +export type DeleteConnectionData = { + connectionId: string; +}; + +export type DeleteConnectionResponse = void; + +export type GetConnectionData = { + connectionId: string; +}; + +export type GetConnectionResponse = ConnectionResponse; + +export type GetConnectionsData = { + limit?: number; + offset?: number; + orderBy?: string; +}; + +export type GetConnectionsResponse = ConnectionCollectionResponse; + +export type GetDagRunData = { + dagId: string; + dagRunId: string; +}; + +export type GetDagRunResponse = DAGRunResponse; + +export type DeleteDagRunData = { + dagId: string; + dagRunId: string; +}; + +export type DeleteDagRunResponse = void; + +export type PatchDagRunStateData = { + dagId: string; + dagRunId: string; + requestBody: DAGRunPatchBody; + updateMask?: Array | null; +}; + +export type PatchDagRunStateResponse = DAGRunResponse; + +export type GetDagSourceData = { + accept?: string; + fileToken: string; +}; + +export type GetDagSourceResponse = DAGSourceResponse; + export type GetEventLogData = { eventLogId: number; }; @@ -771,6 +859,16 @@ export type GetEventLogResponse = EventLogResponse; export type GetHealthResponse = HealthInfoSchema; +export type ListDagWarningsData = { + dagId?: string | null; + limit?: number; + offset?: number; + orderBy?: string; + warningType?: DagWarningType | null; +}; + +export type ListDagWarningsResponse = DAGWarningCollectionResponse; + export type GetPluginsData = { limit?: number; offset?: number; @@ -924,14 +1022,14 @@ export type $OpenApiTs = { }; }; }; - "/public/connections/{connection_id}": { - delete: { - req: DeleteConnectionData; + "/public/backfills/": { + get: { + req: ListBackfillsData; res: { /** * Successful Response */ - 204: void; + 200: unknown; /** * Unauthorized */ @@ -940,23 +1038,19 @@ export type $OpenApiTs = { * Forbidden */ 403: HTTPExceptionResponse; - /** - * Not Found - */ - 404: HTTPExceptionResponse; /** * Validation Error */ 422: HTTPValidationError; }; }; - get: { - req: GetConnectionData; + post: { + req: CreateBackfillData; res: { /** * Successful Response */ - 200: ConnectionResponse; + 200: unknown; /** * Unauthorized */ @@ -969,6 +1063,10 @@ export type $OpenApiTs = { * Not Found */ 404: HTTPExceptionResponse; + /** + * Conflict + */ + 409: HTTPExceptionResponse; /** * Validation Error */ @@ -976,14 +1074,14 @@ export type $OpenApiTs = { }; }; }; - "/public/connections/": { + "/public/backfills/{backfill_id}": { get: { - req: GetConnectionsData; + req: GetBackfillData; res: { /** * Successful Response */ - 200: ConnectionCollectionResponse; + 200: unknown; /** * Unauthorized */ @@ -1003,14 +1101,14 @@ export type $OpenApiTs = { }; }; }; - "/public/dags/{dag_id}/dagRuns/{dag_run_id}": { - get: { - req: GetDagRunData; + "/public/backfills/{backfill_id}/pause": { + put: { + req: PauseBackfillData; res: { /** * Successful Response */ - 200: DAGRunResponse; + 200: unknown; /** * Unauthorized */ @@ -1024,51 +1122,24 @@ export type $OpenApiTs = { */ 404: HTTPExceptionResponse; /** - * Validation Error + * Conflict */ - 422: HTTPValidationError; - }; - }; - delete: { - req: DeleteDagRunData; - res: { - /** - * Successful Response - */ - 204: void; - /** - * Bad Request - */ - 400: HTTPExceptionResponse; - /** - * Unauthorized - */ - 401: HTTPExceptionResponse; - /** - * Forbidden - */ - 403: HTTPExceptionResponse; - /** - * Not Found - */ - 404: HTTPExceptionResponse; + 409: HTTPExceptionResponse; /** * Validation Error */ 422: HTTPValidationError; }; }; - patch: { - req: PatchDagRunStateData; + }; + "/public/backfills/{backfill_id}/unpause": { + put: { + req: UnpauseBackfillData; res: { /** * Successful Response */ - 200: DAGRunResponse; - /** - * Bad Request - */ - 400: HTTPExceptionResponse; + 200: unknown; /** * Unauthorized */ @@ -1081,6 +1152,10 @@ export type $OpenApiTs = { * Not Found */ 404: HTTPExceptionResponse; + /** + * Conflict + */ + 409: HTTPExceptionResponse; /** * Validation Error */ @@ -1088,18 +1163,14 @@ export type $OpenApiTs = { }; }; }; - "/public/dagSources/{file_token}": { - get: { - req: GetDagSourceData; + "/public/backfills/{backfill_id}/cancel": { + put: { + req: CancelBackfillData; res: { /** * Successful Response */ - 200: DAGSourceResponse; - /** - * Bad Request - */ - 400: HTTPExceptionResponse; + 200: unknown; /** * Unauthorized */ @@ -1113,9 +1184,9 @@ export type $OpenApiTs = { */ 404: HTTPExceptionResponse; /** - * Not Acceptable + * Conflict */ - 406: HTTPExceptionResponse; + 409: HTTPExceptionResponse; /** * Validation Error */ @@ -1310,6 +1381,205 @@ export type $OpenApiTs = { }; }; }; + "/public/connections/{connection_id}": { + delete: { + req: DeleteConnectionData; + res: { + /** + * Successful Response + */ + 204: void; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + get: { + req: GetConnectionData; + res: { + /** + * Successful Response + */ + 200: ConnectionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; + "/public/connections/": { + get: { + req: GetConnectionsData; + res: { + /** + * Successful Response + */ + 200: ConnectionCollectionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; + "/public/dags/{dag_id}/dagRuns/{dag_run_id}": { + get: { + req: GetDagRunData; + res: { + /** + * Successful Response + */ + 200: DAGRunResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + delete: { + req: DeleteDagRunData; + res: { + /** + * Successful Response + */ + 204: void; + /** + * Bad Request + */ + 400: HTTPExceptionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + patch: { + req: PatchDagRunStateData; + res: { + /** + * Successful Response + */ + 200: DAGRunResponse; + /** + * Bad Request + */ + 400: HTTPExceptionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; + "/public/dagSources/{file_token}": { + get: { + req: GetDagSourceData; + res: { + /** + * Successful Response + */ + 200: DAGSourceResponse; + /** + * Bad Request + */ + 400: HTTPExceptionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Not Acceptable + */ + 406: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; "/public/eventLogs/{event_log_id}": { get: { req: GetEventLogData; @@ -1347,6 +1617,29 @@ export type $OpenApiTs = { }; }; }; + "/public/dagWarnings": { + get: { + req: ListDagWarningsData; + res: { + /** + * Successful Response + */ + 200: DAGWarningCollectionResponse; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; "/public/plugins/": { get: { req: GetPluginsData; diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index 616f6fbfa29037..7526c340b29f86 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -6,50 +6,6 @@ import type { CamelCasedPropertiesDeep } from "type-fest"; */ export interface paths { - "/backfills": { - get: operations["list_backfills"]; - post: operations["create_backfill"]; - }; - "/backfills/{backfill_id}": { - get: operations["get_backfill"]; - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - }; - "/backfills/{backfill_id}/pause": { - post: operations["pause_backfill"]; - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - }; - "/backfills/{backfill_id}/unpause": { - post: operations["unpause_backfill"]; - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - }; - "/backfills/{backfill_id}/cancel": { - /** - * When a backfill is cancelled, all queued dag runs will be marked as failed. - * Running dag runs will be allowed to continue. - */ - post: operations["cancel_backfill"]; - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - }; "/connections": { get: operations["get_connections"]; post: operations["post_connection"]; @@ -2787,125 +2743,6 @@ export interface components { } export interface operations { - list_backfills: { - parameters: { - query: { - /** List backfills for this dag. */ - dag_id: string; - }; - }; - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["BackfillCollection"]; - }; - }; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - }; - }; - create_backfill: { - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - 400: components["responses"]["BadRequest"]; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - }; - requestBody: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - }; - get_backfill: { - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - 404: components["responses"]["NotFound"]; - }; - }; - pause_backfill: { - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - 404: components["responses"]["NotFound"]; - 409: components["responses"]["Conflict"]; - }; - }; - unpause_backfill: { - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - 404: components["responses"]["NotFound"]; - 409: components["responses"]["Conflict"]; - }; - }; - /** - * When a backfill is cancelled, all queued dag runs will be marked as failed. - * Running dag runs will be allowed to continue. - */ - cancel_backfill: { - parameters: { - path: { - /** The integer id identifying the backfill entity. */ - backfill_id: components["parameters"]["BackfillIdPath"]; - }; - }; - responses: { - /** Success. */ - 200: { - content: { - "application/json": components["schemas"]["Backfill"]; - }; - }; - 401: components["responses"]["Unauthenticated"]; - 403: components["responses"]["PermissionDenied"]; - 404: components["responses"]["NotFound"]; - 409: components["responses"]["Conflict"]; - }; - }; get_connections: { parameters: { query: { @@ -5642,24 +5479,6 @@ export type HealthStatus = CamelCasedPropertiesDeep< export type Operations = operations; /* Types for operation variables */ -export type ListBackfillsVariables = CamelCasedPropertiesDeep< - operations["list_backfills"]["parameters"]["query"] ->; -export type CreateBackfillVariables = CamelCasedPropertiesDeep< - operations["create_backfill"]["requestBody"]["content"]["application/json"] ->; -export type GetBackfillVariables = CamelCasedPropertiesDeep< - operations["get_backfill"]["parameters"]["path"] ->; -export type PauseBackfillVariables = CamelCasedPropertiesDeep< - operations["pause_backfill"]["parameters"]["path"] ->; -export type UnpauseBackfillVariables = CamelCasedPropertiesDeep< - operations["unpause_backfill"]["parameters"]["path"] ->; -export type CancelBackfillVariables = CamelCasedPropertiesDeep< - operations["cancel_backfill"]["parameters"]["path"] ->; export type GetConnectionsVariables = CamelCasedPropertiesDeep< operations["get_connections"]["parameters"]["query"] >; diff --git a/chart/dockerfiles/pgbouncer/Dockerfile b/chart/dockerfiles/pgbouncer/Dockerfile index f561fadc3ebb1d..fcc45da47ba929 100644 --- a/chart/dockerfiles/pgbouncer/Dockerfile +++ b/chart/dockerfiles/pgbouncer/Dockerfile @@ -14,7 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -ARG ALPINE_VERSION="3.14" +ARG ALPINE_VERSION="3.19" FROM alpine:${ALPINE_VERSION} AS builder SHELL ["/bin/ash", "-e", "-x", "-c", "-o", "pipefail"] @@ -26,7 +26,7 @@ ARG PGBOUNCER_SHA256 # Those are build deps only but still we want the latest versions of those # "Pin versions in apk add" https://github.com/hadolint/hadolint/wiki/DL3018 # hadolint ignore=DL3018 -RUN apk --no-cache add make pkgconfig build-base libtool wget gcc g++ libevent-dev libressl-dev c-ares-dev ca-certificates +RUN apk --no-cache add make pkgconfig build-base libtool wget gcc g++ libevent-dev openssl-dev c-ares-dev ca-certificates # We are not using Dash so we can safely ignore the "Dash warning" # "In dash, something is not supported." https://github.com/koalaman/shellcheck/wiki/SC2169 # hadolint ignore=SC2169,SC3060 diff --git a/chart/dockerfiles/pgbouncer/build_and_push.sh b/chart/dockerfiles/pgbouncer/build_and_push.sh index 24ab57882dafca..cede5ab8b0ab3c 100755 --- a/chart/dockerfiles/pgbouncer/build_and_push.sh +++ b/chart/dockerfiles/pgbouncer/build_and_push.sh @@ -22,13 +22,13 @@ readonly DOCKERHUB_USER DOCKERHUB_REPO=${DOCKERHUB_REPO:="airflow"} readonly DOCKERHUB_REPO -PGBOUNCER_VERSION="1.21.0" +PGBOUNCER_VERSION="1.22.1" readonly PGBOUNCER_VERSION -PGBOUNCER_SHA256="7e1dd620c8d85a8490aff25061d5055d7aef9cf3e8bfe2d9e7719b8ee59114e2" +PGBOUNCER_SHA256="2b018aa6ce7f592c9892bb9e0fd90262484eb73937fd2af929770a45373ba215" readonly PGBOUNCER_SHA256 -AIRFLOW_PGBOUNCER_VERSION="2024.01.19" +AIRFLOW_PGBOUNCER_VERSION="2024.09.19" readonly AIRFLOW_PGBOUNCER_VERSION COMMIT_SHA=$(git rev-parse HEAD) diff --git a/chart/values.schema.json b/chart/values.schema.json index ea673a40c2d6ac..b59bb6dbff10f0 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -895,7 +895,7 @@ "tag": { "description": "The PgBouncer image tag.", "type": "string", - "default": "airflow-pgbouncer-2024.01.19-1.21.0" + "default": "airflow-pgbouncer-2024.09.19-1.22.1" }, "pullPolicy": { "description": "The PgBouncer image pull policy.", diff --git a/chart/values.yaml b/chart/values.yaml index 6c3b2aeb9bd1fc..11a8dc035f14bc 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -115,7 +115,7 @@ images: pullPolicy: IfNotPresent pgbouncer: repository: apache/airflow - tag: airflow-pgbouncer-2024.01.19-1.21.0 + tag: airflow-pgbouncer-2024.09.19-1.22.1 pullPolicy: IfNotPresent pgbouncerExporter: repository: apache/airflow diff --git a/dev/breeze/README.md b/dev/breeze/README.md index efc502668953cf..9a3f06066eb221 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -22,6 +22,7 @@ **Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* - [Apache Airflow Breeze](#apache-airflow-breeze) +- [Setting up development env for Breeze](#setting-up-development-env-for-breeze) @@ -62,6 +63,75 @@ available when the package was installed. Since this file becomes part of the in to detect automatically if any of the files have changed. If they did, the user will be warned to upgrade their installations. +Setting up development env for Breeze +------------------------------------- + +> [!NOTE] +> This section is for developers of Breeze. If you are a user of Breeze, you do not need to read this section. + +Breeze is actively developed by Airflow maintainers and contributors, Airflow is an active project +and we are in the process of developing Airflow 3, so breeze requires a lot of adjustments to keep up +the dev environment in sync with Airflow 3 development - this is also why it is part of the same +repository as Airflow - because it needs to be closely synchronized with Airflow development. + +As of November 2024 Airflow switches to using `uv` as the main development environment for Airflow +and for Breeze. So the instructions below are for setting up the development environment for Breeze +using `uv`. However we are using only standard python packaging tools, so you can still use `pip` or +`pipenv` or other build frontends to install Breeze, but we recommend using `uv` as it is the most +convenient way to install, manage python packages and virtual environments. + +Unlike in Airflow, where we manage our own constraints, we use `uv` to manage requirements for Breeze +and we use `uv` to lock the dependencies. This way we can ensure that the dependencies are always +up-to-date and that the development environment is always consistent for different people. This is +why Breeze's `uv.lock` is committed to the repository and is used to install the dependencies by +default by Breeze. Here's how to install breeze with `uv` + + +1. Install `uv` - see [uv documentation](https://docs.astral.sh/uv/getting-started/installation/) + +> [!IMPORTANT] +> All the commands below should be executed while you are in `dev/breeze` directory of the Airflow repository. + +2. Create a new virtual environment for Breeze development: + +```shell +uv venv +``` + +3. Synchronize Breeze dependencies with `uv` to the latest dependencies stored in uv.lock file: + +```shell +uv sync +``` + +After syncing, the `.venv` directory will contain the virtual environment with all the dependencies +installed - you can use that environment to develop Breeze - for example with your favourite IDE +or text editor, you can also use `uv run` to run the scripts in the virtual environment. + +For example to run all tests in the virtual environment you can use: + +```shell +uv run pytest +``` + +4. Add/remove dependencies with `uv`: + +```shell +uv add +uv remove +``` + +5. Update and lock the dependencies (after adding them or periodically to keep them up-to-date): + +```shell +uv lock +``` + +Note that when you update dependencies/lock them you should commit the changes in `pyproject.toml` and `uv.lock`. + +See [uv documentation](https://docs.astral.sh/uv/getting-started/) for more details on using `uv`. + + PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PRE-COMMIT. --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/uv.lock b/dev/breeze/uv.lock new file mode 100644 index 00000000000000..bcfc8c4ce785f1 --- /dev/null +++ b/dev/breeze/uv.lock @@ -0,0 +1,1902 @@ +version = 1 +requires-python = ">=3.9, <4" +resolution-markers = [ + "python_full_version < '3.13'", + "python_full_version >= '3.13'", +] + +[[package]] +name = "anyio" +version = "4.6.2.post1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "exceptiongroup", marker = "python_full_version < '3.11'" }, + { name = "idna" }, + { name = "sniffio" }, + { name = "typing-extensions", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/9f/09/45b9b7a6d4e45c6bcb5bf61d19e3ab87df68e0601fa8c5293de3542546cc/anyio-4.6.2.post1.tar.gz", hash = "sha256:4c8bc31ccdb51c7f7bd251f51c609e038d63e34219b44aa86e47576389880b4c", size = 173422 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e4/f5/f2b75d2fc6f1a260f340f0e7c6a060f4dd2961cc16884ed851b0d18da06a/anyio-4.6.2.post1-py3-none-any.whl", hash = "sha256:6d170c36fba3bdd840c73d3868c1e777e33676a69c3a72cf0a0d5d6d8009b61d", size = 90377 }, +] + +[[package]] +name = "apache-airflow-breeze" +version = "0.0.1" +source = { editable = "." } +dependencies = [ + { name = "black" }, + { name = "click" }, + { name = "filelock" }, + { name = "flit" }, + { name = "gitpython" }, + { name = "google-api-python-client" }, + { name = "google-auth-httplib2" }, + { name = "google-auth-oauthlib" }, + { name = "hatch" }, + { name = "inputimeout" }, + { name = "jinja2" }, + { name = "jsonschema" }, + { name = "packaging" }, + { name = "pipx" }, + { name = "pre-commit" }, + { name = "pre-commit-uv" }, + { name = "psutil" }, + { name = "pygithub" }, + { name = "pytest" }, + { name = "pytest-xdist" }, + { name = "pyyaml" }, + { name = "requests" }, + { name = "rich" }, + { name = "rich-click" }, + { name = "semver" }, + { name = "tabulate" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, + { name = "twine" }, +] + +[package.metadata] +requires-dist = [ + { name = "black", specifier = ">=23.11.0" }, + { name = "click", specifier = ">=8.1.7" }, + { name = "filelock", specifier = ">=3.13.0" }, + { name = "flit", specifier = ">=3.5.0" }, + { name = "gitpython", specifier = ">=3.1.40" }, + { name = "google-api-python-client", specifier = ">=2.142.0" }, + { name = "google-auth-httplib2", specifier = ">=0.2.0" }, + { name = "google-auth-oauthlib", specifier = ">=1.2.0" }, + { name = "hatch", specifier = "==1.9.4" }, + { name = "importlib-resources", marker = "python_full_version < '3.9'", specifier = ">=5.2,!=6.2.0,!=6.3.0,!=6.3.1" }, + { name = "inputimeout", specifier = ">=1.0.4" }, + { name = "jinja2", specifier = ">=3.1.0" }, + { name = "jsonschema", specifier = ">=4.19.1" }, + { name = "packaging", specifier = ">=23.2" }, + { name = "pipx", specifier = ">=1.4.1" }, + { name = "pre-commit", specifier = ">=3.5.0" }, + { name = "pre-commit-uv", specifier = ">=4.1.3" }, + { name = "psutil", specifier = ">=5.9.6" }, + { name = "pygithub", specifier = ">=2.1.1" }, + { name = "pytest", specifier = ">=8.2,<9" }, + { name = "pytest-xdist", specifier = ">=3.3.1" }, + { name = "pyyaml", specifier = ">=6.0.1" }, + { name = "requests", specifier = ">=2.31.0" }, + { name = "rich", specifier = ">=13.6.0" }, + { name = "rich-click", specifier = ">=1.7.1" }, + { name = "semver", specifier = ">=3.0.2" }, + { name = "tabulate", specifier = ">=0.9.0" }, + { name = "tomli", marker = "python_full_version < '3.11'", specifier = ">=2.0.1" }, + { name = "twine", specifier = ">=4.0.2" }, +] + +[[package]] +name = "argcomplete" +version = "3.5.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/5f/39/27605e133e7f4bb0c8e48c9a6b87101515e3446003e0442761f6a02ac35e/argcomplete-3.5.1.tar.gz", hash = "sha256:eb1ee355aa2557bd3d0145de7b06b2a45b0ce461e1e7813f5d066039ab4177b4", size = 82280 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f7/be/a606a6701d491cfae75583c80a6583f8abe9c36c0b9666e867e7cdd62fe8/argcomplete-3.5.1-py3-none-any.whl", hash = "sha256:1a1d148bdaa3e3b93454900163403df41448a248af01b6e849edc5ac08e6c363", size = 43498 }, +] + +[[package]] +name = "attrs" +version = "24.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/fc/0f/aafca9af9315aee06a89ffde799a10a582fe8de76c563ee80bbcdc08b3fb/attrs-24.2.0.tar.gz", hash = "sha256:5cfb1b9148b5b086569baec03f20d7b6bf3bcacc9a42bebf87ffaaca362f6346", size = 792678 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6a/21/5b6702a7f963e95456c0de2d495f67bf5fd62840ac655dc451586d23d39a/attrs-24.2.0-py3-none-any.whl", hash = "sha256:81921eb96de3191c8258c199618104dd27ac608d9366f5e35d011eae1867ede2", size = 63001 }, +] + +[[package]] +name = "backports-tarfile" +version = "1.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/86/72/cd9b395f25e290e633655a100af28cb253e4393396264a98bd5f5951d50f/backports_tarfile-1.2.0.tar.gz", hash = "sha256:d75e02c268746e1b8144c278978b6e98e85de6ad16f8e4b0844a154557eca991", size = 86406 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b9/fa/123043af240e49752f1c4bd24da5053b6bd00cad78c2be53c0d1e8b975bc/backports.tarfile-1.2.0-py3-none-any.whl", hash = "sha256:77e284d754527b01fb1e6fa8a1afe577858ebe4e9dad8919e34c862cb399bc34", size = 30181 }, +] + +[[package]] +name = "black" +version = "24.10.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "click" }, + { name = "mypy-extensions" }, + { name = "packaging" }, + { name = "pathspec" }, + { name = "platformdirs" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, + { name = "typing-extensions", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/d8/0d/cc2fb42b8c50d80143221515dd7e4766995bd07c56c9a3ed30baf080b6dc/black-24.10.0.tar.gz", hash = "sha256:846ea64c97afe3bc677b761787993be4991810ecc7a4a937816dd6bddedc4875", size = 645813 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a3/f3/465c0eb5cddf7dbbfe1fecd9b875d1dcf51b88923cd2c1d7e9ab95c6336b/black-24.10.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e6668650ea4b685440857138e5fe40cde4d652633b1bdffc62933d0db4ed9812", size = 1623211 }, + { url = "https://files.pythonhosted.org/packages/df/57/b6d2da7d200773fdfcc224ffb87052cf283cec4d7102fab450b4a05996d8/black-24.10.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1c536fcf674217e87b8cc3657b81809d3c085d7bf3ef262ead700da345bfa6ea", size = 1457139 }, + { url = "https://files.pythonhosted.org/packages/6e/c5/9023b7673904a5188f9be81f5e129fff69f51f5515655fbd1d5a4e80a47b/black-24.10.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:649fff99a20bd06c6f727d2a27f401331dc0cc861fb69cde910fe95b01b5928f", size = 1753774 }, + { url = "https://files.pythonhosted.org/packages/e1/32/df7f18bd0e724e0d9748829765455d6643ec847b3f87e77456fc99d0edab/black-24.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:fe4d6476887de70546212c99ac9bd803d90b42fc4767f058a0baa895013fbb3e", size = 1414209 }, + { url = "https://files.pythonhosted.org/packages/c2/cc/7496bb63a9b06a954d3d0ac9fe7a73f3bf1cd92d7a58877c27f4ad1e9d41/black-24.10.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5a2221696a8224e335c28816a9d331a6c2ae15a2ee34ec857dcf3e45dbfa99ad", size = 1607468 }, + { url = "https://files.pythonhosted.org/packages/2b/e3/69a738fb5ba18b5422f50b4f143544c664d7da40f09c13969b2fd52900e0/black-24.10.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f9da3333530dbcecc1be13e69c250ed8dfa67f43c4005fb537bb426e19200d50", size = 1437270 }, + { url = "https://files.pythonhosted.org/packages/c9/9b/2db8045b45844665c720dcfe292fdaf2e49825810c0103e1191515fc101a/black-24.10.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:4007b1393d902b48b36958a216c20c4482f601569d19ed1df294a496eb366392", size = 1737061 }, + { url = "https://files.pythonhosted.org/packages/a3/95/17d4a09a5be5f8c65aa4a361444d95edc45def0de887810f508d3f65db7a/black-24.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:394d4ddc64782e51153eadcaaca95144ac4c35e27ef9b0a42e121ae7e57a9175", size = 1423293 }, + { url = "https://files.pythonhosted.org/packages/90/04/bf74c71f592bcd761610bbf67e23e6a3cff824780761f536512437f1e655/black-24.10.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:b5e39e0fae001df40f95bd8cc36b9165c5e2ea88900167bddf258bacef9bbdc3", size = 1644256 }, + { url = "https://files.pythonhosted.org/packages/4c/ea/a77bab4cf1887f4b2e0bce5516ea0b3ff7d04ba96af21d65024629afedb6/black-24.10.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:d37d422772111794b26757c5b55a3eade028aa3fde43121ab7b673d050949d65", size = 1448534 }, + { url = "https://files.pythonhosted.org/packages/4e/3e/443ef8bc1fbda78e61f79157f303893f3fddf19ca3c8989b163eb3469a12/black-24.10.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:14b3502784f09ce2443830e3133dacf2c0110d45191ed470ecb04d0f5f6fcb0f", size = 1761892 }, + { url = "https://files.pythonhosted.org/packages/52/93/eac95ff229049a6901bc84fec6908a5124b8a0b7c26ea766b3b8a5debd22/black-24.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:30d2c30dc5139211dda799758559d1b049f7f14c580c409d6ad925b74a4208a8", size = 1434796 }, + { url = "https://files.pythonhosted.org/packages/d0/a0/a993f58d4ecfba035e61fca4e9f64a2ecae838fc9f33ab798c62173ed75c/black-24.10.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:1cbacacb19e922a1d75ef2b6ccaefcd6e93a2c05ede32f06a21386a04cedb981", size = 1643986 }, + { url = "https://files.pythonhosted.org/packages/37/d5/602d0ef5dfcace3fb4f79c436762f130abd9ee8d950fa2abdbf8bbc555e0/black-24.10.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:1f93102e0c5bb3907451063e08b9876dbeac810e7da5a8bfb7aeb5a9ef89066b", size = 1448085 }, + { url = "https://files.pythonhosted.org/packages/47/6d/a3a239e938960df1a662b93d6230d4f3e9b4a22982d060fc38c42f45a56b/black-24.10.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:ddacb691cdcdf77b96f549cf9591701d8db36b2f19519373d60d31746068dbf2", size = 1760928 }, + { url = "https://files.pythonhosted.org/packages/dd/cf/af018e13b0eddfb434df4d9cd1b2b7892bab119f7a20123e93f6910982e8/black-24.10.0-cp313-cp313-win_amd64.whl", hash = "sha256:680359d932801c76d2e9c9068d05c6b107f2584b2a5b88831c83962eb9984c1b", size = 1436875 }, + { url = "https://files.pythonhosted.org/packages/fe/02/f408c804e0ee78c367dcea0a01aedde4f1712af93b8b6e60df981e0228c7/black-24.10.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:17374989640fbca88b6a448129cd1745c5eb8d9547b464f281b251dd00155ccd", size = 1622516 }, + { url = "https://files.pythonhosted.org/packages/f8/b9/9b706ed2f55bfb28b436225a9c57da35990c9005b90b8c91f03924454ad7/black-24.10.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:63f626344343083322233f175aaf372d326de8436f5928c042639a4afbbf1d3f", size = 1456181 }, + { url = "https://files.pythonhosted.org/packages/0a/1c/314d7f17434a5375682ad097f6f4cc0e3f414f3c95a9b1bb4df14a0f11f9/black-24.10.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:ccfa1d0cb6200857f1923b602f978386a3a2758a65b52e0950299ea014be6800", size = 1752801 }, + { url = "https://files.pythonhosted.org/packages/39/a7/20e5cd9237d28ad0b31438de5d9f01c8b99814576f4c0cda1edd62caf4b0/black-24.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:2cd9c95431d94adc56600710f8813ee27eea544dd118d45896bb734e9d7a0dc7", size = 1413626 }, + { url = "https://files.pythonhosted.org/packages/8d/a7/4b27c50537ebca8bec139b872861f9d2bf501c5ec51fcf897cb924d9e264/black-24.10.0-py3-none-any.whl", hash = "sha256:3bb2b7a1f7b685f85b11fed1ef10f8a9148bceb49853e47a294a3dd963c1dd7d", size = 206898 }, +] + +[[package]] +name = "cachetools" +version = "5.5.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c3/38/a0f315319737ecf45b4319a8cd1f3a908e29d9277b46942263292115eee7/cachetools-5.5.0.tar.gz", hash = "sha256:2cc24fb4cbe39633fb7badd9db9ca6295d766d9c2995f245725a46715d050f2a", size = 27661 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a4/07/14f8ad37f2d12a5ce41206c21820d8cb6561b728e51fad4530dff0552a67/cachetools-5.5.0-py3-none-any.whl", hash = "sha256:02134e8439cdc2ffb62023ce1debca2944c3f289d66bb17ead3ab3dede74b292", size = 9524 }, +] + +[[package]] +name = "certifi" +version = "2024.8.30" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b0/ee/9b19140fe824b367c04c5e1b369942dd754c4c5462d5674002f75c4dedc1/certifi-2024.8.30.tar.gz", hash = "sha256:bec941d2aa8195e248a60b31ff9f0558284cf01a52591ceda73ea9afffd69fd9", size = 168507 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/12/90/3c9ff0512038035f59d279fddeb79f5f1eccd8859f06d6163c58798b9487/certifi-2024.8.30-py3-none-any.whl", hash = "sha256:922820b53db7a7257ffbda3f597266d435245903d80737e34f8a45ff3e3230d8", size = 167321 }, +] + +[[package]] +name = "cffi" +version = "1.17.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pycparser" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/90/07/f44ca684db4e4f08a3fdc6eeb9a0d15dc6883efc7b8c90357fdbf74e186c/cffi-1.17.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:df8b1c11f177bc2313ec4b2d46baec87a5f3e71fc8b45dab2ee7cae86d9aba14", size = 182191 }, + { url = "https://files.pythonhosted.org/packages/08/fd/cc2fedbd887223f9f5d170c96e57cbf655df9831a6546c1727ae13fa977a/cffi-1.17.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8f2cdc858323644ab277e9bb925ad72ae0e67f69e804f4898c070998d50b1a67", size = 178592 }, + { url = "https://files.pythonhosted.org/packages/de/cc/4635c320081c78d6ffc2cab0a76025b691a91204f4aa317d568ff9280a2d/cffi-1.17.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:edae79245293e15384b51f88b00613ba9f7198016a5948b5dddf4917d4d26382", size = 426024 }, + { url = "https://files.pythonhosted.org/packages/b6/7b/3b2b250f3aab91abe5f8a51ada1b717935fdaec53f790ad4100fe2ec64d1/cffi-1.17.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45398b671ac6d70e67da8e4224a065cec6a93541bb7aebe1b198a61b58c7b702", size = 448188 }, + { url = "https://files.pythonhosted.org/packages/d3/48/1b9283ebbf0ec065148d8de05d647a986c5f22586b18120020452fff8f5d/cffi-1.17.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad9413ccdeda48c5afdae7e4fa2192157e991ff761e7ab8fdd8926f40b160cc3", size = 455571 }, + { url = "https://files.pythonhosted.org/packages/40/87/3b8452525437b40f39ca7ff70276679772ee7e8b394934ff60e63b7b090c/cffi-1.17.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5da5719280082ac6bd9aa7becb3938dc9f9cbd57fac7d2871717b1feb0902ab6", size = 436687 }, + { url = "https://files.pythonhosted.org/packages/8d/fb/4da72871d177d63649ac449aec2e8a29efe0274035880c7af59101ca2232/cffi-1.17.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bb1a08b8008b281856e5971307cc386a8e9c5b625ac297e853d36da6efe9c17", size = 446211 }, + { url = "https://files.pythonhosted.org/packages/ab/a0/62f00bcb411332106c02b663b26f3545a9ef136f80d5df746c05878f8c4b/cffi-1.17.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:045d61c734659cc045141be4bae381a41d89b741f795af1dd018bfb532fd0df8", size = 461325 }, + { url = "https://files.pythonhosted.org/packages/36/83/76127035ed2e7e27b0787604d99da630ac3123bfb02d8e80c633f218a11d/cffi-1.17.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6883e737d7d9e4899a8a695e00ec36bd4e5e4f18fabe0aca0efe0a4b44cdb13e", size = 438784 }, + { url = "https://files.pythonhosted.org/packages/21/81/a6cd025db2f08ac88b901b745c163d884641909641f9b826e8cb87645942/cffi-1.17.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:6b8b4a92e1c65048ff98cfe1f735ef8f1ceb72e3d5f0c25fdb12087a23da22be", size = 461564 }, + { url = "https://files.pythonhosted.org/packages/f8/fe/4d41c2f200c4a457933dbd98d3cf4e911870877bd94d9656cc0fcb390681/cffi-1.17.1-cp310-cp310-win32.whl", hash = "sha256:c9c3d058ebabb74db66e431095118094d06abf53284d9c81f27300d0e0d8bc7c", size = 171804 }, + { url = "https://files.pythonhosted.org/packages/d1/b6/0b0f5ab93b0df4acc49cae758c81fe4e5ef26c3ae2e10cc69249dfd8b3ab/cffi-1.17.1-cp310-cp310-win_amd64.whl", hash = "sha256:0f048dcf80db46f0098ccac01132761580d28e28bc0f78ae0d58048063317e15", size = 181299 }, + { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264 }, + { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651 }, + { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259 }, + { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200 }, + { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235 }, + { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721 }, + { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242 }, + { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999 }, + { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242 }, + { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604 }, + { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727 }, + { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400 }, + { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178 }, + { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840 }, + { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803 }, + { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850 }, + { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729 }, + { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256 }, + { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424 }, + { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568 }, + { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736 }, + { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448 }, + { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976 }, + { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989 }, + { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802 }, + { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792 }, + { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893 }, + { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810 }, + { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200 }, + { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447 }, + { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358 }, + { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469 }, + { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475 }, + { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009 }, + { url = "https://files.pythonhosted.org/packages/b9/ea/8bb50596b8ffbc49ddd7a1ad305035daa770202a6b782fc164647c2673ad/cffi-1.17.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b2ab587605f4ba0bf81dc0cb08a41bd1c0a5906bd59243d56bad7668a6fc6c16", size = 182220 }, + { url = "https://files.pythonhosted.org/packages/ae/11/e77c8cd24f58285a82c23af484cf5b124a376b32644e445960d1a4654c3a/cffi-1.17.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:28b16024becceed8c6dfbc75629e27788d8a3f9030691a1dbf9821a128b22c36", size = 178605 }, + { url = "https://files.pythonhosted.org/packages/ed/65/25a8dc32c53bf5b7b6c2686b42ae2ad58743f7ff644844af7cdb29b49361/cffi-1.17.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1d599671f396c4723d016dbddb72fe8e0397082b0a77a4fab8028923bec050e8", size = 424910 }, + { url = "https://files.pythonhosted.org/packages/42/7a/9d086fab7c66bd7c4d0f27c57a1b6b068ced810afc498cc8c49e0088661c/cffi-1.17.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca74b8dbe6e8e8263c0ffd60277de77dcee6c837a3d0881d8c1ead7268c9e576", size = 447200 }, + { url = "https://files.pythonhosted.org/packages/da/63/1785ced118ce92a993b0ec9e0d0ac8dc3e5dbfbcaa81135be56c69cabbb6/cffi-1.17.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f7f5baafcc48261359e14bcd6d9bff6d4b28d9103847c9e136694cb0501aef87", size = 454565 }, + { url = "https://files.pythonhosted.org/packages/74/06/90b8a44abf3556599cdec107f7290277ae8901a58f75e6fe8f970cd72418/cffi-1.17.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:98e3969bcff97cae1b2def8ba499ea3d6f31ddfdb7635374834cf89a1a08ecf0", size = 435635 }, + { url = "https://files.pythonhosted.org/packages/bd/62/a1f468e5708a70b1d86ead5bab5520861d9c7eacce4a885ded9faa7729c3/cffi-1.17.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cdf5ce3acdfd1661132f2a9c19cac174758dc2352bfe37d98aa7512c6b7178b3", size = 445218 }, + { url = "https://files.pythonhosted.org/packages/5b/95/b34462f3ccb09c2594aa782d90a90b045de4ff1f70148ee79c69d37a0a5a/cffi-1.17.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:9755e4345d1ec879e3849e62222a18c7174d65a6a92d5b346b1863912168b595", size = 460486 }, + { url = "https://files.pythonhosted.org/packages/fc/fc/a1e4bebd8d680febd29cf6c8a40067182b64f00c7d105f8f26b5bc54317b/cffi-1.17.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:f1e22e8c4419538cb197e4dd60acc919d7696e5ef98ee4da4e01d3f8cfa4cc5a", size = 437911 }, + { url = "https://files.pythonhosted.org/packages/e6/c3/21cab7a6154b6a5ea330ae80de386e7665254835b9e98ecc1340b3a7de9a/cffi-1.17.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:c03e868a0b3bc35839ba98e74211ed2b05d2119be4e8a0f224fba9384f1fe02e", size = 460632 }, + { url = "https://files.pythonhosted.org/packages/cb/b5/fd9f8b5a84010ca169ee49f4e4ad6f8c05f4e3545b72ee041dbbcb159882/cffi-1.17.1-cp39-cp39-win32.whl", hash = "sha256:e31ae45bc2e29f6b2abd0de1cc3b9d5205aa847cafaecb8af1476a609a2f6eb7", size = 171820 }, + { url = "https://files.pythonhosted.org/packages/8c/52/b08750ce0bce45c143e1b5d7357ee8c55341b52bdef4b0f081af1eb248c2/cffi-1.17.1-cp39-cp39-win_amd64.whl", hash = "sha256:d016c76bdd850f3c626af19b0542c9677ba156e4ee4fccfdd7848803533ef662", size = 181290 }, +] + +[[package]] +name = "cfgv" +version = "3.4.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/11/74/539e56497d9bd1d484fd863dd69cbbfa653cd2aa27abfe35653494d85e94/cfgv-3.4.0.tar.gz", hash = "sha256:e52591d4c5f5dead8e0f673fb16db7949d2cfb3f7da4582893288f0ded8fe560", size = 7114 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c5/55/51844dd50c4fc7a33b653bfaba4c2456f06955289ca770a5dbd5fd267374/cfgv-3.4.0-py2.py3-none-any.whl", hash = "sha256:b7265b1f29fd3316bfcd2b330d63d024f2bfd8bcb8b0272f8e19a504856c48f9", size = 7249 }, +] + +[[package]] +name = "charset-normalizer" +version = "3.4.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f2/4f/e1808dc01273379acc506d18f1504eb2d299bd4131743b9fc54d7be4df1e/charset_normalizer-3.4.0.tar.gz", hash = "sha256:223217c3d4f82c3ac5e29032b3f1c2eb0fb591b72161f86d93f5719079dae93e", size = 106620 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/69/8b/825cc84cf13a28bfbcba7c416ec22bf85a9584971be15b21dd8300c65b7f/charset_normalizer-3.4.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:4f9fc98dad6c2eaa32fc3af1417d95b5e3d08aff968df0cd320066def971f9a6", size = 196363 }, + { url = "https://files.pythonhosted.org/packages/23/81/d7eef6a99e42c77f444fdd7bc894b0ceca6c3a95c51239e74a722039521c/charset_normalizer-3.4.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0de7b687289d3c1b3e8660d0741874abe7888100efe14bd0f9fd7141bcbda92b", size = 125639 }, + { url = "https://files.pythonhosted.org/packages/21/67/b4564d81f48042f520c948abac7079356e94b30cb8ffb22e747532cf469d/charset_normalizer-3.4.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:5ed2e36c3e9b4f21dd9422f6893dec0abf2cca553af509b10cd630f878d3eb99", size = 120451 }, + { url = "https://files.pythonhosted.org/packages/c2/72/12a7f0943dd71fb5b4e7b55c41327ac0a1663046a868ee4d0d8e9c369b85/charset_normalizer-3.4.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:40d3ff7fc90b98c637bda91c89d51264a3dcf210cade3a2c6f838c7268d7a4ca", size = 140041 }, + { url = "https://files.pythonhosted.org/packages/67/56/fa28c2c3e31217c4c52158537a2cf5d98a6c1e89d31faf476c89391cd16b/charset_normalizer-3.4.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1110e22af8ca26b90bd6364fe4c763329b0ebf1ee213ba32b68c73de5752323d", size = 150333 }, + { url = "https://files.pythonhosted.org/packages/f9/d2/466a9be1f32d89eb1554cf84073a5ed9262047acee1ab39cbaefc19635d2/charset_normalizer-3.4.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:86f4e8cca779080f66ff4f191a685ced73d2f72d50216f7112185dc02b90b9b7", size = 142921 }, + { url = "https://files.pythonhosted.org/packages/f8/01/344ec40cf5d85c1da3c1f57566c59e0c9b56bcc5566c08804a95a6cc8257/charset_normalizer-3.4.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f683ddc7eedd742e2889d2bfb96d69573fde1d92fcb811979cdb7165bb9c7d3", size = 144785 }, + { url = "https://files.pythonhosted.org/packages/73/8b/2102692cb6d7e9f03b9a33a710e0164cadfce312872e3efc7cfe22ed26b4/charset_normalizer-3.4.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:27623ba66c183eca01bf9ff833875b459cad267aeeb044477fedac35e19ba907", size = 146631 }, + { url = "https://files.pythonhosted.org/packages/d8/96/cc2c1b5d994119ce9f088a9a0c3ebd489d360a2eb058e2c8049f27092847/charset_normalizer-3.4.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:f606a1881d2663630ea5b8ce2efe2111740df4b687bd78b34a8131baa007f79b", size = 140867 }, + { url = "https://files.pythonhosted.org/packages/c9/27/cde291783715b8ec30a61c810d0120411844bc4c23b50189b81188b273db/charset_normalizer-3.4.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:0b309d1747110feb25d7ed6b01afdec269c647d382c857ef4663bbe6ad95a912", size = 149273 }, + { url = "https://files.pythonhosted.org/packages/3a/a4/8633b0fc1a2d1834d5393dafecce4a1cc56727bfd82b4dc18fc92f0d3cc3/charset_normalizer-3.4.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:136815f06a3ae311fae551c3df1f998a1ebd01ddd424aa5603a4336997629e95", size = 152437 }, + { url = "https://files.pythonhosted.org/packages/64/ea/69af161062166b5975ccbb0961fd2384853190c70786f288684490913bf5/charset_normalizer-3.4.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:14215b71a762336254351b00ec720a8e85cada43b987da5a042e4ce3e82bd68e", size = 150087 }, + { url = "https://files.pythonhosted.org/packages/3b/fd/e60a9d9fd967f4ad5a92810138192f825d77b4fa2a557990fd575a47695b/charset_normalizer-3.4.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:79983512b108e4a164b9c8d34de3992f76d48cadc9554c9e60b43f308988aabe", size = 145142 }, + { url = "https://files.pythonhosted.org/packages/6d/02/8cb0988a1e49ac9ce2eed1e07b77ff118f2923e9ebd0ede41ba85f2dcb04/charset_normalizer-3.4.0-cp310-cp310-win32.whl", hash = "sha256:c94057af19bc953643a33581844649a7fdab902624d2eb739738a30e2b3e60fc", size = 94701 }, + { url = "https://files.pythonhosted.org/packages/d6/20/f1d4670a8a723c46be695dff449d86d6092916f9e99c53051954ee33a1bc/charset_normalizer-3.4.0-cp310-cp310-win_amd64.whl", hash = "sha256:55f56e2ebd4e3bc50442fbc0888c9d8c94e4e06a933804e2af3e89e2f9c1c749", size = 102191 }, + { url = "https://files.pythonhosted.org/packages/9c/61/73589dcc7a719582bf56aae309b6103d2762b526bffe189d635a7fcfd998/charset_normalizer-3.4.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:0d99dd8ff461990f12d6e42c7347fd9ab2532fb70e9621ba520f9e8637161d7c", size = 193339 }, + { url = "https://files.pythonhosted.org/packages/77/d5/8c982d58144de49f59571f940e329ad6e8615e1e82ef84584c5eeb5e1d72/charset_normalizer-3.4.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c57516e58fd17d03ebe67e181a4e4e2ccab1168f8c2976c6a334d4f819fe5944", size = 124366 }, + { url = "https://files.pythonhosted.org/packages/bf/19/411a64f01ee971bed3231111b69eb56f9331a769072de479eae7de52296d/charset_normalizer-3.4.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:6dba5d19c4dfab08e58d5b36304b3f92f3bd5d42c1a3fa37b5ba5cdf6dfcbcee", size = 118874 }, + { url = "https://files.pythonhosted.org/packages/4c/92/97509850f0d00e9f14a46bc751daabd0ad7765cff29cdfb66c68b6dad57f/charset_normalizer-3.4.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bf4475b82be41b07cc5e5ff94810e6a01f276e37c2d55571e3fe175e467a1a1c", size = 138243 }, + { url = "https://files.pythonhosted.org/packages/e2/29/d227805bff72ed6d6cb1ce08eec707f7cfbd9868044893617eb331f16295/charset_normalizer-3.4.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ce031db0408e487fd2775d745ce30a7cd2923667cf3b69d48d219f1d8f5ddeb6", size = 148676 }, + { url = "https://files.pythonhosted.org/packages/13/bc/87c2c9f2c144bedfa62f894c3007cd4530ba4b5351acb10dc786428a50f0/charset_normalizer-3.4.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8ff4e7cdfdb1ab5698e675ca622e72d58a6fa2a8aa58195de0c0061288e6e3ea", size = 141289 }, + { url = "https://files.pythonhosted.org/packages/eb/5b/6f10bad0f6461fa272bfbbdf5d0023b5fb9bc6217c92bf068fa5a99820f5/charset_normalizer-3.4.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3710a9751938947e6327ea9f3ea6332a09bf0ba0c09cae9cb1f250bd1f1549bc", size = 142585 }, + { url = "https://files.pythonhosted.org/packages/3b/a0/a68980ab8a1f45a36d9745d35049c1af57d27255eff8c907e3add84cf68f/charset_normalizer-3.4.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:82357d85de703176b5587dbe6ade8ff67f9f69a41c0733cf2425378b49954de5", size = 144408 }, + { url = "https://files.pythonhosted.org/packages/d7/a1/493919799446464ed0299c8eef3c3fad0daf1c3cd48bff9263c731b0d9e2/charset_normalizer-3.4.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:47334db71978b23ebcf3c0f9f5ee98b8d65992b65c9c4f2d34c2eaf5bcaf0594", size = 139076 }, + { url = "https://files.pythonhosted.org/packages/fb/9d/9c13753a5a6e0db4a0a6edb1cef7aee39859177b64e1a1e748a6e3ba62c2/charset_normalizer-3.4.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:8ce7fd6767a1cc5a92a639b391891bf1c268b03ec7e021c7d6d902285259685c", size = 146874 }, + { url = "https://files.pythonhosted.org/packages/75/d2/0ab54463d3410709c09266dfb416d032a08f97fd7d60e94b8c6ef54ae14b/charset_normalizer-3.4.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:f1a2f519ae173b5b6a2c9d5fa3116ce16e48b3462c8b96dfdded11055e3d6365", size = 150871 }, + { url = "https://files.pythonhosted.org/packages/8d/c9/27e41d481557be53d51e60750b85aa40eaf52b841946b3cdeff363105737/charset_normalizer-3.4.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:63bc5c4ae26e4bc6be6469943b8253c0fd4e4186c43ad46e713ea61a0ba49129", size = 148546 }, + { url = "https://files.pythonhosted.org/packages/ee/44/4f62042ca8cdc0cabf87c0fc00ae27cd8b53ab68be3605ba6d071f742ad3/charset_normalizer-3.4.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:bcb4f8ea87d03bc51ad04add8ceaf9b0f085ac045ab4d74e73bbc2dc033f0236", size = 143048 }, + { url = "https://files.pythonhosted.org/packages/01/f8/38842422988b795220eb8038745d27a675ce066e2ada79516c118f291f07/charset_normalizer-3.4.0-cp311-cp311-win32.whl", hash = "sha256:9ae4ef0b3f6b41bad6366fb0ea4fc1d7ed051528e113a60fa2a65a9abb5b1d99", size = 94389 }, + { url = "https://files.pythonhosted.org/packages/0b/6e/b13bd47fa9023b3699e94abf565b5a2f0b0be6e9ddac9812182596ee62e4/charset_normalizer-3.4.0-cp311-cp311-win_amd64.whl", hash = "sha256:cee4373f4d3ad28f1ab6290684d8e2ebdb9e7a1b74fdc39e4c211995f77bec27", size = 101752 }, + { url = "https://files.pythonhosted.org/packages/d3/0b/4b7a70987abf9b8196845806198975b6aab4ce016632f817ad758a5aa056/charset_normalizer-3.4.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:0713f3adb9d03d49d365b70b84775d0a0d18e4ab08d12bc46baa6132ba78aaf6", size = 194445 }, + { url = "https://files.pythonhosted.org/packages/50/89/354cc56cf4dd2449715bc9a0f54f3aef3dc700d2d62d1fa5bbea53b13426/charset_normalizer-3.4.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:de7376c29d95d6719048c194a9cf1a1b0393fbe8488a22008610b0361d834ecf", size = 125275 }, + { url = "https://files.pythonhosted.org/packages/fa/44/b730e2a2580110ced837ac083d8ad222343c96bb6b66e9e4e706e4d0b6df/charset_normalizer-3.4.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:4a51b48f42d9358460b78725283f04bddaf44a9358197b889657deba38f329db", size = 119020 }, + { url = "https://files.pythonhosted.org/packages/9d/e4/9263b8240ed9472a2ae7ddc3e516e71ef46617fe40eaa51221ccd4ad9a27/charset_normalizer-3.4.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b295729485b06c1a0683af02a9e42d2caa9db04a373dc38a6a58cdd1e8abddf1", size = 139128 }, + { url = "https://files.pythonhosted.org/packages/6b/e3/9f73e779315a54334240353eaea75854a9a690f3f580e4bd85d977cb2204/charset_normalizer-3.4.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ee803480535c44e7f5ad00788526da7d85525cfefaf8acf8ab9a310000be4b03", size = 149277 }, + { url = "https://files.pythonhosted.org/packages/1a/cf/f1f50c2f295312edb8a548d3fa56a5c923b146cd3f24114d5adb7e7be558/charset_normalizer-3.4.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3d59d125ffbd6d552765510e3f31ed75ebac2c7470c7274195b9161a32350284", size = 142174 }, + { url = "https://files.pythonhosted.org/packages/16/92/92a76dc2ff3a12e69ba94e7e05168d37d0345fa08c87e1fe24d0c2a42223/charset_normalizer-3.4.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8cda06946eac330cbe6598f77bb54e690b4ca93f593dee1568ad22b04f347c15", size = 143838 }, + { url = "https://files.pythonhosted.org/packages/a4/01/2117ff2b1dfc61695daf2babe4a874bca328489afa85952440b59819e9d7/charset_normalizer-3.4.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:07afec21bbbbf8a5cc3651aa96b980afe2526e7f048fdfb7f1014d84acc8b6d8", size = 146149 }, + { url = "https://files.pythonhosted.org/packages/f6/9b/93a332b8d25b347f6839ca0a61b7f0287b0930216994e8bf67a75d050255/charset_normalizer-3.4.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6b40e8d38afe634559e398cc32b1472f376a4099c75fe6299ae607e404c033b2", size = 140043 }, + { url = "https://files.pythonhosted.org/packages/ab/f6/7ac4a01adcdecbc7a7587767c776d53d369b8b971382b91211489535acf0/charset_normalizer-3.4.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:b8dcd239c743aa2f9c22ce674a145e0a25cb1566c495928440a181ca1ccf6719", size = 148229 }, + { url = "https://files.pythonhosted.org/packages/9d/be/5708ad18161dee7dc6a0f7e6cf3a88ea6279c3e8484844c0590e50e803ef/charset_normalizer-3.4.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:84450ba661fb96e9fd67629b93d2941c871ca86fc38d835d19d4225ff946a631", size = 151556 }, + { url = "https://files.pythonhosted.org/packages/5a/bb/3d8bc22bacb9eb89785e83e6723f9888265f3a0de3b9ce724d66bd49884e/charset_normalizer-3.4.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:44aeb140295a2f0659e113b31cfe92c9061622cadbc9e2a2f7b8ef6b1e29ef4b", size = 149772 }, + { url = "https://files.pythonhosted.org/packages/f7/fa/d3fc622de05a86f30beea5fc4e9ac46aead4731e73fd9055496732bcc0a4/charset_normalizer-3.4.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:1db4e7fefefd0f548d73e2e2e041f9df5c59e178b4c72fbac4cc6f535cfb1565", size = 144800 }, + { url = "https://files.pythonhosted.org/packages/9a/65/bdb9bc496d7d190d725e96816e20e2ae3a6fa42a5cac99c3c3d6ff884118/charset_normalizer-3.4.0-cp312-cp312-win32.whl", hash = "sha256:5726cf76c982532c1863fb64d8c6dd0e4c90b6ece9feb06c9f202417a31f7dd7", size = 94836 }, + { url = "https://files.pythonhosted.org/packages/3e/67/7b72b69d25b89c0b3cea583ee372c43aa24df15f0e0f8d3982c57804984b/charset_normalizer-3.4.0-cp312-cp312-win_amd64.whl", hash = "sha256:b197e7094f232959f8f20541ead1d9862ac5ebea1d58e9849c1bf979255dfac9", size = 102187 }, + { url = "https://files.pythonhosted.org/packages/f3/89/68a4c86f1a0002810a27f12e9a7b22feb198c59b2f05231349fbce5c06f4/charset_normalizer-3.4.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:dd4eda173a9fcccb5f2e2bd2a9f423d180194b1bf17cf59e3269899235b2a114", size = 194617 }, + { url = "https://files.pythonhosted.org/packages/4f/cd/8947fe425e2ab0aa57aceb7807af13a0e4162cd21eee42ef5b053447edf5/charset_normalizer-3.4.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:e9e3c4c9e1ed40ea53acf11e2a386383c3304212c965773704e4603d589343ed", size = 125310 }, + { url = "https://files.pythonhosted.org/packages/5b/f0/b5263e8668a4ee9becc2b451ed909e9c27058337fda5b8c49588183c267a/charset_normalizer-3.4.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:92a7e36b000bf022ef3dbb9c46bfe2d52c047d5e3f3343f43204263c5addc250", size = 119126 }, + { url = "https://files.pythonhosted.org/packages/ff/6e/e445afe4f7fda27a533f3234b627b3e515a1b9429bc981c9a5e2aa5d97b6/charset_normalizer-3.4.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:54b6a92d009cbe2fb11054ba694bc9e284dad30a26757b1e372a1fdddaf21920", size = 139342 }, + { url = "https://files.pythonhosted.org/packages/a1/b2/4af9993b532d93270538ad4926c8e37dc29f2111c36f9c629840c57cd9b3/charset_normalizer-3.4.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1ffd9493de4c922f2a38c2bf62b831dcec90ac673ed1ca182fe11b4d8e9f2a64", size = 149383 }, + { url = "https://files.pythonhosted.org/packages/fb/6f/4e78c3b97686b871db9be6f31d64e9264e889f8c9d7ab33c771f847f79b7/charset_normalizer-3.4.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:35c404d74c2926d0287fbd63ed5d27eb911eb9e4a3bb2c6d294f3cfd4a9e0c23", size = 142214 }, + { url = "https://files.pythonhosted.org/packages/2b/c9/1c8fe3ce05d30c87eff498592c89015b19fade13df42850aafae09e94f35/charset_normalizer-3.4.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4796efc4faf6b53a18e3d46343535caed491776a22af773f366534056c4e1fbc", size = 144104 }, + { url = "https://files.pythonhosted.org/packages/ee/68/efad5dcb306bf37db7db338338e7bb8ebd8cf38ee5bbd5ceaaaa46f257e6/charset_normalizer-3.4.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e7fdd52961feb4c96507aa649550ec2a0d527c086d284749b2f582f2d40a2e0d", size = 146255 }, + { url = "https://files.pythonhosted.org/packages/0c/75/1ed813c3ffd200b1f3e71121c95da3f79e6d2a96120163443b3ad1057505/charset_normalizer-3.4.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:92db3c28b5b2a273346bebb24857fda45601aef6ae1c011c0a997106581e8a88", size = 140251 }, + { url = "https://files.pythonhosted.org/packages/7d/0d/6f32255c1979653b448d3c709583557a4d24ff97ac4f3a5be156b2e6a210/charset_normalizer-3.4.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:ab973df98fc99ab39080bfb0eb3a925181454d7c3ac8a1e695fddfae696d9e90", size = 148474 }, + { url = "https://files.pythonhosted.org/packages/ac/a0/c1b5298de4670d997101fef95b97ac440e8c8d8b4efa5a4d1ef44af82f0d/charset_normalizer-3.4.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:4b67fdab07fdd3c10bb21edab3cbfe8cf5696f453afce75d815d9d7223fbe88b", size = 151849 }, + { url = "https://files.pythonhosted.org/packages/04/4f/b3961ba0c664989ba63e30595a3ed0875d6790ff26671e2aae2fdc28a399/charset_normalizer-3.4.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:aa41e526a5d4a9dfcfbab0716c7e8a1b215abd3f3df5a45cf18a12721d31cb5d", size = 149781 }, + { url = "https://files.pythonhosted.org/packages/d8/90/6af4cd042066a4adad58ae25648a12c09c879efa4849c705719ba1b23d8c/charset_normalizer-3.4.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:ffc519621dce0c767e96b9c53f09c5d215578e10b02c285809f76509a3931482", size = 144970 }, + { url = "https://files.pythonhosted.org/packages/cc/67/e5e7e0cbfefc4ca79025238b43cdf8a2037854195b37d6417f3d0895c4c2/charset_normalizer-3.4.0-cp313-cp313-win32.whl", hash = "sha256:f19c1585933c82098c2a520f8ec1227f20e339e33aca8fa6f956f6691b784e67", size = 94973 }, + { url = "https://files.pythonhosted.org/packages/65/97/fc9bbc54ee13d33dc54a7fcf17b26368b18505500fc01e228c27b5222d80/charset_normalizer-3.4.0-cp313-cp313-win_amd64.whl", hash = "sha256:707b82d19e65c9bd28b81dde95249b07bf9f5b90ebe1ef17d9b57473f8a64b7b", size = 102308 }, + { url = "https://files.pythonhosted.org/packages/54/2f/28659eee7f5d003e0f5a3b572765bf76d6e0fe6601ab1f1b1dd4cba7e4f1/charset_normalizer-3.4.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:980b4f289d1d90ca5efcf07958d3eb38ed9c0b7676bf2831a54d4f66f9c27dfa", size = 196326 }, + { url = "https://files.pythonhosted.org/packages/d1/18/92869d5c0057baa973a3ee2af71573be7b084b3c3d428fe6463ce71167f8/charset_normalizer-3.4.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:f28f891ccd15c514a0981f3b9db9aa23d62fe1a99997512b0491d2ed323d229a", size = 125614 }, + { url = "https://files.pythonhosted.org/packages/d6/27/327904c5a54a7796bb9f36810ec4173d2df5d88b401d2b95ef53111d214e/charset_normalizer-3.4.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a8aacce6e2e1edcb6ac625fb0f8c3a9570ccc7bfba1f63419b3769ccf6a00ed0", size = 120450 }, + { url = "https://files.pythonhosted.org/packages/a4/23/65af317914a0308495133b2d654cf67b11bbd6ca16637c4e8a38f80a5a69/charset_normalizer-3.4.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bd7af3717683bea4c87acd8c0d3d5b44d56120b26fd3f8a692bdd2d5260c620a", size = 140135 }, + { url = "https://files.pythonhosted.org/packages/f2/41/6190102ad521a8aa888519bb014a74251ac4586cde9b38e790901684f9ab/charset_normalizer-3.4.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5ff2ed8194587faf56555927b3aa10e6fb69d931e33953943bc4f837dfee2242", size = 150413 }, + { url = "https://files.pythonhosted.org/packages/7b/ab/f47b0159a69eab9bd915591106859f49670c75f9a19082505ff16f50efc0/charset_normalizer-3.4.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e91f541a85298cf35433bf66f3fab2a4a2cff05c127eeca4af174f6d497f0d4b", size = 142992 }, + { url = "https://files.pythonhosted.org/packages/28/89/60f51ad71f63aaaa7e51a2a2ad37919985a341a1d267070f212cdf6c2d22/charset_normalizer-3.4.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:309a7de0a0ff3040acaebb35ec45d18db4b28232f21998851cfa709eeff49d62", size = 144871 }, + { url = "https://files.pythonhosted.org/packages/0c/48/0050550275fea585a6e24460b42465020b53375017d8596c96be57bfabca/charset_normalizer-3.4.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:285e96d9d53422efc0d7a17c60e59f37fbf3dfa942073f666db4ac71e8d726d0", size = 146756 }, + { url = "https://files.pythonhosted.org/packages/dc/b5/47f8ee91455946f745e6c9ddbb0f8f50314d2416dd922b213e7d5551ad09/charset_normalizer-3.4.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:5d447056e2ca60382d460a604b6302d8db69476fd2015c81e7c35417cfabe4cd", size = 141034 }, + { url = "https://files.pythonhosted.org/packages/84/79/5c731059ebab43e80bf61fa51666b9b18167974b82004f18c76378ed31a3/charset_normalizer-3.4.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:20587d20f557fe189b7947d8e7ec5afa110ccf72a3128d61a2a387c3313f46be", size = 149434 }, + { url = "https://files.pythonhosted.org/packages/ca/f3/0719cd09fc4dc42066f239cb3c48ced17fc3316afca3e2a30a4756fe49ab/charset_normalizer-3.4.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:130272c698667a982a5d0e626851ceff662565379baf0ff2cc58067b81d4f11d", size = 152443 }, + { url = "https://files.pythonhosted.org/packages/f7/0e/c6357297f1157c8e8227ff337e93fd0a90e498e3d6ab96b2782204ecae48/charset_normalizer-3.4.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:ab22fbd9765e6954bc0bcff24c25ff71dcbfdb185fcdaca49e81bac68fe724d3", size = 150294 }, + { url = "https://files.pythonhosted.org/packages/54/9a/acfa96dc4ea8c928040b15822b59d0863d6e1757fba8bd7de3dc4f761c13/charset_normalizer-3.4.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:7782afc9b6b42200f7362858f9e73b1f8316afb276d316336c0ec3bd73312742", size = 145314 }, + { url = "https://files.pythonhosted.org/packages/73/1c/b10a63032eaebb8d7bcb8544f12f063f41f5f463778ac61da15d9985e8b6/charset_normalizer-3.4.0-cp39-cp39-win32.whl", hash = "sha256:2de62e8801ddfff069cd5c504ce3bc9672b23266597d4e4f50eda28846c322f2", size = 94724 }, + { url = "https://files.pythonhosted.org/packages/c5/77/3a78bf28bfaa0863f9cfef278dbeadf55efe064eafff8c7c424ae3c4c1bf/charset_normalizer-3.4.0-cp39-cp39-win_amd64.whl", hash = "sha256:95c3c157765b031331dd4db3c775e58deaee050a3042fcad72cbc4189d7c8dca", size = 102159 }, + { url = "https://files.pythonhosted.org/packages/bf/9b/08c0432272d77b04803958a4598a51e2a4b51c06640af8b8f0f908c18bf2/charset_normalizer-3.4.0-py3-none-any.whl", hash = "sha256:fe9f97feb71aa9896b81973a7bbada8c49501dc73e58a10fcef6663af95e5079", size = 49446 }, +] + +[[package]] +name = "click" +version = "8.1.7" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "platform_system == 'Windows'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/96/d3/f04c7bfcf5c1862a2a5b845c6b2b360488cf47af55dfa79c98f6a6bf98b5/click-8.1.7.tar.gz", hash = "sha256:ca9853ad459e787e2192211578cc907e7594e294c7ccc834310722b41b9ca6de", size = 336121 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/00/2e/d53fa4befbf2cfa713304affc7ca780ce4fc1fd8710527771b58311a3229/click-8.1.7-py3-none-any.whl", hash = "sha256:ae74fb96c20a0277a1d615f1e4d73c8414f5a98db8b799a7931d1582f3390c28", size = 97941 }, +] + +[[package]] +name = "colorama" +version = "0.4.6" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d8/53/6f443c9a4a8358a93a6792e2acffb9d9d5cb0a5cfd8802644b7b1c9a02e4/colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44", size = 27697 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/d6/3965ed04c63042e047cb6a3e6ed1a63a35087b6a609aa3a15ed8ac56c221/colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6", size = 25335 }, +] + +[[package]] +name = "cryptography" +version = "43.0.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cffi", marker = "platform_python_implementation != 'PyPy'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/0d/05/07b55d1fa21ac18c3a8c79f764e2514e6f6a9698f1be44994f5adf0d29db/cryptography-43.0.3.tar.gz", hash = "sha256:315b9001266a492a6ff443b61238f956b214dbec9910a081ba5b6646a055a805", size = 686989 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1f/f3/01fdf26701a26f4b4dbc337a26883ad5bccaa6f1bbbdd29cd89e22f18a1c/cryptography-43.0.3-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:bf7a1932ac4176486eab36a19ed4c0492da5d97123f1406cf15e41b05e787d2e", size = 6225303 }, + { url = "https://files.pythonhosted.org/packages/a3/01/4896f3d1b392025d4fcbecf40fdea92d3df8662123f6835d0af828d148fd/cryptography-43.0.3-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63efa177ff54aec6e1c0aefaa1a241232dcd37413835a9b674b6e3f0ae2bfd3e", size = 3760905 }, + { url = "https://files.pythonhosted.org/packages/0a/be/f9a1f673f0ed4b7f6c643164e513dbad28dd4f2dcdf5715004f172ef24b6/cryptography-43.0.3-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7e1ce50266f4f70bf41a2c6dc4358afadae90e2a1e5342d3c08883df1675374f", size = 3977271 }, + { url = "https://files.pythonhosted.org/packages/4e/49/80c3a7b5514d1b416d7350830e8c422a4d667b6d9b16a9392ebfd4a5388a/cryptography-43.0.3-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:443c4a81bb10daed9a8f334365fe52542771f25aedaf889fd323a853ce7377d6", size = 3746606 }, + { url = "https://files.pythonhosted.org/packages/0e/16/a28ddf78ac6e7e3f25ebcef69ab15c2c6be5ff9743dd0709a69a4f968472/cryptography-43.0.3-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:74f57f24754fe349223792466a709f8e0c093205ff0dca557af51072ff47ab18", size = 3986484 }, + { url = "https://files.pythonhosted.org/packages/01/f5/69ae8da70c19864a32b0315049866c4d411cce423ec169993d0434218762/cryptography-43.0.3-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:9762ea51a8fc2a88b70cf2995e5675b38d93bf36bd67d91721c309df184f49bd", size = 3852131 }, + { url = "https://files.pythonhosted.org/packages/fd/db/e74911d95c040f9afd3612b1f732e52b3e517cb80de8bf183be0b7d413c6/cryptography-43.0.3-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:81ef806b1fef6b06dcebad789f988d3b37ccaee225695cf3e07648eee0fc6b73", size = 4075647 }, + { url = "https://files.pythonhosted.org/packages/56/48/7b6b190f1462818b324e674fa20d1d5ef3e24f2328675b9b16189cbf0b3c/cryptography-43.0.3-cp37-abi3-win32.whl", hash = "sha256:cbeb489927bd7af4aa98d4b261af9a5bc025bd87f0e3547e11584be9e9427be2", size = 2623873 }, + { url = "https://files.pythonhosted.org/packages/eb/b1/0ebff61a004f7f89e7b65ca95f2f2375679d43d0290672f7713ee3162aff/cryptography-43.0.3-cp37-abi3-win_amd64.whl", hash = "sha256:f46304d6f0c6ab8e52770addfa2fc41e6629495548862279641972b6215451cd", size = 3068039 }, + { url = "https://files.pythonhosted.org/packages/30/d5/c8b32c047e2e81dd172138f772e81d852c51f0f2ad2ae8a24f1122e9e9a7/cryptography-43.0.3-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:8ac43ae87929a5982f5948ceda07001ee5e83227fd69cf55b109144938d96984", size = 6222984 }, + { url = "https://files.pythonhosted.org/packages/2f/78/55356eb9075d0be6e81b59f45c7b48df87f76a20e73893872170471f3ee8/cryptography-43.0.3-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:846da004a5804145a5f441b8530b4bf35afbf7da70f82409f151695b127213d5", size = 3762968 }, + { url = "https://files.pythonhosted.org/packages/2a/2c/488776a3dc843f95f86d2f957ca0fc3407d0242b50bede7fad1e339be03f/cryptography-43.0.3-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0f996e7268af62598f2fc1204afa98a3b5712313a55c4c9d434aef49cadc91d4", size = 3977754 }, + { url = "https://files.pythonhosted.org/packages/7c/04/2345ca92f7a22f601a9c62961741ef7dd0127c39f7310dffa0041c80f16f/cryptography-43.0.3-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:f7b178f11ed3664fd0e995a47ed2b5ff0a12d893e41dd0494f406d1cf555cab7", size = 3749458 }, + { url = "https://files.pythonhosted.org/packages/ac/25/e715fa0bc24ac2114ed69da33adf451a38abb6f3f24ec207908112e9ba53/cryptography-43.0.3-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:c2e6fc39c4ab499049df3bdf567f768a723a5e8464816e8f009f121a5a9f4405", size = 3988220 }, + { url = "https://files.pythonhosted.org/packages/21/ce/b9c9ff56c7164d8e2edfb6c9305045fbc0df4508ccfdb13ee66eb8c95b0e/cryptography-43.0.3-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:e1be4655c7ef6e1bbe6b5d0403526601323420bcf414598955968c9ef3eb7d16", size = 3853898 }, + { url = "https://files.pythonhosted.org/packages/2a/33/b3682992ab2e9476b9c81fff22f02c8b0a1e6e1d49ee1750a67d85fd7ed2/cryptography-43.0.3-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:df6b6c6d742395dd77a23ea3728ab62f98379eff8fb61be2744d4679ab678f73", size = 4076592 }, + { url = "https://files.pythonhosted.org/packages/81/1e/ffcc41b3cebd64ca90b28fd58141c5f68c83d48563c88333ab660e002cd3/cryptography-43.0.3-cp39-abi3-win32.whl", hash = "sha256:d56e96520b1020449bbace2b78b603442e7e378a9b3bd68de65c782db1507995", size = 2623145 }, + { url = "https://files.pythonhosted.org/packages/87/5c/3dab83cc4aba1f4b0e733e3f0c3e7d4386440d660ba5b1e3ff995feb734d/cryptography-43.0.3-cp39-abi3-win_amd64.whl", hash = "sha256:0c580952eef9bf68c4747774cde7ec1d85a6e61de97281f2dba83c7d2c806362", size = 3068026 }, + { url = "https://files.pythonhosted.org/packages/6f/db/d8b8a039483f25fc3b70c90bc8f3e1d4497a99358d610c5067bf3bd4f0af/cryptography-43.0.3-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:d03b5621a135bffecad2c73e9f4deb1a0f977b9a8ffe6f8e002bf6c9d07b918c", size = 3144545 }, + { url = "https://files.pythonhosted.org/packages/93/90/116edd5f8ec23b2dc879f7a42443e073cdad22950d3c8ee834e3b8124543/cryptography-43.0.3-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:a2a431ee15799d6db9fe80c82b055bae5a752bef645bba795e8e52687c69efe3", size = 3679828 }, + { url = "https://files.pythonhosted.org/packages/d8/32/1e1d78b316aa22c0ba6493cc271c1c309969e5aa5c22c830a1d7ce3471e6/cryptography-43.0.3-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:281c945d0e28c92ca5e5930664c1cefd85efe80e5c0d2bc58dd63383fda29f83", size = 3908132 }, + { url = "https://files.pythonhosted.org/packages/91/bb/cd2c13be3332e7af3cdf16154147952d39075b9f61ea5e6b5241bf4bf436/cryptography-43.0.3-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:f18c716be16bc1fea8e95def49edf46b82fccaa88587a45f8dc0ff6ab5d8e0a7", size = 2988811 }, + { url = "https://files.pythonhosted.org/packages/cc/fc/ff7c76afdc4f5933b5e99092528d4783d3d1b131960fc8b31eb38e076ca8/cryptography-43.0.3-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:4a02ded6cd4f0a5562a8887df8b3bd14e822a90f97ac5e544c162899bc467664", size = 3146844 }, + { url = "https://files.pythonhosted.org/packages/d7/29/a233efb3e98b13d9175dcb3c3146988ec990896c8fa07e8467cce27d5a80/cryptography-43.0.3-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:53a583b6637ab4c4e3591a15bc9db855b8d9dee9a669b550f311480acab6eb08", size = 3681997 }, + { url = "https://files.pythonhosted.org/packages/c0/cf/c9eea7791b961f279fb6db86c3355cfad29a73141f46427af71852b23b95/cryptography-43.0.3-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:1ec0bcf7e17c0c5669d881b1cd38c4972fade441b27bda1051665faaa89bdcaa", size = 3905208 }, + { url = "https://files.pythonhosted.org/packages/21/ea/6c38ca546d5b6dab3874c2b8fc6b1739baac29bacdea31a8c6c0513b3cfa/cryptography-43.0.3-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:2ce6fae5bdad59577b44e4dfed356944fbf1d925269114c28be377692643b4ff", size = 2989787 }, +] + +[[package]] +name = "deprecated" +version = "1.2.14" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "wrapt" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/92/14/1e41f504a246fc224d2ac264c227975427a85caf37c3979979edb9b1b232/Deprecated-1.2.14.tar.gz", hash = "sha256:e5323eb936458dccc2582dc6f9c322c852a775a27065ff2b0c4970b9d53d01b3", size = 2974416 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/20/8d/778b7d51b981a96554f29136cd59ca7880bf58094338085bcf2a979a0e6a/Deprecated-1.2.14-py2.py3-none-any.whl", hash = "sha256:6fac8b097794a90302bdbb17b9b815e732d3c4720583ff1b198499d78470466c", size = 9561 }, +] + +[[package]] +name = "distlib" +version = "0.3.9" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/0d/dd/1bec4c5ddb504ca60fc29472f3d27e8d4da1257a854e1d96742f15c1d02d/distlib-0.3.9.tar.gz", hash = "sha256:a60f20dea646b8a33f3e7772f74dc0b2d0772d2837ee1342a00645c81edf9403", size = 613923 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/91/a1/cf2472db20f7ce4a6be1253a81cfdf85ad9c7885ffbed7047fb72c24cf87/distlib-0.3.9-py2.py3-none-any.whl", hash = "sha256:47f8c22fd27c27e25a65601af709b38e4f0a45ea4fc2e710f65755fa8caaaf87", size = 468973 }, +] + +[[package]] +name = "docutils" +version = "0.21.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ae/ed/aefcc8cd0ba62a0560c3c18c33925362d46c6075480bfa4df87b28e169a9/docutils-0.21.2.tar.gz", hash = "sha256:3a6b18732edf182daa3cd12775bbb338cf5691468f91eeeb109deff6ebfa986f", size = 2204444 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8f/d7/9322c609343d929e75e7e5e6255e614fcc67572cfd083959cdef3b7aad79/docutils-0.21.2-py3-none-any.whl", hash = "sha256:dafca5b9e384f0e419294eb4d2ff9fa826435bf15f15b7bd45723e8ad76811b2", size = 587408 }, +] + +[[package]] +name = "editables" +version = "0.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/37/4a/986d35164e2033ddfb44515168a281a7986e260d344cf369c3f52d4c3275/editables-0.5.tar.gz", hash = "sha256:309627d9b5c4adc0e668d8c6fa7bac1ba7c8c5d415c2d27f60f081f8e80d1de2", size = 14744 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6b/be/0f2f4a5e8adc114a02b63d92bf8edbfa24db6fc602fca83c885af2479e0e/editables-0.5-py3-none-any.whl", hash = "sha256:61e5ffa82629e0d8bfe09bc44a07db3c1ab8ed1ce78a6980732870f19b5e7d4c", size = 5098 }, +] + +[[package]] +name = "exceptiongroup" +version = "1.2.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/09/35/2495c4ac46b980e4ca1f6ad6db102322ef3ad2410b79fdde159a4b0f3b92/exceptiongroup-1.2.2.tar.gz", hash = "sha256:47c2edf7c6738fafb49fd34290706d1a1a2f4d1c6df275526b62cbb4aa5393cc", size = 28883 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/02/cc/b7e31358aac6ed1ef2bb790a9746ac2c69bcb3c8588b41616914eb106eaf/exceptiongroup-1.2.2-py3-none-any.whl", hash = "sha256:3111b9d131c238bec2f8f516e123e14ba243563fb135d3fe885990585aa7795b", size = 16453 }, +] + +[[package]] +name = "execnet" +version = "2.1.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/bb/ff/b4c0dc78fbe20c3e59c0c7334de0c27eb4001a2b2017999af398bf730817/execnet-2.1.1.tar.gz", hash = "sha256:5189b52c6121c24feae288166ab41b32549c7e2348652736540b9e6e7d4e72e3", size = 166524 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/43/09/2aea36ff60d16dd8879bdb2f5b3ee0ba8d08cbbdcdfe870e695ce3784385/execnet-2.1.1-py3-none-any.whl", hash = "sha256:26dee51f1b80cebd6d0ca8e74dd8745419761d3bef34163928cbebbdc4749fdc", size = 40612 }, +] + +[[package]] +name = "filelock" +version = "3.16.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9d/db/3ef5bb276dae18d6ec2124224403d1d67bccdbefc17af4cc8f553e341ab1/filelock-3.16.1.tar.gz", hash = "sha256:c249fbfcd5db47e5e2d6d62198e565475ee65e4831e2561c8e313fa7eb961435", size = 18037 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b9/f8/feced7779d755758a52d1f6635d990b8d98dc0a29fa568bbe0625f18fdf3/filelock-3.16.1-py3-none-any.whl", hash = "sha256:2082e5703d51fbf98ea75855d9d5527e33d8ff23099bec374a134febee6946b0", size = 16163 }, +] + +[[package]] +name = "flit" +version = "3.10.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "docutils" }, + { name = "flit-core" }, + { name = "pip" }, + { name = "requests" }, + { name = "tomli-w" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/03/46/f84b8815d161e7392d124d3de6e5880d1d36a74162a77a5e2839dc3c8c68/flit-3.10.1.tar.gz", hash = "sha256:9c6258ae76d218ce60f9e39a43ca42006a3abcc5c44ea6bb2a1daa13857a8f1a", size = 143162 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b6/ba/d88b8f3253b4af5a88559aede6345975cc2b18ed77bf8daf977bbb9df2c5/flit-3.10.1-py3-none-any.whl", hash = "sha256:d79c19c2caae73cc486d3d827af6a11c1a84b9efdfab8d9683b714ec8d1dc1f1", size = 50683 }, +] + +[[package]] +name = "flit-core" +version = "3.10.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d5/ae/09427bea9227a33ec834ed5461432752fd5d02b14f93dd68406c91684622/flit_core-3.10.1.tar.gz", hash = "sha256:66e5b87874a0d6e39691f0e22f09306736b633548670ad3c09ec9db03c5662f7", size = 42842 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/be/2d/293fe6a58e73df57cc2b5e5cf2b17c6bb4fb5b0c390bab8f1e87bdc62529/flit_core-3.10.1-py3-none-any.whl", hash = "sha256:cb31a76e8b31ad3351bb89e531f64ef2b05d1e65bd939183250bf81ddf4922a8", size = 36389 }, +] + +[[package]] +name = "gitdb" +version = "4.0.11" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "smmap" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/19/0d/bbb5b5ee188dec84647a4664f3e11b06ade2bde568dbd489d9d64adef8ed/gitdb-4.0.11.tar.gz", hash = "sha256:bf5421126136d6d0af55bc1e7c1af1c397a34f5b7bd79e776cd3e89785c2b04b", size = 394469 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/fd/5b/8f0c4a5bb9fd491c277c21eff7ccae71b47d43c4446c9d0c6cff2fe8c2c4/gitdb-4.0.11-py3-none-any.whl", hash = "sha256:81a3407ddd2ee8df444cbacea00e2d038e40150acfa3001696fe0dcf1d3adfa4", size = 62721 }, +] + +[[package]] +name = "gitpython" +version = "3.1.43" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "gitdb" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b6/a1/106fd9fa2dd989b6fb36e5893961f82992cf676381707253e0bf93eb1662/GitPython-3.1.43.tar.gz", hash = "sha256:35f314a9f878467f5453cc1fee295c3e18e52f1b99f10f6cf5b1682e968a9e7c", size = 214149 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e9/bd/cc3a402a6439c15c3d4294333e13042b915bbeab54edc457c723931fed3f/GitPython-3.1.43-py3-none-any.whl", hash = "sha256:eec7ec56b92aad751f9912a73404bc02ba212a23adb2c7098ee668417051a1ff", size = 207337 }, +] + +[[package]] +name = "google-api-core" +version = "2.22.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "google-auth" }, + { name = "googleapis-common-protos" }, + { name = "proto-plus" }, + { name = "protobuf" }, + { name = "requests" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/00/c2/425f97c2087affbd452a05d3faa08d97de333f2ca554733e1becab55ee4e/google_api_core-2.22.0.tar.gz", hash = "sha256:26f8d76b96477db42b55fd02a33aae4a42ec8b86b98b94969b7333a2c828bf35", size = 159700 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ba/7b/1137a9811be73d8ff8238eb2d9f60f0bc0bb6a1edd87f9d47557ab937a2b/google_api_core-2.22.0-py3-none-any.whl", hash = "sha256:a6652b6bd51303902494998626653671703c420f6f4c88cfd3f50ed723e9d021", size = 156538 }, +] + +[[package]] +name = "google-api-python-client" +version = "2.151.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "google-api-core" }, + { name = "google-auth" }, + { name = "google-auth-httplib2" }, + { name = "httplib2" }, + { name = "uritemplate" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/7c/87/5a753c932a962f1ac72403608b6840500187fd9d856127a360b7a30c59ec/google_api_python_client-2.151.0.tar.gz", hash = "sha256:a9d26d630810ed4631aea21d1de3e42072f98240aaf184a8a1a874a371115034", size = 12030480 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/75/32/675ec68ed1bd27664d74f980cd262504603da0b683c2dd09c8725f576236/google_api_python_client-2.151.0-py2.py3-none-any.whl", hash = "sha256:4427b2f47cd88b0355d540c2c52215f68c337f3bc9d6aae1ceeae4525977504c", size = 12534219 }, +] + +[[package]] +name = "google-auth" +version = "2.35.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cachetools" }, + { name = "pyasn1-modules" }, + { name = "rsa" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a1/37/c854a8b1b1020cf042db3d67577c6f84cd1e8ff6515e4f5498ae9e444ea5/google_auth-2.35.0.tar.gz", hash = "sha256:f4c64ed4e01e8e8b646ef34c018f8bf3338df0c8e37d8b3bba40e7f574a3278a", size = 267223 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/27/1f/3a72917afcb0d5cd842cbccb81bf7a8a7b45b4c66d8dc4556ccb3b016bfc/google_auth-2.35.0-py2.py3-none-any.whl", hash = "sha256:25df55f327ef021de8be50bad0dfd4a916ad0de96da86cd05661c9297723ad3f", size = 208968 }, +] + +[[package]] +name = "google-auth-httplib2" +version = "0.2.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "google-auth" }, + { name = "httplib2" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/56/be/217a598a818567b28e859ff087f347475c807a5649296fb5a817c58dacef/google-auth-httplib2-0.2.0.tar.gz", hash = "sha256:38aa7badf48f974f1eb9861794e9c0cb2a0511a4ec0679b1f886d108f5640e05", size = 10842 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/be/8a/fe34d2f3f9470a27b01c9e76226965863f153d5fbe276f83608562e49c04/google_auth_httplib2-0.2.0-py2.py3-none-any.whl", hash = "sha256:b65a0a2123300dd71281a7bf6e64d65a0759287df52729bdd1ae2e47dc311a3d", size = 9253 }, +] + +[[package]] +name = "google-auth-oauthlib" +version = "1.2.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "google-auth" }, + { name = "requests-oauthlib" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/cc/0f/1772edb8d75ecf6280f1c7f51cbcebe274e8b17878b382f63738fd96cee5/google_auth_oauthlib-1.2.1.tar.gz", hash = "sha256:afd0cad092a2eaa53cd8e8298557d6de1034c6cb4a740500b5357b648af97263", size = 24970 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1a/8e/22a28dfbd218033e4eeaf3a0533b2b54852b6530da0c0fe934f0cc494b29/google_auth_oauthlib-1.2.1-py2.py3-none-any.whl", hash = "sha256:2d58a27262d55aa1b87678c3ba7142a080098cbc2024f903c62355deb235d91f", size = 24930 }, +] + +[[package]] +name = "googleapis-common-protos" +version = "1.65.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "protobuf" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/53/3b/1599ceafa875ffb951480c8c74f4b77646a6b80e80970698f2aa93c216ce/googleapis_common_protos-1.65.0.tar.gz", hash = "sha256:334a29d07cddc3aa01dee4988f9afd9b2916ee2ff49d6b757155dc0d197852c0", size = 113657 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ec/08/49bfe7cf737952cc1a9c43e80cc258ed45dad7f183c5b8276fc94cb3862d/googleapis_common_protos-1.65.0-py2.py3-none-any.whl", hash = "sha256:2972e6c496f435b92590fd54045060867f3fe9be2c82ab148fc8885035479a63", size = 220890 }, +] + +[[package]] +name = "h11" +version = "0.14.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f5/38/3af3d3633a34a3316095b39c8e8fb4853a28a536e55d347bd8d8e9a14b03/h11-0.14.0.tar.gz", hash = "sha256:8f19fbbe99e72420ff35c00b27a34cb9937e902a8b810e2c88300c6f0a3b699d", size = 100418 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/95/04/ff642e65ad6b90db43e668d70ffb6736436c7ce41fcc549f4e9472234127/h11-0.14.0-py3-none-any.whl", hash = "sha256:e3fe4ac4b851c468cc8363d500db52c2ead036020723024a109d37346efaa761", size = 58259 }, +] + +[[package]] +name = "hatch" +version = "1.9.4" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "click" }, + { name = "hatchling" }, + { name = "httpx" }, + { name = "hyperlink" }, + { name = "keyring" }, + { name = "packaging" }, + { name = "pexpect" }, + { name = "platformdirs" }, + { name = "rich" }, + { name = "shellingham" }, + { name = "tomli-w" }, + { name = "tomlkit" }, + { name = "userpath" }, + { name = "virtualenv" }, + { name = "zstandard" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/17/98/63bf6c592b65f67201db292489053b86310cfb107eb095d345398e00cbd3/hatch-1.9.4.tar.gz", hash = "sha256:9bb7d1c4a7a51cc1f9e16394875c940b45fa84b698f0291529316b27d74e7f32", size = 689598 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/05/38/ba8f90264d19ed39851f37a22f2a4be8e9644a1203f114b16647f954bb02/hatch-1.9.4-py3-none-any.whl", hash = "sha256:461eb86b4b46249e38a9a621c7239e61285fd8e14b5a1b5a727c394893a25300", size = 110812 }, +] + +[[package]] +name = "hatchling" +version = "1.21.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "editables" }, + { name = "packaging" }, + { name = "pathspec" }, + { name = "pluggy" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, + { name = "trove-classifiers" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/d8/a1/7dd1caa87c0b15c04c6291e25112e5d082cce02ee87f221a8be1d594f857/hatchling-1.21.1.tar.gz", hash = "sha256:bba440453a224e7d4478457fa2e8d8c3633765bafa02975a6b53b9bf917980bc", size = 58059 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3a/bb/40528a09a33845bd7fd75c33b3be7faec3b5c8f15f68a58931da67420fb9/hatchling-1.21.1-py3-none-any.whl", hash = "sha256:21e8c13f8458b219a91cb84e5b61c15bf786695d1c4fabc29e91e78f94bfe892", size = 76740 }, +] + +[[package]] +name = "httpcore" +version = "1.0.6" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "certifi" }, + { name = "h11" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b6/44/ed0fa6a17845fb033bd885c03e842f08c1b9406c86a2e60ac1ae1b9206a6/httpcore-1.0.6.tar.gz", hash = "sha256:73f6dbd6eb8c21bbf7ef8efad555481853f5f6acdeaff1edb0694289269ee17f", size = 85180 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/06/89/b161908e2f51be56568184aeb4a880fd287178d176fd1c860d2217f41106/httpcore-1.0.6-py3-none-any.whl", hash = "sha256:27b59625743b85577a8c0e10e55b50b5368a4f2cfe8cc7bcfa9cf00829c2682f", size = 78011 }, +] + +[[package]] +name = "httplib2" +version = "0.22.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pyparsing" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3d/ad/2371116b22d616c194aa25ec410c9c6c37f23599dcd590502b74db197584/httplib2-0.22.0.tar.gz", hash = "sha256:d7a10bc5ef5ab08322488bde8c726eeee5c8618723fdb399597ec58f3d82df81", size = 351116 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a8/6c/d2fbdaaa5959339d53ba38e94c123e4e84b8fbc4b84beb0e70d7c1608486/httplib2-0.22.0-py3-none-any.whl", hash = "sha256:14ae0a53c1ba8f3d37e9e27cf37eabb0fb9980f435ba405d546948b009dd64dc", size = 96854 }, +] + +[[package]] +name = "httpx" +version = "0.27.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "anyio" }, + { name = "certifi" }, + { name = "httpcore" }, + { name = "idna" }, + { name = "sniffio" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/78/82/08f8c936781f67d9e6b9eeb8a0c8b4e406136ea4c3d1f89a5db71d42e0e6/httpx-0.27.2.tar.gz", hash = "sha256:f7c2be1d2f3c3c3160d441802406b206c2b76f5947b11115e6df10c6c65e66c2", size = 144189 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/56/95/9377bcb415797e44274b51d46e3249eba641711cf3348050f76ee7b15ffc/httpx-0.27.2-py3-none-any.whl", hash = "sha256:7bb2708e112d8fdd7829cd4243970f0c223274051cb35ee80c03301ee29a3df0", size = 76395 }, +] + +[[package]] +name = "hyperlink" +version = "21.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "idna" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3a/51/1947bd81d75af87e3bb9e34593a4cf118115a8feb451ce7a69044ef1412e/hyperlink-21.0.0.tar.gz", hash = "sha256:427af957daa58bc909471c6c40f74c5450fa123dd093fc53efd2e91d2705a56b", size = 140743 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6e/aa/8caf6a0a3e62863cbb9dab27135660acba46903b703e224f14f447e57934/hyperlink-21.0.0-py2.py3-none-any.whl", hash = "sha256:e6b14c37ecb73e89c77d78cdb4c2cc8f3fb59a885c5b3f819ff4ed80f25af1b4", size = 74638 }, +] + +[[package]] +name = "identify" +version = "2.6.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/29/bb/25024dbcc93516c492b75919e76f389bac754a3e4248682fba32b250c880/identify-2.6.1.tar.gz", hash = "sha256:91478c5fb7c3aac5ff7bf9b4344f803843dc586832d5f110d672b19aa1984c98", size = 99097 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/7d/0c/4ef72754c050979fdcc06c744715ae70ea37e734816bb6514f79df77a42f/identify-2.6.1-py2.py3-none-any.whl", hash = "sha256:53863bcac7caf8d2ed85bd20312ea5dcfc22226800f6d6881f232d861db5a8f0", size = 98972 }, +] + +[[package]] +name = "idna" +version = "3.10" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f1/70/7703c29685631f5a7590aa73f1f1d3fa9a380e654b86af429e0934a32f7d/idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9", size = 190490 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/76/c6/c88e154df9c4e1a2a66ccf0005a88dfb2650c1dffb6f5ce603dfbd452ce3/idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3", size = 70442 }, +] + +[[package]] +name = "importlib-metadata" +version = "8.5.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "zipp" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/cd/12/33e59336dca5be0c398a7482335911a33aa0e20776128f038019f1a95f1b/importlib_metadata-8.5.0.tar.gz", hash = "sha256:71522656f0abace1d072b9e5481a48f07c138e00f079c38c8f883823f9c26bd7", size = 55304 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a0/d9/a1e041c5e7caa9a05c925f4bdbdfb7f006d1f74996af53467bc394c97be7/importlib_metadata-8.5.0-py3-none-any.whl", hash = "sha256:45e54197d28b7a7f1559e60b95e7c567032b602131fbd588f1497f47880aa68b", size = 26514 }, +] + +[[package]] +name = "iniconfig" +version = "2.0.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d7/4b/cbd8e699e64a6f16ca3a8220661b5f83792b3017d0f79807cb8708d33913/iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3", size = 4646 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ef/a6/62565a6e1cf69e10f5727360368e451d4b7f58beeac6173dc9db836a5b46/iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374", size = 5892 }, +] + +[[package]] +name = "inputimeout" +version = "1.0.4" +source = { registry = "https://pypi.org/simple" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/97/9c/1646ca469bc2dc299ac393c8d31136c6c22a35ca1e373fa462ac01100d37/inputimeout-1.0.4-py3-none-any.whl", hash = "sha256:f4e23d27753cfc25268eefc8d52a3edc46280ad831d226617c51882423475a43", size = 4639 }, +] + +[[package]] +name = "jaraco-classes" +version = "3.4.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "more-itertools" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/06/c0/ed4a27bc5571b99e3cff68f8a9fa5b56ff7df1c2251cc715a652ddd26402/jaraco.classes-3.4.0.tar.gz", hash = "sha256:47a024b51d0239c0dd8c8540c6c7f484be3b8fcf0b2d85c13825780d3b3f3acd", size = 11780 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/7f/66/b15ce62552d84bbfcec9a4873ab79d993a1dd4edb922cbfccae192bd5b5f/jaraco.classes-3.4.0-py3-none-any.whl", hash = "sha256:f662826b6bed8cace05e7ff873ce0f9283b5c924470fe664fff1c2f00f581790", size = 6777 }, +] + +[[package]] +name = "jaraco-context" +version = "6.0.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "backports-tarfile", marker = "python_full_version < '3.12'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/df/ad/f3777b81bf0b6e7bc7514a1656d3e637b2e8e15fab2ce3235730b3e7a4e6/jaraco_context-6.0.1.tar.gz", hash = "sha256:9bae4ea555cf0b14938dc0aee7c9f32ed303aa20a3b73e7dc80111628792d1b3", size = 13912 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ff/db/0c52c4cf5e4bd9f5d7135ec7669a3a767af21b3a308e1ed3674881e52b62/jaraco.context-6.0.1-py3-none-any.whl", hash = "sha256:f797fc481b490edb305122c9181830a3a5b76d84ef6d1aef2fb9b47ab956f9e4", size = 6825 }, +] + +[[package]] +name = "jaraco-functools" +version = "4.1.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "more-itertools" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ab/23/9894b3df5d0a6eb44611c36aec777823fc2e07740dabbd0b810e19594013/jaraco_functools-4.1.0.tar.gz", hash = "sha256:70f7e0e2ae076498e212562325e805204fc092d7b4c17e0e86c959e249701a9d", size = 19159 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9f/4f/24b319316142c44283d7540e76c7b5a6dbd5db623abd86bb7b3491c21018/jaraco.functools-4.1.0-py3-none-any.whl", hash = "sha256:ad159f13428bc4acbf5541ad6dec511f91573b90fba04df61dafa2a1231cf649", size = 10187 }, +] + +[[package]] +name = "jeepney" +version = "0.8.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d6/f4/154cf374c2daf2020e05c3c6a03c91348d59b23c5366e968feb198306fdf/jeepney-0.8.0.tar.gz", hash = "sha256:5efe48d255973902f6badc3ce55e2aa6c5c3b3bc642059ef3a91247bcfcc5806", size = 106005 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ae/72/2a1e2290f1ab1e06f71f3d0f1646c9e4634e70e1d37491535e19266e8dc9/jeepney-0.8.0-py3-none-any.whl", hash = "sha256:c0a454ad016ca575060802ee4d590dd912e35c122fa04e70306de3d076cce755", size = 48435 }, +] + +[[package]] +name = "jinja2" +version = "3.1.4" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "markupsafe" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ed/55/39036716d19cab0747a5020fc7e907f362fbf48c984b14e62127f7e68e5d/jinja2-3.1.4.tar.gz", hash = "sha256:4a3aee7acbbe7303aede8e9648d13b8bf88a429282aa6122a993f0ac800cb369", size = 240245 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/31/80/3a54838c3fb461f6fec263ebf3a3a41771bd05190238de3486aae8540c36/jinja2-3.1.4-py3-none-any.whl", hash = "sha256:bc5dd2abb727a5319567b7a813e6a2e7318c39f4f487cfe6c89c6f9c7d25197d", size = 133271 }, +] + +[[package]] +name = "jsonschema" +version = "4.23.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "attrs" }, + { name = "jsonschema-specifications" }, + { name = "referencing" }, + { name = "rpds-py" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/38/2e/03362ee4034a4c917f697890ccd4aec0800ccf9ded7f511971c75451deec/jsonschema-4.23.0.tar.gz", hash = "sha256:d71497fef26351a33265337fa77ffeb82423f3ea21283cd9467bb03999266bc4", size = 325778 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/69/4a/4f9dbeb84e8850557c02365a0eee0649abe5eb1d84af92a25731c6c0f922/jsonschema-4.23.0-py3-none-any.whl", hash = "sha256:fbadb6f8b144a8f8cf9f0b89ba94501d143e50411a1278633f56a7acf7fd5566", size = 88462 }, +] + +[[package]] +name = "jsonschema-specifications" +version = "2024.10.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "referencing" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/10/db/58f950c996c793472e336ff3655b13fbcf1e3b359dcf52dcf3ed3b52c352/jsonschema_specifications-2024.10.1.tar.gz", hash = "sha256:0f38b83639958ce1152d02a7f062902c41c8fd20d558b0c34344292d417ae272", size = 15561 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/0f/8910b19ac0670a0f80ce1008e5e751c4a57e14d2c4c13a482aa6079fa9d6/jsonschema_specifications-2024.10.1-py3-none-any.whl", hash = "sha256:a09a0680616357d9a0ecf05c12ad234479f549239d0f5b55f3deea67475da9bf", size = 18459 }, +] + +[[package]] +name = "keyring" +version = "25.5.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "importlib-metadata", marker = "python_full_version < '3.12'" }, + { name = "jaraco-classes" }, + { name = "jaraco-context" }, + { name = "jaraco-functools" }, + { name = "jeepney", marker = "sys_platform == 'linux'" }, + { name = "pywin32-ctypes", marker = "sys_platform == 'win32'" }, + { name = "secretstorage", marker = "sys_platform == 'linux'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/f6/24/64447b13df6a0e2797b586dad715766d756c932ce8ace7f67bd384d76ae0/keyring-25.5.0.tar.gz", hash = "sha256:4c753b3ec91717fe713c4edd522d625889d8973a349b0e582622f49766de58e6", size = 62675 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/32/c9/353c156fa2f057e669106e5d6bcdecf85ef8d3536ce68ca96f18dc7b6d6f/keyring-25.5.0-py3-none-any.whl", hash = "sha256:e67f8ac32b04be4714b42fe84ce7dad9c40985b9ca827c592cc303e7c26d9741", size = 39096 }, +] + +[[package]] +name = "markdown-it-py" +version = "3.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "mdurl" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/38/71/3b932df36c1a044d397a1f92d1cf91ee0a503d91e470cbd670aa66b07ed0/markdown-it-py-3.0.0.tar.gz", hash = "sha256:e3f60a94fa066dc52ec76661e37c851cb232d92f9886b15cb560aaada2df8feb", size = 74596 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/42/d7/1ec15b46af6af88f19b8e5ffea08fa375d433c998b8a7639e76935c14f1f/markdown_it_py-3.0.0-py3-none-any.whl", hash = "sha256:355216845c60bd96232cd8d8c40e8f9765cc86f46880e43a8fd22dc1a1a8cab1", size = 87528 }, +] + +[[package]] +name = "markupsafe" +version = "3.0.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b2/97/5d42485e71dfc078108a86d6de8fa46db44a1a9295e89c5d6d4a06e23a62/markupsafe-3.0.2.tar.gz", hash = "sha256:ee55d3edf80167e48ea11a923c7386f4669df67d7994554387f84e7d8b0a2bf0", size = 20537 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/04/90/d08277ce111dd22f77149fd1a5d4653eeb3b3eaacbdfcbae5afb2600eebd/MarkupSafe-3.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7e94c425039cde14257288fd61dcfb01963e658efbc0ff54f5306b06054700f8", size = 14357 }, + { url = "https://files.pythonhosted.org/packages/04/e1/6e2194baeae0bca1fae6629dc0cbbb968d4d941469cbab11a3872edff374/MarkupSafe-3.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9e2d922824181480953426608b81967de705c3cef4d1af983af849d7bd619158", size = 12393 }, + { url = "https://files.pythonhosted.org/packages/1d/69/35fa85a8ece0a437493dc61ce0bb6d459dcba482c34197e3efc829aa357f/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38a9ef736c01fccdd6600705b09dc574584b89bea478200c5fbf112a6b0d5579", size = 21732 }, + { url = "https://files.pythonhosted.org/packages/22/35/137da042dfb4720b638d2937c38a9c2df83fe32d20e8c8f3185dbfef05f7/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbcb445fa71794da8f178f0f6d66789a28d7319071af7a496d4d507ed566270d", size = 20866 }, + { url = "https://files.pythonhosted.org/packages/29/28/6d029a903727a1b62edb51863232152fd335d602def598dade38996887f0/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:57cb5a3cf367aeb1d316576250f65edec5bb3be939e9247ae594b4bcbc317dfb", size = 20964 }, + { url = "https://files.pythonhosted.org/packages/cc/cd/07438f95f83e8bc028279909d9c9bd39e24149b0d60053a97b2bc4f8aa51/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3809ede931876f5b2ec92eef964286840ed3540dadf803dd570c3b7e13141a3b", size = 21977 }, + { url = "https://files.pythonhosted.org/packages/29/01/84b57395b4cc062f9c4c55ce0df7d3108ca32397299d9df00fedd9117d3d/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e07c3764494e3776c602c1e78e298937c3315ccc9043ead7e685b7f2b8d47b3c", size = 21366 }, + { url = "https://files.pythonhosted.org/packages/bd/6e/61ebf08d8940553afff20d1fb1ba7294b6f8d279df9fd0c0db911b4bbcfd/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b424c77b206d63d500bcb69fa55ed8d0e6a3774056bdc4839fc9298a7edca171", size = 21091 }, + { url = "https://files.pythonhosted.org/packages/11/23/ffbf53694e8c94ebd1e7e491de185124277964344733c45481f32ede2499/MarkupSafe-3.0.2-cp310-cp310-win32.whl", hash = "sha256:fcabf5ff6eea076f859677f5f0b6b5c1a51e70a376b0579e0eadef8db48c6b50", size = 15065 }, + { url = "https://files.pythonhosted.org/packages/44/06/e7175d06dd6e9172d4a69a72592cb3f7a996a9c396eee29082826449bbc3/MarkupSafe-3.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:6af100e168aa82a50e186c82875a5893c5597a0c1ccdb0d8b40240b1f28b969a", size = 15514 }, + { url = "https://files.pythonhosted.org/packages/6b/28/bbf83e3f76936960b850435576dd5e67034e200469571be53f69174a2dfd/MarkupSafe-3.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9025b4018f3a1314059769c7bf15441064b2207cb3f065e6ea1e7359cb46db9d", size = 14353 }, + { url = "https://files.pythonhosted.org/packages/6c/30/316d194b093cde57d448a4c3209f22e3046c5bb2fb0820b118292b334be7/MarkupSafe-3.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:93335ca3812df2f366e80509ae119189886b0f3c2b81325d39efdb84a1e2ae93", size = 12392 }, + { url = "https://files.pythonhosted.org/packages/f2/96/9cdafba8445d3a53cae530aaf83c38ec64c4d5427d975c974084af5bc5d2/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2cb8438c3cbb25e220c2ab33bb226559e7afb3baec11c4f218ffa7308603c832", size = 23984 }, + { url = "https://files.pythonhosted.org/packages/f1/a4/aefb044a2cd8d7334c8a47d3fb2c9f328ac48cb349468cc31c20b539305f/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a123e330ef0853c6e822384873bef7507557d8e4a082961e1defa947aa59ba84", size = 23120 }, + { url = "https://files.pythonhosted.org/packages/8d/21/5e4851379f88f3fad1de30361db501300d4f07bcad047d3cb0449fc51f8c/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e084f686b92e5b83186b07e8a17fc09e38fff551f3602b249881fec658d3eca", size = 23032 }, + { url = "https://files.pythonhosted.org/packages/00/7b/e92c64e079b2d0d7ddf69899c98842f3f9a60a1ae72657c89ce2655c999d/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d8213e09c917a951de9d09ecee036d5c7d36cb6cb7dbaece4c71a60d79fb9798", size = 24057 }, + { url = "https://files.pythonhosted.org/packages/f9/ac/46f960ca323037caa0a10662ef97d0a4728e890334fc156b9f9e52bcc4ca/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5b02fb34468b6aaa40dfc198d813a641e3a63b98c2b05a16b9f80b7ec314185e", size = 23359 }, + { url = "https://files.pythonhosted.org/packages/69/84/83439e16197337b8b14b6a5b9c2105fff81d42c2a7c5b58ac7b62ee2c3b1/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0bff5e0ae4ef2e1ae4fdf2dfd5b76c75e5c2fa4132d05fc1b0dabcd20c7e28c4", size = 23306 }, + { url = "https://files.pythonhosted.org/packages/9a/34/a15aa69f01e2181ed8d2b685c0d2f6655d5cca2c4db0ddea775e631918cd/MarkupSafe-3.0.2-cp311-cp311-win32.whl", hash = "sha256:6c89876f41da747c8d3677a2b540fb32ef5715f97b66eeb0c6b66f5e3ef6f59d", size = 15094 }, + { url = "https://files.pythonhosted.org/packages/da/b8/3a3bd761922d416f3dc5d00bfbed11f66b1ab89a0c2b6e887240a30b0f6b/MarkupSafe-3.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:70a87b411535ccad5ef2f1df5136506a10775d267e197e4cf531ced10537bd6b", size = 15521 }, + { url = "https://files.pythonhosted.org/packages/22/09/d1f21434c97fc42f09d290cbb6350d44eb12f09cc62c9476effdb33a18aa/MarkupSafe-3.0.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:9778bd8ab0a994ebf6f84c2b949e65736d5575320a17ae8984a77fab08db94cf", size = 14274 }, + { url = "https://files.pythonhosted.org/packages/6b/b0/18f76bba336fa5aecf79d45dcd6c806c280ec44538b3c13671d49099fdd0/MarkupSafe-3.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:846ade7b71e3536c4e56b386c2a47adf5741d2d8b94ec9dc3e92e5e1ee1e2225", size = 12348 }, + { url = "https://files.pythonhosted.org/packages/e0/25/dd5c0f6ac1311e9b40f4af06c78efde0f3b5cbf02502f8ef9501294c425b/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c99d261bd2d5f6b59325c92c73df481e05e57f19837bdca8413b9eac4bd8028", size = 24149 }, + { url = "https://files.pythonhosted.org/packages/f3/f0/89e7aadfb3749d0f52234a0c8c7867877876e0a20b60e2188e9850794c17/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e17c96c14e19278594aa4841ec148115f9c7615a47382ecb6b82bd8fea3ab0c8", size = 23118 }, + { url = "https://files.pythonhosted.org/packages/d5/da/f2eeb64c723f5e3777bc081da884b414671982008c47dcc1873d81f625b6/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:88416bd1e65dcea10bc7569faacb2c20ce071dd1f87539ca2ab364bf6231393c", size = 22993 }, + { url = "https://files.pythonhosted.org/packages/da/0e/1f32af846df486dce7c227fe0f2398dc7e2e51d4a370508281f3c1c5cddc/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:2181e67807fc2fa785d0592dc2d6206c019b9502410671cc905d132a92866557", size = 24178 }, + { url = "https://files.pythonhosted.org/packages/c4/f6/bb3ca0532de8086cbff5f06d137064c8410d10779c4c127e0e47d17c0b71/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:52305740fe773d09cffb16f8ed0427942901f00adedac82ec8b67752f58a1b22", size = 23319 }, + { url = "https://files.pythonhosted.org/packages/a2/82/8be4c96ffee03c5b4a034e60a31294daf481e12c7c43ab8e34a1453ee48b/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ad10d3ded218f1039f11a75f8091880239651b52e9bb592ca27de44eed242a48", size = 23352 }, + { url = "https://files.pythonhosted.org/packages/51/ae/97827349d3fcffee7e184bdf7f41cd6b88d9919c80f0263ba7acd1bbcb18/MarkupSafe-3.0.2-cp312-cp312-win32.whl", hash = "sha256:0f4ca02bea9a23221c0182836703cbf8930c5e9454bacce27e767509fa286a30", size = 15097 }, + { url = "https://files.pythonhosted.org/packages/c1/80/a61f99dc3a936413c3ee4e1eecac96c0da5ed07ad56fd975f1a9da5bc630/MarkupSafe-3.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:8e06879fc22a25ca47312fbe7c8264eb0b662f6db27cb2d3bbbc74b1df4b9b87", size = 15601 }, + { url = "https://files.pythonhosted.org/packages/83/0e/67eb10a7ecc77a0c2bbe2b0235765b98d164d81600746914bebada795e97/MarkupSafe-3.0.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ba9527cdd4c926ed0760bc301f6728ef34d841f405abf9d4f959c478421e4efd", size = 14274 }, + { url = "https://files.pythonhosted.org/packages/2b/6d/9409f3684d3335375d04e5f05744dfe7e9f120062c9857df4ab490a1031a/MarkupSafe-3.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f8b3d067f2e40fe93e1ccdd6b2e1d16c43140e76f02fb1319a05cf2b79d99430", size = 12352 }, + { url = "https://files.pythonhosted.org/packages/d2/f5/6eadfcd3885ea85fe2a7c128315cc1bb7241e1987443d78c8fe712d03091/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:569511d3b58c8791ab4c2e1285575265991e6d8f8700c7be0e88f86cb0672094", size = 24122 }, + { url = "https://files.pythonhosted.org/packages/0c/91/96cf928db8236f1bfab6ce15ad070dfdd02ed88261c2afafd4b43575e9e9/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15ab75ef81add55874e7ab7055e9c397312385bd9ced94920f2802310c930396", size = 23085 }, + { url = "https://files.pythonhosted.org/packages/c2/cf/c9d56af24d56ea04daae7ac0940232d31d5a8354f2b457c6d856b2057d69/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f3818cb119498c0678015754eba762e0d61e5b52d34c8b13d770f0719f7b1d79", size = 22978 }, + { url = "https://files.pythonhosted.org/packages/2a/9f/8619835cd6a711d6272d62abb78c033bda638fdc54c4e7f4272cf1c0962b/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:cdb82a876c47801bb54a690c5ae105a46b392ac6099881cdfb9f6e95e4014c6a", size = 24208 }, + { url = "https://files.pythonhosted.org/packages/f9/bf/176950a1792b2cd2102b8ffeb5133e1ed984547b75db47c25a67d3359f77/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:cabc348d87e913db6ab4aa100f01b08f481097838bdddf7c7a84b7575b7309ca", size = 23357 }, + { url = "https://files.pythonhosted.org/packages/ce/4f/9a02c1d335caabe5c4efb90e1b6e8ee944aa245c1aaaab8e8a618987d816/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:444dcda765c8a838eaae23112db52f1efaf750daddb2d9ca300bcae1039adc5c", size = 23344 }, + { url = "https://files.pythonhosted.org/packages/ee/55/c271b57db36f748f0e04a759ace9f8f759ccf22b4960c270c78a394f58be/MarkupSafe-3.0.2-cp313-cp313-win32.whl", hash = "sha256:bcf3e58998965654fdaff38e58584d8937aa3096ab5354d493c77d1fdd66d7a1", size = 15101 }, + { url = "https://files.pythonhosted.org/packages/29/88/07df22d2dd4df40aba9f3e402e6dc1b8ee86297dddbad4872bd5e7b0094f/MarkupSafe-3.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:e6a2a455bd412959b57a172ce6328d2dd1f01cb2135efda2e4576e8a23fa3b0f", size = 15603 }, + { url = "https://files.pythonhosted.org/packages/62/6a/8b89d24db2d32d433dffcd6a8779159da109842434f1dd2f6e71f32f738c/MarkupSafe-3.0.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:b5a6b3ada725cea8a5e634536b1b01c30bcdcd7f9c6fff4151548d5bf6b3a36c", size = 14510 }, + { url = "https://files.pythonhosted.org/packages/7a/06/a10f955f70a2e5a9bf78d11a161029d278eeacbd35ef806c3fd17b13060d/MarkupSafe-3.0.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a904af0a6162c73e3edcb969eeeb53a63ceeb5d8cf642fade7d39e7963a22ddb", size = 12486 }, + { url = "https://files.pythonhosted.org/packages/34/cf/65d4a571869a1a9078198ca28f39fba5fbb910f952f9dbc5220afff9f5e6/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4aa4e5faecf353ed117801a068ebab7b7e09ffb6e1d5e412dc852e0da018126c", size = 25480 }, + { url = "https://files.pythonhosted.org/packages/0c/e3/90e9651924c430b885468b56b3d597cabf6d72be4b24a0acd1fa0e12af67/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0ef13eaeee5b615fb07c9a7dadb38eac06a0608b41570d8ade51c56539e509d", size = 23914 }, + { url = "https://files.pythonhosted.org/packages/66/8c/6c7cf61f95d63bb866db39085150df1f2a5bd3335298f14a66b48e92659c/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d16a81a06776313e817c951135cf7340a3e91e8c1ff2fac444cfd75fffa04afe", size = 23796 }, + { url = "https://files.pythonhosted.org/packages/bb/35/cbe9238ec3f47ac9a7c8b3df7a808e7cb50fe149dc7039f5f454b3fba218/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:6381026f158fdb7c72a168278597a5e3a5222e83ea18f543112b2662a9b699c5", size = 25473 }, + { url = "https://files.pythonhosted.org/packages/e6/32/7621a4382488aa283cc05e8984a9c219abad3bca087be9ec77e89939ded9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3d79d162e7be8f996986c064d1c7c817f6df3a77fe3d6859f6f9e7be4b8c213a", size = 24114 }, + { url = "https://files.pythonhosted.org/packages/0d/80/0985960e4b89922cb5a0bac0ed39c5b96cbc1a536a99f30e8c220a996ed9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:131a3c7689c85f5ad20f9f6fb1b866f402c445b220c19fe4308c0b147ccd2ad9", size = 24098 }, + { url = "https://files.pythonhosted.org/packages/82/78/fedb03c7d5380df2427038ec8d973587e90561b2d90cd472ce9254cf348b/MarkupSafe-3.0.2-cp313-cp313t-win32.whl", hash = "sha256:ba8062ed2cf21c07a9e295d5b8a2a5ce678b913b45fdf68c32d95d6c1291e0b6", size = 15208 }, + { url = "https://files.pythonhosted.org/packages/4f/65/6079a46068dfceaeabb5dcad6d674f5f5c61a6fa5673746f42a9f4c233b3/MarkupSafe-3.0.2-cp313-cp313t-win_amd64.whl", hash = "sha256:e444a31f8db13eb18ada366ab3cf45fd4b31e4db1236a4448f68778c1d1a5a2f", size = 15739 }, + { url = "https://files.pythonhosted.org/packages/a7/ea/9b1530c3fdeeca613faeb0fb5cbcf2389d816072fab72a71b45749ef6062/MarkupSafe-3.0.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:eaa0a10b7f72326f1372a713e73c3f739b524b3af41feb43e4921cb529f5929a", size = 14344 }, + { url = "https://files.pythonhosted.org/packages/4b/c2/fbdbfe48848e7112ab05e627e718e854d20192b674952d9042ebd8c9e5de/MarkupSafe-3.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:48032821bbdf20f5799ff537c7ac3d1fba0ba032cfc06194faffa8cda8b560ff", size = 12389 }, + { url = "https://files.pythonhosted.org/packages/f0/25/7a7c6e4dbd4f867d95d94ca15449e91e52856f6ed1905d58ef1de5e211d0/MarkupSafe-3.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a9d3f5f0901fdec14d8d2f66ef7d035f2157240a433441719ac9a3fba440b13", size = 21607 }, + { url = "https://files.pythonhosted.org/packages/53/8f/f339c98a178f3c1e545622206b40986a4c3307fe39f70ccd3d9df9a9e425/MarkupSafe-3.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:88b49a3b9ff31e19998750c38e030fc7bb937398b1f78cfa599aaef92d693144", size = 20728 }, + { url = "https://files.pythonhosted.org/packages/1a/03/8496a1a78308456dbd50b23a385c69b41f2e9661c67ea1329849a598a8f9/MarkupSafe-3.0.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cfad01eed2c2e0c01fd0ecd2ef42c492f7f93902e39a42fc9ee1692961443a29", size = 20826 }, + { url = "https://files.pythonhosted.org/packages/e6/cf/0a490a4bd363048c3022f2f475c8c05582179bb179defcee4766fb3dcc18/MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:1225beacc926f536dc82e45f8a4d68502949dc67eea90eab715dea3a21c1b5f0", size = 21843 }, + { url = "https://files.pythonhosted.org/packages/19/a3/34187a78613920dfd3cdf68ef6ce5e99c4f3417f035694074beb8848cd77/MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3169b1eefae027567d1ce6ee7cae382c57fe26e82775f460f0b2778beaad66c0", size = 21219 }, + { url = "https://files.pythonhosted.org/packages/17/d8/5811082f85bb88410ad7e452263af048d685669bbbfb7b595e8689152498/MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:eb7972a85c54febfb25b5c4b4f3af4dcc731994c7da0d8a0b4a6eb0640e1d178", size = 20946 }, + { url = "https://files.pythonhosted.org/packages/7c/31/bd635fb5989440d9365c5e3c47556cfea121c7803f5034ac843e8f37c2f2/MarkupSafe-3.0.2-cp39-cp39-win32.whl", hash = "sha256:8c4e8c3ce11e1f92f6536ff07154f9d49677ebaaafc32db9db4620bc11ed480f", size = 15063 }, + { url = "https://files.pythonhosted.org/packages/b3/73/085399401383ce949f727afec55ec3abd76648d04b9f22e1c0e99cb4bec3/MarkupSafe-3.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:6e296a513ca3d94054c2c881cc913116e90fd030ad1c656b3869762b754f5f8a", size = 15506 }, +] + +[[package]] +name = "mdurl" +version = "0.1.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d6/54/cfe61301667036ec958cb99bd3efefba235e65cdeb9c84d24a8293ba1d90/mdurl-0.1.2.tar.gz", hash = "sha256:bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba", size = 8729 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b3/38/89ba8ad64ae25be8de66a6d463314cf1eb366222074cfda9ee839c56a4b4/mdurl-0.1.2-py3-none-any.whl", hash = "sha256:84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8", size = 9979 }, +] + +[[package]] +name = "more-itertools" +version = "10.5.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/51/78/65922308c4248e0eb08ebcbe67c95d48615cc6f27854b6f2e57143e9178f/more-itertools-10.5.0.tar.gz", hash = "sha256:5482bfef7849c25dc3c6dd53a6173ae4795da2a41a80faea6700d9f5846c5da6", size = 121020 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/48/7e/3a64597054a70f7c86eb0a7d4fc315b8c1ab932f64883a297bdffeb5f967/more_itertools-10.5.0-py3-none-any.whl", hash = "sha256:037b0d3203ce90cca8ab1defbbdac29d5f993fc20131f3664dc8d6acfa872aef", size = 60952 }, +] + +[[package]] +name = "mypy-extensions" +version = "1.0.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/98/a4/1ab47638b92648243faf97a5aeb6ea83059cc3624972ab6b8d2316078d3f/mypy_extensions-1.0.0.tar.gz", hash = "sha256:75dbf8955dc00442a438fc4d0666508a9a97b6bd41aa2f0ffe9d2f2725af0782", size = 4433 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/2a/e2/5d3f6ada4297caebe1a2add3b126fe800c96f56dbe5d1988a2cbe0b267aa/mypy_extensions-1.0.0-py3-none-any.whl", hash = "sha256:4392f6c0eb8a5668a69e23d168ffa70f0be9ccfd32b5cc2d26a34ae5b844552d", size = 4695 }, +] + +[[package]] +name = "nh3" +version = "0.2.18" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/62/73/10df50b42ddb547a907deeb2f3c9823022580a7a47281e8eae8e003a9639/nh3-0.2.18.tar.gz", hash = "sha256:94a166927e53972a9698af9542ace4e38b9de50c34352b962f4d9a7d4c927af4", size = 15028 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b3/89/1daff5d9ba5a95a157c092c7c5f39b8dd2b1ddb4559966f808d31cfb67e0/nh3-0.2.18-cp37-abi3-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:14c5a72e9fe82aea5fe3072116ad4661af5cf8e8ff8fc5ad3450f123e4925e86", size = 1374474 }, + { url = "https://files.pythonhosted.org/packages/2c/b6/42fc3c69cabf86b6b81e4c051a9b6e249c5ba9f8155590222c2622961f58/nh3-0.2.18-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:7b7c2a3c9eb1a827d42539aa64091640bd275b81e097cd1d8d82ef91ffa2e811", size = 694573 }, + { url = "https://files.pythonhosted.org/packages/45/b9/833f385403abaf0023c6547389ec7a7acf141ddd9d1f21573723a6eab39a/nh3-0.2.18-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:42c64511469005058cd17cc1537578eac40ae9f7200bedcfd1fc1a05f4f8c200", size = 844082 }, + { url = "https://files.pythonhosted.org/packages/05/2b/85977d9e11713b5747595ee61f381bc820749daf83f07b90b6c9964cf932/nh3-0.2.18-cp37-abi3-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:0411beb0589eacb6734f28d5497ca2ed379eafab8ad8c84b31bb5c34072b7164", size = 782460 }, + { url = "https://files.pythonhosted.org/packages/72/f2/5c894d5265ab80a97c68ca36f25c8f6f0308abac649aaf152b74e7e854a8/nh3-0.2.18-cp37-abi3-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:5f36b271dae35c465ef5e9090e1fdaba4a60a56f0bb0ba03e0932a66f28b9189", size = 879827 }, + { url = "https://files.pythonhosted.org/packages/ab/a7/375afcc710dbe2d64cfbd69e31f82f3e423d43737258af01f6a56d844085/nh3-0.2.18-cp37-abi3-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:34c03fa78e328c691f982b7c03d4423bdfd7da69cd707fe572f544cf74ac23ad", size = 841080 }, + { url = "https://files.pythonhosted.org/packages/c2/a8/3bb02d0c60a03ad3a112b76c46971e9480efa98a8946677b5a59f60130ca/nh3-0.2.18-cp37-abi3-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:19aaba96e0f795bd0a6c56291495ff59364f4300d4a39b29a0abc9cb3774a84b", size = 924144 }, + { url = "https://files.pythonhosted.org/packages/1b/63/6ab90d0e5225ab9780f6c9fb52254fa36b52bb7c188df9201d05b647e5e1/nh3-0.2.18-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:de3ceed6e661954871d6cd78b410213bdcb136f79aafe22aa7182e028b8c7307", size = 769192 }, + { url = "https://files.pythonhosted.org/packages/a4/17/59391c28580e2c32272761629893e761442fc7666da0b1cdb479f3b67b88/nh3-0.2.18-cp37-abi3-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6955369e4d9f48f41e3f238a9e60f9410645db7e07435e62c6a9ea6135a4907f", size = 791042 }, + { url = "https://files.pythonhosted.org/packages/a3/da/0c4e282bc3cff4a0adf37005fa1fb42257673fbc1bbf7d1ff639ec3d255a/nh3-0.2.18-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:f0eca9ca8628dbb4e916ae2491d72957fdd35f7a5d326b7032a345f111ac07fe", size = 1010073 }, + { url = "https://files.pythonhosted.org/packages/de/81/c291231463d21da5f8bba82c8167a6d6893cc5419b0639801ee5d3aeb8a9/nh3-0.2.18-cp37-abi3-musllinux_1_2_armv7l.whl", hash = "sha256:3a157ab149e591bb638a55c8c6bcb8cdb559c8b12c13a8affaba6cedfe51713a", size = 1029782 }, + { url = "https://files.pythonhosted.org/packages/63/1d/842fed85cf66c973be0aed8770093d6a04741f65e2c388ddd4c07fd3296e/nh3-0.2.18-cp37-abi3-musllinux_1_2_i686.whl", hash = "sha256:c8b3a1cebcba9b3669ed1a84cc65bf005728d2f0bc1ed2a6594a992e817f3a50", size = 942504 }, + { url = "https://files.pythonhosted.org/packages/eb/61/73a007c74c37895fdf66e0edcd881f5eaa17a348ff02f4bb4bc906d61085/nh3-0.2.18-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:36c95d4b70530b320b365659bb5034341316e6a9b30f0b25fa9c9eff4c27a204", size = 941541 }, + { url = "https://files.pythonhosted.org/packages/78/48/54a788fc9428e481b2f58e0cd8564f6c74ffb6e9ef73d39e8acbeae8c629/nh3-0.2.18-cp37-abi3-win32.whl", hash = "sha256:a7f1b5b2c15866f2db413a3649a8fe4fd7b428ae58be2c0f6bca5eefd53ca2be", size = 573750 }, + { url = "https://files.pythonhosted.org/packages/26/8d/53c5b19c4999bdc6ba95f246f4ef35ca83d7d7423e5e38be43ad66544e5d/nh3-0.2.18-cp37-abi3-win_amd64.whl", hash = "sha256:8ce0f819d2f1933953fca255db2471ad58184a60508f03e6285e5114b6254844", size = 579012 }, +] + +[[package]] +name = "nodeenv" +version = "1.9.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/43/16/fc88b08840de0e0a72a2f9d8c6bae36be573e475a6326ae854bcc549fc45/nodeenv-1.9.1.tar.gz", hash = "sha256:6ec12890a2dab7946721edbfbcd91f3319c6ccc9aec47be7c7e6b7011ee6645f", size = 47437 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d2/1d/1b658dbd2b9fa9c4c9f32accbfc0205d532c8c6194dc0f2a4c0428e7128a/nodeenv-1.9.1-py2.py3-none-any.whl", hash = "sha256:ba11c9782d29c27c70ffbdda2d7415098754709be8a7056d79a737cd901155c9", size = 22314 }, +] + +[[package]] +name = "oauthlib" +version = "3.2.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6d/fa/fbf4001037904031639e6bfbfc02badfc7e12f137a8afa254df6c4c8a670/oauthlib-3.2.2.tar.gz", hash = "sha256:9859c40929662bec5d64f34d01c99e093149682a3f38915dc0655d5a633dd918", size = 177352 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/7e/80/cab10959dc1faead58dc8384a781dfbf93cb4d33d50988f7a69f1b7c9bbe/oauthlib-3.2.2-py3-none-any.whl", hash = "sha256:8139f29aac13e25d502680e9e19963e83f16838d48a0d71c287fe40e7067fbca", size = 151688 }, +] + +[[package]] +name = "packaging" +version = "24.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/51/65/50db4dda066951078f0a96cf12f4b9ada6e4b811516bf0262c0f4f7064d4/packaging-24.1.tar.gz", hash = "sha256:026ed72c8ed3fcce5bf8950572258698927fd1dbda10a5e981cdf0ac37f4f002", size = 148788 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/08/aa/cc0199a5f0ad350994d660967a8efb233fe0416e4639146c089643407ce6/packaging-24.1-py3-none-any.whl", hash = "sha256:5b8f2217dbdbd2f7f384c41c628544e6d52f2d0f53c6d0c3ea61aa5d1d7ff124", size = 53985 }, +] + +[[package]] +name = "pathspec" +version = "0.12.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ca/bc/f35b8446f4531a7cb215605d100cd88b7ac6f44ab3fc94870c120ab3adbf/pathspec-0.12.1.tar.gz", hash = "sha256:a482d51503a1ab33b1c67a6c3813a26953dbdc71c31dacaef9a838c4e29f5712", size = 51043 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/cc/20/ff623b09d963f88bfde16306a54e12ee5ea43e9b597108672ff3a408aad6/pathspec-0.12.1-py3-none-any.whl", hash = "sha256:a0d503e138a4c123b27490a4f7beda6a01c6f288df0e4a8b79c7eb0dc7b4cc08", size = 31191 }, +] + +[[package]] +name = "pexpect" +version = "4.9.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "ptyprocess" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/42/92/cc564bf6381ff43ce1f4d06852fc19a2f11d180f23dc32d9588bee2f149d/pexpect-4.9.0.tar.gz", hash = "sha256:ee7d41123f3c9911050ea2c2dac107568dc43b2d3b0c7557a33212c398ead30f", size = 166450 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772 }, +] + +[[package]] +name = "pip" +version = "24.3.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f4/b1/b422acd212ad7eedddaf7981eee6e5de085154ff726459cf2da7c5a184c1/pip-24.3.1.tar.gz", hash = "sha256:ebcb60557f2aefabc2e0f918751cd24ea0d56d8ec5445fe1807f1d2109660b99", size = 1931073 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ef/7d/500c9ad20238fcfcb4cb9243eede163594d7020ce87bd9610c9e02771876/pip-24.3.1-py3-none-any.whl", hash = "sha256:3790624780082365f47549d032f3770eeb2b1e8bd1f7b2e02dace1afa361b4ed", size = 1822182 }, +] + +[[package]] +name = "pipx" +version = "1.7.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "argcomplete" }, + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "packaging" }, + { name = "platformdirs" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, + { name = "userpath" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/17/21/dd6b9a9c4f0cb659ce3dad991f0e8dde852b2c81922224ef77df4222ab7a/pipx-1.7.1.tar.gz", hash = "sha256:762de134e16a462be92645166d225ecef446afaef534917f5f70008d63584360", size = 291889 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/35/af/66db02a214590a841bcd1df1f02f7ef818dc3f43487acddab0b8c40b25d2/pipx-1.7.1-py3-none-any.whl", hash = "sha256:3933c43bb344e649cb28e10d357e0967ce8572f1c19caf90cf39ae95c2a0afaf", size = 78749 }, +] + +[[package]] +name = "pkginfo" +version = "1.10.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/2f/72/347ec5be4adc85c182ed2823d8d1c7b51e13b9a6b0c1aae59582eca652df/pkginfo-1.10.0.tar.gz", hash = "sha256:5df73835398d10db79f8eecd5cd86b1f6d29317589ea70796994d49399af6297", size = 378457 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/56/09/054aea9b7534a15ad38a363a2bd974c20646ab1582a387a95b8df1bfea1c/pkginfo-1.10.0-py3-none-any.whl", hash = "sha256:889a6da2ed7ffc58ab5b900d888ddce90bce912f2d2de1dc1c26f4cb9fe65097", size = 30392 }, +] + +[[package]] +name = "platformdirs" +version = "4.3.6" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/13/fc/128cc9cb8f03208bdbf93d3aa862e16d376844a14f9a0ce5cf4507372de4/platformdirs-4.3.6.tar.gz", hash = "sha256:357fb2acbc885b0419afd3ce3ed34564c13c9b95c89360cd9563f73aa5e2b907", size = 21302 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3c/a6/bc1012356d8ece4d66dd75c4b9fc6c1f6650ddd5991e421177d9f8f671be/platformdirs-4.3.6-py3-none-any.whl", hash = "sha256:73e575e1408ab8103900836b97580d5307456908a03e92031bab39e4554cc3fb", size = 18439 }, +] + +[[package]] +name = "pluggy" +version = "1.5.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/96/2d/02d4312c973c6050a18b314a5ad0b3210edb65a906f868e31c111dede4a6/pluggy-1.5.0.tar.gz", hash = "sha256:2cffa88e94fdc978c4c574f15f9e59b7f4201d439195c3715ca9e2486f1d0cf1", size = 67955 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/88/5f/e351af9a41f866ac3f1fac4ca0613908d9a41741cfcf2228f4ad853b697d/pluggy-1.5.0-py3-none-any.whl", hash = "sha256:44e1ad92c8ca002de6377e165f3e0f1be63266ab4d554740532335b9d75ea669", size = 20556 }, +] + +[[package]] +name = "pre-commit" +version = "4.0.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cfgv" }, + { name = "identify" }, + { name = "nodeenv" }, + { name = "pyyaml" }, + { name = "virtualenv" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/2e/c8/e22c292035f1bac8b9f5237a2622305bc0304e776080b246f3df57c4ff9f/pre_commit-4.0.1.tar.gz", hash = "sha256:80905ac375958c0444c65e9cebebd948b3cdb518f335a091a670a89d652139d2", size = 191678 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/16/8f/496e10d51edd6671ebe0432e33ff800aa86775d2d147ce7d43389324a525/pre_commit-4.0.1-py2.py3-none-any.whl", hash = "sha256:efde913840816312445dc98787724647c65473daefe420785f885e8ed9a06878", size = 218713 }, +] + +[[package]] +name = "pre-commit-uv" +version = "4.1.4" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pre-commit" }, + { name = "uv" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b1/6c/c3c1d01698c8abb0b546defc0304971fa7fb2ba84ad35587b9dad095d73f/pre_commit_uv-4.1.4.tar.gz", hash = "sha256:3db606a79b226127b27dbbd8381b78c0e30de3ac775a8492c576a68e9250535c", size = 6493 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f1/70/1b65f9118ef64f6ffe5d57a67170bbff25d4f4a3d1cb78e8ed3392e16114/pre_commit_uv-4.1.4-py3-none-any.whl", hash = "sha256:7f01fb494fa1caa5097d20a38f71df7cea0209197b2564699cef9b3f3aa9d135", size = 5578 }, +] + +[[package]] +name = "proto-plus" +version = "1.25.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "protobuf" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/7e/05/74417b2061e1bf1b82776037cad97094228fa1c1b6e82d08a78d3fb6ddb6/proto_plus-1.25.0.tar.gz", hash = "sha256:fbb17f57f7bd05a68b7707e745e26528b0b3c34e378db91eef93912c54982d91", size = 56124 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/dd/25/0b7cc838ae3d76d46539020ec39fc92bfc9acc29367e58fe912702c2a79e/proto_plus-1.25.0-py3-none-any.whl", hash = "sha256:c91fc4a65074ade8e458e95ef8bac34d4008daa7cce4a12d6707066fca648961", size = 50126 }, +] + +[[package]] +name = "protobuf" +version = "5.28.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/74/6e/e69eb906fddcb38f8530a12f4b410699972ab7ced4e21524ece9d546ac27/protobuf-5.28.3.tar.gz", hash = "sha256:64badbc49180a5e401f373f9ce7ab1d18b63f7dd4a9cdc43c92b9f0b481cef7b", size = 422479 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/c5/05163fad52d7c43e124a545f1372d18266db36036377ad29de4271134a6a/protobuf-5.28.3-cp310-abi3-win32.whl", hash = "sha256:0c4eec6f987338617072592b97943fdbe30d019c56126493111cf24344c1cc24", size = 419624 }, + { url = "https://files.pythonhosted.org/packages/9c/4c/4563ebe001ff30dca9d7ed12e471fa098d9759712980cde1fd03a3a44fb7/protobuf-5.28.3-cp310-abi3-win_amd64.whl", hash = "sha256:91fba8f445723fcf400fdbe9ca796b19d3b1242cd873907979b9ed71e4afe868", size = 431464 }, + { url = "https://files.pythonhosted.org/packages/1c/f2/baf397f3dd1d3e4af7e3f5a0382b868d25ac068eefe1ebde05132333436c/protobuf-5.28.3-cp38-abi3-macosx_10_9_universal2.whl", hash = "sha256:a3f6857551e53ce35e60b403b8a27b0295f7d6eb63d10484f12bc6879c715687", size = 414743 }, + { url = "https://files.pythonhosted.org/packages/85/50/cd61a358ba1601f40e7d38bcfba22e053f40ef2c50d55b55926aecc8fec7/protobuf-5.28.3-cp38-abi3-manylinux2014_aarch64.whl", hash = "sha256:3fa2de6b8b29d12c61911505d893afe7320ce7ccba4df913e2971461fa36d584", size = 316511 }, + { url = "https://files.pythonhosted.org/packages/5d/ae/3257b09328c0b4e59535e497b0c7537d4954038bdd53a2f0d2f49d15a7c4/protobuf-5.28.3-cp38-abi3-manylinux2014_x86_64.whl", hash = "sha256:712319fbdddb46f21abb66cd33cb9e491a5763b2febd8f228251add221981135", size = 316624 }, + { url = "https://files.pythonhosted.org/packages/57/b5/ee3d918f536168def73b3f49edeba065429ab3a7e7b033d33e69c46ddff9/protobuf-5.28.3-cp39-cp39-win32.whl", hash = "sha256:135658402f71bbd49500322c0f736145731b16fc79dc8f367ab544a17eab4535", size = 419648 }, + { url = "https://files.pythonhosted.org/packages/53/54/e1bdf6f1d29828ddb6aca0a83bf208ab1d5f88126f34e17e487b2cd20d93/protobuf-5.28.3-cp39-cp39-win_amd64.whl", hash = "sha256:70585a70fc2dd4818c51287ceef5bdba6387f88a578c86d47bb34669b5552c36", size = 431591 }, + { url = "https://files.pythonhosted.org/packages/ad/c3/2377c159e28ea89a91cf1ca223f827ae8deccb2c9c401e5ca233cd73002f/protobuf-5.28.3-py3-none-any.whl", hash = "sha256:cee1757663fa32a1ee673434fcf3bf24dd54763c79690201208bafec62f19eed", size = 169511 }, +] + +[[package]] +name = "psutil" +version = "6.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/26/10/2a30b13c61e7cf937f4adf90710776b7918ed0a9c434e2c38224732af310/psutil-6.1.0.tar.gz", hash = "sha256:353815f59a7f64cdaca1c0307ee13558a0512f6db064e92fe833784f08539c7a", size = 508565 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/01/9e/8be43078a171381953cfee33c07c0d628594b5dbfc5157847b85022c2c1b/psutil-6.1.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:6e2dcd475ce8b80522e51d923d10c7871e45f20918e027ab682f94f1c6351688", size = 247762 }, + { url = "https://files.pythonhosted.org/packages/1d/cb/313e80644ea407f04f6602a9e23096540d9dc1878755f3952ea8d3d104be/psutil-6.1.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:0895b8414afafc526712c498bd9de2b063deaac4021a3b3c34566283464aff8e", size = 248777 }, + { url = "https://files.pythonhosted.org/packages/65/8e/bcbe2025c587b5d703369b6a75b65d41d1367553da6e3f788aff91eaf5bd/psutil-6.1.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9dcbfce5d89f1d1f2546a2090f4fcf87c7f669d1d90aacb7d7582addece9fb38", size = 284259 }, + { url = "https://files.pythonhosted.org/packages/58/4d/8245e6f76a93c98aab285a43ea71ff1b171bcd90c9d238bf81f7021fb233/psutil-6.1.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:498c6979f9c6637ebc3a73b3f87f9eb1ec24e1ce53a7c5173b8508981614a90b", size = 287255 }, + { url = "https://files.pythonhosted.org/packages/27/c2/d034856ac47e3b3cdfa9720d0e113902e615f4190d5d1bdb8df4b2015fb2/psutil-6.1.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d905186d647b16755a800e7263d43df08b790d709d575105d419f8b6ef65423a", size = 288804 }, + { url = "https://files.pythonhosted.org/packages/ea/55/5389ed243c878725feffc0d6a3bc5ef6764312b6fc7c081faaa2cfa7ef37/psutil-6.1.0-cp37-abi3-win32.whl", hash = "sha256:1ad45a1f5d0b608253b11508f80940985d1d0c8f6111b5cb637533a0e6ddc13e", size = 250386 }, + { url = "https://files.pythonhosted.org/packages/11/91/87fa6f060e649b1e1a7b19a4f5869709fbf750b7c8c262ee776ec32f3028/psutil-6.1.0-cp37-abi3-win_amd64.whl", hash = "sha256:a8fb3752b491d246034fa4d279ff076501588ce8cbcdbb62c32fd7a377d996be", size = 254228 }, +] + +[[package]] +name = "ptyprocess" +version = "0.7.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/20/e5/16ff212c1e452235a90aeb09066144d0c5a6a8c0834397e03f5224495c4e/ptyprocess-0.7.0.tar.gz", hash = "sha256:5c5d0a3b48ceee0b48485e0c26037c0acd7d29765ca3fbb5cb3831d347423220", size = 70762 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993 }, +] + +[[package]] +name = "pyasn1" +version = "0.6.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ba/e9/01f1a64245b89f039897cb0130016d79f77d52669aae6ee7b159a6c4c018/pyasn1-0.6.1.tar.gz", hash = "sha256:6f580d2bdd84365380830acf45550f2511469f673cb4a5ae3857a3170128b034", size = 145322 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c8/f1/d6a797abb14f6283c0ddff96bbdd46937f64122b8c925cab503dd37f8214/pyasn1-0.6.1-py3-none-any.whl", hash = "sha256:0d632f46f2ba09143da3a8afe9e33fb6f92fa2320ab7e886e2d0f7672af84629", size = 83135 }, +] + +[[package]] +name = "pyasn1-modules" +version = "0.4.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pyasn1" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/1d/67/6afbf0d507f73c32d21084a79946bfcfca5fbc62a72057e9c23797a737c9/pyasn1_modules-0.4.1.tar.gz", hash = "sha256:c28e2dbf9c06ad61c71a075c7e0f9fd0f1b0bb2d2ad4377f240d33ac2ab60a7c", size = 310028 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/77/89/bc88a6711935ba795a679ea6ebee07e128050d6382eaa35a0a47c8032bdc/pyasn1_modules-0.4.1-py3-none-any.whl", hash = "sha256:49bfa96b45a292b711e986f222502c1c9a5e1f4e568fc30e2574a6c7d07838fd", size = 181537 }, +] + +[[package]] +name = "pycparser" +version = "2.22" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/1d/b2/31537cf4b1ca988837256c910a668b553fceb8f069bedc4b1c826024b52c/pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6", size = 172736 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/13/a3/a812df4e2dd5696d1f351d58b8fe16a405b234ad2886a0dab9183fb78109/pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc", size = 117552 }, +] + +[[package]] +name = "pygithub" +version = "2.4.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "deprecated" }, + { name = "pyjwt", extra = ["crypto"] }, + { name = "pynacl" }, + { name = "requests" }, + { name = "typing-extensions" }, + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/f1/a0/1e8b8ca88df9857836f5bf8e3ee15dfb810d19814ef700b12f99ce11f691/pygithub-2.4.0.tar.gz", hash = "sha256:6601e22627e87bac192f1e2e39c6e6f69a43152cfb8f307cee575879320b3051", size = 3476673 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0a/f3/e185613c411757c0c18b904ea2db173f2872397eddf444a3fe8cdde47077/PyGithub-2.4.0-py3-none-any.whl", hash = "sha256:81935aa4bdc939fba98fee1cb47422c09157c56a27966476ff92775602b9ee24", size = 362599 }, +] + +[[package]] +name = "pygments" +version = "2.18.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/8e/62/8336eff65bcbc8e4cb5d05b55faf041285951b6e80f33e2bff2024788f31/pygments-2.18.0.tar.gz", hash = "sha256:786ff802f32e91311bff3889f6e9a86e81505fe99f2735bb6d60ae0c5004f199", size = 4891905 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f7/3f/01c8b82017c199075f8f788d0d906b9ffbbc5a47dc9918a945e13d5a2bda/pygments-2.18.0-py3-none-any.whl", hash = "sha256:b8e6aca0523f3ab76fee51799c488e38782ac06eafcf95e7ba832985c8e7b13a", size = 1205513 }, +] + +[[package]] +name = "pyjwt" +version = "2.9.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/fb/68/ce067f09fca4abeca8771fe667d89cc347d1e99da3e093112ac329c6020e/pyjwt-2.9.0.tar.gz", hash = "sha256:7e1e5b56cc735432a7369cbfa0efe50fa113ebecdc04ae6922deba8b84582d0c", size = 78825 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/79/84/0fdf9b18ba31d69877bd39c9cd6052b47f3761e9910c15de788e519f079f/PyJWT-2.9.0-py3-none-any.whl", hash = "sha256:3b02fb0f44517787776cf48f2ae25d8e14f300e6d7545a4315cee571a415e850", size = 22344 }, +] + +[package.optional-dependencies] +crypto = [ + { name = "cryptography" }, +] + +[[package]] +name = "pynacl" +version = "1.5.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cffi" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a7/22/27582568be639dfe22ddb3902225f91f2f17ceff88ce80e4db396c8986da/PyNaCl-1.5.0.tar.gz", hash = "sha256:8ac7448f09ab85811607bdd21ec2464495ac8b7c66d146bf545b0f08fb9220ba", size = 3392854 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ce/75/0b8ede18506041c0bf23ac4d8e2971b4161cd6ce630b177d0a08eb0d8857/PyNaCl-1.5.0-cp36-abi3-macosx_10_10_universal2.whl", hash = "sha256:401002a4aaa07c9414132aaed7f6836ff98f59277a234704ff66878c2ee4a0d1", size = 349920 }, + { url = "https://files.pythonhosted.org/packages/59/bb/fddf10acd09637327a97ef89d2a9d621328850a72f1fdc8c08bdf72e385f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_24_aarch64.whl", hash = "sha256:52cb72a79269189d4e0dc537556f4740f7f0a9ec41c1322598799b0bdad4ef92", size = 601722 }, + { url = "https://files.pythonhosted.org/packages/5d/70/87a065c37cca41a75f2ce113a5a2c2aa7533be648b184ade58971b5f7ccc/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a36d4a9dda1f19ce6e03c9a784a2921a4b726b02e1c736600ca9c22029474394", size = 680087 }, + { url = "https://files.pythonhosted.org/packages/ee/87/f1bb6a595f14a327e8285b9eb54d41fef76c585a0edef0a45f6fc95de125/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:0c84947a22519e013607c9be43706dd42513f9e6ae5d39d3613ca1e142fba44d", size = 856678 }, + { url = "https://files.pythonhosted.org/packages/66/28/ca86676b69bf9f90e710571b67450508484388bfce09acf8a46f0b8c785f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:06b8f6fa7f5de8d5d2f7573fe8c863c051225a27b61e6860fd047b1775807858", size = 1133660 }, + { url = "https://files.pythonhosted.org/packages/3d/85/c262db650e86812585e2bc59e497a8f59948a005325a11bbbc9ecd3fe26b/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:a422368fc821589c228f4c49438a368831cb5bbc0eab5ebe1d7fac9dded6567b", size = 663824 }, + { url = "https://files.pythonhosted.org/packages/fd/1a/cc308a884bd299b651f1633acb978e8596c71c33ca85e9dc9fa33a5399b9/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:61f642bf2378713e2c2e1de73444a3778e5f0a38be6fee0fe532fe30060282ff", size = 1117912 }, + { url = "https://files.pythonhosted.org/packages/25/2d/b7df6ddb0c2a33afdb358f8af6ea3b8c4d1196ca45497dd37a56f0c122be/PyNaCl-1.5.0-cp36-abi3-win32.whl", hash = "sha256:e46dae94e34b085175f8abb3b0aaa7da40767865ac82c928eeb9e57e1ea8a543", size = 204624 }, + { url = "https://files.pythonhosted.org/packages/5e/22/d3db169895faaf3e2eda892f005f433a62db2decbcfbc2f61e6517adfa87/PyNaCl-1.5.0-cp36-abi3-win_amd64.whl", hash = "sha256:20f42270d27e1b6a29f54032090b972d97f0a1b0948cc52392041ef7831fee93", size = 212141 }, +] + +[[package]] +name = "pyparsing" +version = "3.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/8c/d5/e5aeee5387091148a19e1145f63606619cb5f20b83fccb63efae6474e7b2/pyparsing-3.2.0.tar.gz", hash = "sha256:cbf74e27246d595d9a74b186b810f6fbb86726dbf3b9532efb343f6d7294fe9c", size = 920984 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/be/ec/2eb3cd785efd67806c46c13a17339708ddc346cbb684eade7a6e6f79536a/pyparsing-3.2.0-py3-none-any.whl", hash = "sha256:93d9577b88da0bbea8cc8334ee8b918ed014968fd2ec383e868fb8afb1ccef84", size = 106921 }, +] + +[[package]] +name = "pytest" +version = "8.3.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "exceptiongroup", marker = "python_full_version < '3.11'" }, + { name = "iniconfig" }, + { name = "packaging" }, + { name = "pluggy" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/8b/6c/62bbd536103af674e227c41a8f3dcd022d591f6eed5facb5a0f31ee33bbc/pytest-8.3.3.tar.gz", hash = "sha256:70b98107bd648308a7952b06e6ca9a50bc660be218d53c257cc1fc94fda10181", size = 1442487 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6b/77/7440a06a8ead44c7757a64362dd22df5760f9b12dc5f11b6188cd2fc27a0/pytest-8.3.3-py3-none-any.whl", hash = "sha256:a6853c7375b2663155079443d2e45de913a911a11d669df02a50814944db57b2", size = 342341 }, +] + +[[package]] +name = "pytest-xdist" +version = "3.6.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "execnet" }, + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/41/c4/3c310a19bc1f1e9ef50075582652673ef2bfc8cd62afef9585683821902f/pytest_xdist-3.6.1.tar.gz", hash = "sha256:ead156a4db231eec769737f57668ef58a2084a34b2e55c4a8fa20d861107300d", size = 84060 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6d/82/1d96bf03ee4c0fdc3c0cbe61470070e659ca78dc0086fb88b66c185e2449/pytest_xdist-3.6.1-py3-none-any.whl", hash = "sha256:9ed4adfb68a016610848639bb7e02c9352d5d9f03d04809919e2dafc3be4cca7", size = 46108 }, +] + +[[package]] +name = "pywin32-ctypes" +version = "0.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/85/9f/01a1a99704853cb63f253eea009390c88e7131c67e66a0a02099a8c917cb/pywin32-ctypes-0.2.3.tar.gz", hash = "sha256:d162dc04946d704503b2edc4d55f3dba5c1d539ead017afa00142c38b9885755", size = 29471 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/de/3d/8161f7711c017e01ac9f008dfddd9410dff3674334c233bde66e7ba65bbf/pywin32_ctypes-0.2.3-py3-none-any.whl", hash = "sha256:8a1513379d709975552d202d942d9837758905c8d01eb82b8bcc30918929e7b8", size = 30756 }, +] + +[[package]] +name = "pyyaml" +version = "6.0.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/54/ed/79a089b6be93607fa5cdaedf301d7dfb23af5f25c398d5ead2525b063e17/pyyaml-6.0.2.tar.gz", hash = "sha256:d584d9ec91ad65861cc08d42e834324ef890a082e591037abe114850ff7bbc3e", size = 130631 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9b/95/a3fac87cb7158e231b5a6012e438c647e1a87f09f8e0d123acec8ab8bf71/PyYAML-6.0.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0a9a2848a5b7feac301353437eb7d5957887edbf81d56e903999a75a3d743086", size = 184199 }, + { url = "https://files.pythonhosted.org/packages/c7/7a/68bd47624dab8fd4afbfd3c48e3b79efe09098ae941de5b58abcbadff5cb/PyYAML-6.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:29717114e51c84ddfba879543fb232a6ed60086602313ca38cce623c1d62cfbf", size = 171758 }, + { url = "https://files.pythonhosted.org/packages/49/ee/14c54df452143b9ee9f0f29074d7ca5516a36edb0b4cc40c3f280131656f/PyYAML-6.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8824b5a04a04a047e72eea5cec3bc266db09e35de6bdfe34c9436ac5ee27d237", size = 718463 }, + { url = "https://files.pythonhosted.org/packages/4d/61/de363a97476e766574650d742205be468921a7b532aa2499fcd886b62530/PyYAML-6.0.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7c36280e6fb8385e520936c3cb3b8042851904eba0e58d277dca80a5cfed590b", size = 719280 }, + { url = "https://files.pythonhosted.org/packages/6b/4e/1523cb902fd98355e2e9ea5e5eb237cbc5f3ad5f3075fa65087aa0ecb669/PyYAML-6.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ec031d5d2feb36d1d1a24380e4db6d43695f3748343d99434e6f5f9156aaa2ed", size = 751239 }, + { url = "https://files.pythonhosted.org/packages/b7/33/5504b3a9a4464893c32f118a9cc045190a91637b119a9c881da1cf6b7a72/PyYAML-6.0.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:936d68689298c36b53b29f23c6dbb74de12b4ac12ca6cfe0e047bedceea56180", size = 695802 }, + { url = "https://files.pythonhosted.org/packages/5c/20/8347dcabd41ef3a3cdc4f7b7a2aff3d06598c8779faa189cdbf878b626a4/PyYAML-6.0.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:23502f431948090f597378482b4812b0caae32c22213aecf3b55325e049a6c68", size = 720527 }, + { url = "https://files.pythonhosted.org/packages/be/aa/5afe99233fb360d0ff37377145a949ae258aaab831bde4792b32650a4378/PyYAML-6.0.2-cp310-cp310-win32.whl", hash = "sha256:2e99c6826ffa974fe6e27cdb5ed0021786b03fc98e5ee3c5bfe1fd5015f42b99", size = 144052 }, + { url = "https://files.pythonhosted.org/packages/b5/84/0fa4b06f6d6c958d207620fc60005e241ecedceee58931bb20138e1e5776/PyYAML-6.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:a4d3091415f010369ae4ed1fc6b79def9416358877534caf6a0fdd2146c87a3e", size = 161774 }, + { url = "https://files.pythonhosted.org/packages/f8/aa/7af4e81f7acba21a4c6be026da38fd2b872ca46226673c89a758ebdc4fd2/PyYAML-6.0.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:cc1c1159b3d456576af7a3e4d1ba7e6924cb39de8f67111c735f6fc832082774", size = 184612 }, + { url = "https://files.pythonhosted.org/packages/8b/62/b9faa998fd185f65c1371643678e4d58254add437edb764a08c5a98fb986/PyYAML-6.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:1e2120ef853f59c7419231f3bf4e7021f1b936f6ebd222406c3b60212205d2ee", size = 172040 }, + { url = "https://files.pythonhosted.org/packages/ad/0c/c804f5f922a9a6563bab712d8dcc70251e8af811fce4524d57c2c0fd49a4/PyYAML-6.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5d225db5a45f21e78dd9358e58a98702a0302f2659a3c6cd320564b75b86f47c", size = 736829 }, + { url = "https://files.pythonhosted.org/packages/51/16/6af8d6a6b210c8e54f1406a6b9481febf9c64a3109c541567e35a49aa2e7/PyYAML-6.0.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5ac9328ec4831237bec75defaf839f7d4564be1e6b25ac710bd1a96321cc8317", size = 764167 }, + { url = "https://files.pythonhosted.org/packages/75/e4/2c27590dfc9992f73aabbeb9241ae20220bd9452df27483b6e56d3975cc5/PyYAML-6.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3ad2a3decf9aaba3d29c8f537ac4b243e36bef957511b4766cb0057d32b0be85", size = 762952 }, + { url = "https://files.pythonhosted.org/packages/9b/97/ecc1abf4a823f5ac61941a9c00fe501b02ac3ab0e373c3857f7d4b83e2b6/PyYAML-6.0.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:ff3824dc5261f50c9b0dfb3be22b4567a6f938ccce4587b38952d85fd9e9afe4", size = 735301 }, + { url = "https://files.pythonhosted.org/packages/45/73/0f49dacd6e82c9430e46f4a027baa4ca205e8b0a9dce1397f44edc23559d/PyYAML-6.0.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:797b4f722ffa07cc8d62053e4cff1486fa6dc094105d13fea7b1de7d8bf71c9e", size = 756638 }, + { url = "https://files.pythonhosted.org/packages/22/5f/956f0f9fc65223a58fbc14459bf34b4cc48dec52e00535c79b8db361aabd/PyYAML-6.0.2-cp311-cp311-win32.whl", hash = "sha256:11d8f3dd2b9c1207dcaf2ee0bbbfd5991f571186ec9cc78427ba5bd32afae4b5", size = 143850 }, + { url = "https://files.pythonhosted.org/packages/ed/23/8da0bbe2ab9dcdd11f4f4557ccaf95c10b9811b13ecced089d43ce59c3c8/PyYAML-6.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:e10ce637b18caea04431ce14fabcf5c64a1c61ec9c56b071a4b7ca131ca52d44", size = 161980 }, + { url = "https://files.pythonhosted.org/packages/86/0c/c581167fc46d6d6d7ddcfb8c843a4de25bdd27e4466938109ca68492292c/PyYAML-6.0.2-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:c70c95198c015b85feafc136515252a261a84561b7b1d51e3384e0655ddf25ab", size = 183873 }, + { url = "https://files.pythonhosted.org/packages/a8/0c/38374f5bb272c051e2a69281d71cba6fdb983413e6758b84482905e29a5d/PyYAML-6.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ce826d6ef20b1bc864f0a68340c8b3287705cae2f8b4b1d932177dcc76721725", size = 173302 }, + { url = "https://files.pythonhosted.org/packages/c3/93/9916574aa8c00aa06bbac729972eb1071d002b8e158bd0e83a3b9a20a1f7/PyYAML-6.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f71ea527786de97d1a0cc0eacd1defc0985dcf6b3f17bb77dcfc8c34bec4dc5", size = 739154 }, + { url = "https://files.pythonhosted.org/packages/95/0f/b8938f1cbd09739c6da569d172531567dbcc9789e0029aa070856f123984/PyYAML-6.0.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9b22676e8097e9e22e36d6b7bda33190d0d400f345f23d4065d48f4ca7ae0425", size = 766223 }, + { url = "https://files.pythonhosted.org/packages/b9/2b/614b4752f2e127db5cc206abc23a8c19678e92b23c3db30fc86ab731d3bd/PyYAML-6.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:80bab7bfc629882493af4aa31a4cfa43a4c57c83813253626916b8c7ada83476", size = 767542 }, + { url = "https://files.pythonhosted.org/packages/d4/00/dd137d5bcc7efea1836d6264f049359861cf548469d18da90cd8216cf05f/PyYAML-6.0.2-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:0833f8694549e586547b576dcfaba4a6b55b9e96098b36cdc7ebefe667dfed48", size = 731164 }, + { url = "https://files.pythonhosted.org/packages/c9/1f/4f998c900485e5c0ef43838363ba4a9723ac0ad73a9dc42068b12aaba4e4/PyYAML-6.0.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8b9c7197f7cb2738065c481a0461e50ad02f18c78cd75775628afb4d7137fb3b", size = 756611 }, + { url = "https://files.pythonhosted.org/packages/df/d1/f5a275fdb252768b7a11ec63585bc38d0e87c9e05668a139fea92b80634c/PyYAML-6.0.2-cp312-cp312-win32.whl", hash = "sha256:ef6107725bd54b262d6dedcc2af448a266975032bc85ef0172c5f059da6325b4", size = 140591 }, + { url = "https://files.pythonhosted.org/packages/0c/e8/4f648c598b17c3d06e8753d7d13d57542b30d56e6c2dedf9c331ae56312e/PyYAML-6.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:7e7401d0de89a9a855c839bc697c079a4af81cf878373abd7dc625847d25cbd8", size = 156338 }, + { url = "https://files.pythonhosted.org/packages/ef/e3/3af305b830494fa85d95f6d95ef7fa73f2ee1cc8ef5b495c7c3269fb835f/PyYAML-6.0.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:efdca5630322a10774e8e98e1af481aad470dd62c3170801852d752aa7a783ba", size = 181309 }, + { url = "https://files.pythonhosted.org/packages/45/9f/3b1c20a0b7a3200524eb0076cc027a970d320bd3a6592873c85c92a08731/PyYAML-6.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:50187695423ffe49e2deacb8cd10510bc361faac997de9efef88badc3bb9e2d1", size = 171679 }, + { url = "https://files.pythonhosted.org/packages/7c/9a/337322f27005c33bcb656c655fa78325b730324c78620e8328ae28b64d0c/PyYAML-6.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0ffe8360bab4910ef1b9e87fb812d8bc0a308b0d0eef8c8f44e0254ab3b07133", size = 733428 }, + { url = "https://files.pythonhosted.org/packages/a3/69/864fbe19e6c18ea3cc196cbe5d392175b4cf3d5d0ac1403ec3f2d237ebb5/PyYAML-6.0.2-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:17e311b6c678207928d649faa7cb0d7b4c26a0ba73d41e99c4fff6b6c3276484", size = 763361 }, + { url = "https://files.pythonhosted.org/packages/04/24/b7721e4845c2f162d26f50521b825fb061bc0a5afcf9a386840f23ea19fa/PyYAML-6.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:70b189594dbe54f75ab3a1acec5f1e3faa7e8cf2f1e08d9b561cb41b845f69d5", size = 759523 }, + { url = "https://files.pythonhosted.org/packages/2b/b2/e3234f59ba06559c6ff63c4e10baea10e5e7df868092bf9ab40e5b9c56b6/PyYAML-6.0.2-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:41e4e3953a79407c794916fa277a82531dd93aad34e29c2a514c2c0c5fe971cc", size = 726660 }, + { url = "https://files.pythonhosted.org/packages/fe/0f/25911a9f080464c59fab9027482f822b86bf0608957a5fcc6eaac85aa515/PyYAML-6.0.2-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:68ccc6023a3400877818152ad9a1033e3db8625d899c72eacb5a668902e4d652", size = 751597 }, + { url = "https://files.pythonhosted.org/packages/14/0d/e2c3b43bbce3cf6bd97c840b46088a3031085179e596d4929729d8d68270/PyYAML-6.0.2-cp313-cp313-win32.whl", hash = "sha256:bc2fa7c6b47d6bc618dd7fb02ef6fdedb1090ec036abab80d4681424b84c1183", size = 140527 }, + { url = "https://files.pythonhosted.org/packages/fa/de/02b54f42487e3d3c6efb3f89428677074ca7bf43aae402517bc7cca949f3/PyYAML-6.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:8388ee1976c416731879ac16da0aff3f63b286ffdd57cdeb95f3f2e085687563", size = 156446 }, + { url = "https://files.pythonhosted.org/packages/65/d8/b7a1db13636d7fb7d4ff431593c510c8b8fca920ade06ca8ef20015493c5/PyYAML-6.0.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:688ba32a1cffef67fd2e9398a2efebaea461578b0923624778664cc1c914db5d", size = 184777 }, + { url = "https://files.pythonhosted.org/packages/0a/02/6ec546cd45143fdf9840b2c6be8d875116a64076218b61d68e12548e5839/PyYAML-6.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a8786accb172bd8afb8be14490a16625cbc387036876ab6ba70912730faf8e1f", size = 172318 }, + { url = "https://files.pythonhosted.org/packages/0e/9a/8cc68be846c972bda34f6c2a93abb644fb2476f4dcc924d52175786932c9/PyYAML-6.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d8e03406cac8513435335dbab54c0d385e4a49e4945d2909a581c83647ca0290", size = 720891 }, + { url = "https://files.pythonhosted.org/packages/e9/6c/6e1b7f40181bc4805e2e07f4abc10a88ce4648e7e95ff1abe4ae4014a9b2/PyYAML-6.0.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f753120cb8181e736c57ef7636e83f31b9c0d1722c516f7e86cf15b7aa57ff12", size = 722614 }, + { url = "https://files.pythonhosted.org/packages/3d/32/e7bd8535d22ea2874cef6a81021ba019474ace0d13a4819c2a4bce79bd6a/PyYAML-6.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b1fdb9dc17f5a7677423d508ab4f243a726dea51fa5e70992e59a7411c89d19", size = 737360 }, + { url = "https://files.pythonhosted.org/packages/d7/12/7322c1e30b9be969670b672573d45479edef72c9a0deac3bb2868f5d7469/PyYAML-6.0.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:0b69e4ce7a131fe56b7e4d770c67429700908fc0752af059838b1cfb41960e4e", size = 699006 }, + { url = "https://files.pythonhosted.org/packages/82/72/04fcad41ca56491995076630c3ec1e834be241664c0c09a64c9a2589b507/PyYAML-6.0.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:a9f8c2e67970f13b16084e04f134610fd1d374bf477b17ec1599185cf611d725", size = 723577 }, + { url = "https://files.pythonhosted.org/packages/ed/5e/46168b1f2757f1fcd442bc3029cd8767d88a98c9c05770d8b420948743bb/PyYAML-6.0.2-cp39-cp39-win32.whl", hash = "sha256:6395c297d42274772abc367baaa79683958044e5d3835486c16da75d2a694631", size = 144593 }, + { url = "https://files.pythonhosted.org/packages/19/87/5124b1c1f2412bb95c59ec481eaf936cd32f0fe2a7b16b97b81c4c017a6a/PyYAML-6.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:39693e1f8320ae4f43943590b49779ffb98acb81f788220ea932a6b6c51004d8", size = 162312 }, +] + +[[package]] +name = "readme-renderer" +version = "44.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "docutils" }, + { name = "nh3" }, + { name = "pygments" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/5a/a9/104ec9234c8448c4379768221ea6df01260cd6c2ce13182d4eac531c8342/readme_renderer-44.0.tar.gz", hash = "sha256:8712034eabbfa6805cacf1402b4eeb2a73028f72d1166d6f5cb7f9c047c5d1e1", size = 32056 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e1/67/921ec3024056483db83953ae8e48079ad62b92db7880013ca77632921dd0/readme_renderer-44.0-py3-none-any.whl", hash = "sha256:2fbca89b81a08526aadf1357a8c2ae889ec05fb03f5da67f9769c9a592166151", size = 13310 }, +] + +[[package]] +name = "referencing" +version = "0.35.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "attrs" }, + { name = "rpds-py" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/99/5b/73ca1f8e72fff6fa52119dbd185f73a907b1989428917b24cff660129b6d/referencing-0.35.1.tar.gz", hash = "sha256:25b42124a6c8b632a425174f24087783efb348a6f1e0008e63cd4466fedf703c", size = 62991 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b7/59/2056f61236782a2c86b33906c025d4f4a0b17be0161b63b70fd9e8775d36/referencing-0.35.1-py3-none-any.whl", hash = "sha256:eda6d3234d62814d1c64e305c1331c9a3a6132da475ab6382eaa997b21ee75de", size = 26684 }, +] + +[[package]] +name = "requests" +version = "2.32.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "certifi" }, + { name = "charset-normalizer" }, + { name = "idna" }, + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/63/70/2bf7780ad2d390a8d301ad0b550f1581eadbd9a20f896afe06353c2a2913/requests-2.32.3.tar.gz", hash = "sha256:55365417734eb18255590a9ff9eb97e9e1da868d4ccd6402399eaf68af20a760", size = 131218 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f9/9b/335f9764261e915ed497fcdeb11df5dfd6f7bf257d4a6a2a686d80da4d54/requests-2.32.3-py3-none-any.whl", hash = "sha256:70761cfe03c773ceb22aa2f671b4757976145175cdfca038c02654d061d6dcc6", size = 64928 }, +] + +[[package]] +name = "requests-oauthlib" +version = "2.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "oauthlib" }, + { name = "requests" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/42/f2/05f29bc3913aea15eb670be136045bf5c5bbf4b99ecb839da9b422bb2c85/requests-oauthlib-2.0.0.tar.gz", hash = "sha256:b3dffaebd884d8cd778494369603a9e7b58d29111bf6b41bdc2dcd87203af4e9", size = 55650 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3b/5d/63d4ae3b9daea098d5d6f5da83984853c1bbacd5dc826764b249fe119d24/requests_oauthlib-2.0.0-py2.py3-none-any.whl", hash = "sha256:7dd8a5c40426b779b0868c404bdef9768deccf22749cde15852df527e6269b36", size = 24179 }, +] + +[[package]] +name = "requests-toolbelt" +version = "1.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "requests" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/f3/61/d7545dafb7ac2230c70d38d31cbfe4cc64f7144dc41f6e4e4b78ecd9f5bb/requests-toolbelt-1.0.0.tar.gz", hash = "sha256:7681a0a3d047012b5bdc0ee37d7f8f07ebe76ab08caeccfc3921ce23c88d5bc6", size = 206888 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3f/51/d4db610ef29373b879047326cbf6fa98b6c1969d6f6dc423279de2b1be2c/requests_toolbelt-1.0.0-py2.py3-none-any.whl", hash = "sha256:cccfdd665f0a24fcf4726e690f65639d272bb0637b9b92dfd91a5568ccf6bd06", size = 54481 }, +] + +[[package]] +name = "rfc3986" +version = "2.0.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/85/40/1520d68bfa07ab5a6f065a186815fb6610c86fe957bc065754e47f7b0840/rfc3986-2.0.0.tar.gz", hash = "sha256:97aacf9dbd4bfd829baad6e6309fa6573aaf1be3f6fa735c8ab05e46cecb261c", size = 49026 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ff/9a/9afaade874b2fa6c752c36f1548f718b5b83af81ed9b76628329dab81c1b/rfc3986-2.0.0-py2.py3-none-any.whl", hash = "sha256:50b1502b60e289cb37883f3dfd34532b8873c7de9f49bb546641ce9cbd256ebd", size = 31326 }, +] + +[[package]] +name = "rich" +version = "13.9.4" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "markdown-it-py" }, + { name = "pygments" }, + { name = "typing-extensions", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ab/3a/0316b28d0761c6734d6bc14e770d85506c986c85ffb239e688eeaab2c2bc/rich-13.9.4.tar.gz", hash = "sha256:439594978a49a09530cff7ebc4b5c7103ef57baf48d5ea3184f21d9a2befa098", size = 223149 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/19/71/39c7c0d87f8d4e6c020a393182060eaefeeae6c01dab6a84ec346f2567df/rich-13.9.4-py3-none-any.whl", hash = "sha256:6049d5e6ec054bf2779ab3358186963bac2ea89175919d699e378b99738c2a90", size = 242424 }, +] + +[[package]] +name = "rich-click" +version = "1.8.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "click" }, + { name = "rich" }, + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3a/a9/a1f1af87e83832d794342fbc09c96cc7cd6798b8dfb8adfbe6ccbef8d70c/rich_click-1.8.3.tar.gz", hash = "sha256:6d75bdfa7aa9ed2c467789a0688bc6da23fbe3a143e19aa6ad3f8bac113d2ab3", size = 38209 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c6/ea/5a0c5a8e6532e971983d1b0fc99268eb66a10f489da35d9022ce01044191/rich_click-1.8.3-py3-none-any.whl", hash = "sha256:636d9c040d31c5eee242201b5bf4f2d358bfae4db14bb22ec1cafa717cfd02cd", size = 35032 }, +] + +[[package]] +name = "rpds-py" +version = "0.20.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/25/cb/8e919951f55d109d658f81c9b49d0cc3b48637c50792c5d2e77032b8c5da/rpds_py-0.20.1.tar.gz", hash = "sha256:e1791c4aabd117653530dccd24108fa03cc6baf21f58b950d0a73c3b3b29a350", size = 25931 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ae/0e/d7e7e9280988a7bc56fd326042baca27f4f55fad27dc8aa64e5e0e894e5d/rpds_py-0.20.1-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:a649dfd735fff086e8a9d0503a9f0c7d01b7912a333c7ae77e1515c08c146dad", size = 327335 }, + { url = "https://files.pythonhosted.org/packages/4c/72/027185f213d53ae66765c575229829b202fbacf3d55fe2bd9ff4e29bb157/rpds_py-0.20.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f16bc1334853e91ddaaa1217045dd7be166170beec337576818461268a3de67f", size = 318250 }, + { url = "https://files.pythonhosted.org/packages/2b/e7/b4eb3e6ff541c83d3b46f45f855547e412ab60c45bef64520fafb00b9b42/rpds_py-0.20.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:14511a539afee6f9ab492b543060c7491c99924314977a55c98bfa2ee29ce78c", size = 361206 }, + { url = "https://files.pythonhosted.org/packages/e7/80/cb9a4b4cad31bcaa37f38dae7a8be861f767eb2ca4f07a146b5ffcfbee09/rpds_py-0.20.1-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3ccb8ac2d3c71cda472b75af42818981bdacf48d2e21c36331b50b4f16930163", size = 369921 }, + { url = "https://files.pythonhosted.org/packages/95/1b/463b11e7039e18f9e778568dbf7338c29bbc1f8996381115201c668eb8c8/rpds_py-0.20.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c142b88039b92e7e0cb2552e8967077e3179b22359e945574f5e2764c3953dcf", size = 403673 }, + { url = "https://files.pythonhosted.org/packages/86/98/1ef4028e9d5b76470bf7f8f2459be07ac5c9621270a2a5e093f8d8a8cc2c/rpds_py-0.20.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f19169781dddae7478a32301b499b2858bc52fc45a112955e798ee307e294977", size = 430267 }, + { url = "https://files.pythonhosted.org/packages/25/8e/41d7e3e6d3a4a6c94375020477705a3fbb6515717901ab8f94821cf0a0d9/rpds_py-0.20.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:13c56de6518e14b9bf6edde23c4c39dac5b48dcf04160ea7bce8fca8397cdf86", size = 360569 }, + { url = "https://files.pythonhosted.org/packages/4f/6a/8839340464d4e1bbfaf0482e9d9165a2309c2c17427e4dcb72ce3e5cc5d6/rpds_py-0.20.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:925d176a549f4832c6f69fa6026071294ab5910e82a0fe6c6228fce17b0706bd", size = 382584 }, + { url = "https://files.pythonhosted.org/packages/64/96/7a7f938d3796a6a3ec08ed0e8a5ecd436fbd516a3684ab1fa22d46d6f6cc/rpds_py-0.20.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:78f0b6877bfce7a3d1ff150391354a410c55d3cdce386f862926a4958ad5ab7e", size = 546560 }, + { url = "https://files.pythonhosted.org/packages/15/c7/19fb4f1247a3c90a99eca62909bf76ee988f9b663e47878a673d9854ec5c/rpds_py-0.20.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:3dd645e2b0dcb0fd05bf58e2e54c13875847687d0b71941ad2e757e5d89d4356", size = 549359 }, + { url = "https://files.pythonhosted.org/packages/d2/4c/445eb597a39a883368ea2f341dd6e48a9d9681b12ebf32f38a827b30529b/rpds_py-0.20.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:4f676e21db2f8c72ff0936f895271e7a700aa1f8d31b40e4e43442ba94973899", size = 527567 }, + { url = "https://files.pythonhosted.org/packages/4f/71/4c44643bffbcb37311fc7fe221bcf139c8d660bc78f746dd3a05741372c8/rpds_py-0.20.1-cp310-none-win32.whl", hash = "sha256:648386ddd1e19b4a6abab69139b002bc49ebf065b596119f8f37c38e9ecee8ff", size = 200412 }, + { url = "https://files.pythonhosted.org/packages/f4/33/9d0529d74099e090ec9ab15eb0a049c56cca599eaaca71bfedbdbca656a9/rpds_py-0.20.1-cp310-none-win_amd64.whl", hash = "sha256:d9ecb51120de61e4604650666d1f2b68444d46ae18fd492245a08f53ad2b7711", size = 218563 }, + { url = "https://files.pythonhosted.org/packages/a0/2e/a6ded84019a05b8f23e0fe6a632f62ae438a8c5e5932d3dfc90c73418414/rpds_py-0.20.1-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:762703bdd2b30983c1d9e62b4c88664df4a8a4d5ec0e9253b0231171f18f6d75", size = 327194 }, + { url = "https://files.pythonhosted.org/packages/68/11/d3f84c69de2b2086be3d6bd5e9d172825c096b13842ab7e5f8f39f06035b/rpds_py-0.20.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:0b581f47257a9fce535c4567782a8976002d6b8afa2c39ff616edf87cbeff712", size = 318126 }, + { url = "https://files.pythonhosted.org/packages/18/c0/13f1bce9c901511e5e4c0b77a99dbb946bb9a177ca88c6b480e9cb53e304/rpds_py-0.20.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:842c19a6ce894493563c3bd00d81d5100e8e57d70209e84d5491940fdb8b9e3a", size = 361119 }, + { url = "https://files.pythonhosted.org/packages/06/31/3bd721575671f22a37476c2d7b9e34bfa5185bdcee09f7fedde3b29f3adb/rpds_py-0.20.1-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:42cbde7789f5c0bcd6816cb29808e36c01b960fb5d29f11e052215aa85497c93", size = 369532 }, + { url = "https://files.pythonhosted.org/packages/20/22/3eeb0385f33251b4fd0f728e6a3801dc8acc05e714eb7867cefe635bf4ab/rpds_py-0.20.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6c8e9340ce5a52f95fa7d3b552b35c7e8f3874d74a03a8a69279fd5fca5dc751", size = 403703 }, + { url = "https://files.pythonhosted.org/packages/10/e1/8dde6174e7ac5b9acd3269afca2e17719bc7e5088c68f44874d2ad9e4560/rpds_py-0.20.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8ba6f89cac95c0900d932c9efb7f0fb6ca47f6687feec41abcb1bd5e2bd45535", size = 429868 }, + { url = "https://files.pythonhosted.org/packages/19/51/a3cc1a5238acfc2582033e8934d034301f9d4931b9bf7c7ccfabc4ca0880/rpds_py-0.20.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4a916087371afd9648e1962e67403c53f9c49ca47b9680adbeef79da3a7811b0", size = 360539 }, + { url = "https://files.pythonhosted.org/packages/cd/8c/3c87471a44bd4114e2b0aec90f298f6caaac4e8db6af904d5dd2279f5c61/rpds_py-0.20.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:200a23239781f46149e6a415f1e870c5ef1e712939fe8fa63035cd053ac2638e", size = 382467 }, + { url = "https://files.pythonhosted.org/packages/d0/9b/95073fe3e0f130e6d561e106818b6568ef1f2df3352e7f162ab912da837c/rpds_py-0.20.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:58b1d5dd591973d426cbb2da5e27ba0339209832b2f3315928c9790e13f159e8", size = 546669 }, + { url = "https://files.pythonhosted.org/packages/de/4c/7ab3669e02bb06fedebcfd64d361b7168ba39dfdf385e4109440f2e7927b/rpds_py-0.20.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:6b73c67850ca7cae0f6c56f71e356d7e9fa25958d3e18a64927c2d930859b8e4", size = 549304 }, + { url = "https://files.pythonhosted.org/packages/f1/e8/ad5da336cd42adbdafe0ecd40dcecdae01fd3d703c621c7637615a008d3a/rpds_py-0.20.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:d8761c3c891cc51e90bc9926d6d2f59b27beaf86c74622c8979380a29cc23ac3", size = 527637 }, + { url = "https://files.pythonhosted.org/packages/02/f1/1b47b9e5b941c2659c9b7e4ef41b6f07385a6500c638fa10c066e4616ecb/rpds_py-0.20.1-cp311-none-win32.whl", hash = "sha256:cd945871335a639275eee904caef90041568ce3b42f402c6959b460d25ae8732", size = 200488 }, + { url = "https://files.pythonhosted.org/packages/85/f6/c751c1adfa31610055acfa1cc667cf2c2d7011a73070679c448cf5856905/rpds_py-0.20.1-cp311-none-win_amd64.whl", hash = "sha256:7e21b7031e17c6b0e445f42ccc77f79a97e2687023c5746bfb7a9e45e0921b84", size = 218475 }, + { url = "https://files.pythonhosted.org/packages/e7/10/4e8dcc08b58a548098dbcee67a4888751a25be7a6dde0a83d4300df48bfa/rpds_py-0.20.1-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:36785be22066966a27348444b40389f8444671630063edfb1a2eb04318721e17", size = 329749 }, + { url = "https://files.pythonhosted.org/packages/d2/e4/61144f3790e12fd89e6153d77f7915ad26779735fef8ee9c099cba6dfb4a/rpds_py-0.20.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:142c0a5124d9bd0e2976089484af5c74f47bd3298f2ed651ef54ea728d2ea42c", size = 321032 }, + { url = "https://files.pythonhosted.org/packages/fa/e0/99205aabbf3be29ef6c58ef9b08feed51ba6532fdd47461245cb58dd9897/rpds_py-0.20.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dbddc10776ca7ebf2a299c41a4dde8ea0d8e3547bfd731cb87af2e8f5bf8962d", size = 363931 }, + { url = "https://files.pythonhosted.org/packages/ac/bd/bce2dddb518b13a7e77eed4be234c9af0c9c6d403d01c5e6ae8eb447ab62/rpds_py-0.20.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:15a842bb369e00295392e7ce192de9dcbf136954614124a667f9f9f17d6a216f", size = 373343 }, + { url = "https://files.pythonhosted.org/packages/43/15/112b7c553066cb91264691ba7fb119579c440a0ae889da222fa6fc0d411a/rpds_py-0.20.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be5ef2f1fc586a7372bfc355986226484e06d1dc4f9402539872c8bb99e34b01", size = 406304 }, + { url = "https://files.pythonhosted.org/packages/af/8d/2da52aef8ae5494a382b0c0025ba5b68f2952db0f2a4c7534580e8ca83cc/rpds_py-0.20.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dbcf360c9e3399b056a238523146ea77eeb2a596ce263b8814c900263e46031a", size = 423022 }, + { url = "https://files.pythonhosted.org/packages/c8/1b/f23015cb293927c93bdb4b94a48bfe77ad9d57359c75db51f0ff0cf482ff/rpds_py-0.20.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ecd27a66740ffd621d20b9a2f2b5ee4129a56e27bfb9458a3bcc2e45794c96cb", size = 364937 }, + { url = "https://files.pythonhosted.org/packages/7b/8b/6da8636b2ea2e2f709e56656e663b6a71ecd9a9f9d9dc21488aade122026/rpds_py-0.20.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d0b937b2a1988f184a3e9e577adaa8aede21ec0b38320d6009e02bd026db04fa", size = 386301 }, + { url = "https://files.pythonhosted.org/packages/20/af/2ae192797bffd0d6d558145b5a36e7245346ff3e44f6ddcb82f0eb8512d4/rpds_py-0.20.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6889469bfdc1eddf489729b471303739bf04555bb151fe8875931f8564309afc", size = 549452 }, + { url = "https://files.pythonhosted.org/packages/07/dd/9f6520712a5108cd7d407c9db44a3d59011b385c58e320d58ebf67757a9e/rpds_py-0.20.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:19b73643c802f4eaf13d97f7855d0fb527fbc92ab7013c4ad0e13a6ae0ed23bd", size = 554370 }, + { url = "https://files.pythonhosted.org/packages/5e/0e/b1bdc7ea0db0946d640ab8965146099093391bb5d265832994c47461e3c5/rpds_py-0.20.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:3c6afcf2338e7f374e8edc765c79fbcb4061d02b15dd5f8f314a4af2bdc7feb5", size = 530940 }, + { url = "https://files.pythonhosted.org/packages/ae/d3/ffe907084299484fab60a7955f7c0e8a295c04249090218c59437010f9f4/rpds_py-0.20.1-cp312-none-win32.whl", hash = "sha256:dc73505153798c6f74854aba69cc75953888cf9866465196889c7cdd351e720c", size = 203164 }, + { url = "https://files.pythonhosted.org/packages/1f/ba/9cbb57423c4bfbd81c473913bebaed151ad4158ee2590a4e4b3e70238b48/rpds_py-0.20.1-cp312-none-win_amd64.whl", hash = "sha256:8bbe951244a838a51289ee53a6bae3a07f26d4e179b96fc7ddd3301caf0518eb", size = 220750 }, + { url = "https://files.pythonhosted.org/packages/b5/01/fee2e1d1274c92fff04aa47d805a28d62c2aa971d1f49f5baea1c6e670d9/rpds_py-0.20.1-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:6ca91093a4a8da4afae7fe6a222c3b53ee4eef433ebfee4d54978a103435159e", size = 329359 }, + { url = "https://files.pythonhosted.org/packages/b0/cf/4aeffb02b7090029d7aeecbffb9a10e1c80f6f56d7e9a30e15481dc4099c/rpds_py-0.20.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:b9c2fe36d1f758b28121bef29ed1dee9b7a2453e997528e7d1ac99b94892527c", size = 320543 }, + { url = "https://files.pythonhosted.org/packages/17/69/85cf3429e9ccda684ba63ff36b5866d5f9451e921cc99819341e19880334/rpds_py-0.20.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f009c69bc8c53db5dfab72ac760895dc1f2bc1b62ab7408b253c8d1ec52459fc", size = 363107 }, + { url = "https://files.pythonhosted.org/packages/ef/de/7df88dea9c3eeb832196d23b41f0f6fc5f9a2ee9b2080bbb1db8731ead9c/rpds_py-0.20.1-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:6740a3e8d43a32629bb9b009017ea5b9e713b7210ba48ac8d4cb6d99d86c8ee8", size = 372027 }, + { url = "https://files.pythonhosted.org/packages/d1/b8/88675399d2038580743c570a809c43a900e7090edc6553f8ffb66b23c965/rpds_py-0.20.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:32b922e13d4c0080d03e7b62991ad7f5007d9cd74e239c4b16bc85ae8b70252d", size = 405031 }, + { url = "https://files.pythonhosted.org/packages/e1/aa/cca639f6d17caf00bab51bdc70fcc0bdda3063e5662665c4fdf60443c474/rpds_py-0.20.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fe00a9057d100e69b4ae4a094203a708d65b0f345ed546fdef86498bf5390982", size = 422271 }, + { url = "https://files.pythonhosted.org/packages/c4/07/bf8a949d2ec4626c285579c9d6b356c692325f1a4126e947736b416e1fc4/rpds_py-0.20.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:49fe9b04b6fa685bd39237d45fad89ba19e9163a1ccaa16611a812e682913496", size = 363625 }, + { url = "https://files.pythonhosted.org/packages/11/f0/06675c6a58d6ce34547879138810eb9aab0c10e5607ea6c2e4dc56b703c8/rpds_py-0.20.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:aa7ac11e294304e615b43f8c441fee5d40094275ed7311f3420d805fde9b07b4", size = 385906 }, + { url = "https://files.pythonhosted.org/packages/bf/ac/2d1f50374eb8e41030fad4e87f81751e1c39e3b5d4bee8c5618830d8a6ac/rpds_py-0.20.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6aa97af1558a9bef4025f8f5d8c60d712e0a3b13a2fe875511defc6ee77a1ab7", size = 549021 }, + { url = "https://files.pythonhosted.org/packages/f7/d4/a7d70a7cc71df772eeadf4bce05e32e780a9fe44a511a5b091c7a85cb767/rpds_py-0.20.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:483b29f6f7ffa6af845107d4efe2e3fa8fb2693de8657bc1849f674296ff6a5a", size = 553800 }, + { url = "https://files.pythonhosted.org/packages/87/81/dc30bc449ccba63ad23a0f6633486d4e0e6955f45f3715a130dacabd6ad0/rpds_py-0.20.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:37fe0f12aebb6a0e3e17bb4cd356b1286d2d18d2e93b2d39fe647138458b4bcb", size = 531076 }, + { url = "https://files.pythonhosted.org/packages/50/80/fb62ab48f3b5cfe704ead6ad372da1922ddaa76397055e02eb507054c979/rpds_py-0.20.1-cp313-none-win32.whl", hash = "sha256:a624cc00ef2158e04188df5e3016385b9353638139a06fb77057b3498f794782", size = 202804 }, + { url = "https://files.pythonhosted.org/packages/d9/30/a3391e76d0b3313f33bdedd394a519decae3a953d2943e3dabf80ae32447/rpds_py-0.20.1-cp313-none-win_amd64.whl", hash = "sha256:b71b8666eeea69d6363248822078c075bac6ed135faa9216aa85f295ff009b1e", size = 220502 }, + { url = "https://files.pythonhosted.org/packages/d6/87/e7e0fcbfdc0d0e261534bcc885f6ae6253095b972e32f8b8b1278c78a2a9/rpds_py-0.20.1-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:b41b6321805c472f66990c2849e152aff7bc359eb92f781e3f606609eac877ad", size = 327867 }, + { url = "https://files.pythonhosted.org/packages/93/a0/17836b7961fc82586e9b818abdee2a27e2e605a602bb8c0d43f02092f8c2/rpds_py-0.20.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0a90c373ea2975519b58dece25853dbcb9779b05cc46b4819cb1917e3b3215b6", size = 318893 }, + { url = "https://files.pythonhosted.org/packages/dc/03/deb81d8ea3a8b974e7b03cfe8c8c26616ef8f4980dd430d8dd0a2f1b4d8e/rpds_py-0.20.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:16d4477bcb9fbbd7b5b0e4a5d9b493e42026c0bf1f06f723a9353f5153e75d30", size = 361664 }, + { url = "https://files.pythonhosted.org/packages/16/49/d9938603731745c7b6babff97ca61ff3eb4619e7128b5ab0111ad4e91d6d/rpds_py-0.20.1-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:84b8382a90539910b53a6307f7c35697bc7e6ffb25d9c1d4e998a13e842a5e83", size = 369796 }, + { url = "https://files.pythonhosted.org/packages/87/d2/480b36c69cdc373853401b6aab6a281cf60f6d72b1545d82c0d23d9dd77c/rpds_py-0.20.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4888e117dd41b9d34194d9e31631af70d3d526efc363085e3089ab1a62c32ed1", size = 403860 }, + { url = "https://files.pythonhosted.org/packages/31/7c/f6d909cb57761293308dbef14f1663d84376f2e231892a10aafc57b42037/rpds_py-0.20.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5265505b3d61a0f56618c9b941dc54dc334dc6e660f1592d112cd103d914a6db", size = 430793 }, + { url = "https://files.pythonhosted.org/packages/d4/62/c9bd294c4b5f84d9cc2c387b548ae53096ad7e71ac5b02b6310e9dc85aa4/rpds_py-0.20.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e75ba609dba23f2c95b776efb9dd3f0b78a76a151e96f96cc5b6b1b0004de66f", size = 360927 }, + { url = "https://files.pythonhosted.org/packages/c1/a7/15d927d83a44da8307a432b1cac06284b6657706d099a98cc99fec34ad51/rpds_py-0.20.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:1791ff70bc975b098fe6ecf04356a10e9e2bd7dc21fa7351c1742fdeb9b4966f", size = 382660 }, + { url = "https://files.pythonhosted.org/packages/4c/28/0630719c18456238bb07d59c4302fed50a13aa8035ec23dbfa80d116f9bc/rpds_py-0.20.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:d126b52e4a473d40232ec2052a8b232270ed1f8c9571aaf33f73a14cc298c24f", size = 546888 }, + { url = "https://files.pythonhosted.org/packages/b9/75/3c9bda11b9c15d680b315f898af23825159314d4b56568f24b53ace8afcd/rpds_py-0.20.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:c14937af98c4cc362a1d4374806204dd51b1e12dded1ae30645c298e5a5c4cb1", size = 550088 }, + { url = "https://files.pythonhosted.org/packages/70/f1/8fe7d04c194218171220a412057429defa9e2da785de0777c4d39309337e/rpds_py-0.20.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:3d089d0b88996df627693639d123c8158cff41c0651f646cd8fd292c7da90eaf", size = 528270 }, + { url = "https://files.pythonhosted.org/packages/d6/62/41b0020f4b00af042b008e679dbe25a2f5bce655139a81f8b812f9068e52/rpds_py-0.20.1-cp39-none-win32.whl", hash = "sha256:653647b8838cf83b2e7e6a0364f49af96deec64d2a6578324db58380cff82aca", size = 200658 }, + { url = "https://files.pythonhosted.org/packages/05/01/e64bb8889f2dcc951e53de33d8b8070456397ae4e10edc35e6cb9908f5c8/rpds_py-0.20.1-cp39-none-win_amd64.whl", hash = "sha256:fa41a64ac5b08b292906e248549ab48b69c5428f3987b09689ab2441f267d04d", size = 218883 }, + { url = "https://files.pythonhosted.org/packages/b6/fa/7959429e69569d0f6e7d27f80451402da0409349dd2b07f6bcbdd5fad2d3/rpds_py-0.20.1-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:7a07ced2b22f0cf0b55a6a510078174c31b6d8544f3bc00c2bcee52b3d613f74", size = 328209 }, + { url = "https://files.pythonhosted.org/packages/25/97/5dfdb091c30267ff404d2fd9e70c7a6d6ffc65ca77fffe9456e13b719066/rpds_py-0.20.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:68cb0a499f2c4a088fd2f521453e22ed3527154136a855c62e148b7883b99f9a", size = 319499 }, + { url = "https://files.pythonhosted.org/packages/7c/98/cf2608722400f5f9bb4c82aa5ac09026f3ac2ebea9d4059d3533589ed0b6/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa3060d885657abc549b2a0f8e1b79699290e5d83845141717c6c90c2df38311", size = 361795 }, + { url = "https://files.pythonhosted.org/packages/89/de/0e13dd43c785c60e63933e96fbddda0b019df6862f4d3019bb49c3861131/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:95f3b65d2392e1c5cec27cff08fdc0080270d5a1a4b2ea1d51d5f4a2620ff08d", size = 370604 }, + { url = "https://files.pythonhosted.org/packages/8a/fc/fe3c83c77f82b8059eeec4e998064913d66212b69b3653df48f58ad33d3d/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2cc3712a4b0b76a1d45a9302dd2f53ff339614b1c29603a911318f2357b04dd2", size = 404177 }, + { url = "https://files.pythonhosted.org/packages/94/30/5189518bfb80a41f664daf32b46645c7fbdcc89028a0f1bfa82e806e0fbb/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5d4eea0761e37485c9b81400437adb11c40e13ef513375bbd6973e34100aeb06", size = 430108 }, + { url = "https://files.pythonhosted.org/packages/67/0e/6f069feaff5c298375cd8c55e00ecd9bd79c792ce0893d39448dc0097857/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f5179583d7a6cdb981151dd349786cbc318bab54963a192692d945dd3f6435d", size = 361184 }, + { url = "https://files.pythonhosted.org/packages/27/9f/ce3e2ae36f392c3ef1988c06e9e0b4c74f64267dad7c223003c34da11adb/rpds_py-0.20.1-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:2fbb0ffc754490aff6dabbf28064be47f0f9ca0b9755976f945214965b3ace7e", size = 384140 }, + { url = "https://files.pythonhosted.org/packages/5f/d5/89d44504d0bc7a1135062cb520a17903ff002f458371b8d9160af3b71e52/rpds_py-0.20.1-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:a94e52537a0e0a85429eda9e49f272ada715506d3b2431f64b8a3e34eb5f3e75", size = 546589 }, + { url = "https://files.pythonhosted.org/packages/8f/8f/e1c2db4fcca3947d9a28ec9553700b4dc8038f0eff575f579e75885b0661/rpds_py-0.20.1-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:92b68b79c0da2a980b1c4197e56ac3dd0c8a149b4603747c4378914a68706979", size = 550059 }, + { url = "https://files.pythonhosted.org/packages/67/29/00a9e986df36721b5def82fff60995c1ee8827a7d909a6ec8929fb4cc668/rpds_py-0.20.1-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:93da1d3db08a827eda74356f9f58884adb254e59b6664f64cc04cdff2cc19b0d", size = 529131 }, + { url = "https://files.pythonhosted.org/packages/a3/32/95364440560ec476b19c6a2704259e710c223bf767632ebaa72cc2a1760f/rpds_py-0.20.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:754bbed1a4ca48479e9d4182a561d001bbf81543876cdded6f695ec3d465846b", size = 219677 }, + { url = "https://files.pythonhosted.org/packages/ed/bf/ad8492e972c90a3d48a38e2b5095c51a8399d5b57e83f2d5d1649490f72b/rpds_py-0.20.1-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:ca449520e7484534a2a44faf629362cae62b660601432d04c482283c47eaebab", size = 328046 }, + { url = "https://files.pythonhosted.org/packages/75/fd/84f42386165d6d555acb76c6d39c90b10c9dcf25116daf4f48a0a9d6867a/rpds_py-0.20.1-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:9c4cb04a16b0f199a8c9bf807269b2f63b7b5b11425e4a6bd44bd6961d28282c", size = 319306 }, + { url = "https://files.pythonhosted.org/packages/6c/8a/abcd5119a0573f9588ad71a3fde3c07ddd1d1393cfee15a6ba7495c256f1/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb63804105143c7e24cee7db89e37cb3f3941f8e80c4379a0b355c52a52b6780", size = 362558 }, + { url = "https://files.pythonhosted.org/packages/9d/65/1c2bb10afd4bd32800227a658ae9097bc1d08a4e5048a57a9bd2efdf6306/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:55cd1fa4ecfa6d9f14fbd97ac24803e6f73e897c738f771a9fe038f2f11ff07c", size = 370811 }, + { url = "https://files.pythonhosted.org/packages/6c/ee/f4bab2b9e51ced30351cfd210647885391463ae682028c79760e7db28e4e/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0f8f741b6292c86059ed175d80eefa80997125b7c478fb8769fd9ac8943a16c0", size = 404660 }, + { url = "https://files.pythonhosted.org/packages/48/0f/9d04d0939682f0c97be827fc51ff986555ffb573e6781bd5132441f0ce25/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0fc212779bf8411667234b3cdd34d53de6c2b8b8b958e1e12cb473a5f367c338", size = 430490 }, + { url = "https://files.pythonhosted.org/packages/0d/f2/e9b90fd8416d59941b6a12f2c2e1d898b63fd092f5a7a6f98236cb865764/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ad56edabcdb428c2e33bbf24f255fe2b43253b7d13a2cdbf05de955217313e6", size = 361448 }, + { url = "https://files.pythonhosted.org/packages/0b/83/1cc776dce7bedb17d6f4ea62eafccee8a57a4678f4fac414ab69fb9b6b0b/rpds_py-0.20.1-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:0a3a1e9ee9728b2c1734f65d6a1d376c6f2f6fdcc13bb007a08cc4b1ff576dc5", size = 383681 }, + { url = "https://files.pythonhosted.org/packages/17/5c/e0cdd6b0a8373fdef3667af2778dd9ff3abf1bbb9c7bd92c603c91440eb0/rpds_py-0.20.1-pp39-pypy39_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:e13de156137b7095442b288e72f33503a469aa1980ed856b43c353ac86390519", size = 546203 }, + { url = "https://files.pythonhosted.org/packages/1b/a8/81fc9cbc01e7ef6d10652aedc1de4e8473934773e2808ba49094e03575df/rpds_py-0.20.1-pp39-pypy39_pp73-musllinux_1_2_i686.whl", hash = "sha256:07f59760ef99f31422c49038964b31c4dfcfeb5d2384ebfc71058a7c9adae2d2", size = 549855 }, + { url = "https://files.pythonhosted.org/packages/b3/87/99648693d3c1bbce088119bc61ecaab62e5f9c713894edc604ffeca5ae88/rpds_py-0.20.1-pp39-pypy39_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:59240685e7da61fb78f65a9f07f8108e36a83317c53f7b276b4175dc44151684", size = 528625 }, + { url = "https://files.pythonhosted.org/packages/05/c3/10c68a08849f1fa45d205e54141fa75d316013e3d701ef01770ee1220bb8/rpds_py-0.20.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:83cba698cfb3c2c5a7c3c6bac12fe6c6a51aae69513726be6411076185a8b24a", size = 219991 }, +] + +[[package]] +name = "rsa" +version = "4.9" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pyasn1" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/aa/65/7d973b89c4d2351d7fb232c2e452547ddfa243e93131e7cfa766da627b52/rsa-4.9.tar.gz", hash = "sha256:e38464a49c6c85d7f1351b0126661487a7e0a14a50f1675ec50eb34d4f20ef21", size = 29711 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/49/97/fa78e3d2f65c02c8e1268b9aba606569fe97f6c8f7c2d74394553347c145/rsa-4.9-py3-none-any.whl", hash = "sha256:90260d9058e514786967344d0ef75fa8727eed8a7d2e43ce9f4bcf1b536174f7", size = 34315 }, +] + +[[package]] +name = "secretstorage" +version = "3.3.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cryptography" }, + { name = "jeepney" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/53/a4/f48c9d79cb507ed1373477dbceaba7401fd8a23af63b837fa61f1dcd3691/SecretStorage-3.3.3.tar.gz", hash = "sha256:2403533ef369eca6d2ba81718576c5e0f564d5cca1b58f73a8b23e7d4eeebd77", size = 19739 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/54/24/b4293291fa1dd830f353d2cb163295742fa87f179fcc8a20a306a81978b7/SecretStorage-3.3.3-py3-none-any.whl", hash = "sha256:f356e6628222568e3af06f2eba8df495efa13b3b63081dafd4f7d9a7b7bc9f99", size = 15221 }, +] + +[[package]] +name = "semver" +version = "3.0.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/41/6c/a536cc008f38fd83b3c1b98ce19ead13b746b5588c9a0cb9dd9f6ea434bc/semver-3.0.2.tar.gz", hash = "sha256:6253adb39c70f6e51afed2fa7152bcd414c411286088fb4b9effb133885ab4cc", size = 214988 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9a/77/0cc7a8a3bc7e53d07e8f47f147b92b0960e902b8254859f4aee5c4d7866b/semver-3.0.2-py3-none-any.whl", hash = "sha256:b1ea4686fe70b981f85359eda33199d60c53964284e0cfb4977d243e37cf4bf4", size = 17099 }, +] + +[[package]] +name = "shellingham" +version = "1.5.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/58/15/8b3609fd3830ef7b27b655beb4b4e9c62313a4e8da8c676e142cc210d58e/shellingham-1.5.4.tar.gz", hash = "sha256:8dbca0739d487e5bd35ab3ca4b36e11c4078f3a234bfce294b0a0291363404de", size = 10310 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e0/f9/0595336914c5619e5f28a1fb793285925a8cd4b432c9da0a987836c7f822/shellingham-1.5.4-py2.py3-none-any.whl", hash = "sha256:7ecfff8f2fd72616f7481040475a65b2bf8af90a56c89140852d1120324e8686", size = 9755 }, +] + +[[package]] +name = "smmap" +version = "5.0.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/88/04/b5bf6d21dc4041000ccba7eb17dd3055feb237e7ffc2c20d3fae3af62baa/smmap-5.0.1.tar.gz", hash = "sha256:dceeb6c0028fdb6734471eb07c0cd2aae706ccaecab45965ee83f11c8d3b1f62", size = 22291 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a7/a5/10f97f73544edcdef54409f1d839f6049a0d79df68adbc1ceb24d1aaca42/smmap-5.0.1-py3-none-any.whl", hash = "sha256:e6d8668fa5f93e706934a62d7b4db19c8d9eb8cf2adbb75ef1b675aa332b69da", size = 24282 }, +] + +[[package]] +name = "sniffio" +version = "1.3.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a2/87/a6771e1546d97e7e041b6ae58d80074f81b7d5121207425c964ddf5cfdbd/sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc", size = 20372 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e9/44/75a9c9421471a6c4805dbf2356f7c181a29c1879239abab1ea2cc8f38b40/sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2", size = 10235 }, +] + +[[package]] +name = "tabulate" +version = "0.9.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ec/fe/802052aecb21e3797b8f7902564ab6ea0d60ff8ca23952079064155d1ae1/tabulate-0.9.0.tar.gz", hash = "sha256:0095b12bf5966de529c0feb1fa08671671b3368eec77d7ef7ab114be2c068b3c", size = 81090 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/40/44/4a5f08c96eb108af5cb50b41f76142f0afa346dfa99d5296fe7202a11854/tabulate-0.9.0-py3-none-any.whl", hash = "sha256:024ca478df22e9340661486f85298cff5f6dcdba14f3813e8830015b9ed1948f", size = 35252 }, +] + +[[package]] +name = "tomli" +version = "2.0.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/35/b9/de2a5c0144d7d75a57ff355c0c24054f965b2dc3036456ae03a51ea6264b/tomli-2.0.2.tar.gz", hash = "sha256:d46d457a85337051c36524bc5349dd91b1877838e2979ac5ced3e710ed8a60ed", size = 16096 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/cf/db/ce8eda256fa131af12e0a76d481711abe4681b6923c27efb9a255c9e4594/tomli-2.0.2-py3-none-any.whl", hash = "sha256:2ebe24485c53d303f690b0ec092806a085f07af5a5aa1464f3931eec36caaa38", size = 13237 }, +] + +[[package]] +name = "tomli-w" +version = "1.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d4/19/b65f1a088ee23e37cdea415b357843eca8b1422a7b11a9eee6e35d4ec273/tomli_w-1.1.0.tar.gz", hash = "sha256:49e847a3a304d516a169a601184932ef0f6b61623fe680f836a2aa7128ed0d33", size = 6929 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c4/ac/ce90573ba446a9bbe65838ded066a805234d159b4446ae9f8ec5bbd36cbd/tomli_w-1.1.0-py3-none-any.whl", hash = "sha256:1403179c78193e3184bfaade390ddbd071cba48a32a2e62ba11aae47490c63f7", size = 6440 }, +] + +[[package]] +name = "tomlkit" +version = "0.13.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b1/09/a439bec5888f00a54b8b9f05fa94d7f901d6735ef4e55dcec9bc37b5d8fa/tomlkit-0.13.2.tar.gz", hash = "sha256:fff5fe59a87295b278abd31bec92c15d9bc4a06885ab12bcea52c71119392e79", size = 192885 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f9/b6/a447b5e4ec71e13871be01ba81f5dfc9d0af7e473da256ff46bc0e24026f/tomlkit-0.13.2-py3-none-any.whl", hash = "sha256:7a974427f6e119197f670fbbbeae7bef749a6c14e793db934baefc1b5f03efde", size = 37955 }, +] + +[[package]] +name = "trove-classifiers" +version = "2024.10.21.16" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/99/85/92c2667cf221b37648041ce9319427f92fa76cbec634aad844e67e284706/trove_classifiers-2024.10.21.16.tar.gz", hash = "sha256:17cbd055d67d5e9d9de63293a8732943fabc21574e4c7b74edf112b4928cf5f3", size = 16153 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/35/35/5055ab8d215af853d07bbff1a74edf48f91ed308f037380a5ca52dd73348/trove_classifiers-2024.10.21.16-py3-none-any.whl", hash = "sha256:0fb11f1e995a757807a8ef1c03829fbd4998d817319abcef1f33165750f103be", size = 13546 }, +] + +[[package]] +name = "twine" +version = "5.1.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "importlib-metadata" }, + { name = "keyring" }, + { name = "pkginfo" }, + { name = "readme-renderer" }, + { name = "requests" }, + { name = "requests-toolbelt" }, + { name = "rfc3986" }, + { name = "rich" }, + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/77/68/bd982e5e949ef8334e6f7dcf76ae40922a8750aa2e347291ae1477a4782b/twine-5.1.1.tar.gz", hash = "sha256:9aa0825139c02b3434d913545c7b847a21c835e11597f5255842d457da2322db", size = 225531 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/5d/ec/00f9d5fd040ae29867355e559a94e9a8429225a0284a3f5f091a3878bfc0/twine-5.1.1-py3-none-any.whl", hash = "sha256:215dbe7b4b94c2c50a7315c0275d2258399280fbb7d04182c7e55e24b5f93997", size = 38650 }, +] + +[[package]] +name = "typing-extensions" +version = "4.12.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/df/db/f35a00659bc03fec321ba8bce9420de607a1d37f8342eee1863174c69557/typing_extensions-4.12.2.tar.gz", hash = "sha256:1a7ead55c7e559dd4dee8856e3a88b41225abfe1ce8df57b7c13915fe121ffb8", size = 85321 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/26/9f/ad63fc0248c5379346306f8668cda6e2e2e9c95e01216d2b8ffd9ff037d0/typing_extensions-4.12.2-py3-none-any.whl", hash = "sha256:04e5ca0351e0f3f85c6853954072df659d0d13fac324d0072316b67d7794700d", size = 37438 }, +] + +[[package]] +name = "uritemplate" +version = "4.1.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d2/5a/4742fdba39cd02a56226815abfa72fe0aa81c33bed16ed045647d6000eba/uritemplate-4.1.1.tar.gz", hash = "sha256:4346edfc5c3b79f694bccd6d6099a322bbeb628dbf2cd86eea55a456ce5124f0", size = 273898 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/81/c0/7461b49cd25aeece13766f02ee576d1db528f1c37ce69aee300e075b485b/uritemplate-4.1.1-py2.py3-none-any.whl", hash = "sha256:830c08b8d99bdd312ea4ead05994a38e8936266f84b9a7878232db50b044e02e", size = 10356 }, +] + +[[package]] +name = "urllib3" +version = "2.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ed/63/22ba4ebfe7430b76388e7cd448d5478814d3032121827c12a2cc287e2260/urllib3-2.2.3.tar.gz", hash = "sha256:e7d814a81dad81e6caf2ec9fdedb284ecc9c73076b62654547cc64ccdcae26e9", size = 300677 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ce/d9/5f4c13cecde62396b0d3fe530a50ccea91e7dfc1ccf0e09c228841bb5ba8/urllib3-2.2.3-py3-none-any.whl", hash = "sha256:ca899ca043dcb1bafa3e262d73aa25c465bfb49e0bd9dd5d59f1d0acba2f8fac", size = 126338 }, +] + +[[package]] +name = "userpath" +version = "1.9.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "click" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/d5/b7/30753098208505d7ff9be5b3a32112fb8a4cb3ddfccbbb7ba9973f2e29ff/userpath-1.9.2.tar.gz", hash = "sha256:6c52288dab069257cc831846d15d48133522455d4677ee69a9781f11dbefd815", size = 11140 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/43/99/3ec6335ded5b88c2f7ed25c56ffd952546f7ed007ffb1e1539dc3b57015a/userpath-1.9.2-py3-none-any.whl", hash = "sha256:2cbf01a23d655a1ff8fc166dfb78da1b641d1ceabf0fe5f970767d380b14e89d", size = 9065 }, +] + +[[package]] +name = "uv" +version = "0.4.29" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6a/23/6e8d8177112b40d4905a49c03d397c5b93eb030f87cdddf0c5d4be599fc9/uv-0.4.29.tar.gz", hash = "sha256:9c559b6fdc042add463e86afa1c210716f7020bfc2e96b00df5af7afcb587ce7", size = 2102901 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1c/8d/78b6927a3e511a4bc05347714c8917896477537bf09a6301e84de08b7a59/uv-0.4.29-py3-none-linux_armv6l.whl", hash = "sha256:287dc3fd3f78093a5a82136f01cbd9f224e0905b38d3dcffdc96c08fbbe48ee9", size = 13250618 }, + { url = "https://files.pythonhosted.org/packages/d8/2f/1bbfc3c15933fcf07c222e063044696320f5a9fe3d5c584960ed0c490cf8/uv-0.4.29-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:6b03859068aaa08ca9907a51d403d54b0a9d8054091646845a9192f213f099d4", size = 13316211 }, + { url = "https://files.pythonhosted.org/packages/fb/1a/1c862cc36f29cf58b22758f31eb5f9611ee86429d470c8e4c0fd235592ec/uv-0.4.29-py3-none-macosx_11_0_arm64.whl", hash = "sha256:950bbfe1954e9c3a5d6c4777bb778b4c23d0dea9ad9f77622c45d4fbba433355", size = 12363705 }, + { url = "https://files.pythonhosted.org/packages/a1/0e/76e947db1135fa2436b11cc1ca927de187601be7ec65b0102f42a6a58211/uv-0.4.29-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:3473b05142ba436ac30d036b7ab5e9bcfa97f63df5d1382f92e0a3e4aaa391bc", size = 12622825 }, + { url = "https://files.pythonhosted.org/packages/41/3d/b54226b11eb935e4e57585905cf3ded2ac7d972c551bef1c3a000d4c5e47/uv-0.4.29-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7060dfbad0bc26e9cecbb4f8482445c958071511f23728948478f81acfb29048", size = 13054445 }, + { url = "https://files.pythonhosted.org/packages/bf/00/02fa712a3991957d2a65d043173d06d3a429acb3c4e54976f4385c034d97/uv-0.4.29-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:df35d9cbe4cfbb7bce287f56e3bb7a7cef0b7b5173ed889d936d4c470f2b1b83", size = 13655646 }, + { url = "https://files.pythonhosted.org/packages/61/85/f6796032396bbd350648747c984376c8c8add14c75476ed8d5a3438a9c76/uv-0.4.29-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:cfb797a87b55d96cc0593e9f29ab5d58454be74598ea0158e1b2f4f2dc97cede", size = 14281147 }, + { url = "https://files.pythonhosted.org/packages/17/48/3314a03c6580d0b05bd1b9122ff9a9fbde5163815cf84f5a220fc013cea1/uv-0.4.29-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:668d3e6095c6f0cac6a831ef4030f7ad79442d1c84b9569f01f50b60c2d51a77", size = 14004714 }, + { url = "https://files.pythonhosted.org/packages/11/e0/456bc5271f09ff385c57570628705757a59f9a3f8205ff029dc9b2213dbd/uv-0.4.29-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0be21afa0e582ddc5badff6ef40c3c6784efc5feae4ad568307b668d40dc49bd", size = 18032241 }, + { url = "https://files.pythonhosted.org/packages/ef/6c/db10ff7f178ee93a832941e1cddbf38bfb1b0e30fd07580db10eb909f19d/uv-0.4.29-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f6224a322267570e0470c61008fd1c8e2f50bf073b339f4c3010da86aef3c44c", size = 13787528 }, + { url = "https://files.pythonhosted.org/packages/1b/cf/501cd6aeeae0413e83ed0c112a362e44c05fa01144ecfd05c6fb3533778d/uv-0.4.29-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:24cccff9c248864ba0ab3429bae56314146c9494ce66a881d70ea8cf2805945f", size = 12789635 }, + { url = "https://files.pythonhosted.org/packages/8d/8d/3103af713c6369b6c1afe2bd8415eb43ea2cd4d11aa823f2e5747736b410/uv-0.4.29-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:68d4967b5f0af8bd46085e0f3ded229026700668a97734a21c3d11a5fc350c47", size = 13022589 }, + { url = "https://files.pythonhosted.org/packages/4f/4d/e9a0da7c43301f27503ed0af881afb9059e3700bd374d1c7c6579ff9fb29/uv-0.4.29-py3-none-musllinux_1_1_i686.whl", hash = "sha256:75927da78f74bb935314d236dc61ecdc192e878e06eb79585b6d9d5ee9829f98", size = 13367805 }, + { url = "https://files.pythonhosted.org/packages/be/70/a78cd7cdac7581cf0a7e027cf3c69d07ca5b6b83d39f571411cc73f1590f/uv-0.4.29-py3-none-musllinux_1_1_ppc64le.whl", hash = "sha256:246da468ac0d51e7fb257cd038db2f8d6376ae269a44d01f56776e32108aa9da", size = 15158094 }, + { url = "https://files.pythonhosted.org/packages/e6/93/3bcb18a54a9823c8bfadd362022b1c480da10c0bcd86398101f9a124e0a7/uv-0.4.29-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:8c71663c7df4f512c697de39a4926dc191897f5fede73644bb2329f532c1ebfa", size = 13917229 }, + { url = "https://files.pythonhosted.org/packages/8a/38/bd90e265f868ddbc2dd3cc9625e2d8670d3ac35984a078491be11be754f3/uv-0.4.29-py3-none-win32.whl", hash = "sha256:b5775db128b98251c3ea7874367fc20dce9f9aac3dbfa635e3ef4a1c56842d9c", size = 13203439 }, + { url = "https://files.pythonhosted.org/packages/cb/4f/446a0fe5901b110093f3888e93c8ebee1b08f35ba1699bbaf3645b553865/uv-0.4.29-py3-none-win_amd64.whl", hash = "sha256:67dcfd253020e25ed1c49e5bd06406205c37264f99e14002de53a357cd1cdadf", size = 14902665 }, +] + +[[package]] +name = "virtualenv" +version = "20.27.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "distlib" }, + { name = "filelock" }, + { name = "platformdirs" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/8c/b3/7b6a79c5c8cf6d90ea681310e169cf2db2884f4d583d16c6e1d5a75a4e04/virtualenv-20.27.1.tar.gz", hash = "sha256:142c6be10212543b32c6c45d3d3893dff89112cc588b7d0879ae5a1ec03a47ba", size = 6491145 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ae/92/78324ff89391e00c8f4cf6b8526c41c6ef36b4ea2d2c132250b1a6fc2b8d/virtualenv-20.27.1-py3-none-any.whl", hash = "sha256:f11f1b8a29525562925f745563bfd48b189450f61fb34c4f9cc79dd5aa32a1f4", size = 3117838 }, +] + +[[package]] +name = "wrapt" +version = "1.16.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/95/4c/063a912e20bcef7124e0df97282a8af3ff3e4b603ce84c481d6d7346be0a/wrapt-1.16.0.tar.gz", hash = "sha256:5f370f952971e7d17c7d1ead40e49f32345a7f7a5373571ef44d800d06b1899d", size = 53972 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a8/c6/5375258add3777494671d8cec27cdf5402abd91016dee24aa2972c61fedf/wrapt-1.16.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:ffa565331890b90056c01db69c0fe634a776f8019c143a5ae265f9c6bc4bd6d4", size = 37315 }, + { url = "https://files.pythonhosted.org/packages/32/12/e11adfde33444986135d8881b401e4de6cbb4cced046edc6b464e6ad7547/wrapt-1.16.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e4fdb9275308292e880dcbeb12546df7f3e0f96c6b41197e0cf37d2826359020", size = 38160 }, + { url = "https://files.pythonhosted.org/packages/70/7d/3dcc4a7e96f8d3e398450ec7703db384413f79bd6c0196e0e139055ce00f/wrapt-1.16.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb2dee3874a500de01c93d5c71415fcaef1d858370d405824783e7a8ef5db440", size = 80419 }, + { url = "https://files.pythonhosted.org/packages/d1/c4/8dfdc3c2f0b38be85c8d9fdf0011ebad2f54e40897f9549a356bebb63a97/wrapt-1.16.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2a88e6010048489cda82b1326889ec075a8c856c2e6a256072b28eaee3ccf487", size = 72669 }, + { url = "https://files.pythonhosted.org/packages/49/83/b40bc1ad04a868b5b5bcec86349f06c1ee1ea7afe51dc3e46131e4f39308/wrapt-1.16.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ac83a914ebaf589b69f7d0a1277602ff494e21f4c2f743313414378f8f50a4cf", size = 80271 }, + { url = "https://files.pythonhosted.org/packages/19/d4/cd33d3a82df73a064c9b6401d14f346e1d2fb372885f0295516ec08ed2ee/wrapt-1.16.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:73aa7d98215d39b8455f103de64391cb79dfcad601701a3aa0dddacf74911d72", size = 84748 }, + { url = "https://files.pythonhosted.org/packages/ef/58/2fde309415b5fa98fd8f5f4a11886cbf276824c4c64d45a39da342fff6fe/wrapt-1.16.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:807cc8543a477ab7422f1120a217054f958a66ef7314f76dd9e77d3f02cdccd0", size = 77522 }, + { url = "https://files.pythonhosted.org/packages/07/44/359e4724a92369b88dbf09878a7cde7393cf3da885567ea898e5904049a3/wrapt-1.16.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:bf5703fdeb350e36885f2875d853ce13172ae281c56e509f4e6eca049bdfb136", size = 84780 }, + { url = "https://files.pythonhosted.org/packages/88/8f/706f2fee019360cc1da652353330350c76aa5746b4e191082e45d6838faf/wrapt-1.16.0-cp310-cp310-win32.whl", hash = "sha256:f6b2d0c6703c988d334f297aa5df18c45e97b0af3679bb75059e0e0bd8b1069d", size = 35335 }, + { url = "https://files.pythonhosted.org/packages/19/2b/548d23362e3002ebbfaefe649b833fa43f6ca37ac3e95472130c4b69e0b4/wrapt-1.16.0-cp310-cp310-win_amd64.whl", hash = "sha256:decbfa2f618fa8ed81c95ee18a387ff973143c656ef800c9f24fb7e9c16054e2", size = 37528 }, + { url = "https://files.pythonhosted.org/packages/fd/03/c188ac517f402775b90d6f312955a5e53b866c964b32119f2ed76315697e/wrapt-1.16.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:1a5db485fe2de4403f13fafdc231b0dbae5eca4359232d2efc79025527375b09", size = 37313 }, + { url = "https://files.pythonhosted.org/packages/0f/16/ea627d7817394db04518f62934a5de59874b587b792300991b3c347ff5e0/wrapt-1.16.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:75ea7d0ee2a15733684badb16de6794894ed9c55aa5e9903260922f0482e687d", size = 38164 }, + { url = "https://files.pythonhosted.org/packages/7f/a7/f1212ba098f3de0fd244e2de0f8791ad2539c03bef6c05a9fcb03e45b089/wrapt-1.16.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a452f9ca3e3267cd4d0fcf2edd0d035b1934ac2bd7e0e57ac91ad6b95c0c6389", size = 80890 }, + { url = "https://files.pythonhosted.org/packages/b7/96/bb5e08b3d6db003c9ab219c487714c13a237ee7dcc572a555eaf1ce7dc82/wrapt-1.16.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:43aa59eadec7890d9958748db829df269f0368521ba6dc68cc172d5d03ed8060", size = 73118 }, + { url = "https://files.pythonhosted.org/packages/6e/52/2da48b35193e39ac53cfb141467d9f259851522d0e8c87153f0ba4205fb1/wrapt-1.16.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:72554a23c78a8e7aa02abbd699d129eead8b147a23c56e08d08dfc29cfdddca1", size = 80746 }, + { url = "https://files.pythonhosted.org/packages/11/fb/18ec40265ab81c0e82a934de04596b6ce972c27ba2592c8b53d5585e6bcd/wrapt-1.16.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:d2efee35b4b0a347e0d99d28e884dfd82797852d62fcd7ebdeee26f3ceb72cf3", size = 85668 }, + { url = "https://files.pythonhosted.org/packages/0f/ef/0ecb1fa23145560431b970418dce575cfaec555ab08617d82eb92afc7ccf/wrapt-1.16.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:6dcfcffe73710be01d90cae08c3e548d90932d37b39ef83969ae135d36ef3956", size = 78556 }, + { url = "https://files.pythonhosted.org/packages/25/62/cd284b2b747f175b5a96cbd8092b32e7369edab0644c45784871528eb852/wrapt-1.16.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:eb6e651000a19c96f452c85132811d25e9264d836951022d6e81df2fff38337d", size = 85712 }, + { url = "https://files.pythonhosted.org/packages/e5/a7/47b7ff74fbadf81b696872d5ba504966591a3468f1bc86bca2f407baef68/wrapt-1.16.0-cp311-cp311-win32.whl", hash = "sha256:66027d667efe95cc4fa945af59f92c5a02c6f5bb6012bff9e60542c74c75c362", size = 35327 }, + { url = "https://files.pythonhosted.org/packages/cf/c3/0084351951d9579ae83a3d9e38c140371e4c6b038136909235079f2e6e78/wrapt-1.16.0-cp311-cp311-win_amd64.whl", hash = "sha256:aefbc4cb0a54f91af643660a0a150ce2c090d3652cf4052a5397fb2de549cd89", size = 37523 }, + { url = "https://files.pythonhosted.org/packages/92/17/224132494c1e23521868cdd57cd1e903f3b6a7ba6996b7b8f077ff8ac7fe/wrapt-1.16.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:5eb404d89131ec9b4f748fa5cfb5346802e5ee8836f57d516576e61f304f3b7b", size = 37614 }, + { url = "https://files.pythonhosted.org/packages/6a/d7/cfcd73e8f4858079ac59d9db1ec5a1349bc486ae8e9ba55698cc1f4a1dff/wrapt-1.16.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9090c9e676d5236a6948330e83cb89969f433b1943a558968f659ead07cb3b36", size = 38316 }, + { url = "https://files.pythonhosted.org/packages/7e/79/5ff0a5c54bda5aec75b36453d06be4f83d5cd4932cc84b7cb2b52cee23e2/wrapt-1.16.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:94265b00870aa407bd0cbcfd536f17ecde43b94fb8d228560a1e9d3041462d73", size = 86322 }, + { url = "https://files.pythonhosted.org/packages/c4/81/e799bf5d419f422d8712108837c1d9bf6ebe3cb2a81ad94413449543a923/wrapt-1.16.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f2058f813d4f2b5e3a9eb2eb3faf8f1d99b81c3e51aeda4b168406443e8ba809", size = 79055 }, + { url = "https://files.pythonhosted.org/packages/62/62/30ca2405de6a20448ee557ab2cd61ab9c5900be7cbd18a2639db595f0b98/wrapt-1.16.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98b5e1f498a8ca1858a1cdbffb023bfd954da4e3fa2c0cb5853d40014557248b", size = 87291 }, + { url = "https://files.pythonhosted.org/packages/49/4e/5d2f6d7b57fc9956bf06e944eb00463551f7d52fc73ca35cfc4c2cdb7aed/wrapt-1.16.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:14d7dc606219cdd7405133c713f2c218d4252f2a469003f8c46bb92d5d095d81", size = 90374 }, + { url = "https://files.pythonhosted.org/packages/a6/9b/c2c21b44ff5b9bf14a83252a8b973fb84923764ff63db3e6dfc3895cf2e0/wrapt-1.16.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:49aac49dc4782cb04f58986e81ea0b4768e4ff197b57324dcbd7699c5dfb40b9", size = 83896 }, + { url = "https://files.pythonhosted.org/packages/14/26/93a9fa02c6f257df54d7570dfe8011995138118d11939a4ecd82cb849613/wrapt-1.16.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:418abb18146475c310d7a6dc71143d6f7adec5b004ac9ce08dc7a34e2babdc5c", size = 91738 }, + { url = "https://files.pythonhosted.org/packages/a2/5b/4660897233eb2c8c4de3dc7cefed114c61bacb3c28327e64150dc44ee2f6/wrapt-1.16.0-cp312-cp312-win32.whl", hash = "sha256:685f568fa5e627e93f3b52fda002c7ed2fa1800b50ce51f6ed1d572d8ab3e7fc", size = 35568 }, + { url = "https://files.pythonhosted.org/packages/5c/cc/8297f9658506b224aa4bd71906447dea6bb0ba629861a758c28f67428b91/wrapt-1.16.0-cp312-cp312-win_amd64.whl", hash = "sha256:dcdba5c86e368442528f7060039eda390cc4091bfd1dca41e8046af7c910dda8", size = 37653 }, + { url = "https://files.pythonhosted.org/packages/70/cc/b92e1da2cad6a9f8ee481000ece07a35e3b24e041e60ff8b850c079f0ebf/wrapt-1.16.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9b201ae332c3637a42f02d1045e1d0cccfdc41f1f2f801dafbaa7e9b4797bfc2", size = 37314 }, + { url = "https://files.pythonhosted.org/packages/4a/cc/3402bcc897978be00fef608cd9e3e39ec8869c973feeb5e1e277670e5ad2/wrapt-1.16.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:2076fad65c6736184e77d7d4729b63a6d1ae0b70da4868adeec40989858eb3fb", size = 38162 }, + { url = "https://files.pythonhosted.org/packages/28/d3/4f079f649c515727c127c987b2ec2e0816b80d95784f2d28d1a57d2a1029/wrapt-1.16.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c5cd603b575ebceca7da5a3a251e69561bec509e0b46e4993e1cac402b7247b8", size = 80235 }, + { url = "https://files.pythonhosted.org/packages/a3/1c/226c2a4932e578a2241dcb383f425995f80224b446f439c2e112eb51c3a6/wrapt-1.16.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b47cfad9e9bbbed2339081f4e346c93ecd7ab504299403320bf85f7f85c7d46c", size = 72553 }, + { url = "https://files.pythonhosted.org/packages/b1/e7/459a8a4f40f2fa65eb73cb3f339e6d152957932516d18d0e996c7ae2d7ae/wrapt-1.16.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f8212564d49c50eb4565e502814f694e240c55551a5f1bc841d4fcaabb0a9b8a", size = 80129 }, + { url = "https://files.pythonhosted.org/packages/da/6f/6d0b3c4983f1fc764a422989dabc268ee87d937763246cd48aa92f1eed1e/wrapt-1.16.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:5f15814a33e42b04e3de432e573aa557f9f0f56458745c2074952f564c50e664", size = 84550 }, + { url = "https://files.pythonhosted.org/packages/96/e8/27ef35cf61e5147c1c3abcb89cfbb8d691b2bb8364803fcc950140bc14d8/wrapt-1.16.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:db2e408d983b0e61e238cf579c09ef7020560441906ca990fe8412153e3b291f", size = 77352 }, + { url = "https://files.pythonhosted.org/packages/b6/ad/7a0766341081bfd9f18a7049e4d6d45586ae5c5bb0a640f05e2f558e849c/wrapt-1.16.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:edfad1d29c73f9b863ebe7082ae9321374ccb10879eeabc84ba3b69f2579d537", size = 84626 }, + { url = "https://files.pythonhosted.org/packages/09/43/b26852e9c45a1aac0d14b1080b25b612fa840ba99739c5fc55db07b7ce08/wrapt-1.16.0-cp39-cp39-win32.whl", hash = "sha256:ed867c42c268f876097248e05b6117a65bcd1e63b779e916fe2e33cd6fd0d3c3", size = 35327 }, + { url = "https://files.pythonhosted.org/packages/74/f2/96ed140b08743f7f68d5bda35a2a589600781366c3da96f056043d258b1a/wrapt-1.16.0-cp39-cp39-win_amd64.whl", hash = "sha256:eb1b046be06b0fce7249f1d025cd359b4b80fc1c3e24ad9eca33e0dcdb2e4a35", size = 37526 }, + { url = "https://files.pythonhosted.org/packages/ff/21/abdedb4cdf6ff41ebf01a74087740a709e2edb146490e4d9beea054b0b7a/wrapt-1.16.0-py3-none-any.whl", hash = "sha256:6906c4100a8fcbf2fa735f6059214bb13b97f75b1a61777fcf6432121ef12ef1", size = 23362 }, +] + +[[package]] +name = "zipp" +version = "3.20.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/54/bf/5c0000c44ebc80123ecbdddba1f5dcd94a5ada602a9c225d84b5aaa55e86/zipp-3.20.2.tar.gz", hash = "sha256:bc9eb26f4506fda01b81bcde0ca78103b6e62f991b381fec825435c836edbc29", size = 24199 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/62/8b/5ba542fa83c90e09eac972fc9baca7a88e7e7ca4b221a89251954019308b/zipp-3.20.2-py3-none-any.whl", hash = "sha256:a817ac80d6cf4b23bf7f2828b7cabf326f15a001bea8b1f9b49631780ba28350", size = 9200 }, +] + +[[package]] +name = "zstandard" +version = "0.23.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cffi", marker = "platform_python_implementation == 'PyPy'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ed/f6/2ac0287b442160a89d726b17a9184a4c615bb5237db763791a7fd16d9df1/zstandard-0.23.0.tar.gz", hash = "sha256:b2d8c62d08e7255f68f7a740bae85b3c9b8e5466baa9cbf7f57f1cde0ac6bc09", size = 681701 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/2a/55/bd0487e86679db1823fc9ee0d8c9c78ae2413d34c0b461193b5f4c31d22f/zstandard-0.23.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bf0a05b6059c0528477fba9054d09179beb63744355cab9f38059548fedd46a9", size = 788701 }, + { url = "https://files.pythonhosted.org/packages/e1/8a/ccb516b684f3ad987dfee27570d635822e3038645b1a950c5e8022df1145/zstandard-0.23.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:fc9ca1c9718cb3b06634c7c8dec57d24e9438b2aa9a0f02b8bb36bf478538880", size = 633678 }, + { url = "https://files.pythonhosted.org/packages/12/89/75e633d0611c028e0d9af6df199423bf43f54bea5007e6718ab7132e234c/zstandard-0.23.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77da4c6bfa20dd5ea25cbf12c76f181a8e8cd7ea231c673828d0386b1740b8dc", size = 4941098 }, + { url = "https://files.pythonhosted.org/packages/4a/7a/bd7f6a21802de358b63f1ee636ab823711c25ce043a3e9f043b4fcb5ba32/zstandard-0.23.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b2170c7e0367dde86a2647ed5b6f57394ea7f53545746104c6b09fc1f4223573", size = 5308798 }, + { url = "https://files.pythonhosted.org/packages/79/3b/775f851a4a65013e88ca559c8ae42ac1352db6fcd96b028d0df4d7d1d7b4/zstandard-0.23.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c16842b846a8d2a145223f520b7e18b57c8f476924bda92aeee3a88d11cfc391", size = 5341840 }, + { url = "https://files.pythonhosted.org/packages/09/4f/0cc49570141dd72d4d95dd6fcf09328d1b702c47a6ec12fbed3b8aed18a5/zstandard-0.23.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:157e89ceb4054029a289fb504c98c6a9fe8010f1680de0201b3eb5dc20aa6d9e", size = 5440337 }, + { url = "https://files.pythonhosted.org/packages/e7/7c/aaa7cd27148bae2dc095191529c0570d16058c54c4597a7d118de4b21676/zstandard-0.23.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:203d236f4c94cd8379d1ea61db2fce20730b4c38d7f1c34506a31b34edc87bdd", size = 4861182 }, + { url = "https://files.pythonhosted.org/packages/ac/eb/4b58b5c071d177f7dc027129d20bd2a44161faca6592a67f8fcb0b88b3ae/zstandard-0.23.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:dc5d1a49d3f8262be192589a4b72f0d03b72dcf46c51ad5852a4fdc67be7b9e4", size = 4932936 }, + { url = "https://files.pythonhosted.org/packages/44/f9/21a5fb9bb7c9a274b05ad700a82ad22ce82f7ef0f485980a1e98ed6e8c5f/zstandard-0.23.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:752bf8a74412b9892f4e5b58f2f890a039f57037f52c89a740757ebd807f33ea", size = 5464705 }, + { url = "https://files.pythonhosted.org/packages/49/74/b7b3e61db3f88632776b78b1db597af3f44c91ce17d533e14a25ce6a2816/zstandard-0.23.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:80080816b4f52a9d886e67f1f96912891074903238fe54f2de8b786f86baded2", size = 4857882 }, + { url = "https://files.pythonhosted.org/packages/4a/7f/d8eb1cb123d8e4c541d4465167080bec88481ab54cd0b31eb4013ba04b95/zstandard-0.23.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:84433dddea68571a6d6bd4fbf8ff398236031149116a7fff6f777ff95cad3df9", size = 4697672 }, + { url = "https://files.pythonhosted.org/packages/5e/05/f7dccdf3d121309b60342da454d3e706453a31073e2c4dac8e1581861e44/zstandard-0.23.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:ab19a2d91963ed9e42b4e8d77cd847ae8381576585bad79dbd0a8837a9f6620a", size = 5206043 }, + { url = "https://files.pythonhosted.org/packages/86/9d/3677a02e172dccd8dd3a941307621c0cbd7691d77cb435ac3c75ab6a3105/zstandard-0.23.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:59556bf80a7094d0cfb9f5e50bb2db27fefb75d5138bb16fb052b61b0e0eeeb0", size = 5667390 }, + { url = "https://files.pythonhosted.org/packages/41/7e/0012a02458e74a7ba122cd9cafe491facc602c9a17f590367da369929498/zstandard-0.23.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:27d3ef2252d2e62476389ca8f9b0cf2bbafb082a3b6bfe9d90cbcbb5529ecf7c", size = 5198901 }, + { url = "https://files.pythonhosted.org/packages/65/3a/8f715b97bd7bcfc7342d8adcd99a026cb2fb550e44866a3b6c348e1b0f02/zstandard-0.23.0-cp310-cp310-win32.whl", hash = "sha256:5d41d5e025f1e0bccae4928981e71b2334c60f580bdc8345f824e7c0a4c2a813", size = 430596 }, + { url = "https://files.pythonhosted.org/packages/19/b7/b2b9eca5e5a01111e4fe8a8ffb56bdcdf56b12448a24effe6cfe4a252034/zstandard-0.23.0-cp310-cp310-win_amd64.whl", hash = "sha256:519fbf169dfac1222a76ba8861ef4ac7f0530c35dd79ba5727014613f91613d4", size = 495498 }, + { url = "https://files.pythonhosted.org/packages/9e/40/f67e7d2c25a0e2dc1744dd781110b0b60306657f8696cafb7ad7579469bd/zstandard-0.23.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:34895a41273ad33347b2fc70e1bff4240556de3c46c6ea430a7ed91f9042aa4e", size = 788699 }, + { url = "https://files.pythonhosted.org/packages/e8/46/66d5b55f4d737dd6ab75851b224abf0afe5774976fe511a54d2eb9063a41/zstandard-0.23.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:77ea385f7dd5b5676d7fd943292ffa18fbf5c72ba98f7d09fc1fb9e819b34c23", size = 633681 }, + { url = "https://files.pythonhosted.org/packages/63/b6/677e65c095d8e12b66b8f862b069bcf1f1d781b9c9c6f12eb55000d57583/zstandard-0.23.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:983b6efd649723474f29ed42e1467f90a35a74793437d0bc64a5bf482bedfa0a", size = 4944328 }, + { url = "https://files.pythonhosted.org/packages/59/cc/e76acb4c42afa05a9d20827116d1f9287e9c32b7ad58cc3af0721ce2b481/zstandard-0.23.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80a539906390591dd39ebb8d773771dc4db82ace6372c4d41e2d293f8e32b8db", size = 5311955 }, + { url = "https://files.pythonhosted.org/packages/78/e4/644b8075f18fc7f632130c32e8f36f6dc1b93065bf2dd87f03223b187f26/zstandard-0.23.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:445e4cb5048b04e90ce96a79b4b63140e3f4ab5f662321975679b5f6360b90e2", size = 5344944 }, + { url = "https://files.pythonhosted.org/packages/76/3f/dbafccf19cfeca25bbabf6f2dd81796b7218f768ec400f043edc767015a6/zstandard-0.23.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd30d9c67d13d891f2360b2a120186729c111238ac63b43dbd37a5a40670b8ca", size = 5442927 }, + { url = "https://files.pythonhosted.org/packages/0c/c3/d24a01a19b6733b9f218e94d1a87c477d523237e07f94899e1c10f6fd06c/zstandard-0.23.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d20fd853fbb5807c8e84c136c278827b6167ded66c72ec6f9a14b863d809211c", size = 4864910 }, + { url = "https://files.pythonhosted.org/packages/1c/a9/cf8f78ead4597264f7618d0875be01f9bc23c9d1d11afb6d225b867cb423/zstandard-0.23.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:ed1708dbf4d2e3a1c5c69110ba2b4eb6678262028afd6c6fbcc5a8dac9cda68e", size = 4935544 }, + { url = "https://files.pythonhosted.org/packages/2c/96/8af1e3731b67965fb995a940c04a2c20997a7b3b14826b9d1301cf160879/zstandard-0.23.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:be9b5b8659dff1f913039c2feee1aca499cfbc19e98fa12bc85e037c17ec6ca5", size = 5467094 }, + { url = "https://files.pythonhosted.org/packages/ff/57/43ea9df642c636cb79f88a13ab07d92d88d3bfe3e550b55a25a07a26d878/zstandard-0.23.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:65308f4b4890aa12d9b6ad9f2844b7ee42c7f7a4fd3390425b242ffc57498f48", size = 4860440 }, + { url = "https://files.pythonhosted.org/packages/46/37/edb78f33c7f44f806525f27baa300341918fd4c4af9472fbc2c3094be2e8/zstandard-0.23.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:98da17ce9cbf3bfe4617e836d561e433f871129e3a7ac16d6ef4c680f13a839c", size = 4700091 }, + { url = "https://files.pythonhosted.org/packages/c1/f1/454ac3962671a754f3cb49242472df5c2cced4eb959ae203a377b45b1a3c/zstandard-0.23.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:8ed7d27cb56b3e058d3cf684d7200703bcae623e1dcc06ed1e18ecda39fee003", size = 5208682 }, + { url = "https://files.pythonhosted.org/packages/85/b2/1734b0fff1634390b1b887202d557d2dd542de84a4c155c258cf75da4773/zstandard-0.23.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:b69bb4f51daf461b15e7b3db033160937d3ff88303a7bc808c67bbc1eaf98c78", size = 5669707 }, + { url = "https://files.pythonhosted.org/packages/52/5a/87d6971f0997c4b9b09c495bf92189fb63de86a83cadc4977dc19735f652/zstandard-0.23.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:034b88913ecc1b097f528e42b539453fa82c3557e414b3de9d5632c80439a473", size = 5201792 }, + { url = "https://files.pythonhosted.org/packages/79/02/6f6a42cc84459d399bd1a4e1adfc78d4dfe45e56d05b072008d10040e13b/zstandard-0.23.0-cp311-cp311-win32.whl", hash = "sha256:f2d4380bf5f62daabd7b751ea2339c1a21d1c9463f1feb7fc2bdcea2c29c3160", size = 430586 }, + { url = "https://files.pythonhosted.org/packages/be/a2/4272175d47c623ff78196f3c10e9dc7045c1b9caf3735bf041e65271eca4/zstandard-0.23.0-cp311-cp311-win_amd64.whl", hash = "sha256:62136da96a973bd2557f06ddd4e8e807f9e13cbb0bfb9cc06cfe6d98ea90dfe0", size = 495420 }, + { url = "https://files.pythonhosted.org/packages/7b/83/f23338c963bd9de687d47bf32efe9fd30164e722ba27fb59df33e6b1719b/zstandard-0.23.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:b4567955a6bc1b20e9c31612e615af6b53733491aeaa19a6b3b37f3b65477094", size = 788713 }, + { url = "https://files.pythonhosted.org/packages/5b/b3/1a028f6750fd9227ee0b937a278a434ab7f7fdc3066c3173f64366fe2466/zstandard-0.23.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1e172f57cd78c20f13a3415cc8dfe24bf388614324d25539146594c16d78fcc8", size = 633459 }, + { url = "https://files.pythonhosted.org/packages/26/af/36d89aae0c1f95a0a98e50711bc5d92c144939efc1f81a2fcd3e78d7f4c1/zstandard-0.23.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b0e166f698c5a3e914947388c162be2583e0c638a4703fc6a543e23a88dea3c1", size = 4945707 }, + { url = "https://files.pythonhosted.org/packages/cd/2e/2051f5c772f4dfc0aae3741d5fc72c3dcfe3aaeb461cc231668a4db1ce14/zstandard-0.23.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:12a289832e520c6bd4dcaad68e944b86da3bad0d339ef7989fb7e88f92e96072", size = 5306545 }, + { url = "https://files.pythonhosted.org/packages/0a/9e/a11c97b087f89cab030fa71206963090d2fecd8eb83e67bb8f3ffb84c024/zstandard-0.23.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d50d31bfedd53a928fed6707b15a8dbeef011bb6366297cc435accc888b27c20", size = 5337533 }, + { url = "https://files.pythonhosted.org/packages/fc/79/edeb217c57fe1bf16d890aa91a1c2c96b28c07b46afed54a5dcf310c3f6f/zstandard-0.23.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:72c68dda124a1a138340fb62fa21b9bf4848437d9ca60bd35db36f2d3345f373", size = 5436510 }, + { url = "https://files.pythonhosted.org/packages/81/4f/c21383d97cb7a422ddf1ae824b53ce4b51063d0eeb2afa757eb40804a8ef/zstandard-0.23.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:53dd9d5e3d29f95acd5de6802e909ada8d8d8cfa37a3ac64836f3bc4bc5512db", size = 4859973 }, + { url = "https://files.pythonhosted.org/packages/ab/15/08d22e87753304405ccac8be2493a495f529edd81d39a0870621462276ef/zstandard-0.23.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:6a41c120c3dbc0d81a8e8adc73312d668cd34acd7725f036992b1b72d22c1772", size = 4936968 }, + { url = "https://files.pythonhosted.org/packages/eb/fa/f3670a597949fe7dcf38119a39f7da49a8a84a6f0b1a2e46b2f71a0ab83f/zstandard-0.23.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:40b33d93c6eddf02d2c19f5773196068d875c41ca25730e8288e9b672897c105", size = 5467179 }, + { url = "https://files.pythonhosted.org/packages/4e/a9/dad2ab22020211e380adc477a1dbf9f109b1f8d94c614944843e20dc2a99/zstandard-0.23.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9206649ec587e6b02bd124fb7799b86cddec350f6f6c14bc82a2b70183e708ba", size = 4848577 }, + { url = "https://files.pythonhosted.org/packages/08/03/dd28b4484b0770f1e23478413e01bee476ae8227bbc81561f9c329e12564/zstandard-0.23.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:76e79bc28a65f467e0409098fa2c4376931fd3207fbeb6b956c7c476d53746dd", size = 4693899 }, + { url = "https://files.pythonhosted.org/packages/2b/64/3da7497eb635d025841e958bcd66a86117ae320c3b14b0ae86e9e8627518/zstandard-0.23.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:66b689c107857eceabf2cf3d3fc699c3c0fe8ccd18df2219d978c0283e4c508a", size = 5199964 }, + { url = "https://files.pythonhosted.org/packages/43/a4/d82decbab158a0e8a6ebb7fc98bc4d903266bce85b6e9aaedea1d288338c/zstandard-0.23.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9c236e635582742fee16603042553d276cca506e824fa2e6489db04039521e90", size = 5655398 }, + { url = "https://files.pythonhosted.org/packages/f2/61/ac78a1263bc83a5cf29e7458b77a568eda5a8f81980691bbc6eb6a0d45cc/zstandard-0.23.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a8fffdbd9d1408006baaf02f1068d7dd1f016c6bcb7538682622c556e7b68e35", size = 5191313 }, + { url = "https://files.pythonhosted.org/packages/e7/54/967c478314e16af5baf849b6ee9d6ea724ae5b100eb506011f045d3d4e16/zstandard-0.23.0-cp312-cp312-win32.whl", hash = "sha256:dc1d33abb8a0d754ea4763bad944fd965d3d95b5baef6b121c0c9013eaf1907d", size = 430877 }, + { url = "https://files.pythonhosted.org/packages/75/37/872d74bd7739639c4553bf94c84af7d54d8211b626b352bc57f0fd8d1e3f/zstandard-0.23.0-cp312-cp312-win_amd64.whl", hash = "sha256:64585e1dba664dc67c7cdabd56c1e5685233fbb1fc1966cfba2a340ec0dfff7b", size = 495595 }, + { url = "https://files.pythonhosted.org/packages/80/f1/8386f3f7c10261fe85fbc2c012fdb3d4db793b921c9abcc995d8da1b7a80/zstandard-0.23.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:576856e8594e6649aee06ddbfc738fec6a834f7c85bf7cadd1c53d4a58186ef9", size = 788975 }, + { url = "https://files.pythonhosted.org/packages/16/e8/cbf01077550b3e5dc86089035ff8f6fbbb312bc0983757c2d1117ebba242/zstandard-0.23.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:38302b78a850ff82656beaddeb0bb989a0322a8bbb1bf1ab10c17506681d772a", size = 633448 }, + { url = "https://files.pythonhosted.org/packages/06/27/4a1b4c267c29a464a161aeb2589aff212b4db653a1d96bffe3598f3f0d22/zstandard-0.23.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d2240ddc86b74966c34554c49d00eaafa8200a18d3a5b6ffbf7da63b11d74ee2", size = 4945269 }, + { url = "https://files.pythonhosted.org/packages/7c/64/d99261cc57afd9ae65b707e38045ed8269fbdae73544fd2e4a4d50d0ed83/zstandard-0.23.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2ef230a8fd217a2015bc91b74f6b3b7d6522ba48be29ad4ea0ca3a3775bf7dd5", size = 5306228 }, + { url = "https://files.pythonhosted.org/packages/7a/cf/27b74c6f22541f0263016a0fd6369b1b7818941de639215c84e4e94b2a1c/zstandard-0.23.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:774d45b1fac1461f48698a9d4b5fa19a69d47ece02fa469825b442263f04021f", size = 5336891 }, + { url = "https://files.pythonhosted.org/packages/fa/18/89ac62eac46b69948bf35fcd90d37103f38722968e2981f752d69081ec4d/zstandard-0.23.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6f77fa49079891a4aab203d0b1744acc85577ed16d767b52fc089d83faf8d8ed", size = 5436310 }, + { url = "https://files.pythonhosted.org/packages/a8/a8/5ca5328ee568a873f5118d5b5f70d1f36c6387716efe2e369010289a5738/zstandard-0.23.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ac184f87ff521f4840e6ea0b10c0ec90c6b1dcd0bad2f1e4a9a1b4fa177982ea", size = 4859912 }, + { url = "https://files.pythonhosted.org/packages/ea/ca/3781059c95fd0868658b1cf0440edd832b942f84ae60685d0cfdb808bca1/zstandard-0.23.0-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:c363b53e257246a954ebc7c488304b5592b9c53fbe74d03bc1c64dda153fb847", size = 4936946 }, + { url = "https://files.pythonhosted.org/packages/ce/11/41a58986f809532742c2b832c53b74ba0e0a5dae7e8ab4642bf5876f35de/zstandard-0.23.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:e7792606d606c8df5277c32ccb58f29b9b8603bf83b48639b7aedf6df4fe8171", size = 5466994 }, + { url = "https://files.pythonhosted.org/packages/83/e3/97d84fe95edd38d7053af05159465d298c8b20cebe9ccb3d26783faa9094/zstandard-0.23.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a0817825b900fcd43ac5d05b8b3079937073d2b1ff9cf89427590718b70dd840", size = 4848681 }, + { url = "https://files.pythonhosted.org/packages/6e/99/cb1e63e931de15c88af26085e3f2d9af9ce53ccafac73b6e48418fd5a6e6/zstandard-0.23.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:9da6bc32faac9a293ddfdcb9108d4b20416219461e4ec64dfea8383cac186690", size = 4694239 }, + { url = "https://files.pythonhosted.org/packages/ab/50/b1e703016eebbc6501fc92f34db7b1c68e54e567ef39e6e59cf5fb6f2ec0/zstandard-0.23.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:fd7699e8fd9969f455ef2926221e0233f81a2542921471382e77a9e2f2b57f4b", size = 5200149 }, + { url = "https://files.pythonhosted.org/packages/aa/e0/932388630aaba70197c78bdb10cce2c91fae01a7e553b76ce85471aec690/zstandard-0.23.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:d477ed829077cd945b01fc3115edd132c47e6540ddcd96ca169facff28173057", size = 5655392 }, + { url = "https://files.pythonhosted.org/packages/02/90/2633473864f67a15526324b007a9f96c96f56d5f32ef2a56cc12f9548723/zstandard-0.23.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:fa6ce8b52c5987b3e34d5674b0ab529a4602b632ebab0a93b07bfb4dfc8f8a33", size = 5191299 }, + { url = "https://files.pythonhosted.org/packages/b0/4c/315ca5c32da7e2dc3455f3b2caee5c8c2246074a61aac6ec3378a97b7136/zstandard-0.23.0-cp313-cp313-win32.whl", hash = "sha256:a9b07268d0c3ca5c170a385a0ab9fb7fdd9f5fd866be004c4ea39e44edce47dd", size = 430862 }, + { url = "https://files.pythonhosted.org/packages/a2/bf/c6aaba098e2d04781e8f4f7c0ba3c7aa73d00e4c436bcc0cf059a66691d1/zstandard-0.23.0-cp313-cp313-win_amd64.whl", hash = "sha256:f3513916e8c645d0610815c257cbfd3242adfd5c4cfa78be514e5a3ebb42a41b", size = 495578 }, + { url = "https://files.pythonhosted.org/packages/fb/96/4fcafeb7e013a2386d22f974b5b97a0b9a65004ed58c87ae001599bfbd48/zstandard-0.23.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3aa014d55c3af933c1315eb4bb06dd0459661cc0b15cd61077afa6489bec63bb", size = 788697 }, + { url = "https://files.pythonhosted.org/packages/83/ff/a52ce725be69b86a2967ecba0497a8184540cc284c0991125515449e54e2/zstandard-0.23.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0a7f0804bb3799414af278e9ad51be25edf67f78f916e08afdb983e74161b916", size = 633679 }, + { url = "https://files.pythonhosted.org/packages/34/0f/3dc62db122f6a9c481c335fff6fc9f4e88d8f6e2d47321ee3937328addb4/zstandard-0.23.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fb2b1ecfef1e67897d336de3a0e3f52478182d6a47eda86cbd42504c5cbd009a", size = 4940416 }, + { url = "https://files.pythonhosted.org/packages/1d/e5/9fe0dd8c85fdc2f635e6660d07872a5dc4b366db566630161e39f9f804e1/zstandard-0.23.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:837bb6764be6919963ef41235fd56a6486b132ea64afe5fafb4cb279ac44f259", size = 5307693 }, + { url = "https://files.pythonhosted.org/packages/73/bf/fe62c0cd865c171ee8ed5bc83174b5382a2cb729c8d6162edfb99a83158b/zstandard-0.23.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1516c8c37d3a053b01c1c15b182f3b5f5eef19ced9b930b684a73bad121addf4", size = 5341236 }, + { url = "https://files.pythonhosted.org/packages/39/86/4fe79b30c794286110802a6cd44a73b6a314ac8196b9338c0fbd78c2407d/zstandard-0.23.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:48ef6a43b1846f6025dde6ed9fee0c24e1149c1c25f7fb0a0585572b2f3adc58", size = 5439101 }, + { url = "https://files.pythonhosted.org/packages/72/ed/cacec235c581ebf8c608c7fb3d4b6b70d1b490d0e5128ea6996f809ecaef/zstandard-0.23.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:11e3bf3c924853a2d5835b24f03eeba7fc9b07d8ca499e247e06ff5676461a15", size = 4860320 }, + { url = "https://files.pythonhosted.org/packages/f6/1e/2c589a2930f93946b132fc852c574a19d5edc23fad2b9e566f431050c7ec/zstandard-0.23.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2fb4535137de7e244c230e24f9d1ec194f61721c86ebea04e1581d9d06ea1269", size = 4931933 }, + { url = "https://files.pythonhosted.org/packages/8e/f5/30eadde3686d902b5d4692bb5f286977cbc4adc082145eb3f49d834b2eae/zstandard-0.23.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8c24f21fa2af4bb9f2c492a86fe0c34e6d2c63812a839590edaf177b7398f700", size = 5463878 }, + { url = "https://files.pythonhosted.org/packages/e0/c8/8aed1f0ab9854ef48e5ad4431367fcb23ce73f0304f7b72335a8edc66556/zstandard-0.23.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:a8c86881813a78a6f4508ef9daf9d4995b8ac2d147dcb1a450448941398091c9", size = 4857192 }, + { url = "https://files.pythonhosted.org/packages/a8/c6/55e666cfbcd032b9e271865e8578fec56e5594d4faeac379d371526514f5/zstandard-0.23.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:fe3b385d996ee0822fd46528d9f0443b880d4d05528fd26a9119a54ec3f91c69", size = 4696513 }, + { url = "https://files.pythonhosted.org/packages/dc/bd/720b65bea63ec9de0ac7414c33b9baf271c8de8996e5ff324dc93fc90ff1/zstandard-0.23.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:82d17e94d735c99621bf8ebf9995f870a6b3e6d14543b99e201ae046dfe7de70", size = 5204823 }, + { url = "https://files.pythonhosted.org/packages/d8/40/d678db1556e3941d330cd4e95623a63ef235b18547da98fa184cbc028ecf/zstandard-0.23.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:c7c517d74bea1a6afd39aa612fa025e6b8011982a0897768a2f7c8ab4ebb78a2", size = 5666490 }, + { url = "https://files.pythonhosted.org/packages/ed/cc/c89329723d7515898a1fc7ef5d251264078548c505719d13e9511800a103/zstandard-0.23.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1fd7e0f1cfb70eb2f95a19b472ee7ad6d9a0a992ec0ae53286870c104ca939e5", size = 5196622 }, + { url = "https://files.pythonhosted.org/packages/78/4c/634289d41e094327a94500dfc919e58841b10ea3a9efdfafbac614797ec2/zstandard-0.23.0-cp39-cp39-win32.whl", hash = "sha256:43da0f0092281bf501f9c5f6f3b4c975a8a0ea82de49ba3f7100e64d422a1274", size = 430620 }, + { url = "https://files.pythonhosted.org/packages/a2/e2/0b0c5a0f4f7699fecd92c1ba6278ef9b01f2b0b0dd46f62bfc6729c05659/zstandard-0.23.0-cp39-cp39-win_amd64.whl", hash = "sha256:f8346bfa098532bc1fb6c7ef06783e969d87a99dd1d2a5a18a892c1d7a643c58", size = 495528 }, +] diff --git a/newsfragments/43612.significant.rst b/newsfragments/43612.significant.rst new file mode 100644 index 00000000000000..8c0ec597c11766 --- /dev/null +++ b/newsfragments/43612.significant.rst @@ -0,0 +1,8 @@ +Virtualenv installation uses ``uv`` now per default if ``uv`` is available. + +If you want to control how the virtualenv is created, you can use the +AIRFLOW__STANDARD__VENV_INSTALL_METHOD option. The possible values are: + +- ``auto``: Automatically select, use ``uv`` if available, otherwise use ``pip``. +- ``pip``: Use pip to install the virtual environment. +- ``uv``: Use uv to install the virtual environment. Must be available in environment PATH. diff --git a/providers/src/airflow/providers/standard/provider.yaml b/providers/src/airflow/providers/standard/provider.yaml index ee069165dfe38c..007cfd812b7afc 100644 --- a/providers/src/airflow/providers/standard/provider.yaml +++ b/providers/src/airflow/providers/standard/provider.yaml @@ -62,3 +62,20 @@ hooks: - airflow.providers.standard.hooks.filesystem - airflow.providers.standard.hooks.package_index - airflow.providers.standard.hooks.subprocess + +config: + standard: + description: Options for the standard provider operators. + options: + venv_install_method: + description: | + Which python tooling should be used to install the virtual environment. + + The following options are available: + - ``auto``: Automatically select, use ``uv`` if available, otherwise use ``pip``. + - ``pip``: Use pip to install the virtual environment. + - ``uv``: Use uv to install the virtual environment. Must be available in environment PATH. + version_added: ~ + type: string + example: uv + default: auto diff --git a/providers/src/airflow/providers/standard/utils/python_virtualenv.py b/providers/src/airflow/providers/standard/utils/python_virtualenv.py index b9c7be4c2832d7..9d03e43367a49e 100644 --- a/providers/src/airflow/providers/standard/utils/python_virtualenv.py +++ b/providers/src/airflow/providers/standard/utils/python_virtualenv.py @@ -20,15 +20,51 @@ from __future__ import annotations import os +import shutil import sys from pathlib import Path import jinja2 from jinja2 import select_autoescape +from airflow.configuration import conf from airflow.utils.process_utils import execute_in_subprocess +def _is_uv_installed() -> bool: + """ + Verify whether the uv tool is installed by checking if it's included in the system PATH or installed as a package. + + :return: True if it is. Whichever way of checking it works, is fine. + """ + return bool(shutil.which("uv")) + + +def _use_uv() -> bool: + """ + Check if the uv tool should be used. + + :return: True if uv should be used. + """ + venv_install_method = conf.get("standard", "venv_install_method", fallback="auto").lower() + if venv_install_method == "auto": + return _is_uv_installed() + elif venv_install_method == "uv": + return True + return False + + +def _generate_uv_cmd(tmp_dir: str, python_bin: str, system_site_packages: bool) -> list[str]: + """Build the command to install the venv via UV.""" + cmd = ["uv", "venv", "--allow-existing", "--seed"] + if python_bin is not None: + cmd += ["--python", python_bin] + if system_site_packages: + cmd.append("--system-site-packages") + cmd.append(tmp_dir) + return cmd + + def _generate_venv_cmd(tmp_dir: str, python_bin: str, system_site_packages: bool) -> list[str]: """We are using venv command instead of venv module to allow creation of venv for different python versions.""" if python_bin is None: @@ -39,12 +75,33 @@ def _generate_venv_cmd(tmp_dir: str, python_bin: str, system_site_packages: bool return cmd +def _generate_uv_install_cmd_from_file( + tmp_dir: str, requirements_file_path: str, pip_install_options: list[str] +) -> list[str]: + return [ + "uv", + "pip", + "install", + "--python", + f"{tmp_dir}/bin/python", + *pip_install_options, + "-r", + requirements_file_path, + ] + + def _generate_pip_install_cmd_from_file( tmp_dir: str, requirements_file_path: str, pip_install_options: list[str] ) -> list[str]: return [f"{tmp_dir}/bin/pip", "install", *pip_install_options, "-r", requirements_file_path] +def _generate_uv_install_cmd_from_list( + tmp_dir: str, requirements: list[str], pip_install_options: list[str] +) -> list[str]: + return ["uv", "pip", "install", "--python", f"{tmp_dir}/bin/python", *pip_install_options, *requirements] + + def _generate_pip_install_cmd_from_list( tmp_dir: str, requirements: list[str], pip_install_options: list[str] ) -> list[str]: @@ -88,22 +145,33 @@ def prepare_virtualenv( if pip_install_options is None: pip_install_options = [] + if requirements is not None and requirements_file_path is not None: + raise ValueError("Either requirements OR requirements_file_path has to be passed, but not both") + if index_urls is not None: _generate_pip_conf(Path(venv_directory) / "pip.conf", index_urls) - venv_cmd = _generate_venv_cmd(venv_directory, python_bin, system_site_packages) + if _use_uv(): + venv_cmd = _generate_uv_cmd(venv_directory, python_bin, system_site_packages) + else: + venv_cmd = _generate_venv_cmd(venv_directory, python_bin, system_site_packages) execute_in_subprocess(venv_cmd) - if requirements is not None and requirements_file_path is not None: - raise ValueError("Either requirements OR requirements_file_path has to be passed, but not both") - pip_cmd = None if requirements is not None and len(requirements) != 0: - pip_cmd = _generate_pip_install_cmd_from_list(venv_directory, requirements, pip_install_options) + if _use_uv(): + pip_cmd = _generate_uv_install_cmd_from_list(venv_directory, requirements, pip_install_options) + else: + pip_cmd = _generate_pip_install_cmd_from_list(venv_directory, requirements, pip_install_options) if requirements_file_path is not None and requirements_file_path: - pip_cmd = _generate_pip_install_cmd_from_file( - venv_directory, requirements_file_path, pip_install_options - ) + if _use_uv(): + pip_cmd = _generate_uv_install_cmd_from_file( + venv_directory, requirements_file_path, pip_install_options + ) + else: + pip_cmd = _generate_pip_install_cmd_from_file( + venv_directory, requirements_file_path, pip_install_options + ) if pip_cmd: execute_in_subprocess(pip_cmd) diff --git a/providers/tests/fab/auth_manager/api_endpoints/test_backfill_endpoint.py b/providers/tests/fab/auth_manager/api_endpoints/test_backfill_endpoint.py deleted file mode 100644 index d09f94c4653451..00000000000000 --- a/providers/tests/fab/auth_manager/api_endpoints/test_backfill_endpoint.py +++ /dev/null @@ -1,269 +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. -from __future__ import annotations - -import os -from datetime import datetime -from unittest import mock - -import pendulum -import pytest - -from airflow.models import DagBag, DagModel -from airflow.models.dag import DAG -from airflow.models.serialized_dag import SerializedDagModel -from airflow.operators.empty import EmptyOperator -from airflow.security import permissions -from airflow.utils import timezone -from airflow.utils.session import provide_session - -from providers.tests.fab.auth_manager.api_endpoints.api_connexion_utils import create_user, delete_user -from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS -from tests_common.test_utils.db import ( - clear_db_backfills, - clear_db_dags, - clear_db_runs, - clear_db_serialized_dags, -) - -try: - from airflow.models.backfill import Backfill -except ImportError: - if AIRFLOW_V_3_0_PLUS: - raise - else: - pass - -pytestmark = [ - pytest.mark.db_test, - pytest.mark.skipif(not AIRFLOW_V_3_0_PLUS, reason="Test requires Airflow 3.0+"), -] - - -DAG_ID = "test_dag" -TASK_ID = "op1" -DAG2_ID = "test_dag2" -DAG3_ID = "test_dag3" -UTC_JSON_REPR = "UTC" if pendulum.__version__.startswith("3") else "Timezone('UTC')" - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_auth_api): - app = minimal_app_for_auth_api - - create_user(app, username="test_granular_permissions", role_name="TestGranularDag") - app.appbuilder.sm.sync_perm_for_dag( - "TEST_DAG_1", - access_control={ - "TestGranularDag": { - permissions.RESOURCE_DAG: {permissions.ACTION_CAN_EDIT, permissions.ACTION_CAN_READ} - }, - }, - ) - - with DAG( - DAG_ID, - schedule=None, - start_date=datetime(2020, 6, 15), - doc_md="details", - params={"foo": 1}, - tags=["example"], - ) as dag: - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG2_ID, schedule=None, start_date=datetime(2020, 6, 15)) as dag2: # no doc_md - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG3_ID, schedule=None) as dag3: # DAG start_date set to None - EmptyOperator(task_id=TASK_ID, start_date=datetime(2019, 6, 12)) - - dag_bag = DagBag(os.devnull, include_examples=False) - dag_bag.dags = {dag.dag_id: dag, dag2.dag_id: dag2, dag3.dag_id: dag3} - - app.dag_bag = dag_bag - - yield app - - delete_user(app, username="test_granular_permissions") - - -class TestBackfillEndpoint: - @staticmethod - def clean_db(): - clear_db_backfills() - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.clean_db() - self.app = configured_app - self.client = self.app.test_client() # type:ignore - self.dag_id = DAG_ID - self.dag2_id = DAG2_ID - self.dag3_id = DAG3_ID - - def teardown_method(self) -> None: - self.clean_db() - - @provide_session - def _create_dag_models(self, *, count=1, dag_id_prefix="TEST_DAG", is_paused=False, session=None): - dags = [] - for num in range(1, count + 1): - dag_model = DagModel( - dag_id=f"{dag_id_prefix}_{num}", - fileloc=f"/tmp/dag_{num}.py", - is_active=True, - timetable_summary="0 0 * * *", - is_paused=is_paused, - ) - session.add(dag_model) - dags.append(dag_model) - return dags - - @provide_session - def _create_deactivated_dag(self, session=None): - dag_model = DagModel( - dag_id="TEST_DAG_DELETED_1", - fileloc="/tmp/dag_del_1.py", - schedule_interval="2 2 * * *", - is_active=False, - ) - session.add(dag_model) - - -class TestListBackfills(TestBackfillEndpoint): - def test_should_respond_200_with_granular_dag_access(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - b = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - - session.add(b) - session.commit() - kwargs = {} - kwargs.update(environ_overrides={"REMOTE_USER": "test_granular_permissions"}) - response = self.client.get("/api/v1/backfills?dag_id=TEST_DAG_1", **kwargs) - assert response.status_code == 200 - - -class TestGetBackfill(TestBackfillEndpoint): - def test_should_respond_200_with_granular_dag_access(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - kwargs.update(environ_overrides={"REMOTE_USER": "test_granular_permissions"}) - response = self.client.get(f"/api/v1/backfills/{backfill.id}", **kwargs) - assert response.status_code == 200 - - -class TestCreateBackfill(TestBackfillEndpoint): - def test_create_backfill(self, session, dag_maker): - with dag_maker(session=session, dag_id="TEST_DAG_1", schedule="0 * * * *") as dag: - EmptyOperator(task_id="mytask") - session.add(SerializedDagModel(dag)) - session.commit() - session.query(DagModel).all() - from_date = pendulum.parse("2024-01-01") - from_date_iso = from_date.isoformat() - to_date = pendulum.parse("2024-02-01") - to_date_iso = to_date.isoformat() - max_active_runs = 5 - data = { - "dag_id": dag.dag_id, - "from_date": f"{from_date_iso}", - "to_date": f"{to_date_iso}", - "max_active_runs": max_active_runs, - "reverse": False, - } - kwargs = {} - kwargs.update(environ_overrides={"REMOTE_USER": "test_granular_permissions"}) - - response = self.client.post( - "/api/v1/backfills", - **kwargs, - json=data, - ) - assert response.status_code == 200 - assert response.json == { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": None, - "from_date": from_date_iso, - "id": mock.ANY, - "is_paused": False, - "reprocess_behavior": "none", - "max_active_runs": 5, - "to_date": to_date_iso, - "updated_at": mock.ANY, - } - - -class TestPauseBackfill(TestBackfillEndpoint): - def test_should_respond_200_with_granular_dag_access(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - kwargs.update(environ_overrides={"REMOTE_USER": "test_granular_permissions"}) - response = self.client.post(f"/api/v1/backfills/{backfill.id}/pause", **kwargs) - assert response.status_code == 200 - - -class TestCancelBackfill(TestBackfillEndpoint): - def test_should_respond_200_with_granular_dag_access(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - kwargs.update(environ_overrides={"REMOTE_USER": "test_granular_permissions"}) - response = self.client.post(f"/api/v1/backfills/{backfill.id}/cancel", **kwargs) - assert response.status_code == 200 - # now it is marked as completed - assert pendulum.parse(response.json["completed_at"]) - - # get conflict when canceling already-canceled backfill - response = self.client.post(f"/api/v1/backfills/{backfill.id}/cancel", **kwargs) - assert response.status_code == 409 diff --git a/providers/tests/standard/utils/test_python_virtualenv.py b/providers/tests/standard/utils/test_python_virtualenv.py index da9cf757d74f23..0e10dcf5305ccc 100644 --- a/providers/tests/standard/utils/test_python_virtualenv.py +++ b/providers/tests/standard/utils/test_python_virtualenv.py @@ -22,11 +22,28 @@ import pytest -from airflow.providers.standard.utils.python_virtualenv import _generate_pip_conf, prepare_virtualenv +from airflow.providers.standard.utils.python_virtualenv import _generate_pip_conf, _use_uv, prepare_virtualenv from airflow.utils.decorators import remove_task_decorator +from tests_common.test_utils.config import conf_vars + class TestPrepareVirtualenv: + @mock.patch("shutil.which") + def test_use_uv(self, mock_shutil_which): + with conf_vars({("standard", "venv_install_method"): "auto"}): + mock_shutil_which.side_effect = [True] + assert _use_uv() is True + + mock_shutil_which.side_effect = [False] + assert _use_uv() is False + + with conf_vars({("standard", "venv_install_method"): "uv"}): + assert _use_uv() is True + + with conf_vars({("standard", "venv_install_method"): "pip"}): + assert _use_uv() is False + @pytest.mark.parametrize( ("index_urls", "expected_pip_conf_content", "unexpected_pip_conf_content"), [ @@ -60,7 +77,8 @@ def test_generate_pip_conf( assert term not in generated_conf @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") - def test_should_create_virtualenv(self, mock_execute_in_subprocess): + @conf_vars({("standard", "venv_install_method"): "pip"}) + def test_should_create_virtualenv_pip(self, mock_execute_in_subprocess): python_bin = prepare_virtualenv( venv_directory="/VENV", python_bin="pythonVER", system_site_packages=False, requirements=[] ) @@ -68,7 +86,19 @@ def test_should_create_virtualenv(self, mock_execute_in_subprocess): mock_execute_in_subprocess.assert_called_once_with(["pythonVER", "-m", "venv", "/VENV"]) @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") - def test_should_create_virtualenv_with_system_packages(self, mock_execute_in_subprocess): + @conf_vars({("standard", "venv_install_method"): "uv"}) + def test_should_create_virtualenv_uv(self, mock_execute_in_subprocess): + python_bin = prepare_virtualenv( + venv_directory="/VENV", python_bin="pythonVER", system_site_packages=False, requirements=[] + ) + assert "/VENV/bin/python" == python_bin + mock_execute_in_subprocess.assert_called_once_with( + ["uv", "venv", "--allow-existing", "--seed", "--python", "pythonVER", "/VENV"] + ) + + @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") + @conf_vars({("standard", "venv_install_method"): "pip"}) + def test_should_create_virtualenv_with_system_packages_pip(self, mock_execute_in_subprocess): python_bin = prepare_virtualenv( venv_directory="/VENV", python_bin="pythonVER", system_site_packages=True, requirements=[] ) @@ -78,7 +108,28 @@ def test_should_create_virtualenv_with_system_packages(self, mock_execute_in_sub ) @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") - def test_pip_install_options(self, mock_execute_in_subprocess): + @conf_vars({("standard", "venv_install_method"): "uv"}) + def test_should_create_virtualenv_with_system_packages_uv(self, mock_execute_in_subprocess): + python_bin = prepare_virtualenv( + venv_directory="/VENV", python_bin="pythonVER", system_site_packages=True, requirements=[] + ) + assert "/VENV/bin/python" == python_bin + mock_execute_in_subprocess.assert_called_once_with( + [ + "uv", + "venv", + "--allow-existing", + "--seed", + "--python", + "pythonVER", + "--system-site-packages", + "/VENV", + ] + ) + + @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") + @conf_vars({("standard", "venv_install_method"): "pip"}) + def test_pip_install_options_pip(self, mock_execute_in_subprocess): pip_install_options = ["--no-deps"] python_bin = prepare_virtualenv( venv_directory="/VENV", @@ -89,15 +140,30 @@ def test_pip_install_options(self, mock_execute_in_subprocess): ) assert "/VENV/bin/python" == python_bin - mock_execute_in_subprocess.assert_any_call( - ["pythonVER", "-m", "venv", "/VENV", "--system-site-packages"] - ) mock_execute_in_subprocess.assert_called_with( ["/VENV/bin/pip", "install", *pip_install_options, "apache-beam[gcp]"] ) @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") - def test_should_create_virtualenv_with_extra_packages(self, mock_execute_in_subprocess): + @conf_vars({("standard", "venv_install_method"): "uv"}) + def test_pip_install_options_uv(self, mock_execute_in_subprocess): + pip_install_options = ["--no-deps"] + python_bin = prepare_virtualenv( + venv_directory="/VENV", + python_bin="pythonVER", + system_site_packages=True, + requirements=["apache-beam[gcp]"], + pip_install_options=pip_install_options, + ) + + assert "/VENV/bin/python" == python_bin + mock_execute_in_subprocess.assert_called_with( + ["uv", "pip", "install", "--python", "/VENV/bin/python", *pip_install_options, "apache-beam[gcp]"] + ) + + @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") + @conf_vars({("standard", "venv_install_method"): "pip"}) + def test_should_create_virtualenv_with_extra_packages_pip(self, mock_execute_in_subprocess): python_bin = prepare_virtualenv( venv_directory="/VENV", python_bin="pythonVER", @@ -110,6 +176,21 @@ def test_should_create_virtualenv_with_extra_packages(self, mock_execute_in_subp mock_execute_in_subprocess.assert_called_with(["/VENV/bin/pip", "install", "apache-beam[gcp]"]) + @mock.patch("airflow.providers.standard.utils.python_virtualenv.execute_in_subprocess") + @conf_vars({("standard", "venv_install_method"): "uv"}) + def test_should_create_virtualenv_with_extra_packages_uv(self, mock_execute_in_subprocess): + python_bin = prepare_virtualenv( + venv_directory="/VENV", + python_bin="pythonVER", + system_site_packages=False, + requirements=["apache-beam[gcp]"], + ) + assert "/VENV/bin/python" == python_bin + + mock_execute_in_subprocess.assert_called_with( + ["uv", "pip", "install", "--python", "/VENV/bin/python", "apache-beam[gcp]"] + ) + def test_remove_task_decorator(self): py_source = "@task.virtualenv(use_dill=True)\ndef f():\nimport funcsigs" res = remove_task_decorator(python_source=py_source, task_decorator_name="@task.virtualenv") diff --git a/scripts/ci/docker-compose/providers-and-tests-sources.yml b/scripts/ci/docker-compose/providers-and-tests-sources.yml index 29498ae27eb942..8a06f2fcc0d1fc 100644 --- a/scripts/ci/docker-compose/providers-and-tests-sources.yml +++ b/scripts/ci/docker-compose/providers-and-tests-sources.yml @@ -21,6 +21,7 @@ services: tty: true # docker run -t environment: - AIRFLOW__CORE__PLUGINS_FOLDER=/files/plugins + - LINK_PROVIDERS_TO_AIRFLOW_PACKAGE=true # We only mount tests folder volumes: - ../../../.bash_aliases:/root/.bash_aliases:cached @@ -30,8 +31,8 @@ services: - ../../../empty:/opt/airflow/airflow # but keep tests - ../../../tests/:/opt/airflow/tests:cached - # and providers - - ../../../providers/src/airflow/providers:/opt/airflow/airflow/providers:cached + # Mount providers to make sure that we have the latest providers - both tests and sources + - ../../../providers/:/opt/airflow/providers:cached # and entrypoint and in_container scripts for testing - ../../../scripts/docker/entrypoint_ci.sh:/entrypoint - ../../../scripts/in_container/:/opt/airflow/scripts/in_container diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index cbd7bdce141ef0..8e864ba83121b7 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -236,6 +236,13 @@ function determine_airflow_to_use() { --constraint https://raw.githubusercontent.com/apache/airflow/constraints-main/constraints-${PYTHON_MAJOR_MINOR_VERSION}.txt # Some packages might leave legacy typing module which causes test issues pip uninstall -y typing || true + if [[ ${LINK_PROVIDERS_TO_AIRFLOW_PACKAGE=} == "true" ]]; then + echo + echo "${COLOR_BLUE}Linking providers to airflow package as we are using them from mounted sources.${COLOR_RESET}" + echo + rm -rf /usr/local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/providers + ln -s "${AIRFLOW_SOURCES}/providers/src/airflow/providers" "/usr/local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/providers" + fi fi if [[ "${USE_AIRFLOW_VERSION}" =~ ^2\.2\..*|^2\.1\..*|^2\.0\..* && "${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN=}" != "" ]]; then diff --git a/tests/api_connexion/endpoints/test_backfill_endpoint.py b/tests/api_connexion/endpoints/test_backfill_endpoint.py deleted file mode 100644 index 14a5078c539a33..00000000000000 --- a/tests/api_connexion/endpoints/test_backfill_endpoint.py +++ /dev/null @@ -1,442 +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. -from __future__ import annotations - -import os -from datetime import datetime -from unittest import mock - -import pendulum -import pytest - -from airflow.models import DagBag, DagModel -from airflow.models.backfill import Backfill, ReprocessBehavior -from airflow.models.dag import DAG -from airflow.operators.empty import EmptyOperator -from airflow.utils import timezone -from airflow.utils.session import provide_session - -from tests_common.test_utils.api_connexion_utils import create_user, delete_user -from tests_common.test_utils.db import ( - clear_db_backfills, - clear_db_dags, - clear_db_runs, - clear_db_serialized_dags, -) - -pytestmark = [pytest.mark.db_test, pytest.mark.need_serialized_dag] - - -DAG_ID = "test_dag" -TASK_ID = "op1" -DAG2_ID = "test_dag2" -DAG3_ID = "test_dag3" -UTC_JSON_REPR = "UTC" if pendulum.__version__.startswith("3") else "Timezone('UTC')" - - -def _clean_db(): - clear_db_backfills() - clear_db_runs() - clear_db_dags() - clear_db_serialized_dags() - - -@pytest.fixture(autouse=True) -def clean_db(): - _clean_db() - yield - _clean_db() - - -@pytest.fixture(scope="module") -def configured_app(minimal_app_for_api): - app = minimal_app_for_api - - create_user( - app, - username="test", - role_name="admin", - ) - create_user(app, username="test_no_permissions", role_name=None) - - with DAG( - DAG_ID, - schedule=None, - start_date=datetime(2020, 6, 15), - doc_md="details", - params={"foo": 1}, - tags=["example"], - ) as dag: - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG2_ID, schedule=None, start_date=datetime(2020, 6, 15)) as dag2: # no doc_md - EmptyOperator(task_id=TASK_ID) - - with DAG(DAG3_ID, schedule=None) as dag3: # DAG start_date set to None - EmptyOperator(task_id=TASK_ID, start_date=datetime(2019, 6, 12)) - - dag_bag = DagBag(os.devnull, include_examples=False) - dag_bag.dags = {dag.dag_id: dag, dag2.dag_id: dag2, dag3.dag_id: dag3} - - app.dag_bag = dag_bag - - yield app - - delete_user(app, username="test") - delete_user(app, username="test_no_permissions") - - -class TestBackfillEndpoint: - @pytest.fixture(autouse=True) - def setup_attrs(self, configured_app) -> None: - self.app = configured_app - self.client = self.app.test_client() # type:ignore - self.dag_id = DAG_ID - self.dag2_id = DAG2_ID - self.dag3_id = DAG3_ID - - @provide_session - def _create_dag_models(self, *, count=1, dag_id_prefix="TEST_DAG", is_paused=False, session=None): - dags = [] - for num in range(1, count + 1): - dag_model = DagModel( - dag_id=f"{dag_id_prefix}_{num}", - fileloc=f"/tmp/dag_{num}.py", - is_active=True, - timetable_summary="0 0 * * *", - is_paused=is_paused, - ) - session.add(dag_model) - dags.append(dag_model) - return dags - - @provide_session - def _create_deactivated_dag(self, session=None): - dag_model = DagModel( - dag_id="TEST_DAG_DELETED_1", - fileloc="/tmp/dag_del_1.py", - schedule_interval="2 2 * * *", - is_active=False, - ) - session.add(dag_model) - - -class TestListBackfills(TestBackfillEndpoint): - def test_should_respond_200(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - b = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) - session.add(b) - session.commit() - response = self.client.get( - f"/api/v1/backfills?dag_id={dag.dag_id}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "backfills": [ - { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": None, - "from_date": from_date.isoformat(), - "id": b.id, - "is_paused": False, - "reprocess_behavior": "none", - "max_active_runs": 10, - "to_date": to_date.isoformat(), - "updated_at": mock.ANY, - } - ], - "total_entries": 1, - } - - @pytest.mark.parametrize( - "user, expected", - [ - ("test_no_permissions", 403), - ("test", 200), - (None, 401), - ], - ) - def test_should_respond_200_with_granular_dag_access(self, user, expected, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - b = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - - session.add(b) - session.commit() - kwargs = {} - if user: - kwargs.update(environ_overrides={"REMOTE_USER": user}) - response = self.client.get("/api/v1/backfills?dag_id=TEST_DAG_1", **kwargs) - assert response.status_code == expected - - -class TestGetBackfill(TestBackfillEndpoint): - def test_should_respond_200(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) - session.add(backfill) - session.commit() - response = self.client.get( - f"/api/v1/backfills/{backfill.id}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": None, - "from_date": from_date.isoformat(), - "id": backfill.id, - "is_paused": False, - "reprocess_behavior": "none", - "max_active_runs": 10, - "to_date": to_date.isoformat(), - "updated_at": mock.ANY, - } - - def test_no_exist(self, session): - response = self.client.get( - f"/api/v1/backfills/{23198409834208}", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 404 - assert response.json.get("title") == "Backfill not found" - - @pytest.mark.parametrize( - "user, expected", - [ - ("test_no_permissions", 403), - ("test", 200), - (None, 401), - ], - ) - def test_should_respond_200_with_granular_dag_access(self, user, expected, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - if user: - kwargs.update(environ_overrides={"REMOTE_USER": user}) - response = self.client.get(f"/api/v1/backfills/{backfill.id}", **kwargs) - assert response.status_code == expected - - -class TestCreateBackfill(TestBackfillEndpoint): - @pytest.mark.parametrize( - "user, expected", - [ - ("test_no_permissions", 403), - ("test", 200), - (None, 401), - ], - ) - @pytest.mark.parametrize( - "repro_act, repro_exp", - [ - (None, ReprocessBehavior.NONE), - ("none", ReprocessBehavior.NONE), - ("failed", ReprocessBehavior.FAILED), - ("completed", ReprocessBehavior.COMPLETED), - ], - ) - def test_create_backfill(self, repro_act, repro_exp, user, expected, session, dag_maker): - if repro_act is not None and expected > 300: - pytest.skip("this combination not needed") - with dag_maker(session=session, dag_id="TEST_DAG_1", schedule="0 * * * *") as dag: - EmptyOperator(task_id="mytask") - session.query(DagModel).all() - from_date = pendulum.parse("2024-01-01") - from_date_iso = from_date.isoformat() - to_date = pendulum.parse("2024-02-01") - to_date_iso = to_date.isoformat() - max_active_runs = 5 - data = { - "dag_id": dag.dag_id, - "from_date": f"{from_date_iso}", - "to_date": f"{to_date_iso}", - "max_active_runs": max_active_runs, - "reverse": False, - "dag_run_conf": {"param1": "val1", "param2": True}, - } - if repro_act is not None: - data["reprocess_behavior"] = repro_act - kwargs = {} - if user: - kwargs.update(environ_overrides={"REMOTE_USER": user}) - - response = self.client.post( - "/api/v1/backfills", - json=data, - **kwargs, - ) - assert response.status_code == expected - if expected < 300: - assert response.json == { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": {"param1": "val1", "param2": True}, - "from_date": from_date_iso, - "id": mock.ANY, - "is_paused": False, - "reprocess_behavior": repro_exp, - "max_active_runs": 5, - "to_date": to_date_iso, - "updated_at": mock.ANY, - } - - -class TestPauseBackfill(TestBackfillEndpoint): - def test_should_respond_200(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) - session.add(backfill) - session.commit() - response = self.client.post( - f"/api/v1/backfills/{backfill.id}/pause", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": None, - "from_date": from_date.isoformat(), - "id": backfill.id, - "is_paused": True, - "reprocess_behavior": "none", - "max_active_runs": 10, - "to_date": to_date.isoformat(), - "updated_at": mock.ANY, - } - - @pytest.mark.parametrize( - "user, expected", - [ - ("test_no_permissions", 403), - ("test", 200), - (None, 401), - ], - ) - def test_should_respond_200_with_granular_dag_access(self, user, expected, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - if user: - kwargs.update(environ_overrides={"REMOTE_USER": user}) - response = self.client.post(f"/api/v1/backfills/{backfill.id}/pause", **kwargs) - assert response.status_code == expected - - -class TestCancelBackfill(TestBackfillEndpoint): - def test_should_respond_200(self, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) - session.add(backfill) - session.commit() - response = self.client.post( - f"/api/v1/backfills/{backfill.id}/cancel", - environ_overrides={"REMOTE_USER": "test"}, - ) - assert response.status_code == 200 - assert response.json == { - "completed_at": mock.ANY, - "created_at": mock.ANY, - "dag_id": "TEST_DAG_1", - "dag_run_conf": None, - "from_date": from_date.isoformat(), - "id": backfill.id, - "is_paused": True, - "reprocess_behavior": "none", - "max_active_runs": 10, - "to_date": to_date.isoformat(), - "updated_at": mock.ANY, - } - assert pendulum.parse(response.json["completed_at"]) - # now it is marked as completed - assert pendulum.parse(response.json["completed_at"]) - - # get conflict when canceling already-canceled backfill - response = self.client.post( - f"/api/v1/backfills/{backfill.id}/cancel", environ_overrides={"REMOTE_USER": "test"} - ) - assert response.status_code == 409 - - @pytest.mark.parametrize( - "user, expected", - [ - ("test_no_permissions", 403), - ("test", 200), - (None, 401), - ], - ) - def test_should_respond_200_with_granular_dag_access(self, user, expected, session): - (dag,) = self._create_dag_models() - from_date = timezone.utcnow() - to_date = timezone.utcnow() - backfill = Backfill( - dag_id=dag.dag_id, - from_date=from_date, - to_date=to_date, - ) - session.add(backfill) - session.commit() - kwargs = {} - if user: - kwargs.update(environ_overrides={"REMOTE_USER": user}) - response = self.client.post(f"/api/v1/backfills/{backfill.id}/cancel", **kwargs) - assert response.status_code == expected - if response.status_code < 300: - # now it is marked as completed - assert pendulum.parse(response.json["completed_at"]) - - # get conflict when canceling already-canceled backfill - response = self.client.post(f"/api/v1/backfills/{backfill.id}/cancel", **kwargs) - assert response.status_code == 409 diff --git a/tests/api_connexion/schemas/test_backfill_schema.py b/tests/api_connexion/schemas/test_backfill_schema.py deleted file mode 100644 index b9c4e91a59f81b..00000000000000 --- a/tests/api_connexion/schemas/test_backfill_schema.py +++ /dev/null @@ -1,55 +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. -from __future__ import annotations - -from airflow.api_connexion.schemas.backfill_schema import BackfillCollection, backfill_collection_schema -from airflow.models.backfill import Backfill -from airflow.utils import timezone - - -class TestBackfillSchema: - def test_serialize_direct(self): - now = timezone.utcnow() - now_iso = now.isoformat() - b1 = Backfill( - dag_id="hi", - created_at=now, - completed_at=now, - from_date=now, - to_date=now, - updated_at=now, - ) - bc = BackfillCollection(backfills=[b1], total_entries=1) - out = backfill_collection_schema.dump(bc) - assert out == { - "backfills": [ - { - "completed_at": now_iso, - "created_at": now_iso, - "dag_id": "hi", - "dag_run_conf": None, - "from_date": now_iso, - "id": None, - "is_paused": None, - "reprocess_behavior": None, - "max_active_runs": None, - "to_date": now_iso, - "updated_at": now_iso, - } - ], - "total_entries": 1, - } diff --git a/tests/api_fastapi/core_api/routes/public/test_backfills.py b/tests/api_fastapi/core_api/routes/public/test_backfills.py new file mode 100644 index 00000000000000..1c64b10848f4b6 --- /dev/null +++ b/tests/api_fastapi/core_api/routes/public/test_backfills.py @@ -0,0 +1,309 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import os +from datetime import datetime +from unittest import mock + +import pendulum +import pytest +from sqlalchemy import select + +from airflow.models import DagBag, DagModel, DagRun +from airflow.models.backfill import Backfill, BackfillDagRun, ReprocessBehavior, _create_backfill +from airflow.models.dag import DAG +from airflow.operators.empty import EmptyOperator +from airflow.providers.standard.operators.python import PythonOperator +from airflow.utils import timezone +from airflow.utils.session import provide_session +from airflow.utils.state import DagRunState + +from tests_common.test_utils.db import ( + clear_db_backfills, + clear_db_dags, + clear_db_runs, + clear_db_serialized_dags, +) + +pytestmark = [pytest.mark.db_test, pytest.mark.need_serialized_dag] + + +DAG_ID = "test_dag" +TASK_ID = "op1" +DAG2_ID = "test_dag2" +DAG3_ID = "test_dag3" + + +def _clean_db(): + clear_db_backfills() + clear_db_runs() + clear_db_dags() + clear_db_serialized_dags() + + +@pytest.fixture(autouse=True) +def clean_db(): + _clean_db() + yield + _clean_db() + + +def make_dags(): + with DAG( + DAG_ID, + schedule=None, + start_date=datetime(2020, 6, 15), + doc_md="details", + params={"foo": 1}, + tags=["example"], + ) as dag: + EmptyOperator(task_id=TASK_ID) + + with DAG(DAG2_ID, schedule=None, start_date=datetime(2020, 6, 15)) as dag2: # no doc_md + EmptyOperator(task_id=TASK_ID) + + with DAG(DAG3_ID, schedule=None) as dag3: # DAG start_date set to None + EmptyOperator(task_id=TASK_ID, start_date=datetime(2019, 6, 12)) + + dag_bag = DagBag(os.devnull, include_examples=False) + dag_bag.dags = {dag.dag_id: dag, dag2.dag_id: dag2, dag3.dag_id: dag3} + + +def to_iso(val): + return pendulum.instance(val).to_iso8601_string() + + +class TestBackfillEndpoint: + @provide_session + def _create_dag_models(self, *, count=1, dag_id_prefix="TEST_DAG", is_paused=False, session=None): + dags = [] + for num in range(1, count + 1): + dag_model = DagModel( + dag_id=f"{dag_id_prefix}_{num}", + fileloc=f"/tmp/dag_{num}.py", + is_active=True, + timetable_summary="0 0 * * *", + is_paused=is_paused, + ) + session.add(dag_model) + dags.append(dag_model) + return dags + + +class TestListBackfills(TestBackfillEndpoint): + def test_list_backfill(self, test_client, session): + (dag,) = self._create_dag_models() + from_date = timezone.utcnow() + to_date = timezone.utcnow() + b = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) + session.add(b) + session.commit() + response = test_client.get(f"/public/backfills?dag_id={dag.dag_id}") + assert response.status_code == 200 + assert response.json() == { + "backfills": [ + { + "completed_at": mock.ANY, + "created_at": mock.ANY, + "dag_id": "TEST_DAG_1", + "dag_run_conf": {}, + "from_date": to_iso(from_date), + "id": b.id, + "is_paused": False, + "reprocess_behavior": "none", + "max_active_runs": 10, + "to_date": to_iso(to_date), + "updated_at": mock.ANY, + } + ], + "total_entries": 1, + } + + +class TestGetBackfill(TestBackfillEndpoint): + def test_get_backfill(self, session, test_client): + (dag,) = self._create_dag_models() + from_date = timezone.utcnow() + to_date = timezone.utcnow() + backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) + session.add(backfill) + session.commit() + response = test_client.get(f"/public/backfills/{backfill.id}") + assert response.status_code == 200 + assert response.json() == { + "completed_at": mock.ANY, + "created_at": mock.ANY, + "dag_id": "TEST_DAG_1", + "dag_run_conf": {}, + "from_date": to_iso(from_date), + "id": backfill.id, + "is_paused": False, + "reprocess_behavior": "none", + "max_active_runs": 10, + "to_date": to_iso(to_date), + "updated_at": mock.ANY, + } + + def test_no_exist(self, session, test_client): + response = test_client.get(f"/public/backfills/{231984098}") + assert response.status_code == 404 + assert response.json().get("detail") == "Backfill not found" + + +class TestCreateBackfill(TestBackfillEndpoint): + @pytest.mark.parametrize( + "repro_act, repro_exp", + [ + (None, ReprocessBehavior.NONE), + ("none", ReprocessBehavior.NONE), + ("failed", ReprocessBehavior.FAILED), + ("completed", ReprocessBehavior.COMPLETED), + ], + ) + def test_create_backfill(self, repro_act, repro_exp, session, dag_maker, test_client): + with dag_maker(session=session, dag_id="TEST_DAG_1", schedule="0 * * * *") as dag: + EmptyOperator(task_id="mytask") + session.query(DagModel).all() + session.commit() + from_date = pendulum.parse("2024-01-01") + from_date_iso = to_iso(from_date) + to_date = pendulum.parse("2024-02-01") + to_date_iso = to_iso(to_date) + max_active_runs = 5 + data = { + "dag_id": dag.dag_id, + "from_date": f"{from_date_iso}", + "to_date": f"{to_date_iso}", + "max_active_runs": max_active_runs, + "run_backwards": False, + "dag_run_conf": {"param1": "val1", "param2": True}, + } + if repro_act is not None: + data["reprocess_behavior"] = repro_act + response = test_client.post( + url="/public/backfills", + json=data, + ) + assert response.status_code == 200 + assert response.json() == { + "completed_at": mock.ANY, + "created_at": mock.ANY, + "dag_id": "TEST_DAG_1", + "dag_run_conf": {"param1": "val1", "param2": True}, + "from_date": from_date_iso, + "id": mock.ANY, + "is_paused": False, + "reprocess_behavior": repro_exp, + "max_active_runs": 5, + "to_date": to_date_iso, + "updated_at": mock.ANY, + } + + +class TestCancelBackfill(TestBackfillEndpoint): + def test_cancel_backfill(self, session, test_client): + (dag,) = self._create_dag_models() + from_date = timezone.utcnow() + to_date = timezone.utcnow() + backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) + session.add(backfill) + session.commit() + response = test_client.put( + f"/public/backfills/{backfill.id}/cancel", + ) + assert response.status_code == 200 + assert response.json() == { + "completed_at": mock.ANY, + "created_at": mock.ANY, + "dag_id": "TEST_DAG_1", + "dag_run_conf": {}, + "from_date": to_iso(from_date), + "id": backfill.id, + "is_paused": True, + "reprocess_behavior": "none", + "max_active_runs": 10, + "to_date": to_iso(to_date), + "updated_at": mock.ANY, + } + assert pendulum.parse(response.json()["completed_at"]) + # now it is marked as completed + assert pendulum.parse(response.json()["completed_at"]) + + # get conflict when canceling already-canceled backfill + response = test_client.put(f"/public/backfills/{backfill.id}/cancel") + assert response.status_code == 409 + + def test_cancel_backfill_end_states(self, dag_maker, session, test_client): + """ + Queued runs should be marked *failed*. + Every other dag run should be left alone. + """ + with dag_maker(schedule="@daily") as dag: + PythonOperator(task_id="hi", python_callable=print) + b = _create_backfill( + dag_id=dag.dag_id, + from_date=timezone.datetime(2021, 1, 1), + to_date=timezone.datetime(2021, 1, 5), + max_active_runs=2, + reverse=False, + dag_run_conf={}, + ) + query = ( + select(DagRun) + .join(BackfillDagRun.dag_run) + .where(BackfillDagRun.backfill_id == b.id) + .order_by(BackfillDagRun.sort_ordinal) + ) + dag_runs = session.scalars(query).all() + dates = [str(x.logical_date.date()) for x in dag_runs] + expected_dates = ["2021-01-01", "2021-01-02", "2021-01-03", "2021-01-04", "2021-01-05"] + assert dates == expected_dates + assert all(x.state == DagRunState.QUEUED for x in dag_runs) + dag_runs[0].state = "running" + session.commit() + response = test_client.put(f"/public/backfills/{b.id}/cancel") + assert response.status_code == 200 + session.expunge_all() + dag_runs = session.scalars(query).all() + states = [x.state for x in dag_runs] + assert states == ["running", "failed", "failed", "failed", "failed"] + + +class TestPauseBackfill(TestBackfillEndpoint): + def test_pause_backfill(self, session, test_client): + (dag,) = self._create_dag_models() + from_date = timezone.utcnow() + to_date = timezone.utcnow() + backfill = Backfill(dag_id=dag.dag_id, from_date=from_date, to_date=to_date) + session.add(backfill) + session.commit() + response = test_client.put(f"/public/backfills/{backfill.id}/pause") + assert response.status_code == 200 + assert response.json() == { + "completed_at": mock.ANY, + "created_at": mock.ANY, + "dag_id": "TEST_DAG_1", + "dag_run_conf": {}, + "from_date": to_iso(from_date), + "id": backfill.id, + "is_paused": True, + "reprocess_behavior": "none", + "max_active_runs": 10, + "to_date": to_iso(to_date), + "updated_at": mock.ANY, + } diff --git a/tests/api_fastapi/core_api/routes/public/test_dag_warning.py b/tests/api_fastapi/core_api/routes/public/test_dag_warning.py new file mode 100644 index 00000000000000..61237bd10299a2 --- /dev/null +++ b/tests/api_fastapi/core_api/routes/public/test_dag_warning.py @@ -0,0 +1,85 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +import pytest + +from airflow.models.dag import DagModel +from airflow.models.dagwarning import DagWarning +from airflow.utils.session import provide_session + +from tests_common.test_utils.db import clear_db_dag_warnings, clear_db_dags + +pytestmark = pytest.mark.db_test + +DAG1_ID = "test_dag1" +DAG1_MESSAGE = "test message 1" +DAG2_ID = "test_dag2" +DAG2_MESSAGE = "test message 2" +DAG3_ID = "test_dag3" +DAG3_MESSAGE = "test message 3" +DAG_WARNING_TYPE = "non-existent pool" + + +@pytest.fixture(autouse=True) +@provide_session +def setup(dag_maker, session=None) -> None: + clear_db_dags() + clear_db_dag_warnings() + + session.add(DagModel(dag_id=DAG1_ID)) + session.add(DagModel(dag_id=DAG2_ID)) + session.add(DagModel(dag_id=DAG3_ID)) + session.add(DagWarning(DAG1_ID, DAG_WARNING_TYPE, DAG1_MESSAGE)) + session.add(DagWarning(DAG2_ID, DAG_WARNING_TYPE, DAG2_MESSAGE)) + session.add(DagWarning(DAG3_ID, DAG_WARNING_TYPE, DAG3_MESSAGE)) + session.commit() + + +class TestGetDagWarnings: + @pytest.mark.parametrize( + "query_params, expected_total_entries, expected_messages", + [ + ({}, 3, [DAG1_MESSAGE, DAG2_MESSAGE, DAG3_MESSAGE]), + ({"dag_id": DAG1_ID}, 1, [DAG1_MESSAGE]), + ({"warning_type": DAG_WARNING_TYPE}, 3, [DAG1_MESSAGE, DAG2_MESSAGE, DAG3_MESSAGE]), + ({"limit": 1, "order_by": "message"}, 3, [DAG1_MESSAGE]), + ({"limit": 1, "offset": 1, "order_by": "message"}, 3, [DAG2_MESSAGE]), + ({"limit": 1, "offset": 2, "order_by": "dag_id"}, 3, [DAG3_MESSAGE]), + ({"limit": 1, "offset": 2, "order_by": "-dag_id"}, 3, [DAG1_MESSAGE]), + ({"limit": 1, "order_by": "timestamp"}, 3, [DAG1_MESSAGE]), + ({"limit": 1, "order_by": "-timestamp"}, 3, [DAG3_MESSAGE]), + ({"order_by": "timestamp"}, 3, [DAG1_MESSAGE, DAG2_MESSAGE, DAG3_MESSAGE]), + ({"order_by": "-timestamp"}, 3, [DAG3_MESSAGE, DAG2_MESSAGE, DAG1_MESSAGE]), + ({"order_by": "dag_id"}, 3, [DAG1_MESSAGE, DAG2_MESSAGE, DAG3_MESSAGE]), + ({"order_by": "-dag_id"}, 3, [DAG3_MESSAGE, DAG2_MESSAGE, DAG1_MESSAGE]), + ], + ) + def test_get_dag_warnings(self, test_client, query_params, expected_total_entries, expected_messages): + response = test_client.get("/public/dagWarnings", params=query_params) + assert response.status_code == 200 + response_json = response.json() + assert response_json["total_entries"] == expected_total_entries + assert len(response_json["dag_warnings"]) == len(expected_messages) + assert [dag_warning["message"] for dag_warning in response_json["dag_warnings"]] == expected_messages + + def test_get_dag_warnings_bad_request(self, test_client): + response = test_client.get("/public/dagWarnings", params={"warning_type": "invalid"}) + response_json = response.json() + assert response.status_code == 422 + assert response_json["detail"][0]["msg"] == "Input should be 'asset conflict' or 'non-existent pool'" diff --git a/tests/api_fastapi/core_api/routes/public/test_task_instances.py b/tests/api_fastapi/core_api/routes/public/test_task_instances.py index 717f17ca278ab8..0f27abd567e2b8 100644 --- a/tests/api_fastapi/core_api/routes/public/test_task_instances.py +++ b/tests/api_fastapi/core_api/routes/public/test_task_instances.py @@ -19,6 +19,7 @@ import datetime as dt import urllib +from unittest import mock import pendulum import pytest @@ -180,6 +181,7 @@ def test_should_respond_200(self, test_client, session): "executor": None, "executor_config": "{}", "hostname": "", + "id": mock.ANY, "map_index": -1, "max_tries": 0, "note": "placeholder-note", @@ -237,6 +239,7 @@ def test_should_respond_200_with_task_state_in_deferred(self, test_client, sessi "executor": None, "executor_config": "{}", "hostname": "", + "id": mock.ANY, "map_index": -1, "max_tries": 0, "note": "placeholder-note", @@ -283,6 +286,7 @@ def test_should_respond_200_with_task_state_in_removed(self, test_client, sessio "executor": None, "executor_config": "{}", "hostname": "", + "id": mock.ANY, "map_index": -1, "max_tries": 0, "note": "placeholder-note", @@ -325,6 +329,7 @@ def test_should_respond_200_task_instance_with_rendered(self, test_client, sessi "executor": None, "executor_config": "{}", "hostname": "", + "id": mock.ANY, "map_index": -1, "max_tries": 0, "note": "placeholder-note", @@ -424,6 +429,7 @@ def test_should_respond_200_mapped_task_instance_with_rtif(self, test_client, se "executor": None, "executor_config": "{}", "hostname": "", + "id": mock.ANY, "map_index": map_index, "max_tries": 0, "note": "placeholder-note", diff --git a/tests/models/test_backfill.py b/tests/models/test_backfill.py index a67590e91a57f5..a32db07808e824 100644 --- a/tests/models/test_backfill.py +++ b/tests/models/test_backfill.py @@ -32,7 +32,6 @@ BackfillDagRun, BackfillDagRunExceptionReason, ReprocessBehavior, - _cancel_backfill, _create_backfill, ) from airflow.providers.standard.operators.python import PythonOperator @@ -323,41 +322,6 @@ def test_active_dag_run(dag_maker, session): ) -def test_cancel_backfill(dag_maker, session): - """ - Queued runs should be marked *failed*. - Every other dag run should be left alone. - """ - with dag_maker(schedule="@daily") as dag: - PythonOperator(task_id="hi", python_callable=print) - b = _create_backfill( - dag_id=dag.dag_id, - from_date=timezone.datetime(2021, 1, 1), - to_date=timezone.datetime(2021, 1, 5), - max_active_runs=2, - reverse=False, - dag_run_conf={}, - ) - query = ( - select(DagRun) - .join(BackfillDagRun.dag_run) - .where(BackfillDagRun.backfill_id == b.id) - .order_by(BackfillDagRun.sort_ordinal) - ) - dag_runs = session.scalars(query).all() - dates = [str(x.logical_date.date()) for x in dag_runs] - expected_dates = ["2021-01-01", "2021-01-02", "2021-01-03", "2021-01-04", "2021-01-05"] - assert dates == expected_dates - assert all(x.state == DagRunState.QUEUED for x in dag_runs) - dag_runs[0].state = "running" - session.commit() - _cancel_backfill(backfill_id=b.id) - session.expunge_all() - dag_runs = session.scalars(query).all() - states = [x.state for x in dag_runs] - assert states == ["running", "failed", "failed", "failed", "failed"] - - def create_next_run( *, is_backfill: bool, next_date: datetime, dag_id: str, dag_maker, reprocess=None, session: Session ):