diff --git a/.github/ISSUE_TEMPLATE/1-airflow_bug_report.yml b/.github/ISSUE_TEMPLATE/1-airflow_bug_report.yml index a9c8978740344..e5060dbe3671d 100644 --- a/.github/ISSUE_TEMPLATE/1-airflow_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/1-airflow_bug_report.yml @@ -192,6 +192,7 @@ body: - redis - salesforce - samba + - sdk-java - segment - sendgrid - sftp diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index 21b44a9fb840f..5bd46a1c9dbcb 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -189,6 +189,9 @@ labelPRBasedOnFilePath: provider:keycloak: - providers/keycloak/** + provider:sdk-java: + - providers/sdk/java/** + provider:microsoft-azure: - providers/microsoft/azure/** @@ -261,6 +264,9 @@ labelPRBasedOnFilePath: provider:samba: - providers/samba/** + provider:sdk-executable: + - providers/sdk/executable/** + provider:segment: - providers/segment/** diff --git a/airflow-core/docs/extra-packages-ref.rst b/airflow-core/docs/extra-packages-ref.rst index 2646b0a7c3079..9fb579c9b08ec 100644 --- a/airflow-core/docs/extra-packages-ref.rst +++ b/airflow-core/docs/extra-packages-ref.rst @@ -178,6 +178,17 @@ all the ``airflow`` packages together - similarly to what happened in Airflow 2. ``airflow-task-sdk`` separately, if you want to install providers, you need to install them separately as ``apache-airflow-providers-*`` distribution packages. +Multi-Language extras +===================== + +These are extras that add dependencies needed for integration with other languages runtimes. Currently we have only Java SDK related extra, but in the future we might add more extras related to other languages runtimes. + ++----------+------------------------------------------+------------------------------------------------------------------+ +| extra | install command | enables | ++==========+==========================================+==================================================================+ +| sdk.java | ``pip install apache-airflow[sdk.java]`` | JavaCoordinator for both dag processing and workload execution. | ++----------+------------------------------------------+------------------------------------------------------------------+ + Apache Software extras ====================== diff --git a/airflow-core/pyproject.toml b/airflow-core/pyproject.toml index a875d9e9c77e7..0d4d45d7e40db 100644 --- a/airflow-core/pyproject.toml +++ b/airflow-core/pyproject.toml @@ -249,6 +249,7 @@ exclude = [ "../shared/serialization/src/airflow_shared/serialization" = "src/airflow/_shared/serialization" "../shared/state/src/airflow_shared/state" = "src/airflow/_shared/state" "../shared/timezones/src/airflow_shared/timezones" = "src/airflow/_shared/timezones" +"../shared/workloads/src/airflow_shared/workloads" = "src/airflow/_shared/workloads" "../shared/listeners/src/airflow_shared/listeners" = "src/airflow/_shared/listeners" "../shared/plugins_manager/src/airflow_shared/plugins_manager" = "src/airflow/_shared/plugins_manager" "../shared/providers_discovery/src/airflow_shared/providers_discovery" = "src/airflow/_shared/providers_discovery" @@ -337,6 +338,7 @@ shared_distributions = [ "apache-airflow-shared-serialization", "apache-airflow-shared-state", "apache-airflow-shared-timezones", + "apache-airflow-shared-workloads", "apache-airflow-shared-plugins-manager", "apache-airflow-shared-providers-discovery", ] diff --git a/airflow-core/src/airflow/_shared/workloads b/airflow-core/src/airflow/_shared/workloads new file mode 120000 index 0000000000000..f25d9e16ea1d9 --- /dev/null +++ b/airflow-core/src/airflow/_shared/workloads @@ -0,0 +1 @@ +../../../../shared/workloads/src/airflow_shared/workloads \ No newline at end of file diff --git a/airflow-core/src/airflow/config_templates/config.yml b/airflow-core/src/airflow/config_templates/config.yml index 8a060007978ee..e7812fcfa3b00 100644 --- a/airflow-core/src/airflow/config_templates/config.yml +++ b/airflow-core/src/airflow/config_templates/config.yml @@ -1967,6 +1967,21 @@ workers: type: integer example: ~ default: "60" +sdk: + description: Settings for non-Python SDK runtime coordination + options: + queue_to_sdk: + description: | + JSON mapping of queue names to SDK runtime coordinator names. + + When a task's ``language`` field is not set, this mapping is checked + to route the task to a non-Python runtime coordinator based on its + queue. This is useful when queues are used as environment or + isolation identifiers (e.g. ``foo``, ``bar``). + version_added: 3.1.7 + type: string + example: '{"foo": "java", "bar": "java", "go-queue": "go"}' + default: "{{}}" api_auth: description: Settings relating to authentication on the Airflow APIs options: diff --git a/airflow-core/src/airflow/dag_processing/manager.py b/airflow-core/src/airflow/dag_processing/manager.py index 8d497ca7508e3..c2dab6e3fe6d5 100644 --- a/airflow-core/src/airflow/dag_processing/manager.py +++ b/airflow-core/src/airflow/dag_processing/manager.py @@ -66,7 +66,7 @@ from airflow.sdk import SecretCache from airflow.sdk.log import init_log_file, logging_processors from airflow.typing_compat import assert_never -from airflow.utils.file import list_py_file_paths, might_contain_dag +from airflow.utils.file import might_contain_dag from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.net import get_hostname from airflow.utils.process_utils import ( @@ -88,6 +88,9 @@ from airflow.sdk.api.client import Client +log = logging.getLogger(__name__) + + class DagParsingStat(NamedTuple): """Information on processing progress.""" @@ -158,6 +161,62 @@ def utc_epoch() -> datetime: return result +def discover_dag_file_paths( + directory: str | os.PathLike[str] | None, + bundle_name: str = "", + safe_mode: bool = conf.getboolean("core", "DAG_DISCOVERY_SAFE_MODE", fallback=True), +) -> list[str]: + """ + Discover paths of DAG files within a directory. + + Walks ``directory`` (honouring ``.airflowignore``) and returns each file that is + either a Python DAG candidate (``.py`` source or ZIP archive that passes + :func:`~airflow.utils.file.might_contain_dag`) or accepted by a registered coordinator's + :meth:`~airflow.sdk.execution_time.coordinator.BaseCoordinator.can_handle_dag_file` + (e.g. a ``.jar`` for the Java SDK, a self-contained executable for the Go SDK). + + Coordinator handling takes precedence over the generic ZIP heuristic so that, + for example, a ``.jar`` is delegated to its coordinator rather than being + scanned for embedded ``.py`` modules. + + :param directory: Directory to scan, or a single file path. ``None`` returns + an empty list. A single file is returned as-is without filtering. + :param bundle_name: Bundle name forwarded to ``can_handle_dag_file``. + :param safe_mode: Whether to apply the Python DAG heuristic; see + :func:`~airflow.utils.file.might_contain_dag`. + :return: Absolute paths discovered as DAG sources. + """ + if directory is None: + return [] + if os.path.isfile(directory): + return [str(directory)] + if not os.path.isdir(directory): + return [] + + from airflow._shared.module_loading.file_discovery import find_path_from_directory + from airflow.providers_manager import ProvidersManager + + coordinators = ProvidersManager().coordinators + ignore_file_syntax = conf.get_mandatory_value("core", "DAG_IGNORE_FILE_SYNTAX", fallback="glob") + + file_paths: list[str] = [] + for file_path in find_path_from_directory(directory, ".airflowignore", ignore_file_syntax): + path = Path(file_path) + try: + if not path.is_file(): + continue + if path.suffix == ".py": + if might_contain_dag(file_path, safe_mode): + file_paths.append(file_path) + elif any(c.can_handle_dag_file(bundle_name, file_path) for c in coordinators): + file_paths.append(file_path) + elif zipfile.is_zipfile(path) and might_contain_dag(file_path, safe_mode): + file_paths.append(file_path) + except Exception: + log.exception("Error while examining %s", file_path) + return file_paths + + class _StubSelector(selectors.BaseSelector): """ Stub to stand in until the real selector is created. @@ -808,9 +867,11 @@ def _refresh_dag_bundles(self, known_files: dict[str, set[DagFileInfo]]): def _find_files_in_bundle(self, bundle: BaseDagBundle) -> list[Path]: """Get relative paths for dag files from bundle dir.""" - # Build up a list of Python files that could contain DAGs self.log.info("Searching for files in %s at %s", bundle.name, bundle.path) - rel_paths = [Path(x).relative_to(bundle.path) for x in list_py_file_paths(bundle.path)] + rel_paths = [ + Path(x).relative_to(bundle.path) + for x in discover_dag_file_paths(bundle.path, bundle_name=bundle.name) + ] self.log.info("Found %s files for bundle %s", len(rel_paths), bundle.name) return rel_paths @@ -822,7 +883,13 @@ def _get_observed_filelocs(self, present: set[DagFileInfo]) -> set[str]: For regular files this includes the relative file path. For ZIP archives this includes DAG-like inner paths such as ``archive.zip/dag.py``. + + Files claimed by a registered runtime coordinator (e.g. ``.jar``) + are treated as opaque files rather than ZIP archives. """ + from airflow.providers_manager import ProvidersManager + + coordinators = ProvidersManager().coordinators def find_zipped_dags(abs_path: os.PathLike) -> Iterator[str]: """Yield absolute paths for DAG-like files inside a ZIP archive.""" @@ -837,7 +904,10 @@ def find_zipped_dags(abs_path: os.PathLike) -> Iterator[str]: observed_filelocs: set[str] = set() for info in present: abs_path = str(info.absolute_path) - if abs_path.endswith(".py") or not zipfile.is_zipfile(abs_path): + handled_by_coordinator = any( + c.can_handle_dag_file(info.bundle_name, abs_path) for c in coordinators + ) + if abs_path.endswith(".py") or handled_by_coordinator or not zipfile.is_zipfile(abs_path): observed_filelocs.add(str(info.rel_path)) else: if TYPE_CHECKING: diff --git a/airflow-core/src/airflow/dag_processing/processor.py b/airflow-core/src/airflow/dag_processing/processor.py index aa9f07411f87d..4733bf26db21e 100644 --- a/airflow-core/src/airflow/dag_processing/processor.py +++ b/airflow-core/src/airflow/dag_processing/processor.py @@ -17,6 +17,7 @@ from __future__ import annotations import contextlib +import functools import importlib import logging import os @@ -76,8 +77,6 @@ from airflow.utils.state import TaskInstanceState if TYPE_CHECKING: - from socket import socket - from structlog.typing import FilteringBoundLogger from airflow.api_fastapi.execution_api.app import InProcessExecutionAPI @@ -86,6 +85,7 @@ from airflow.sdk.definitions.context import Context from airflow.sdk.definitions.dag import DAG from airflow.sdk.definitions.mappedoperator import MappedOperator + from airflow.sdk.execution_time.supervisor import SelectorCallback from airflow.typing_compat import Self @@ -553,7 +553,14 @@ def start( # type: ignore[override] ) -> Self: logger = kwargs["logger"] - _pre_import_airflow_modules(os.fspath(path), logger) + # Check if a provider-registered runtime coordinator should handle this file + logger.debug("Checking for provider-registered runtime coordinator entrypoint for file", path=path) + resolved_target = cls._resolve_processor_target(path, bundle_name, bundle_path, logger) + if resolved_target is not None: + target = resolved_target + logger.debug("Resolved provider-registered runtime coordinator entrypoint for file", path=path) + else: + _pre_import_airflow_modules(os.fspath(path), logger) proc: Self = super().start( target=target, @@ -566,6 +573,53 @@ def start( # type: ignore[override] proc._on_child_started(callbacks, path, bundle_path, bundle_name) return proc + @staticmethod + def _resolve_processor_target( + path: str | os.PathLike[str], + bundle_name: str, + bundle_path: Path, + log: FilteringBoundLogger, + ) -> Callable[[], None] | None: + """ + Return the entrypoint of the first provider runtime coordinator that can handle *path*. + + The returned callable is a ``functools.partial`` that binds *path*, *bundle_name* + and *bundle_path* so the supervisor can pass it as a no-arg ``target`` to + ``WatchedSubprocess.start``. + """ + from airflow.providers_manager import ProvidersManager + + for coordinator_cls in ProvidersManager().coordinators: + try: + log.debug( + "Checking runtime coordinator %s for file %s", + coordinator_cls, + path, + ) + if coordinator_cls.can_handle_dag_file(bundle_name, path): + log.debug( + "Using runtime coordinator %s for file %s", + coordinator_cls, + path, + ) + return functools.partial( + coordinator_cls.run_dag_parsing, + path=os.fspath(path), + bundle_name=bundle_name, + bundle_path=os.fspath(bundle_path), + ) + log.debug( + "Runtime coordinator %s cannot handle file %s with bundle name %s", + coordinator_cls, + path, + bundle_name, + ) + except Exception: + log.warning("Failed to check runtime coordinator %s", coordinator_cls, exc_info=True) + + log.debug("No runtime coordinator found for file %s, using default processor", path) + return None + def _on_child_started( self, callbacks: list[CallbackRequest], @@ -591,7 +645,7 @@ def _get_target_loggers(self) -> tuple[FilteringBoundLogger, ...]: def _create_log_forwarder( self, loggers: tuple[FilteringBoundLogger, ...], name: str, log_level: int = logging.INFO - ) -> Callable[[socket], bool]: + ) -> SelectorCallback: return super()._create_log_forwarder(loggers, name.replace("task.", "dag_processor.", 1), log_level) def _handle_request(self, msg: ToManager, log: FilteringBoundLogger, req_id: int) -> None: diff --git a/airflow-core/src/airflow/executors/base_executor.py b/airflow-core/src/airflow/executors/base_executor.py index 9c9487c5377bc..b2a284ad1cc36 100644 --- a/airflow-core/src/airflow/executors/base_executor.py +++ b/airflow-core/src/airflow/executors/base_executor.py @@ -651,10 +651,8 @@ def run_workload( if isinstance(workload, ExecuteTask): from airflow.sdk.execution_time.supervisor import supervise_task - # workload.ti is a TaskInstanceDTO which duck-types as TaskInstance. - # TODO: Create a protocol for this. return supervise_task( - ti=workload.ti, # type: ignore[arg-type] + ti=workload.ti, bundle_info=workload.bundle_info, dag_rel_path=workload.dag_rel_path, token=workload.token, diff --git a/airflow-core/src/airflow/executors/workloads/task.py b/airflow-core/src/airflow/executors/workloads/task.py index d05affe433096..89a171d9244df 100644 --- a/airflow-core/src/airflow/executors/workloads/task.py +++ b/airflow-core/src/airflow/executors/workloads/task.py @@ -22,8 +22,9 @@ from pathlib import Path from typing import TYPE_CHECKING, Literal -from pydantic import BaseModel, Field +from pydantic import Field +from airflow._shared.workloads import TaskInstanceDTO as _BaseTaskInstanceDTO from airflow.executors.workloads.base import BaseDagBundleWorkload, BundleInfo from airflow.utils.state import TaskInstanceState @@ -33,8 +34,13 @@ from airflow.models.taskinstancekey import TaskInstanceKey -class TaskInstanceDTO(BaseModel): - """Schema for TaskInstance with minimal required fields needed for Executors and Task SDK.""" +class TaskInstanceDTO(_BaseTaskInstanceDTO): + """ + TaskInstanceDTO with executor-specific ``key`` property. + + Extends the shared :class:`~airflow._shared.workloads.TaskInstanceDTO` + to add the :attr:`key` property used by executors for workload tracking. + """ id: uuid.UUID dag_version_id: uuid.UUID diff --git a/airflow-core/src/airflow/models/dagcode.py b/airflow-core/src/airflow/models/dagcode.py index 60ee91c8b59b5..528859f4cd311 100644 --- a/airflow-core/src/airflow/models/dagcode.py +++ b/airflow-core/src/airflow/models/dagcode.py @@ -119,6 +119,16 @@ def code(cls, dag_id, session: Session = NEW_SESSION) -> str: @staticmethod def get_code_from_file(fileloc): + # Try from runtime coordinator first (classes are pre-loaded by ProvidersManager) + from airflow.providers_manager import ProvidersManager + + for coordinator_cls in ProvidersManager().coordinators: + # TODO: Perhaps the `can_handle_dag_file` interface should just accept `path` only? + # Or maybe we can have different granularity for this. that 1 with bundle + path, another with just path + if coordinator_cls.can_handle_dag_file("", fileloc): + return coordinator_cls.get_code_from_file(fileloc) + + # Then fallback to python native try: with open_maybe_zipped(fileloc, "r") as f: code = f.read() diff --git a/airflow-core/src/airflow/provider.yaml.schema.json b/airflow-core/src/airflow/provider.yaml.schema.json index 5714b8db658c5..1c41b906289cf 100644 --- a/airflow-core/src/airflow/provider.yaml.schema.json +++ b/airflow-core/src/airflow/provider.yaml.schema.json @@ -624,6 +624,13 @@ } } }, + "coordinators": { + "type": "array", + "description": "Runtime Coordinator class names (BaseCoordinator subclasses)", + "items": { + "type": "string" + } + }, "source-date-epoch": { "type": "integer", "description": "Source date epoch - seconds since epoch (gmtime) when the release documentation was prepared. Used to generate reproducible package builds with flint.", diff --git a/airflow-core/src/airflow/provider_info.schema.json b/airflow-core/src/airflow/provider_info.schema.json index 86fc726a05168..92601fc58af74 100644 --- a/airflow-core/src/airflow/provider_info.schema.json +++ b/airflow-core/src/airflow/provider_info.schema.json @@ -446,6 +446,13 @@ "type": "string" } } + }, + "coordinators": { + "type": "array", + "description": "Runtime Coordinator class names (BaseCoordinator subclasses)", + "items": { + "type": "string" + } } }, "definitions": { diff --git a/airflow-core/src/airflow/providers_manager.py b/airflow-core/src/airflow/providers_manager.py index 6fefcbc39b06d..8945589b4b046 100644 --- a/airflow-core/src/airflow/providers_manager.py +++ b/airflow-core/src/airflow/providers_manager.py @@ -41,6 +41,7 @@ if TYPE_CHECKING: from airflow.cli.cli_config import CLICommand + from airflow.sdk.execution_time.coordinator import BaseCoordinator log = logging.getLogger(__name__) @@ -448,6 +449,7 @@ def __init__(self): ) # Set of plugins contained in providers self._plugins_set: set[PluginInfo] = set() + self._coordinators: list[type[BaseCoordinator]] = [] self._init_airflow_core_hooks() self._runtime_manager = None @@ -625,6 +627,12 @@ def initialize_providers_configuration(self): self.initialize_providers_list() self._discover_config() + @provider_info_cache("coordinators") + def initialize_providers_coordinators(self): + """Lazy initialization of providers runtime coordinators.""" + self.initialize_providers_list() + self._discover_coordinators() + @provider_info_cache("plugins") def initialize_providers_plugins(self): self.initialize_providers_list() @@ -1280,6 +1288,19 @@ def _discover_config(self) -> None: if provider.data.get("config"): self._provider_configs[provider_package] = provider.data.get("config") # type: ignore[assignment] + def _discover_coordinators(self) -> None: + """Retrieve and pre-load all coordinators defined in the providers.""" + seen: set[str] = set() + for provider_package, provider in self._provider_dict.items(): + for coordinator_class_path in provider.data.get("coordinators", []): + if coordinator_class_path in seen: + continue + coordinator_cls = _correctness_check(provider_package, coordinator_class_path, provider) + if coordinator_cls: + seen.add(coordinator_class_path) + self._coordinators.append(coordinator_cls) + self._coordinators = sorted(self._coordinators, key=lambda c: c.__qualname__) + def _discover_plugins(self) -> None: """Retrieve all plugins defined in the providers.""" for provider_package, provider in self._provider_dict.items(): @@ -1477,6 +1498,12 @@ def db_managers(self) -> list[str]: self.initialize_providers_db_managers() return sorted(self._db_manager_class_name_set) + @property + def coordinators(self) -> list[type[BaseCoordinator]]: + """Returns pre-loaded coordinator classes available in providers.""" + self.initialize_providers_coordinators() + return self._coordinators + @property def filesystem_module_names(self) -> list[str]: self.initialize_providers_filesystems() @@ -1548,6 +1575,7 @@ def _cleanup(self): self._trigger_info_set.clear() self._notification_info_set.clear() self._plugins_set.clear() + self._coordinators.clear() self._cli_command_functions_set.clear() self._cli_command_provider_name_set.clear() diff --git a/airflow-core/src/airflow/serialization/definitions/baseoperator.py b/airflow-core/src/airflow/serialization/definitions/baseoperator.py index 6bafc5891235a..9eaf9cc3ed906 100644 --- a/airflow-core/src/airflow/serialization/definitions/baseoperator.py +++ b/airflow-core/src/airflow/serialization/definitions/baseoperator.py @@ -195,6 +195,7 @@ def get_serialized_fields(cls): "ignore_first_depends_on_past", "inlets", "is_setup", + "sdk", "is_teardown", "map_index_template", "max_active_tis_per_dag", diff --git a/airflow-core/src/airflow/utils/file.py b/airflow-core/src/airflow/utils/file.py index c614cfff0ad96..25e191cdccfd8 100644 --- a/airflow-core/src/airflow/utils/file.py +++ b/airflow-core/src/airflow/utils/file.py @@ -19,7 +19,6 @@ import ast import hashlib -import logging import os import re import zipfile @@ -30,8 +29,6 @@ from airflow.configuration import conf -log = logging.getLogger(__name__) - MODIFIED_DAG_MODULE_NAME = "unusual_prefix_{path_hash}_{module_name}" @@ -74,49 +71,6 @@ def open_maybe_zipped(fileloc, mode="r"): return open(fileloc, mode=mode) -def list_py_file_paths( - directory: str | os.PathLike[str] | None, - safe_mode: bool = conf.getboolean("core", "DAG_DISCOVERY_SAFE_MODE", fallback=True), -) -> list[str]: - """ - Traverse a directory and look for Python files. - - :param directory: the directory to traverse - :param safe_mode: whether to use a heuristic to determine whether a file - contains Airflow DAG definitions. If not provided, use the - core.DAG_DISCOVERY_SAFE_MODE configuration setting. If not set, default - to safe. - :return: a list of paths to Python files in the specified directory - """ - file_paths: list[str] = [] - if directory is None: - file_paths = [] - elif os.path.isfile(directory): - file_paths = [str(directory)] - elif os.path.isdir(directory): - file_paths.extend(find_dag_file_paths(directory, safe_mode)) - return file_paths - - -def find_dag_file_paths(directory: str | os.PathLike[str], safe_mode: bool) -> list[str]: - """Find file paths of all DAG files.""" - from airflow._shared.module_loading.file_discovery import find_path_from_directory - - file_paths = [] - ignore_file_syntax = conf.get_mandatory_value("core", "DAG_IGNORE_FILE_SYNTAX", fallback="glob") - - for file_path in find_path_from_directory(directory, ".airflowignore", ignore_file_syntax): - path = Path(file_path) - try: - if path.is_file() and (path.suffix == ".py" or zipfile.is_zipfile(path)): - if might_contain_dag(file_path, safe_mode): - file_paths.append(file_path) - except Exception: - log.exception("Error while examining %s", file_path) - - return file_paths - - COMMENT_PATTERN = re.compile(r"\s*#.*") diff --git a/airflow-core/tests/unit/always/test_providers_manager.py b/airflow-core/tests/unit/always/test_providers_manager.py index afa473e80a4f0..b13930c98d1c2 100644 --- a/airflow-core/tests/unit/always/test_providers_manager.py +++ b/airflow-core/tests/unit/always/test_providers_manager.py @@ -258,6 +258,34 @@ def test_dialects(self): assert len(dialect_class_names) == 3 assert dialect_class_names == ["default", "mssql", "postgresql"] + @patch("airflow.providers_manager.import_string") + def test_coordinators(self, mock_import_string): + class ACoordinator: + pass + + class ZCoordinator: + pass + + mock_import_string.side_effect = lambda path: { + "airflow.providers.sdk.java.coordinator.ACoordinator": ACoordinator, + "airflow.providers.sdk.java.coordinator.ZCoordinator": ZCoordinator, + }[path] + providers_manager = ProvidersManager() + providers_manager._provider_dict = LazyDictWithCache() + providers_manager._provider_dict["apache-airflow-providers-sdk-java"] = ProviderInfo( + version="0.0.1", + data={ + "coordinators": [ + "airflow.providers.sdk.java.coordinator.ZCoordinator", + "airflow.providers.sdk.java.coordinator.ACoordinator", + "airflow.providers.sdk.java.coordinator.ZCoordinator", + ] + }, + ) + + with patch.object(providers_manager, "initialize_providers_list"): + assert providers_manager.coordinators == [ACoordinator, ZCoordinator] + class TestWithoutCheckProviderManager: @pytest.fixture(autouse=True) diff --git a/airflow-core/tests/unit/dag_processing/test_manager.py b/airflow-core/tests/unit/dag_processing/test_manager.py index c10e73473bfef..281a378b26c1e 100644 --- a/airflow-core/tests/unit/dag_processing/test_manager.py +++ b/airflow-core/tests/unit/dag_processing/test_manager.py @@ -31,6 +31,7 @@ from collections import defaultdict, deque from datetime import datetime, timedelta from pathlib import Path +from pprint import pformat from socket import socket, socketpair from unittest import mock from unittest.mock import MagicMock @@ -41,6 +42,7 @@ from sqlalchemy import func, select from uuid6 import uuid7 +from airflow._shared.module_loading import find_path_from_directory from airflow._shared.timezones import timezone from airflow.callbacks.callback_requests import DagCallbackRequest from airflow.dag_processing.bundles.base import BaseDagBundle @@ -51,6 +53,7 @@ DagFileInfo, DagFileProcessorManager, DagFileStat, + discover_dag_file_paths, ) from airflow.dag_processing.processor import DagFileParsingResult, DagFileProcessorProcess from airflow.models import DagModel, DbCallbackRequest @@ -60,6 +63,7 @@ from airflow.models.dagcode import DagCode from airflow.models.serialized_dag import SerializedDagModel from airflow.models.team import Team +from airflow.utils import file as file_utils from airflow.utils.net import get_hostname from airflow.utils.session import create_session @@ -85,6 +89,11 @@ DEFAULT_DATE = timezone.datetime(2016, 1, 1) +def might_contain_dag(file_path: str, zip_file: zipfile.ZipFile | None = None): + """Custom callable injected via conf_vars in TestDagFileDiscovery.test_might_contain_dag.""" + return False + + def _get_file_infos(files: list[str | Path]) -> list[DagFileInfo]: return [DagFileInfo(bundle_name="testing", bundle_path=TEST_DAGS_FOLDER, rel_path=Path(f)) for f in files] @@ -114,6 +123,34 @@ def encode_mtime_in_filename(val): return out +class _FakeCoordinator: + """Test double recording every can_handle_dag_file call and matching by extension.""" + + file_extension: str = ".fakeext" + invocations: list[tuple[str, str]] = [] + + @classmethod + def reset(cls) -> None: + cls.invocations = [] + + @classmethod + def can_handle_dag_file(cls, bundle_name: str, path) -> bool: + cls.invocations.append((bundle_name, str(path))) + return str(path).endswith(cls.file_extension) + + +@pytest.fixture +def fake_coordinator(): + """Inject a fake coordinator into ProvidersManager.coordinators for the duration of a test.""" + _FakeCoordinator.reset() + with mock.patch( + "airflow.providers_manager.ProvidersManager.coordinators", + new_callable=mock.PropertyMock, + return_value=[_FakeCoordinator], + ): + yield _FakeCoordinator + + def _create_zip_bundle_with_valid_and_broken_dags(zip_path: Path) -> None: with zipfile.ZipFile(zip_path, "w") as zf: zf.writestr( @@ -285,6 +322,43 @@ def test_get_observed_filelocs_expands_zip_inner_paths(self, tmp_path): "test_zip.zip/broken_dag.py", } + def test_get_observed_filelocs_treats_coordinator_handled_zip_as_opaque(self, tmp_path, fake_coordinator): + """A coordinator-claimed file that happens to be a ZIP must NOT be expanded into inner paths.""" + # Coordinator handles ".fakeext"; the file is a real ZIP archive so + # without the coordinator check it would be enumerated like a dag-zip. + bundle_file = tmp_path / "bundle.fakeext" + _create_zip_bundle_with_valid_and_broken_dags(bundle_file) + + manager = DagFileProcessorManager(max_runs=1) + observed_filelocs = manager._get_observed_filelocs( + { + DagFileInfo( + bundle_name="testing", + rel_path=Path("bundle.fakeext"), + bundle_path=tmp_path, + ) + } + ) + + assert observed_filelocs == {"bundle.fakeext"} + + def test_get_observed_filelocs_forwards_bundle_name_to_coordinator(self, tmp_path, fake_coordinator): + bundle_file = tmp_path / "bundle.fakeext" + bundle_file.write_bytes(b"opaque payload") + + manager = DagFileProcessorManager(max_runs=1) + manager._get_observed_filelocs( + { + DagFileInfo( + bundle_name="my_bundle", + rel_path=Path("bundle.fakeext"), + bundle_path=tmp_path, + ) + } + ) + + assert fake_coordinator.invocations == [("my_bundle", str(bundle_file))] + @pytest.mark.usefixtures("clear_parse_import_errors") def test_refresh_dag_bundles_keeps_zip_inner_file_errors(self, session, tmp_path, configure_dag_bundles): bundle_path = tmp_path / "bundleone" @@ -2462,3 +2536,162 @@ def test_refresh_dag_bundles_update_bundle_state_failure_still_scans_files(self) # _bundle_versions must NOT advance — DB still holds the old version, so the next # iteration will see a version mismatch and re-refresh rather than skip incorrectly assert "mock_bundle" not in manager._bundle_versions + + +class TestDagFileDiscovery: + def test_find_path_from_directory_regex_ignore(self): + should_ignore = [ + "test_invalid_cron.py", + "test_invalid_param.py", + "test_ignore_this.py", + ] + files = find_path_from_directory(TEST_DAGS_FOLDER, ".airflowignore") + + assert files + assert all(os.path.basename(file) not in should_ignore for file in files) + + def test_find_path_from_directory_glob_ignore(self): + should_ignore = { + "should_ignore_this.py", + "test_explicit_ignore.py", + "test_invalid_cron.py", + "test_invalid_param.py", + "test_ignore_this.py", + "test_prev_dagrun_dep.py", + "test_nested_dag.py", + ".airflowignore", + } + should_not_ignore = { + "test_on_kill.py", + "test_negate_ignore.py", + "test_dont_ignore_this.py", + "test_nested_negate_ignore.py", + "test_explicit_dont_ignore.py", + } + actual_files = list(find_path_from_directory(TEST_DAGS_FOLDER, ".airflowignore_glob", "glob")) + + assert actual_files + assert all(os.path.basename(file) not in should_ignore for file in actual_files) + actual_included_filenames = { + os.path.basename(f) for f in actual_files if os.path.basename(f) in should_not_ignore + } + assert actual_included_filenames == should_not_ignore, ( + f"actual_included_filenames: {pformat(actual_included_filenames)}\nexpected_included_filenames: {pformat(should_not_ignore)}" + ) + + def test_might_contain_dag_with_default_callable(self): + file_path_with_dag = os.path.join(TEST_DAGS_FOLDER, "test_scheduler_dags.py") + + assert file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=True) + + @conf_vars({("core", "might_contain_dag_callable"): "unit.dag_processing.test_manager.might_contain_dag"}) + def test_might_contain_dag(self): + """Test might_contain_dag_callable""" + file_path_with_dag = os.path.join(TEST_DAGS_FOLDER, "test_scheduler_dags.py") + + # There is a DAG defined in the file_path_with_dag, however, the might_contain_dag_callable + # returns False no matter what, which is used to test might_contain_dag_callable actually + # overrides the default function + assert not file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=True) + + # With safe_mode is False, the user defined callable won't be invoked + assert file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=False) + + def test_get_modules(self): + file_path = os.path.join(TEST_DAGS_FOLDER, "test_imports.py") + + modules = list(file_utils.iter_airflow_imports(file_path)) + + assert len(modules) == 4 + assert "airflow.utils" in modules + assert "airflow.decorators" in modules + assert "airflow.models" in modules + assert "airflow.sensors" in modules + # this one is a local import, we don't want it. + assert "airflow.local_import" not in modules + # this one is in a comment, we don't want it + assert "airflow.in_comment" not in modules + # we don't want imports under conditions + assert "airflow.if_branch" not in modules + assert "airflow.else_branch" not in modules + + def test_get_modules_from_invalid_file(self): + file_path = os.path.join(TEST_DAGS_FOLDER, "README.md") # just getting a non-python file + + # should not error + modules = list(file_utils.iter_airflow_imports(file_path)) + + assert len(modules) == 0 + + def test_discover_dag_file_paths(self, test_zip_path): + expected_files = set() + # No_dags is empty, _invalid_ is ignored by .airflowignore + ignored_files = { + "no_dags.py", + "should_ignore_this.py", + "test_explicit_ignore.py", + "test_invalid_cron.py", + "test_invalid_dup_task.py", + "test_ignore_this.py", + "test_invalid_param.py", + "test_invalid_param2.py", + "test_invalid_param3.py", + "test_invalid_param4.py", + "test_nested_dag.py", + "test_imports.py", + "test_nested_negate_ignore.py", + "file_no_airflow_dag.py", # no_dag test case in test_zip folder + "test.py", # no_dag test case in test_zip_module folder + "__init__.py", + } + for root, _, files in os.walk(TEST_DAGS_FOLDER): + for file_name in files: + if file_name.endswith((".py", ".zip")): + if file_name not in ignored_files: + expected_files.add(f"{root}/{file_name}") + detected_files = set(discover_dag_file_paths(str(TEST_DAGS_FOLDER))) + assert detected_files == expected_files, ( + f"Detected files mismatched expected files:\ndetected_files: {pformat(detected_files)}\nexpected_files: {pformat(expected_files)}" + ) + + def test_discover_returns_empty_for_none(self): + assert discover_dag_file_paths(None) == [] + + def test_discover_returns_empty_for_missing_path(self, tmp_path): + assert discover_dag_file_paths(tmp_path / "does_not_exist") == [] + + def test_discover_returns_single_file_as_is(self, tmp_path): + single = tmp_path / "anything.bin" + single.write_bytes(b"opaque") + assert discover_dag_file_paths(single) == [str(single)] + + def test_discover_includes_coordinator_handled_files(self, tmp_path, fake_coordinator): + coord_file = tmp_path / "bundle.fakeext" + coord_file.write_bytes(b"opaque payload") + py_file = tmp_path / "dag.py" + py_file.write_text("from airflow.sdk import DAG\nDAG('d')") + + assert set(discover_dag_file_paths(tmp_path)) == {str(coord_file), str(py_file)} + + def test_discover_coordinator_takes_precedence_over_zip_heuristic(self, tmp_path, fake_coordinator): + """A coordinator-claimed file that is also a ZIP must NOT also be included via the generic ZIP path.""" + coord_zip = tmp_path / "bundle.fakeext" + _create_zip_bundle_with_valid_and_broken_dags(coord_zip) + + # File appears exactly once: claimed by coordinator, generic zip branch skipped. + assert discover_dag_file_paths(tmp_path) == [str(coord_zip)] + + def test_discover_forwards_bundle_name_to_coordinator(self, tmp_path, fake_coordinator): + coord_file = tmp_path / "bundle.fakeext" + coord_file.write_bytes(b"opaque payload") + + discover_dag_file_paths(tmp_path, bundle_name="my_bundle") + + # Only one non-.py file, so exactly one coordinator invocation, with the bundle name. + assert fake_coordinator.invocations == [("my_bundle", str(coord_file))] + + def test_discover_skips_non_matching_unknown_file(self, tmp_path, fake_coordinator): + """A file no coordinator claims and that isn't .py / a ZIP must not appear in results.""" + (tmp_path / "random.bin").write_bytes(b"unknown payload") + + assert discover_dag_file_paths(tmp_path) == [] diff --git a/airflow-core/tests/unit/models/test_dag.py b/airflow-core/tests/unit/models/test_dag.py index b34ab12dd4aef..5666ad9dd3212 100644 --- a/airflow-core/tests/unit/models/test_dag.py +++ b/airflow-core/tests/unit/models/test_dag.py @@ -41,6 +41,7 @@ from airflow._shared.timezones.timezone import datetime as datetime_tz from airflow.configuration import conf from airflow.dag_processing.dagbag import BundleDagBag, DagBag +from airflow.dag_processing.manager import discover_dag_file_paths from airflow.exceptions import AirflowException from airflow.models.asset import ( AssetAliasModel, @@ -91,7 +92,6 @@ NullTimetable, OnceTimetable, ) -from airflow.utils.file import list_py_file_paths from airflow.utils.session import create_session from airflow.utils.state import DagRunState, State, TaskInstanceState from airflow.utils.types import DagRunTriggeredByType, DagRunType @@ -1135,7 +1135,7 @@ def test_dag_is_deactivated_upon_dagfile_deletion(self, dag_maker): DagModel.deactivate_deleted_dags( bundle_name=orm_dag.bundle_name, - rel_filelocs=list_py_file_paths(settings.DAGS_FOLDER), + rel_filelocs=discover_dag_file_paths(settings.DAGS_FOLDER), ) orm_dag = session.scalar(select(DagModel).where(DagModel.dag_id == dag_id)) diff --git a/airflow-core/tests/unit/utils/test_file.py b/airflow-core/tests/unit/utils/test_file.py index cc55c1ac0632e..a1e14d22a45f0 100644 --- a/airflow-core/tests/unit/utils/test_file.py +++ b/airflow-core/tests/unit/utils/test_file.py @@ -18,29 +18,18 @@ from __future__ import annotations import os -import zipfile -from pprint import pformat from unittest import mock import pytest -from airflow._shared.module_loading import find_path_from_directory from airflow.utils import file as file_utils from airflow.utils.file import ( correct_maybe_zipped, - list_py_file_paths, open_maybe_zipped, ) -from tests_common.test_utils.config import conf_vars from unit.models import TEST_DAGS_FOLDER -TEST_DAG_FOLDER = os.environ["AIRFLOW__CORE__DAGS_FOLDER"] - - -def might_contain_dag(file_path: str, zip_file: zipfile.ZipFile | None = None): - return False - class TestCorrectMaybeZipped: @mock.patch("zipfile.is_zipfile") @@ -95,124 +84,6 @@ def test_open_maybe_zipped_archive(self, test_zip_path): assert isinstance(content, str) -class TestListPyFilesPath: - def test_find_path_from_directory_regex_ignore(self): - should_ignore = [ - "test_invalid_cron.py", - "test_invalid_param.py", - "test_ignore_this.py", - ] - files = find_path_from_directory(TEST_DAGS_FOLDER, ".airflowignore") - - assert files - assert all(os.path.basename(file) not in should_ignore for file in files) - - def test_find_path_from_directory_glob_ignore(self): - should_ignore = { - "should_ignore_this.py", - "test_explicit_ignore.py", - "test_invalid_cron.py", - "test_invalid_param.py", - "test_ignore_this.py", - "test_prev_dagrun_dep.py", - "test_nested_dag.py", - ".airflowignore", - } - should_not_ignore = { - "test_on_kill.py", - "test_negate_ignore.py", - "test_dont_ignore_this.py", - "test_nested_negate_ignore.py", - "test_explicit_dont_ignore.py", - } - actual_files = list(find_path_from_directory(TEST_DAGS_FOLDER, ".airflowignore_glob", "glob")) - - assert actual_files - assert all(os.path.basename(file) not in should_ignore for file in actual_files) - actual_included_filenames = set( - [os.path.basename(f) for f in actual_files if os.path.basename(f) in should_not_ignore] - ) - assert actual_included_filenames == should_not_ignore, ( - f"actual_included_filenames: {pformat(actual_included_filenames)}\nexpected_included_filenames: {pformat(should_not_ignore)}" - ) - - def test_might_contain_dag_with_default_callable(self): - file_path_with_dag = os.path.join(TEST_DAGS_FOLDER, "test_scheduler_dags.py") - - assert file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=True) - - @conf_vars({("core", "might_contain_dag_callable"): "unit.utils.test_file.might_contain_dag"}) - def test_might_contain_dag(self): - """Test might_contain_dag_callable""" - file_path_with_dag = os.path.join(TEST_DAGS_FOLDER, "test_scheduler_dags.py") - - # There is a DAG defined in the file_path_with_dag, however, the might_contain_dag_callable - # returns False no matter what, which is used to test might_contain_dag_callable actually - # overrides the default function - assert not file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=True) - - # With safe_mode is False, the user defined callable won't be invoked - assert file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=False) - - def test_get_modules(self): - file_path = os.path.join(TEST_DAGS_FOLDER, "test_imports.py") - - modules = list(file_utils.iter_airflow_imports(file_path)) - - assert len(modules) == 4 - assert "airflow.utils" in modules - assert "airflow.decorators" in modules - assert "airflow.models" in modules - assert "airflow.sensors" in modules - # this one is a local import, we don't want it. - assert "airflow.local_import" not in modules - # this one is in a comment, we don't want it - assert "airflow.in_comment" not in modules - # we don't want imports under conditions - assert "airflow.if_branch" not in modules - assert "airflow.else_branch" not in modules - - def test_get_modules_from_invalid_file(self): - file_path = os.path.join(TEST_DAGS_FOLDER, "README.md") # just getting a non-python file - - # should not error - modules = list(file_utils.iter_airflow_imports(file_path)) - - assert len(modules) == 0 - - def test_list_py_file_paths(self, test_zip_path): - detected_files = set() - expected_files = set() - # No_dags is empty, _invalid_ is ignored by .airflowignore - ignored_files = { - "no_dags.py", - "should_ignore_this.py", - "test_explicit_ignore.py", - "test_invalid_cron.py", - "test_invalid_dup_task.py", - "test_ignore_this.py", - "test_invalid_param.py", - "test_invalid_param2.py", - "test_invalid_param3.py", - "test_invalid_param4.py", - "test_nested_dag.py", - "test_imports.py", - "test_nested_negate_ignore.py", - "file_no_airflow_dag.py", # no_dag test case in test_zip folder - "test.py", # no_dag test case in test_zip_module folder - "__init__.py", - } - for root, _, files in os.walk(TEST_DAG_FOLDER): - for file_name in files: - if file_name.endswith((".py", ".zip")): - if file_name not in ignored_files: - expected_files.add(f"{root}/{file_name}") - detected_files = set(list_py_file_paths(TEST_DAG_FOLDER)) - assert detected_files == expected_files, ( - f"Detected files mismatched expected files:\ndetected_files: {pformat(detected_files)}\nexpected_files: {pformat(expected_files)}" - ) - - @pytest.mark.parametrize( ("edge_filename", "expected_modification"), [ diff --git a/dev/breeze/doc/images/output_build-docs.svg b/dev/breeze/doc/images/output_build-docs.svg index 1858bbb097e91..2a0812c05163e 100644 --- a/dev/breeze/doc/images/output_build-docs.svg +++ b/dev/breeze/doc/images/output_build-docs.svg @@ -240,8 +240,8 @@ hashicorp | helm-chart | http | imap | influxdb | informatica | jdbc | jenkins | keycloak | microsoft.azure |        microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    -salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |    -tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...             +salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh |    +standard | tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...  Build documents. diff --git a/dev/breeze/doc/images/output_build-docs.txt b/dev/breeze/doc/images/output_build-docs.txt index 54d8d4e3f39bb..247bee9e56ff6 100644 --- a/dev/breeze/doc/images/output_build-docs.txt +++ b/dev/breeze/doc/images/output_build-docs.txt @@ -1 +1 @@ -c5f2067ec852773089ed0ca7b8d1d533 +b4c249b4d1f7605a443774262109694a diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.svg b/dev/breeze/doc/images/output_release-management_add-back-references.svg index f17f7f47ed43b..37e9086660253 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.svg +++ b/dev/breeze/doc/images/output_release-management_add-back-references.svg @@ -155,8 +155,8 @@ hashicorp | helm-chart | http | imap | influxdb | informatica | jdbc | jenkins | keycloak | microsoft.azure |        microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    -salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |    -tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...             +salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh |    +standard | tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...  Command to add back references for documentation to make it backward compatible. diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.txt b/dev/breeze/doc/images/output_release-management_add-back-references.txt index ffc7eeea6018b..a43ec033fc2a6 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.txt +++ b/dev/breeze/doc/images/output_release-management_add-back-references.txt @@ -1 +1 @@ -3df401aef0085547b08fe896a9a65381 +a44de0a6fcf0ad832e0b2a73a883f0a0 diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg index 8fe24cdf434e6..6566b6c97716f 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg @@ -149,9 +149,9 @@ github | google | grpc | hashicorp | http | imap | influxdb | informatica | jdbc | jenkins | keycloak |                microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai |         openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres |     -presto | qdrant | redis | salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake |    -sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb |          -zendesk]...                                                                                                            +presto | qdrant | redis | salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp |     +snowflake | sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex |    +ydb | zendesk]...                                                                                                      Generates content for issue to test the release. diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt index c6189be26338f..0c327de82828f 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt @@ -1 +1 @@ -a85c889b710aa347eb6c47fc36b11720 +ee99c790838efb1d5e5a3b06e6c49846 diff --git a/dev/breeze/doc/images/output_release-management_generate-providers-metadata.svg b/dev/breeze/doc/images/output_release-management_generate-providers-metadata.svg index 867b9fedc0357..742e316f5a754 100644 --- a/dev/breeze/doc/images/output_release-management_generate-providers-metadata.svg +++ b/dev/breeze/doc/images/output_release-management_generate-providers-metadata.svg @@ -1,4 +1,4 @@ - + keycloak | microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql  | neo4j | odbc | openai | openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty |  papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce | samba |  -segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |  -tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk) ---provider-versionProvider version to generate metadata for. Only used when --provider-id is specified. Limits     -running metadata generation to only this version of the provider. (TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---verbose-vPrint verbose information about performed steps. ---help   -hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +sdk.java | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh |  +standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb |  +zendesk) +--provider-versionProvider version to generate metadata for. Only used when --provider-id is specified. Limits     +running metadata generation to only this version of the provider. (TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--verbose-vPrint verbose information about performed steps. +--help   -hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_release-management_generate-providers-metadata.txt b/dev/breeze/doc/images/output_release-management_generate-providers-metadata.txt index 3615848d57819..6e5c7fd64de14 100644 --- a/dev/breeze/doc/images/output_release-management_generate-providers-metadata.txt +++ b/dev/breeze/doc/images/output_release-management_generate-providers-metadata.txt @@ -1 +1 @@ -fdfdca32a5248d3b91cb29e14cc538b4 +de007da2573c2e6066fd2b0d26d14874 diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.svg index 3661b47f2a46d..dbabcc063abf1 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.svg @@ -197,9 +197,9 @@ github | google | grpc | hashicorp | http | imap | influxdb | informatica | jdbc | jenkins | keycloak |                microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai |         openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres |     -presto | qdrant | redis | salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake |    -sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb |          -zendesk]...                                                                                                            +presto | qdrant | redis | salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp |     +snowflake | sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex |    +ydb | zendesk]...                                                                                                      Prepare sdist/whl distributions of Airflow Providers. Each provider directory is wiped with `git clean -fdx (preserving .venv, .idea, .vscode) before build to keep in-tree generated files out of the artifact. See dev/breeze  diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.txt index f10fd70bd89fa..aa4a21a6dcebd 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-distributions.txt @@ -1 +1 @@ -18d45fa2bec60ab0557f04fb4427b35e +71c54d02659478978d0aa40b2baf4fef diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg index c4454038e4a78..ae36d4fde676e 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg @@ -218,9 +218,9 @@ github | google | grpc | hashicorp | http | imap | influxdb | informatica | jdbc | jenkins | keycloak |                microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai |         openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres |     -presto | qdrant | redis | salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake |    -sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb |          -zendesk]...                                                                                                            +presto | qdrant | redis | salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp |     +snowflake | sqlite | ssh | standard | tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex |    +ydb | zendesk]...                                                                                                      Prepare CHANGELOG, README and COMMITS information for providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt index 5586a29b8136c..68e5927948e4f 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt @@ -1 +1 @@ -622441d283775edefeda685820e7169a +542fd516d5584cf5bc1b6aa945338a8c diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.svg b/dev/breeze/doc/images/output_release-management_publish-docs.svg index d119da2013d75..291f8b0d144e3 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.svg +++ b/dev/breeze/doc/images/output_release-management_publish-docs.svg @@ -194,8 +194,8 @@ hashicorp | helm-chart | http | imap | influxdb | informatica | jdbc | jenkins | keycloak | microsoft.azure |        microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    -salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |    -tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...             +salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh |    +standard | tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...  Command to publish generated documentation to airflow-site diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.txt b/dev/breeze/doc/images/output_release-management_publish-docs.txt index c73c7846664c8..487f7d9fef5f8 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.txt +++ b/dev/breeze/doc/images/output_release-management_publish-docs.txt @@ -1 +1 @@ -4521ec02334b8909f66e82c460a69446 +6a7fed8b89fffc1e9d8856bf1a2d5f2d diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg index fd62a65b513d4..ec6cd73739017 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg @@ -189,9 +189,9 @@ | grpc | hashicorp | http | imap | influxdb | informatica | jdbc | jenkins | keycloak |  microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j |  odbc | openai | openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty | papermill  -| pgvector | pinecone | postgres | presto | qdrant | redis | salesforce | samba | segment |  -sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard | tableau |  -telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk) +| pgvector | pinecone | postgres | presto | qdrant | redis | salesforce | samba | sdk.java |  +segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |  +tableau | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk) --provider-versionProvider version to generate the requirements for i.e `2.1.0`. `latest` is also a supported      value to account for the most recent version of the provider (TEXT) --force           Force update providers requirements even if they already exist. diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt index a7761ea29d68a..8a0c324836340 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt @@ -1 +1 @@ -fa98bbcd73f9160c29eff1b6779a23bc +c4babe6a19ea7748ed3488c930187a8e diff --git a/dev/breeze/doc/images/output_workflow-run_publish-docs.svg b/dev/breeze/doc/images/output_workflow-run_publish-docs.svg index 511790e79d721..0b42e92e917d3 100644 --- a/dev/breeze/doc/images/output_workflow-run_publish-docs.svg +++ b/dev/breeze/doc/images/output_workflow-run_publish-docs.svg @@ -200,8 +200,8 @@ hashicorp | helm-chart | http | imap | influxdb | informatica | jdbc | jenkins | keycloak | microsoft.azure |        microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    -salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | standard |    -tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...             +salesforce | samba | sdk.java | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh |    +standard | tableau | task-sdk | telegram | teradata | trino | vertica | vespa | weaviate | yandex | ydb | zendesk]...  Trigger publish docs to S3 workflow diff --git a/dev/breeze/doc/images/output_workflow-run_publish-docs.txt b/dev/breeze/doc/images/output_workflow-run_publish-docs.txt index 6a433f7935a96..cbb67ce0a1df9 100644 --- a/dev/breeze/doc/images/output_workflow-run_publish-docs.txt +++ b/dev/breeze/doc/images/output_workflow-run_publish-docs.txt @@ -1 +1 @@ -6ff7091e58988c6273e51f372bb8a1a6 +a4876e7e49973aad884a0270de53885a diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 4deaa3bf598b9..ce35be9d26a3a 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -789,25 +789,25 @@ def get_airflow_extras(): { "python-version": "3.10", "airflow-version": "2.11.1", - "remove-providers": "common.messaging edge3 fab git keycloak informatica common.ai opensearch", + "remove-providers": "common.messaging edge3 fab git keycloak informatica common.ai opensearch sdk.java", "run-unit-tests": "true", }, { "python-version": "3.10", "airflow-version": "3.0.6", - "remove-providers": "", + "remove-providers": "sdk.java", "run-unit-tests": "true", }, { "python-version": "3.10", "airflow-version": "3.1.8", - "remove-providers": "", + "remove-providers": "sdk.java", "run-unit-tests": "true", }, { "python-version": "3.10", "airflow-version": "3.2.1", - "remove-providers": "", + "remove-providers": "sdk.java", "run-unit-tests": "true", }, ] diff --git a/dev/registry/extract_metadata.py b/dev/registry/extract_metadata.py index 463a9c408082b..5d9f635f74ea3 100644 --- a/dev/registry/extract_metadata.py +++ b/dev/registry/extract_metadata.py @@ -46,7 +46,7 @@ try: import tomllib # Python 3.11+ stdlib except ModuleNotFoundError: # pragma: no cover -- Python 3.10 fallback - import tomli as tomllib + import tomli as tomllib # type: ignore[no-redef] import yaml from registry_contract_models import validate_providers_catalog diff --git a/dev/registry/extract_versions.py b/dev/registry/extract_versions.py index d9dc4e166dcf1..2908b22b32e6a 100644 --- a/dev/registry/extract_versions.py +++ b/dev/registry/extract_versions.py @@ -49,7 +49,7 @@ try: import tomllib # Python 3.11+ stdlib except ModuleNotFoundError: # pragma: no cover -- Python 3.10 fallback - import tomli as tomllib + import tomli as tomllib # type: ignore[no-redef] from registry_contract_models import validate_provider_version_metadata try: diff --git a/devel-common/src/docs/provider_conf.py b/devel-common/src/docs/provider_conf.py index 6bc9da15f5f61..b730e8f20a417 100644 --- a/devel-common/src/docs/provider_conf.py +++ b/devel-common/src/docs/provider_conf.py @@ -151,7 +151,7 @@ # List of patterns, relative to source directory, that match files and # directories to ignore when looking for source files. -empty_subpackages = ["apache", "atlassian", "common", "cncf", "dbt", "microsoft"] +empty_subpackages = ["apache", "atlassian", "common", "cncf", "dbt", "microsoft", "sdk"] exclude_patterns = [ "operators/_partials", "_api/airflow/index.rst", diff --git a/devel-common/src/tests_common/pytest_plugin.py b/devel-common/src/tests_common/pytest_plugin.py index 98a871ebb12d2..35bf28195bad1 100644 --- a/devel-common/src/tests_common/pytest_plugin.py +++ b/devel-common/src/tests_common/pytest_plugin.py @@ -2510,7 +2510,6 @@ def execute(self, context): from uuid6 import uuid7 from airflow.sdk import DAG - from airflow.sdk.api.datamodels._generated import TaskInstance from airflow.sdk.execution_time.comms import BundleInfo, StartupDetails from airflow.timetables.base import TimeRestriction @@ -2538,6 +2537,13 @@ def _create_task_instance( should_retry: bool | None = None, max_tries: int | None = None, ) -> RuntimeTaskInstance: + from tests_common.test_utils.version_compat import AIRFLOW_V_3_3_PLUS + + if AIRFLOW_V_3_3_PLUS: + from airflow.sdk._shared.workloads import TaskInstanceDTO + else: + from airflow.executors.workloads.task import TaskInstanceDTO # type: ignore[no-redef,assignment] + from airflow.sdk.api.datamodels._generated import DagRun, DagRunState, TIRunContext from airflow.utils.types import DagRunType @@ -2615,14 +2621,17 @@ def _create_task_instance( } startup_details = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=ti_id, task_id=task.task_id, dag_id=dag_id, run_id=run_id, try_number=try_number, - map_index=map_index, + map_index=map_index if map_index is not None else -1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=BundleInfo(name="anything", version="any"), diff --git a/go-sdk/adr/0001-bundle-packing-options.md b/go-sdk/adr/0001-bundle-packing-options.md new file mode 100644 index 0000000000000..86f2642f541ab --- /dev/null +++ b/go-sdk/adr/0001-bundle-packing-options.md @@ -0,0 +1,292 @@ + + +# 1. Post-build bundle-packing options for the Go SDK + +Date: 2026-04-30 + +## Status + +Accepted as the option register. The packer-mechanism decision is +recorded in [ADR 0002](0002-use-go-tool-directive-for-bundle-packer.md): +Option H (Go 1.24 `tool` directive) for delivery, paired with Option A +(standalone `airflow-go-pack` binary) and Option D (standardised +`--dump-bundle-spec` introspection contract). + +The container-format assumption running through this ADR — that the +output is a ZIP archive — is superseded by +[ADR 0004](0004-self-contained-executable-bundle.md), which embeds the +source and manifest in a footer appended to the executable. The +options below still describe valid *packer mechanisms*; only the +artefact each one writes has changed from a ZIP to a footer-augmented +executable. + +## Context + +The executable provider's bundle spec +([`providers/sdk/executable/docs/bundle-spec.rst`](../../providers/sdk/executable/docs/bundle-spec.rst)) +defines a deployment artifact as a ZIP archive containing: + +1. `airflow-metadata.yaml` declaring `format_version`, `sdk` (language/version), + `source` (archive-relative path to the DAG source file), `executable` + (archive-relative path to the compiled binary), and `dags` (a mapping of + `dag_id` to `{tasks: [task_id, ...]}`). +2. The primary DAG source file, included verbatim. +3. The compiled native executable, which speaks the coordinator protocol + (`--comm=` / `--logs=`). + +Bundle authors today produce the executable with a plain `go build` +(see [`go-sdk/example/bundle/Justfile`](../example/bundle/Justfile)). There is +no SDK-provided way to produce the conforming ZIP, so each author would need +to hand-roll one. + +The bundle binary already exposes a `--bundle-metadata` flag (defined in +[`bundle/bundlev1/bundlev1server/server.go`](../bundle/bundlev1/bundlev1server/server.go)) +that prints the `BundleInfo{Name, Version}` returned by the author's +`BundleProvider.GetBundleVersion()`. It does **not** currently invoke +`RegisterDags`, so it does not yet enumerate `dag_id` / `task_id` for the +manifest. This is relevant context: the binary itself is the authoritative +source of dag/task identity at runtime, and the SDK can extend the +introspection path cheaply. + +The user's initial framing was `go build -toolexec`. `-toolexec` wraps each +toolchain invocation (compile, asm, link) and does not have visibility into +the final `-o` output path or a single "build finished" hook, so it is a poor +fit for producing the final ZIP. The options below cover the mechanisms that +do fit, plus the `-toolexec` variant for completeness. + +A packing mechanism has two sub-decisions: + +- **Where the packing logic runs.** In the bundle binary itself + (self-pack), in a separate SDK CLI, or in build tooling outside the SDK + (Makefile/Justfile snippet). +- **How dag/task IDs reach the manifest.** Runtime introspection of the + built binary (call into `RegisterDags` against an in-memory + registry recorder), static AST scan of the source file, or + hand-written manifest. + +The options below combine those two sub-decisions in different ways. + +## Options + +### Option A: Standalone SDK packer CLI (`airflow-go-pack`) + +A new binary under `go-sdk/cmd/airflow-go-pack` that takes +already-built inputs and writes the ZIP: + +``` +airflow-go-pack \ + --source ./example/bundle/main.go \ + --executable ./bin/example-dag-bundle \ + --output ./bin/example.zip +``` + +Manifest population: the packer execs the supplied executable with +`--bundle-metadata` and reads the JSON from stdout to fill `sdk.version`, +and a new `--dump-dags` (or extended `--bundle-metadata`) flag to enumerate +`dags`. Source language is hard-coded to `go`; SDK version is read from the +build info embedded in the binary or from a build-time `-ldflags` value. + +- **Pros:** simple, single-purpose binary; works against any binary the user + built however they like (CGO, cross-compile, custom `-ldflags`); no + coupling to `go build` invocation; trivially callable from `just`, + `make`, CI, or `go generate`. +- **Cons:** two-step UX (`go build` then `airflow-go-pack`); user has to + install or `go run` the tool; nothing prevents pack/build mismatch + (e.g. packing yesterday's binary). + +### Option B: All-in-one SDK CLI with a `build` subcommand + +A single SDK CLI (`airflow-go`) with subcommands that wrap `go build` and +then pack: + +``` +airflow-go build ./example/bundle --output ./bin/example.zip +``` + +Internally: spawn `go build -o /bundle `, then run the same +introspection step as Option A, then write the ZIP. + +- **Pros:** single command; no chance of pack/build skew; easy to add + related subcommands later (`airflow-go new`, `airflow-go run`, + `airflow-go validate`); good defaults for `-ldflags` (e.g. + `-X main.bundleVersion=...`) without the author having to know them. +- **Cons:** the SDK now owns a `go build` wrapper and inherits + responsibility for forwarding the long tail of `go build` flags + (`-tags`, `-trimpath`, `GOOS`/`GOARCH` env, `-ldflags` passthrough, + `-buildvcs`, etc.); harder to integrate with non-trivial existing build + systems that already drive `go build` themselves. + +### Option C: Self-packing binary (`--pack-bundle `) + +Extend `bundlev1server.Serve` so that when the binary is invoked with +`--pack-bundle `, it builds the ZIP itself: it knows its own +executable path (`os.Executable()`), its embedded source (via `//go:embed` +of the DAG source file at build time), and its dag/task list (by +calling `RegisterDags` against an in-memory recorder). After writing +the archive, it exits. + +- **Pros:** zero extra tools; the binary is fully self-describing; pack + output is provably consistent with the binary's runtime behaviour. +- **Cons:** requires the author's `main` package to embed its own source + (`//go:embed main.go` or similar), which is awkward when the DAG is + spread across multiple files or the source path is non-obvious; + bloats every bundle binary with packing code and an embedded copy of + the source; mixes build-time concerns into a runtime entrypoint. + +### Option D: Two-phase external introspection (introspection binary + packer) + +Same shape as Option A or B, but standardise the introspection contract: +the SDK guarantees that every bundle binary supports +`--dump-bundle-spec` (or a richer `--bundle-metadata`) which prints a +JSON blob containing `sdk.language`, `sdk.version`, and the full `dags` +mapping. The packer's only job is to combine that JSON, the source +file path the user passes in, and the binary itself into a ZIP. + +This is really a refinement of A/B that fixes the introspection contract +in the SDK protocol, rather than an independent option, but is worth +calling out because the shape of the introspection flag is itself a +decision (single flag vs. several; JSON vs. YAML; pretty vs. compact). + +- **Pros:** decouples "how do we enumerate dags" from "how do we ZIP"; + any future packer (third-party CI plugin, IDE, etc.) can rely on the + same contract; trivial to unit-test. +- **Cons:** locks in a wire format the SDK has to keep stable; slightly + more code in the bundle binary than today. + +### Option E: Static AST scan, no introspection + +Parser-only packer: walk the DAG source AST, find `dagbag.AddDag("X")` +calls and the `.AddTask(fn)` calls chained off them, and synthesise the +manifest without running the binary. + +- **Pros:** no runtime dependency on the binary (works even if it + doesn't build for the host platform, e.g. cross-compiled for Linux on + a macOS dev box); fast. +- **Cons:** brittle to anything dynamic (`for _, name := range names { + dagbag.AddDag(name) }`, helper functions, generated code); the SDK + becomes the second source of truth for dag/task identity, which can + drift from `RegisterDags`; users will hit "I added a DAG and the + packer didn't see it" failures. + +### Option F: `go generate` directive + +Document a recommended `//go:generate` line in the author's `main.go`: + +```go +//go:generate go run github.com/apache/airflow/go-sdk/cmd/airflow-go-pack ... +``` + +`go generate` is then the build-time entrypoint. + +- **Pros:** stdlib-blessed mechanism; no new tool installation needed + (`go run` fetches the packer from the module cache); discoverable from + the source file itself. +- **Cons:** `go generate` has to be run explicitly; users frequently + forget; doesn't actually pack the *binary*, only triggers a tool that + must still build and pack it; fits awkwardly because the natural + ordering is `pack` -> `build`, not `build` -> `pack`. + +### Option G: `go build -toolexec` wrapper + +Provide a binary that the user passes as +`go build -toolexec=airflow-go-toolexec ...`. The wrapper proxies every +toolchain call and, on detecting the final `link` invocation, copies the +linker's output path, then runs the packing step against it. + +- **Pros:** single `go build` invocation; nominally fits into existing + `go build` workflows. +- **Cons:** `-toolexec` was not designed for this. It receives the + toolchain executable path and an argv that varies across Go versions; + the wrapper has to parse the linker's `-o` to discover the binary + location, which is undocumented/internal behaviour and changes + between releases. It also runs once per package compile, so the + wrapper must distinguish "real" link invocations from intermediate + ones. Operationally fragile; recommended against. + +### Option H: `go.mod` `tool` directive (Go 1.24+) + +Register the packer in the consuming project's `go.mod` via the +`tool` directive and invoke it as `go tool airflow-go-pack`. This is +orthogonal to A/B/D (it's a *delivery* mechanism, not a different +implementation) but is worth listing because it changes the install +story significantly. + +- **Pros:** version-pinned per project; no `uv tool`-style global install; + works the same in every checkout; aligns with `breeze`'s direction + ([ADR 0017](../../dev/breeze/doc/adr/0017-use-uvx-to-run-breeze-from-local-sources.md) + for the Python side). +- **Cons:** requires Go 1.24 in the consumer's toolchain; one more line + the author has to add to `go.mod`. + +### Option I: Build-system recipe only (no SDK code) + +Ship a documented `Justfile` / `Makefile` / `Taskfile` snippet that +sequences `go build` -> `zip` -> manifest write, and let users copy it +into their projects. The SDK provides only the spec and an example. + +- **Pros:** zero new code in the SDK; users see exactly what is + happening. +- **Cons:** every project re-implements (and slowly diverges from) the + spec; manifest generation in shell is painful; no introspection of + dag/task IDs without re-running the binary anyway, so the recipe + ends up calling some helper, which is just Option A by another name. + +## Cross-cutting sub-decisions + +These apply to whichever top-level option is chosen: + +1. **Manifest source of truth.** Runtime introspection (D) is the only + approach that cannot drift from `RegisterDags`. Everything else + trades that guarantee for some other property (no binary needed, + no extra flag, etc.). +2. **Source-file discovery.** The spec requires the source file path + to appear in the manifest's `source` field and the file itself to + be present in the ZIP. The packer needs either (a) an explicit + `--source` argument, (b) a convention (e.g. the `main.go` of the + `main` package being built), or (c) a `//go:embed`-d copy inside + the binary (Option C). +3. **SDK version reporting.** The bundle binary should expose the SDK + version it linked against. This can come from `runtime/debug.ReadBuildInfo` + walking the deps for `github.com/apache/airflow/go-sdk`, or from + a build-time `-ldflags -X` value the SDK documents. +4. **Reproducibility.** The ZIP should be deterministic for a given + set of inputs (sorted entries, fixed mtimes, no host-specific + metadata) so two builds of the same bundle hash identically. This + is independent of which option is picked but easiest to enforce + inside SDK-owned code (A/B/C/D) than in a shell recipe (I). +5. **Executable bit.** The spec says the archive entry SHOULD preserve + the executable bit via the ZIP external-attributes field. The + packer must set `0755` (or similar) on the executable entry; this + is trivial in Go's `archive/zip` but easy to get wrong in shell. + +## Decision + +Recorded in [ADR 0002](0002-use-go-tool-directive-for-bundle-packer.md). +Summary: deliver the packer via the Go 1.24 `tool` directive (Option H); +implement it as a standalone binary at `cmd/airflow-go-pack` (Option A); +populate the manifest by execing the bundle binary with a standardised +`--dump-bundle-spec` introspection flag (Option D). + +## Consequences + +Listing the options here, rather than only landing the chosen one, +keeps the rejected alternatives discoverable for future SDKs (Rust, +C++, Zig) which will face the same question, and documents why +`-toolexec` and AST-only scanning were considered and dropped. diff --git a/go-sdk/adr/0002-use-go-tool-directive-for-bundle-packer.md b/go-sdk/adr/0002-use-go-tool-directive-for-bundle-packer.md new file mode 100644 index 0000000000000..beb680bdcec70 --- /dev/null +++ b/go-sdk/adr/0002-use-go-tool-directive-for-bundle-packer.md @@ -0,0 +1,222 @@ + + +# 2. Use the Go 1.24 `tool` directive to deliver the bundle packer + +Date: 2026-04-30 + +## Status + +Accepted. Selects from the option register in +[ADR 0001](0001-bundle-packing-options.md). + +The output-format portion of this ADR (the packer writes a ZIP archive +following the bundle spec) is superseded by +[ADR 0004](0004-self-contained-executable-bundle.md): the packer now +writes a self-contained executable with an appended footer carrying +the source bytes and the manifest. The packer's *mechanism* (Option +A standalone binary + Option D introspection contract + Option H +`tool` directive) is unchanged. The decision sketches below mention +ZIP output; read them with the ADR 0004 substitution in mind, and +treat ADR 0004 as authoritative wherever the two disagree. + +## Context + +[ADR 0001](0001-bundle-packing-options.md) enumerated nine candidate +mechanisms for producing a conforming bundle ZIP +([`providers/sdk/executable/docs/bundle-spec.rst`](../../providers/sdk/executable/docs/bundle-spec.rst)) +from a Go SDK build. Two reasons drive the choice: + +1. **The repository already requires Go 1.24.** `go-sdk/go.mod` declares + `go 1.24.0` with `toolchain go1.24.6`, so language features added in + 1.24 are available to every consumer of the SDK by construction. +2. **Contributors expect Go-native workflows.** The Go 1.24 `tool` + directive is the toolchain's native answer to "depend on a + build-time CLI without polluting the global PATH." It pins the tool + version per-project in `go.mod`, resolves it through the standard + module cache, and exposes it as `go tool `, with no extra + installer and no per-worktree drift. The same problem on the Python + side led `breeze` to switch to `uvx` in + [ADR 0017](../../dev/breeze/doc/adr/0017-use-uvx-to-run-breeze-from-local-sources.md); + `tool` is the analogous answer here. + +The `tool` directive is a delivery mechanism. It still needs an +underlying implementation. We pair it with two of the implementation +options from ADR 0001, with a UX twist: + +- **Option A (standalone packer):** a single-purpose binary at + `go-sdk/cmd/airflow-go-pack`. It still operates as one process with + a clear input/output contract, but it drives `go build` internally + by default so that the common case is one command: + `go tool airflow-go-pack ./pkg`. Authors who already produce their + own binary can opt out via `--executable ` and skip the build + phase. This is closer to Option B's ergonomics than the original + ADR 0001 sketch, but kept inside the standalone-packer shape so the + SDK does not own a fully general `go build` wrapper. +- **Option D (standardised introspection contract):** every bundle + binary supports a `--dump-bundle-spec` flag that prints JSON + containing `sdk.language`, `sdk.version`, and the full `dags` mapping. + The packer execs the freshly built binary with this flag to populate + the manifest, which keeps `RegisterDags` as the single authoritative + source of dag/task identity. + +Options C, E, F, G, and I from ADR 0001 are rejected for the reasons +recorded there. Option B (a full `airflow-go build` wrapping `go +build`) is rejected as a separate top-level command, but its core +ergonomic win (one command for the common case) is folded into the +Option A packer through default behaviour, with a `--` passthrough +convention so authors can forward arbitrary flags to the underlying +`go build` without the SDK having to enumerate them. + +## Decision + +1. **Add `cmd/airflow-go-pack` to the go-sdk module.** Its default + invocation is one command: + + ```sh + go tool airflow-go-pack [./path/to/pkg] [-- ...] + ``` + + The single positional argument is the Go package containing the + bundle's `main` package (defaults to `.`, the current directory). + Anything after `--` is forwarded verbatim to the internal + `go build` invocation, so authors keep full control over `-tags`, + `-trimpath`, `-ldflags`, `GOOS`/`GOARCH` (via env), `-buildvcs`, + etc. without the packer having to enumerate them. + + With no further flags, the packer: + + 1. Resolves the target package and locates the file in that + package that defines `func main()`. That file becomes the + manifest's `source` and is copied verbatim into the ZIP. If + `main` is split across multiple files, the packer errors and + asks the author to specify `--source `. + 2. Runs `go build [forwarded flags] -o / `. + 3. Executes the freshly built binary with `--dump-bundle-spec` to + obtain `sdk.{language,version}` and the `dags` mapping. + 4. Writes a deterministic ZIP next to the working directory at + `.zip`, where `` comes from the + binary's `BundleInfo.Name` (already exposed via + `--bundle-metadata`). + + Optional overrides, for advanced or pre-built workflows: + + - `--source `: override the auto-detected source file. + - `--executable `: skip the internal `go build` and pack a + pre-built binary. Mutually exclusive with `--` build-flag + passthrough. + - `--output `: override the default output ZIP path. + + Examples: + + ```sh + # Common case: build and pack in one command from the package dir. + go tool airflow-go-pack + + # Pack a different package, with extra go build flags. + go tool airflow-go-pack ./cmd/my-bundle -- -trimpath -tags=prod + + # Pack an already-built binary (skips go build). + go tool airflow-go-pack --executable ./build/example --source main.go + ``` + +2. **Extend `bundlev1server.Serve` with `--dump-bundle-spec`.** The + flag prints a JSON document of the form: + + ```json + { + "format_version": "1.0", + "sdk": {"language": "go", "version": ""}, + "dags": { + "": {"tasks": ["", "..."]} + } + } + ``` + + `sdk.version` is read from `runtime/debug.ReadBuildInfo()` by + walking deps for `github.com/apache/airflow/go-sdk`. The `dags` + mapping is populated by calling the bundle's `RegisterDags` against + an in-memory recording registry, then enumerating the recorded + dags and their tasks. The existing `--bundle-metadata` flag is + retained for backward compatibility and continues to print + `BundleInfo`. + +3. **Bundle authors register the packer in their own `go.mod`:** + + ``` + tool github.com/apache/airflow/go-sdk/cmd/airflow-go-pack + ``` + + and pack in one step: + + ```sh + go tool airflow-go-pack + ``` + +4. **Update `go-sdk/example/bundle/Justfile`** to demonstrate the + recipe end-to-end, including the `tool` directive in the example's + own `go.mod`. + +## Consequences + +- Per-project, per-worktree version pinning of the packer through + `go.mod`. No global install. Two checkouts on different SDK versions + pack with the right packer for each. +- Authors get a Go-native, single-command workflow for the common case + (`go tool airflow-go-pack`), with a `--` passthrough escape hatch + for arbitrary `go build` flags and `--executable` for pre-built + workflows. CI and other build systems can use whichever shape fits. +- The `--dump-bundle-spec` JSON becomes a stable wire format that the + SDK has to keep backward-compatible. It is versioned implicitly + through the bundle spec's `format_version` field, so additive + changes are safe. +- Third-party tooling (IDE plugins, alternative packers, CI plugins) + can rely on the same introspection contract without taking a Go + dependency on the SDK. +- The packer takes on responsibility for locating the `main` source + file and choosing a sensible default output path. Both are + heuristics; both are overridable. Drift between the heuristic and + the spec is the main maintenance cost introduced by this option. +- Requires Go 1.24 in any consumer project. Already a project-wide + assumption. + +## Implementation notes + +- The ZIP must be deterministic: sorted entries, fixed mtimes, no + host-specific metadata, so two builds of identical inputs hash the + same. +- The executable entry's external-attributes field must encode mode + `0755` (or similar), per the bundle spec's executable-bit + requirement. +- The packer should validate that the manifest's `dags` is non-empty + and warn (not fail) on empty `tasks` lists, matching the bundle + spec's "permitted but discouraged" wording. +- `--dump-bundle-spec` runs `RegisterDags` but must not start the + gRPC server or contact any external services; the in-memory + registry recorder is the only side effect. +- Source-file detection uses `go/packages` (or `go list -json`) to + load the target package, then picks the file whose AST contains a + top-level `func main()`. If the package has zero or more than one + such file, the packer errors with a clear message and asks for + `--source`. +- The internal `go build` runs in a temp directory so the host's + working tree is not polluted with build artefacts; the temp dir is + cleaned up after the ZIP is written. +- `go build` flag passthrough uses the standard `--` separator + convention so the packer's own flag set stays small and stable. diff --git a/go-sdk/adr/0003-coordinator-protocol-msgpack-ipc.md b/go-sdk/adr/0003-coordinator-protocol-msgpack-ipc.md new file mode 100644 index 0000000000000..9bb77256b5268 --- /dev/null +++ b/go-sdk/adr/0003-coordinator-protocol-msgpack-ipc.md @@ -0,0 +1,381 @@ + + +# 3. Dual-mode bundle binary: msgpack-over-IPC coordinator protocol alongside the existing go-plugin/Edge-Worker path + +Date: 2026-04-30 + +## Status + +Accepted. + +## Context + +A Go SDK bundle binary today (the artefact built from +[`go-sdk/example/bundle/main.go`](../example/bundle/main.go) via +`bundlev1server.Serve`) speaks exactly one protocol: HashiCorp +[`go-plugin`](https://github.com/hashicorp/go-plugin) gRPC over a +stdio-negotiated socket, gated by the magic-cookie handshake declared in +[`pkg/bundles/shared/handshake.go`](../pkg/bundles/shared/handshake.go). +The Airflow Go *Edge Worker* +([`cmd/airflow-go-edge-worker`](../cmd/airflow-go-edge-worker/main.go), +[`edge/`](../edge)) is the consumer of that protocol — it execs the +bundle binary as a child process, completes the go-plugin handshake, +opens the `DagBundle` gRPC client, and drives `GetMetadata`/`Execute` +([`bundle/bundlev1/bundlev1server/impl/plugin.go`](../bundle/bundlev1/bundlev1server/impl/plugin.go)). +The bundle binary never listens on a public socket; the protocol is +local-process only. + +Meanwhile, the Python side of Airflow has standardised on a different +wire protocol for non-Python language runtimes — the *coordinator +protocol* — pioneered by the Java SDK and described in +[java-sdk ADR 0002](../../../feature-java-sdk-worktrees/feature-java-all/java-sdk/adr/0002-dag-parsing.md) +and +[java-sdk ADR 0003](../../../feature-java-sdk-worktrees/feature-java-all/java-sdk/adr/0003-workload-execution.md). +Its shape is: + +- The runtime is launched with `--comm=` and + `--logs=` CLI arguments. +- It connects out (TCP, loopback) to both addresses. +- Frames on the comm channel are length-prefixed msgpack: a 4-byte + big-endian length followed by the msgpack payload. Requests are + `[id, body]`; responses are `[id, body, error]`. +- Two workloads share one channel, distinguished by the first inbound + frame: `DagFileParseRequest` (one-shot, returns + `DagFileParsingResult` and exits) or `StartupDetails` (multi-round + task execution: the runtime sends `GetConnection` / `GetVariable` / + `GetXCom` / `SetXCom` and terminates with `SucceedTask` or + `TaskState`). +- The logs channel carries structured JSON log records emitted by the + runtime. + +The Python-side launcher is +[`ExecutableCoordinator`](../../providers/sdk/executable/src/airflow/providers/sdk/executable/coordinator.py), +which already builds command lines of the form +` --comm= --logs=` for both `dag_parsing_runtime_cmd` +and `task_execution_runtime_cmd`. The bundle-spec contract +([`providers/sdk/executable/docs/bundle-spec.rst`](../../providers/sdk/executable/docs/bundle-spec.rst)) +ratifies that any compiled SDK shipping a ZIP bundle "MUST honour the +SDK coordinator protocol (`--comm=` / `--logs=` +socket-based IPC)". The Java SDK satisfies this contract; the Go SDK +currently does not. + +The two protocols target different deployment shapes: + +- **go-plugin / Edge Worker.** The Go-native worker is itself a long-running + process that loads bundles in-process and dispatches tasks to them + over gRPC. It is the only consumer that speaks go-plugin to a Go + bundle today, and it owns the full task-runtime stack on the worker + host (no Python in the data path). This is the path + [`go-sdk/example/bundle/main.go`](../example/bundle/main.go) was + written for and the path that + [`pkg/worker`](../pkg/worker) drives. +- **Coordinator / `ExecutableCoordinator`.** The Python task + runner forks a child that runs ` --comm=… --logs=…`, + bridges its socket to the Airflow supervisor's fd 0, and proxies + Airflow service calls (`GetConnection`, `GetVariable`, ...) through + to the Execution API. This is how Airflow runs non-Python tasks + *without* a per-language worker — the same way Java runs today, and + the same way Rust/C++/Zig will run in the future. It is also the + only path the executable provider's bundle spec recognises. + +Today these two paths require two different binaries, even though the +DAG/task definitions, the registry, the worker plumbing, and the +serialisation surfaces overlap almost entirely. That is the gap this +ADR closes. + +The user-written `main()` is one line — +`bundlev1server.Serve(&myBundle{})` — and we want to keep it one line. +Whichever protocol the binary should speak must be decided inside +`Serve` based on how it was invoked, not by branching in user code. + +## Decision + +Make the SDK bundle binary **dual-mode**. A single +`bundlev1server.Serve(bundle, opts...)` call dispatches to one of two +protocol servers based on its CLI arguments and process environment. +User code does not change. + +### Invocation matrix + +`Serve` parses flags first, then chooses a mode in this order: + +| Trigger | Mode | Behaviour | +|--------------------------------------------------------|-----------------|-----------| +| `--bundle-metadata` | metadata-dump | Existing flag (ADR 0001 / `server.go:37`). Prints `BundleInfo` JSON and exits. | +| `--dump-bundle-spec` | spec-dump | Existing flag added by [ADR 0002](0002-use-go-tool-directive-for-bundle-packer.md). Prints the full bundle spec JSON (`sdk`, `dags`) used by `airflow-go-pack`. | +| `--comm= --logs=` | **coordinator** | New. Speaks the msgpack-over-IPC coordinator protocol. Both flags are required; partial use is a hard error. | +| `AIRFLOW_BUNDLE_MAGIC_COOKIE` env var present (default) | go-plugin | Existing behaviour. Hands off to `plugin.Serve` which performs the handshake and serves `DagBundle` gRPC to the Edge Worker. | +| Otherwise | error | Print usage to stderr and exit non-zero. Today this case implicitly errors via go-plugin's failed handshake; we make the diagnostic explicit so authors running the binary directly get a clear message. | + +The two server modes share the same `bundlev1.BundleProvider` +implementation and the same lazy `RegisterDags` recorder cache that +`impl.server` already maintains (`impl/plugin.go:99-121`). Only the +front door changes. + +### Coordinator mode: protocol details + +When `Serve` enters coordinator mode it: + +1. **Parses and validates the addresses.** Both `--comm` and `--logs` + are `host:port` strings. `127.0.0.1` is the only host the coordinator + protocol is designed for, but we do not pin it — the value is whatever + `_runtime_subprocess_entrypoint` chose on the Python side. + +2. **Connects out** to the comm address, then to the logs address. Both + are TCP. We dial; we do not listen. The launcher already has both + listeners up before exec'ing the binary + ([java-sdk ADR 0002, "What the Base Class Handles Automatically"](../../../feature-java-sdk-worktrees/feature-java-all/java-sdk/adr/0002-dag-parsing.md#what-the-base-class-handles-automatically)). + +3. **Routes structured logs to the logs socket.** A new + `slog.Handler` writes JSON-line records (one record per line, UTF-8, + newline-terminated) to the logs connection, replacing the + `hclog`/stderr handler used in go-plugin mode. `slog.SetDefault` is + called before any user code runs so `log` arguments injected into + tasks land on the right channel. On disconnect the handler falls + back to stderr so the binary never deadlocks on a closed sink. + +4. **Reads the first comm frame and dispatches by message type.** The + first frame's body has a `type` field per the Java SDK's encoding + ([java-sdk ADR 0003, "Task SDK Protocol Messages"](../../../feature-java-sdk-worktrees/feature-java-all/java-sdk/adr/0003-workload-execution.md#task-sdk-protocol-messages)). + Two values are valid here: + + - `DagFileParseRequest` → DAG-parsing one-shot. + - `StartupDetails` → task execution. + + Any other type is an error frame back to the supervisor and + `os.Exit(1)`. + +#### DAG-parsing path (`DagFileParseRequest` → `DagFileParsingResult`) + +```text +Supervisor Bundle binary (Go) + │ │ + ├── [4B len][msgpack: id, ─────────────►│ + │ {type: "DagFileParseRequest", │ + │ file: ""}] │ + │ │ + │ ├── BundleProvider.RegisterDags(reg) + │ │ (cached, same as gRPC path) + │ │ + │ ├── serialise(reg) → + │ │ DagFileParsingResult + │ │ in DagSerialization v3 JSON + │ │ (see java-sdk ADR-0002) + │ │ + │◄────────────────[4B len][msgpack: ────┤ + │ id, {type: "DagFileParsingResult", + │ fileloc: "...", + │ serialized_dags: [...] }] │ + │ │ + │ └── close + exit(0) +``` + +The serialised DAG payload must match Python's `SerializedDAG.serialize_dag` +output **exactly**, including the `__type` / `__var` wrapping rules, +unwrapping of "non-decorated" fields (`start_date`, `end_date`, `tags`), +and the timetable encoding listed in +[java-sdk ADR 0002, "DagFileParsingResult Format"](../../../feature-java-sdk-worktrees/feature-java-all/java-sdk/adr/0002-dag-parsing.md#dagfileparsingresult-format). +The Go SDK gains a `serde` package that performs this encoding from +`bundlev1.Bundle` / `bundlev1.Task`, validated against +`validation/serialization/test_dags.yaml` (the same fixture set the Java +SDK uses), so the Go and Java outputs are byte-identical for shared +inputs. + +#### Task-execution path (`StartupDetails` → multi-round → `SucceedTask` / `TaskState`) + +```text +Supervisor Bundle binary (Go) + │ │ + ├── StartupDetails ────────────────────►│ + │ (ti, dag_rel_path, bundle_info, │ + │ start_date, ti_context) │ + │ │ + │ ├── lookup task: + │ │ bundle.dags[ti.dag_id] + │ │ .tasks[ti.task_id] + │ │ (returns TaskState{state:"removed"} + │ │ if not found, mirroring Java) + │ │ + │ ├── construct sdk.Client whose + │ │ GetConnection / GetVariable / + │ │ GetXCom / SetXCom calls block on + │ │ request/response over the + │ │ comm socket + │ │ + │◄── GetConnection(conn_id) ────────────┤ + ├── ConnectionResult ──────────────────►│ + │◄── GetVariable(key) ──────────────────┤ + ├── VariableResult ────────────────────►│ + │◄── GetXCom(...) ──────────────────────┤ + ├── XComResult ────────────────────────►│ + │◄── SetXCom(...) ──────────────────────┤ + ├── (empty response) ──────────────────►│ + │ │ + │ ├── task fn returns: + │ │ err == nil → SucceedTask + │ │ err != nil → TaskState{"failed"} + │ │ (panic recovered → "failed") + │ │ + │◄── SucceedTask / TaskState ───────────┤ + │ │ + │ └── close + exit(0) +``` + +Concretely, this reuses +[`pkg/worker.Worker`](../pkg/worker/runner.go) for task lookup and +parameter injection — `extract(ctx, sdk.Client, *slog.Logger)`, +`transform(ctx, sdk.VariableClient, *slog.Logger)`, and `load() error` +in the example bundle work unchanged. The injected `sdk.Client` +implementation is swapped: in go-plugin mode it talks to the Execution +API directly via the URL from viper (`impl/plugin.go:182`), in +coordinator mode it talks to the supervisor over the comm socket. +Both implement the same `sdk.Client` / `sdk.VariableClient` interfaces, +so user task code is identical between the two modes. + +Frame correlation, error envelopes, and request `id` numbering follow +java-sdk ADR 0003 verbatim. Re-implementing rather than reusing those +is a deliberate cost of having a separate Go runtime; the validation +fixtures keep the encoders honest. + +### go-plugin mode: unchanged + +When neither dump flag nor `--comm`/`--logs` is set, `Serve` falls +through to the existing call site: + +```go +plugin.Serve(&plugin.ServeConfig{ + HandshakeConfig: shared.Handshake, + Plugins: plugin.PluginSet{"dag-bundle": &impl.BundleGRPCPlugin{...}}, + GRPCServer: plugin.DefaultGRPCServer, +}) +``` + +The handshake env var (`AIRFLOW_BUNDLE_MAGIC_COOKIE`) gates the path +the same way it does today, so an Edge Worker that execs the binary +gets exactly the same protocol it gets today. The `DagBundle` gRPC +service, the registry cache, the `--bundle-metadata` flag, and the +worker injection in +[`impl/plugin.go:178`](../bundle/bundlev1/bundlev1server/impl/plugin.go) +are untouched. + +### Code organisation + +A new internal package +`go-sdk/bundle/bundlev1/bundlev1server/impl/coord` owns the +coordinator-mode server: frame codec, log-sink handler, dag-parse +handler, task-execution handler, and the `sdk.Client` adapter that +proxies to the comm socket. It depends on a new +`go-sdk/bundle/bundlev1/serde` package for DagSerialization v3 +encoding. The frame codec is small enough to keep first-party rather +than pulling a new msgpack dependency at the API surface; we use +[`github.com/vmihailenco/msgpack/v5`](https://github.com/vmihailenco/msgpack) +internally. + +`bundlev1server.Serve` becomes: + +```go +func Serve(bundle bundlev1.BundleProvider, opts ...ServeOpt) error { + config.SetupViper("") + flag.Parse() + + switch mode := decideMode(); mode { + case modeMetadataDump: + return dumpBundleMetadata(bundle) // existing + case modeSpecDump: + return dumpBundleSpec(bundle) // ADR 0002 + case modeCoordinator: + return coord.Serve(bundle, *commAddr, *logsAddr) // NEW + case modePlugin: + return servePlugin(bundle) // existing + } +} +``` + +User code (`main.go`) is the same one line: + +```go +func main() { bundlev1server.Serve(&myBundle{}) } +``` + +## Consequences + +### Capability gains + +- A single binary built from one `bundlev1server.Serve` entry point now + runs under both the Go-native Edge Worker (go-plugin) and the + Python-native task runner via `ExecutableCoordinator` + (msgpack-over-IPC). Authors do not pick a deployment shape at build + time. +- The bundle ZIP produced by `airflow-go-pack` (ADR 0002) becomes + spec-conformant + ([`providers/sdk/executable/docs/bundle-spec.rst`](../../providers/sdk/executable/docs/bundle-spec.rst)) + without further changes, because the binary now honours + `--comm=`/`--logs=` as the spec demands. +- Mixed-language pipelines (Python `@task.stub` DAGs delegating to a Go + task) work without a Go worker on the executor host — the same + coordinator the Java SDK rides on now carries Go. + +### Compatibility + +- The go-plugin path is unchanged at the wire and at the source level. + Existing Edge Worker deployments do not need to be rebuilt or + reconfigured. The protocol selector keys off CLI flags and the + go-plugin magic-cookie env var, both of which the Edge Worker + already sets. +- `--bundle-metadata` still emits the same JSON. `--dump-bundle-spec` + (ADR 0002) is unaffected. Adding a binary with this ADR's changes + into an older Edge Worker deployment is safe; adding an older binary + into an `ExecutableCoordinator` deployment fails fast with a + clear "unknown flag: --comm" stderr message rather than hanging. + +### New ongoing costs + +- The Go SDK now owns a second wire protocol. Encoder drift between + Python's `SerializedDAG.serialize_dag` and the Go `serde` package is + the largest maintenance hazard. We mitigate it by sharing + `validation/serialization/test_dags.yaml` with the Java SDK and + running the same `compare.py` step in CI for Go output. +- The Task SDK message catalogue (`GetConnection`, `GetVariable`, + `GetXCom`, `SetXCom`, `SucceedTask`, `TaskState`, + `ConnectionResult`, `VariableResult`, `XComResult`, `ErrorResponse`, + `StartupDetails`, `DagFileParseRequest`, `DagFileParsingResult`) is + duplicated from the Java SDK's Kotlin definitions. Schema changes on + the Python side need both SDKs updated together; a single + `task-sdk/protocol/` JSON-schema source of truth is a reasonable + follow-up but is out of scope here. +- A new transitive dependency on `vmihailenco/msgpack/v5`. It is + pure-Go and stable; the cost is acceptable. +- The `sdk.Client` interface gains a second backend (comm-socket). + Tests that previously injected a fake `sdk.VariableClient` (see + [`example/bundle/main_test.go`](../example/bundle/main_test.go)) keep + working unchanged — the swap is below the SDK surface. + +### Out of scope + +- The logs channel format. We emit JSON-line records to match the Java + SDK; a richer protocol (severity-aware framing, attachment of trace + ids) is deferred until the Python supervisor side standardises one. +- OTel context propagation. The `context_carrier` field on + `TaskInstance` is still TODO in + [`impl/plugin.go:151`](../bundle/bundlev1/bundlev1server/impl/plugin.go#L151) + and remains TODO in coordinator mode for now. +- A Go-side equivalent of the Java SDK's `Supervisor.kt` (the + no-Python-in-the-loop execution path). The Edge Worker already fills + that role for Go via go-plugin; we do not need a second one. diff --git a/go-sdk/adr/0004-self-contained-executable-bundle.md b/go-sdk/adr/0004-self-contained-executable-bundle.md new file mode 100644 index 0000000000000..5d73bcaff9590 --- /dev/null +++ b/go-sdk/adr/0004-self-contained-executable-bundle.md @@ -0,0 +1,304 @@ + + +# 4. Self-contained executable bundle (footer-embedded source and metadata) + +Date: 2026-05-04 + +## Status + +Accepted. Supersedes the ZIP-archive container portion of +[ADR 0001](0001-bundle-packing-options.md) and the ZIP output sketched +in [ADR 0002](0002-use-go-tool-directive-for-bundle-packer.md). The +packer mechanism (Option A standalone packer + Option D introspection +contract + Option H `tool` directive) is unchanged; only the artefact +the packer writes is changed. + +## Context + +ADR 0001 / ADR 0002 picked a ZIP archive as the bundle container, +following the executable provider's existing +[bundle spec](../../providers/sdk/executable/docs/bundle-spec.rst). +A conforming bundle today is `bundle.zip` with three required entries: +`airflow-metadata.yaml`, the primary DAG source file, and the compiled +executable. + +That layout has three properties we want to preserve: + +1. **Discovery without execution.** The scanner must be able to read + `dag_id` / `task_id` and the SDK language/version from a bundle on + disk without running the binary. ADR 0002 already enforces this — + `airflow-go-pack` runs the binary once at build time, captures its + `--dump-bundle-spec` output into the manifest, and the scanner reads + the manifest at deploy time. +2. **Source available for the UI.** The Airflow UI's source-view + panel needs to render the DAG file. The current spec ships it as a + verbatim ZIP entry referenced by the manifest's `source` field. +3. **Single deployment unit.** Drop one file in + `[executable] bundles_folder` and the scanner picks it up. + +What the ZIP container costs us: + +- **Two artefacts in flight.** `go build` produces a binary; the + packer wraps it into a ZIP. Anything that touches the binary after + it is wrapped (re-strip, re-sign, swap-in a debug build) drifts from + the manifest unless the wrapping is redone. The wrapping step is + cheap but the drift mode is real. +- **A second container format on the consumer side.** The scanner + must open archives, find members by name, and materialise the + executable into a transient cache before the runtime can exec it. + That is `archive/zip` on the Python side plus a per-bundle cache + directory. +- **Inspection requires a different tool than running.** `unzip` to + inspect, then run; or run, then `unzip` to debug. Two muscle memories. + +Native-executable SDKs (Go, Rust, C++, Zig) all produce a single +self-contained binary by static linking. The binary itself is already +the only thing that has to land on the worker host to run a task. The +manifest and the source file are small data the scanner needs but the +runtime doesn't. Both can ride along in a footer appended to the +binary, with the binary remaining a runnable executable. + +This is the same pattern self-extracting installers, `goreleaser`-style +self-update images, and embedded-asset binaries already use: append +data after the OS-recognised binary structure, leave a fixed-size +trailer at the very end so a reader can locate the data, and validate +with a magic value. + +The user-facing claim becomes "the executable *is* the bundle." A +bundle directory looks like: + +``` +/opt/airflow/executable-bundles/ +├── example +├── pipeline +└── analytics +``` + +(Filenames follow OS conventions: no extension on Linux/macOS, `.exe` +on Windows. The scanner identifies bundles by the trailer's magic, not +by the filename.) + +## Decision + +Replace the bundle's ZIP container with a footer appended to the +compiled executable. The executable's normal byte content is unchanged +and it remains directly runnable; the footer is data that follows the +last byte the OS loader cares about. + +### Footer layout + +A bundle file is laid out as: + +```text ++---------------------------------+ +| | ++---------------------------------+ <- end of "binary" region +| source bytes (variable length) | raw root source file, UTF-8, +| | length = source_len; MAY be 0 ++---------------------------------+ +| metadata bytes (variable length)| airflow-metadata.yaml content, +| | UTF-8, length = metadata_len ++---------------------------------+ +| trailer (32 bytes, little-endian fixed layout): | +| bytes 0..3 source_len u32 | +| bytes 4..7 metadata_len u32 | +| bytes 8..11 footer_ver u32 (= 1) | +| bytes 12..23 reserved 12 bytes, zero | +| bytes 24..31 magic 8 bytes ASCII "AFBNDL01" | ++---------------------------------+ <- EOF +``` + +`AFBNDL01` is `0x41 0x46 0x42 0x4E 0x44 0x4C 0x30 0x31`. The two +trailing ASCII digits are the footer-format version, repeated for human +inspection (`tail -c 8 ./mybundle | xxd`); the binary `footer_ver` +field is the source of truth for parsing. + +Reader algorithm: + +1. Open the file. Seek to `EOF - 32`. Read 32 bytes. +2. Compare bytes 24..31 against `AFBNDL01`. If different, the file is + not a bundle; the scanner ignores it. +3. Parse `footer_ver`. If unknown, fail with a versioning error. +4. Compute `metadata_start = filesize - 32 - metadata_len` and + `source_start = metadata_start - source_len`. +5. Read `metadata_len` bytes from `metadata_start` for the manifest. +6. Read `source_len` bytes from `source_start` for the source view. + If `source_len == 0`, no source is embedded; the UI falls back to + "(source not available)". +7. Validate that `source_start >= 0` and that the implied "binary + region" (bytes `[0, source_start)`) is non-empty. + +Ordering note: source comes *before* metadata so a future +`format_version` can introduce extra trailing blobs (e.g. signed +checksums, compressed deps) by extending the trailer rather than +inserting between existing blobs. + +### Manifest schema changes + +The manifest content is the same YAML as today, with two field-level +changes that follow from the footer container: + +- **Drop `executable`.** The binary *is* the file; there is no + archive-relative path to record. +- **Redefine `source` as a display filename, not a path.** The source + bytes live in the footer; the manifest's `source` field carries the + original filename (e.g. `example.go`) so the UI can show it as a + filename in the source-view panel and pick a syntax-highlighting + mode from the extension. + +Everything else (`format_version`, `sdk.language`, `sdk.version`, +`dags`, the open-additivity rule for unknown keys) is unchanged. + +### Build pipeline + +The packer's behaviour from ADR 0002 changes only at the final write +step: + +1. Resolve target package, locate the file with `func main()`. (No + change.) +2. Run `go build [forwarded flags] -o `. (No change.) +3. Exec the freshly built binary with `--dump-bundle-spec` to obtain + the manifest. (No change.) +4. **New:** read the source file's bytes; serialise the manifest to + YAML; append `` to ``. +5. Default output path becomes `` (or `.exe` + on Windows), not `.zip`. + +Ordering against post-build steps: + +- **Strip:** must run *before* append. Stripping a file that already + has a footer either leaves the footer intact (most strip + implementations stop at the OS-defined end of the binary) or + truncates it; do not rely on either. +- **Code-sign:** must run *after* append on platforms whose signature + covers the entire file (Linux dm-verity, macOS post-Big-Sur for + certain notarisation flows, Windows Authenticode). The signature + then attests to the footer's contents along with the binary, which + is the property we want. +- **Compressors (UPX, etc.):** unsupported. UPX rewrites the file end + to end, destroying the trailer. Bundle binaries should not be + compressor-wrapped; this matches typical production deployment + practice. + +Determinism: the footer is byte-identical for byte-identical inputs +(source bytes, manifest YAML, layout), so a deterministic `go build` +plus a deterministic manifest serialisation produces a byte-identical +bundle file. We canonicalise the manifest as sorted-key YAML at write +time to avoid map-order non-determinism on the Go side. + +### Cross-language scope + +The bundle spec is language-agnostic by design. Every native-SDK +language we currently target (Go, Rust, C++, Zig) emits a single +statically-linked native executable; appending a fixed-format footer +is a few lines of code in each. The footer layout above is the +contract every SDK packer implements; the consumer-side scanner reads +it identically regardless of source language. + +Interpreted languages without a single binary artefact are out of +scope for the executable provider and therefore for this ADR. + +### Consumer-side changes + +The scanner currently iterates `*.zip` in `bundles_folder` and opens +each as an archive. It now iterates *all* regular files, reads the +last 32 bytes of each, and treats files whose magic matches as +bundles. Files without the magic are silently ignored (so a stray +README in the directory does not fail the scan). + +The runtime no longer has to materialise an executable from an +archive. It execs the bundle file directly, which removes the +transient cache directory and the chmod-after-extract step from +the spec. + +## Consequences + +### What this buys + +- **One artefact.** No `.zip` wrapper around a binary; the binary is + the deployment unit. `cp ./mybundle /opt/airflow/executable-bundles/` + is the deploy workflow. +- **No drift between binary and manifest.** They are produced and + committed in the same step and physically attached. +- **Atomic deploy.** A partially written file fails the magic check; + the scanner skips it cleanly instead of seeing half a manifest. +- **Smaller consumer surface.** No `archive/zip` dependency, no + per-bundle cache directory, no chmod-after-extract path, no + external-attributes handling for the executable bit. +- **Simpler runtime.** Exec the file directly. + +### What this costs + +- **Inspection needs a tool.** With ZIP, `unzip -p bundle.zip + airflow-metadata.yaml` worked from any shell. With the footer + format, ops needs a small CLI (`go tool airflow-go-pack inspect + ./mybundle` or equivalent) to dump the manifest and source. Cheap + to implement; the obligation is "ship it alongside the packer." +- **Build pipeline ordering matters.** Strip-then-append-then-sign is + the only correct order. Documented in the packer and in this ADR; + failure modes (stripped trailer, signature over the wrong bytes) + are loud (magic check fails, signature verification fails) rather + than silent. +- **Compressor incompatibility.** UPX and similar are not supported + for bundle binaries. Acceptable; production deployments do not + typically compress executables this way. +- **Magic-collision handling.** A non-bundle file in + `bundles_folder` whose last 8 bytes happen to be `AFBNDL01` would + be picked up as a bundle. Probability is negligible for a fixed + 8-byte ASCII string, and the next parse step (`footer_ver` check, + bounds check, YAML parse) catches a false positive cleanly. Not + worth a checksum in v1. +- **Footer format is now a wire format the SDK has to keep stable.** + `footer_ver = 1` is the only currently defined value; future + versions append fields after the version field but before the + reserved region, or use the reserved region. Older readers reject + unknown `footer_ver` rather than guessing. + +### Out of scope + +- Signed checksums in the footer. We rely on platform code-signing + (Authenticode, codesign, dm-verity) for tamper detection. A + bundle-level checksum could be added in a future `footer_ver` if + signing-free tamper detection becomes a requirement. +- Multiple source files. Only the root file (the file containing + `func main()`) is embedded. DAGs split across multiple source + files keep the rest of their sources outside the bundle; the UI + source-view shows only the entry file. Revisit if user feedback + requests broader source visibility. +- Compression of the source/metadata blobs. Both are tiny (kilobytes) + next to the binary; deflating them adds reader complexity for no + measurable space win. + +## Implementation notes + +- The append step is `os.OpenFile(out, O_RDWR|O_APPEND, 0)` plus three + writes (source, metadata, trailer) followed by `Close`. No mmap + needed. +- The executable bit on the output file is set by `go build` itself. + The append step preserves it (we write through, not truncate). +- The packer's existing reproducibility guarantees (sorted entries, + fixed mtimes) reduce to "write a deterministic YAML manifest"; the + ZIP-specific concerns (entry ordering, entry mtimes, external + attributes) go away. +- The Python-side scanner's bundle-detection helper lives next to + `BundleScanner`; it reads 32 bytes per file and parses YAML for + matching files. Keep it tolerant of trailing whitespace or short + files (anything `< 32` bytes is not a bundle). diff --git a/go-sdk/bundle/bundlev1/bundlev1server/server.go b/go-sdk/bundle/bundlev1/bundlev1server/server.go index 67d212ff06976..8e3612c2b602b 100644 --- a/go-sdk/bundle/bundlev1/bundlev1server/server.go +++ b/go-sdk/bundle/bundlev1/bundlev1server/server.go @@ -22,6 +22,7 @@ import ( "fmt" "log/slog" "os" + "runtime/debug" "github.com/evanphx/go-hclog-slog/hclogslog" "github.com/hashicorp/go-hclog" @@ -32,9 +33,34 @@ import ( "github.com/apache/airflow/go-sdk/bundle/bundlev1/bundlev1server/impl" "github.com/apache/airflow/go-sdk/pkg/bundles/shared" "github.com/apache/airflow/go-sdk/pkg/config" + "github.com/apache/airflow/go-sdk/pkg/execution" ) -var versionInfo *bool = flag.Bool("bundle-metadata", false, "show the embedded bundle info") +// sdkModulePath is the import path of the SDK module. Used to identify the +// SDK version from the bundle binary's build info dependencies. +const sdkModulePath = "github.com/apache/airflow/go-sdk" + +// Flags. The bundle-metadata flag is the existing ADR 0001 introspection +// hook; --comm and --logs select the coordinator-mode protocol added by +// ADR 0003. All three are read by Serve to choose a server mode below. +var ( + versionInfo = flag.Bool("bundle-metadata", false, "show the embedded bundle info") + dumpSpec = flag.Bool( + "dump-bundle-spec", + false, + "print the bundle spec JSON (sdk + dags) used by airflow-go-pack and exit", + ) + commAddr = flag.String( + "comm", + "", + "host:port of the supervisor's coordinator comm channel (selects coordinator mode)", + ) + logsAddr = flag.String( + "logs", + "", + "host:port of the supervisor's coordinator logs channel (selects coordinator mode)", + ) +) // ServeOpt is an interface for defining options that can be passed to the // Serve function. Each implementation modifies the ServeConfig being @@ -52,24 +78,31 @@ func (s serveConfigFunc) ApplyServeOpt(in *ServerConfig) error { type ServerConfig struct{} -// Serve is the entrypoint for your bundle, and sets it up ready for Airflow's Go Worker to use +// serveMode tags the protocol the binary will speak this run. +type serveMode int + +const ( + modePlugin serveMode = iota // go-plugin gRPC (existing Edge Worker path) + modeMetadataDump // --bundle-metadata: print BundleInfo JSON + modeSpecDump // --dump-bundle-spec: print bundle spec JSON (ADR 0002) + modeCoordinator // --comm/--logs: msgpack-over-IPC (ADR 0003) + modeUsageError // misuse: print usage and exit non-zero +) + +// Serve is the entrypoint for your bundle, and sets it up ready for Airflow's +// Go Worker (go-plugin) or Python supervisor (coordinator protocol) to use. // -// Zero or more options to configure the server may also be passed. There are no options yet, this is to allow -// future changes without breaking compatibility +// The mode is decided from CLI flags and process environment, so user code is +// always one line: +// +// func main() { bundlev1server.Serve(&myBundle{}) } +// +// Zero or more options to configure the server may also be passed. There are +// no options yet; the parameter exists to allow future additions without +// breaking compatibility. func Serve(bundle bundlev1.BundleProvider, opts ...ServeOpt) error { config.SetupViper("") - hcLogger := hclog.New(&hclog.LoggerOptions{ - Level: hclog.Trace, - Output: os.Stderr, - JSONFormat: true, - IncludeLocation: true, - AdditionalLocationOffset: 3, - }) - - log := slog.New(hclogslog.Adapt(hcLogger)) - slog.SetDefault(log) - flag.Parse() serveConfig := &ServerConfig{} @@ -77,16 +110,156 @@ func Serve(bundle bundlev1.BundleProvider, opts ...ServeOpt) error { c.ApplyServeOpt(serveConfig) } + switch decideMode() { + case modeMetadataDump: + return dumpBundleMetadata(bundle) + case modeSpecDump: + return dumpBundleSpec(bundle) + case modeCoordinator: + // In coordinator mode the supervisor reads the logs channel for + // structured records, so configuring the hclog/stderr default + // logger here is unnecessary — execution.Serve installs its own + // slog handler against the logs socket before any user code runs. + return execution.Serve(bundle, *commAddr, *logsAddr) + case modePlugin: + installPluginLogger() + return servePlugin(bundle) + case modeUsageError: + fmt.Fprintln(os.Stderr, "error: --comm and --logs must be supplied together") + flag.CommandLine.SetOutput(os.Stderr) + flag.Usage() + os.Exit(2) + } + return nil +} + +func decideMode() serveMode { if *versionInfo { - meta := bundle.GetBundleVersion() - data, err := json.MarshalIndent(meta, "", " ") - if err != nil { - return err + return modeMetadataDump + } + if *dumpSpec { + return modeSpecDump + } + commSet := *commAddr != "" + logsSet := *logsAddr != "" + if commSet && logsSet { + return modeCoordinator + } + if commSet || logsSet { + // Partial use is a hard error per ADR 0003: both flags are + // required, otherwise the supervisor is misconfigured and the + // runtime should fail loudly rather than fall through to + // go-plugin (which would hang on the missing magic-cookie). + return modeUsageError + } + return modePlugin +} + +func dumpBundleMetadata(bundle bundlev1.BundleProvider) error { + meta := bundle.GetBundleVersion() + data, err := json.MarshalIndent(meta, "", " ") + if err != nil { + return err + } + fmt.Println(string(data)) + return nil +} + +// bundleSpec is the wire shape printed by --dump-bundle-spec. The schema is +// stable per ADR 0002 and consumed by airflow-go-pack to populate the +// bundle's airflow-metadata.yaml at build time. +type bundleSpec struct { + FormatVersion string `json:"format_version"` + SDK bundleSpecSDK `json:"sdk"` + Dags map[string]bundleSpecDag `json:"dags"` +} + +type bundleSpecSDK struct { + Language string `json:"language"` + Version string `json:"version"` +} + +type bundleSpecDag struct { + Tasks []string `json:"tasks"` +} + +// dumpBundleSpec runs the bundle's RegisterDags against an in-memory recorder +// and writes the bundle spec JSON to stdout. It must not start the gRPC +// server or contact any external services; the recorder is the only side +// effect. +func dumpBundleSpec(bundle bundlev1.BundleProvider) error { + reg := bundlev1.New() + if err := bundle.RegisterDags(reg); err != nil { + return fmt.Errorf("registering dags: %w", err) + } + + enum, ok := reg.(bundlev1.EnumerableBundle) + if !ok { + return fmt.Errorf("registry does not implement EnumerableBundle") + } + + spec := bundleSpec{ + FormatVersion: "1.0", + SDK: bundleSpecSDK{ + Language: "go", + Version: sdkVersion(), + }, + Dags: make(map[string]bundleSpecDag), + } + for _, dag := range enum.OrderedDags() { + taskIDs := make([]string, 0, len(dag.Tasks)) + for _, t := range dag.Tasks { + taskIDs = append(taskIDs, t.ID) } - fmt.Println(string(data)) - return nil + spec.Dags[dag.DagID] = bundleSpecDag{Tasks: taskIDs} + } + + data, err := json.MarshalIndent(spec, "", " ") + if err != nil { + return err } + fmt.Println(string(data)) + return nil +} +// sdkVersion returns the version of the SDK module linked into this binary, +// derived from runtime/debug.ReadBuildInfo. Falls back to "(devel)" when +// build info is unavailable (e.g. tests, bundle binaries built from a local +// replace directive). +func sdkVersion() string { + info, ok := debug.ReadBuildInfo() + if !ok { + return "(devel)" + } + if info.Main.Path == sdkModulePath && info.Main.Version != "" { + return info.Main.Version + } + for _, dep := range info.Deps { + if dep.Path == sdkModulePath { + if dep.Replace != nil && dep.Replace.Version != "" { + return dep.Replace.Version + } + if dep.Version != "" { + return dep.Version + } + } + } + return "(devel)" +} + +func installPluginLogger() { + hcLogger := hclog.New(&hclog.LoggerOptions{ + Level: hclog.Trace, + Output: os.Stderr, + JSONFormat: true, + IncludeLocation: true, + AdditionalLocationOffset: 3, + }) + log := slog.New(hclogslog.Adapt(hcLogger)) + slog.SetDefault(log) +} + +func servePlugin(bundle bundlev1.BundleProvider) error { pluginConfig := &plugin.ServeConfig{ HandshakeConfig: shared.Handshake, Plugins: plugin.PluginSet{ @@ -99,6 +272,5 @@ func Serve(bundle bundlev1.BundleProvider, opts ...ServeOpt) error { // Likely never returns plugin.Serve(pluginConfig) - return nil } diff --git a/go-sdk/bundle/bundlev1/registry.go b/go-sdk/bundle/bundlev1/registry.go index 8d902efa081f6..9e302c679332b 100644 --- a/go-sdk/bundle/bundlev1/registry.go +++ b/go-sdk/bundle/bundlev1/registry.go @@ -23,6 +23,7 @@ import ( "runtime" "strings" "sync" + "time" "github.com/apache/airflow/go-sdk/pkg/worker" ) @@ -32,20 +33,129 @@ type ( Bundle = worker.Bundle Dag interface { - AddTask(fn any) - AddTaskWithName(taskId string, fn any) + // AddTask registers fn as a task in this Dag using fn's Go name as + // the task id. spec carries optional per-task configuration (pass + // TaskSpec{} for defaults). depends lists task ids in the same Dag + // that must run before this one; each must already be registered. + // Pass nil for no dependencies. + AddTask(fn any, spec TaskSpec, depends []string) + // AddTaskWithName is AddTask with an explicit task id. + AddTaskWithName(taskId string, fn any, spec TaskSpec, depends []string) } // Registry defines the interface that lets user code add dags and tasks, and extends Bundle for execution // time Registry interface { Bundle - AddDag(dagId string) Dag + AddDag(dagId string, spec ...DagSpec) Dag + } + + // TaskSpec is the optional configuration applied to a task at registration + // time. Every field is optional: a zero value means "unset" and the + // scheduler falls back to its serialization-schema default. The field + // names mirror the keys defined under "operator" in + // airflow-core/src/airflow/serialization/schema.json. + TaskSpec struct { + Queue string + Pool string + PoolSlots int + Retries int + RetryDelay time.Duration + MaxRetryDelay time.Duration + RetryExponentialBackoff float64 + PriorityWeight int + WeightRule string + TriggerRule string + Owner string + ExecutionTimeout time.Duration + Executor string + StartDate time.Time + EndDate time.Time + DependsOnPast bool + WaitForDownstream bool + // DoXComPush, EmailOnFailure, and EmailOnRetry default to true in the + // scheduler. A nil pointer means "unset" so the field is omitted from + // the serialized payload; pass Bool(false) to explicitly opt out. + DoXComPush *bool + EmailOnFailure *bool + EmailOnRetry *bool + DocMD string + MapIndexTemplate string + MaxActiveTisPerDag int + MaxActiveTisPerDagrun int + } + + // DagSpec is the optional configuration applied to a DAG at registration + // time. Every field is optional: a zero value means "unset" and the + // scheduler falls back to its serialization-schema default. The field + // names mirror the keys defined under "dag" in + // airflow-core/src/airflow/serialization/schema.json. + DagSpec struct { + // Schedule is "@once", "@continuous", a cron expression, or "" for + // NullTimetable (no schedule). + Schedule string + Description string + StartDate time.Time + EndDate time.Time + Tags []string + DagDisplayName string + DocMD string + MaxActiveTasks int + MaxActiveRuns int + MaxConsecutiveFailedDagRuns int + DagrunTimeout time.Duration + Catchup bool + FailFast bool + RenderTemplateAsNativeObj bool + DisableBundleVersioning bool + // IsPausedUponCreation has no schema default. nil means "unset"; pass + // Bool(true) or Bool(false) to set it explicitly. + IsPausedUponCreation *bool + } + + // TaskInfo describes a registered task. Coordinator-mode DAG parsing uses + // it to render the per-task block of a DagFileParsingResult. + TaskInfo struct { + // ID is the user-visible task id (the function name unless overridden + // via AddTaskWithName). + ID string + // TypeName is the unqualified Go function name (e.g. "extract"). + TypeName string + // PkgPath is the Go package path (e.g. "main", "github.com/x/y"). + PkgPath string + // Spec carries the optional per-task configuration supplied at + // registration. The zero value means "no overrides". + Spec TaskSpec + // Downstream lists task ids that depend on this task, populated as + // later tasks declare this id in their AddTask `depends` argument. + // Order is registration order; the serializer sorts before emit. + Downstream []string + } + + // DagInfo describes a registered dag together with its tasks in + // registration order. + DagInfo struct { + DagID string + // Spec carries the optional per-dag configuration supplied at + // registration. The zero value means "no overrides". + Spec DagSpec + Tasks []TaskInfo + } + + // EnumerableBundle exposes the dag/task identity recorded by + // RegisterDags. The default registry implements it; the coordinator-mode + // runtime relies on it for the DAG-parse one-shot. + EnumerableBundle interface { + OrderedDags() []DagInfo } registry struct { sync.RWMutex taskFuncMap map[string]map[string]Task + taskInfo map[string]map[string]TaskInfo + dagSpec map[string]DagSpec + dagOrder []string + taskOrder map[string][]string } ) @@ -54,36 +164,75 @@ type dagShim struct { registry *registry } -func (d dagShim) AddTask(fn any) { - d.registry.registerTask(d.dagId, fn) +func (d dagShim) AddTask(fn any, spec TaskSpec, depends []string) { + d.registry.registerTask(d.dagId, fn, spec, depends) } -func (d dagShim) AddTaskWithName(taskId string, fn any) { - d.registry.registerTaskWithName(d.dagId, taskId, fn) +func (d dagShim) AddTaskWithName(taskId string, fn any, spec TaskSpec, depends []string) { + d.registry.registerTaskWithName(d.dagId, taskId, fn, spec, depends) +} + +// Bool returns a pointer to b. Use it for the *bool fields on TaskSpec / +// DagSpec where nil means "leave at schema default": +// +// v1.TaskSpec{DoXComPush: v1.Bool(false)} +func Bool(b bool) *bool { + return &b +} + +func optionalSpec[T any](specs []T, caller string) T { + switch len(specs) { + case 0: + var zero T + return zero + case 1: + return specs[0] + default: + panic(fmt.Errorf("%s accepts at most one spec, got %d", caller, len(specs))) + } } // Function New creates a new bundle on which Dag and Tasks can be registered func New() Registry { - return ®istry{taskFuncMap: make(map[string]map[string]Task)} + return ®istry{ + taskFuncMap: make(map[string]map[string]Task), + taskInfo: make(map[string]map[string]TaskInfo), + dagSpec: make(map[string]DagSpec), + taskOrder: make(map[string][]string), + } +} + +func splitFullName(fullName string) (typeName, pkgPath string) { + // fullName looks like "main.extract" or "github.com/x/y.MyTask"; method + // values get a "-fm" suffix. + lastDot := strings.LastIndex(fullName, ".") + if lastDot < 0 { + return strings.TrimSuffix(fullName, "-fm"), "" + } + return strings.TrimSuffix(fullName[lastDot+1:], "-fm"), fullName[:lastDot] } func getFnName(fn reflect.Value) string { fullName := runtime.FuncForPC(fn.Pointer()).Name() - parts := strings.Split(fullName, ".") - fnName := parts[len(parts)-1] - // Go adds `-fm` suffix to a method names - return strings.TrimSuffix(fnName, "-fm") + name, _ := splitFullName(fullName) + return name } -func (r *registry) AddDag(dagId string) Dag { +func (r *registry) AddDag(dagId string, spec ...DagSpec) Dag { + dagSpec := optionalSpec(spec, "AddDag") + r.RWMutex.Lock() + defer r.RWMutex.Unlock() if _, exists := r.taskFuncMap[dagId]; exists { panic(fmt.Errorf("Dag %q already exists in bundle", dagId)) } r.taskFuncMap[dagId] = make(map[string]Task) + r.taskInfo[dagId] = make(map[string]TaskInfo) + r.dagSpec[dagId] = dagSpec + r.dagOrder = append(r.dagOrder, dagId) return dagShim{dagId, r} } -func (r *registry) registerTask(dagId string, fn any) { +func (r *registry) registerTask(dagId string, fn any, spec TaskSpec, depends []string) { val := reflect.ValueOf(fn) if val.Kind() != reflect.Func { @@ -92,30 +241,67 @@ func (r *registry) registerTask(dagId string, fn any) { fnName := getFnName(val) - r.registerTaskWithName(dagId, fnName, fn) + r.registerTaskWithName(dagId, fnName, fn, spec, depends) } -func (r *registry) registerTaskWithName(dagId, taskId string, fn any) { +func (r *registry) registerTaskWithName( + dagId, taskId string, + fn any, + spec TaskSpec, + depends []string, +) { task, err := NewTaskFunction(fn) if err != nil { panic(fmt.Errorf("error registering task %q for DAG %q: %w", taskId, dagId, err)) } + val := reflect.ValueOf(fn) + fullName := runtime.FuncForPC(val.Pointer()).Name() + typeName, pkgPath := splitFullName(fullName) + + info := TaskInfo{ID: taskId, TypeName: typeName, PkgPath: pkgPath, Spec: spec} + r.RWMutex.Lock() defer r.RWMutex.Unlock() dagTasks, exists := r.taskFuncMap[dagId] - if !exists { dagTasks = make(map[string]Task) r.taskFuncMap[dagId] = dagTasks + r.taskInfo[dagId] = make(map[string]TaskInfo) + r.dagOrder = append(r.dagOrder, dagId) } - _, exists = dagTasks[taskId] - if exists { + if _, exists := dagTasks[taskId]; exists { panic(fmt.Errorf("taskId %q is already registered for DAG %q", taskId, dagId)) } + + // Resolve depends to upstream TaskInfo entries, validating each exists. + // We dedupe so a repeated id in `depends` only records one downstream + // edge on the parent. + seen := make(map[string]bool, len(depends)) + for _, dep := range depends { + if dep == taskId { + panic(fmt.Errorf("task %q cannot depend on itself in DAG %q", taskId, dagId)) + } + if seen[dep] { + continue + } + seen[dep] = true + parent, ok := r.taskInfo[dagId][dep] + if !ok { + panic(fmt.Errorf( + "task %q depends on unknown task %q in DAG %q; register upstream tasks first", + taskId, dep, dagId, + )) + } + parent.Downstream = append(parent.Downstream, taskId) + r.taskInfo[dagId][dep] = parent + } + dagTasks[taskId] = task + r.taskInfo[dagId][taskId] = info + r.taskOrder[dagId] = append(r.taskOrder[dagId], taskId) } func (r *registry) LookupTask(dagId, taskId string) (task Task, exists bool) { @@ -129,3 +315,22 @@ func (r *registry) LookupTask(dagId, taskId string) (task Task, exists bool) { task, exists = dagTasks[taskId] return task, exists } + +// OrderedDags returns the registered dags in the order AddDag was called, +// each with its tasks in the order AddTask / AddTaskWithName was called. The +// returned slice is freshly allocated; callers may mutate it freely. +func (r *registry) OrderedDags() []DagInfo { + r.RLock() + defer r.RUnlock() + + out := make([]DagInfo, 0, len(r.dagOrder)) + for _, dagID := range r.dagOrder { + taskIDs := r.taskOrder[dagID] + tasks := make([]TaskInfo, 0, len(taskIDs)) + for _, tid := range taskIDs { + tasks = append(tasks, r.taskInfo[dagID][tid]) + } + out = append(out, DagInfo{DagID: dagID, Spec: r.dagSpec[dagID], Tasks: tasks}) + } + return out +} diff --git a/go-sdk/bundle/bundlev1/registry_test.go b/go-sdk/bundle/bundlev1/registry_test.go index 25105cd145598..31f4ce2b7326e 100644 --- a/go-sdk/bundle/bundlev1/registry_test.go +++ b/go-sdk/bundle/bundlev1/registry_test.go @@ -67,14 +67,14 @@ func (s *RegistrySuite) TestAddDag_DuplicatePanics() { } func (s *RegistrySuite) TestAddTask_RegistersAndFindsTask() { - s.dag.AddTask(myTask) + s.dag.AddTask(myTask, TaskSpec{}, nil) task, exists := s.reg.LookupTask("dag1", "myTask") s.True(exists) s.NotNil(task) } func (s *RegistrySuite) TestAddTaskWithName_RegistersAndFindsTask() { - s.dag.AddTaskWithName("special", myTask) + s.dag.AddTaskWithName("special", myTask, TaskSpec{}, nil) task, exists := s.reg.LookupTask("dag1", "special") s.True(exists) s.NotNil(task) @@ -85,20 +85,20 @@ func (s *RegistrySuite) TestAddTaskWithName_RegistersAndFindsTask() { } func (s *RegistrySuite) TestRegisterTaskWithName_DuplicatePanics() { - s.dag.AddTaskWithName("special", myTask) + s.dag.AddTaskWithName("special", myTask, TaskSpec{}, nil) s.PanicsWithError("taskId \"special\" is already registered for DAG \"dag1\"", func() { - s.dag.AddTaskWithName("special", myTask) + s.dag.AddTaskWithName("special", myTask, TaskSpec{}, nil) }) } func (s *RegistrySuite) TestAddTask_NonFuncPanics() { s.PanicsWithError("task fn was a string, not a func", func() { - s.dag.AddTask("not a func") + s.dag.AddTask("not a func", TaskSpec{}, nil) }) } func (s *RegistrySuite) TestAddTaskWithArgs_BindsCorrectArgs() { - s.dag.AddTask(myTaskWithArgs) + s.dag.AddTask(myTaskWithArgs, TaskSpec{}, nil) task, exists := s.reg.LookupTask("dag1", "myTaskWithArgs") s.True(exists) s.NotNil(task) @@ -108,13 +108,131 @@ func (s *RegistrySuite) TestAddTask_InvalidReturnType() { s.PanicsWithError( "error registering task \"NotErrorRet\" for DAG \"dag1\": expected task function github.com/apache/airflow/go-sdk/bundle/bundlev1.NotErrorRet last return value to return error but found int", func() { - s.dag.AddTask(NotErrorRet) + s.dag.AddTask(NotErrorRet, TaskSpec{}, nil) }, ) } func (s *RegistrySuite) TestAddTask_ErrorReturnType() { - s.dag.AddTask(errorTask) + s.dag.AddTask(errorTask, TaskSpec{}, nil) _, exists := s.reg.LookupTask("dag1", "errorTask") s.True(exists) } + +func (s *RegistrySuite) TestAddTask_WithSpec() { + s.dag.AddTask(myTask, TaskSpec{Queue: "high_mem", Retries: 3, DoXComPush: Bool(false)}, nil) + enum, ok := s.reg.(EnumerableBundle) + s.Require().True(ok) + dags := enum.OrderedDags() + s.Require().Len(dags, 1) + s.Require().Len(dags[0].Tasks, 1) + got := dags[0].Tasks[0] + s.Equal("myTask", got.ID) + s.Equal("high_mem", got.Spec.Queue) + s.Equal(3, got.Spec.Retries) + s.Require().NotNil(got.Spec.DoXComPush) + s.False(*got.Spec.DoXComPush) +} + +func (s *RegistrySuite) TestAddTaskWithName_WithSpec() { + s.dag.AddTaskWithName("special", myTask, TaskSpec{Queue: "gpu", Pool: "gpu_pool"}, nil) + enum, ok := s.reg.(EnumerableBundle) + s.Require().True(ok) + dags := enum.OrderedDags() + s.Require().Len(dags, 1) + s.Require().Len(dags[0].Tasks, 1) + got := dags[0].Tasks[0] + s.Equal("special", got.ID) + s.Equal("gpu", got.Spec.Queue) + s.Equal("gpu_pool", got.Spec.Pool) +} + +func (s *RegistrySuite) TestAddTask_DependsRecordsDownstream() { + s.dag.AddTaskWithName("extract", myTask, TaskSpec{}, nil) + s.dag.AddTaskWithName("transform", myTask, TaskSpec{}, []string{"extract"}) + s.dag.AddTaskWithName("load", myTask, TaskSpec{}, []string{"transform"}) + + enum := s.reg.(EnumerableBundle) + tasks := enum.OrderedDags()[0].Tasks + byID := make(map[string]TaskInfo, len(tasks)) + for _, t := range tasks { + byID[t.ID] = t + } + s.Equal([]string{"transform"}, byID["extract"].Downstream) + s.Equal([]string{"load"}, byID["transform"].Downstream) + s.Nil(byID["load"].Downstream) +} + +func (s *RegistrySuite) TestAddTask_FanOutFanIn() { + s.dag.AddTaskWithName("extract", myTask, TaskSpec{}, nil) + s.dag.AddTaskWithName("transform_a", myTask, TaskSpec{}, []string{"extract"}) + s.dag.AddTaskWithName("transform_b", myTask, TaskSpec{}, []string{"extract"}) + s.dag.AddTaskWithName("load", myTask, TaskSpec{}, []string{"transform_a", "transform_b"}) + + enum := s.reg.(EnumerableBundle) + tasks := enum.OrderedDags()[0].Tasks + byID := make(map[string]TaskInfo, len(tasks)) + for _, t := range tasks { + byID[t.ID] = t + } + s.ElementsMatch([]string{"transform_a", "transform_b"}, byID["extract"].Downstream) + s.Equal([]string{"load"}, byID["transform_a"].Downstream) + s.Equal([]string{"load"}, byID["transform_b"].Downstream) +} + +func (s *RegistrySuite) TestAddTask_DependsDuplicatesIgnored() { + s.dag.AddTaskWithName("extract", myTask, TaskSpec{}, nil) + s.dag.AddTaskWithName("load", myTask, TaskSpec{}, []string{"extract", "extract"}) + + enum := s.reg.(EnumerableBundle) + tasks := enum.OrderedDags()[0].Tasks + byID := make(map[string]TaskInfo, len(tasks)) + for _, t := range tasks { + byID[t.ID] = t + } + s.Equal([]string{"load"}, byID["extract"].Downstream) +} + +func (s *RegistrySuite) TestAddTask_DependsUnknownPanics() { + s.PanicsWithError( + `task "load" depends on unknown task "extract" in DAG "dag1"; register upstream tasks first`, + func() { + s.dag.AddTaskWithName("load", myTask, TaskSpec{}, []string{"extract"}) + }, + ) +} + +func (s *RegistrySuite) TestAddTask_DependsOnSelfPanics() { + s.PanicsWithError(`task "self" cannot depend on itself in DAG "dag1"`, func() { + s.dag.AddTaskWithName("self", myTask, TaskSpec{}, []string{"self"}) + }) +} + +func (s *RegistrySuite) TestAddDag_WithSpec() { + dag2 := s.reg.AddDag( + "dag2", + DagSpec{Schedule: "@daily", Tags: []string{"etl"}, MaxActiveRuns: 4}, + ) + s.NotNil(dag2) + enum, ok := s.reg.(EnumerableBundle) + s.Require().True(ok) + dags := enum.OrderedDags() + s.Require().Len(dags, 2) + var got DagInfo + for _, d := range dags { + if d.DagID == "dag2" { + got = d + break + } + } + s.Equal("dag2", got.DagID) + s.Equal("@daily", got.Spec.Schedule) + s.Equal([]string{"etl"}, got.Spec.Tags) + s.Equal(4, got.Spec.MaxActiveRuns) +} + +func (s *RegistrySuite) TestAddDag_TooManySpecsPanics() { + s.PanicsWithError("AddDag accepts at most one spec, got 2", func() { + s.reg.AddDag("dag3", DagSpec{}, DagSpec{}) + }) +} diff --git a/go-sdk/bundle/bundlev1/task.go b/go-sdk/bundle/bundlev1/task.go index 4271f4892bb6f..5277f40681cad 100644 --- a/go-sdk/bundle/bundlev1/task.go +++ b/go-sdk/bundle/bundlev1/task.go @@ -45,7 +45,12 @@ func NewTaskFunction(fn any) (Task, error) { func (f *taskFunction) Execute(ctx context.Context, logger *slog.Logger) error { fnType := f.fn.Type() - sdkClient := sdk.NewClient() + var sdkClient sdk.Client + if injected, ok := ctx.Value(sdkcontext.SdkClientContextKey).(sdk.Client); ok { + sdkClient = injected + } else { + sdkClient = sdk.NewClient() + } reflectArgs := make([]reflect.Value, fnType.NumIn()) for i := range reflectArgs { diff --git a/go-sdk/cmd/airflow-go-pack/inspect.go b/go-sdk/cmd/airflow-go-pack/inspect.go new file mode 100644 index 0000000000000..f9cca6d488b22 --- /dev/null +++ b/go-sdk/cmd/airflow-go-pack/inspect.go @@ -0,0 +1,57 @@ +// 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. + +package main + +import ( + "fmt" + + "github.com/spf13/cobra" + + "github.com/apache/airflow/go-sdk/internal/bundlefooter" +) + +func newInspectCmd() *cobra.Command { + var showSource bool + cmd := &cobra.Command{ + Use: "inspect ", + Short: "Print the manifest (and optionally source) embedded in a bundle", + Args: cobra.ExactArgs(1), + RunE: func(cmd *cobra.Command, args []string) error { + source, manifest, err := bundlefooter.Read(args[0]) + if err != nil { + return err + } + out := cmd.OutOrStdout() + if showSource { + fmt.Fprintln(out, "# --- source ---") + out.Write(source) + if len(source) > 0 && source[len(source)-1] != '\n' { + fmt.Fprintln(out) + } + fmt.Fprintln(out, "# --- manifest ---") + } + out.Write(manifest) + if len(manifest) > 0 && manifest[len(manifest)-1] != '\n' { + fmt.Fprintln(out) + } + return nil + }, + } + cmd.Flags().BoolVar(&showSource, "source", false, "also print the embedded source file") + return cmd +} diff --git a/go-sdk/cmd/airflow-go-pack/main.go b/go-sdk/cmd/airflow-go-pack/main.go new file mode 100644 index 0000000000000..3be934f18b2f0 --- /dev/null +++ b/go-sdk/cmd/airflow-go-pack/main.go @@ -0,0 +1,112 @@ +// 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. + +// Command airflow-go-pack builds a self-contained Airflow bundle from a Go +// package. It runs `go build`, exec's the freshly built binary with +// `--dump-bundle-spec` to obtain the manifest, and appends the source plus +// manifest plus AFBNDL01 trailer to the executable as specified by ADR 0004. +// +// Usage: +// +// go tool airflow-go-pack [./path/to/pkg] [-- ...] +// go tool airflow-go-pack --executable ./build/example --source main.go +// go tool airflow-go-pack inspect ./mybundle +// +// See go-sdk/adr/0002-use-go-tool-directive-for-bundle-packer.md and +// go-sdk/adr/0004-self-contained-executable-bundle.md. +package main + +import ( + "fmt" + "os" + + "github.com/spf13/cobra" +) + +func main() { + if err := newRootCmd().Execute(); err != nil { + fmt.Fprintln(os.Stderr, "error:", err) + os.Exit(1) + } +} + +func newRootCmd() *cobra.Command { + opts := &packOptions{} + + root := &cobra.Command{ + Use: "airflow-go-pack [package]", + Short: "Build a self-contained Airflow bundle from a Go package", + Long: `airflow-go-pack builds a Go bundle binary, queries it for its DAG/task +identity via --dump-bundle-spec, and appends the source plus an +airflow-metadata.yaml manifest plus an AFBNDL01 trailer to the +executable. The result is a single self-contained file that drops into +[executable] bundles_folder. + +By default the packer builds the package in the current directory. Pass +a different package as the positional argument; pass extra go build +flags after a "--" separator. + +Examples: + go tool airflow-go-pack + go tool airflow-go-pack ./cmd/my-bundle -- -trimpath -tags=prod + go tool airflow-go-pack --executable ./build/example --source main.go +`, + // Only count args BEFORE "--" toward the positional limit; args + // after "--" are forwarded verbatim to `go build` and must not + // inflate the count (e.g. `-- -ldflags "-X main.foo=bar"`). + Args: func(cmd *cobra.Command, args []string) error { + dashAt := cmd.ArgsLenAtDash() + pkgArgs := args + if dashAt >= 0 { + pkgArgs = args[:dashAt] + } + return cobra.MaximumNArgs(1)(cmd, pkgArgs) + }, + RunE: func(cmd *cobra.Command, args []string) error { + // Anything after "--" is forwarded to the internal `go build` + // invocation. ArgsLenAtDash() returns the count of args before + // the dash, or -1 if the dash isn't present. + dashAt := cmd.ArgsLenAtDash() + var pkgArgs, buildArgs []string + if dashAt < 0 { + pkgArgs = args + } else { + pkgArgs = args[:dashAt] + buildArgs = args[dashAt:] + } + opts.pkg = "." + if len(pkgArgs) == 1 { + opts.pkg = pkgArgs[0] + } + opts.buildArgs = buildArgs + return runPack(cmd.OutOrStdout(), cmd.ErrOrStderr(), opts) + }, + } + + root.Flags().StringVar(&opts.source, "source", + "", + "path to the DAG source file (defaults to the file in the target package containing func main)") + root.Flags().StringVar(&opts.executable, "executable", + "", + "pack a pre-built executable instead of running go build") + root.Flags().StringVar(&opts.output, "output", + "", + "output bundle path (defaults to ./)") + + root.AddCommand(newInspectCmd()) + return root +} diff --git a/go-sdk/cmd/airflow-go-pack/pack.go b/go-sdk/cmd/airflow-go-pack/pack.go new file mode 100644 index 0000000000000..16640faa899d3 --- /dev/null +++ b/go-sdk/cmd/airflow-go-pack/pack.go @@ -0,0 +1,407 @@ +// 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. + +package main + +import ( + "bytes" + "encoding/json" + "fmt" + "go/ast" + "go/parser" + "go/token" + "io" + "os" + "os/exec" + "path/filepath" + "runtime" + "sort" + + "gopkg.in/yaml.v3" + + "github.com/apache/airflow/go-sdk/internal/bundlefooter" +) + +// packOptions are the flags accepted by the root pack command. +type packOptions struct { + pkg string // target package (default ".") + source string // override the auto-detected DAG source file + executable string // pack a pre-built binary instead of building + output string // override the default output path + buildArgs []string // forwarded verbatim to `go build` (already includes the leading "--") +} + +// bundleSpec mirrors the JSON printed by --dump-bundle-spec. +type bundleSpec struct { + FormatVersion string `json:"format_version"` + SDK bundleSpecSDK `json:"sdk"` + Dags map[string]bundleSpecDag `json:"dags"` +} + +type bundleSpecSDK struct { + Language string `json:"language"` + Version string `json:"version"` +} + +type bundleSpecDag struct { + Tasks []string `json:"tasks"` +} + +// bundleMetadata mirrors --bundle-metadata's BundleInfo JSON. +type bundleMetadata struct { + Name string `json:"Name"` + Version *string `json:"Version,omitempty"` +} + +func runPack(stdout, stderr io.Writer, opts *packOptions) error { + if opts.executable != "" && len(opts.buildArgs) > 0 { + return fmt.Errorf("--executable is mutually exclusive with go build flags after \"--\"") + } + + sourcePath := opts.source + var execPath string + cleanupExec := func() {} + defer func() { cleanupExec() }() + + if opts.executable != "" { + execPath = opts.executable + if sourcePath == "" { + return fmt.Errorf( + "--executable requires --source: cannot infer the DAG source for a pre-built binary", + ) + } + } else { + discovered, err := discoverMainSource(opts.pkg) + if err != nil { + return fmt.Errorf("locating DAG source file: %w", err) + } + if sourcePath == "" { + sourcePath = discovered + } + + tmp, cleanup, err := buildPackage(stderr, opts.pkg, opts.buildArgs) + if err != nil { + return err + } + execPath = tmp + cleanupExec = cleanup + } + + if _, err := os.Stat(execPath); err != nil { + return fmt.Errorf("executable %s: %w", execPath, err) + } + if _, err := os.Stat(sourcePath); err != nil { + return fmt.Errorf("source file %s: %w", sourcePath, err) + } + + meta, err := readBundleMetadata(execPath) + if err != nil { + return fmt.Errorf("--bundle-metadata: %w", err) + } + if meta.Name == "" { + return fmt.Errorf( + "bundle binary returned an empty BundleInfo.Name; set bundleName at build time", + ) + } + + spec, err := readBundleSpec(execPath) + if err != nil { + return fmt.Errorf("--dump-bundle-spec: %w", err) + } + if len(spec.Dags) == 0 { + return fmt.Errorf("bundle exposes no dags: nothing to pack") + } + for dagID, dag := range spec.Dags { + if len(dag.Tasks) == 0 { + fmt.Fprintf(stderr, "warning: dag %q has no tasks\n", dagID) + } + } + + manifest, err := renderManifest(spec, filepath.Base(sourcePath)) + if err != nil { + return fmt.Errorf("rendering manifest: %w", err) + } + sourceBytes, err := os.ReadFile(sourcePath) + if err != nil { + return fmt.Errorf("reading source file: %w", err) + } + + output := opts.output + if output == "" { + output = defaultOutputPath(meta.Name) + } + + // Copy the executable to the output path before appending so we never + // mutate the build artefact in the temp dir or the user-supplied + // --executable file. + if err := copyFile(execPath, output, 0o755); err != nil { + return fmt.Errorf("writing %s: %w", output, err) + } + if err := bundlefooter.Append(output, sourceBytes, manifest); err != nil { + return err + } + + fmt.Fprintf(stdout, "Wrote bundle %s (sdk=%s/%s, dags=%d)\n", + output, spec.SDK.Language, spec.SDK.Version, len(spec.Dags)) + return nil +} + +func defaultOutputPath(bundleName string) string { + if runtime.GOOS == "windows" { + return bundleName + ".exe" + } + return bundleName +} + +// discoverMainSource locates the file in the given package whose AST contains +// a top-level `func main()`. Returns an error if the package has zero or +// more than one such file, mirroring ADR 0002's discovery contract. +func discoverMainSource(pkg string) (string, error) { + cmd := exec.Command("go", "list", "-f", "{{.Dir}}\n{{range .GoFiles}}{{.}}\n{{end}}", pkg) + var stdout, stderr bytes.Buffer + cmd.Stdout = &stdout + cmd.Stderr = &stderr + if err := cmd.Run(); err != nil { + return "", fmt.Errorf("go list %s: %w: %s", pkg, err, stderr.String()) + } + + lines := splitNonEmpty(stdout.String()) + if len(lines) < 2 { + return "", fmt.Errorf("package %s has no Go source files", pkg) + } + dir := lines[0] + files := lines[1:] + + fset := token.NewFileSet() + var matches []string + for _, name := range files { + full := filepath.Join(dir, name) + f, err := parser.ParseFile(fset, full, nil, parser.SkipObjectResolution) + if err != nil { + return "", fmt.Errorf("parsing %s: %w", full, err) + } + if hasMainFunc(f) { + matches = append(matches, full) + } + } + switch len(matches) { + case 0: + return "", fmt.Errorf("no file in package %s defines func main()", pkg) + case 1: + return matches[0], nil + default: + return "", fmt.Errorf( + "multiple files in package %s define func main(): %v; use --source to disambiguate", + pkg, + matches, + ) + } +} + +func hasMainFunc(f *ast.File) bool { + for _, decl := range f.Decls { + fn, ok := decl.(*ast.FuncDecl) + if !ok { + continue + } + if fn.Recv != nil { + continue + } + if fn.Name.Name != "main" { + continue + } + if fn.Type.Params != nil && len(fn.Type.Params.List) != 0 { + continue + } + return true + } + return false +} + +func splitNonEmpty(s string) []string { + var out []string + for _, line := range bytes.Split([]byte(s), []byte("\n")) { + t := bytes.TrimSpace(line) + if len(t) > 0 { + out = append(out, string(t)) + } + } + return out +} + +// buildPackage runs `go build [extraArgs...] -o /bundle ` and +// returns the path to the freshly built executable plus a cleanup function. +// extraArgs is the slice that comes after the "--" separator on the +// airflow-go-pack command line; we drop the leading "--" before forwarding. +func buildPackage(stderr io.Writer, pkg string, extraArgs []string) (string, func(), error) { + tmpDir, err := os.MkdirTemp("", "airflow-go-pack-*") + if err != nil { + return "", nil, fmt.Errorf("creating temp dir: %w", err) + } + cleanup := func() { _ = os.RemoveAll(tmpDir) } + + binName := "bundle" + if runtime.GOOS == "windows" { + binName += ".exe" + } + outPath := filepath.Join(tmpDir, binName) + + args := []string{"build"} + for _, a := range extraArgs { + if a == "--" { + continue + } + args = append(args, a) + } + args = append(args, "-o", outPath, pkg) + + cmd := exec.Command("go", args...) + cmd.Stdout = stderr + cmd.Stderr = stderr + if err := cmd.Run(); err != nil { + cleanup() + return "", nil, fmt.Errorf("go build failed: %w", err) + } + return outPath, cleanup, nil +} + +func readBundleMetadata(execPath string) (bundleMetadata, error) { + out, err := runIntrospect(execPath, "--bundle-metadata") + if err != nil { + return bundleMetadata{}, err + } + var meta bundleMetadata + if err := json.Unmarshal(out, &meta); err != nil { + return bundleMetadata{}, fmt.Errorf("decoding --bundle-metadata JSON: %w", err) + } + return meta, nil +} + +func readBundleSpec(execPath string) (bundleSpec, error) { + out, err := runIntrospect(execPath, "--dump-bundle-spec") + if err != nil { + return bundleSpec{}, err + } + var spec bundleSpec + if err := json.Unmarshal(out, &spec); err != nil { + return bundleSpec{}, fmt.Errorf("decoding --dump-bundle-spec JSON: %w", err) + } + return spec, nil +} + +func runIntrospect(execPath string, flag string) ([]byte, error) { + cmd := exec.Command(execPath, flag) + var stdout, stderr bytes.Buffer + cmd.Stdout = &stdout + cmd.Stderr = &stderr + if err := cmd.Run(); err != nil { + return nil, fmt.Errorf("%s %s: %w: %s", execPath, flag, err, stderr.String()) + } + return stdout.Bytes(), nil +} + +// renderManifest serialises the bundle spec as deterministic, sorted-key +// YAML matching the schema in providers/sdk/executable/docs/bundle-spec.rst. +func renderManifest(spec bundleSpec, sourceName string) ([]byte, error) { + if spec.FormatVersion == "" { + spec.FormatVersion = "1.0" + } + + dagIDs := make([]string, 0, len(spec.Dags)) + for id := range spec.Dags { + dagIDs = append(dagIDs, id) + } + sort.Strings(dagIDs) + + dagsNode := &yaml.Node{Kind: yaml.MappingNode} + for _, id := range dagIDs { + tasks := spec.Dags[id].Tasks + taskItems := make([]*yaml.Node, 0, len(tasks)) + for _, t := range tasks { + taskItems = append(taskItems, scalar(t)) + } + dagsNode.Content = append(dagsNode.Content, + scalar(id), + &yaml.Node{ + Kind: yaml.MappingNode, + Content: []*yaml.Node{ + scalar("tasks"), + {Kind: yaml.SequenceNode, Content: taskItems}, + }, + }, + ) + } + + root := &yaml.Node{Kind: yaml.DocumentNode} + manifest := &yaml.Node{ + Kind: yaml.MappingNode, + Content: []*yaml.Node{ + scalar("format_version"), quotedScalar(spec.FormatVersion), + scalar("sdk"), + { + Kind: yaml.MappingNode, + Content: []*yaml.Node{ + scalar("language"), scalar(spec.SDK.Language), + scalar("version"), quotedScalar(spec.SDK.Version), + }, + }, + scalar("source"), scalar(sourceName), + scalar("dags"), dagsNode, + }, + } + root.Content = []*yaml.Node{manifest} + + var buf bytes.Buffer + enc := yaml.NewEncoder(&buf) + enc.SetIndent(2) + if err := enc.Encode(root); err != nil { + return nil, err + } + if err := enc.Close(); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +func scalar(value string) *yaml.Node { + return &yaml.Node{Kind: yaml.ScalarNode, Value: value} +} + +func quotedScalar(value string) *yaml.Node { + return &yaml.Node{Kind: yaml.ScalarNode, Value: value, Style: yaml.DoubleQuotedStyle} +} + +// copyFile copies src to dst, truncating dst if it already exists. +func copyFile(src, dst string, mode os.FileMode) error { + in, err := os.Open(src) + if err != nil { + return err + } + defer in.Close() + out, err := os.OpenFile(dst, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, mode) + if err != nil { + return err + } + if _, err := io.Copy(out, in); err != nil { + out.Close() + return err + } + if err := out.Close(); err != nil { + return err + } + return os.Chmod(dst, mode) +} diff --git a/go-sdk/cmd/airflow-go-pack/pack_test.go b/go-sdk/cmd/airflow-go-pack/pack_test.go new file mode 100644 index 0000000000000..b6dc9f9280e69 --- /dev/null +++ b/go-sdk/cmd/airflow-go-pack/pack_test.go @@ -0,0 +1,99 @@ +// 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. + +package main + +import ( + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestRenderManifest_DeterministicDagOrdering(t *testing.T) { + spec := bundleSpec{ + FormatVersion: "1.0", + SDK: bundleSpecSDK{Language: "go", Version: "0.1.0"}, + Dags: map[string]bundleSpecDag{ + "zeta_dag": {Tasks: []string{"a", "b"}}, + "alpha_dag": {Tasks: []string{"x"}}, + }, + } + + got1, err := renderManifest(spec, "main.go") + require.NoError(t, err) + got2, err := renderManifest(spec, "main.go") + require.NoError(t, err) + + assert.Equal(t, got1, got2, "manifest should be byte-identical for identical input") + + expected := `format_version: "1.0" +sdk: + language: go + version: "0.1.0" +source: main.go +dags: + alpha_dag: + tasks: + - x + zeta_dag: + tasks: + - a + - b +` + assert.Equal(t, expected, string(got1)) +} + +func TestRenderManifest_EmptyDags(t *testing.T) { + spec := bundleSpec{ + FormatVersion: "1.0", + SDK: bundleSpecSDK{Language: "go", Version: "0.1.0"}, + Dags: map[string]bundleSpecDag{}, + } + got, err := renderManifest(spec, "main.go") + require.NoError(t, err) + assert.Contains(t, string(got), "dags: {}") +} + +// TestRootArgs_AllowsBuildFlagsAfterDoubleDash regression-tests the +// positional-arg validator: forwarded `go build` flags after "--" must not +// be counted against MaximumNArgs(1). The runtime call would otherwise fail +// with `accepts at most 1 arg(s), received N`. +func TestRootArgs_AllowsBuildFlagsAfterDoubleDash(t *testing.T) { + cases := [][]string{ + {"--", "-ldflags", "-X main.dagId=foo"}, + {"./pkg", "--", "-ldflags", "-X main.dagId=foo"}, + {"--", "-trimpath", "-tags=prod"}, + } + for _, argv := range cases { + cmd := newRootCmd() + // Stop the command from actually running; we only want arg validation. + cmd.RunE = func(*cobra.Command, []string) error { return nil } + cmd.SetArgs(argv) + assert.NoError(t, cmd.Execute(), "args=%v should validate", argv) + } +} + +func TestRootArgs_RejectsExtraPositionalBeforeDash(t *testing.T) { + cmd := newRootCmd() + cmd.RunE = func(*cobra.Command, []string) error { return nil } + cmd.SetArgs([]string{"./pkg1", "./pkg2", "--", "-ldflags", "-X main.dagId=foo"}) + err := cmd.Execute() + require.Error(t, err) + assert.Contains(t, err.Error(), "accepts at most 1 arg") +} diff --git a/go-sdk/example/bundle/Justfile b/go-sdk/example/bundle/Justfile index ca211b304b104..5ed6ef5d34a1a 100644 --- a/go-sdk/example/bundle/Justfile +++ b/go-sdk/example/bundle/Justfile @@ -21,13 +21,32 @@ default: @just --list -# Build the example bundle +# Build the example bundle (raw go build, no footer; for Edge Worker testing) build: @echo "Building example DAG bundle..." go build -o ../../bin/example-dag-bundle . - -# Build with specific name and version +# Build with specific name and version (raw go build, no footer) build-with name="data_processing_example" version="1.0.0": @echo "Building example DAG bundle with name={{name}} version={{version}}..." go build -ldflags="-X 'main.bundleName={{name}}' -X 'main.bundleVersion={{version}}'" -o ../../bin/{{name}}-{{version}} . + +# One-step build + pack. The single `go tool airflow-go-pack` +# invocation runs `go build` internally, queries the binary for its +# DAG/task identity via --dump-bundle-spec, and appends the source plus +# airflow-metadata.yaml plus AFBNDL01 trailer. The output is a single +# self-contained executable bundle, named after BundleInfo.Name and +# written to the current directory. Drop it into [executable] +# bundles_folder to deploy. +pack: + @echo "Packing example DAG bundle..." + go tool airflow-go-pack --output ../../bin/example_dags + +# Pack with extra go build flags forwarded after "--". +pack-release: + @echo "Packing example DAG bundle (release flags)..." + go tool airflow-go-pack --output ../../bin/example_dags -- -trimpath -ldflags="-s -w" + +# Inspect a packed bundle's embedded manifest. +inspect bundle="../../bin/example_dags": + go tool airflow-go-pack inspect {{bundle}} diff --git a/go-sdk/example/bundle/main.go b/go-sdk/example/bundle/main.go index 5e970da54951b..c3cc777a6b511 100644 --- a/go-sdk/example/bundle/main.go +++ b/go-sdk/example/bundle/main.go @@ -29,10 +29,14 @@ import ( "github.com/apache/airflow/go-sdk/sdk" ) -// Set by `-ldflags` at build time +// Set by `-ldflags` at build time. Override dagId to produce the same +// example bundle under a different DAG name — e.g. build once with +// `-X main.dagId=simple_dag` for the pure-Go scenario, and again with +// `-X main.dagId=go_multi_lang` for the Python-stub scenario. var ( bundleName = "example_dags" bundleVersion = "0.0" + dagId = "simple_dag" ) type myBundle struct{} @@ -45,10 +49,14 @@ func (m *myBundle) GetBundleVersion() v1.BundleInfo { } func (m *myBundle) RegisterDags(dagbag v1.Registry) error { - simpleDag := dagbag.AddDag("simple_dag") - simpleDag.AddTask(extract) - simpleDag.AddTask(transform) - simpleDag.AddTask(load) + dag := dagbag.AddDag(dagId, v1.DagSpec{ + Schedule: "@daily", + Description: "Example Go-authored Dag", + Tags: []string{"example", "go-sdk"}, + }) + dag.AddTask(extract, v1.TaskSpec{Queue: "go-task", Retries: 2}, nil) + dag.AddTask(transform, v1.TaskSpec{Queue: "go-task"}, []string{"extract"}) + dag.AddTask(load, v1.TaskSpec{Queue: "go-task"}, []string{"transform"}) return nil } diff --git a/go-sdk/go.mod b/go-sdk/go.mod index bfb400eee9432..cc738e4c3f862 100644 --- a/go-sdk/go.mod +++ b/go-sdk/go.mod @@ -4,6 +4,8 @@ go 1.24.0 toolchain go1.24.6 +tool github.com/apache/airflow/go-sdk/cmd/airflow-go-pack + require ( github.com/cappuccinotm/slogx v1.4.2 github.com/golang-jwt/jwt/v5 v5.3.0 @@ -16,6 +18,7 @@ require ( github.com/spf13/pflag v1.0.10 github.com/spf13/viper v1.20.1 github.com/stretchr/testify v1.11.1 + github.com/vmihailenco/msgpack/v5 v5.4.1 google.golang.org/grpc v1.79.3 google.golang.org/protobuf v1.36.10 resty.dev/v3 v3.0.0-beta.2 @@ -38,6 +41,7 @@ require ( github.com/spf13/afero v1.12.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/subosito/gotenv v1.6.0 // indirect + github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect go.opentelemetry.io/otel v1.41.0 // indirect go.opentelemetry.io/otel/trace v1.41.0 // indirect go.uber.org/multierr v1.10.0 // indirect @@ -58,5 +62,5 @@ require ( github.com/samber/slog-http v1.8.2 golang.org/x/sys v0.39.0 // indirect golang.org/x/text v0.32.0 // indirect - gopkg.in/yaml.v3 v3.0.1 // indirect + gopkg.in/yaml.v3 v3.0.1 ) diff --git a/go-sdk/go.sum b/go-sdk/go.sum index 5b7940672b1ca..a275d6b63c8be 100644 --- a/go-sdk/go.sum +++ b/go-sdk/go.sum @@ -114,6 +114,10 @@ github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= +github.com/vmihailenco/msgpack/v5 v5.4.1 h1:cQriyiUvjTwOHg8QZaPihLWeRAAVoCpE00IUPn0Bjt8= +github.com/vmihailenco/msgpack/v5 v5.4.1/go.mod h1:GaZTsDaehaPpQVyxrf5mtQlH+pc21PIudVV/E3rRQok= +github.com/vmihailenco/tagparser/v2 v2.0.0 h1:y09buUbR+b5aycVFQs/g70pqKVZNBmxwAhO7/IwNM9g= +github.com/vmihailenco/tagparser/v2 v2.0.0/go.mod h1:Wri+At7QHww0WTrCBeu4J6bNtoV6mEfg5OIWRZA9qds= go.opentelemetry.io/auto/sdk v1.2.1 h1:jXsnJ4Lmnqd11kwkBV2LgLoFMZKizbCi5fNZ/ipaZ64= go.opentelemetry.io/auto/sdk v1.2.1/go.mod h1:KRTj+aOaElaLi+wW1kO/DZRXwkF4C5xPbEe3ZiIhN7Y= go.opentelemetry.io/otel v1.41.0 h1:YlEwVsGAlCvczDILpUXpIpPSL/VPugt7zHThEMLce1c= diff --git a/go-sdk/internal/bundlefooter/footer.go b/go-sdk/internal/bundlefooter/footer.go new file mode 100644 index 0000000000000..75d518e027d1f --- /dev/null +++ b/go-sdk/internal/bundlefooter/footer.go @@ -0,0 +1,222 @@ +// 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. + +// Package bundlefooter implements the AFBNDL01 trailer described in +// ADR 0004 (and providers/sdk/executable/docs/bundle-spec.rst). A bundle +// file is the compiled executable with three appended regions: the source +// bytes, the manifest bytes, and a fixed 32-byte trailer that locates them. +// +// The trailer layout (all little-endian) is: +// +// bytes 0..3 source_len uint32 +// bytes 4..7 metadata_len uint32 +// bytes 8..11 footer_ver uint32 (= 1) +// bytes 12..23 reserved 12 bytes, zero +// bytes 24..31 magic 8 bytes ASCII "AFBNDL01" +package bundlefooter + +import ( + "encoding/binary" + "errors" + "fmt" + "io" + "math" + "os" +) + +const ( + // TrailerSize is the fixed length of the trailer, in bytes. + TrailerSize = 32 + + // FooterVersion is the currently defined trailer-format version. + FooterVersion = 1 + + // MaxRegionSize is the largest source or metadata region this footer + // format can address (uint32 length field). + MaxRegionSize = math.MaxUint32 +) + +// Magic is the 8-byte ASCII tag that identifies a file as a bundle. +var Magic = [8]byte{'A', 'F', 'B', 'N', 'D', 'L', '0', '1'} + +// ErrNotBundle is returned by Read when the file does not end with the +// AFBNDL01 magic. +var ErrNotBundle = errors.New("bundlefooter: not a bundle (magic mismatch)") + +// ErrUnknownVersion is returned by Read when the trailer's footer_ver field +// is something other than FooterVersion. +var ErrUnknownVersion = errors.New("bundlefooter: unknown footer version") + +// Trailer carries the parsed contents of a bundle's 32-byte trailer. +type Trailer struct { + SourceLen uint32 + MetadataLen uint32 + FooterVersion uint32 +} + +// Append writes the source bytes, metadata bytes, and trailer to the end of +// the file at execPath. The file's existing contents (the executable) are +// left intact and its mode bits are preserved. source MAY be nil/empty. +func Append(execPath string, source, metadata []byte) error { + if int64(len(source)) > MaxRegionSize { + return fmt.Errorf( + "bundlefooter: source region too large (%d bytes, max %d)", + len(source), + MaxRegionSize, + ) + } + if int64(len(metadata)) > MaxRegionSize { + return fmt.Errorf( + "bundlefooter: metadata region too large (%d bytes, max %d)", + len(metadata), + MaxRegionSize, + ) + } + + f, err := os.OpenFile(execPath, os.O_RDWR|os.O_APPEND, 0) + if err != nil { + return fmt.Errorf("bundlefooter: opening %s: %w", execPath, err) + } + defer f.Close() + + if len(source) > 0 { + if _, err := f.Write(source); err != nil { + return fmt.Errorf("bundlefooter: writing source region: %w", err) + } + } + if len(metadata) > 0 { + if _, err := f.Write(metadata); err != nil { + return fmt.Errorf("bundlefooter: writing metadata region: %w", err) + } + } + + trailer := encodeTrailer(uint32(len(source)), uint32(len(metadata))) + if _, err := f.Write(trailer[:]); err != nil { + return fmt.Errorf("bundlefooter: writing trailer: %w", err) + } + return nil +} + +// Read parses the trailer of the file at path and returns the embedded +// source and metadata regions. Returns ErrNotBundle if the magic does not +// match (so callers may silently ignore non-bundle files). +func Read(path string) (source, metadata []byte, err error) { + f, err := os.Open(path) + if err != nil { + return nil, nil, fmt.Errorf("bundlefooter: opening %s: %w", path, err) + } + defer f.Close() + + stat, err := f.Stat() + if err != nil { + return nil, nil, fmt.Errorf("bundlefooter: stat %s: %w", path, err) + } + size := stat.Size() + if size < TrailerSize { + return nil, nil, ErrNotBundle + } + + var trailer [TrailerSize]byte + if _, err := f.ReadAt(trailer[:], size-TrailerSize); err != nil { + return nil, nil, fmt.Errorf("bundlefooter: reading trailer: %w", err) + } + + t, err := decodeTrailer(trailer) + if err != nil { + return nil, nil, err + } + + metadataStart := size - TrailerSize - int64(t.MetadataLen) + sourceStart := metadataStart - int64(t.SourceLen) + if sourceStart < 0 { + return nil, nil, fmt.Errorf( + "bundlefooter: trailer reports regions larger than file (source_len=%d metadata_len=%d size=%d)", + t.SourceLen, + t.MetadataLen, + size, + ) + } + if sourceStart == 0 { + return nil, nil, fmt.Errorf("bundlefooter: empty binary region") + } + + if t.SourceLen > 0 { + source = make([]byte, t.SourceLen) + if _, err := f.ReadAt(source, sourceStart); err != nil && !errors.Is(err, io.EOF) { + return nil, nil, fmt.Errorf("bundlefooter: reading source region: %w", err) + } + } + if t.MetadataLen > 0 { + metadata = make([]byte, t.MetadataLen) + if _, err := f.ReadAt(metadata, metadataStart); err != nil && !errors.Is(err, io.EOF) { + return nil, nil, fmt.Errorf("bundlefooter: reading metadata region: %w", err) + } + } + return source, metadata, nil +} + +// IsBundle reports whether the file at path ends with the AFBNDL01 magic. +// It does not validate the trailer beyond the magic check, so a file with a +// matching magic but a corrupt trailer body still returns true. +func IsBundle(path string) (bool, error) { + f, err := os.Open(path) + if err != nil { + return false, err + } + defer f.Close() + + stat, err := f.Stat() + if err != nil { + return false, err + } + if stat.Size() < TrailerSize { + return false, nil + } + + var tail [8]byte + if _, err := f.ReadAt(tail[:], stat.Size()-int64(len(tail))); err != nil { + return false, err + } + return tail == Magic, nil +} + +func encodeTrailer(sourceLen, metadataLen uint32) [TrailerSize]byte { + var t [TrailerSize]byte + binary.LittleEndian.PutUint32(t[0:4], sourceLen) + binary.LittleEndian.PutUint32(t[4:8], metadataLen) + binary.LittleEndian.PutUint32(t[8:12], FooterVersion) + // bytes 12..23 are reserved, zero + copy(t[24:32], Magic[:]) + return t +} + +func decodeTrailer(b [TrailerSize]byte) (Trailer, error) { + var magic [8]byte + copy(magic[:], b[24:32]) + if magic != Magic { + return Trailer{}, ErrNotBundle + } + t := Trailer{ + SourceLen: binary.LittleEndian.Uint32(b[0:4]), + MetadataLen: binary.LittleEndian.Uint32(b[4:8]), + FooterVersion: binary.LittleEndian.Uint32(b[8:12]), + } + if t.FooterVersion != FooterVersion { + return Trailer{}, fmt.Errorf("%w: %d", ErrUnknownVersion, t.FooterVersion) + } + return t, nil +} diff --git a/go-sdk/internal/bundlefooter/footer_test.go b/go-sdk/internal/bundlefooter/footer_test.go new file mode 100644 index 0000000000000..33cbcafffd8ef --- /dev/null +++ b/go-sdk/internal/bundlefooter/footer_test.go @@ -0,0 +1,139 @@ +// 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. + +package bundlefooter + +import ( + "errors" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func writeTempBinary(t *testing.T, contents []byte) string { + t.Helper() + dir := t.TempDir() + path := filepath.Join(dir, "fake-binary") + require.NoError(t, os.WriteFile(path, contents, 0o755)) + return path +} + +func TestAppendAndRead_RoundTrip(t *testing.T) { + binary := []byte("\x7FELFnot-really-an-elf-but-good-enough") + source := []byte("package main\n\nfunc main() {}\n") + metadata := []byte("format_version: \"1.0\"\nsdk:\n language: go\n") + + path := writeTempBinary(t, binary) + require.NoError(t, Append(path, source, metadata)) + + got := mustRead(t, path) + assert.Equal(t, len(binary)+len(source)+len(metadata)+TrailerSize, got.size) + + gotSource, gotMetadata, err := Read(path) + require.NoError(t, err) + assert.Equal(t, source, gotSource) + assert.Equal(t, metadata, gotMetadata) + + ok, err := IsBundle(path) + require.NoError(t, err) + assert.True(t, ok) +} + +func TestAppend_ZeroLengthSource(t *testing.T) { + binary := []byte("\x7FELFstub") + metadata := []byte("manifest") + + path := writeTempBinary(t, binary) + require.NoError(t, Append(path, nil, metadata)) + + source, gotMetadata, err := Read(path) + require.NoError(t, err) + assert.Empty(t, source) + assert.Equal(t, metadata, gotMetadata) +} + +func TestAppend_DeterministicOutput(t *testing.T) { + binary := []byte("\x7FELFstub-binary-bytes") + source := []byte("source") + metadata := []byte("manifest") + + pathA := writeTempBinary(t, binary) + pathB := writeTempBinary(t, binary) + require.NoError(t, Append(pathA, source, metadata)) + require.NoError(t, Append(pathB, source, metadata)) + + a, err := os.ReadFile(pathA) + require.NoError(t, err) + b, err := os.ReadFile(pathB) + require.NoError(t, err) + assert.Equal(t, a, b) +} + +func TestRead_NotBundle(t *testing.T) { + path := writeTempBinary(t, []byte("just a regular file with no footer")) + + _, _, err := Read(path) + require.ErrorIs(t, err, ErrNotBundle) + + ok, err := IsBundle(path) + require.NoError(t, err) + assert.False(t, ok) +} + +func TestRead_TooShort(t *testing.T) { + path := writeTempBinary(t, []byte("hi")) + + _, _, err := Read(path) + require.ErrorIs(t, err, ErrNotBundle) +} + +func TestRead_UnknownVersion(t *testing.T) { + binary := []byte("\x7FELFstub") + source := []byte("src") + metadata := []byte("md") + path := writeTempBinary(t, binary) + require.NoError(t, Append(path, source, metadata)) + + // Mutate the version byte in the trailer. + f, err := os.OpenFile(path, os.O_RDWR, 0) + require.NoError(t, err) + stat, err := f.Stat() + require.NoError(t, err) + // footer_ver lives at bytes 8..11 of the trailer. + versionOffset := stat.Size() - TrailerSize + 8 + _, err = f.WriteAt([]byte{99, 0, 0, 0}, versionOffset) + require.NoError(t, err) + require.NoError(t, f.Close()) + + _, _, err = Read(path) + require.Error(t, err) + assert.True(t, errors.Is(err, ErrUnknownVersion)) +} + +type bundleStat struct { + size int +} + +func mustRead(t *testing.T, path string) bundleStat { + t.Helper() + stat, err := os.Stat(path) + require.NoError(t, err) + return bundleStat{size: int(stat.Size())} +} diff --git a/go-sdk/pkg/execution/client.go b/go-sdk/pkg/execution/client.go new file mode 100644 index 0000000000000..0e047803b6859 --- /dev/null +++ b/go-sdk/pkg/execution/client.go @@ -0,0 +1,183 @@ +// 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. + +package execution + +import ( + "context" + "encoding/json" + "fmt" + + "github.com/apache/airflow/go-sdk/pkg/api" + "github.com/apache/airflow/go-sdk/sdk" +) + +// CoordinatorClient implements sdk.Client by communicating with the Airflow supervisor +// over the comm socket using msgpack-framed IPC instead of HTTP. +type CoordinatorClient struct { + comm *CoordinatorComm + details *StartupDetails +} + +var _ sdk.Client = (*CoordinatorClient)(nil) + +// NewCoordinatorClient creates a new client backed by the comm socket. +func NewCoordinatorClient(comm *CoordinatorComm, details *StartupDetails) *CoordinatorClient { + return &CoordinatorClient{ + comm: comm, + details: details, + } +} + +// GetVariable requests a variable value from the supervisor. +func (c *CoordinatorClient) GetVariable(_ context.Context, key string) (string, error) { + resp, err := c.comm.Communicate(GetVariableMsg{Key: key}.toMap()) + if err != nil { + return "", err + } + + result, err := decodeVariableResult(resp) + if err != nil { + return "", fmt.Errorf("decoding variable result: %w", err) + } + + if result.Value == nil { + return "", fmt.Errorf("%w: %q", sdk.VariableNotFound, key) + } + + switch v := result.Value.(type) { + case string: + return v, nil + default: + // If the value is not a string, marshal it to JSON. + b, err := json.Marshal(v) + if err != nil { + return "", fmt.Errorf("marshaling variable value: %w", err) + } + return string(b), nil + } +} + +// UnmarshalJSONVariable gets a variable and unmarshals its JSON value. +func (c *CoordinatorClient) UnmarshalJSONVariable( + ctx context.Context, + key string, + pointer any, +) error { + val, err := c.GetVariable(ctx, key) + if err != nil { + return err + } + return json.Unmarshal([]byte(val), pointer) +} + +// GetConnection requests a connection from the supervisor. +func (c *CoordinatorClient) GetConnection( + _ context.Context, + connID string, +) (sdk.Connection, error) { + resp, err := c.comm.Communicate(GetConnectionMsg{ConnID: connID}.toMap()) + if err != nil { + return sdk.Connection{}, err + } + + result, err := decodeConnectionResult(resp) + if err != nil { + return sdk.Connection{}, fmt.Errorf("decoding connection result: %w", err) + } + + conn := sdk.Connection{ + ID: result.ConnID, + Type: result.ConnType, + Host: result.Host, + Port: result.Port, + Path: result.Schema, + } + + if result.Login != "" { + login := result.Login + conn.Login = &login + } + if result.Password != "" { + password := result.Password + conn.Password = &password + } + if result.Extra != "" { + conn.Extra = map[string]any{} + if err := json.Unmarshal([]byte(result.Extra), &conn.Extra); err != nil { + return conn, fmt.Errorf("parsing connection extra: %w", err) + } + } + + return conn, nil +} + +// GetXCom requests an XCom value from the supervisor. +func (c *CoordinatorClient) GetXCom( + _ context.Context, + dagId, runId, taskId string, + mapIndex *int, + key string, + _ any, +) (any, error) { + msg := GetXComMsg{ + Key: key, + DagID: dagId, + TaskID: taskId, + RunID: runId, + } + if mapIndex != nil { + msg.MapIndex = mapIndex + } + + resp, err := c.comm.Communicate(msg.toMap()) + if err != nil { + return nil, err + } + + result, err := decodeXComResult(resp) + if err != nil { + return nil, fmt.Errorf("decoding xcom result: %w", err) + } + + return result.Value, nil +} + +// PushXCom sends an XCom value to the supervisor. +func (c *CoordinatorClient) PushXCom( + _ context.Context, + ti api.TaskInstance, + key string, + value any, +) error { + mapIndex := -1 + if ti.MapIndex != nil && *ti.MapIndex != -1 { + mapIndex = *ti.MapIndex + } + + msg := SetXComMsg{ + Key: key, + Value: value, + DagID: ti.DagId, + TaskID: ti.TaskId, + RunID: ti.RunId, + MapIndex: mapIndex, + } + + _, err := c.comm.Communicate(msg.toMap()) + return err +} diff --git a/go-sdk/pkg/execution/comms.go b/go-sdk/pkg/execution/comms.go new file mode 100644 index 0000000000000..81b47caf1600c --- /dev/null +++ b/go-sdk/pkg/execution/comms.go @@ -0,0 +1,128 @@ +// 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. + +package execution + +import ( + "fmt" + "io" + "log/slog" + "sync" + "sync/atomic" +) + +// CoordinatorComm manages bidirectional communication with the Airflow supervisor +// over a length-prefixed msgpack socket connection. +type CoordinatorComm struct { + reader io.Reader + writer io.Writer + nextID atomic.Int32 + logger *slog.Logger + + wmu sync.Mutex // serialises writes + rmu sync.Mutex // serialises reads +} + +// NewCoordinatorComm creates a new communication channel. +func NewCoordinatorComm(reader io.Reader, writer io.Writer, logger *slog.Logger) *CoordinatorComm { + return &CoordinatorComm{ + reader: reader, + writer: writer, + logger: logger, + } +} + +// ReadMessage reads and decodes one frame from the comm socket. +// It returns the raw IncomingFrame with decoded map bodies. +func (c *CoordinatorComm) ReadMessage() (IncomingFrame, error) { + c.rmu.Lock() + defer c.rmu.Unlock() + frame, err := readFrame(c.reader) + if err != nil { + return IncomingFrame{}, fmt.Errorf("reading frame: %w", err) + } + c.logger.Debug("Received frame", "id", frame.ID) + return frame, nil +} + +// SendRequest sends a request frame (2-element: [id, body]) to the supervisor. +func (c *CoordinatorComm) SendRequest(id int, body map[string]any) error { + payload, err := encodeRequest(id, body) + if err != nil { + return fmt.Errorf("encoding request: %w", err) + } + c.logger.Debug("Sending request", "id", id) + c.wmu.Lock() + defer c.wmu.Unlock() + return writeFrame(c.writer, payload) +} + +// Communicate sends a request and waits for the corresponding response. +// This is a synchronous request-response: the caller sends a request and blocks +// until the next frame arrives. The protocol is single-threaded on the comm socket. +// +// If the response contains an error element, it is returned as an ApiError. +// Otherwise, the response body map is returned. +func (c *CoordinatorComm) Communicate(body map[string]any) (map[string]any, error) { + id := int(c.nextID.Add(1) - 1) + + if err := c.SendRequest(id, body); err != nil { + return nil, err + } + + frame, err := c.ReadMessage() + if err != nil { + return nil, err + } + + // Check for error in the response. + if frame.Err != nil { + errResp := decodeErrorResponse(frame.Err) + if errResp != nil { + return nil, &ApiError{ + Err: errResp.Error, + Detail: errResp.Detail, + } + } + } + + // Also check if the body itself is an ErrorResponse. + if frame.Body != nil { + if typ, _ := frame.Body["type"].(string); typ == "ErrorResponse" { + errResp := decodeErrorResponse(frame.Body) + return nil, &ApiError{ + Err: errResp.Error, + Detail: errResp.Detail, + } + } + } + + return frame.Body, nil +} + +// ApiError represents an error returned by the supervisor over the comm socket. +type ApiError struct { + Err string + Detail any +} + +func (e *ApiError) Error() string { + if e.Detail != nil { + return fmt.Sprintf("[%s] %v", e.Err, e.Detail) + } + return e.Err +} diff --git a/go-sdk/pkg/execution/comms_test.go b/go-sdk/pkg/execution/comms_test.go new file mode 100644 index 0000000000000..a5360b7358d22 --- /dev/null +++ b/go-sdk/pkg/execution/comms_test.go @@ -0,0 +1,166 @@ +// 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. + +package execution + +import ( + "bytes" + "io" + "log/slog" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/vmihailenco/msgpack/v5" +) + +func TestCoordinatorCommReadMessage(t *testing.T) { + body := map[string]any{ + "type": "DagFileParseRequest", + "file": "/path/to/dags.go", + } + payload, err := encodeRequest(0, body) + require.NoError(t, err) + + var buf bytes.Buffer + require.NoError(t, writeFrame(&buf, payload)) + + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(&buf, io.Discard, logger) + + frame, err := comm.ReadMessage() + require.NoError(t, err) + assert.Equal(t, 0, frame.ID) + assert.Equal(t, "DagFileParseRequest", frame.Body["type"]) +} + +func TestCoordinatorCommSendRequest(t *testing.T) { + var buf bytes.Buffer + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(bytes.NewReader(nil), &buf, logger) + + body := map[string]any{ + "type": "GetVariable", + "key": "test_key", + } + err := comm.SendRequest(5, body) + require.NoError(t, err) + + frame, err := readFrame(&buf) + require.NoError(t, err) + assert.Equal(t, 5, frame.ID) + assert.Equal(t, "GetVariable", frame.Body["type"]) + assert.Equal(t, "test_key", frame.Body["key"]) +} + +func TestCoordinatorCommCommunicate(t *testing.T) { + // Prepare a response frame for the mock supervisor to "return". + responseBody := map[string]any{ + "type": "VariableResult", + "key": "my_var", + "value": "my_value", + } + responsePayload, err := encodeRequest(0, responseBody) + require.NoError(t, err) + + var responseBuf bytes.Buffer + require.NoError(t, writeFrame(&responseBuf, responsePayload)) + + var requestBuf bytes.Buffer + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(&responseBuf, &requestBuf, logger) + + result, err := comm.Communicate(GetVariableMsg{Key: "my_var"}.toMap()) + require.NoError(t, err) + assert.Equal(t, "VariableResult", result["type"]) + assert.Equal(t, "my_value", result["value"]) + + // Verify the request was sent. + sentFrame, err := readFrame(&requestBuf) + require.NoError(t, err) + assert.Equal(t, "GetVariable", sentFrame.Body["type"]) +} + +func TestCoordinatorCommCommunicateError(t *testing.T) { + // Build a 3-element response frame with an error. + responsePayload := encodeResponseFrame(t, 0, nil, map[string]any{ + "type": "ErrorResponse", + "error": "not_found", + "detail": "Variable not found", + }) + + var responseBuf bytes.Buffer + require.NoError(t, writeFrame(&responseBuf, responsePayload)) + + var requestBuf bytes.Buffer + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(&responseBuf, &requestBuf, logger) + + _, err := comm.Communicate(GetVariableMsg{Key: "missing"}.toMap()) + require.Error(t, err) + + apiErr, ok := err.(*ApiError) + require.True(t, ok) + assert.Equal(t, "not_found", apiErr.Err) +} + +func TestCoordinatorCommCommunicateBodyError(t *testing.T) { + // Error can also come in the body element of a 2-element frame. + errorBody := map[string]any{ + "type": "ErrorResponse", + "error": "server_error", + "detail": "internal failure", + } + responsePayload, err := encodeRequest(0, errorBody) + require.NoError(t, err) + + var responseBuf bytes.Buffer + require.NoError(t, writeFrame(&responseBuf, responsePayload)) + + var requestBuf bytes.Buffer + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(&responseBuf, &requestBuf, logger) + + _, err = comm.Communicate(GetVariableMsg{Key: "test"}.toMap()) + require.Error(t, err) + + apiErr, ok := err.(*ApiError) + require.True(t, ok) + assert.Equal(t, "server_error", apiErr.Err) +} + +func TestApiErrorFormat(t *testing.T) { + err := &ApiError{Err: "not_found", Detail: "Variable 'x' not found"} + assert.Equal(t, "[not_found] Variable 'x' not found", err.Error()) + + err2 := &ApiError{Err: "server_error"} + assert.Equal(t, "server_error", err2.Error()) +} + +// encodeResponseFrame encodes a 3-element response frame for testing. +func encodeResponseFrame(t *testing.T, id int, body, errBody map[string]any) []byte { + t.Helper() + var buf bytes.Buffer + enc := msgpack.NewEncoder(&buf) + enc.UseCompactInts(true) + + require.NoError(t, enc.EncodeArrayLen(3)) + require.NoError(t, enc.EncodeInt(int64(id))) + require.NoError(t, enc.Encode(body)) + require.NoError(t, enc.Encode(errBody)) + return buf.Bytes() +} diff --git a/go-sdk/pkg/execution/dag_parser.go b/go-sdk/pkg/execution/dag_parser.go new file mode 100644 index 0000000000000..734a5ded77f72 --- /dev/null +++ b/go-sdk/pkg/execution/dag_parser.go @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package execution + +import ( + "github.com/apache/airflow/go-sdk/bundle/bundlev1" +) + +// ParseDags processes a DagFileParseRequest by serialising every dag +// registered on bundle to DagSerialization v3 and returning the result as a +// DagFileParsingResult body. bundle is the materialised registry produced by +// running BundleProvider.RegisterDags. +func ParseDags(bundle bundlev1.Bundle, req *DagFileParseRequest) map[string]any { + fileloc := req.File + bundlePath := req.BundlePath + relativeFileloc := computeRelativeFileloc(fileloc, bundlePath) + + var dags []bundlev1.DagInfo + if enum, ok := bundle.(bundlev1.EnumerableBundle); ok { + dags = enum.OrderedDags() + } + + serializedDags := make([]any, 0, len(dags)) + for _, d := range dags { + serializedDags = append(serializedDags, map[string]any{ + "data": map[string]any{ + "__version": 3, + "dag": SerializeDag(d, fileloc, relativeFileloc), + }, + }) + } + + return map[string]any{ + "type": "DagFileParsingResult", + "fileloc": fileloc, + "serialized_dags": serializedDags, + } +} diff --git a/go-sdk/pkg/execution/frames.go b/go-sdk/pkg/execution/frames.go new file mode 100644 index 0000000000000..7458532d5aa18 --- /dev/null +++ b/go-sdk/pkg/execution/frames.go @@ -0,0 +1,229 @@ +// 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. + +package execution + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + + "github.com/vmihailenco/msgpack/v5" +) + +// IncomingFrame represents a decoded frame received from the comm socket. +type IncomingFrame struct { + ID int + Body map[string]any + Err map[string]any // non-nil only for response frames (3-element arrays) +} + +// encodeRequest encodes a request frame (2-element msgpack array: [id, body]). +func encodeRequest(id int, body map[string]any) ([]byte, error) { + var buf bytes.Buffer + enc := msgpack.NewEncoder(&buf) + enc.UseCompactInts(true) + + if err := enc.EncodeArrayLen(2); err != nil { + return nil, err + } + if err := enc.EncodeInt(int64(id)); err != nil { + return nil, err + } + if err := enc.Encode(body); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +// writeFrame writes a length-prefixed msgpack payload to the writer. +// Format: [4-byte big-endian length][payload bytes] +func writeFrame(w io.Writer, payload []byte) error { + prefix := make([]byte, 4) + binary.BigEndian.PutUint32(prefix, uint32(len(payload))) + if _, err := w.Write(prefix); err != nil { + return fmt.Errorf("writing length prefix: %w", err) + } + if _, err := w.Write(payload); err != nil { + return fmt.Errorf("writing payload: %w", err) + } + return nil +} + +// readFrame reads one length-prefixed msgpack frame from the reader and decodes it. +func readFrame(r io.Reader) (IncomingFrame, error) { + // Read 4-byte big-endian length prefix. + prefix := make([]byte, 4) + if _, err := io.ReadFull(r, prefix); err != nil { + return IncomingFrame{}, fmt.Errorf("reading length prefix: %w", err) + } + payloadLen := binary.BigEndian.Uint32(prefix) + + // Read the payload. + payload := make([]byte, payloadLen) + if _, err := io.ReadFull(r, payload); err != nil { + return IncomingFrame{}, fmt.Errorf("reading payload (%d bytes): %w", payloadLen, err) + } + + return decodeFrame(payload) +} + +// decodeFrame decodes a msgpack payload into an IncomingFrame. +func decodeFrame(data []byte) (IncomingFrame, error) { + dec := msgpack.NewDecoder(bytes.NewReader(data)) + + arrLen, err := dec.DecodeArrayLen() + if err != nil { + return IncomingFrame{}, fmt.Errorf("decoding array header: %w", err) + } + if arrLen < 2 { + return IncomingFrame{}, fmt.Errorf("unexpected frame arity %d, need at least 2", arrLen) + } + + id64, err := dec.DecodeInt64() + if err != nil { + return IncomingFrame{}, fmt.Errorf("decoding frame id: %w", err) + } + + // Decode the body element. + bodyRaw, err := dec.DecodeInterface() + if err != nil { + return IncomingFrame{}, fmt.Errorf("decoding body: %w", err) + } + body, _ := toStringMap(bodyRaw) + + // For response frames (3-element), decode the error element. + var errMap map[string]any + if arrLen >= 3 { + errRaw, err := dec.DecodeInterface() + if err != nil { + return IncomingFrame{}, fmt.Errorf("decoding error element: %w", err) + } + errMap, _ = toStringMap(errRaw) + } + + return IncomingFrame{ + ID: int(id64), + Body: body, + Err: errMap, + }, nil +} + +// toStringMap converts a decoded interface{} to map[string]any. +// Returns nil, false if the input is nil or not a map. +func toStringMap(v any) (map[string]any, bool) { + if v == nil { + return nil, false + } + switch m := v.(type) { + case map[string]any: + return m, true + case map[any]any: + result := make(map[string]any, len(m)) + for k, val := range m { + result[fmt.Sprint(k)] = val + } + return result, true + default: + return nil, false + } +} + +// mapString extracts a string value from a map. +func mapString(m map[string]any, key string) (string, error) { + v, ok := m[key] + if !ok { + return "", fmt.Errorf("missing key %q", key) + } + s, ok := v.(string) + if !ok { + return "", fmt.Errorf("key %q: expected string, got %T", key, v) + } + return s, nil +} + +// mapIntOr extracts an int value from a map, returning the default if missing. +func mapIntOr(m map[string]any, key string, def int) int { + v, ok := m[key] + if !ok { + return def + } + n, err := toInt(v) + if err != nil { + return def + } + return n +} + +// mapStringOr extracts a string value from a map, returning the default if missing. +func mapStringOr(m map[string]any, key string, def string) string { + v, ok := m[key] + if !ok { + return def + } + s, ok := v.(string) + if !ok { + return def + } + return s +} + +// mapMap extracts a nested map from a map. +func mapMap(m map[string]any, key string) map[string]any { + v, ok := m[key] + if !ok || v == nil { + return nil + } + sub, ok := toStringMap(v) + if !ok { + return nil + } + return sub +} + +// toInt converts various numeric types from msgpack decoding to int. +func toInt(v any) (int, error) { + switch n := v.(type) { + case int: + return n, nil + case int8: + return int(n), nil + case int16: + return int(n), nil + case int32: + return int(n), nil + case int64: + return int(n), nil + case uint: + return int(n), nil + case uint8: + return int(n), nil + case uint16: + return int(n), nil + case uint32: + return int(n), nil + case uint64: + return int(n), nil + case float32: + return int(n), nil + case float64: + return int(n), nil + default: + return 0, fmt.Errorf("expected numeric, got %T", v) + } +} diff --git a/go-sdk/pkg/execution/frames_test.go b/go-sdk/pkg/execution/frames_test.go new file mode 100644 index 0000000000000..e0800dd8191ac --- /dev/null +++ b/go-sdk/pkg/execution/frames_test.go @@ -0,0 +1,200 @@ +// 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. + +package execution + +import ( + "bytes" + "encoding/binary" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/vmihailenco/msgpack/v5" +) + +func TestEncodeRequest(t *testing.T) { + body := map[string]any{ + "type": "GetVariable", + "key": "my_var", + } + + data, err := encodeRequest(42, body) + require.NoError(t, err) + + // Decode and verify structure. + dec := msgpack.NewDecoder(bytes.NewReader(data)) + arrLen, err := dec.DecodeArrayLen() + require.NoError(t, err) + assert.Equal(t, 2, arrLen, "request frame should be 2-element array") + + id, err := dec.DecodeInt64() + require.NoError(t, err) + assert.Equal(t, int64(42), id) + + var decodedBody map[string]any + err = dec.Decode(&decodedBody) + require.NoError(t, err) + assert.Equal(t, "GetVariable", decodedBody["type"]) + assert.Equal(t, "my_var", decodedBody["key"]) +} + +func TestWriteAndReadFrame(t *testing.T) { + body := map[string]any{ + "type": "GetConnection", + "conn_id": "my_db", + } + + payload, err := encodeRequest(7, body) + require.NoError(t, err) + + // Write to buffer with length prefix. + var buf bytes.Buffer + err = writeFrame(&buf, payload) + require.NoError(t, err) + + // Verify length prefix. + prefix := buf.Bytes()[:4] + expectedLen := uint32(len(payload)) + assert.Equal(t, expectedLen, binary.BigEndian.Uint32(prefix)) + + // Read back. + frame, err := readFrame(&buf) + require.NoError(t, err) + assert.Equal(t, 7, frame.ID) + assert.Equal(t, "GetConnection", frame.Body["type"]) + assert.Equal(t, "my_db", frame.Body["conn_id"]) + assert.Nil(t, frame.Err) +} + +func TestDecodeResponseFrame(t *testing.T) { + // Encode a 3-element response frame: [id, body, error] + var buf bytes.Buffer + enc := msgpack.NewEncoder(&buf) + enc.UseCompactInts(true) + + require.NoError(t, enc.EncodeArrayLen(3)) + require.NoError(t, enc.EncodeInt(5)) + require.NoError(t, enc.Encode(map[string]any{ + "type": "ConnectionResult", + "conn_id": "test_conn", + "host": "localhost", + })) + require.NoError(t, enc.Encode(nil)) // no error + + frame, err := decodeFrame(buf.Bytes()) + require.NoError(t, err) + assert.Equal(t, 5, frame.ID) + assert.Equal(t, "ConnectionResult", frame.Body["type"]) + assert.Equal(t, "localhost", frame.Body["host"]) + assert.Nil(t, frame.Err) +} + +func TestDecodeResponseFrameWithError(t *testing.T) { + var buf bytes.Buffer + enc := msgpack.NewEncoder(&buf) + enc.UseCompactInts(true) + + require.NoError(t, enc.EncodeArrayLen(3)) + require.NoError(t, enc.EncodeInt(3)) + require.NoError(t, enc.Encode(nil)) // nil body + require.NoError(t, enc.Encode(map[string]any{ + "type": "ErrorResponse", + "error": "not_found", + "detail": "Variable 'x' not found", + })) + + frame, err := decodeFrame(buf.Bytes()) + require.NoError(t, err) + assert.Equal(t, 3, frame.ID) + assert.Nil(t, frame.Body) + assert.NotNil(t, frame.Err) + assert.Equal(t, "not_found", frame.Err["error"]) +} + +func TestRoundTripMultipleFrames(t *testing.T) { + var buf bytes.Buffer + + // Write two frames. + bodies := []map[string]any{ + {"type": "GetVariable", "key": "v1"}, + {"type": "GetVariable", "key": "v2"}, + } + for i, body := range bodies { + payload, err := encodeRequest(i, body) + require.NoError(t, err) + require.NoError(t, writeFrame(&buf, payload)) + } + + // Read them back. + for i, expected := range bodies { + frame, err := readFrame(&buf) + require.NoError(t, err) + assert.Equal(t, i, frame.ID) + assert.Equal(t, expected["key"], frame.Body["key"]) + } +} + +func TestToStringMap(t *testing.T) { + tests := []struct { + name string + input any + want map[string]any + ok bool + }{ + {"nil", nil, nil, false}, + {"string map", map[string]any{"a": 1}, map[string]any{"a": 1}, true}, + {"any key map", map[any]any{"b": 2}, map[string]any{"b": 2}, true}, + {"not a map", "hello", nil, false}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, ok := toStringMap(tt.input) + assert.Equal(t, tt.ok, ok) + if tt.ok { + assert.Equal(t, tt.want, got) + } + }) + } +} + +func TestToInt(t *testing.T) { + tests := []struct { + input any + want int + }{ + {int8(42), 42}, + {int16(42), 42}, + {int32(42), 42}, + {int64(42), 42}, + {uint8(42), 42}, + {uint16(42), 42}, + {uint32(42), 42}, + {uint64(42), 42}, + {float32(42.0), 42}, + {float64(42.0), 42}, + {int(42), 42}, + } + for _, tt := range tests { + got, err := toInt(tt.input) + require.NoError(t, err) + assert.Equal(t, tt.want, got) + } + + _, err := toInt("not a number") + assert.Error(t, err) +} diff --git a/go-sdk/pkg/execution/integration_test.go b/go-sdk/pkg/execution/integration_test.go new file mode 100644 index 0000000000000..4bfc0e06fb181 --- /dev/null +++ b/go-sdk/pkg/execution/integration_test.go @@ -0,0 +1,358 @@ +// 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. + +package execution + +import ( + "bytes" + "errors" + "io" + "log/slog" + "net" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/apache/airflow/go-sdk/bundle/bundlev1" +) + +// --- Test task functions --- + +func failingTask() error { + return errors.New("task failed intentionally") +} + +func panicTask() error { + panic("something went wrong") +} + +func simpleTask() error { + return nil +} + +// buildBundle wires a bundlev1.Registry from a closure and returns it as a +// bundlev1.Bundle (the materialised registry). +func buildBundle(t *testing.T, register func(bundlev1.Registry)) bundlev1.Bundle { + t.Helper() + reg := bundlev1.New() + register(reg) + return reg +} + +// --- Tests --- + +func TestDagParsing(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + d := r.AddDag("test_dag") + d.AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + }) + + req := &DagFileParseRequest{ + File: "/bundles/test/main.go", + BundlePath: "/bundles/test", + } + + result := ParseDags(bundle, req) + + assert.Equal(t, "DagFileParsingResult", result["type"]) + assert.Equal(t, "/bundles/test/main.go", result["fileloc"]) + + serializedDags, ok := result["serialized_dags"].([]any) + require.True(t, ok) + require.Len(t, serializedDags, 1) + + dagEntry := serializedDags[0].(map[string]any) + data := dagEntry["data"].(map[string]any) + assert.Equal(t, 3, data["__version"]) + + dagMap := data["dag"].(map[string]any) + assert.Equal(t, "test_dag", dagMap["dag_id"]) + + tt := dagMap["timetable"].(map[string]any) + assert.Equal(t, "airflow.timetables.simple.NullTimetable", tt["__type"]) + + tasks := dagMap["tasks"].([]any) + require.Len(t, tasks, 1) + taskMap := tasks[0].(map[string]any) + assert.Equal(t, "operator", taskMap["__type"]) + taskData := taskMap["__var"].(map[string]any) + assert.Equal(t, "simpleTask", taskData["task_id"]) + assert.Equal(t, "go", taskData["language"]) +} + +func TestDagParsingMultipleDagsPreservesOrder(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + r.AddDag("dag1").AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + r.AddDag("dag2").AddTask(failingTask, bundlev1.TaskSpec{}, nil) + }) + + req := &DagFileParseRequest{File: "/bundle/main.go", BundlePath: "/bundle"} + result := ParseDags(bundle, req) + + serializedDags := result["serialized_dags"].([]any) + require.Len(t, serializedDags, 2) + + dag1Data := serializedDags[0].(map[string]any)["data"].(map[string]any)["dag"].(map[string]any) + assert.Equal(t, "dag1", dag1Data["dag_id"]) + + dag2Data := serializedDags[1].(map[string]any)["data"].(map[string]any)["dag"].(map[string]any) + assert.Equal(t, "dag2", dag2Data["dag_id"]) +} + +func TestTaskRunnerSuccess(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + r.AddDag("test_dag").AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + }) + + details := &StartupDetails{ + TI: TaskInstanceInfo{ + ID: "550e8400-e29b-41d4-a716-446655440000", + DagID: "test_dag", + TaskID: "simpleTask", + RunID: "run1", + MapIndex: -1, + }, + BundleInfo: BundleInfoMsg{Name: "test", Version: "1.0"}, + } + + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(bytes.NewReader(nil), io.Discard, logger) + + result := RunTask(bundle, details, comm, logger) + assert.Equal(t, "SucceedTask", result["type"]) +} + +func TestTaskRunnerFailure(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + r.AddDag("test_dag").AddTask(failingTask, bundlev1.TaskSpec{}, nil) + }) + + details := &StartupDetails{ + TI: TaskInstanceInfo{ + ID: "550e8400-e29b-41d4-a716-446655440000", + DagID: "test_dag", + TaskID: "failingTask", + RunID: "run1", + MapIndex: -1, + }, + BundleInfo: BundleInfoMsg{Name: "test", Version: "1.0"}, + } + + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(bytes.NewReader(nil), io.Discard, logger) + + result := RunTask(bundle, details, comm, logger) + assert.Equal(t, "TaskState", result["type"]) + assert.Equal(t, "failed", result["state"]) +} + +func TestTaskRunnerTaskNotFound(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + r.AddDag("test_dag").AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + }) + + details := &StartupDetails{ + TI: TaskInstanceInfo{ + ID: "550e8400-e29b-41d4-a716-446655440000", + DagID: "test_dag", + TaskID: "nonexistent", + RunID: "run1", + }, + BundleInfo: BundleInfoMsg{Name: "test", Version: "1.0"}, + } + + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(bytes.NewReader(nil), io.Discard, logger) + + result := RunTask(bundle, details, comm, logger) + assert.Equal(t, "TaskState", result["type"]) + assert.Equal(t, "removed", result["state"]) +} + +func TestTaskRunnerPanic(t *testing.T) { + bundle := buildBundle(t, func(r bundlev1.Registry) { + r.AddDag("test_dag").AddTask(panicTask, bundlev1.TaskSpec{}, nil) + }) + + details := &StartupDetails{ + TI: TaskInstanceInfo{ + ID: "550e8400-e29b-41d4-a716-446655440000", + DagID: "test_dag", + TaskID: "panicTask", + RunID: "run1", + MapIndex: -1, + }, + BundleInfo: BundleInfoMsg{Name: "test", Version: "1.0"}, + } + + logger := slog.New(slog.NewTextHandler(io.Discard, nil)) + comm := NewCoordinatorComm(bytes.NewReader(nil), io.Discard, logger) + + result := RunTask(bundle, details, comm, logger) + assert.Equal(t, "TaskState", result["type"]) + assert.Equal(t, "failed", result["state"]) +} + +// --- End-to-end Serve test against a fake supervisor --- + +// fakeProvider implements bundlev1.BundleProvider; it lets a test inject the +// registration closure and a synthetic version. +type fakeProvider struct { + register func(bundlev1.Registry) error +} + +func (f *fakeProvider) GetBundleVersion() bundlev1.BundleInfo { + v := "1.0" + return bundlev1.BundleInfo{Name: "fake", Version: &v} +} + +func (f *fakeProvider) RegisterDags(reg bundlev1.Registry) error { + if f.register == nil { + return nil + } + return f.register(reg) +} + +func startSupervisor( + t *testing.T, +) (commAddr, logsAddr string, commCh, logsCh chan net.Conn, cleanup func()) { + t.Helper() + commLn, err := net.Listen("tcp", "127.0.0.1:0") + require.NoError(t, err) + logsLn, err := net.Listen("tcp", "127.0.0.1:0") + require.NoError(t, err) + + commCh = make(chan net.Conn, 1) + logsCh = make(chan net.Conn, 1) + go func() { + c, err := commLn.Accept() + if err == nil { + commCh <- c + } + close(commCh) + }() + go func() { + c, err := logsLn.Accept() + if err == nil { + logsCh <- c + } + close(logsCh) + }() + cleanup = func() { + commLn.Close() + logsLn.Close() + } + return commLn.Addr().String(), logsLn.Addr().String(), commCh, logsCh, cleanup +} + +func TestServeDagFileParseEndToEnd(t *testing.T) { + commAddr, logsAddr, commCh, logsCh, cleanup := startSupervisor(t) + defer cleanup() + + provider := &fakeProvider{ + register: func(r bundlev1.Registry) error { + d := r.AddDag("simple_dag") + d.AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + return nil + }, + } + + done := make(chan error, 1) + go func() { done <- Serve(provider, commAddr, logsAddr) }() + + commConn := <-commCh + require.NotNil(t, commConn) + defer commConn.Close() + logsConn := <-logsCh + require.NotNil(t, logsConn) + defer logsConn.Close() + + // Send DagFileParseRequest as a request frame. + payload, err := encodeRequest(0, map[string]any{ + "type": "DagFileParseRequest", + "file": "/bundle/main.go", + "bundle_path": "/bundle", + }) + require.NoError(t, err) + require.NoError(t, writeFrame(commConn, payload)) + + frame, err := readFrame(commConn) + require.NoError(t, err) + assert.Equal(t, 0, frame.ID) + require.Nil(t, frame.Err) + assert.Equal(t, "DagFileParsingResult", frame.Body["type"]) + + dags := frame.Body["serialized_dags"].([]any) + require.Len(t, dags, 1) + dag := dags[0].(map[string]any)["data"].(map[string]any)["dag"].(map[string]any) + assert.Equal(t, "simple_dag", dag["dag_id"]) + + select { + case err := <-done: + require.NoError(t, err) + case <-time.After(2 * time.Second): + t.Fatal("Serve did not return after parse result") + } +} + +func TestServeStartupDetailsEndToEnd(t *testing.T) { + commAddr, logsAddr, commCh, logsCh, cleanup := startSupervisor(t) + defer cleanup() + + provider := &fakeProvider{ + register: func(r bundlev1.Registry) error { + r.AddDag("dag1").AddTask(simpleTask, bundlev1.TaskSpec{}, nil) + return nil + }, + } + + done := make(chan error, 1) + go func() { done <- Serve(provider, commAddr, logsAddr) }() + + commConn := <-commCh + defer commConn.Close() + logsConn := <-logsCh + defer logsConn.Close() + + payload, err := encodeRequest(0, map[string]any{ + "type": "StartupDetails", + "ti": map[string]any{ + "id": "550e8400-e29b-41d4-a716-446655440000", + "dag_id": "dag1", + "task_id": "simpleTask", + "run_id": "run1", + "try_number": 1, + }, + "bundle_info": map[string]any{"name": "fake", "version": "1.0"}, + }) + require.NoError(t, err) + require.NoError(t, writeFrame(commConn, payload)) + + frame, err := readFrame(commConn) + require.NoError(t, err) + require.Nil(t, frame.Err) + assert.Equal(t, "SucceedTask", frame.Body["type"]) + + select { + case err := <-done: + require.NoError(t, err) + case <-time.After(2 * time.Second): + t.Fatal("Serve did not return after task completion") + } +} diff --git a/go-sdk/pkg/execution/logger.go b/go-sdk/pkg/execution/logger.go new file mode 100644 index 0000000000000..172cbba161f55 --- /dev/null +++ b/go-sdk/pkg/execution/logger.go @@ -0,0 +1,158 @@ +// 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. + +package execution + +import ( + "context" + "encoding/json" + "io" + "log/slog" + "strings" + "sync" + "time" +) + +// SocketLogHandler is an slog.Handler that streams structured JSON log lines +// to the logs TCP socket. Each log entry is a single JSON object followed by +// a newline, matching the Airflow log streaming format. +// +// Key mapping: +// - "event" for the log message (not "msg") +// - "level" in lowercase (not "INFO"/"ERROR") +// - "timestamp" in RFC3339Nano format (not "time") +// - Additional attributes are included as top-level fields +type SocketLogHandler struct { + shared *socketLogHandlerShared + level slog.Level + attrs []slog.Attr + groups []string +} + +// socketLogHandlerShared holds the writer and buffer that must remain shared +// across WithAttrs / WithGroup clones; otherwise the sync.Mutex would be +// copied (which the runtime detector flags as a bug). +type socketLogHandlerShared struct { + mu sync.Mutex + writer io.Writer + buf [][]byte + connected bool +} + +var _ slog.Handler = (*SocketLogHandler)(nil) + +// NewSocketLogHandler creates a new handler. If writer is nil, messages are +// buffered until Connect() is called. +func NewSocketLogHandler(writer io.Writer, level slog.Level) *SocketLogHandler { + shared := &socketLogHandlerShared{} + if writer != nil { + shared.writer = writer + shared.connected = true + } + return &SocketLogHandler{ + shared: shared, + level: level, + } +} + +// Connect sets the writer and flushes any buffered log messages. +func (h *SocketLogHandler) Connect(w io.Writer) { + h.shared.mu.Lock() + defer h.shared.mu.Unlock() + + h.shared.writer = w + h.shared.connected = true + + for _, line := range h.shared.buf { + _, _ = w.Write(line) + } + h.shared.buf = nil +} + +func (h *SocketLogHandler) Enabled(_ context.Context, level slog.Level) bool { + return level >= h.level +} + +func (h *SocketLogHandler) Handle(_ context.Context, r slog.Record) error { + entry := make(map[string]any) + + // Set standard fields. + entry["event"] = r.Message + entry["level"] = strings.ToLower(r.Level.String()) + if !r.Time.IsZero() { + entry["timestamp"] = r.Time.Format(time.RFC3339Nano) + } + + // Apply pre-configured attrs. + for _, a := range h.attrs { + key := h.prefixedKey(a.Key) + entry[key] = a.Value.Any() + } + + // Apply record attrs. + r.Attrs(func(a slog.Attr) bool { + key := h.prefixedKey(a.Key) + entry[key] = a.Value.Any() + return true + }) + + line, err := json.Marshal(entry) + if err != nil { + return err + } + line = append(line, '\n') + + h.shared.mu.Lock() + defer h.shared.mu.Unlock() + + if !h.shared.connected { + h.shared.buf = append(h.shared.buf, line) + return nil + } + + _, err = h.shared.writer.Write(line) + return err +} + +func (h *SocketLogHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + return &SocketLogHandler{ + shared: h.shared, + level: h.level, + attrs: append(append([]slog.Attr{}, h.attrs...), attrs...), + groups: h.groups, + } +} + +func (h *SocketLogHandler) WithGroup(name string) slog.Handler { + if name == "" { + return h + } + return &SocketLogHandler{ + shared: h.shared, + level: h.level, + attrs: h.attrs, + groups: append(append([]string{}, h.groups...), name), + } +} + +// prefixedKey prepends any active group names to the attribute key. +func (h *SocketLogHandler) prefixedKey(key string) string { + if len(h.groups) == 0 { + return key + } + return strings.Join(h.groups, ".") + "." + key +} diff --git a/go-sdk/pkg/execution/logger_test.go b/go-sdk/pkg/execution/logger_test.go new file mode 100644 index 0000000000000..cf925ad13744a --- /dev/null +++ b/go-sdk/pkg/execution/logger_test.go @@ -0,0 +1,134 @@ +// 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. + +package execution + +import ( + "bytes" + "encoding/json" + "log/slog" + "strings" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestSocketLogHandlerBasicOutput(t *testing.T) { + var buf bytes.Buffer + handler := NewSocketLogHandler(&buf, slog.LevelDebug) + logger := slog.New(handler) + + logger.Info("test message", "key1", "val1") + + // Parse the output. + output := buf.String() + assert.True(t, strings.HasSuffix(output, "\n"), "output should end with newline") + + var entry map[string]any + require.NoError(t, json.Unmarshal([]byte(strings.TrimSpace(output)), &entry)) + + assert.Equal(t, "test message", entry["event"]) + assert.Equal(t, "info", entry["level"]) + assert.Equal(t, "val1", entry["key1"]) + assert.Contains(t, entry, "timestamp") +} + +func TestSocketLogHandlerLevelFiltering(t *testing.T) { + var buf bytes.Buffer + handler := NewSocketLogHandler(&buf, slog.LevelWarn) + logger := slog.New(handler) + + logger.Debug("should be filtered") + logger.Info("also filtered") + logger.Warn("should appear") + + lines := strings.Split(strings.TrimSpace(buf.String()), "\n") + assert.Len(t, lines, 1) + + var entry map[string]any + require.NoError(t, json.Unmarshal([]byte(lines[0]), &entry)) + assert.Equal(t, "should appear", entry["event"]) + assert.Equal(t, "warn", entry["level"]) +} + +func TestSocketLogHandlerBuffering(t *testing.T) { + // Create handler without a writer — messages should be buffered. + handler := NewSocketLogHandler(nil, slog.LevelDebug) + logger := slog.New(handler) + + logger.Info("buffered message 1") + logger.Info("buffered message 2") + + // Connect the writer — buffered messages should flush. + var buf bytes.Buffer + handler.Connect(&buf) + + output := buf.String() + lines := strings.Split(strings.TrimSpace(output), "\n") + assert.Len(t, lines, 2) + + // New messages should write directly. + logger.Info("direct message") + lines = strings.Split(strings.TrimSpace(buf.String()), "\n") + assert.Len(t, lines, 3) +} + +func TestSocketLogHandlerWithAttrs(t *testing.T) { + var buf bytes.Buffer + handler := NewSocketLogHandler(&buf, slog.LevelDebug) + logger := slog.New(handler).With("component", "test") + + logger.Info("with attrs") + + var entry map[string]any + require.NoError(t, json.Unmarshal([]byte(strings.TrimSpace(buf.String())), &entry)) + assert.Equal(t, "test", entry["component"]) +} + +func TestSocketLogHandlerWithGroup(t *testing.T) { + var buf bytes.Buffer + handler := NewSocketLogHandler(&buf, slog.LevelDebug) + logger := slog.New(handler).WithGroup("grp") + + logger.Info("grouped", "key", "val") + + var entry map[string]any + require.NoError(t, json.Unmarshal([]byte(strings.TrimSpace(buf.String())), &entry)) + assert.Equal(t, "val", entry["grp.key"]) +} + +func TestSocketLogHandlerKeyMapping(t *testing.T) { + var buf bytes.Buffer + handler := NewSocketLogHandler(&buf, slog.LevelDebug) + logger := slog.New(handler) + + logger.Error("an error occurred") + + var entry map[string]any + require.NoError(t, json.Unmarshal([]byte(strings.TrimSpace(buf.String())), &entry)) + + // Check key mapping: "event" not "msg", "level" lowercase, "timestamp" not "time" + assert.Equal(t, "an error occurred", entry["event"]) + assert.Equal(t, "error", entry["level"]) + _, hasTimestamp := entry["timestamp"] + assert.True(t, hasTimestamp) + _, hasMsg := entry["msg"] + assert.False(t, hasMsg) + _, hasTime := entry["time"] + assert.False(t, hasTime) +} diff --git a/go-sdk/pkg/execution/messages.go b/go-sdk/pkg/execution/messages.go new file mode 100644 index 0000000000000..ae5f6a6c7045c --- /dev/null +++ b/go-sdk/pkg/execution/messages.go @@ -0,0 +1,394 @@ +// 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. + +package execution + +import ( + "fmt" + "time" +) + +// --- Inbound messages (Supervisor -> Runtime) --- + +// DagFileParseRequest is sent by the supervisor to request DAG parsing. +type DagFileParseRequest struct { + File string + BundlePath string +} + +func decodeDagFileParseRequest(m map[string]any) (*DagFileParseRequest, error) { + file, err := mapString(m, "file") + if err != nil { + return nil, err + } + bundlePath := mapStringOr(m, "bundle_path", "") + return &DagFileParseRequest{File: file, BundlePath: bundlePath}, nil +} + +// TaskInstanceInfo holds task instance details from StartupDetails. +type TaskInstanceInfo struct { + ID string + TaskID string + DagID string + RunID string + TryNumber int + DagVersionID string + MapIndex int + ContextCarrier map[string]any +} + +func decodeTaskInstanceInfo(m map[string]any) (TaskInstanceInfo, error) { + if m == nil { + return TaskInstanceInfo{}, fmt.Errorf("nil task instance map") + } + id, err := mapString(m, "id") + if err != nil { + return TaskInstanceInfo{}, fmt.Errorf("ti.id: %w", err) + } + taskID, err := mapString(m, "task_id") + if err != nil { + return TaskInstanceInfo{}, fmt.Errorf("ti.task_id: %w", err) + } + dagID, err := mapString(m, "dag_id") + if err != nil { + return TaskInstanceInfo{}, fmt.Errorf("ti.dag_id: %w", err) + } + runID, err := mapString(m, "run_id") + if err != nil { + return TaskInstanceInfo{}, fmt.Errorf("ti.run_id: %w", err) + } + tryNumber := mapIntOr(m, "try_number", 1) + dagVersionID := mapStringOr(m, "dag_version_id", "") + mapIndex := mapIntOr(m, "map_index", -1) + contextCarrier := mapMap(m, "context_carrier") + + return TaskInstanceInfo{ + ID: id, + TaskID: taskID, + DagID: dagID, + RunID: runID, + TryNumber: tryNumber, + DagVersionID: dagVersionID, + MapIndex: mapIndex, + ContextCarrier: contextCarrier, + }, nil +} + +// BundleInfoMsg holds bundle identification from StartupDetails. +type BundleInfoMsg struct { + Name string + Version string +} + +func decodeBundleInfo(m map[string]any) BundleInfoMsg { + if m == nil { + return BundleInfoMsg{} + } + return BundleInfoMsg{ + Name: mapStringOr(m, "name", ""), + Version: mapStringOr(m, "version", ""), + } +} + +// TIRunContext holds the runtime context for a task instance. +type TIRunContext struct { + LogicalDate *time.Time + DataIntervalStart *time.Time + DataIntervalEnd *time.Time +} + +func decodeTIRunContext(m map[string]any) TIRunContext { + if m == nil { + return TIRunContext{} + } + ctx := TIRunContext{} + if t, err := asTime(m["logical_date"]); err == nil { + ctx.LogicalDate = &t + } + if t, err := asTime(m["data_interval_start"]); err == nil { + ctx.DataIntervalStart = &t + } + if t, err := asTime(m["data_interval_end"]); err == nil { + ctx.DataIntervalEnd = &t + } + return ctx +} + +// StartupDetails is sent by the supervisor to initiate task execution. +type StartupDetails struct { + TI TaskInstanceInfo + DagRelPath string + BundleInfo BundleInfoMsg + StartDate time.Time + TIContext TIRunContext + SentryIntegration string +} + +func decodeStartupDetails(m map[string]any) (*StartupDetails, error) { + tiMap := mapMap(m, "ti") + ti, err := decodeTaskInstanceInfo(tiMap) + if err != nil { + return nil, fmt.Errorf("decoding ti: %w", err) + } + + dagRelPath := mapStringOr(m, "dag_rel_path", "") + bundleInfo := decodeBundleInfo(mapMap(m, "bundle_info")) + + var startDate time.Time + if t, err := asTime(m["start_date"]); err == nil { + startDate = t + } + + tiContext := decodeTIRunContext(mapMap(m, "ti_context")) + sentryIntegration := mapStringOr(m, "sentry_integration", "") + + return &StartupDetails{ + TI: ti, + DagRelPath: dagRelPath, + BundleInfo: bundleInfo, + StartDate: startDate, + TIContext: tiContext, + SentryIntegration: sentryIntegration, + }, nil +} + +// --- Response types (for runtime-initiated requests) --- + +// ConnectionResult is the response to GetConnection. +type ConnectionResult struct { + ConnID string + ConnType string + Host string + Schema string + Login string + Password string + Port int + Extra string +} + +func decodeConnectionResult(m map[string]any) (*ConnectionResult, error) { + return &ConnectionResult{ + ConnID: mapStringOr(m, "conn_id", ""), + ConnType: mapStringOr(m, "conn_type", ""), + Host: mapStringOr(m, "host", ""), + Schema: mapStringOr(m, "schema", ""), + Login: mapStringOr(m, "login", ""), + Password: mapStringOr(m, "password", ""), + Port: mapIntOr(m, "port", 0), + Extra: mapStringOr(m, "extra", ""), + }, nil +} + +// VariableResult is the response to GetVariable. +type VariableResult struct { + Key string + Value any +} + +func decodeVariableResult(m map[string]any) (*VariableResult, error) { + return &VariableResult{ + Key: mapStringOr(m, "key", ""), + Value: m["value"], + }, nil +} + +// XComResult is the response to GetXCom. +type XComResult struct { + Key string + Value any +} + +func decodeXComResult(m map[string]any) (*XComResult, error) { + return &XComResult{ + Key: mapStringOr(m, "key", ""), + Value: m["value"], + }, nil +} + +// ErrorResponse represents an error returned by the supervisor. +type ErrorResponse struct { + Error string + Detail any +} + +func decodeErrorResponse(m map[string]any) *ErrorResponse { + if m == nil { + return nil + } + return &ErrorResponse{ + Error: mapStringOr(m, "error", ""), + Detail: m["detail"], + } +} + +// --- Outbound messages (Runtime -> Supervisor) --- + +// GetConnectionMsg is sent to request a connection from the supervisor. +type GetConnectionMsg struct { + ConnID string +} + +func (m GetConnectionMsg) toMap() map[string]any { + return map[string]any{ + "type": "GetConnection", + "conn_id": m.ConnID, + } +} + +// GetVariableMsg is sent to request a variable from the supervisor. +type GetVariableMsg struct { + Key string +} + +func (m GetVariableMsg) toMap() map[string]any { + return map[string]any{ + "type": "GetVariable", + "key": m.Key, + } +} + +// GetXComMsg is sent to request an XCom value from the supervisor. +type GetXComMsg struct { + Key string + DagID string + TaskID string + RunID string + MapIndex *int + IncludePriorDates bool +} + +func (m GetXComMsg) toMap() map[string]any { + result := map[string]any{ + "type": "GetXCom", + "key": m.Key, + "dag_id": m.DagID, + "task_id": m.TaskID, + "run_id": m.RunID, + "include_prior_dates": m.IncludePriorDates, + } + if m.MapIndex != nil { + result["map_index"] = *m.MapIndex + } + return result +} + +// SetXComMsg is sent to set an XCom value. +type SetXComMsg struct { + Key string + Value any + DagID string + TaskID string + RunID string + MapIndex int + MappedLength *int +} + +func (m SetXComMsg) toMap() map[string]any { + result := map[string]any{ + "type": "SetXCom", + "key": m.Key, + "value": m.Value, + "dag_id": m.DagID, + "task_id": m.TaskID, + "run_id": m.RunID, + "map_index": m.MapIndex, + } + if m.MappedLength != nil { + result["mapped_length"] = *m.MappedLength + } + return result +} + +// SucceedTaskMsg is sent as a terminal message when a task succeeds. +type SucceedTaskMsg struct { + EndDate time.Time + TaskOutlets []any + OutletEvents []any +} + +func (m SucceedTaskMsg) toMap() map[string]any { + taskOutlets := m.TaskOutlets + if taskOutlets == nil { + taskOutlets = []any{} + } + outletEvents := m.OutletEvents + if outletEvents == nil { + outletEvents = []any{} + } + return map[string]any{ + "type": "SucceedTask", + "end_date": m.EndDate.UTC().Format(time.RFC3339), + "task_outlets": taskOutlets, + "outlet_events": outletEvents, + } +} + +// TaskStateMsg is sent as a terminal message for failed/removed/skipped tasks. +type TaskStateMsg struct { + State string // "failed", "removed", "skipped" + EndDate time.Time +} + +func (m TaskStateMsg) toMap() map[string]any { + return map[string]any{ + "type": "TaskState", + "state": m.State, + "end_date": m.EndDate.UTC().Format(time.RFC3339), + } +} + +// --- Message dispatch --- + +// decodeIncomingBody dispatches decoding of a body map based on its "type" field. +func decodeIncomingBody(m map[string]any) (any, error) { + if m == nil { + return nil, nil + } + typ, _ := m["type"].(string) + switch typ { + case "DagFileParseRequest": + return decodeDagFileParseRequest(m) + case "StartupDetails": + return decodeStartupDetails(m) + case "ConnectionResult": + return decodeConnectionResult(m) + case "VariableResult": + return decodeVariableResult(m) + case "XComResult": + return decodeXComResult(m) + case "ErrorResponse": + return decodeErrorResponse(m), nil + default: + return nil, fmt.Errorf("unknown message type %q", typ) + } +} + +// asTime parses a time value that may be a time.Time (from msgpack timestamp ext) +// or a string (ISO 8601 format). +func asTime(v any) (time.Time, error) { + if v == nil { + return time.Time{}, fmt.Errorf("nil time value") + } + switch t := v.(type) { + case time.Time: + return t, nil + case string: + return time.Parse(time.RFC3339Nano, t) + default: + return time.Time{}, fmt.Errorf("expected time, got %T", v) + } +} diff --git a/go-sdk/pkg/execution/messages_test.go b/go-sdk/pkg/execution/messages_test.go new file mode 100644 index 0000000000000..78270dc2f4fd1 --- /dev/null +++ b/go-sdk/pkg/execution/messages_test.go @@ -0,0 +1,258 @@ +// 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. + +package execution + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestDecodeDagFileParseRequest(t *testing.T) { + m := map[string]any{ + "type": "DagFileParseRequest", + "file": "/path/to/dags.go", + "bundle_path": "/bundles/my_bundle", + } + + req, err := decodeDagFileParseRequest(m) + require.NoError(t, err) + assert.Equal(t, "/path/to/dags.go", req.File) + assert.Equal(t, "/bundles/my_bundle", req.BundlePath) +} + +func TestDecodeStartupDetails(t *testing.T) { + m := map[string]any{ + "type": "StartupDetails", + "ti": map[string]any{ + "id": "550e8400-e29b-41d4-a716-446655440000", + "task_id": "extract", + "dag_id": "tutorial_dag", + "run_id": "manual__2024-01-15", + "try_number": int64(1), + "dag_version_id": "abc-123", + "map_index": int64(-1), + }, + "dag_rel_path": "dags/tutorial.go", + "bundle_info": map[string]any{ + "name": "example_dags", + "version": "1.0.0", + }, + "start_date": "2024-01-15T10:30:00Z", + "sentry_integration": "", + "ti_context": map[string]any{ + "logical_date": "2024-01-15T00:00:00Z", + "data_interval_start": "2024-01-14T00:00:00Z", + "data_interval_end": "2024-01-15T00:00:00Z", + }, + } + + details, err := decodeStartupDetails(m) + require.NoError(t, err) + + assert.Equal(t, "550e8400-e29b-41d4-a716-446655440000", details.TI.ID) + assert.Equal(t, "extract", details.TI.TaskID) + assert.Equal(t, "tutorial_dag", details.TI.DagID) + assert.Equal(t, "manual__2024-01-15", details.TI.RunID) + assert.Equal(t, 1, details.TI.TryNumber) + assert.Equal(t, -1, details.TI.MapIndex) + assert.Equal(t, "dags/tutorial.go", details.DagRelPath) + assert.Equal(t, "example_dags", details.BundleInfo.Name) + assert.Equal(t, "1.0.0", details.BundleInfo.Version) + assert.NotNil(t, details.TIContext.LogicalDate) +} + +func TestDecodeConnectionResult(t *testing.T) { + m := map[string]any{ + "type": "ConnectionResult", + "conn_id": "my_db", + "conn_type": "postgres", + "host": "db.example.com", + "schema": "mydb", + "login": "user", + "password": "secret", + "port": int64(5432), + "extra": `{"sslmode":"require"}`, + } + + result, err := decodeConnectionResult(m) + require.NoError(t, err) + assert.Equal(t, "my_db", result.ConnID) + assert.Equal(t, "postgres", result.ConnType) + assert.Equal(t, "db.example.com", result.Host) + assert.Equal(t, "mydb", result.Schema) + assert.Equal(t, "user", result.Login) + assert.Equal(t, "secret", result.Password) + assert.Equal(t, 5432, result.Port) +} + +func TestDecodeVariableResult(t *testing.T) { + m := map[string]any{ + "type": "VariableResult", + "key": "my_var", + "value": "hello", + } + + result, err := decodeVariableResult(m) + require.NoError(t, err) + assert.Equal(t, "my_var", result.Key) + assert.Equal(t, "hello", result.Value) +} + +func TestDecodeXComResult(t *testing.T) { + m := map[string]any{ + "type": "XComResult", + "key": "return_value", + "value": map[string]any{"data": "processed"}, + } + + result, err := decodeXComResult(m) + require.NoError(t, err) + assert.Equal(t, "return_value", result.Key) + valMap, ok := result.Value.(map[string]any) + require.True(t, ok) + assert.Equal(t, "processed", valMap["data"]) +} + +func TestDecodeErrorResponseNil(t *testing.T) { + assert.Nil(t, decodeErrorResponse(nil)) +} + +func TestGetConnectionMsgToMap(t *testing.T) { + msg := GetConnectionMsg{ConnID: "my_db"} + m := msg.toMap() + assert.Equal(t, "GetConnection", m["type"]) + assert.Equal(t, "my_db", m["conn_id"]) +} + +func TestGetVariableMsgToMap(t *testing.T) { + msg := GetVariableMsg{Key: "my_var"} + m := msg.toMap() + assert.Equal(t, "GetVariable", m["type"]) + assert.Equal(t, "my_var", m["key"]) +} + +func TestGetXComMsgToMapWithMapIndex(t *testing.T) { + mapIdx := 3 + msg := GetXComMsg{ + Key: "result", + DagID: "dag1", + TaskID: "task1", + RunID: "run1", + MapIndex: &mapIdx, + IncludePriorDates: true, + } + m := msg.toMap() + assert.Equal(t, "GetXCom", m["type"]) + assert.Equal(t, 3, m["map_index"]) + assert.Equal(t, true, m["include_prior_dates"]) +} + +func TestGetXComMsgToMapNilMapIndex(t *testing.T) { + msg := GetXComMsg{Key: "result", DagID: "d", TaskID: "t", RunID: "r"} + m := msg.toMap() + _, hasMapIndex := m["map_index"] + assert.False(t, hasMapIndex) +} + +func TestSetXComMsgToMap(t *testing.T) { + msg := SetXComMsg{ + Key: "output", Value: 42, + DagID: "dag1", TaskID: "task1", RunID: "run1", MapIndex: -1, + } + m := msg.toMap() + assert.Equal(t, "SetXCom", m["type"]) + assert.Equal(t, 42, m["value"]) + _, hasMappedLength := m["mapped_length"] + assert.False(t, hasMappedLength) +} + +func TestSucceedTaskMsgToMap(t *testing.T) { + endDate := time.Date(2024, 1, 15, 10, 30, 0, 0, time.UTC) + msg := SucceedTaskMsg{EndDate: endDate} + m := msg.toMap() + assert.Equal(t, "SucceedTask", m["type"]) + assert.Equal(t, "2024-01-15T10:30:00Z", m["end_date"]) + assert.Equal(t, []any{}, m["task_outlets"]) + assert.Equal(t, []any{}, m["outlet_events"]) +} + +func TestTaskStateMsgToMap(t *testing.T) { + endDate := time.Date(2024, 1, 15, 10, 30, 0, 0, time.UTC) + msg := TaskStateMsg{State: "failed", EndDate: endDate} + m := msg.toMap() + assert.Equal(t, "TaskState", m["type"]) + assert.Equal(t, "failed", m["state"]) +} + +func TestDecodeIncomingBodyDispatch(t *testing.T) { + t.Run("DagFileParseRequest", func(t *testing.T) { + body := map[string]any{"type": "DagFileParseRequest", "file": "x", "bundle_path": "y"} + result, err := decodeIncomingBody(body) + require.NoError(t, err) + _, ok := result.(*DagFileParseRequest) + assert.True(t, ok) + }) + + t.Run("ConnectionResult", func(t *testing.T) { + body := map[string]any{"type": "ConnectionResult", "conn_id": "x"} + result, err := decodeIncomingBody(body) + require.NoError(t, err) + _, ok := result.(*ConnectionResult) + assert.True(t, ok) + }) + + t.Run("nil", func(t *testing.T) { + result, err := decodeIncomingBody(nil) + require.NoError(t, err) + assert.Nil(t, result) + }) + + t.Run("unknown type", func(t *testing.T) { + _, err := decodeIncomingBody(map[string]any{"type": "UnknownMsg"}) + assert.Error(t, err) + }) +} + +func TestAsTime(t *testing.T) { + t.Run("from string", func(t *testing.T) { + ts, err := asTime("2024-01-15T10:30:00Z") + require.NoError(t, err) + assert.Equal(t, 2024, ts.Year()) + assert.Equal(t, time.January, ts.Month()) + }) + + t.Run("from time.Time", func(t *testing.T) { + now := time.Now() + ts, err := asTime(now) + require.NoError(t, err) + assert.Equal(t, now, ts) + }) + + t.Run("nil", func(t *testing.T) { + _, err := asTime(nil) + assert.Error(t, err) + }) + + t.Run("wrong type", func(t *testing.T) { + _, err := asTime(42) + assert.Error(t, err) + }) +} diff --git a/go-sdk/pkg/execution/serde.go b/go-sdk/pkg/execution/serde.go new file mode 100644 index 0000000000000..0d9618c3990eb --- /dev/null +++ b/go-sdk/pkg/execution/serde.go @@ -0,0 +1,426 @@ +// 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. + +package execution + +import ( + "fmt" + "path/filepath" + "reflect" + "sort" + "time" + + "github.com/apache/airflow/go-sdk/bundle/bundlev1" +) + +// serializeValue recursively serializes a value with Airflow's type/var encoding. +// This matches Python's BaseSerialization.serialize() output: +// - primitives (string, bool, int, float) pass through unchanged +// - time.Time -> {"__type": "datetime", "__var": epoch_seconds_float} +// - time.Duration -> {"__type": "timedelta", "__var": total_seconds_float} +// - map[string]any -> {"__type": "dict", "__var": {k: serialize(v), ...}} +// - []any -> direct array with each element serialized +func serializeValue(value any) any { + if value == nil { + return nil + } + switch v := value.(type) { + case string, bool: + return v + case int: + return v + case int8: + return int(v) + case int16: + return int(v) + case int32: + return int(v) + case int64: + return v + case float32: + return float64(v) + case float64: + return v + case time.Time: + epochSec := float64(v.Unix()) + float64(v.Nanosecond())/1e9 + return map[string]any{ + "__type": "datetime", + "__var": epochSec, + } + case time.Duration: + return map[string]any{ + "__type": "timedelta", + "__var": v.Seconds(), + } + case map[string]any: + serialized := make(map[string]any, len(v)) + for k, val := range v { + serialized[k] = serializeValue(val) + } + return map[string]any{ + "__type": "dict", + "__var": serialized, + } + case []string: + result := make([]any, len(v)) + for i, item := range v { + result[i] = serializeValue(item) + } + return result + case []any: + result := make([]any, len(v)) + for i, item := range v { + result[i] = serializeValue(item) + } + return result + default: + // Use reflection to handle typed maps and slices that don't match + // the concrete types above (e.g., map[string]map[string][]string). + rv := reflect.ValueOf(v) + switch rv.Kind() { + case reflect.Map: + serialized := make(map[string]any, rv.Len()) + for _, key := range rv.MapKeys() { + serialized[fmt.Sprint(key.Interface())] = serializeValue(rv.MapIndex(key).Interface()) + } + return map[string]any{ + "__type": "dict", + "__var": serialized, + } + case reflect.Slice, reflect.Array: + result := make([]any, rv.Len()) + for i := range result { + result[i] = serializeValue(rv.Index(i).Interface()) + } + return result + default: + return v + } + } +} + +// unwrapTypeEncoding extracts the "__var" part from a type-encoded value. +// In Python's serialize_to_json, non-decorated fields are serialized then unwrapped. +func unwrapTypeEncoding(value any) any { + m, ok := value.(map[string]any) + if !ok { + return value + } + if _, hasType := m["__type"]; !hasType { + return value + } + if v, hasVar := m["__var"]; hasVar { + return v + } + return value +} + +// serializeTimetable converts a schedule string to the Airflow timetable format. +func serializeTimetable(schedule *string) map[string]any { + if schedule == nil { + return map[string]any{ + "__type": "airflow.timetables.simple.NullTimetable", + "__var": map[string]any{}, + } + } + switch *schedule { + case "@once": + return map[string]any{ + "__type": "airflow.timetables.simple.OnceTimetable", + "__var": map[string]any{}, + } + case "@continuous": + return map[string]any{ + "__type": "airflow.timetables.simple.ContinuousTimetable", + "__var": map[string]any{}, + } + default: + return map[string]any{ + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": map[string]any{ + "expression": *schedule, + "timezone": "UTC", + "interval": 0.0, + "run_immediately": false, + }, + } + } +} + +// serializeTask converts a task to the Airflow serialization format. The +// downstream_task_ids slice is read from info.Downstream (populated by the +// registry from each task's `depends` argument) and sorted for stable JSON. +func serializeTask(info bundlev1.TaskInfo) map[string]any { + typeName := info.TypeName + if typeName == "" { + typeName = info.ID + } + pkgPath := info.PkgPath + if pkgPath == "" { + pkgPath = "main" + } + data := map[string]any{ + "task_id": info.ID, + "task_type": typeName, + "_task_module": pkgPath, + "language": "go", + } + applyTaskSpec(data, info.Spec) + if len(info.Downstream) > 0 { + sorted := make([]string, len(info.Downstream)) + copy(sorted, info.Downstream) + sort.Strings(sorted) + data["downstream_task_ids"] = sorted + } + return map[string]any{ + "__type": "operator", + "__var": data, + } +} + +// applyTaskSpec mirrors Python BaseSerialization's "omit hard-coded default" +// behavior: each TaskSpec field is written into data only when it differs +// from the schema default declared in +// airflow-core/src/airflow/serialization/schema.json. A zero-valued field is +// always considered "unset" and is skipped. +func applyTaskSpec(data map[string]any, s bundlev1.TaskSpec) { + if s.Queue != "" && s.Queue != "default" { + data["queue"] = s.Queue + } + if s.Pool != "" && s.Pool != "default_pool" { + data["pool"] = s.Pool + } + if s.PoolSlots != 0 && s.PoolSlots != 1 { + data["pool_slots"] = s.PoolSlots + } + if s.Retries != 0 { + data["retries"] = s.Retries + } + if s.RetryDelay != 0 && s.RetryDelay != 300*time.Second { + data["retry_delay"] = unwrapTypeEncoding(serializeValue(s.RetryDelay)) + } + if s.MaxRetryDelay != 0 { + data["max_retry_delay"] = unwrapTypeEncoding(serializeValue(s.MaxRetryDelay)) + } + if s.RetryExponentialBackoff != 0 { + data["retry_exponential_backoff"] = s.RetryExponentialBackoff + } + if s.PriorityWeight != 0 && s.PriorityWeight != 1 { + data["priority_weight"] = s.PriorityWeight + } + if s.WeightRule != "" && s.WeightRule != "downstream" { + data["weight_rule"] = s.WeightRule + } + if s.TriggerRule != "" && s.TriggerRule != "all_success" { + data["trigger_rule"] = s.TriggerRule + } + if s.Owner != "" && s.Owner != "airflow" { + data["owner"] = s.Owner + } + if s.ExecutionTimeout != 0 { + data["execution_timeout"] = unwrapTypeEncoding(serializeValue(s.ExecutionTimeout)) + } + if s.Executor != "" { + data["executor"] = s.Executor + } + if !s.StartDate.IsZero() { + data["start_date"] = unwrapTypeEncoding(serializeValue(s.StartDate)) + } + if !s.EndDate.IsZero() { + data["end_date"] = unwrapTypeEncoding(serializeValue(s.EndDate)) + } + if s.DependsOnPast { + data["depends_on_past"] = true + } + if s.WaitForDownstream { + data["wait_for_downstream"] = true + } + // do_xcom_push / email_on_failure / email_on_retry default to true; only + // emit when an explicit false overrides the default. + if s.DoXComPush != nil && !*s.DoXComPush { + data["do_xcom_push"] = false + } + if s.EmailOnFailure != nil && !*s.EmailOnFailure { + data["email_on_failure"] = false + } + if s.EmailOnRetry != nil && !*s.EmailOnRetry { + data["email_on_retry"] = false + } + if s.DocMD != "" { + data["doc_md"] = s.DocMD + } + if s.MapIndexTemplate != "" { + data["map_index_template"] = s.MapIndexTemplate + } + if s.MaxActiveTisPerDag != 0 { + data["max_active_tis_per_dag"] = s.MaxActiveTisPerDag + } + if s.MaxActiveTisPerDagrun != 0 { + data["max_active_tis_per_dagrun"] = s.MaxActiveTisPerDagrun + } +} + +// applyDagSpec writes optional DAG-level fields onto data, omitting any +// field equal to its schema default. See applyTaskSpec for the convention. +func applyDagSpec(data map[string]any, s bundlev1.DagSpec) { + if s.Description != "" { + data["description"] = s.Description + } + if !s.StartDate.IsZero() { + data["start_date"] = unwrapTypeEncoding(serializeValue(s.StartDate)) + } + if !s.EndDate.IsZero() { + data["end_date"] = unwrapTypeEncoding(serializeValue(s.EndDate)) + } + if len(s.Tags) > 0 { + tags := make([]any, len(s.Tags)) + for i, t := range s.Tags { + tags[i] = t + } + data["tags"] = tags + } + if s.DagDisplayName != "" { + data["dag_display_name"] = s.DagDisplayName + } + if s.DocMD != "" { + data["doc_md"] = s.DocMD + } + if s.MaxActiveTasks != 0 && s.MaxActiveTasks != 16 { + data["max_active_tasks"] = s.MaxActiveTasks + } + if s.MaxActiveRuns != 0 && s.MaxActiveRuns != 16 { + data["max_active_runs"] = s.MaxActiveRuns + } + if s.MaxConsecutiveFailedDagRuns != 0 { + data["max_consecutive_failed_dag_runs"] = s.MaxConsecutiveFailedDagRuns + } + if s.DagrunTimeout != 0 { + data["dagrun_timeout"] = unwrapTypeEncoding(serializeValue(s.DagrunTimeout)) + } + if s.Catchup { + data["catchup"] = true + } + if s.FailFast { + data["fail_fast"] = true + } + if s.RenderTemplateAsNativeObj { + data["render_template_as_native_obj"] = true + } + if s.DisableBundleVersioning { + data["disable_bundle_versioning"] = true + } + if s.IsPausedUponCreation != nil { + data["is_paused_upon_creation"] = *s.IsPausedUponCreation + } +} + +// serializeTaskGroup creates a flat root task group containing all task IDs. +func serializeTaskGroup(taskIDs []string) map[string]any { + children := make(map[string]any, len(taskIDs)) + for _, id := range taskIDs { + children[id] = []any{"operator", id} + } + return map[string]any{ + "_group_id": nil, + "group_display_name": "", + "prefix_group_id": true, + "tooltip": "", + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "children": children, + "upstream_group_ids": []any{}, + "downstream_group_ids": []any{}, + "upstream_task_ids": []any{}, + "downstream_task_ids": []any{}, + } +} + +// serializeParams converts DAG params to Airflow's serialization format. +func serializeParams(params map[string]any) []any { + if len(params) == 0 { + return []any{} + } + result := make([]any, 0, len(params)) + for k, v := range params { + result = append(result, []any{ + k, + map[string]any{ + "__class": "airflow.sdk.definitions.param.Param", + "default": serializeValue(v), + "description": nil, + "schema": serializeValue(map[string]any{}), + "source": nil, + }, + }) + } + return result +} + +// SerializeDag converts a bundlev1.DagInfo to Airflow DagSerialization v3 +// format. Required fields are always present; optional fields from +// info.Spec are emitted only when they differ from their schema default +// (see applyDagSpec). +func SerializeDag(info bundlev1.DagInfo, fileloc, relativeFileloc string) map[string]any { + taskIDs := make([]string, len(info.Tasks)) + tasks := make([]any, len(info.Tasks)) + for i, t := range info.Tasks { + taskIDs[i] = t.ID + tasks[i] = serializeTask(t) + } + + var schedule *string + if info.Spec.Schedule != "" { + s := info.Spec.Schedule + schedule = &s + } + + result := map[string]any{ + // Required fields (always present) + "dag_id": info.DagID, + "fileloc": fileloc, + "relative_fileloc": relativeFileloc, + "timezone": "UTC", + "timetable": serializeTimetable(schedule), + "tasks": tasks, + "dag_dependencies": []any{}, + "task_group": serializeTaskGroup(taskIDs), + "edge_info": map[string]any{}, + "params": serializeParams(nil), + "deadline": nil, + "allowed_run_types": nil, + } + applyDagSpec(result, info.Spec) + return result +} + +// computeRelativeFileloc computes the relative file location from the bundle path. +func computeRelativeFileloc(fileloc, bundlePath string) string { + if fileloc == "" { + return "" + } + if bundlePath == "" { + return "." + } + rel, err := filepath.Rel(bundlePath, fileloc) + if err != nil { + return "." + } + if rel == "" { + return "." + } + return rel +} diff --git a/go-sdk/pkg/execution/serde_test.go b/go-sdk/pkg/execution/serde_test.go new file mode 100644 index 0000000000000..6ad434e539b10 --- /dev/null +++ b/go-sdk/pkg/execution/serde_test.go @@ -0,0 +1,410 @@ +// 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. + +package execution + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/apache/airflow/go-sdk/bundle/bundlev1" +) + +func TestSerializeValuePrimitives(t *testing.T) { + assert.Nil(t, serializeValue(nil)) + assert.Equal(t, "hello", serializeValue("hello")) + assert.Equal(t, true, serializeValue(true)) + assert.Equal(t, 42, serializeValue(42)) + assert.Equal(t, float64(3.14), serializeValue(3.14)) +} + +func TestSerializeValueDatetime(t *testing.T) { + ts := time.Date(2024, 1, 15, 10, 30, 0, 500000000, time.UTC) + result := serializeValue(ts) + m, ok := result.(map[string]any) + require.True(t, ok) + assert.Equal(t, "datetime", m["__type"]) + epochSec := m["__var"].(float64) + expected := float64(ts.Unix()) + 0.5 + assert.InDelta(t, expected, epochSec, 0.001) +} + +func TestSerializeValueTimedelta(t *testing.T) { + dur := 90 * time.Second + result := serializeValue(dur) + m, ok := result.(map[string]any) + require.True(t, ok) + assert.Equal(t, "timedelta", m["__type"]) + assert.Equal(t, 90.0, m["__var"]) +} + +func TestSerializeValueMap(t *testing.T) { + input := map[string]any{ + "key1": "val1", + "key2": 42, + } + result := serializeValue(input) + m, ok := result.(map[string]any) + require.True(t, ok) + assert.Equal(t, "dict", m["__type"]) + inner := m["__var"].(map[string]any) + assert.Equal(t, "val1", inner["key1"]) + assert.Equal(t, 42, inner["key2"]) +} + +func TestSerializeValueSlice(t *testing.T) { + input := []any{"a", 1, true} + result := serializeValue(input) + arr, ok := result.([]any) + require.True(t, ok) + assert.Len(t, arr, 3) + assert.Equal(t, "a", arr[0]) +} + +func TestUnwrapTypeEncoding(t *testing.T) { + wrapped := map[string]any{ + "__type": "datetime", + "__var": 1705313400.5, + } + assert.Equal(t, 1705313400.5, unwrapTypeEncoding(wrapped)) + + assert.Equal(t, "hello", unwrapTypeEncoding("hello")) + assert.Equal(t, 42, unwrapTypeEncoding(42)) +} + +func TestSerializeTimetable(t *testing.T) { + t.Run("nil schedule", func(t *testing.T) { + result := serializeTimetable(nil) + assert.Equal(t, "airflow.timetables.simple.NullTimetable", result["__type"]) + }) + + t.Run("@once", func(t *testing.T) { + s := "@once" + result := serializeTimetable(&s) + assert.Equal(t, "airflow.timetables.simple.OnceTimetable", result["__type"]) + }) + + t.Run("@continuous", func(t *testing.T) { + s := "@continuous" + result := serializeTimetable(&s) + assert.Equal(t, "airflow.timetables.simple.ContinuousTimetable", result["__type"]) + }) + + t.Run("cron expression", func(t *testing.T) { + s := "0 12 * * *" + result := serializeTimetable(&s) + assert.Equal(t, "airflow.timetables.trigger.CronTriggerTimetable", result["__type"]) + v := result["__var"].(map[string]any) + assert.Equal(t, "0 12 * * *", v["expression"]) + assert.Equal(t, "UTC", v["timezone"]) + assert.Equal(t, 0.0, v["interval"]) + assert.Equal(t, false, v["run_immediately"]) + }) +} + +func TestSerializeTask(t *testing.T) { + result := serializeTask(bundlev1.TaskInfo{ + ID: "extract", TypeName: "extract", PkgPath: "main", + Downstream: []string{"transform"}, + }) + assert.Equal(t, "operator", result["__type"]) + data := result["__var"].(map[string]any) + assert.Equal(t, "extract", data["task_id"]) + assert.Equal(t, "extract", data["task_type"]) + assert.Equal(t, "main", data["_task_module"]) + assert.Equal(t, "go", data["language"]) + assert.Equal(t, []string{"transform"}, data["downstream_task_ids"]) + _, hasQueue := data["queue"] + assert.False(t, hasQueue, "queue should be omitted when unset") +} + +func TestSerializeTaskDownstreamSorted(t *testing.T) { + result := serializeTask(bundlev1.TaskInfo{ + ID: "extract", TypeName: "extract", PkgPath: "main", + Downstream: []string{"transform", "audit", "load"}, + }) + data := result["__var"].(map[string]any) + assert.Equal(t, []string{"audit", "load", "transform"}, data["downstream_task_ids"]) +} + +func TestSerializeTaskNoDownstream(t *testing.T) { + result := serializeTask(bundlev1.TaskInfo{ID: "load", TypeName: "load", PkgPath: "main"}) + data := result["__var"].(map[string]any) + _, hasDownstream := data["downstream_task_ids"] + assert.False(t, hasDownstream) +} + +func TestSerializeTaskCustomQueue(t *testing.T) { + result := serializeTask(bundlev1.TaskInfo{ + ID: "extract", TypeName: "extract", PkgPath: "main", + Spec: bundlev1.TaskSpec{Queue: "high_mem"}, + }) + data := result["__var"].(map[string]any) + assert.Equal(t, "high_mem", data["queue"]) +} + +func TestSerializeTaskDefaultQueueOmitted(t *testing.T) { + result := serializeTask(bundlev1.TaskInfo{ + ID: "extract", TypeName: "extract", PkgPath: "main", + Spec: bundlev1.TaskSpec{Queue: "default"}, + }) + data := result["__var"].(map[string]any) + _, hasQueue := data["queue"] + assert.False(t, hasQueue, "queue=\"default\" matches the schema default and should be omitted") +} + +func TestApplyTaskSpec_EmitsAndOmits(t *testing.T) { + spec := bundlev1.TaskSpec{ + Queue: "gpu", + Pool: "gpu_pool", + PoolSlots: 4, + Retries: 3, + RetryDelay: 60 * time.Second, + MaxRetryDelay: 10 * time.Minute, + RetryExponentialBackoff: 2.0, + PriorityWeight: 5, + WeightRule: "upstream", + TriggerRule: "all_done", + Owner: "data-eng", + ExecutionTimeout: 45 * time.Second, + Executor: "KubernetesExecutor", + DependsOnPast: true, + WaitForDownstream: true, + DoXComPush: bundlev1.Bool(false), + EmailOnFailure: bundlev1.Bool(false), + EmailOnRetry: bundlev1.Bool(false), + DocMD: "## task", + MapIndexTemplate: "{{ task.task_id }}", + MaxActiveTisPerDag: 2, + MaxActiveTisPerDagrun: 1, + } + data := map[string]any{} + applyTaskSpec(data, spec) + + assert.Equal(t, "gpu", data["queue"]) + assert.Equal(t, "gpu_pool", data["pool"]) + assert.Equal(t, 4, data["pool_slots"]) + assert.Equal(t, 3, data["retries"]) + assert.Equal(t, 60.0, data["retry_delay"]) + assert.Equal(t, 600.0, data["max_retry_delay"]) + assert.Equal(t, 2.0, data["retry_exponential_backoff"]) + assert.Equal(t, 5, data["priority_weight"]) + assert.Equal(t, "upstream", data["weight_rule"]) + assert.Equal(t, "all_done", data["trigger_rule"]) + assert.Equal(t, "data-eng", data["owner"]) + assert.Equal(t, 45.0, data["execution_timeout"]) + assert.Equal(t, "KubernetesExecutor", data["executor"]) + assert.Equal(t, true, data["depends_on_past"]) + assert.Equal(t, true, data["wait_for_downstream"]) + assert.Equal(t, false, data["do_xcom_push"]) + assert.Equal(t, false, data["email_on_failure"]) + assert.Equal(t, false, data["email_on_retry"]) + assert.Equal(t, "## task", data["doc_md"]) + assert.Equal(t, "{{ task.task_id }}", data["map_index_template"]) + assert.Equal(t, 2, data["max_active_tis_per_dag"]) + assert.Equal(t, 1, data["max_active_tis_per_dagrun"]) +} + +func TestApplyTaskSpec_OmitsSchemaDefaults(t *testing.T) { + // Values equal to schema defaults must be dropped. + spec := bundlev1.TaskSpec{ + Queue: "default", + Pool: "default_pool", + PoolSlots: 1, + Retries: 0, + RetryDelay: 300 * time.Second, + PriorityWeight: 1, + WeightRule: "downstream", + TriggerRule: "all_success", + Owner: "airflow", + DoXComPush: bundlev1.Bool(true), + EmailOnFailure: bundlev1.Bool(true), + EmailOnRetry: bundlev1.Bool(true), + } + data := map[string]any{} + applyTaskSpec(data, spec) + assert.Empty(t, data, "all fields equal schema defaults; nothing should be emitted") +} + +func TestApplyTaskSpec_EmptySpecNoOp(t *testing.T) { + data := map[string]any{} + applyTaskSpec(data, bundlev1.TaskSpec{}) + assert.Empty(t, data) +} + +func TestSerializeTaskGroup(t *testing.T) { + result := serializeTaskGroup([]string{"t1", "t2"}) + assert.Nil(t, result["_group_id"]) + assert.Equal(t, true, result["prefix_group_id"]) + assert.Equal(t, "CornflowerBlue", result["ui_color"]) + + children := result["children"].(map[string]any) + assert.Equal(t, []any{"operator", "t1"}, children["t1"]) + assert.Equal(t, []any{"operator", "t2"}, children["t2"]) +} + +func TestSerializeParams(t *testing.T) { + t.Run("empty", func(t *testing.T) { + result := serializeParams(nil) + assert.Equal(t, []any{}, result) + }) + + t.Run("with values", func(t *testing.T) { + result := serializeParams(map[string]any{"key1": "default_val"}) + assert.Len(t, result, 1) + pair := result[0].([]any) + assert.Equal(t, "key1", pair[0]) + paramMap := pair[1].(map[string]any) + assert.Equal(t, "airflow.sdk.definitions.param.Param", paramMap["__class"]) + assert.Equal(t, "default_val", paramMap["default"]) + }) +} + +func TestSerializeDagMinimal(t *testing.T) { + info := bundlev1.DagInfo{DagID: "test_dag"} + result := SerializeDag(info, "/path/to/bundle", ".") + + assert.Equal(t, "test_dag", result["dag_id"]) + assert.Equal(t, "/path/to/bundle", result["fileloc"]) + assert.Equal(t, ".", result["relative_fileloc"]) + assert.Equal(t, "UTC", result["timezone"]) + + tt := result["timetable"].(map[string]any) + assert.Equal(t, "airflow.timetables.simple.NullTimetable", tt["__type"]) + + _, hasDesc := result["description"] + assert.False(t, hasDesc) + _, hasCatchup := result["catchup"] + assert.False(t, hasCatchup) +} + +func TestSerializeDagWithTasks(t *testing.T) { + info := bundlev1.DagInfo{ + DagID: "etl", + Tasks: []bundlev1.TaskInfo{ + { + ID: "extract", TypeName: "extract", PkgPath: "main", + Downstream: []string{"load"}, + }, + { + ID: "load", TypeName: "load", PkgPath: "main", + Spec: bundlev1.TaskSpec{Queue: "high_mem"}, + }, + }, + } + result := SerializeDag(info, "/bundle/main.go", "main.go") + + tasks := result["tasks"].([]any) + require.Len(t, tasks, 2) + first := tasks[0].(map[string]any) + v := first["__var"].(map[string]any) + assert.Equal(t, "extract", v["task_id"]) + assert.Equal(t, "extract", v["task_type"]) + assert.Equal(t, "main", v["_task_module"]) + assert.Equal(t, "go", v["language"]) + _, hasQueue := v["queue"] + assert.False(t, hasQueue, "extract has no queue set; field should be omitted") + assert.Equal(t, []string{"load"}, v["downstream_task_ids"]) + + second := tasks[1].(map[string]any)["__var"].(map[string]any) + assert.Equal(t, "high_mem", second["queue"]) + _, hasDownstream := second["downstream_task_ids"] + assert.False(t, hasDownstream, "leaf task has no downstream") + + tg := result["task_group"].(map[string]any) + children := tg["children"].(map[string]any) + assert.Contains(t, children, "extract") + assert.Contains(t, children, "load") +} + +func TestSerializeDagWithSpec(t *testing.T) { + start := time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC) + info := bundlev1.DagInfo{ + DagID: "etl", + Spec: bundlev1.DagSpec{ + Schedule: "@daily", + Description: "Extract, transform, load", + StartDate: start, + Tags: []string{"prod", "etl"}, + DagDisplayName: "ETL Pipeline", + DocMD: "## ETL", + MaxActiveTasks: 32, + MaxActiveRuns: 4, + MaxConsecutiveFailedDagRuns: 3, + DagrunTimeout: 2 * time.Hour, + Catchup: true, + FailFast: true, + RenderTemplateAsNativeObj: true, + DisableBundleVersioning: true, + IsPausedUponCreation: bundlev1.Bool(true), + }, + } + result := SerializeDag(info, "/bundle/main.go", "main.go") + + tt := result["timetable"].(map[string]any) + assert.Equal(t, "airflow.timetables.trigger.CronTriggerTimetable", tt["__type"]) + v := tt["__var"].(map[string]any) + assert.Equal(t, "@daily", v["expression"]) + + assert.Equal(t, "Extract, transform, load", result["description"]) + assert.Equal(t, []any{"prod", "etl"}, result["tags"]) + assert.Equal(t, "ETL Pipeline", result["dag_display_name"]) + assert.Equal(t, "## ETL", result["doc_md"]) + assert.Equal(t, 32, result["max_active_tasks"]) + assert.Equal(t, 4, result["max_active_runs"]) + assert.Equal(t, 3, result["max_consecutive_failed_dag_runs"]) + assert.Equal(t, (2 * time.Hour).Seconds(), result["dagrun_timeout"]) + assert.Equal(t, true, result["catchup"]) + assert.Equal(t, true, result["fail_fast"]) + assert.Equal(t, true, result["render_template_as_native_obj"]) + assert.Equal(t, true, result["disable_bundle_versioning"]) + assert.Equal(t, true, result["is_paused_upon_creation"]) + + // start_date is a raw epoch number, not the type-wrapped form. + startDate := result["start_date"].(float64) + assert.InDelta(t, float64(start.Unix()), startDate, 0.001) +} + +func TestApplyDagSpec_OmitsSchemaDefaults(t *testing.T) { + spec := bundlev1.DagSpec{ + MaxActiveTasks: 16, + MaxActiveRuns: 16, + } + data := map[string]any{} + applyDagSpec(data, spec) + assert.Empty(t, data, "values equal to schema defaults must be omitted") +} + +func TestComputeRelativeFileloc(t *testing.T) { + tests := []struct { + fileloc string + bundlePath string + want string + }{ + {"", "", ""}, + {"/a/b/c.go", "", "."}, + {"/bundles/my/dags.go", "/bundles/my", "dags.go"}, + {"/bundles/my/sub/dags.go", "/bundles/my", "sub/dags.go"}, + } + for _, tt := range tests { + result := computeRelativeFileloc(tt.fileloc, tt.bundlePath) + assert.Equal(t, tt.want, result, "fileloc=%q bundlePath=%q", tt.fileloc, tt.bundlePath) + } +} diff --git a/go-sdk/pkg/execution/server.go b/go-sdk/pkg/execution/server.go new file mode 100644 index 0000000000000..bee35a028d9de --- /dev/null +++ b/go-sdk/pkg/execution/server.go @@ -0,0 +1,155 @@ +// 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. + +// Package execution implements the SDK coordinator-protocol runtime +// (msgpack-over-IPC). It is the second mode of bundlev1server.Serve: when +// the bundle binary is launched with --comm/--logs by the Airflow supervisor +// (Python ExecutableCoordinator), bundlev1server.Serve dispatches here. +// +// The first inbound frame on the comm socket selects between two +// sub-protocols: +// +// - DagFileParseRequest: one-shot, returns DagFileParsingResult and exits. +// - StartupDetails: multi-round task execution. +// +// See go-sdk/adr/0003-coordinator-protocol-msgpack-ipc.md. +package execution + +import ( + "fmt" + "log/slog" + "net" + "sync" + + "github.com/apache/airflow/go-sdk/bundle/bundlev1" +) + +// Serve runs the bundle binary in coordinator mode. It dials the supervisor's +// comm and logs sockets, installs an slog handler that writes JSON-line +// records to the logs connection, and dispatches on the first frame. +// +// Serve returns nil on a clean shutdown (one-shot DAG parse or task execution +// completed); a non-nil error indicates a protocol-level failure (connection +// loss, malformed frames, unknown first message type). +func Serve(provider bundlev1.BundleProvider, commAddr, logsAddr string) error { + if commAddr == "" { + return fmt.Errorf("missing --comm=host:port argument") + } + if logsAddr == "" { + return fmt.Errorf("missing --logs=host:port argument") + } + + // Buffer log records until the logs socket is connected. Anything the + // runtime emits between Connect-time and the first frame still gets + // flushed. + logHandler := NewSocketLogHandler(nil, slog.LevelDebug) + logger := slog.New(logHandler) + slog.SetDefault(logger) + + // Connect to both sockets concurrently so the supervisor can accept them + // in either order. + var commConn, logsConn net.Conn + var commErr, logsErr error + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + commConn, commErr = net.Dial("tcp", commAddr) + }() + go func() { + defer wg.Done() + logsConn, logsErr = net.Dial("tcp", logsAddr) + }() + wg.Wait() + + if commErr != nil { + return fmt.Errorf("connecting to comm socket %s: %w", commAddr, commErr) + } + defer commConn.Close() + if logsErr != nil { + return fmt.Errorf("connecting to logs socket %s: %w", logsAddr, logsErr) + } + defer logsConn.Close() + + logHandler.Connect(logsConn) + logger.Debug("Connected", "comm", commAddr, "logs", logsAddr) + + // Materialise the bundle (RegisterDags) up front. Both protocol paths + // need the registry, and doing it once before the first frame keeps the + // dispatcher simple. + bundle, err := materialiseBundle(provider) + if err != nil { + return fmt.Errorf("registering dags: %w", err) + } + + comm := NewCoordinatorComm(commConn, commConn, logger) + + frame, err := comm.ReadMessage() + if err != nil { + return fmt.Errorf("reading initial message: %w", err) + } + + if frame.Err != nil { + errResp := decodeErrorResponse(frame.Err) + if errResp != nil { + return fmt.Errorf( + "received error from supervisor: [%s] %v", + errResp.Error, + errResp.Detail, + ) + } + } + + body, err := decodeIncomingBody(frame.Body) + if err != nil { + return fmt.Errorf("decoding initial message: %w", err) + } + + switch msg := body.(type) { + case *DagFileParseRequest: + logger.Debug("DAG parsing mode", "file", msg.File) + result := ParseDags(bundle, msg) + if err := comm.SendRequest(frame.ID, result); err != nil { + return fmt.Errorf("sending parse result: %w", err) + } + logger.Debug("DAG parsing complete") + + case *StartupDetails: + logger.Debug("Task execution mode", + "dag_id", msg.TI.DagID, + "task_id", msg.TI.TaskID, + ) + result := RunTask(bundle, msg, comm, logger) + if err := comm.SendRequest(frame.ID, result); err != nil { + return fmt.Errorf("sending task result: %w", err) + } + logger.Debug("Task execution complete") + + default: + return fmt.Errorf("unexpected initial message type: %T", body) + } + + return nil +} + +func materialiseBundle(provider bundlev1.BundleProvider) (bundlev1.Bundle, error) { + reg := bundlev1.New() + if err := provider.RegisterDags(reg); err != nil { + return nil, err + } + return reg, nil +} diff --git a/go-sdk/pkg/execution/task_runner.go b/go-sdk/pkg/execution/task_runner.go new file mode 100644 index 0000000000000..cae7613aa67d9 --- /dev/null +++ b/go-sdk/pkg/execution/task_runner.go @@ -0,0 +1,117 @@ +// 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. + +package execution + +import ( + "context" + "fmt" + "log/slog" + "runtime/debug" + "time" + + "github.com/google/uuid" + + "github.com/apache/airflow/go-sdk/bundle/bundlev1" + "github.com/apache/airflow/go-sdk/pkg/api" + "github.com/apache/airflow/go-sdk/pkg/sdkcontext" + "github.com/apache/airflow/go-sdk/sdk" +) + +// RunTask executes a task based on StartupDetails received from the supervisor. +// +// It looks up the task in the bundle, creates a CoordinatorClient for SDK +// calls, executes the task, and returns a terminal message body +// (SucceedTaskMsg or TaskStateMsg) ready to ship as the final response frame. +// +// The supervisor owns the Execution-API state transitions in coordinator +// mode, so we deliberately bypass worker.ExecuteTaskWorkload (which drives +// Run / UpdateState itself) and only invoke the user's task function. +func RunTask( + bundle bundlev1.Bundle, + details *StartupDetails, + comm *CoordinatorComm, + logger *slog.Logger, +) map[string]any { + task, exists := bundle.LookupTask(details.TI.DagID, details.TI.TaskID) + if !exists { + logger.Error("Task not registered", + "dag_id", details.TI.DagID, + "task_id", details.TI.TaskID, + ) + return TaskStateMsg{State: "removed", EndDate: time.Now().UTC()}.toMap() + } + + client := NewCoordinatorClient(comm, details) + + // taskFunction.sendXcom reads the workload from context to get the task + // instance ids; populate it the same shape the gRPC path uses. + tiUUID, _ := uuid.Parse(details.TI.ID) + mapIndex := details.TI.MapIndex + workload := api.ExecuteTaskWorkload{ + TI: api.TaskInstance{ + Id: tiUUID, + DagId: details.TI.DagID, + RunId: details.TI.RunID, + TaskId: details.TI.TaskID, + TryNumber: details.TI.TryNumber, + MapIndex: &mapIndex, + }, + BundleInfo: api.BundleInfo{ + Name: details.BundleInfo.Name, + Version: &details.BundleInfo.Version, + }, + } + + ctx := context.Background() + ctx = context.WithValue(ctx, sdkcontext.WorkloadContextKey, workload) + ctx = context.WithValue(ctx, sdkcontext.SdkClientContextKey, sdk.Client(client)) + + return executeTask(ctx, task, logger) +} + +// executeTask runs the task and handles success, failure, and panics. +func executeTask( + ctx context.Context, + task bundlev1.Task, + logger *slog.Logger, +) (result map[string]any) { + defer func() { + if r := recover(); r != nil { + logger.Error("Recovered panic in task", + "error", r, + "stack", string(debug.Stack()), + ) + result = TaskStateMsg{ + State: "failed", + EndDate: time.Now().UTC(), + }.toMap() + } + }() + + if err := task.Execute(ctx, logger); err != nil { + logger.Error("Task failed", "error", fmt.Sprintf("%v", err)) + return TaskStateMsg{ + State: "failed", + EndDate: time.Now().UTC(), + }.toMap() + } + + return SucceedTaskMsg{ + EndDate: time.Now().UTC(), + }.toMap() +} diff --git a/go-sdk/pkg/sdkcontext/keys.go b/go-sdk/pkg/sdkcontext/keys.go index 0dbc2c6019487..ad83dfce3bd6e 100644 --- a/go-sdk/pkg/sdkcontext/keys.go +++ b/go-sdk/pkg/sdkcontext/keys.go @@ -22,6 +22,7 @@ type ( apiClientContextKey struct{} workerContextKey struct{} runtimeTIContextKey struct{} + sdkClientContextKey struct{} ) var ( @@ -32,4 +33,11 @@ var ( RuntimeTIContextKey = runtimeTIContextKey{} ApiClientContextKey = apiClientContextKey{} WorkerContextKey = workerContextKey{} + + // SdkClientContextKey, when present, holds an sdk.Client implementation + // that should be injected into task functions instead of constructing a + // default HTTP-backed client. The coordinator-mode runtime uses this to + // route task SDK calls (GetVariable, GetConnection, ...) over the + // supervisor comm socket rather than to the Execution API. + SdkClientContextKey = sdkClientContextKey{} ) diff --git a/go-sdk/pkg/worker/runner_test.go b/go-sdk/pkg/worker/runner_test.go index 72673c00df376..f5026e84a2017 100644 --- a/go-sdk/pkg/worker/runner_test.go +++ b/go-sdk/pkg/worker/runner_test.go @@ -161,7 +161,7 @@ func (s *WorkerSuite) TestStartContextErrorTaskDoesntStart() { s.registry.AddDag(testWorkload.TI.DagId).AddTaskWithName(testWorkload.TI.TaskId, func() error { wasCalled = true return nil - }) + }, bundlev1.TaskSpec{}, nil) // Setup the mock s.ti.EXPECT(). @@ -194,7 +194,7 @@ func (s *WorkerSuite) TestTaskHeartbeatsWhileRunning() { s.registry.AddDag(testWorkload.TI.DagId).AddTaskWithName(testWorkload.TI.TaskId, func() error { time.Sleep(time.Second) return nil - }) + }, bundlev1.TaskSpec{}, nil) s.ExpectTaskRun(id) s.ExpectTaskState(id, api.TerminalTIStateSuccess) diff --git a/go-sdk/sdk/client.go b/go-sdk/sdk/client.go index d2aa1f40351f3..3e10c913d1c3d 100644 --- a/go-sdk/sdk/client.go +++ b/go-sdk/sdk/client.go @@ -84,7 +84,7 @@ func (*client) GetConnection(ctx context.Context, connID string) (Connection, er return Connection{}, err } - return connFromAPIResponse(resp) + return ConnFromAPIResponse(resp) } func (c *client) PushXCom( diff --git a/go-sdk/sdk/connection.go b/go-sdk/sdk/connection.go index 35835c2d523d4..1d0bcf30b6f89 100644 --- a/go-sdk/sdk/connection.go +++ b/go-sdk/sdk/connection.go @@ -110,7 +110,11 @@ func (c Connection) GetURI() *url.URL { return uri } -func connFromAPIResponse(resp *api.ConnectionResponse) (Connection, error) { +// ConnFromAPIResponse converts an Execution-API ConnectionResponse into the +// SDK's Connection type. It is exported so other internal SDK packages (for +// example, the coordinator-mode runtime in bundlev1server/impl/coord) can +// reuse the same conversion. +func ConnFromAPIResponse(resp *api.ConnectionResponse) (Connection, error) { var err error conn := Connection{ ID: resp.ConnId, diff --git a/providers/edge3/src/airflow/providers/edge3/worker_api/v2-edge-generated.yaml b/providers/edge3/src/airflow/providers/edge3/worker_api/v2-edge-generated.yaml index 01c8149d1dad8..b041c1daa44e9 100644 --- a/providers/edge3/src/airflow/providers/edge3/worker_api/v2-edge-generated.yaml +++ b/providers/edge3/src/airflow/providers/edge3/worker_api/v2-edge-generated.yaml @@ -1257,8 +1257,12 @@ components: - queue - priority_weight title: TaskInstanceDTO - description: Schema for TaskInstance with minimal required fields needed for - Executors and Task SDK. + description: 'TaskInstanceDTO with executor-specific ``key`` property. + + + Extends the shared :class:`~airflow._shared.workloads.TaskInstanceDTO` + + to add the :attr:`key` property used by executors for workload tracking.' TaskInstanceState: type: string enum: diff --git a/providers/sdk/executable/.gitignore b/providers/sdk/executable/.gitignore new file mode 100644 index 0000000000000..bff2d7629604d --- /dev/null +++ b/providers/sdk/executable/.gitignore @@ -0,0 +1 @@ +*.iml diff --git a/providers/sdk/executable/LICENSE b/providers/sdk/executable/LICENSE new file mode 100644 index 0000000000000..11069edd79019 --- /dev/null +++ b/providers/sdk/executable/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +Licensed 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. diff --git a/providers/sdk/executable/NOTICE b/providers/sdk/executable/NOTICE new file mode 100644 index 0000000000000..a51bd9390d030 --- /dev/null +++ b/providers/sdk/executable/NOTICE @@ -0,0 +1,5 @@ +Apache Airflow +Copyright 2016-2026 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/providers/sdk/executable/README.rst b/providers/sdk/executable/README.rst new file mode 100644 index 0000000000000..60050ac7a4ab9 --- /dev/null +++ b/providers/sdk/executable/README.rst @@ -0,0 +1,25 @@ + +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +``apache-airflow-providers-sdk-executable`` +=========================================== + +Native executable language support for Apache Airflow runtime coordinators. + +Supports any compiled binary (Go, Rust, etc.) that implements the Airflow +SDK coordinator protocol (``--comm``/``--logs`` socket-based IPC). diff --git a/providers/sdk/executable/docs/airflow-metadata.schema.json b/providers/sdk/executable/docs/airflow-metadata.schema.json new file mode 100644 index 0000000000000..489d0c1649cf9 --- /dev/null +++ b/providers/sdk/executable/docs/airflow-metadata.schema.json @@ -0,0 +1,65 @@ +{ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "$id": "https://airflow.apache.org/schemas/sdk-executable/airflow-metadata-1.0.schema.json", + "title": "Airflow Executable SDK Bundle Metadata", + "description": "Build-time manifest declaring DAG and task identifiers exposed by an Airflow native-executable SDK bundle. See the Bundle Spec Format documentation in apache-airflow-providers-sdk-executable.", + "type": "object", + "required": ["format_version", "sdk", "source", "dags"], + "additionalProperties": true, + "properties": { + "format_version": { + "type": "string", + "description": "Bundle-spec version this manifest conforms to (currently '1.0').", + "pattern": "^[0-9]+\\.[0-9]+(\\.[0-9]+)?$" + }, + "sdk": { + "type": "object", + "description": "Identifies the SDK that produced the bundle.", + "required": ["language", "version"], + "additionalProperties": true, + "properties": { + "language": { + "type": "string", + "description": "Lower-case source-language identifier (e.g. 'go', 'rust', 'cpp', 'zig').", + "pattern": "^[a-z][a-z0-9_+.\\-]*$" + }, + "version": { + "type": "string", + "description": "SDK version used at build time.", + "minLength": 1 + } + } + }, + "source": { + "type": "string", + "description": "Original filename of the primary DAG source file (e.g. 'example.go'). The file's bytes are embedded in the bundle's source region; this field is a display name used by the Airflow UI.", + "minLength": 1 + }, + "dags": { + "type": "object", + "description": "Mapping of dag_id to DAG entry. Every dag_id the bundle exposes must appear here.", + "minProperties": 1, + "additionalProperties": { + "$ref": "#/$defs/dagEntry" + } + } + }, + "$defs": { + "dagEntry": { + "type": "object", + "description": "Static description of a single DAG declared in the bundle.", + "required": ["tasks"], + "additionalProperties": true, + "properties": { + "tasks": { + "type": "array", + "description": "Static list of task_ids declared in the DAG.", + "items": { + "type": "string", + "minLength": 1 + } + } + } + } + } +} diff --git a/providers/sdk/executable/docs/bundle-spec.rst b/providers/sdk/executable/docs/bundle-spec.rst new file mode 100644 index 0000000000000..8e5a168c5a992 --- /dev/null +++ b/providers/sdk/executable/docs/bundle-spec.rst @@ -0,0 +1,250 @@ + .. 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. + +Bundle Spec Format +================== + +This document specifies the on-disk format of a build artifact produced by an +Airflow native-executable SDK (Go, Rust, C++, Zig, ...) and consumed by +:class:`~airflow.providers.sdk.executable.coordinator.ExecutableCoordinator` +at deployment time. + +The goal is a single, language-agnostic *bundle* shape so that scheduler, +worker, and UI behave identically regardless of which compiled SDK produced +the DAG. + +Format version: ``1.0``. + +Container +--------- + +A bundle is **the compiled executable itself, with a fixed-format footer +appended after the binary's normal end-of-file**. The executable remains +directly runnable; the footer is data that follows the last byte the OS +loader cares about and is invisible to ``exec()``. There is no enclosing +archive. + +A bundle file therefore has three regions, in order from offset 0: + +1. The native executable (ELF / Mach-O / PE), including any code-signing + structures the platform appends. +2. The primary DAG source file, embedded verbatim (UTF-8). MAY have length 0. +3. The build-time manifest (``airflow-metadata.yaml`` content, UTF-8). + +The file ends with a fixed 32-byte trailer that locates regions (2) and (3) +and identifies the file as a bundle. See :ref:`bundle-trailer-layout`. + +Filenames follow OS conventions for executables: no extension on Linux/macOS, +``.exe`` on Windows. The scanner identifies bundles by the trailer's magic, +not by the filename. + +.. _bundle-trailer-layout: + +Trailer Layout +-------------- + +The last 32 bytes of a conforming bundle are the trailer. All multi-byte +integers are little-endian. + +:: + + bytes 0..3 source_len uint32 length of the source region in bytes + bytes 4..7 metadata_len uint32 length of the metadata region in bytes + bytes 8..11 footer_ver uint32 currently 1 + bytes 12..23 reserved 12 bytes, MUST be zero + bytes 24..31 magic 8 bytes ASCII "AFBNDL01" + +The magic is the byte sequence ``0x41 0x46 0x42 0x4E 0x44 0x4C 0x30 0x31`` +(``"AFBNDL01"``). The trailing ``01`` is the footer-format version repeated +in ASCII so a human can identify a bundle at a glance +(``tail -c 8 ./mybundle | xxd``); the binary ``footer_ver`` field is the +authoritative source of truth for parsing. + +Reader algorithm: + +1. Open the file. Seek to ``EOF - 32``. Read 32 bytes. +2. Compare bytes ``24..31`` against ``"AFBNDL01"``. If different, the file + is not a bundle; the scanner MUST ignore it. +3. Parse ``footer_ver``. If unknown, fail with a versioning error. +4. Compute ``metadata_start = filesize - 32 - metadata_len`` and + ``source_start = metadata_start - source_len``. +5. Read ``metadata_len`` bytes from ``metadata_start`` for the manifest. +6. Read ``source_len`` bytes from ``source_start`` for the source view. + If ``source_len == 0``, no source is embedded; the UI displays + "(source not available)". +7. Validate ``source_start >= 0`` and that the implied binary region + (``[0, source_start)``) is non-empty. + +Source comes *before* metadata so a future ``footer_ver`` MAY introduce +additional trailing blobs (e.g. signed checksums, compressed deps) by +extending the trailer rather than inserting between existing blobs. + +.. _bundle-metadata-schema: + +``airflow-metadata.yaml`` schema +-------------------------------- + +The metadata region carries the same YAML manifest documented previously, +produced at build time from a static scan of the DAG source. A +machine-readable JSON Schema is published at +:download:`airflow-metadata.schema.json` for use by build tooling, validators, +and editors. + +.. code-block:: yaml + + format_version: "1.0" + sdk: + language: go + version: "0.1.0" + source: example.go + dags: + example_dag: + tasks: + - extract + - transform + - load + another_dag: + tasks: + - run + +Top-level keys: + +``format_version`` (string, required) + The bundle-spec version this manifest conforms to. Currently ``"1.0"``. + +``sdk`` (mapping, required) + Identifies the SDK that produced the bundle. + + - ``language`` (string, required): lower-case source-language identifier + (e.g. ``go``, ``rust``, ``cpp``, ``zig``). + - ``version`` (string, required): SDK version used at build time. + +``source`` (string, required) + Original filename of the primary DAG source file (e.g. ``example.go``). + The file's bytes live in the source region of the bundle, not at this + path; this field is a display name the Airflow UI uses to label the + source-view panel and pick a syntax-highlighting mode from the + extension. + +``dags`` (mapping, required) + Mapping of ``dag_id`` to a *DAG entry*. Every ``dag_id`` the bundle + exposes MUST appear here. The scanner uses these keys to match a DAG + parsing or task-execution request to the bundle that owns it. + +DAG entry fields: + +``tasks`` (list of strings, required) + Static list of ``task_id``\ s declared in the DAG. Empty lists are + permitted but discouraged. + +Unrecognised top-level or DAG-entry keys MUST be ignored by the consumer so +that future SDK versions can extend the manifest without breaking older +runtimes. + +Examples +-------- + +Go bundle:: + + example + ├── ELF/Mach-O/PE executable + ├── source region: contents of example.go + ├── metadata region: airflow-metadata.yaml (source: example.go) + └── trailer (32 B): AFBNDL01 magic + lengths + +Rust bundle:: + + pipeline + ├── ELF/Mach-O/PE executable + ├── source region: contents of main.rs + ├── metadata region: airflow-metadata.yaml (source: main.rs) + └── trailer (32 B): AFBNDL01 magic + lengths + +The bundle is one file. ``./example`` runs the binary; the appended data +is invisible to ``exec()``. + +Build Pipeline Ordering +----------------------- + +The footer is appended after the executable is otherwise complete. Producers +that perform additional post-build steps MUST observe the following order: + +- **Strip** debug symbols *before* appending the footer. Strip + implementations operate on the binary's defined end and either leave + trailing data intact or truncate it; do not rely on either behaviour. +- **Code-sign** *after* appending the footer on platforms whose signature + covers the entire file (Authenticode, certain notarisation flows). The + signature then attests to the footer's contents along with the binary. +- **Compressors** such as UPX are NOT supported. They rewrite the file + end-to-end and destroy the trailer. + +Determinism: the trailer is byte-identical for byte-identical inputs, so a +deterministic build plus a canonical (sorted-key) manifest serialisation +yields a byte-identical bundle file. + +Deployment Layout +----------------- + +Bundle files are placed **as-is** in the directory configured by +``[executable] bundles_folder``. The scanner enumerates regular files in +this directory, reads the last 32 bytes of each, and treats files whose +magic matches ``"AFBNDL01"`` as bundles. Files without the magic are +silently ignored, so non-bundle files (READMEs, dotfiles) MAY share the +directory without interfering with the scan. + +:: + + /opt/airflow/executable-bundles/ + ├── example + ├── pipeline + └── analytics + +At task-execution time the runtime execs the bundle file directly with the +coordinator arguments (``--comm=`` / ``--logs=``). No extraction, +no transient cache directory, no chmod-after-extract step is required: the +file is already a runnable executable with the appropriate permission bits +preserved by the build pipeline. + +The compiled executable MUST honour the SDK coordinator protocol — +``--comm=`` / ``--logs=`` socket-based IPC. + +See :class:`~airflow.providers.sdk.executable.bundle_scanner.BundleScanner` +for the consumer-side scanner. + +Inspection +---------- + +Because the bundle is a single executable rather than an archive, +inspecting the embedded source and manifest requires a small CLI rather +than an off-the-shelf ``unzip``. The Go SDK's ``airflow-go-pack`` tool +provides an ``inspect`` subcommand that dumps both regions; equivalent +helpers are expected from each language's packer. + +Compatibility and Versioning +---------------------------- + +- The current bundle-spec format version is ``1.0``; the current trailer + format version is ``1`` (``footer_ver = 1``). +- Backward-incompatible bundle-spec changes increment the major component + of ``format_version`` and are gated behind an explicit opt-in on the + consumer side. +- New optional manifest fields MAY be added in minor versions and MUST be + ignored by older consumers. +- New trailer-format versions append fields after ``footer_ver`` (consuming + the reserved region) or extend the trailer with additional trailing + blobs ahead of the magic. Older readers MUST reject unknown + ``footer_ver`` rather than guessing. diff --git a/providers/sdk/executable/docs/changelog.rst b/providers/sdk/executable/docs/changelog.rst new file mode 100644 index 0000000000000..abbcf8dddd86c --- /dev/null +++ b/providers/sdk/executable/docs/changelog.rst @@ -0,0 +1,25 @@ + .. 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. + + +Changelog +--------- + +0.1.0 +..... + +Initial release of the ``apache-airflow-providers-sdk-executable`` provider. diff --git a/providers/sdk/executable/docs/commits.rst b/providers/sdk/executable/docs/commits.rst new file mode 100644 index 0000000000000..1e9802470054a --- /dev/null +++ b/providers/sdk/executable/docs/commits.rst @@ -0,0 +1,18 @@ + .. 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. + + .. THIS FILE IS UPDATED AUTOMATICALLY_AT_RELEASE_TIME diff --git a/providers/sdk/executable/docs/conf.py b/providers/sdk/executable/docs/conf.py new file mode 100644 index 0000000000000..3c29b9e1d9cca --- /dev/null +++ b/providers/sdk/executable/docs/conf.py @@ -0,0 +1,27 @@ +# Disable Flake8 because of all the sphinx imports +# +# 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. +"""Configuration of Providers docs building.""" + +from __future__ import annotations + +import os + +os.environ["AIRFLOW_PACKAGE_NAME"] = "apache-airflow-providers-sdk-executable" + +from docs.provider_conf import * # noqa: F403 diff --git a/providers/sdk/executable/docs/index.rst b/providers/sdk/executable/docs/index.rst new file mode 100644 index 0000000000000..83f716b3b4246 --- /dev/null +++ b/providers/sdk/executable/docs/index.rst @@ -0,0 +1,122 @@ + + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +``apache-airflow-providers-sdk-executable`` +================================================= + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Basics + + Home + Changelog + Security + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: References + + Bundle Spec Format + Python API <_api/airflow/providers/sdk/executable/index> + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Resources + + PyPI Repository + Installing from sources + + + +.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Commits + + Detailed list of commits + + +apache-airflow-providers-sdk-executable package +------------------------------------------------------ + +Native executable language support for Apache Airflow runtime coordinators. +Supports any compiled binary (Go, Rust, etc.) that implements the Airflow +SDK coordinator protocol (--comm/--logs socket-based IPC). + + +Release: 0.1.0 + +Provider package +---------------- + +This package is for the ``sdk.executable`` provider. +All classes for this package are included in the ``airflow.providers.sdk.executable`` python package. + +Installation +------------ + +You can install this package on top of an existing Airflow installation via +``pip install apache-airflow-providers-sdk-executable``. +For the minimum Airflow version supported, see ``Requirements`` below. + +Requirements +------------ + +The minimum Apache Airflow version supported by this provider distribution is ``3.0.0``. + +================== ================== +PIP package Version required +================== ================== +``apache-airflow`` ``>=3.0.0`` +``pyyaml`` +================== ================== + +Cross provider package dependencies +----------------------------------- + +Those are dependencies that might be needed in order to use all the features of the package. +You need to install the specified provider distributions in order to use them. + +You can install such cross-provider dependencies when installing from PyPI. For example: + +.. code-block:: bash + + pip install apache-airflow-providers-sdk-executable[common.compat] + + +================================================================================================================== ================= +Dependent package Extra +================================================================================================================== ================= +`apache-airflow-providers-common-compat `_ ``common.compat`` +================================================================================================================== ================= + +Downloading official packages +----------------------------- + +You can download officially released packages and verify their checksums and signatures from the +`Official Apache Download site `_ + +* `The apache-airflow-providers-sdk-executable 0.1.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-sdk-executable 0.1.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/providers/sdk/executable/docs/installing-providers-from-sources.rst b/providers/sdk/executable/docs/installing-providers-from-sources.rst new file mode 100644 index 0000000000000..a72b45ffaa6e8 --- /dev/null +++ b/providers/sdk/executable/docs/installing-providers-from-sources.rst @@ -0,0 +1,18 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. include:: /../../../devel-common/src/sphinx_exts/includes/installing-providers-from-sources.rst diff --git a/providers/sdk/executable/docs/security.rst b/providers/sdk/executable/docs/security.rst new file mode 100644 index 0000000000000..15a0ebbb2d054 --- /dev/null +++ b/providers/sdk/executable/docs/security.rst @@ -0,0 +1,18 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. include:: /../../../devel-common/src/sphinx_exts/includes/security.rst diff --git a/providers/sdk/executable/provider.yaml b/providers/sdk/executable/provider.yaml new file mode 100644 index 0000000000000..22be9cb9a0d0c --- /dev/null +++ b/providers/sdk/executable/provider.yaml @@ -0,0 +1,56 @@ +# 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. + +--- +package-name: apache-airflow-providers-sdk-executable +name: "SDK: Executable" +description: | + Native executable language support for Apache Airflow runtime coordinators. + Supports any compiled binary (Go, Rust, etc.) that implements the Airflow + SDK coordinator protocol (--comm/--logs socket-based IPC). + +state: ready +lifecycle: incubation +source-date-epoch: 1775631151 +versions: + - 0.1.0 + +integrations: + - integration-name: Native Executable + external-doc-url: https://airflow.apache.org/ + tags: + - software + +config: + executable: + description: "Options for the native executable language provider." + options: + bundles_folder: + description: | + Path to the directory containing native executable DAG bundles. + When using Python stub DAGs that delegate task execution to a + native runtime, the coordinator scans this directory for files + whose AFBNDL01 trailer marks them as Airflow executable bundles + and selects the one whose embedded manifest declares the target + dag_id. + type: string + version_added: ~ + example: ~/airflow/executable-bundles + default: "" + +coordinators: + - airflow.providers.sdk.executable.coordinator.ExecutableCoordinator diff --git a/providers/sdk/executable/pyproject.toml b/providers/sdk/executable/pyproject.toml new file mode 100644 index 0000000000000..56c22089eaad5 --- /dev/null +++ b/providers/sdk/executable/pyproject.toml @@ -0,0 +1,132 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! + +# IF YOU WANT TO MODIFY THIS FILE EXCEPT DEPENDENCIES, YOU SHOULD MODIFY THE TEMPLATE +# `pyproject_TEMPLATE.toml.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY +[build-system] +requires = ["flit_core==3.12.0"] +build-backend = "flit_core.buildapi" + +[project] +name = "apache-airflow-providers-sdk-executable" +version = "0.1.0" +description = "Provider package apache-airflow-providers-sdk-executable for Apache Airflow" +readme = "README.rst" +license = "Apache-2.0" +license-files = ['LICENSE', 'NOTICE'] +authors = [ + {name="Apache Software Foundation", email="dev@airflow.apache.org"}, +] +maintainers = [ + {name="Apache Software Foundation", email="dev@airflow.apache.org"}, +] +keywords = [ "airflow-provider", "sdk.executable", "airflow", "integration" ] +classifiers = [ + "Development Status :: 5 - Production/Stable", + "Environment :: Console", + "Environment :: Web Environment", + "Intended Audience :: Developers", + "Intended Audience :: System Administrators", + "Framework :: Apache Airflow", + "Framework :: Apache Airflow :: Provider", + "Programming Language :: Python :: 3.10", + "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: 3.12", + "Programming Language :: Python :: 3.13", + "Programming Language :: Python :: 3.14", + "Topic :: System :: Monitoring", +] +requires-python = ">=3.10" + +# The dependencies should be modified in place in the generated file. +# Any change in the dependencies is preserved when the file is regenerated +# Make sure to run ``prek update-providers-dependencies --all-files`` +# After you modify the dependencies, and rebuild your Breeze CI image with ``breeze ci-image build`` +dependencies = [ + "apache-airflow>=3.0.0", + "pyyaml", +] + +# The optional dependencies should be modified in place in the generated file +# Any change in the dependencies is preserved when the file is regenerated +[project.optional-dependencies] +"common.compat" = [ + "apache-airflow-providers-common-compat" +] + +[dependency-groups] +dev = [ + "apache-airflow", + "apache-airflow-task-sdk", + "apache-airflow-devel-common", + "apache-airflow-providers-common-compat", + # Additional devel dependencies (do not remove this line and add extra development dependencies) +] + +# To build docs: +# +# uv run --group docs build-docs +# +# To enable auto-refreshing build with server: +# +# uv run --group docs build-docs --autobuild +# +# To see more options: +# +# uv run --group docs build-docs --help +# +docs = [ + "apache-airflow-devel-common[docs]" +] + +[tool.uv.sources] +# These names must match the names as defined in the pyproject.toml of the workspace items, +# *not* the workspace folder paths +apache-airflow = {workspace = true} +apache-airflow-devel-common = {workspace = true} +apache-airflow-task-sdk = {workspace = true} +apache-airflow-providers-common-sql = {workspace = true} +apache-airflow-providers-standard = {workspace = true} + +[project.urls] +"Documentation" = "https://airflow.apache.org/docs/apache-airflow-providers-sdk-executable/0.1.0" +"Changelog" = "https://airflow.apache.org/docs/apache-airflow-providers-sdk-executable/0.1.0/changelog.html" +"Bug Tracker" = "https://github.com/apache/airflow/issues" +"Source Code" = "https://github.com/apache/airflow" +"Slack Chat" = "https://s.apache.org/airflow-slack" +"Mastodon" = "https://fosstodon.org/@airflow" +"YouTube" = "https://www.youtube.com/channel/UCSXwxpWZQ7XZ1WL3wqevChA/" + +[project.entry-points."apache_airflow_provider"] +provider_info = "airflow.providers.sdk.executable.get_provider_info:get_provider_info" + +[tool.flit.module] +name = "airflow.providers.sdk.executable" + +# Explicit sdist contents so the build does not rely on VCS information +# (flit 4.0 makes --no-use-vcs the default — see https://github.com/pypa/flit/pull/782). +[tool.flit.sdist] +include = [ + "docs/", + "provider.yaml", + "src/airflow/__init__.py", + "src/airflow/providers/__init__.py", + "src/airflow/providers/sdk/__init__.py", + "tests/", +] diff --git a/providers/sdk/executable/src/airflow/__init__.py b/providers/sdk/executable/src/airflow/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/executable/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/executable/src/airflow/providers/__init__.py b/providers/sdk/executable/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/executable/src/airflow/providers/sdk/__init__.py b/providers/sdk/executable/src/airflow/providers/sdk/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/sdk/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/executable/src/airflow/providers/sdk/executable/__init__.py b/providers/sdk/executable/src/airflow/providers/sdk/executable/__init__.py new file mode 100644 index 0000000000000..4160892edd09e --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/sdk/executable/__init__.py @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE +# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. +# +# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE +# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY +# +from __future__ import annotations + +import packaging.version + +from airflow import __version__ as airflow_version + +__all__ = ["__version__"] + +__version__ = "0.1.0" + +if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( + "3.0.0" +): + raise RuntimeError( + f"The package `apache-airflow-providers-sdk-executable:{__version__}` needs Apache Airflow 3.0.0+" + ) diff --git a/providers/sdk/executable/src/airflow/providers/sdk/executable/bundle_scanner.py b/providers/sdk/executable/src/airflow/providers/sdk/executable/bundle_scanner.py new file mode 100644 index 0000000000000..1a29b4c35486e --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/sdk/executable/bundle_scanner.py @@ -0,0 +1,200 @@ +# +# 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. +""" +Scan directories for native executable Airflow SDK bundles. + +A bundle is a single self-contained executable with a fixed-format trailer +appended after the binary. The last 32 bytes of the file form the trailer +and locate two preceding regions: the embedded DAG source and the +``airflow-metadata.yaml`` manifest. See :doc:`bundle-spec` for the wire +format. + +Detection is by the trailer magic ``AFBNDL01``; files without it are +silently ignored, so non-bundle entries (READMEs, dotfiles, ...) MAY share +the directory. +""" + +from __future__ import annotations + +import os +import struct +from pathlib import Path +from typing import Any, NamedTuple + +import yaml + +FOOTER_MAGIC = b"AFBNDL01" +FOOTER_SIZE = 32 +FOOTER_VERSION = 1 + + +class _Footer(NamedTuple): + source_len: int + metadata_len: int + footer_ver: int + + +def _read_footer(path: Path) -> _Footer | None: + """ + Parse the trailer at the end of *path*. + + :returns: a :class:`_Footer` when the trailer's magic matches and the + declared regions are within bounds; ``None`` when the file is too + small or the magic does not match (i.e. it is not a bundle). + :raises ValueError: when the magic matches but the trailer is otherwise + malformed (unknown ``footer_ver`` or out-of-bounds region offsets). + """ + try: + size = path.stat().st_size + except OSError: + return None + if size < FOOTER_SIZE: + return None + try: + with open(path, "rb") as f: + f.seek(size - FOOTER_SIZE) + trailer = f.read(FOOTER_SIZE) + except OSError: + return None + if len(trailer) != FOOTER_SIZE or trailer[24:32] != FOOTER_MAGIC: + return None + source_len, metadata_len, footer_ver = struct.unpack_from(" dict[str, Any] | None: + """ + Return the parsed ``airflow-metadata.yaml`` manifest embedded in *path*. + + Returns ``None`` when *path* is not a bundle, when the metadata bytes + are not valid UTF-8 YAML, or when the manifest does not deserialise to + a mapping. + """ + try: + footer = _read_footer(path) + except ValueError: + return None + if footer is None: + return None + metadata_start = path.stat().st_size - FOOTER_SIZE - footer.metadata_len + with open(path, "rb") as f: + f.seek(metadata_start) + metadata_bytes = f.read(footer.metadata_len) + try: + data = yaml.safe_load(metadata_bytes.decode("utf-8")) + except (UnicodeDecodeError, yaml.YAMLError): + return None + if not isinstance(data, dict): + return None + return data + + +def read_source_code(path: Path) -> str | None: + """ + Return the embedded DAG source from a bundle, decoded as UTF-8. + + Returns ``None`` when *path* is not a bundle or carries an empty source + region (``source_len == 0``). + """ + try: + footer = _read_footer(path) + except ValueError: + return None + if footer is None or footer.source_len == 0: + return None + source_start = path.stat().st_size - FOOTER_SIZE - footer.metadata_len - footer.source_len + with open(path, "rb") as f: + f.seek(source_start) + source_bytes = f.read(footer.source_len) + try: + return source_bytes.decode("utf-8") + except UnicodeDecodeError: + return None + + +def _dag_ids(metadata: dict[str, Any]) -> set[str]: + dags = metadata.get("dags") + if not isinstance(dags, dict): + return set() + return set(dags.keys()) + + +class BundleScanner: + """ + Locate Airflow native executable bundles inside a directory. + + The scanner enumerates every regular, executable file in *bundles_dir*, + reads the last 32 bytes of each, and treats files whose magic matches + ``AFBNDL01`` as bundles. Non-bundle files are silently ignored. + """ + + def __init__(self, bundles_dir: Path) -> None: + self._bundles_dir = bundles_dir + + def resolve(self, dag_id: str) -> str: + """ + Return the executable path of the bundle whose manifest declares *dag_id*. + + :raises FileNotFoundError: if no matching bundle is found. + """ + for candidate in self._candidate_files(): + metadata = read_bundle_metadata(candidate) + if metadata is None: + continue + if dag_id in _dag_ids(metadata): + return str(candidate.resolve()) + + raise FileNotFoundError( + f"No executable bundle containing dag_id={dag_id!r} found in {self._bundles_dir}" + ) + + @staticmethod + def resolve_executable(path: Path) -> str | None: + """ + Validate that *path* is an Airflow executable bundle. + + Returns the resolved executable path when *path* is a regular, + executable file whose trailer matches ``AFBNDL01`` and whose + embedded manifest declares at least one DAG; ``None`` otherwise. + """ + resolved = path.resolve() + if not resolved.is_file() or not os.access(resolved, os.X_OK): + return None + metadata = read_bundle_metadata(resolved) + if metadata is None: + return None + if not _dag_ids(metadata): + return None + return str(resolved) + + def _candidate_files(self) -> list[Path]: + if not self._bundles_dir.is_dir(): + return [] + return sorted(p for p in self._bundles_dir.iterdir() if p.is_file() and os.access(p, os.X_OK)) diff --git a/providers/sdk/executable/src/airflow/providers/sdk/executable/coordinator.py b/providers/sdk/executable/src/airflow/providers/sdk/executable/coordinator.py new file mode 100644 index 0000000000000..f1826262ccf9e --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/sdk/executable/coordinator.py @@ -0,0 +1,135 @@ +# +# 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. +"""Native executable coordinator for DAG file processing and task execution.""" + +from __future__ import annotations + +import os +from pathlib import Path +from typing import TYPE_CHECKING + +from structlog import get_logger + +from airflow.providers.sdk.executable.bundle_scanner import BundleScanner, read_source_code +from airflow.sdk.execution_time.coordinator import BaseCoordinator + +if TYPE_CHECKING: + from airflow.sdk.api.datamodels._generated import BundleInfo, TaskInstance + +log = get_logger(__name__) + + +class ExecutableCoordinator(BaseCoordinator): + """Coordinator that launches a native executable subprocess for DAG parsing and task execution.""" + + sdk = "executable" + file_extension = "" + + @classmethod + def can_handle_dag_file(cls, bundle_name: str, path: str | os.PathLike[str]) -> bool: + """ + Return ``True`` when *path* is a self-contained executable bundle. + + Detection is by the ``AFBNDL01`` trailer magic appended by the SDK + packer; non-bundle files are silently rejected. + """ + try: + return BundleScanner.resolve_executable(Path(path)) is not None + except OSError: + return False + + @classmethod + def get_code_from_file(cls, fileloc: str) -> str: + """Read the DAG source embedded in the bundle's footer.""" + code = read_source_code(Path(fileloc)) + if code is None: + raise FileNotFoundError(f"No source code found for executable: {fileloc}") + return code + + @classmethod + def dag_parsing_cmd( + cls, + *, + dag_file_path: str, + bundle_name: str, + bundle_path: str, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """Build the subprocess command for parsing a native executable bundle.""" + return [ + dag_file_path, + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] + + @classmethod + def task_execution_cmd( + cls, + *, + what: TaskInstance, + dag_file_path: str, + bundle_path: str, + bundle_info: BundleInfo, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """Build the subprocess command for executing a task in a native executable bundle.""" + # Case 1: Pure executable DAG — dag_file_path is itself a bundle. + # A ``.py`` extension always indicates a Python stub, so short-circuit + # there before validating the AFBNDL01 trailer. Both checks are needed: + # ``os.access(_, X_OK)`` alone is not reliable because bind-mounted + # Python stubs can satisfy X_OK (e.g. when running as root inside + # Breeze) yet are not real bundles. + log.debug( + "Resolving executable for task execution", + what=what, + dag_file_path=dag_file_path, + bundle_path=bundle_path, + bundle_info=bundle_info, + comm_addr=comm_addr, + logs_addr=logs_addr, + ) + if ( + not dag_file_path.endswith(".py") + and (resolved := BundleScanner.resolve_executable(Path(dag_file_path))) is not None + ): + return [ + resolved, + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] + + # Case 2: Python Stub DAG — the dag_file_path is a Python file but + # the task delegates to a native runtime. The actual binary lives + # in the provider's configured ``[executable] bundles_folder``. + from airflow.providers.common.compat.sdk import conf + + bundles_folder = conf.get("executable", "bundles_folder", fallback=None) + if not bundles_folder: + raise ValueError( + "The [executable] bundles_folder config must be set for Python stub DAGs " + "that delegate to native executable task execution." + ) + + executable_path = BundleScanner(Path(bundles_folder)).resolve(dag_id=what.dag_id) + return [ + executable_path, + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] diff --git a/providers/sdk/executable/src/airflow/providers/sdk/executable/get_provider_info.py b/providers/sdk/executable/src/airflow/providers/sdk/executable/get_provider_info.py new file mode 100644 index 0000000000000..c2f7738bca818 --- /dev/null +++ b/providers/sdk/executable/src/airflow/providers/sdk/executable/get_provider_info.py @@ -0,0 +1,51 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! +# +# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE +# `get_provider_info_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY + + +def get_provider_info(): + return { + "package-name": "apache-airflow-providers-sdk-executable", + "name": "SDK: Executable", + "description": "Native executable language support for Apache Airflow runtime coordinators.\nSupports any compiled binary (Go, Rust, etc.) that implements the Airflow\nSDK coordinator protocol (--comm/--logs socket-based IPC).\n", + "integrations": [ + { + "integration-name": "Native Executable", + "external-doc-url": "https://airflow.apache.org/", + "tags": ["software"], + } + ], + "config": { + "executable": { + "description": "Options for the native executable language provider.", + "options": { + "bundles_folder": { + "description": "Path to the directory containing native executable DAG bundles.\nWhen using Python stub DAGs that delegate task execution to a\nnative runtime, the coordinator scans this directory for files\nwhose AFBNDL01 trailer marks them as Airflow executable bundles\nand selects the one whose embedded manifest declares the target\ndag_id.\n", + "type": "string", + "version_added": None, + "example": "~/airflow/executable-bundles", + "default": "", + } + }, + } + }, + "coordinators": ["airflow.providers.sdk.executable.coordinator.ExecutableCoordinator"], + } diff --git a/providers/sdk/executable/tests/conftest.py b/providers/sdk/executable/tests/conftest.py new file mode 100644 index 0000000000000..f56ccce0a3f69 --- /dev/null +++ b/providers/sdk/executable/tests/conftest.py @@ -0,0 +1,19 @@ +# 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 + +pytest_plugins = "tests_common.pytest_plugin" diff --git a/providers/sdk/executable/tests/unit/__init__.py b/providers/sdk/executable/tests/unit/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/executable/tests/unit/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/executable/tests/unit/sdk/__init__.py b/providers/sdk/executable/tests/unit/sdk/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/executable/tests/unit/sdk/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/executable/tests/unit/sdk/executable/__init__.py b/providers/sdk/executable/tests/unit/sdk/executable/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/providers/sdk/executable/tests/unit/sdk/executable/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/providers/sdk/executable/tests/unit/sdk/executable/test_executable_provider.py b/providers/sdk/executable/tests/unit/sdk/executable/test_executable_provider.py new file mode 100644 index 0000000000000..af262e2a505bb --- /dev/null +++ b/providers/sdk/executable/tests/unit/sdk/executable/test_executable_provider.py @@ -0,0 +1,449 @@ +# +# 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 stat +import struct +from pathlib import Path +from unittest.mock import MagicMock, patch + +import pytest +import yaml + +from airflow.providers.sdk.executable.bundle_scanner import ( + FOOTER_MAGIC, + FOOTER_SIZE, + BundleScanner, + read_bundle_metadata, + read_source_code, +) +from airflow.providers.sdk.executable.coordinator import ExecutableCoordinator +from airflow.providers.sdk.executable.get_provider_info import get_provider_info + + +def test_get_provider_info_exposes_executable_runtime_components(): + info = get_provider_info() + assert info["package-name"] == "apache-airflow-providers-sdk-executable" + assert info["name"] == "SDK: Executable" + assert info["coordinators"] == [ + "airflow.providers.sdk.executable.coordinator.ExecutableCoordinator", + ] + assert info["integrations"] == [ + { + "integration-name": "Native Executable", + "external-doc-url": "https://airflow.apache.org/", + "tags": ["software"], + } + ] + assert "bundles_folder" in info["config"]["executable"]["options"] + + +def test_executable_provider_entrypoints_are_importable(): + assert ExecutableCoordinator.sdk == "executable" + assert ExecutableCoordinator.file_extension == "" + + +_DEFAULT_BINARY_PAYLOAD = b"\x7fELF" + b"binary-stub-payload" + + +def _make_metadata(dag_ids, source_filename: str = "example.go") -> dict: + return { + "format_version": "1.0", + "sdk": {"language": "go", "version": "0.1.0"}, + "source": source_filename, + "dags": {dag_id: {"tasks": ["task1"]} for dag_id in dag_ids}, + } + + +def _build_bundle( + path: Path, + *, + dag_ids=("tutorial_dag",), + source: str | bytes = "package main\n\nfunc main() {}\n", + source_filename: str = "example.go", + metadata: dict | bytes | None = None, + binary_bytes: bytes = _DEFAULT_BINARY_PAYLOAD, + footer_ver: int = 1, + magic: bytes = FOOTER_MAGIC, + reserved: bytes = b"\x00" * 12, +) -> Path: + """Write a self-contained bundle at *path* and return it. + + The binary region is a short opaque stub; tests only care that it has + non-zero length so the trailer's bounds-check passes. + """ + if isinstance(source, str): + source_bytes = source.encode("utf-8") + else: + source_bytes = source + + if metadata is None: + metadata_dict = _make_metadata(dag_ids, source_filename=source_filename) + metadata_bytes = yaml.safe_dump(metadata_dict, sort_keys=True).encode("utf-8") + elif isinstance(metadata, (bytes, bytearray)): + metadata_bytes = bytes(metadata) + else: + metadata_bytes = yaml.safe_dump(metadata, sort_keys=True).encode("utf-8") + + if len(reserved) != 12: + raise ValueError("reserved must be exactly 12 bytes") + trailer = struct.pack(" Path: + """Create a non-bundle executable file (no AFBNDL01 trailer).""" + path.write_bytes(b"#!/bin/sh\nexit 0\n") + path.chmod(path.stat().st_mode | stat.S_IEXEC | stat.S_IXGRP | stat.S_IXOTH) + return path + + +class TestReadBundleMetadata: + def test_parses_embedded_manifest(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag", "other_dag"]) + + metadata = read_bundle_metadata(binary) + assert metadata is not None + assert metadata["sdk"] == {"language": "go", "version": "0.1.0"} + assert set(metadata["dags"].keys()) == {"tutorial_dag", "other_dag"} + + def test_non_bundle_file_returns_none(self, tmp_path): + regular = _make_executable(tmp_path / "not_a_bundle") + assert read_bundle_metadata(regular) is None + + def test_short_file_returns_none(self, tmp_path): + short = tmp_path / "tiny" + short.write_bytes(b"hi") + assert read_bundle_metadata(short) is None + + def test_unknown_footer_version_returns_none(self, tmp_path): + binary = _build_bundle(tmp_path / "future_bundle", footer_ver=99) + assert read_bundle_metadata(binary) is None + + def test_corrupted_yaml_returns_none(self, tmp_path): + binary = _build_bundle(tmp_path / "broken", metadata=b"::: not: yaml: [") + assert read_bundle_metadata(binary) is None + + +class TestReadSourceCode: + def test_returns_embedded_source(self, tmp_path): + binary = _build_bundle( + tmp_path / "my_bundle", + source='package main\n\nfunc main() { println("hi") }\n', + ) + + assert read_source_code(binary) == 'package main\n\nfunc main() { println("hi") }\n' + + def test_empty_source_region_returns_none(self, tmp_path): + binary = _build_bundle(tmp_path / "no_source", source="") + assert read_source_code(binary) is None + + def test_non_bundle_file_returns_none(self, tmp_path): + regular = _make_executable(tmp_path / "not_a_bundle") + assert read_source_code(regular) is None + + def test_invalid_utf8_source_returns_none(self, tmp_path): + binary = _build_bundle(tmp_path / "binary_source", source=b"\xff\xfe\x00\x00not utf-8") + assert read_source_code(binary) is None + + +class TestCanHandleDagFile: + def test_valid_bundle_with_dag(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag"]) + + assert ExecutableCoordinator.can_handle_dag_file("test_bundle", str(binary)) is True + + def test_executable_without_footer(self, tmp_path): + regular = _make_executable(tmp_path / "my_bundle") + + assert ExecutableCoordinator.can_handle_dag_file("test_bundle", str(regular)) is False + + def test_bundle_with_empty_dags(self, tmp_path): + binary = _build_bundle(tmp_path / "empty_bundle", dag_ids=[]) + + assert ExecutableCoordinator.can_handle_dag_file("test_bundle", str(binary)) is False + + def test_nonexistent_path(self): + assert ExecutableCoordinator.can_handle_dag_file("test_bundle", "/nonexistent/path") is False + + def test_python_file_not_handled(self, tmp_path): + py_file = tmp_path / "my_dag.py" + py_file.write_text("# a python dag") + + assert ExecutableCoordinator.can_handle_dag_file("test_bundle", str(py_file)) is False + + +class TestDagParsingCmd: + def test_builds_correct_command(self): + cmd = ExecutableCoordinator.dag_parsing_cmd( + dag_file_path="/path/to/my_bundle", + bundle_name="test_bundle", + bundle_path="/path/to", + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + assert cmd == [ + "/path/to/my_bundle", + "--comm=127.0.0.1:12345", + "--logs=127.0.0.1:12346", + ] + + +class TestTaskExecutionCmd: + def test_pure_executable_dag(self, tmp_path): + """When dag_file_path points directly to the bundle binary.""" + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag"]) + + what = MagicMock(spec=["dag_id"]) + what.dag_id = "tutorial_dag" + + bundle_info = MagicMock(spec=["name", "version"]) + + cmd = ExecutableCoordinator.task_execution_cmd( + what=what, + dag_file_path=str(binary), + bundle_path=str(tmp_path), + bundle_info=bundle_info, + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + assert cmd == [ + str(binary.resolve()), + "--comm=127.0.0.1:12345", + "--logs=127.0.0.1:12346", + ] + + def test_python_stub_with_exec_bit_set_falls_through_to_bundles_folder(self, tmp_path): + """Python stub with X_OK set (e.g. bind-mounted in Breeze running as root) + must still be recognised as a stub and resolve via bundles_folder. + + Regression test for the EACCES seen when running stub-mode DAGs from a + directory where files inherit the bind-mount's executable bits. + """ + bundles_dir = tmp_path / "bundles" + bundles_dir.mkdir() + binary = _build_bundle(bundles_dir / "my_bundle", dag_ids=["tutorial_dag"]) + + what = MagicMock(spec=["dag_id"]) + what.dag_id = "tutorial_dag" + + bundle_info = MagicMock(spec=["name", "version"]) + + py_file = tmp_path / "stub_dag.py" + py_file.write_text("# stub dag") + py_file.chmod(py_file.stat().st_mode | stat.S_IEXEC | stat.S_IXGRP | stat.S_IXOTH) + + with patch("airflow.providers.common.compat.sdk.conf") as mock_conf: + mock_conf.get.return_value = str(bundles_dir) + + cmd = ExecutableCoordinator.task_execution_cmd( + what=what, + dag_file_path=str(py_file), + bundle_path=str(tmp_path), + bundle_info=bundle_info, + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + + assert cmd[0] == str(binary.resolve()) + assert cmd[0] != str(py_file) + + def test_py_extension_short_circuits_even_if_trailer_matches(self, tmp_path): + """A ``.py`` path is always treated as a stub, even if its bytes happen + to satisfy the AFBNDL01 trailer check. Defensive against a + pathologically-crafted Python file that could otherwise hit Case 1. + """ + bundles_dir = tmp_path / "bundles" + bundles_dir.mkdir() + real_binary = _build_bundle(bundles_dir / "my_bundle", dag_ids=["tutorial_dag"]) + + # Build a file with the AFBNDL01 trailer but a .py name. Even with the + # trailer present, the .py extension forces the stub path. + rogue_py = _build_bundle(tmp_path / "stub_dag.py", dag_ids=["tutorial_dag"]) + + what = MagicMock(spec=["dag_id"]) + what.dag_id = "tutorial_dag" + + bundle_info = MagicMock(spec=["name", "version"]) + + with patch("airflow.providers.common.compat.sdk.conf") as mock_conf: + mock_conf.get.return_value = str(bundles_dir) + + cmd = ExecutableCoordinator.task_execution_cmd( + what=what, + dag_file_path=str(rogue_py), + bundle_path=str(tmp_path), + bundle_info=bundle_info, + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + + assert cmd[0] == str(real_binary.resolve()) + assert cmd[0] != str(rogue_py) + + def test_python_stub_dag_with_bundles_folder(self, tmp_path): + """When dag_file_path is a .py file, resolve from the configured bundles_folder.""" + bundles_dir = tmp_path / "bundles" + bundles_dir.mkdir() + binary = _build_bundle(bundles_dir / "my_bundle", dag_ids=["tutorial_dag"]) + + what = MagicMock(spec=["dag_id"]) + what.dag_id = "tutorial_dag" + + bundle_info = MagicMock(spec=["name", "version"]) + + py_file = tmp_path / "stub_dag.py" + py_file.write_text("# stub dag") + + with patch("airflow.providers.common.compat.sdk.conf") as mock_conf: + mock_conf.get.return_value = str(bundles_dir) + + cmd = ExecutableCoordinator.task_execution_cmd( + what=what, + dag_file_path=str(py_file), + bundle_path=str(tmp_path), + bundle_info=bundle_info, + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + + assert cmd == [ + str(binary.resolve()), + "--comm=127.0.0.1:12345", + "--logs=127.0.0.1:12346", + ] + + def test_python_stub_dag_without_bundles_folder_raises(self, tmp_path): + """When dag_file_path is not executable and no bundles_folder configured.""" + py_file = tmp_path / "stub_dag.py" + py_file.write_text("# stub dag") + + what = MagicMock(spec=["dag_id"]) + what.dag_id = "tutorial_dag" + + bundle_info = MagicMock(spec=["name", "version"]) + + with patch("airflow.providers.common.compat.sdk.conf") as mock_conf: + mock_conf.get.return_value = None + + with pytest.raises(ValueError, match="bundles_folder config must be set"): + ExecutableCoordinator.task_execution_cmd( + what=what, + dag_file_path=str(py_file), + bundle_path=str(tmp_path), + bundle_info=bundle_info, + comm_addr="127.0.0.1:12345", + logs_addr="127.0.0.1:12346", + ) + + +class TestBundleScanner: + def test_resolve_finds_matching_dag_id(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag", "other_dag"]) + + scanner = BundleScanner(tmp_path) + assert scanner.resolve("tutorial_dag") == str(binary.resolve()) + + def test_resolve_picks_matching_bundle_among_many(self, tmp_path): + _build_bundle(tmp_path / "alpha", dag_ids=["alpha_dag"]) + beta = _build_bundle(tmp_path / "beta", dag_ids=["beta_dag"]) + _build_bundle(tmp_path / "gamma", dag_ids=["gamma_dag"]) + + scanner = BundleScanner(tmp_path) + assert scanner.resolve("beta_dag") == str(beta.resolve()) + + def test_resolve_skips_non_bundle_files(self, tmp_path): + (tmp_path / "README.md").write_text("not a bundle") + _make_executable(tmp_path / "stray_executable") + binary = _build_bundle(tmp_path / "real_bundle", dag_ids=["tutorial_dag"]) + + scanner = BundleScanner(tmp_path) + assert scanner.resolve("tutorial_dag") == str(binary.resolve()) + + def test_resolve_skips_non_executable_files(self, tmp_path): + # A bundle file without the executable bit cannot be exec'd, so the + # scanner must skip it even if its trailer would otherwise match. + non_exec = _build_bundle(tmp_path / "non_exec", dag_ids=["tutorial_dag"]) + non_exec.chmod(non_exec.stat().st_mode & ~(stat.S_IEXEC | stat.S_IXGRP | stat.S_IXOTH)) + + scanner = BundleScanner(tmp_path) + with pytest.raises(FileNotFoundError, match="No executable bundle"): + scanner.resolve("tutorial_dag") + + def test_resolve_raises_when_not_found(self, tmp_path): + scanner = BundleScanner(tmp_path) + with pytest.raises(FileNotFoundError, match="No executable bundle"): + scanner.resolve("nonexistent_dag") + + def test_resolve_raises_when_directory_missing(self, tmp_path): + scanner = BundleScanner(tmp_path / "does_not_exist") + with pytest.raises(FileNotFoundError, match="No executable bundle"): + scanner.resolve("tutorial_dag") + + def test_resolve_executable_valid(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag"]) + + result = BundleScanner.resolve_executable(binary) + assert result == str(binary.resolve()) + + def test_resolve_executable_not_a_bundle(self, tmp_path): + regular = _make_executable(tmp_path / "my_bundle") + + assert BundleScanner.resolve_executable(regular) is None + + def test_resolve_executable_empty_dags(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=[]) + + assert BundleScanner.resolve_executable(binary) is None + + def test_resolve_executable_non_executable_returns_none(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", dag_ids=["tutorial_dag"]) + binary.chmod(binary.stat().st_mode & ~(stat.S_IEXEC | stat.S_IXGRP | stat.S_IXOTH)) + + assert BundleScanner.resolve_executable(binary) is None + + def test_resolve_executable_directory_returns_none(self, tmp_path): + bundle_dir = tmp_path / "my_bundle" + bundle_dir.mkdir() + + assert BundleScanner.resolve_executable(bundle_dir) is None + + +class TestGetCodeFromFile: + def test_reads_embedded_source(self, tmp_path): + source = "package main\n\nfunc main() {}\n" + binary = _build_bundle(tmp_path / "my_bundle", source=source) + + result = ExecutableCoordinator.get_code_from_file(str(binary)) + assert result == source + + def test_no_source_raises(self, tmp_path): + binary = _build_bundle(tmp_path / "my_bundle", source="") + + with pytest.raises(FileNotFoundError, match="No source code found"): + ExecutableCoordinator.get_code_from_file(str(binary)) + + def test_non_bundle_raises(self, tmp_path): + regular = _make_executable(tmp_path / "not_a_bundle") + + with pytest.raises(FileNotFoundError, match="No source code found"): + ExecutableCoordinator.get_code_from_file(str(regular)) diff --git a/providers/sdk/java/.gitignore b/providers/sdk/java/.gitignore new file mode 100644 index 0000000000000..bff2d7629604d --- /dev/null +++ b/providers/sdk/java/.gitignore @@ -0,0 +1 @@ +*.iml diff --git a/providers/sdk/java/LICENSE b/providers/sdk/java/LICENSE new file mode 100644 index 0000000000000..11069edd79019 --- /dev/null +++ b/providers/sdk/java/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +Licensed 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. diff --git a/providers/sdk/java/NOTICE b/providers/sdk/java/NOTICE new file mode 100644 index 0000000000000..a51bd9390d030 --- /dev/null +++ b/providers/sdk/java/NOTICE @@ -0,0 +1,5 @@ +Apache Airflow +Copyright 2016-2026 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/providers/sdk/java/README.rst b/providers/sdk/java/README.rst new file mode 100644 index 0000000000000..ba3081bb6cb53 --- /dev/null +++ b/providers/sdk/java/README.rst @@ -0,0 +1,60 @@ + +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! + +.. IF YOU WANT TO MODIFY TEMPLATE FOR THIS FILE, YOU SHOULD MODIFY THE TEMPLATE + ``PROVIDER_README_TEMPLATE.rst.jinja2`` IN the ``dev/breeze/src/airflow_breeze/templates`` DIRECTORY + +Package ``apache-airflow-providers-sdk-java`` + +Release: ``0.1.0`` + + +Java Coordinator + + +Provider package +---------------- + +This is a provider package for ``sdk.java`` provider. All classes for this provider package +are in ``airflow.providers.sdk.java`` python package. + +You can find package information and changelog for the provider +in the `documentation `_. + +Installation +------------ + +You can install this package on top of an existing Airflow installation (see ``Requirements`` below +for the minimum Airflow version supported) via +``pip install apache-airflow-providers-sdk-java`` + +The package supports the following python versions: 3.10,3.11,3.12,3.13,3.14 + +Requirements +------------ + +================== ================== +PIP package Version required +================== ================== +``apache-airflow`` ``>=3.0.0`` +================== ================== + +The changelog for the provider package can be found in the +`changelog `_. diff --git a/providers/sdk/java/docs/.latest-doc-only-change.txt b/providers/sdk/java/docs/.latest-doc-only-change.txt new file mode 100644 index 0000000000000..2c1ab461a9c8e --- /dev/null +++ b/providers/sdk/java/docs/.latest-doc-only-change.txt @@ -0,0 +1 @@ +da9caffdbbeab1917e1cec5726e50af5f14a5206 diff --git a/providers/sdk/java/docs/changelog.rst b/providers/sdk/java/docs/changelog.rst new file mode 100644 index 0000000000000..c5aa1ad337ef8 --- /dev/null +++ b/providers/sdk/java/docs/changelog.rst @@ -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. + + +.. NOTE TO CONTRIBUTORS: + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes + and you want to add an explanation to the users on how they are supposed to deal with them. + The changelog is updated and maintained semi-automatically by release manager. + + +``apache-airflow-providers-sdk-java`` + + +Changelog +--------- + +0.1.0 +..... + +Features +~~~~~~~~ + +* ``Add the initial Java coordinator interface`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): diff --git a/providers/sdk/java/docs/commits.rst b/providers/sdk/java/docs/commits.rst new file mode 100644 index 0000000000000..6b84d751e94e3 --- /dev/null +++ b/providers/sdk/java/docs/commits.rst @@ -0,0 +1,35 @@ + + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! + + .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE + `PROVIDER_COMMITS_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY + + .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN! + +Package apache-airflow-providers-sdk-java +------------------------------------------------------ + +Java Coordinator + + +This is detailed commit list of changes for versions provider package: ``sdk.java``. +For high-level changelog, see :doc:`package information including changelog `. + +.. airflow-providers-commits:: diff --git a/providers/sdk/java/docs/conf.py b/providers/sdk/java/docs/conf.py new file mode 100644 index 0000000000000..596c5b5c7b5f3 --- /dev/null +++ b/providers/sdk/java/docs/conf.py @@ -0,0 +1,27 @@ +# Disable Flake8 because of all the sphinx imports +# +# 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. +"""Configuration of Providers docs building.""" + +from __future__ import annotations + +import os + +os.environ["AIRFLOW_PACKAGE_NAME"] = "apache-airflow-providers-sdk-java" + +from docs.provider_conf import * # noqa: F403 diff --git a/providers/sdk/java/docs/index.rst b/providers/sdk/java/docs/index.rst new file mode 100644 index 0000000000000..7d4854e09b430 --- /dev/null +++ b/providers/sdk/java/docs/index.rst @@ -0,0 +1,116 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +``apache-airflow-providers-sdk-java`` +=========================================== + +The SDK: Java provider registers Java-specific task coordinator and DAG file processor classes for Apache Airflow. + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Basics + + Home + Changelog + Security + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: References + + Python API <_api/airflow/providers/sdk/java/index> + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Resources + + PyPI Repository + Installing from sources + +.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Commits + + Detailed list of commits + + +apache-airflow-providers-sdk-java package +------------------------------------------------------ + +Java SDK support for Apache Airflow runtime coordinators. + + +Release: 0.1.0 + +Provider package +---------------- + +This package is for the ``sdk.java`` provider. +All classes for this package are included in the ``airflow.providers.sdk.java`` python package. + +Installation +------------ + +You can install this package on top of an existing Airflow installation via +``pip install apache-airflow-providers-sdk-java``. +For the minimum Airflow version supported, see ``Requirements`` below. + +Requirements +------------ + +The minimum Apache Airflow version supported by this provider distribution is ``3.3.0``. + +================== ================== +PIP package Version required +================== ================== +``apache-airflow`` ``>=3.3.0`` +================== ================== + +Cross provider package dependencies +----------------------------------- + +Those are dependencies that might be needed in order to use all the features of the package. +You need to install the specified provider distributions in order to use them. + +You can install such cross-provider dependencies when installing from PyPI. For example: + +.. code-block:: bash + + pip install apache-airflow-providers-sdk-java[common.compat] + + +================================================================================================================== ================= +Dependent package Extra +================================================================================================================== ================= +`apache-airflow-providers-common-compat `_ ``common.compat`` +================================================================================================================== ================= + +Downloading official packages +----------------------------- + +You can download officially released packages and verify their checksums and signatures from the +`Official Apache Download site `_ + +* `The apache-airflow-providers-sdk-java 0.1.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-sdk-java 0.1.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/providers/sdk/java/docs/installing-providers-from-sources.rst b/providers/sdk/java/docs/installing-providers-from-sources.rst new file mode 100644 index 0000000000000..fdbb17d017579 --- /dev/null +++ b/providers/sdk/java/docs/installing-providers-from-sources.rst @@ -0,0 +1,18 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. include:: /../../../../devel-common/src/sphinx_exts/includes/installing-providers-from-sources.rst diff --git a/providers/sdk/java/docs/security.rst b/providers/sdk/java/docs/security.rst new file mode 100644 index 0000000000000..351ff007ebf2f --- /dev/null +++ b/providers/sdk/java/docs/security.rst @@ -0,0 +1,18 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. include:: /../../../../devel-common/src/sphinx_exts/includes/security.rst diff --git a/providers/sdk/java/provider.yaml b/providers/sdk/java/provider.yaml new file mode 100644 index 0000000000000..d10f841962034 --- /dev/null +++ b/providers/sdk/java/provider.yaml @@ -0,0 +1,58 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +--- +package-name: apache-airflow-providers-sdk-java +name: "SDK: Java" +description: | + Java SDK support for Apache Airflow runtime coordinators. + +state: ready +lifecycle: incubation +source-date-epoch: 1775631151 +# Note that those versions are maintained by release manager - do not update them manually +# with the exception of case where other provider in sources has >= new provider version. +# In such case adding >= NEW_VERSION and bumping to NEW_VERSION in a provider have +# to be done in the same PR +versions: + - 0.1.0 + +integrations: + - integration-name: Java + external-doc-url: https://openjdk.org/ + tags: + - software + +config: + java: + description: "Options for the Java SDK provider." + options: + bundles_folder: + description: | + Path to the directory containing Java DAG bundle JARs. + When using Python stub DAGs that delegate task execution to Java, + the coordinator scans this directory to find the JAR bundle matching + the target dag_id. Each immediate subdirectory is treated as a + separate bundle home, and the directory itself is also checked + (flat layout). + type: string + version_added: ~ + example: ~/airflow/java-bundles + default: "" + +coordinators: + - airflow.providers.sdk.java.coordinator.JavaCoordinator diff --git a/providers/sdk/java/pyproject.toml b/providers/sdk/java/pyproject.toml new file mode 100644 index 0000000000000..6baca6f81fdd4 --- /dev/null +++ b/providers/sdk/java/pyproject.toml @@ -0,0 +1,131 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! + +# IF YOU WANT TO MODIFY THIS FILE EXCEPT DEPENDENCIES, YOU SHOULD MODIFY THE TEMPLATE +# `pyproject_TEMPLATE.toml.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY +[build-system] +requires = ["flit_core==3.12.0"] +build-backend = "flit_core.buildapi" + +[project] +name = "apache-airflow-providers-sdk-java" +version = "0.1.0" +description = "Provider package apache-airflow-providers-sdk-java for Apache Airflow" +readme = "README.rst" +license = "Apache-2.0" +license-files = ['LICENSE', 'NOTICE'] +authors = [ + {name="Apache Software Foundation", email="dev@airflow.apache.org"}, +] +maintainers = [ + {name="Apache Software Foundation", email="dev@airflow.apache.org"}, +] +keywords = [ "airflow-provider", "sdk.java", "airflow", "integration" ] +classifiers = [ + "Development Status :: 5 - Production/Stable", + "Environment :: Console", + "Environment :: Web Environment", + "Intended Audience :: Developers", + "Intended Audience :: System Administrators", + "Framework :: Apache Airflow", + "Framework :: Apache Airflow :: Provider", + "Programming Language :: Python :: 3.10", + "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: 3.12", + "Programming Language :: Python :: 3.13", + "Programming Language :: Python :: 3.14", + "Topic :: System :: Monitoring", +] +requires-python = ">=3.10" + +# The dependencies should be modified in place in the generated file. +# Any change in the dependencies is preserved when the file is regenerated +# Make sure to run ``prek update-providers-dependencies --all-files`` +# After you modify the dependencies, and rebuild your Breeze CI image with ``breeze ci-image build`` +dependencies = [ + "apache-airflow>=3.3.0", +] + +# The optional dependencies should be modified in place in the generated file +# Any change in the dependencies is preserved when the file is regenerated +[project.optional-dependencies] +"common.compat" = [ + "apache-airflow-providers-common-compat" +] + +[dependency-groups] +dev = [ + "apache-airflow", + "apache-airflow-task-sdk", + "apache-airflow-devel-common", + "apache-airflow-providers-common-compat", + # Additional devel dependencies (do not remove this line and add extra development dependencies) +] + +# To build docs: +# +# uv run --group docs build-docs +# +# To enable auto-refreshing build with server: +# +# uv run --group docs build-docs --autobuild +# +# To see more options: +# +# uv run --group docs build-docs --help +# +docs = [ + "apache-airflow-devel-common[docs]" +] + +[tool.uv.sources] +# These names must match the names as defined in the pyproject.toml of the workspace items, +# *not* the workspace folder paths +apache-airflow = {workspace = true} +apache-airflow-devel-common = {workspace = true} +apache-airflow-task-sdk = {workspace = true} +apache-airflow-providers-common-sql = {workspace = true} +apache-airflow-providers-standard = {workspace = true} + +[project.urls] +"Documentation" = "https://airflow.apache.org/docs/apache-airflow-providers-sdk-java/0.1.0" +"Changelog" = "https://airflow.apache.org/docs/apache-airflow-providers-sdk-java/0.1.0/changelog.html" +"Bug Tracker" = "https://github.com/apache/airflow/issues" +"Source Code" = "https://github.com/apache/airflow" +"Slack Chat" = "https://s.apache.org/airflow-slack" +"Mastodon" = "https://fosstodon.org/@airflow" +"YouTube" = "https://www.youtube.com/channel/UCSXwxpWZQ7XZ1WL3wqevChA/" + +[project.entry-points."apache_airflow_provider"] +provider_info = "airflow.providers.sdk.java.get_provider_info:get_provider_info" + +[tool.flit.module] +name = "airflow.providers.sdk.java" + +# Explicit sdist contents so the build does not rely on VCS information +# (flit 4.0 makes --no-use-vcs the default — see https://github.com/pypa/flit/pull/782). +[tool.flit.sdist] +include = [ + "docs/", + "provider.yaml", + "src/airflow/__init__.py", + "src/airflow/providers/__init__.py", + "src/airflow/providers/sdk/__init__.py", + "tests/", +] diff --git a/providers/sdk/java/src/airflow/__init__.py b/providers/sdk/java/src/airflow/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/java/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/java/src/airflow/providers/__init__.py b/providers/sdk/java/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/java/src/airflow/providers/sdk/__init__.py b/providers/sdk/java/src/airflow/providers/sdk/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/sdk/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/java/src/airflow/providers/sdk/java/__init__.py b/providers/sdk/java/src/airflow/providers/sdk/java/__init__.py new file mode 100644 index 0000000000000..1c942bc68df44 --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/sdk/java/__init__.py @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE +# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. +# +# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE +# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY +# +from __future__ import annotations + +import packaging.version + +from airflow import __version__ as airflow_version + +__all__ = ["__version__"] + +__version__ = "0.1.0" + +if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( + "3.3.0" +): + raise RuntimeError( + f"The package `apache-airflow-providers-sdk-java:{__version__}` needs Apache Airflow 3.3.0+" + ) diff --git a/providers/sdk/java/src/airflow/providers/sdk/java/bundle_scanner.py b/providers/sdk/java/src/airflow/providers/sdk/java/bundle_scanner.py new file mode 100644 index 0000000000000..87bbf518b8e5b --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/sdk/java/bundle_scanner.py @@ -0,0 +1,220 @@ +# +# 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. +""" +Scan directories for Airflow Java SDK bundle JARs. + +Mirrors the Java SDK's ``BundleScanner`` — checks each JAR's manifest for +``Airflow-Java-SDK-Metadata``, reads the embedded metadata YAML, and +resolves the main class and classpath needed to launch the bundle process. +""" + +from __future__ import annotations + +import email +import os +import zipfile +from pathlib import Path +from typing import NamedTuple + +import yaml + +MANIFEST_PATH = "META-INF/MANIFEST.MF" +METADATA_MANIFEST_KEY = "Airflow-Java-SDK-Metadata" +SDK_VERSION_MANIFEST_KEY = "Airflow-Java-SDK-Version" +DAG_CODE_MANIFEST_KEY = "Airflow-Java-SDK-Dag-Code" +MAIN_CLASS_MANIFEST_KEY = "Main-Class" + + +class ResolvedJarBundle(NamedTuple): + """A resolved Java DAG bundle: everything needed to start the bundle process.""" + + main_class: str + classpath: str + + +class BundleScanner: + """ + Locate Airflow Java SDK bundles inside a directory tree. + + Supports two directory layouts: + + - **Nested** - each immediate subdirectory of *bundles_dir* is a bundle home. + - **Flat** — *bundles_dir* itself contains the bundle JARs. + + Within a bundle home the JVM convention of a ``lib/`` subdirectory for + dependency JARs is respected automatically. + """ + + def __init__(self, bundles_dir: Path) -> None: + self._bundles_dir = bundles_dir + + def resolve(self, dag_id: str) -> ResolvedJarBundle: + """ + Find the bundle whose metadata YAML lists *dag_id*. + + :raises FileNotFoundError: if no matching bundle is found. + """ + for bundle_home in self._candidate_homes(): + jars = _jar_files(bundle_home) + if not jars: + continue + + for jar_path in jars: + result = _read_bundle_jar(jar_path) + if result is None: + continue + main_class, dag_ids = result + if dag_id in dag_ids: + classpath = os.pathsep.join(str(j.resolve()) for j in jars) + return ResolvedJarBundle(main_class=main_class, classpath=classpath) + + raise FileNotFoundError(f"No JAR bundle containing dag_id={dag_id!r} found in {self._bundles_dir}") + + @staticmethod + def resolve_jar(jar_path: Path) -> str: + """ + Read ``Main-Class`` from a single bundle JAR, validating SDK attributes. + + :raises FileNotFoundError: if the JAR is not a valid Airflow Java SDK bundle. + """ + result = _read_bundle_jar(jar_path) + if result is None: + raise FileNotFoundError( + f"Not a valid Airflow Java SDK bundle: {jar_path} " + f"(requires {METADATA_MANIFEST_KEY} and {MAIN_CLASS_MANIFEST_KEY})" + ) + return result[0] + + def _candidate_homes(self) -> list[Path]: + """Return normalised bundle-home directories to inspect.""" + candidates: list[Path] = [] + + # Each subdirectory is a potential bundle home (nested layout). + if self._bundles_dir.is_dir(): + for child in sorted(self._bundles_dir.iterdir()): + if child.is_dir(): + candidates.append(_normalize_bundle_home(child)) + + # The directory itself (flat layout). + candidates.append(_normalize_bundle_home(self._bundles_dir)) + return candidates + + +def _jar_files(directory: Path) -> list[Path]: + """List all ``.jar`` files in *directory*, sorted by name.""" + if not directory.is_dir(): + return [] + return sorted(p for p in directory.iterdir() if p.is_file() and p.suffix == ".jar") + + +def _normalize_bundle_home(path: Path) -> Path: + """ + Normalize a bundle path to the directory containing JARs. + + Handles the common JVM distribution layout where dependency JARs + live in a ``lib/`` subdirectory (Gradle ``application`` plugin, + Maven Assembly, sbt-native-packager, etc.). + + - If *path* points to a JAR file, use its parent directory. + - If the directory has a ``lib/`` subdirectory containing JARs, use that. + - Otherwise, return the directory as-is. + """ + normalized = path.resolve() + if normalized.is_file() and normalized.suffix == ".jar": + return normalized.parent + lib = normalized / "lib" + if lib.is_dir() and any(p.suffix == ".jar" for p in lib.iterdir()): + return lib + return normalized + + +def _read_bundle_jar(jar_path: Path) -> tuple[str, set[str]] | None: + """ + Read ``Main-Class`` and dag IDs from a JAR's manifest and embedded metadata. + + Returns ``(main_class, dag_ids)`` when the JAR carries valid + ``Airflow-Java-SDK-Metadata`` and ``Main-Class`` manifest attributes + and the referenced metadata YAML contains at least one dag ID. + Returns ``None`` otherwise. + """ + try: + with zipfile.ZipFile(jar_path) as zf: + try: + with zf.open(MANIFEST_PATH) as f: + manifest = email.message_from_binary_file(f) + except KeyError: + return None + + metadata_file = manifest.get(METADATA_MANIFEST_KEY) + if not metadata_file: + return None + + main_class = manifest.get(MAIN_CLASS_MANIFEST_KEY) + if not main_class: + return None + + try: + with zf.open(metadata_file) as f: + content = f.read().decode() + except KeyError: + return None + except zipfile.BadZipFile: + return None + + dag_ids = _parse_dag_ids_from_metadata(content) + if not dag_ids: + return None + + return main_class, dag_ids + + +def read_dag_code(jar_path: Path) -> str | None: + """ + Read the DAG source code embedded in a JAR bundle. + + Returns the source code string when the JAR carries a valid + ``Airflow-Java-SDK-Dag-Code`` manifest attribute pointing to an + embedded source file. Returns ``None`` otherwise. + """ + try: + with zipfile.ZipFile(jar_path) as zf: + try: + with zf.open(MANIFEST_PATH) as f: + manifest = email.message_from_binary_file(f) + except KeyError: + return None + + dag_code_path = manifest.get(DAG_CODE_MANIFEST_KEY) + if not dag_code_path: + return None + + try: + with zf.open(dag_code_path) as f: + return f.read().decode() + except KeyError: + return None + except zipfile.BadZipFile: + return None + + +def _parse_dag_ids_from_metadata(yaml_content: str) -> set[str]: + """Parse dag IDs from an ``airflow-metadata.yaml`` content string.""" + data = yaml.safe_load(yaml_content) + if not isinstance(data, dict) or "dags" not in data: + return set() + return set(data["dags"].keys()) diff --git a/providers/sdk/java/src/airflow/providers/sdk/java/coordinator.py b/providers/sdk/java/src/airflow/providers/sdk/java/coordinator.py new file mode 100644 index 0000000000000..e5f31b29f7575 --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/sdk/java/coordinator.py @@ -0,0 +1,131 @@ +# +# 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. +"""Java runtime coordinator that launches a JVM subprocess for Dag file processing and task execution.""" + +from __future__ import annotations + +import contextlib +import os +import zipfile +from pathlib import Path +from typing import TYPE_CHECKING + +from airflow.providers.sdk.java.bundle_scanner import BundleScanner, read_dag_code +from airflow.sdk.execution_time.coordinator import BaseCoordinator + +if TYPE_CHECKING: + from airflow.sdk._shared.workloads import TaskInstanceDTO + from airflow.sdk.api.datamodels._generated import BundleInfo + + +class JavaCoordinator(BaseCoordinator): + """Coordinator that launches a JVM subprocess for DAG parsing and task execution.""" + + sdk = "java" + file_extension = ".jar" + + @classmethod + def can_handle_dag_file(cls, bundle_name: str, path: str | os.PathLike[str]) -> bool: + """Return ``True`` when *path* is a JAR with valid Airflow Java SDK manifest attributes.""" + if not os.fspath(path).endswith(cls.file_extension): + return False + with contextlib.suppress(FileNotFoundError, NotADirectoryError, zipfile.BadZipFile, KeyError): + return BundleScanner.resolve_jar(Path(path)) is not None + return False + + @classmethod + def get_code_from_file(cls, fileloc: str) -> str: + """Read embedded DAG source code from a JAR bundle.""" + code = read_dag_code(Path(fileloc)) + if code is None: + raise FileNotFoundError(f"No DAG source code found in JAR: {fileloc}") + return code + + @classmethod + def dag_parsing_cmd( + cls, + *, + dag_file_path: str, + bundle_name: str, + bundle_path: str, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """Build the ``java`` command for parsing a JAR bundle.""" + jar_path = Path(dag_file_path) + # Java bundles are typically thin JARs: the main JAR only contains + # the bundle's own classes while its dependencies (the Airflow Java + # SDK, logging libraries, etc.) are separate JARs that live alongside + # it. Using ``/*`` lets the JVM load every JAR in the directory. + classpath = f"{bundle_path}/*" + return [ + "java", + "-classpath", + classpath, + BundleScanner.resolve_jar(jar_path), + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] + + @classmethod + def task_execution_cmd( + cls, + *, + what: TaskInstanceDTO, + dag_file_path: str, + bundle_path: str, + bundle_info: BundleInfo, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """Build the ``java`` command for executing a task in a JAR bundle.""" + if dag_file_path.endswith(".jar"): + # Case 1: Pure Java Dag — the dag_file_path points directly to a + # bundle JAR inside the Airflow Core Dag Bundle. + jar_path = Path(dag_file_path) + classpath = f"{bundle_path}/*" + return [ + "java", + "-classpath", + classpath, + BundleScanner.resolve_jar(jar_path), + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] + + # Case 2: Python Stub Dag — the dag_file_path is a Python file but + # the task delegates to a Java runtime. The actual JAR bundle lives + # in the provider's configured ``[java] bundles_folder``. + from airflow.providers.common.compat.sdk import conf + + bundles_folder = conf.get("java", "bundles_folder", fallback=None) + if not bundles_folder: + raise ValueError( + "The [java] bundles_folder config must be set for Python stub DAGs " + "that delegate to Java task execution." + ) + + resolved = BundleScanner(Path(bundles_folder)).resolve(dag_id=what.dag_id) + return [ + "java", + "-classpath", + resolved.classpath, + resolved.main_class, + f"--comm={comm_addr}", + f"--logs={logs_addr}", + ] diff --git a/providers/sdk/java/src/airflow/providers/sdk/java/get_provider_info.py b/providers/sdk/java/src/airflow/providers/sdk/java/get_provider_info.py new file mode 100644 index 0000000000000..89df45102b732 --- /dev/null +++ b/providers/sdk/java/src/airflow/providers/sdk/java/get_provider_info.py @@ -0,0 +1,47 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! +# +# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE +# `get_provider_info_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY + + +def get_provider_info(): + return { + "package-name": "apache-airflow-providers-sdk-java", + "name": "SDK: Java", + "description": "Java SDK support for Apache Airflow runtime coordinators.\n", + "integrations": [ + {"integration-name": "Java", "external-doc-url": "https://openjdk.org/", "tags": ["software"]} + ], + "config": { + "java": { + "description": "Options for the Java SDK provider.", + "options": { + "bundles_folder": { + "description": "Path to the directory containing Java DAG bundle JARs.\nWhen using Python stub DAGs that delegate task execution to Java,\nthe coordinator scans this directory to find the JAR bundle matching\nthe target dag_id. Each immediate subdirectory is treated as a\nseparate bundle home, and the directory itself is also checked\n(flat layout).\n", + "type": "string", + "version_added": None, + "example": "~/airflow/java-bundles", + "default": "", + } + }, + } + }, + "coordinators": ["airflow.providers.sdk.java.coordinator.JavaCoordinator"], + } diff --git a/providers/sdk/java/tests/conftest.py b/providers/sdk/java/tests/conftest.py new file mode 100644 index 0000000000000..f56ccce0a3f69 --- /dev/null +++ b/providers/sdk/java/tests/conftest.py @@ -0,0 +1,19 @@ +# 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 + +pytest_plugins = "tests_common.pytest_plugin" diff --git a/providers/sdk/java/tests/unit/__init__.py b/providers/sdk/java/tests/unit/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/java/tests/unit/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/java/tests/unit/sdk/__init__.py b/providers/sdk/java/tests/unit/sdk/__init__.py new file mode 100644 index 0000000000000..5966d6b1d5261 --- /dev/null +++ b/providers/sdk/java/tests/unit/sdk/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) diff --git a/providers/sdk/java/tests/unit/sdk/java/__init__.py b/providers/sdk/java/tests/unit/sdk/java/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/providers/sdk/java/tests/unit/sdk/java/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/providers/sdk/java/tests/unit/sdk/java/test_bundle_scanner.py b/providers/sdk/java/tests/unit/sdk/java/test_bundle_scanner.py new file mode 100644 index 0000000000000..5c042036143da --- /dev/null +++ b/providers/sdk/java/tests/unit/sdk/java/test_bundle_scanner.py @@ -0,0 +1,337 @@ +# +# 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 +import zipfile +from pathlib import Path + +import pytest +import yaml + +from airflow.providers.sdk.java.bundle_scanner import ( + DAG_CODE_MANIFEST_KEY, + MAIN_CLASS_MANIFEST_KEY, + MANIFEST_PATH, + METADATA_MANIFEST_KEY, + SDK_VERSION_MANIFEST_KEY, + BundleScanner, + ResolvedJarBundle, + _jar_files, + _normalize_bundle_home, + _parse_dag_ids_from_metadata, + _read_bundle_jar, + read_dag_code, +) + +METADATA_YAML_PATH = "META-INF/airflow-metadata.yaml" +DAG_CODE_PATH = "JavaExample.java" +TEST_MAIN_CLASS = "com.example.MyDag" +TEST_SDK_VERSION = "1.0.0" + + +def _make_manifest( + *, + main_class: str | None = TEST_MAIN_CLASS, + metadata_path: str | None = METADATA_YAML_PATH, + sdk_version: str | None = TEST_SDK_VERSION, + dag_code_path: str | None = None, +) -> str: + lines = ["Manifest-Version: 1.0"] + if main_class: + lines.append(f"{MAIN_CLASS_MANIFEST_KEY}: {main_class}") + if metadata_path: + lines.append(f"{METADATA_MANIFEST_KEY}: {metadata_path}") + if sdk_version: + lines.append(f"{SDK_VERSION_MANIFEST_KEY}: {sdk_version}") + if dag_code_path: + lines.append(f"{DAG_CODE_MANIFEST_KEY}: {dag_code_path}") + return "\n".join(lines) + "\n" + + +def _make_metadata_yaml(dag_ids: list[str]) -> str: + return yaml.dump({"dags": {dag_id: {} for dag_id in dag_ids}}) + + +def _create_bundle_jar( + jar_path: Path, + *, + dag_ids: list[str] | None = None, + main_class: str | None = TEST_MAIN_CLASS, + include_metadata: bool = True, + include_manifest: bool = True, + dag_code: str | None = None, +) -> Path: + """Create a minimal JAR (zip) file with Airflow Java SDK manifest attributes.""" + with zipfile.ZipFile(jar_path, "w") as zf: + if include_manifest: + dag_code_path = DAG_CODE_PATH if dag_code else None + manifest = _make_manifest( + main_class=main_class, + metadata_path=METADATA_YAML_PATH if include_metadata else None, + dag_code_path=dag_code_path, + ) + zf.writestr(MANIFEST_PATH, manifest) + + if include_metadata and dag_ids is not None: + zf.writestr(METADATA_YAML_PATH, _make_metadata_yaml(dag_ids)) + + if dag_code: + zf.writestr(DAG_CODE_PATH, dag_code) + return jar_path + + +class TestJarFiles: + def test_lists_jar_files_sorted(self, tmp_path: Path): + (tmp_path / "b.jar").touch() + (tmp_path / "a.jar").touch() + (tmp_path / "c.txt").touch() + result = _jar_files(tmp_path) + assert result == [tmp_path / "a.jar", tmp_path / "b.jar"] + + def test_returns_empty_for_nonexistent_directory(self, tmp_path: Path): + assert _jar_files(tmp_path / "nonexistent") == [] + + def test_returns_empty_for_directory_with_no_jars(self, tmp_path: Path): + (tmp_path / "readme.txt").touch() + assert _jar_files(tmp_path) == [] + + def test_ignores_jar_directories(self, tmp_path: Path): + (tmp_path / "fake.jar").mkdir() + assert _jar_files(tmp_path) == [] + + +class TestNormalizeBundleHome: + def test_jar_file_returns_parent(self, tmp_path: Path): + jar = tmp_path / "bundle.jar" + jar.touch() + assert _normalize_bundle_home(jar) == tmp_path.resolve() + + def test_dir_with_lib_containing_jars(self, tmp_path: Path): + lib = tmp_path / "lib" + lib.mkdir() + (lib / "dep.jar").touch() + assert _normalize_bundle_home(tmp_path) == lib.resolve() + + def test_dir_with_empty_lib(self, tmp_path: Path): + lib = tmp_path / "lib" + lib.mkdir() + assert _normalize_bundle_home(tmp_path) == tmp_path.resolve() + + def test_plain_directory(self, tmp_path: Path): + assert _normalize_bundle_home(tmp_path) == tmp_path.resolve() + + +class TestParseDagIdsFromMetadata: + def test_parses_dag_ids(self): + content = yaml.dump({"dags": {"dag_a": {}, "dag_b": {"key": "val"}}}) + assert _parse_dag_ids_from_metadata(content) == {"dag_a", "dag_b"} + + @pytest.mark.parametrize( + "yaml_content", + [ + pytest.param(yaml.dump({"other": 1}), id="missing_dags_key"), + pytest.param("just a string", id="non_dict"), + pytest.param(yaml.dump({"dags": {}}), id="empty_dags"), + ], + ) + def test_returns_empty_set(self, yaml_content): + assert _parse_dag_ids_from_metadata(yaml_content) == set() + + +class TestReadBundleJar: + def test_valid_jar(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "valid.jar", dag_ids=["my_dag"]) + result = _read_bundle_jar(jar) + assert result is not None + main_class, dag_ids = result + assert main_class == TEST_MAIN_CLASS + assert dag_ids == {"my_dag"} + + def test_returns_none_for_missing_manifest(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_manifest.jar", include_manifest=False) + assert _read_bundle_jar(jar) is None + + def test_returns_none_for_missing_metadata_key(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_meta.jar", include_metadata=False) + assert _read_bundle_jar(jar) is None + + def test_returns_none_for_missing_main_class(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_main.jar", dag_ids=["d"], main_class=None) + assert _read_bundle_jar(jar) is None + + def test_returns_none_for_missing_metadata_file(self, tmp_path: Path): + """Manifest references a metadata file that does not exist inside the JAR.""" + jar = tmp_path / "missing_meta_file.jar" + with zipfile.ZipFile(jar, "w") as zf: + manifest = _make_manifest(metadata_path="nonexistent.yaml") + zf.writestr(MANIFEST_PATH, manifest) + assert _read_bundle_jar(jar) is None + + def test_returns_none_for_bad_zip(self, tmp_path: Path): + bad = tmp_path / "bad.jar" + bad.write_text("not a zip file") + assert _read_bundle_jar(bad) is None + + def test_returns_none_for_empty_dag_ids(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "empty_dags.jar", dag_ids=[]) + assert _read_bundle_jar(jar) is None + + def test_multiple_dag_ids(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "multi.jar", dag_ids=["dag_1", "dag_2", "dag_3"]) + result = _read_bundle_jar(jar) + assert result is not None + _, dag_ids = result + assert dag_ids == {"dag_1", "dag_2", "dag_3"} + + +class TestReadDagCode: + def test_reads_embedded_dag_code(self, tmp_path: Path): + code = "public class MyDag {}" + jar = _create_bundle_jar(tmp_path / "with_code.jar", dag_ids=["d"], dag_code=code) + assert read_dag_code(jar) == code + + def test_returns_none_for_missing_dag_code_key(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_code.jar", dag_ids=["d"]) + assert read_dag_code(jar) is None + + def test_returns_none_for_missing_manifest(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_manifest.jar", include_manifest=False) + assert read_dag_code(jar) is None + + def test_returns_none_for_bad_zip(self, tmp_path: Path): + bad = tmp_path / "bad.jar" + bad.write_text("not a zip") + assert read_dag_code(bad) is None + + def test_returns_none_when_code_file_missing(self, tmp_path: Path): + """Manifest references a dag code file that does not exist inside the JAR.""" + jar = tmp_path / "broken_code.jar" + with zipfile.ZipFile(jar, "w") as zf: + manifest = _make_manifest(dag_code_path="missing_source.py") + zf.writestr(MANIFEST_PATH, manifest) + assert read_dag_code(jar) is None + + +class TestBundleScannerResolveJar: + def test_returns_main_class(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "bundle.jar", dag_ids=["d"]) + assert BundleScanner.resolve_jar(jar) == TEST_MAIN_CLASS + + def test_raises_for_invalid_jar(self, tmp_path: Path): + jar = tmp_path / "not_bundle.jar" + jar.write_text("not a zip") + with pytest.raises(FileNotFoundError, match="Not a valid Airflow Java SDK bundle"): + BundleScanner.resolve_jar(jar) + + +class TestBundleScannerCandidateHomes: + def test_nested_layout(self, tmp_path: Path): + sub_a = tmp_path / "bundle_a" + sub_a.mkdir() + (sub_a / "app.jar").touch() + + sub_b = tmp_path / "bundle_b" + sub_b.mkdir() + (sub_b / "app.jar").touch() + + scanner = BundleScanner(tmp_path) + homes = scanner._candidate_homes() + # Nested subdirs + the bundles_dir itself + assert len(homes) == 3 + assert sub_a.resolve() in homes + assert sub_b.resolve() in homes + assert tmp_path.resolve() in homes + + def test_flat_layout(self, tmp_path: Path): + (tmp_path / "app.jar").touch() + scanner = BundleScanner(tmp_path) + homes = scanner._candidate_homes() + # Only the directory itself (no subdirectories) + assert homes == [tmp_path.resolve()] + + def test_nested_with_lib_subdir(self, tmp_path: Path): + sub = tmp_path / "my_bundle" + sub.mkdir() + lib = sub / "lib" + lib.mkdir() + (lib / "dep.jar").touch() + + scanner = BundleScanner(tmp_path) + homes = scanner._candidate_homes() + # _normalize_bundle_home should redirect to lib/ + assert lib.resolve() in homes + + +class TestBundleScannerResolve: + def test_finds_matching_dag(self, tmp_path: Path): + bundle_dir = tmp_path / "my_bundle" + bundle_dir.mkdir() + _create_bundle_jar(bundle_dir / "app.jar", dag_ids=["target_dag"]) + + scanner = BundleScanner(tmp_path) + result = scanner.resolve("target_dag") + assert isinstance(result, ResolvedJarBundle) + assert result.main_class == TEST_MAIN_CLASS + assert str((bundle_dir / "app.jar").resolve()) in result.classpath + + def test_raises_when_no_match(self, tmp_path: Path): + bundle_dir = tmp_path / "my_bundle" + bundle_dir.mkdir() + _create_bundle_jar(bundle_dir / "app.jar", dag_ids=["other_dag"]) + + scanner = BundleScanner(tmp_path) + with pytest.raises(FileNotFoundError, match="No JAR bundle containing dag_id='missing'"): + scanner.resolve("missing") + + def test_classpath_includes_all_jars(self, tmp_path: Path): + bundle_dir = tmp_path / "my_bundle" + bundle_dir.mkdir() + _create_bundle_jar(bundle_dir / "app.jar", dag_ids=["my_dag"]) + # Create a dependency JAR (no SDK metadata, just a plain JAR) + with zipfile.ZipFile(bundle_dir / "dep.jar", "w") as zf: + zf.writestr("dummy.class", b"") + + scanner = BundleScanner(tmp_path) + result = scanner.resolve("my_dag") + parts = result.classpath.split(os.pathsep) + assert len(parts) == 2 + + def test_flat_layout_resolve(self, tmp_path: Path): + _create_bundle_jar(tmp_path / "app.jar", dag_ids=["flat_dag"]) + + scanner = BundleScanner(tmp_path) + result = scanner.resolve("flat_dag") + assert result.main_class == TEST_MAIN_CLASS + + def test_skips_non_bundle_jars(self, tmp_path: Path): + bundle_dir = tmp_path / "my_bundle" + bundle_dir.mkdir() + # Non-bundle JAR (no manifest) + with zipfile.ZipFile(bundle_dir / "plain.jar", "w") as zf: + zf.writestr("dummy.class", b"") + _create_bundle_jar(bundle_dir / "real.jar", dag_ids=["real_dag"]) + + scanner = BundleScanner(tmp_path) + result = scanner.resolve("real_dag") + assert result.main_class == TEST_MAIN_CLASS + + def test_empty_bundles_dir(self, tmp_path: Path): + scanner = BundleScanner(tmp_path) + with pytest.raises(FileNotFoundError): + scanner.resolve("any_dag") diff --git a/providers/sdk/java/tests/unit/sdk/java/test_coordinator.py b/providers/sdk/java/tests/unit/sdk/java/test_coordinator.py new file mode 100644 index 0000000000000..2c42574dd3ed0 --- /dev/null +++ b/providers/sdk/java/tests/unit/sdk/java/test_coordinator.py @@ -0,0 +1,242 @@ +# +# 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 uuid +import zipfile +from pathlib import Path +from unittest.mock import patch + +import pytest +import yaml + +from airflow._shared.workloads import TaskInstanceDTO +from airflow.providers.sdk.java.bundle_scanner import ( + MAIN_CLASS_MANIFEST_KEY, + MANIFEST_PATH, + METADATA_MANIFEST_KEY, + SDK_VERSION_MANIFEST_KEY, +) +from airflow.providers.sdk.java.coordinator import JavaCoordinator +from airflow.sdk.api.datamodels._generated import BundleInfo + +from tests_common.test_utils.version_compat import AIRFLOW_V_3_3_PLUS + +if not AIRFLOW_V_3_3_PLUS: + pytest.skip("Coordinator is only compatible with Airflow >= 3.3.0", allow_module_level=True) + +METADATA_YAML_PATH = "META-INF/airflow-metadata.yaml" +DAG_CODE_PATH = "dag_source.py" +TEST_MAIN_CLASS = "com.example.MyBundle" + + +def _make_manifest( + *, + main_class: str | None = TEST_MAIN_CLASS, + metadata_path: str | None = METADATA_YAML_PATH, + dag_code_path: str | None = None, +) -> str: + lines = ["Manifest-Version: 1.0"] + if main_class: + lines.append(f"{MAIN_CLASS_MANIFEST_KEY}: {main_class}") + if metadata_path: + lines.append(f"{METADATA_MANIFEST_KEY}: {metadata_path}") + lines.append(f"{SDK_VERSION_MANIFEST_KEY}: 1.0.0") + if dag_code_path: + lines.append(f"Airflow-Java-SDK-Dag-Code: {dag_code_path}") + return "\n".join(lines) + "\n" + + +def _create_bundle_jar( + jar_path: Path, + *, + dag_ids: list[str] | None = None, + dag_code: str | None = None, +) -> Path: + with zipfile.ZipFile(jar_path, "w") as zf: + dag_code_path = DAG_CODE_PATH if dag_code else None + manifest = _make_manifest(dag_code_path=dag_code_path) + zf.writestr(MANIFEST_PATH, manifest) + if dag_ids is not None: + metadata = yaml.dump({"dags": {d: {} for d in dag_ids}}) + zf.writestr(METADATA_YAML_PATH, metadata) + if dag_code: + zf.writestr(DAG_CODE_PATH, dag_code) + return jar_path + + +def _make_ti(dag_id: str = "test_dag") -> TaskInstanceDTO: + return TaskInstanceDTO( + id=uuid.uuid4(), + dag_version_id=uuid.uuid4(), + task_id="task_1", + dag_id=dag_id, + run_id="run_1", + try_number=1, + map_index=-1, + pool_slots=1, + queue="default", + priority_weight=1, + ) + + +class TestJavaCoordinatorAttributes: + def test_sdk(self): + assert JavaCoordinator.sdk == "java" + + def test_file_extension(self): + assert JavaCoordinator.file_extension == ".jar" + + +class TestCanHandleDagFile: + def test_valid_jar_returns_true(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "valid.jar", dag_ids=["d"]) + assert JavaCoordinator.can_handle_dag_file("bundle", str(jar)) is True + + def test_non_jar_file_returns_false(self, tmp_path: Path): + py_file = tmp_path / "dag.py" + py_file.write_text("from airflow import DAG") + assert JavaCoordinator.can_handle_dag_file("bundle", str(py_file)) is False + + def test_missing_file_returns_false(self, tmp_path: Path): + assert JavaCoordinator.can_handle_dag_file("bundle", str(tmp_path / "missing.jar")) is False + + def test_bad_zip_returns_false(self, tmp_path: Path): + bad = tmp_path / "bad.jar" + bad.write_text("not a zip") + assert JavaCoordinator.can_handle_dag_file("bundle", str(bad)) is False + + def test_jar_without_sdk_manifest_returns_false(self, tmp_path: Path): + jar = tmp_path / "plain.jar" + with zipfile.ZipFile(jar, "w") as zf: + zf.writestr("dummy.class", b"") + assert JavaCoordinator.can_handle_dag_file("bundle", str(jar)) is False + + +class TestGetCodeFromFile: + def test_returns_embedded_code(self, tmp_path: Path): + code = "from airflow import DAG\ndag = DAG('my_dag')" + jar = _create_bundle_jar(tmp_path / "with_code.jar", dag_ids=["d"], dag_code=code) + assert JavaCoordinator.get_code_from_file(str(jar)) == code + + def test_raises_when_no_code(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "no_code.jar", dag_ids=["d"]) + with pytest.raises(FileNotFoundError, match="No DAG source code found in JAR"): + JavaCoordinator.get_code_from_file(str(jar)) + + +class TestDagParsingCmd: + def test_builds_java_command(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "app.jar", dag_ids=["d"]) + bundle_path = str(tmp_path) + cmd = JavaCoordinator.dag_parsing_cmd( + dag_file_path=str(jar), + bundle_name="my_bundle", + bundle_path=bundle_path, + comm_addr="localhost:1234", + logs_addr="localhost:5678", + ) + assert cmd == [ + "java", + "-classpath", + f"{bundle_path}/*", + TEST_MAIN_CLASS, + "--comm=localhost:1234", + "--logs=localhost:5678", + ] + + +class TestTaskExecutionCmd: + def test_pure_java_dag(self, tmp_path: Path): + jar = _create_bundle_jar(tmp_path / "app.jar", dag_ids=["test_dag"]) + bundle_path = str(tmp_path) + ti = _make_ti() + bundle_info = BundleInfo(name="my_bundle") + + cmd = JavaCoordinator.task_execution_cmd( + what=ti, + dag_file_path=str(jar), + bundle_path=bundle_path, + bundle_info=bundle_info, + comm_addr="localhost:1234", + logs_addr="localhost:5678", + ) + assert cmd == [ + "java", + "-classpath", + f"{bundle_path}/*", + TEST_MAIN_CLASS, + "--comm=localhost:1234", + "--logs=localhost:5678", + ] + + def test_python_stub_dag_with_bundles_folder(self, tmp_path: Path): + bundles_folder = tmp_path / "java_bundles" + bundle_sub = bundles_folder / "my_bundle" + bundle_sub.mkdir(parents=True) + _create_bundle_jar(bundle_sub / "app.jar", dag_ids=["stub_dag"]) + + ti = _make_ti(dag_id="stub_dag") + bundle_info = BundleInfo(name="my_bundle") + + with patch( + "airflow.providers.common.compat.sdk.conf.get", + return_value=str(bundles_folder), + ): + cmd = JavaCoordinator.task_execution_cmd( + what=ti, + dag_file_path="/dags/stub_dag.py", + bundle_path="/some/bundle/path", + bundle_info=bundle_info, + comm_addr="localhost:1234", + logs_addr="localhost:5678", + ) + + assert cmd == [ + "java", + "-classpath", + f"{bundles_folder}/my_bundle/app.jar", + TEST_MAIN_CLASS, + "--comm=localhost:1234", + "--logs=localhost:5678", + ] + + @pytest.mark.parametrize( + "config_value", + [ + pytest.param(None, id="none"), + pytest.param("", id="empty_string"), + ], + ) + def test_python_stub_dag_invalid_config_raises(self, config_value): + ti = _make_ti() + bundle_info = BundleInfo(name="my_bundle") + + with patch( + "airflow.providers.common.compat.sdk.conf.get", + return_value=config_value, + ): + with pytest.raises(ValueError, match="bundles_folder config must be set"): + JavaCoordinator.task_execution_cmd( + what=ti, + dag_file_path="/dags/stub_dag.py", + bundle_path="/some/bundle/path", + bundle_info=bundle_info, + comm_addr="localhost:1234", + logs_addr="localhost:5678", + ) diff --git a/providers/sdk/java/tests/unit/sdk/java/test_java_provider.py b/providers/sdk/java/tests/unit/sdk/java/test_java_provider.py new file mode 100644 index 0000000000000..e0489ada7cc17 --- /dev/null +++ b/providers/sdk/java/tests/unit/sdk/java/test_java_provider.py @@ -0,0 +1,58 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import pytest + +from airflow.providers.sdk.java.coordinator import JavaCoordinator +from airflow.providers.sdk.java.get_provider_info import get_provider_info + +from tests_common.test_utils.version_compat import AIRFLOW_V_3_3_PLUS + +if not AIRFLOW_V_3_3_PLUS: + pytest.skip("Coordinator is only compatible with Airflow >= 3.3.0", allow_module_level=True) + + +def test_get_provider_info_exposes_java_runtime_components(): + assert get_provider_info() == { + "package-name": "apache-airflow-providers-sdk-java", + "name": "SDK: Java", + "description": "Java SDK support for Apache Airflow runtime coordinators.\n", + "integrations": [ + {"integration-name": "Java", "external-doc-url": "https://openjdk.org/", "tags": ["software"]} + ], + "config": { + "java": { + "description": "Options for the Java SDK provider.", + "options": { + "bundles_folder": { + "description": "Path to the directory containing Java DAG bundle JARs.\nWhen using Python stub DAGs that delegate task execution to Java,\nthe coordinator scans this directory to find the JAR bundle matching\nthe target dag_id. Each immediate subdirectory is treated as a\nseparate bundle home, and the directory itself is also checked\n(flat layout).\n", + "type": "string", + "version_added": None, + "example": "~/airflow/java-bundles", + "default": "", + } + }, + } + }, + "coordinators": ["airflow.providers.sdk.java.coordinator.JavaCoordinator"], + } + + +def test_java_provider_entrypoints_are_importable(): + assert JavaCoordinator.sdk == "java" diff --git a/providers/standard/src/airflow/providers/standard/decorators/stub.py b/providers/standard/src/airflow/providers/standard/decorators/stub.py index f29d123c740c1..a5e63d925f795 100644 --- a/providers/standard/src/airflow/providers/standard/decorators/stub.py +++ b/providers/standard/src/airflow/providers/standard/decorators/stub.py @@ -85,7 +85,6 @@ def stub( Stub tasks exist in the Dag graph only, but the execution must happen in an external environment via the Task Execution Interface. - """ return task_decorator_factory( decorated_operator_class=_StubOperator, diff --git a/pyproject.toml b/pyproject.toml index 4a988db2d3ea7..3a25fc450976e 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -338,6 +338,12 @@ apache-airflow = "airflow.__main__:main" "samba" = [ "apache-airflow-providers-samba>=4.9.0" ] +"sdk.executable" = [ + "apache-airflow-providers-sdk-executable>=0.1.0" +] +"sdk.java" = [ + "apache-airflow-providers-sdk-java>=0.1.0" +] "segment" = [ "apache-airflow-providers-segment>=3.7.0" ] @@ -481,6 +487,8 @@ apache-airflow = "airflow.__main__:main" "apache-airflow-providers-redis>=4.0.0", "apache-airflow-providers-salesforce>=5.9.0", "apache-airflow-providers-samba>=4.9.0", + "apache-airflow-providers-sdk-executable>=0.1.0", + "apache-airflow-providers-sdk-java>=0.1.0", "apache-airflow-providers-segment>=3.7.0", "apache-airflow-providers-sendgrid>=4.0.0", "apache-airflow-providers-sftp>=5.0.0", @@ -748,6 +756,7 @@ testing = ["dev", "providers.tests", "tests_common", "tests", "system", "unit", "providers/cncf/*/src/airflow/providers/cncf/__init__.py" = ["I002"] "providers/dbt/*/src/airflow/providers/dbt/__init__.py" = ["I002"] "providers/microsoft/*/src/airflow/providers/microsoft/__init__.py" = ["I002"] +"providers/sdk/*/src/airflow/providers/sdk/__init__.py" = ["I002"] # The test_python.py is needed because adding __future__.annotations breaks runtime checks that are # needed for the test to work @@ -1218,6 +1227,10 @@ mypy_path = [ "$MYPY_CONFIG_FILE_DIR/providers/salesforce/tests", "$MYPY_CONFIG_FILE_DIR/providers/samba/src", "$MYPY_CONFIG_FILE_DIR/providers/samba/tests", + "$MYPY_CONFIG_FILE_DIR/providers/sdk/executable/src", + "$MYPY_CONFIG_FILE_DIR/providers/sdk/executable/tests", + "$MYPY_CONFIG_FILE_DIR/providers/sdk/java/src", + "$MYPY_CONFIG_FILE_DIR/providers/sdk/java/tests", "$MYPY_CONFIG_FILE_DIR/providers/segment/src", "$MYPY_CONFIG_FILE_DIR/providers/segment/tests", "$MYPY_CONFIG_FILE_DIR/providers/sendgrid/src", @@ -1329,6 +1342,7 @@ dev = [ "apache-airflow-shared-state", "apache-airflow-shared-template-rendering", "apache-airflow-shared-timezones", + "apache-airflow-shared-workloads", ] # To build docs: @@ -1461,6 +1475,8 @@ apache-airflow-providers-qdrant = false apache-airflow-providers-redis = false apache-airflow-providers-salesforce = false apache-airflow-providers-samba = false +apache-airflow-providers-sdk-executable = false +apache-airflow-providers-sdk-java = false apache-airflow-providers-segment = false apache-airflow-providers-sendgrid = false apache-airflow-providers-sftp = false @@ -1496,6 +1512,7 @@ apache-airflow-shared-serialization = false apache-airflow-shared-state = false apache-airflow-shared-template-rendering = false apache-airflow-shared-timezones = false +apache-airflow-shared-workloads = false apache-airflow-task-sdk = false apache-airflow-task-sdk-integration-tests = false apache-aurflow-docker-stack = false @@ -1612,6 +1629,8 @@ apache-airflow-providers-qdrant = false apache-airflow-providers-redis = false apache-airflow-providers-salesforce = false apache-airflow-providers-samba = false +apache-airflow-providers-sdk-executable = false +apache-airflow-providers-sdk-java = false apache-airflow-providers-segment = false apache-airflow-providers-sendgrid = false apache-airflow-providers-sftp = false @@ -1647,6 +1666,7 @@ apache-airflow-shared-serialization = false apache-airflow-shared-state = false apache-airflow-shared-template-rendering = false apache-airflow-shared-timezones = false +apache-airflow-shared-workloads = false apache-airflow-task-sdk = false apache-airflow-task-sdk-integration-tests = false apache-aurflow-docker-stack = false @@ -1692,6 +1712,7 @@ apache-airflow-shared-serialization = { workspace = true } apache-airflow-shared-state = { workspace = true } apache-airflow-shared-template-rendering = { workspace = true } apache-airflow-shared-timezones = { workspace = true } +apache-airflow-shared-workloads = { workspace = true } # Automatically generated provider workspace items (update_airflow_pyproject_toml.py) apache-airflow-providers-airbyte = { workspace = true } apache-airflow-providers-akeyless = { workspace = true } @@ -1773,6 +1794,8 @@ apache-airflow-providers-qdrant = { workspace = true } apache-airflow-providers-redis = { workspace = true } apache-airflow-providers-salesforce = { workspace = true } apache-airflow-providers-samba = { workspace = true } +apache-airflow-providers-sdk-executable = { workspace = true } +apache-airflow-providers-sdk-java = { workspace = true } apache-airflow-providers-segment = { workspace = true } apache-airflow-providers-sendgrid = { workspace = true } apache-airflow-providers-sftp = { workspace = true } @@ -1829,6 +1852,7 @@ members = [ "shared/state", "shared/template_rendering", "shared/timezones", + "shared/workloads", # Automatically generated provider workspace members (update_airflow_pyproject_toml.py) "providers/airbyte", "providers/akeyless", @@ -1910,6 +1934,8 @@ members = [ "providers/redis", "providers/salesforce", "providers/samba", + "providers/sdk/executable", + "providers/sdk/java", "providers/segment", "providers/sendgrid", "providers/sftp", diff --git a/scripts/ci/docker-compose/remove-sources.yml b/scripts/ci/docker-compose/remove-sources.yml index a2f7d3a035766..51db303510fb6 100644 --- a/scripts/ci/docker-compose/remove-sources.yml +++ b/scripts/ci/docker-compose/remove-sources.yml @@ -107,6 +107,8 @@ services: - ../../../empty:/opt/airflow/providers/redis/src - ../../../empty:/opt/airflow/providers/salesforce/src - ../../../empty:/opt/airflow/providers/samba/src + - ../../../empty:/opt/airflow/providers/sdk/executable/src + - ../../../empty:/opt/airflow/providers/sdk/java/src - ../../../empty:/opt/airflow/providers/segment/src - ../../../empty:/opt/airflow/providers/sendgrid/src - ../../../empty:/opt/airflow/providers/sftp/src diff --git a/scripts/ci/docker-compose/tests-sources.yml b/scripts/ci/docker-compose/tests-sources.yml index 9c02d1c271412..b1ae51dd87ebc 100644 --- a/scripts/ci/docker-compose/tests-sources.yml +++ b/scripts/ci/docker-compose/tests-sources.yml @@ -120,6 +120,8 @@ services: - ../../../providers/redis/tests:/opt/airflow/providers/redis/tests - ../../../providers/salesforce/tests:/opt/airflow/providers/salesforce/tests - ../../../providers/samba/tests:/opt/airflow/providers/samba/tests + - ../../../providers/sdk/executable/tests:/opt/airflow/providers/sdk/executable/tests + - ../../../providers/sdk/java/tests:/opt/airflow/providers/sdk/java/tests - ../../../providers/segment/tests:/opt/airflow/providers/segment/tests - ../../../providers/sendgrid/tests:/opt/airflow/providers/sendgrid/tests - ../../../providers/sftp/tests:/opt/airflow/providers/sftp/tests diff --git a/scripts/ci/prek/common_prek_utils.py b/scripts/ci/prek/common_prek_utils.py index 9e3d4ceb2dfcf..bfb427ccd1832 100644 --- a/scripts/ci/prek/common_prek_utils.py +++ b/scripts/ci/prek/common_prek_utils.py @@ -41,7 +41,7 @@ AIRFLOW_TASK_SDK_SOURCES_PATH = AIRFLOW_TASK_SDK_ROOT_PATH / "src" # Here we should add the second level paths that we want to have sub-packages in -KNOWN_SECOND_LEVEL_PATHS = ["apache", "atlassian", "common", "cncf", "dbt", "microsoft"] +KNOWN_SECOND_LEVEL_PATHS = ["apache", "atlassian", "common", "cncf", "dbt", "microsoft", "sdk"] DEFAULT_PYTHON_MAJOR_MINOR_VERSION = "3.10" diff --git a/scripts/in_container/install_airflow_and_providers.py b/scripts/in_container/install_airflow_and_providers.py index c8223f3eeff10..84847ea3e3041 100755 --- a/scripts/in_container/install_airflow_and_providers.py +++ b/scripts/in_container/install_airflow_and_providers.py @@ -1064,6 +1064,7 @@ def install_airflow_and_providers( "apache-airflow-providers-common-messaging", "apache-airflow-providers-git", "apache-airflow-providers-edge3", + "apache-airflow-providers-sdk-java", ] run_command( ["uv", "pip", "uninstall", *providers_to_uninstall_for_airflow_2], diff --git a/scripts/in_container/java_sdk_setup.sh b/scripts/in_container/java_sdk_setup.sh new file mode 100644 index 0000000000000..b3437b7fc4200 --- /dev/null +++ b/scripts/in_container/java_sdk_setup.sh @@ -0,0 +1,73 @@ +#!/usr/bin/env bash +# 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. + + + +# 1. Check Java +check_java() { + local java_bin="/files/openjdk/bin/java" + local version_output + + # First check if the locally installed OpenJDK exists and works. + if [ -x "$java_bin" ] && version_output=$("$java_bin" -version 2>&1); then + echo "Found existing OpenJDK at $java_bin. OK." + return + fi + + # On macOS, /usr/bin/java exists as a shim even without a JDK installed, + # so we must test with `java -version` directly. + if ! version_output=$(java -version 2>&1); then + echo "Java is not installed." + install_java + return + fi + + local java_version + java_version=$(echo "$version_output" | head -n1 | sed -E 's/.*"([0-9]+)(\.[0-9]+)*.*/\1/') + + if ! [[ "$java_version" =~ ^[0-9]+$ ]]; then + echo "Could not determine Java version." + install_java + return + fi + + if [ "$java_version" -ge 11 ]; then + echo "Java $java_version detected. OK." + else + echo "Java version $java_version found, but >= 11 is required." + install_java + fi +} + + +install_java() { + echo "Installing OpenJDK 11 in Breeze..." + + curl -L -o /files/openjdk-11-aarch64.tar.gz \ + https://github.com/adoptium/temurin11-binaries/releases/download/jdk-11.0.30+7/OpenJDK11U-jdk_aarch64_linux_hotspot_11.0.30_7.tar.gz + + rm -rf /files/openjdk && mkdir -p /files/openjdk && \ + tar -xzf /files/openjdk-11-aarch64.tar.gz --strip-components=1 -C /files/openjdk + + /files/openjdk/bin/java -version + echo "" +} + +check_java +# Install Java Provider +pip install -e /opt/airflow/providers/languages/java/ diff --git a/shared/workloads/.gitignore b/shared/workloads/.gitignore new file mode 100644 index 0000000000000..bff2d7629604d --- /dev/null +++ b/shared/workloads/.gitignore @@ -0,0 +1 @@ +*.iml diff --git a/shared/workloads/.pre-commit-config.yaml b/shared/workloads/.pre-commit-config.yaml new file mode 100644 index 0000000000000..7ce635750ca3c --- /dev/null +++ b/shared/workloads/.pre-commit-config.yaml @@ -0,0 +1,31 @@ +# 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. +--- +default_stages: [pre-commit, pre-push] +minimum_prek_version: '0.3.4' +default_language_version: + python: python3 +repos: + - repo: local + hooks: + - id: mypy-shared-workloads + name: Run mypy for shared-workloads + language: python + entry: ../../scripts/ci/prek/run_mypy_full_dist_local_venv_or_breeze_in_ci.py shared/workloads + pass_filenames: false + files: ^.*\.py$ + require_serial: true diff --git a/shared/workloads/pyproject.toml b/shared/workloads/pyproject.toml new file mode 100644 index 0000000000000..1a1c1dded04be --- /dev/null +++ b/shared/workloads/pyproject.toml @@ -0,0 +1,55 @@ +# 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. + +[project] +name = "apache-airflow-shared-workloads" +description = "Shared workload data transfer objects for Airflow distributions" +version = "0.0" +classifiers = [ + "Private :: Do Not Upload", +] + +dependencies = [ + "pydantic>=2.10.0", +] + +[dependency-groups] +dev = [ + "apache-airflow-devel-common", +] + +[build-system] +requires = [ + "hatchling==1.29.0", + "packaging==26.0", + "pathspec==1.0.4", + "pluggy==1.6.0", + "tomli==2.4.1; python_version < '3.11'", + "trove-classifiers==2026.1.14.14", +] +build-backend = "hatchling.build" + +[tool.hatch.build.targets.wheel] +packages = ["src/airflow_shared"] + +[tool.ruff] +extend = "../../pyproject.toml" +src = ["src"] + +[tool.ruff.lint.per-file-ignores] +# Ignore Doc rules et al for anything outside of tests +"!src/*" = ["D", "S101", "TRY002"] diff --git a/shared/workloads/src/airflow_shared/workloads/__init__.py b/shared/workloads/src/airflow_shared/workloads/__init__.py new file mode 100644 index 0000000000000..1a30b11c4b6af --- /dev/null +++ b/shared/workloads/src/airflow_shared/workloads/__init__.py @@ -0,0 +1,46 @@ +# 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. +"""Shared workload data transfer objects for Airflow distributions.""" + +from __future__ import annotations + +import uuid + +from pydantic import BaseModel, Field + + +class TaskInstanceDTO(BaseModel): + """Schema for TaskInstance with minimal required fields needed for Executors and Task SDK.""" + + id: uuid.UUID + dag_version_id: uuid.UUID + task_id: str + dag_id: str + run_id: str + try_number: int + map_index: int = -1 + + pool_slots: int + queue: str + priority_weight: int + executor_config: dict | None = Field(default=None, exclude=True) + + parent_context_carrier: dict | None = None + context_carrier: dict | None = None + + +__all__ = ["TaskInstanceDTO"] diff --git a/shared/workloads/tests/conftest.py b/shared/workloads/tests/conftest.py new file mode 100644 index 0000000000000..8b61b1b99b90d --- /dev/null +++ b/shared/workloads/tests/conftest.py @@ -0,0 +1,22 @@ +# +# 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 + +os.environ["_AIRFLOW__AS_LIBRARY"] = "true" diff --git a/shared/workloads/tests/workloads/__init__.py b/shared/workloads/tests/workloads/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/shared/workloads/tests/workloads/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/shared/workloads/tests/workloads/test_task_instance_dto.py b/shared/workloads/tests/workloads/test_task_instance_dto.py new file mode 100644 index 0000000000000..4788386dd6861 --- /dev/null +++ b/shared/workloads/tests/workloads/test_task_instance_dto.py @@ -0,0 +1,56 @@ +# +# 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. +"""Tests for shared TaskInstanceDTO.""" + +from __future__ import annotations + +import uuid + +import pytest + +from airflow_shared.workloads import TaskInstanceDTO + + +@pytest.fixture +def minimal_ti_kwargs(): + return { + "id": uuid.uuid4(), + "dag_version_id": uuid.uuid4(), + "task_id": "my_task", + "dag_id": "my_dag", + "run_id": "run_1", + "try_number": 1, + "pool_slots": 1, + "queue": "default", + "priority_weight": 1, + } + + +class TestTaskInstanceDTO: + """Test the shared TaskInstanceDTO model.""" + + def test_create_with_required_fields(self, minimal_ti_kwargs): + ti = TaskInstanceDTO(**minimal_ti_kwargs) + assert ti is not None + assert ti.task_id == "my_task" + assert ti.dag_id == "my_dag" + assert ti.run_id == "run_1" + assert ti.try_number == 1 + assert ti.pool_slots == 1 + assert ti.queue == "default" + assert ti.priority_weight == 1 diff --git a/shared/workloads/uv.lock b/shared/workloads/uv.lock new file mode 100644 index 0000000000000..60097a87185e6 --- /dev/null +++ b/shared/workloads/uv.lock @@ -0,0 +1,2125 @@ +version = 1 +revision = 3 +requires-python = ">=3.12" + +[options] +exclude-newer = "2026-04-17T08:44:15.45316Z" +exclude-newer-span = "P4D" + +[options.exclude-newer-package] +apache-airflow-providers-informatica = false +apache-airflow-providers-amazon = false +apache-airflow-providers-elasticsearch = false +apache-airflow-providers-microsoft-winrm = false +apache-airflow-docker-tests = false +apache-airflow-providers = false +apache-airflow-providers-fab = false +apache-airflow-providers-openlineage = false +apache-airflow-providers-sftp = false +apache-airflow-e2e-tests = false +apache-airflow-shared-logging = false +apache-airflow-providers-apache-drill = false +apache-airflow-providers-pgvector = false +apache-airflow-providers-imap = false +apache-airflow-providers-qdrant = false +apache-airflow-providers-edge3 = false +apache-airflow-providers-neo4j = false +apache-airflow-providers-discord = false +apache-airflow-providers-opensearch = false +apache-airflow-providers-samba = false +apache-airflow-providers-arangodb = false +apache-airflow-providers-apache-spark = false +apache-airflow-providers-ftp = false +apache-airflow-helm-tests = false +apache-airflow-providers-jenkins = false +apache-airflow-shared-listeners = false +apache-airflow-providers-telegram = false +apache-airflow-shared-providers-discovery = false +apache-airflow-providers-celery = false +apache-airflow-providers-docker = false +apache-airflow-providers-sendgrid = false +apache-airflow-providers-common-ai = false +apache-airflow = false +apache-airflow-shared-observability = false +apache-airflow-dev = false +apache-airflow-providers-dbt-cloud = false +apache-airflow-providers-openfaas = false +apache-airflow-devel-common = false +apache-airflow-providers-apache-cassandra = false +apache-airflow-providers-asana = false +apache-airflow-providers-oracle = false +apache-airflow-providers-mysql = false +apache-airflow-providers-alibaba = false +apache-airflow-providers-microsoft-mssql = false +apache-airflow-providers-teradata = false +apache-airflow-providers-jdbc = false +apache-airflow-providers-common-io = false +apache-airflow-providers-cohere = false +apache-airflow-providers-pinecone = false +apache-airflow-providers-segment = false +apache-airflow-providers-redis = false +apache-airflow-shared-dagnode = false +apache-airflow-providers-apache-pinot = false +apache-airflow-providers-weaviate = false +apache-airflow-providers-salesforce = false +apache-airflow-providers-ssh = false +apache-airflow-providers-papermill = false +apache-airflow-providers-google = false +apache-airflow-providers-microsoft-psrp = false +apache-airflow-providers-vertica = false +apache-airflow-providers-apache-hdfs = false +apache-airflow-shared-template-rendering = false +apache-airflow-providers-http = false +apache-airflow-providers-slack = false +apache-airflow-providers-vespa = false +apache-airflow-providers-databricks = false +apache-airflow-providers-sqlite = false +apache-airflow-shared-module-loading = false +apache-airflow-providers-yandex = false +apache-airflow-shared-serialization = false +apache-airflow-scripts = false +apache-airflow-providers-exasol = false +apache-airflow-providers-mongo = false +apache-airflow-providers-apprise = false +apache-airflow-providers-apache-impala = false +apache-airflow-ctl = false +apache-airflow-providers-github = false +apache-airflow-providers-snowflake = false +apache-airflow-providers-zendesk = false +apache-airflow-providers-presto = false +apache-airflow-providers-airbyte = false +apache-airflow-providers-apache-hive = false +apache-airflow-kubernetes-tests = false +apache-airflow-providers-grpc = false +apache-airflow-providers-apache-druid = false +apache-airflow-providers-cncf-kubernetes = false +apache-airflow-providers-apache-flink = false +apache-airflow-providers-apache-pig = false +apache-airflow-providers-apache-tinkerpop = false +apache-airflow-shared-timezones = false +apache-airflow-providers-apache-iceberg = false +apache-airflow-breeze = false +apache-airflow-providers-opsgenie = false +apache-airflow-providers-apache-livy = false +apache-airflow-core = false +apache-airflow-providers-hashicorp = false +apache-airflow-providers-pagerduty = false +apache-airflow-providers-datadog = false +apache-airflow-providers-apache-kafka = false +apache-airflow-providers-influxdb = false +apache-airflow-providers-keycloak = false +apache-airflow-providers-trino = false +apache-airflow-providers-common-messaging = false +apache-airflow-providers-standard = false +apache-airflow-providers-languages-java = false +apache-airflow-providers-singularity = false +apache-airflow-providers-common-compat = false +apache-airflow-ctl-tests = false +apache-airflow-providers-tableau = false +apache-airflow-providers-common-sql = false +apache-airflow-shared-configuration = false +apache-airflow-providers-facebook = false +apache-airflow-providers-ydb = false +apache-airflow-providers-microsoft-azure = false +apache-airflow-shared-plugins-manager = false +apache-airflow-shared-secrets-backend = false +apache-airflow-shared-secrets-masker = false +apache-airflow-providers-git = false +apache-airflow-task-sdk = false +apache-airflow-providers-atlassian-jira = false +apache-airflow-providers-odbc = false +apache-airflow-providers-postgres = false +apache-airflow-providers-openai = false +apache-airflow-task-sdk-integration-tests = false +apache-airflow-providers-smtp = false +apache-airflow-providers-dingding = false +apache-airflow-providers-apache-kylin = false +apache-airflow-providers-cloudant = false +apache-aurflow-docker-stack = false + +[[package]] +name = "aiohappyeyeballs" +version = "2.6.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/26/30/f84a107a9c4331c14b2b586036f40965c128aa4fee4dda5d3d51cb14ad54/aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558", size = 22760, upload-time = "2025-03-12T01:42:48.764Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0f/15/5bf3b99495fb160b63f95972b81750f18f7f4e02ad051373b669d17d44f2/aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8", size = 15265, upload-time = "2025-03-12T01:42:47.083Z" }, +] + +[[package]] +name = "aiohttp" +version = "3.13.5" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "aiohappyeyeballs" }, + { name = "aiosignal" }, + { name = "attrs" }, + { name = "frozenlist" }, + { name = "multidict" }, + { name = "propcache" }, + { name = "yarl" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/77/9a/152096d4808df8e4268befa55fba462f440f14beab85e8ad9bf990516918/aiohttp-3.13.5.tar.gz", hash = "sha256:9d98cc980ecc96be6eb4c1994ce35d28d8b1f5e5208a23b421187d1209dbb7d1", size = 7858271, upload-time = "2026-03-31T22:01:03.343Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/be/6f/353954c29e7dcce7cf00280a02c75f30e133c00793c7a2ed3776d7b2f426/aiohttp-3.13.5-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:023ecba036ddd840b0b19bf195bfae970083fd7024ce1ac22e9bba90464620e9", size = 748876, upload-time = "2026-03-31T21:57:36.319Z" }, + { url = "https://files.pythonhosted.org/packages/f5/1b/428a7c64687b3b2e9cd293186695affc0e1e54a445d0361743b231f11066/aiohttp-3.13.5-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:15c933ad7920b7d9a20de151efcd05a6e38302cbf0e10c9b2acb9a42210a2416", size = 499557, upload-time = "2026-03-31T21:57:38.236Z" }, + { url = "https://files.pythonhosted.org/packages/29/47/7be41556bfbb6917069d6a6634bb7dd5e163ba445b783a90d40f5ac7e3a7/aiohttp-3.13.5-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ab2899f9fa2f9f741896ebb6fa07c4c883bfa5c7f2ddd8cf2aafa86fa981b2d2", size = 500258, upload-time = "2026-03-31T21:57:39.923Z" }, + { url = "https://files.pythonhosted.org/packages/67/84/c9ecc5828cb0b3695856c07c0a6817a99d51e2473400f705275a2b3d9239/aiohttp-3.13.5-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:a60eaa2d440cd4707696b52e40ed3e2b0f73f65be07fd0ef23b6b539c9c0b0b4", size = 1749199, upload-time = "2026-03-31T21:57:41.938Z" }, + { url = "https://files.pythonhosted.org/packages/f0/d3/3c6d610e66b495657622edb6ae7c7fd31b2e9086b4ec50b47897ad6042a9/aiohttp-3.13.5-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:55b3bdd3292283295774ab585160c4004f4f2f203946997f49aac032c84649e9", size = 1721013, upload-time = "2026-03-31T21:57:43.904Z" }, + { url = "https://files.pythonhosted.org/packages/49/a0/24409c12217456df0bae7babe3b014e460b0b38a8e60753d6cb339f6556d/aiohttp-3.13.5-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c2b2355dc094e5f7d45a7bb262fe7207aa0460b37a0d87027dcf21b5d890e7d5", size = 1781501, upload-time = "2026-03-31T21:57:46.285Z" }, + { url = "https://files.pythonhosted.org/packages/98/9d/b65ec649adc5bccc008b0957a9a9c691070aeac4e41cea18559fef49958b/aiohttp-3.13.5-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:b38765950832f7d728297689ad78f5f2cf79ff82487131c4d26fe6ceecdc5f8e", size = 1878981, upload-time = "2026-03-31T21:57:48.734Z" }, + { url = "https://files.pythonhosted.org/packages/57/d8/8d44036d7eb7b6a8ec4c5494ea0c8c8b94fbc0ed3991c1a7adf230df03bf/aiohttp-3.13.5-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b18f31b80d5a33661e08c89e202edabf1986e9b49c42b4504371daeaa11b47c1", size = 1767934, upload-time = "2026-03-31T21:57:51.171Z" }, + { url = "https://files.pythonhosted.org/packages/31/04/d3f8211f273356f158e3464e9e45484d3fb8c4ce5eb2f6fe9405c3273983/aiohttp-3.13.5-cp312-cp312-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:33add2463dde55c4f2d9635c6ab33ce154e5ecf322bd26d09af95c5f81cfa286", size = 1566671, upload-time = "2026-03-31T21:57:53.326Z" }, + { url = "https://files.pythonhosted.org/packages/41/db/073e4ebe00b78e2dfcacff734291651729a62953b48933d765dc513bf798/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:327cc432fdf1356fb4fbc6fe833ad4e9f6aacb71a8acaa5f1855e4b25910e4a9", size = 1705219, upload-time = "2026-03-31T21:57:55.385Z" }, + { url = "https://files.pythonhosted.org/packages/48/45/7dfba71a2f9fd97b15c95c06819de7eb38113d2cdb6319669195a7d64270/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:7c35b0bf0b48a70b4cb4fc5d7bed9b932532728e124874355de1a0af8ec4bc88", size = 1743049, upload-time = "2026-03-31T21:57:57.341Z" }, + { url = "https://files.pythonhosted.org/packages/18/71/901db0061e0f717d226386a7f471bb59b19566f2cae5f0d93874b017271f/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:df23d57718f24badef8656c49743e11a89fd6f5358fa8a7b96e728fda2abf7d3", size = 1749557, upload-time = "2026-03-31T21:57:59.626Z" }, + { url = "https://files.pythonhosted.org/packages/08/d5/41eebd16066e59cd43728fe74bce953d7402f2b4ddfdfef2c0e9f17ca274/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_riscv64.whl", hash = "sha256:02e048037a6501a5ec1f6fc9736135aec6eb8a004ce48838cb951c515f32c80b", size = 1558931, upload-time = "2026-03-31T21:58:01.972Z" }, + { url = "https://files.pythonhosted.org/packages/30/e6/4a799798bf05740e66c3a1161079bda7a3dd8e22ca392481d7a7f9af82a6/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:31cebae8b26f8a615d2b546fee45d5ffb76852ae6450e2a03f42c9102260d6fe", size = 1774125, upload-time = "2026-03-31T21:58:04.007Z" }, + { url = "https://files.pythonhosted.org/packages/84/63/7749337c90f92bc2cb18f9560d67aa6258c7060d1397d21529b8004fcf6f/aiohttp-3.13.5-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:888e78eb5ca55a615d285c3c09a7a91b42e9dd6fc699b166ebd5dee87c9ccf14", size = 1732427, upload-time = "2026-03-31T21:58:06.337Z" }, + { url = "https://files.pythonhosted.org/packages/98/de/cf2f44ff98d307e72fb97d5f5bbae3bfcb442f0ea9790c0bf5c5c2331404/aiohttp-3.13.5-cp312-cp312-win32.whl", hash = "sha256:8bd3ec6376e68a41f9f95f5ed170e2fcf22d4eb27a1f8cb361d0508f6e0557f3", size = 433534, upload-time = "2026-03-31T21:58:08.712Z" }, + { url = "https://files.pythonhosted.org/packages/aa/ca/eadf6f9c8fa5e31d40993e3db153fb5ed0b11008ad5d9de98a95045bed84/aiohttp-3.13.5-cp312-cp312-win_amd64.whl", hash = "sha256:110e448e02c729bcebb18c60b9214a87ba33bac4a9fa5e9a5f139938b56c6cb1", size = 460446, upload-time = "2026-03-31T21:58:10.945Z" }, + { url = "https://files.pythonhosted.org/packages/78/e9/d76bf503005709e390122d34e15256b88f7008e246c4bdbe915cd4f1adce/aiohttp-3.13.5-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:a5029cc80718bbd545123cd8fe5d15025eccaaaace5d0eeec6bd556ad6163d61", size = 742930, upload-time = "2026-03-31T21:58:13.155Z" }, + { url = "https://files.pythonhosted.org/packages/57/00/4b7b70223deaebd9bb85984d01a764b0d7bd6526fcdc73cca83bcbe7243e/aiohttp-3.13.5-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:4bb6bf5811620003614076bdc807ef3b5e38244f9d25ca5fe888eaccea2a9832", size = 496927, upload-time = "2026-03-31T21:58:15.073Z" }, + { url = "https://files.pythonhosted.org/packages/9c/f5/0fb20fb49f8efdcdce6cd8127604ad2c503e754a8f139f5e02b01626523f/aiohttp-3.13.5-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a84792f8631bf5a94e52d9cc881c0b824ab42717165a5579c760b830d9392ac9", size = 497141, upload-time = "2026-03-31T21:58:17.009Z" }, + { url = "https://files.pythonhosted.org/packages/3b/86/b7c870053e36a94e8951b803cb5b909bfbc9b90ca941527f5fcafbf6b0fa/aiohttp-3.13.5-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:57653eac22c6a4c13eb22ecf4d673d64a12f266e72785ab1c8b8e5940d0e8090", size = 1732476, upload-time = "2026-03-31T21:58:18.925Z" }, + { url = "https://files.pythonhosted.org/packages/b5/e5/4e161f84f98d80c03a238671b4136e6530453d65262867d989bbe78244d0/aiohttp-3.13.5-cp313-cp313-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:e5e5f7debc7a57af53fdf5c5009f9391d9f4c12867049d509bf7bb164a6e295b", size = 1706507, upload-time = "2026-03-31T21:58:21.094Z" }, + { url = "https://files.pythonhosted.org/packages/d4/56/ea11a9f01518bd5a2a2fcee869d248c4b8a0cfa0bb13401574fa31adf4d4/aiohttp-3.13.5-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c719f65bebcdf6716f10e9eff80d27567f7892d8988c06de12bbbd39307c6e3a", size = 1773465, upload-time = "2026-03-31T21:58:23.159Z" }, + { url = "https://files.pythonhosted.org/packages/eb/40/333ca27fb74b0383f17c90570c748f7582501507307350a79d9f9f3c6eb1/aiohttp-3.13.5-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:d97f93fdae594d886c5a866636397e2bcab146fd7a132fd6bb9ce182224452f8", size = 1873523, upload-time = "2026-03-31T21:58:25.59Z" }, + { url = "https://files.pythonhosted.org/packages/f0/d2/e2f77eef1acb7111405433c707dc735e63f67a56e176e72e9e7a2cd3f493/aiohttp-3.13.5-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:3df334e39d4c2f899a914f1dba283c1aadc311790733f705182998c6f7cae665", size = 1754113, upload-time = "2026-03-31T21:58:27.624Z" }, + { url = "https://files.pythonhosted.org/packages/fb/56/3f653d7f53c89669301ec9e42c95233e2a0c0a6dd051269e6e678db4fdb0/aiohttp-3.13.5-cp313-cp313-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:fe6970addfea9e5e081401bcbadf865d2b6da045472f58af08427e108d618540", size = 1562351, upload-time = "2026-03-31T21:58:29.918Z" }, + { url = "https://files.pythonhosted.org/packages/ec/a6/9b3e91eb8ae791cce4ee736da02211c85c6f835f1bdfac0594a8a3b7018c/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:7becdf835feff2f4f335d7477f121af787e3504b48b449ff737afb35869ba7bb", size = 1693205, upload-time = "2026-03-31T21:58:32.214Z" }, + { url = "https://files.pythonhosted.org/packages/98/fc/bfb437a99a2fcebd6b6eaec609571954de2ed424f01c352f4b5504371dd3/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:676e5651705ad5d8a70aeb8eb6936c436d8ebbd56e63436cb7dd9bb36d2a9a46", size = 1730618, upload-time = "2026-03-31T21:58:34.728Z" }, + { url = "https://files.pythonhosted.org/packages/e4/b6/c8534862126191a034f68153194c389addc285a0f1347d85096d349bbc15/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:9b16c653d38eb1a611cc898c41e76859ca27f119d25b53c12875fd0474ae31a8", size = 1745185, upload-time = "2026-03-31T21:58:36.909Z" }, + { url = "https://files.pythonhosted.org/packages/0b/93/4ca8ee2ef5236e2707e0fd5fecb10ce214aee1ff4ab307af9c558bda3b37/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_riscv64.whl", hash = "sha256:999802d5fa0389f58decd24b537c54aa63c01c3219ce17d1214cbda3c2b22d2d", size = 1557311, upload-time = "2026-03-31T21:58:39.38Z" }, + { url = "https://files.pythonhosted.org/packages/57/ae/76177b15f18c5f5d094f19901d284025db28eccc5ae374d1d254181d33f4/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:ec707059ee75732b1ba130ed5f9580fe10ff75180c812bc267ded039db5128c6", size = 1773147, upload-time = "2026-03-31T21:58:41.476Z" }, + { url = "https://files.pythonhosted.org/packages/01/a4/62f05a0a98d88af59d93b7fcac564e5f18f513cb7471696ac286db970d6a/aiohttp-3.13.5-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:2d6d44a5b48132053c2f6cd5c8cb14bc67e99a63594e336b0f2af81e94d5530c", size = 1730356, upload-time = "2026-03-31T21:58:44.049Z" }, + { url = "https://files.pythonhosted.org/packages/e4/85/fc8601f59dfa8c9523808281f2da571f8b4699685f9809a228adcc90838d/aiohttp-3.13.5-cp313-cp313-win32.whl", hash = "sha256:329f292ed14d38a6c4c435e465f48bebb47479fd676a0411936cc371643225cc", size = 432637, upload-time = "2026-03-31T21:58:46.167Z" }, + { url = "https://files.pythonhosted.org/packages/c0/1b/ac685a8882896acf0f6b31d689e3792199cfe7aba37969fa91da63a7fa27/aiohttp-3.13.5-cp313-cp313-win_amd64.whl", hash = "sha256:69f571de7500e0557801c0b51f4780482c0ec5fe2ac851af5a92cfce1af1cb83", size = 458896, upload-time = "2026-03-31T21:58:48.119Z" }, + { url = "https://files.pythonhosted.org/packages/5d/ce/46572759afc859e867a5bc8ec3487315869013f59281ce61764f76d879de/aiohttp-3.13.5-cp314-cp314-macosx_10_13_universal2.whl", hash = "sha256:eb4639f32fd4a9904ab8fb45bf3383ba71137f3d9d4ba25b3b3f3109977c5b8c", size = 745721, upload-time = "2026-03-31T21:58:50.229Z" }, + { url = "https://files.pythonhosted.org/packages/13/fe/8a2efd7626dbe6049b2ef8ace18ffda8a4dfcbe1bcff3ac30c0c7575c20b/aiohttp-3.13.5-cp314-cp314-macosx_10_13_x86_64.whl", hash = "sha256:7e5dc4311bd5ac493886c63cbf76ab579dbe4641268e7c74e48e774c74b6f2be", size = 497663, upload-time = "2026-03-31T21:58:52.232Z" }, + { url = "https://files.pythonhosted.org/packages/9b/91/cc8cc78a111826c54743d88651e1687008133c37e5ee615fee9b57990fac/aiohttp-3.13.5-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:756c3c304d394977519824449600adaf2be0ccee76d206ee339c5e76b70ded25", size = 499094, upload-time = "2026-03-31T21:58:54.566Z" }, + { url = "https://files.pythonhosted.org/packages/0a/33/a8362cb15cf16a3af7e86ed11962d5cd7d59b449202dc576cdc731310bde/aiohttp-3.13.5-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ecc26751323224cf8186efcf7fbcbc30f4e1d8c7970659daf25ad995e4032a56", size = 1726701, upload-time = "2026-03-31T21:58:56.864Z" }, + { url = "https://files.pythonhosted.org/packages/45/0c/c091ac5c3a17114bd76cbf85d674650969ddf93387876cf67f754204bd77/aiohttp-3.13.5-cp314-cp314-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:10a75acfcf794edf9d8db50e5a7ec5fc818b2a8d3f591ce93bc7b1210df016d2", size = 1683360, upload-time = "2026-03-31T21:58:59.072Z" }, + { url = "https://files.pythonhosted.org/packages/23/73/bcee1c2b79bc275e964d1446c55c54441a461938e70267c86afaae6fba27/aiohttp-3.13.5-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:0f7a18f258d124cd678c5fe072fe4432a4d5232b0657fca7c1847f599233c83a", size = 1773023, upload-time = "2026-03-31T21:59:01.776Z" }, + { url = "https://files.pythonhosted.org/packages/c7/ef/720e639df03004fee2d869f771799d8c23046dec47d5b81e396c7cda583a/aiohttp-3.13.5-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:df6104c009713d3a89621096f3e3e88cc323fd269dbd7c20afe18535094320be", size = 1853795, upload-time = "2026-03-31T21:59:04.568Z" }, + { url = "https://files.pythonhosted.org/packages/bd/c9/989f4034fb46841208de7aeeac2c6d8300745ab4f28c42f629ba77c2d916/aiohttp-3.13.5-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:241a94f7de7c0c3b616627aaad530fe2cb620084a8b144d3be7b6ecfe95bae3b", size = 1730405, upload-time = "2026-03-31T21:59:07.221Z" }, + { url = "https://files.pythonhosted.org/packages/ce/75/ee1fd286ca7dc599d824b5651dad7b3be7ff8d9a7e7b3fe9820d9180f7db/aiohttp-3.13.5-cp314-cp314-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:c974fb66180e58709b6fc402846f13791240d180b74de81d23913abe48e96d94", size = 1558082, upload-time = "2026-03-31T21:59:09.484Z" }, + { url = "https://files.pythonhosted.org/packages/c3/20/1e9e6650dfc436340116b7aa89ff8cb2bbdf0abc11dfaceaad8f74273a10/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:6e27ea05d184afac78aabbac667450c75e54e35f62238d44463131bd3f96753d", size = 1692346, upload-time = "2026-03-31T21:59:12.068Z" }, + { url = "https://files.pythonhosted.org/packages/d8/40/8ebc6658d48ea630ac7903912fe0dd4e262f0e16825aa4c833c56c9f1f56/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:a79a6d399cef33a11b6f004c67bb07741d91f2be01b8d712d52c75711b1e07c7", size = 1698891, upload-time = "2026-03-31T21:59:14.552Z" }, + { url = "https://files.pythonhosted.org/packages/d8/78/ea0ae5ec8ba7a5c10bdd6e318f1ba5e76fcde17db8275188772afc7917a4/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:c632ce9c0b534fbe25b52c974515ed674937c5b99f549a92127c85f771a78772", size = 1742113, upload-time = "2026-03-31T21:59:17.068Z" }, + { url = "https://files.pythonhosted.org/packages/8a/66/9d308ed71e3f2491be1acb8769d96c6f0c47d92099f3bc9119cada27b357/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_riscv64.whl", hash = "sha256:fceedde51fbd67ee2bcc8c0b33d0126cc8b51ef3bbde2f86662bd6d5a6f10ec5", size = 1553088, upload-time = "2026-03-31T21:59:19.541Z" }, + { url = "https://files.pythonhosted.org/packages/da/a6/6cc25ed8dfc6e00c90f5c6d126a98e2cf28957ad06fa1036bd34b6f24a2c/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:f92995dfec9420bb69ae629abf422e516923ba79ba4403bc750d94fb4a6c68c1", size = 1757976, upload-time = "2026-03-31T21:59:22.311Z" }, + { url = "https://files.pythonhosted.org/packages/c1/2b/cce5b0ffe0de99c83e5e36d8f828e4161e415660a9f3e58339d07cce3006/aiohttp-3.13.5-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:20ae0ff08b1f2c8788d6fb85afcb798654ae6ba0b747575f8562de738078457b", size = 1712444, upload-time = "2026-03-31T21:59:24.635Z" }, + { url = "https://files.pythonhosted.org/packages/6c/cf/9e1795b4160c58d29421eafd1a69c6ce351e2f7c8d3c6b7e4ca44aea1a5b/aiohttp-3.13.5-cp314-cp314-win32.whl", hash = "sha256:b20df693de16f42b2472a9c485e1c948ee55524786a0a34345511afdd22246f3", size = 438128, upload-time = "2026-03-31T21:59:27.291Z" }, + { url = "https://files.pythonhosted.org/packages/22/4d/eaedff67fc805aeba4ba746aec891b4b24cebb1a7d078084b6300f79d063/aiohttp-3.13.5-cp314-cp314-win_amd64.whl", hash = "sha256:f85c6f327bf0b8c29da7d93b1cabb6363fb5e4e160a32fa241ed2dce21b73162", size = 464029, upload-time = "2026-03-31T21:59:29.429Z" }, + { url = "https://files.pythonhosted.org/packages/79/11/c27d9332ee20d68dd164dc12a6ecdef2e2e35ecc97ed6cf0d2442844624b/aiohttp-3.13.5-cp314-cp314t-macosx_10_13_universal2.whl", hash = "sha256:1efb06900858bb618ff5cee184ae2de5828896c448403d51fb633f09e109be0a", size = 778758, upload-time = "2026-03-31T21:59:31.547Z" }, + { url = "https://files.pythonhosted.org/packages/04/fb/377aead2e0a3ba5f09b7624f702a964bdf4f08b5b6728a9799830c80041e/aiohttp-3.13.5-cp314-cp314t-macosx_10_13_x86_64.whl", hash = "sha256:fee86b7c4bd29bdaf0d53d14739b08a106fdda809ca5fe032a15f52fae5fe254", size = 512883, upload-time = "2026-03-31T21:59:34.098Z" }, + { url = "https://files.pythonhosted.org/packages/bb/a6/aa109a33671f7a5d3bd78b46da9d852797c5e665bfda7d6b373f56bff2ec/aiohttp-3.13.5-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:20058e23909b9e65f9da62b396b77dfa95965cbe840f8def6e572538b1d32e36", size = 516668, upload-time = "2026-03-31T21:59:36.497Z" }, + { url = "https://files.pythonhosted.org/packages/79/b3/ca078f9f2fa9563c36fb8ef89053ea2bb146d6f792c5104574d49d8acb63/aiohttp-3.13.5-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:8cf20a8d6868cb15a73cab329ffc07291ba8c22b1b88176026106ae39aa6df0f", size = 1883461, upload-time = "2026-03-31T21:59:38.723Z" }, + { url = "https://files.pythonhosted.org/packages/b7/e3/a7ad633ca1ca497b852233a3cce6906a56c3225fb6d9217b5e5e60b7419d/aiohttp-3.13.5-cp314-cp314t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:330f5da04c987f1d5bdb8ae189137c77139f36bd1cb23779ca1a354a4b027800", size = 1747661, upload-time = "2026-03-31T21:59:41.187Z" }, + { url = "https://files.pythonhosted.org/packages/33/b9/cd6fe579bed34a906d3d783fe60f2fa297ef55b27bb4538438ee49d4dc41/aiohttp-3.13.5-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:6f1cbf0c7926d315c3c26c2da41fd2b5d2fe01ac0e157b78caefc51a782196cf", size = 1863800, upload-time = "2026-03-31T21:59:43.84Z" }, + { url = "https://files.pythonhosted.org/packages/c0/3f/2c1e2f5144cefa889c8afd5cf431994c32f3b29da9961698ff4e3811b79a/aiohttp-3.13.5-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:53fc049ed6390d05423ba33103ded7281fe897cf97878f369a527070bd95795b", size = 1958382, upload-time = "2026-03-31T21:59:46.187Z" }, + { url = "https://files.pythonhosted.org/packages/66/1d/f31ec3f1013723b3babe3609e7f119c2c2fb6ef33da90061a705ef3e1bc8/aiohttp-3.13.5-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:898703aa2667e3c5ca4c54ca36cd73f58b7a38ef87a5606414799ebce4d3fd3a", size = 1803724, upload-time = "2026-03-31T21:59:48.656Z" }, + { url = "https://files.pythonhosted.org/packages/0e/b4/57712dfc6f1542f067daa81eb61da282fab3e6f1966fca25db06c4fc62d5/aiohttp-3.13.5-cp314-cp314t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:0494a01ca9584eea1e5fbd6d748e61ecff218c51b576ee1999c23db7066417d8", size = 1640027, upload-time = "2026-03-31T21:59:51.284Z" }, + { url = "https://files.pythonhosted.org/packages/25/3c/734c878fb43ec083d8e31bf029daae1beafeae582d1b35da234739e82ee7/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:6cf81fe010b8c17b09495cbd15c1d35afbc8fb405c0c9cf4738e5ae3af1d65be", size = 1806644, upload-time = "2026-03-31T21:59:53.753Z" }, + { url = "https://files.pythonhosted.org/packages/20/a5/f671e5cbec1c21d044ff3078223f949748f3a7f86b14e34a365d74a5d21f/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:c564dd5f09ddc9d8f2c2d0a301cd30a79a2cc1b46dd1a73bef8f0038863d016b", size = 1791630, upload-time = "2026-03-31T21:59:56.239Z" }, + { url = "https://files.pythonhosted.org/packages/0b/63/fb8d0ad63a0b8a99be97deac8c04dacf0785721c158bdf23d679a87aa99e/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:2994be9f6e51046c4f864598fd9abeb4fba6e88f0b2152422c9666dcd4aea9c6", size = 1809403, upload-time = "2026-03-31T21:59:59.103Z" }, + { url = "https://files.pythonhosted.org/packages/59/0c/bfed7f30662fcf12206481c2aac57dedee43fe1c49275e85b3a1e1742294/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_riscv64.whl", hash = "sha256:157826e2fa245d2ef46c83ea8a5faf77ca19355d278d425c29fda0beb3318037", size = 1634924, upload-time = "2026-03-31T22:00:02.116Z" }, + { url = "https://files.pythonhosted.org/packages/17/d6/fd518d668a09fd5a3319ae5e984d4d80b9a4b3df4e21c52f02251ef5a32e/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:a8aca50daa9493e9e13c0f566201a9006f080e7c50e5e90d0b06f53146a54500", size = 1836119, upload-time = "2026-03-31T22:00:04.756Z" }, + { url = "https://files.pythonhosted.org/packages/78/b7/15fb7a9d52e112a25b621c67b69c167805cb1f2ab8f1708a5c490d1b52fe/aiohttp-3.13.5-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:3b13560160d07e047a93f23aaa30718606493036253d5430887514715b67c9d9", size = 1772072, upload-time = "2026-03-31T22:00:07.494Z" }, + { url = "https://files.pythonhosted.org/packages/7e/df/57ba7f0c4a553fc2bd8b6321df236870ec6fd64a2a473a8a13d4f733214e/aiohttp-3.13.5-cp314-cp314t-win32.whl", hash = "sha256:9a0f4474b6ea6818b41f82172d799e4b3d29e22c2c520ce4357856fced9af2f8", size = 471819, upload-time = "2026-03-31T22:00:10.277Z" }, + { url = "https://files.pythonhosted.org/packages/62/29/2f8418269e46454a26171bfdd6a055d74febf32234e474930f2f60a17145/aiohttp-3.13.5-cp314-cp314t-win_amd64.whl", hash = "sha256:18a2f6c1182c51baa1d28d68fea51513cb2a76612f038853c0ad3c145423d3d9", size = 505441, upload-time = "2026-03-31T22:00:12.791Z" }, +] + +[[package]] +name = "aioresponses" +version = "0.7.8" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "aiohttp" }, + { name = "packaging" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/de/03/532bbc645bdebcf3b6af3b25d46655259d66ce69abba7720b71ebfabbade/aioresponses-0.7.8.tar.gz", hash = "sha256:b861cdfe5dc58f3b8afac7b0a6973d5d7b2cb608dd0f6253d16b8ee8eaf6df11", size = 40253, upload-time = "2025-01-19T18:14:03.222Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/12/b7/584157e43c98aa89810bc2f7099e7e01c728ecf905a66cf705106009228f/aioresponses-0.7.8-py2.py3-none-any.whl", hash = "sha256:b73bd4400d978855e55004b23a3a84cb0f018183bcf066a85ad392800b5b9a94", size = 12518, upload-time = "2025-01-19T18:13:59.633Z" }, +] + +[[package]] +name = "aiosignal" +version = "1.4.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "frozenlist" }, + { name = "typing-extensions", marker = "python_full_version < '3.13'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/61/62/06741b579156360248d1ec624842ad0edf697050bbaf7c3e46394e106ad1/aiosignal-1.4.0.tar.gz", hash = "sha256:f47eecd9468083c2029cc99945502cb7708b082c232f9aca65da147157b251c7", size = 25007, upload-time = "2025-07-03T22:54:43.528Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/fb/76/641ae371508676492379f16e2fa48f4e2c11741bd63c48be4b12a6b09cba/aiosignal-1.4.0-py3-none-any.whl", hash = "sha256:053243f8b92b990551949e63930a839ff0cf0b0ebbe0597b0f3fb19e1a0fe82e", size = 7490, upload-time = "2025-07-03T22:54:42.156Z" }, +] + +[[package]] +name = "annotated-types" +version = "0.7.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ee/67/531ea369ba64dcff5ec9c3402f9f51bf748cec26dde048a2f973a4eea7f5/annotated_types-0.7.0.tar.gz", hash = "sha256:aff07c09a53a08bc8cfccb9c85b05f1aa9a2a6f23728d790723543408344ce89", size = 16081, upload-time = "2024-05-20T21:33:25.928Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/78/b6/6307fbef88d9b5ee7421e68d78a9f162e0da4900bc5f5793f6d3d0e34fb8/annotated_types-0.7.0-py3-none-any.whl", hash = "sha256:1f02e8b43a8fbbc3f3e0d4f0f4bfc8131bcb4eebe8849b8e5c773f3a1c582a53", size = 13643, upload-time = "2024-05-20T21:33:24.1Z" }, +] + +[[package]] +name = "apache-airflow-devel-common" +version = "0.1.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "aioresponses" }, + { name = "black" }, + { name = "coverage" }, + { name = "filelock" }, + { name = "ipdb" }, + { name = "jmespath" }, + { name = "kgb" }, + { name = "mypy" }, + { name = "pdbr" }, + { name = "pytest" }, + { name = "pytest-asyncio" }, + { name = "pytest-cov" }, + { name = "pytest-custom-exit-code" }, + { name = "pytest-icdiff" }, + { name = "pytest-instafail" }, + { name = "pytest-mock" }, + { name = "pytest-rerunfailures" }, + { name = "pytest-timeouts" }, + { name = "pytest-unordered" }, + { name = "pytest-xdist" }, + { name = "requests-mock" }, + { name = "rich" }, + { name = "ruff" }, + { name = "semver" }, + { name = "time-machine" }, + { name = "types-aiofiles" }, + { name = "types-certifi" }, + { name = "types-croniter" }, + { name = "types-deprecated" }, + { name = "types-docutils" }, + { name = "types-markdown" }, + { name = "types-paramiko" }, + { name = "types-protobuf" }, + { name = "types-pymysql" }, + { name = "types-python-dateutil" }, + { name = "types-python-slugify" }, + { name = "types-pytz" }, + { name = "types-pyyaml" }, + { name = "types-redis" }, + { name = "types-requests" }, + { name = "types-setuptools" }, + { name = "types-tabulate" }, + { name = "types-toml" }, + { name = "wheel" }, + { name = "yamllint" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/63/cf/b1f97e6b6f2e57adda012429c541f1881610061e646945e75dfe7d811a9c/apache_airflow_devel_common-0.1.1.tar.gz", hash = "sha256:b066b2d3fdf940b4c2859af23036a7ef31f8e81694209b1ab0e056530e286232", size = 63846, upload-time = "2025-05-16T20:05:09.303Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a8/df/9f05b0e6fc132e59b5dec26f4ec610bc611b5f256495b7db25066f5b743b/apache_airflow_devel_common-0.1.1-py2.py3-none-any.whl", hash = "sha256:951ab206cf2f95768d08ad8c15b1cfaa7df404308c9e32998e3e16d217d2b032", size = 100909, upload-time = "2025-05-16T20:05:07.52Z" }, +] + +[[package]] +name = "apache-airflow-shared-workloads" +version = "0.0" +source = { editable = "." } +dependencies = [ + { name = "pydantic" }, +] + +[package.dev-dependencies] +dev = [ + { name = "apache-airflow-devel-common" }, +] + +[package.metadata] +requires-dist = [{ name = "pydantic", specifier = ">=2.10.0" }] + +[package.metadata.requires-dev] +dev = [{ name = "apache-airflow-devel-common" }] + +[[package]] +name = "asttokens" +version = "3.0.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/be/a5/8e3f9b6771b0b408517c82d97aed8f2036509bc247d46114925e32fe33f0/asttokens-3.0.1.tar.gz", hash = "sha256:71a4ee5de0bde6a31d64f6b13f2293ac190344478f081c3d1bccfcf5eacb0cb7", size = 62308, upload-time = "2025-11-15T16:43:48.578Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d2/39/e7eaf1799466a4aef85b6a4fe7bd175ad2b1c6345066aa33f1f58d4b18d0/asttokens-3.0.1-py3-none-any.whl", hash = "sha256:15a3ebc0f43c2d0a50eeafea25e19046c68398e487b9f1f5b517f7c0f40f976a", size = 27047, upload-time = "2025-11-15T16:43:16.109Z" }, +] + +[[package]] +name = "attrs" +version = "26.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9a/8e/82a0fe20a541c03148528be8cac2408564a6c9a0cc7e9171802bc1d26985/attrs-26.1.0.tar.gz", hash = "sha256:d03ceb89cb322a8fd706d4fb91940737b6642aa36998fe130a9bc96c985eff32", size = 952055, upload-time = "2026-03-19T14:22:25.026Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/64/b4/17d4b0b2a2dc85a6df63d1157e028ed19f90d4cd97c36717afef2bc2f395/attrs-26.1.0-py3-none-any.whl", hash = "sha256:c647aa4a12dfbad9333ca4e71fe62ddc36f4e63b2d260a37a8b83d2f043ac309", size = 67548, upload-time = "2026-03-19T14:22:23.645Z" }, +] + +[[package]] +name = "black" +version = "26.3.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "click" }, + { name = "mypy-extensions" }, + { name = "packaging" }, + { name = "pathspec" }, + { name = "platformdirs" }, + { name = "pytokens" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/e1/c5/61175d618685d42b005847464b8fb4743a67b1b8fdb75e50e5a96c31a27a/black-26.3.1.tar.gz", hash = "sha256:2c50f5063a9641c7eed7795014ba37b0f5fa227f3d408b968936e24bc0566b07", size = 666155, upload-time = "2026-03-12T03:36:03.593Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/dc/f8/da5eae4fc75e78e6dceb60624e1b9662ab00d6b452996046dfa9b8a6025b/black-26.3.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:b5e6f89631eb88a7302d416594a32faeee9fb8fb848290da9d0a5f2903519fc1", size = 1895920, upload-time = "2026-03-12T03:40:13.921Z" }, + { url = "https://files.pythonhosted.org/packages/2c/9f/04e6f26534da2e1629b2b48255c264cabf5eedc5141d04516d9d68a24111/black-26.3.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:41cd2012d35b47d589cb8a16faf8a32ef7a336f56356babd9fcf70939ad1897f", size = 1718499, upload-time = "2026-03-12T03:40:15.239Z" }, + { url = "https://files.pythonhosted.org/packages/04/91/a5935b2a63e31b331060c4a9fdb5a6c725840858c599032a6f3aac94055f/black-26.3.1-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0f76ff19ec5297dd8e66eb64deda23631e642c9393ab592826fd4bdc97a4bce7", size = 1794994, upload-time = "2026-03-12T03:40:17.124Z" }, + { url = "https://files.pythonhosted.org/packages/e7/0a/86e462cdd311a3c2a8ece708d22aba17d0b2a0d5348ca34b40cdcbea512e/black-26.3.1-cp312-cp312-win_amd64.whl", hash = "sha256:ddb113db38838eb9f043623ba274cfaf7d51d5b0c22ecb30afe58b1bb8322983", size = 1420867, upload-time = "2026-03-12T03:40:18.83Z" }, + { url = "https://files.pythonhosted.org/packages/5b/e5/22515a19cb7eaee3440325a6b0d95d2c0e88dd180cb011b12ae488e031d1/black-26.3.1-cp312-cp312-win_arm64.whl", hash = "sha256:dfdd51fc3e64ea4f35873d1b3fb25326773d55d2329ff8449139ebaad7357efb", size = 1230124, upload-time = "2026-03-12T03:40:20.425Z" }, + { url = "https://files.pythonhosted.org/packages/f5/77/5728052a3c0450c53d9bb3945c4c46b91baa62b2cafab6801411b6271e45/black-26.3.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:855822d90f884905362f602880ed8b5df1b7e3ee7d0db2502d4388a954cc8c54", size = 1895034, upload-time = "2026-03-12T03:40:21.813Z" }, + { url = "https://files.pythonhosted.org/packages/52/73/7cae55fdfdfbe9d19e9a8d25d145018965fe2079fa908101c3733b0c55a0/black-26.3.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:8a33d657f3276328ce00e4d37fe70361e1ec7614da5d7b6e78de5426cb56332f", size = 1718503, upload-time = "2026-03-12T03:40:23.666Z" }, + { url = "https://files.pythonhosted.org/packages/e1/87/af89ad449e8254fdbc74654e6467e3c9381b61472cc532ee350d28cfdafb/black-26.3.1-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:f1cd08e99d2f9317292a311dfe578fd2a24b15dbce97792f9c4d752275c1fa56", size = 1793557, upload-time = "2026-03-12T03:40:25.497Z" }, + { url = "https://files.pythonhosted.org/packages/43/10/d6c06a791d8124b843bf325ab4ac7d2f5b98731dff84d6064eafd687ded1/black-26.3.1-cp313-cp313-win_amd64.whl", hash = "sha256:c7e72339f841b5a237ff14f7d3880ddd0fc7f98a1199e8c4327f9a4f478c1839", size = 1422766, upload-time = "2026-03-12T03:40:27.14Z" }, + { url = "https://files.pythonhosted.org/packages/59/4f/40a582c015f2d841ac24fed6390bd68f0fc896069ff3a886317959c9daf8/black-26.3.1-cp313-cp313-win_arm64.whl", hash = "sha256:afc622538b430aa4c8c853f7f63bc582b3b8030fd8c80b70fb5fa5b834e575c2", size = 1232140, upload-time = "2026-03-12T03:40:28.882Z" }, + { url = "https://files.pythonhosted.org/packages/d5/da/e36e27c9cebc1311b7579210df6f1c86e50f2d7143ae4fcf8a5017dc8809/black-26.3.1-cp314-cp314-macosx_10_15_x86_64.whl", hash = "sha256:2d6bfaf7fd0993b420bed691f20f9492d53ce9a2bcccea4b797d34e947318a78", size = 1889234, upload-time = "2026-03-12T03:40:30.964Z" }, + { url = "https://files.pythonhosted.org/packages/0e/7b/9871acf393f64a5fa33668c19350ca87177b181f44bb3d0c33b2d534f22c/black-26.3.1-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:f89f2ab047c76a9c03f78d0d66ca519e389519902fa27e7a91117ef7611c0568", size = 1720522, upload-time = "2026-03-12T03:40:32.346Z" }, + { url = "https://files.pythonhosted.org/packages/03/87/e766c7f2e90c07fb7586cc787c9ae6462b1eedab390191f2b7fc7f6170a9/black-26.3.1-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b07fc0dab849d24a80a29cfab8d8a19187d1c4685d8a5e6385a5ce323c1f015f", size = 1787824, upload-time = "2026-03-12T03:40:33.636Z" }, + { url = "https://files.pythonhosted.org/packages/ac/94/2424338fb2d1875e9e83eed4c8e9c67f6905ec25afd826a911aea2b02535/black-26.3.1-cp314-cp314-win_amd64.whl", hash = "sha256:0126ae5b7c09957da2bdbd91a9ba1207453feada9e9fe51992848658c6c8e01c", size = 1445855, upload-time = "2026-03-12T03:40:35.442Z" }, + { url = "https://files.pythonhosted.org/packages/86/43/0c3338bd928afb8ee7471f1a4eec3bdbe2245ccb4a646092a222e8669840/black-26.3.1-cp314-cp314-win_arm64.whl", hash = "sha256:92c0ec1f2cc149551a2b7b47efc32c866406b6891b0ee4625e95967c8f4acfb1", size = 1258109, upload-time = "2026-03-12T03:40:36.832Z" }, + { url = "https://files.pythonhosted.org/packages/8e/0d/52d98722666d6fc6c3dd4c76df339501d6efd40e0ff95e6186a7b7f0befd/black-26.3.1-py3-none-any.whl", hash = "sha256:2bd5aa94fc267d38bb21a70d7410a89f1a1d318841855f698746f8e7f51acd1b", size = 207542, upload-time = "2026-03-12T03:36:01.668Z" }, +] + +[[package]] +name = "certifi" +version = "2026.2.25" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/af/2d/7bf41579a8986e348fa033a31cdd0e4121114f6bce2457e8876010b092dd/certifi-2026.2.25.tar.gz", hash = "sha256:e887ab5cee78ea814d3472169153c2d12cd43b14bd03329a39a9c6e2e80bfba7", size = 155029, upload-time = "2026-02-25T02:54:17.342Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9a/3c/c17fb3ca2d9c3acff52e30b309f538586f9f5b9c9cf454f3845fc9af4881/certifi-2026.2.25-py3-none-any.whl", hash = "sha256:027692e4402ad994f1c42e52a4997a9763c646b73e4096e4d5d6db8af1d6f0fa", size = 153684, upload-time = "2026-02-25T02:54:15.766Z" }, +] + +[[package]] +name = "cffi" +version = "2.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pycparser", marker = "implementation_name != 'PyPy'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/eb/56/b1ba7935a17738ae8453301356628e8147c79dbb825bcbc73dc7401f9846/cffi-2.0.0.tar.gz", hash = "sha256:44d1b5909021139fe36001ae048dbdde8214afa20200eda0f64c068cac5d5529", size = 523588, upload-time = "2025-09-08T23:24:04.541Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ea/47/4f61023ea636104d4f16ab488e268b93008c3d0bb76893b1b31db1f96802/cffi-2.0.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:6d02d6655b0e54f54c4ef0b94eb6be0607b70853c45ce98bd278dc7de718be5d", size = 185271, upload-time = "2025-09-08T23:22:44.795Z" }, + { url = "https://files.pythonhosted.org/packages/df/a2/781b623f57358e360d62cdd7a8c681f074a71d445418a776eef0aadb4ab4/cffi-2.0.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8eca2a813c1cb7ad4fb74d368c2ffbbb4789d377ee5bb8df98373c2cc0dee76c", size = 181048, upload-time = "2025-09-08T23:22:45.938Z" }, + { url = "https://files.pythonhosted.org/packages/ff/df/a4f0fbd47331ceeba3d37c2e51e9dfc9722498becbeec2bd8bc856c9538a/cffi-2.0.0-cp312-cp312-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:21d1152871b019407d8ac3985f6775c079416c282e431a4da6afe7aefd2bccbe", size = 212529, upload-time = "2025-09-08T23:22:47.349Z" }, + { url = "https://files.pythonhosted.org/packages/d5/72/12b5f8d3865bf0f87cf1404d8c374e7487dcf097a1c91c436e72e6badd83/cffi-2.0.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:b21e08af67b8a103c71a250401c78d5e0893beff75e28c53c98f4de42f774062", size = 220097, upload-time = "2025-09-08T23:22:48.677Z" }, + { url = "https://files.pythonhosted.org/packages/c2/95/7a135d52a50dfa7c882ab0ac17e8dc11cec9d55d2c18dda414c051c5e69e/cffi-2.0.0-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:1e3a615586f05fc4065a8b22b8152f0c1b00cdbc60596d187c2a74f9e3036e4e", size = 207983, upload-time = "2025-09-08T23:22:50.06Z" }, + { url = "https://files.pythonhosted.org/packages/3a/c8/15cb9ada8895957ea171c62dc78ff3e99159ee7adb13c0123c001a2546c1/cffi-2.0.0-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:81afed14892743bbe14dacb9e36d9e0e504cd204e0b165062c488942b9718037", size = 206519, upload-time = "2025-09-08T23:22:51.364Z" }, + { url = "https://files.pythonhosted.org/packages/78/2d/7fa73dfa841b5ac06c7b8855cfc18622132e365f5b81d02230333ff26e9e/cffi-2.0.0-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:3e17ed538242334bf70832644a32a7aae3d83b57567f9fd60a26257e992b79ba", size = 219572, upload-time = "2025-09-08T23:22:52.902Z" }, + { url = "https://files.pythonhosted.org/packages/07/e0/267e57e387b4ca276b90f0434ff88b2c2241ad72b16d31836adddfd6031b/cffi-2.0.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:3925dd22fa2b7699ed2617149842d2e6adde22b262fcbfada50e3d195e4b3a94", size = 222963, upload-time = "2025-09-08T23:22:54.518Z" }, + { url = "https://files.pythonhosted.org/packages/b6/75/1f2747525e06f53efbd878f4d03bac5b859cbc11c633d0fb81432d98a795/cffi-2.0.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2c8f814d84194c9ea681642fd164267891702542f028a15fc97d4674b6206187", size = 221361, upload-time = "2025-09-08T23:22:55.867Z" }, + { url = "https://files.pythonhosted.org/packages/7b/2b/2b6435f76bfeb6bbf055596976da087377ede68df465419d192acf00c437/cffi-2.0.0-cp312-cp312-win32.whl", hash = "sha256:da902562c3e9c550df360bfa53c035b2f241fed6d9aef119048073680ace4a18", size = 172932, upload-time = "2025-09-08T23:22:57.188Z" }, + { url = "https://files.pythonhosted.org/packages/f8/ed/13bd4418627013bec4ed6e54283b1959cf6db888048c7cf4b4c3b5b36002/cffi-2.0.0-cp312-cp312-win_amd64.whl", hash = "sha256:da68248800ad6320861f129cd9c1bf96ca849a2771a59e0344e88681905916f5", size = 183557, upload-time = "2025-09-08T23:22:58.351Z" }, + { url = "https://files.pythonhosted.org/packages/95/31/9f7f93ad2f8eff1dbc1c3656d7ca5bfd8fb52c9d786b4dcf19b2d02217fa/cffi-2.0.0-cp312-cp312-win_arm64.whl", hash = "sha256:4671d9dd5ec934cb9a73e7ee9676f9362aba54f7f34910956b84d727b0d73fb6", size = 177762, upload-time = "2025-09-08T23:22:59.668Z" }, + { url = "https://files.pythonhosted.org/packages/4b/8d/a0a47a0c9e413a658623d014e91e74a50cdd2c423f7ccfd44086ef767f90/cffi-2.0.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:00bdf7acc5f795150faa6957054fbbca2439db2f775ce831222b66f192f03beb", size = 185230, upload-time = "2025-09-08T23:23:00.879Z" }, + { url = "https://files.pythonhosted.org/packages/4a/d2/a6c0296814556c68ee32009d9c2ad4f85f2707cdecfd7727951ec228005d/cffi-2.0.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:45d5e886156860dc35862657e1494b9bae8dfa63bf56796f2fb56e1679fc0bca", size = 181043, upload-time = "2025-09-08T23:23:02.231Z" }, + { url = "https://files.pythonhosted.org/packages/b0/1e/d22cc63332bd59b06481ceaac49d6c507598642e2230f201649058a7e704/cffi-2.0.0-cp313-cp313-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:07b271772c100085dd28b74fa0cd81c8fb1a3ba18b21e03d7c27f3436a10606b", size = 212446, upload-time = "2025-09-08T23:23:03.472Z" }, + { url = "https://files.pythonhosted.org/packages/a9/f5/a2c23eb03b61a0b8747f211eb716446c826ad66818ddc7810cc2cc19b3f2/cffi-2.0.0-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:d48a880098c96020b02d5a1f7d9251308510ce8858940e6fa99ece33f610838b", size = 220101, upload-time = "2025-09-08T23:23:04.792Z" }, + { url = "https://files.pythonhosted.org/packages/f2/7f/e6647792fc5850d634695bc0e6ab4111ae88e89981d35ac269956605feba/cffi-2.0.0-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:f93fd8e5c8c0a4aa1f424d6173f14a892044054871c771f8566e4008eaa359d2", size = 207948, upload-time = "2025-09-08T23:23:06.127Z" }, + { url = "https://files.pythonhosted.org/packages/cb/1e/a5a1bd6f1fb30f22573f76533de12a00bf274abcdc55c8edab639078abb6/cffi-2.0.0-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:dd4f05f54a52fb558f1ba9f528228066954fee3ebe629fc1660d874d040ae5a3", size = 206422, upload-time = "2025-09-08T23:23:07.753Z" }, + { url = "https://files.pythonhosted.org/packages/98/df/0a1755e750013a2081e863e7cd37e0cdd02664372c754e5560099eb7aa44/cffi-2.0.0-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:c8d3b5532fc71b7a77c09192b4a5a200ea992702734a2e9279a37f2478236f26", size = 219499, upload-time = "2025-09-08T23:23:09.648Z" }, + { url = "https://files.pythonhosted.org/packages/50/e1/a969e687fcf9ea58e6e2a928ad5e2dd88cc12f6f0ab477e9971f2309b57c/cffi-2.0.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:d9b29c1f0ae438d5ee9acb31cadee00a58c46cc9c0b2f9038c6b0b3470877a8c", size = 222928, upload-time = "2025-09-08T23:23:10.928Z" }, + { url = "https://files.pythonhosted.org/packages/36/54/0362578dd2c9e557a28ac77698ed67323ed5b9775ca9d3fe73fe191bb5d8/cffi-2.0.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:6d50360be4546678fc1b79ffe7a66265e28667840010348dd69a314145807a1b", size = 221302, upload-time = "2025-09-08T23:23:12.42Z" }, + { url = "https://files.pythonhosted.org/packages/eb/6d/bf9bda840d5f1dfdbf0feca87fbdb64a918a69bca42cfa0ba7b137c48cb8/cffi-2.0.0-cp313-cp313-win32.whl", hash = "sha256:74a03b9698e198d47562765773b4a8309919089150a0bb17d829ad7b44b60d27", size = 172909, upload-time = "2025-09-08T23:23:14.32Z" }, + { url = "https://files.pythonhosted.org/packages/37/18/6519e1ee6f5a1e579e04b9ddb6f1676c17368a7aba48299c3759bbc3c8b3/cffi-2.0.0-cp313-cp313-win_amd64.whl", hash = "sha256:19f705ada2530c1167abacb171925dd886168931e0a7b78f5bffcae5c6b5be75", size = 183402, upload-time = "2025-09-08T23:23:15.535Z" }, + { url = "https://files.pythonhosted.org/packages/cb/0e/02ceeec9a7d6ee63bb596121c2c8e9b3a9e150936f4fbef6ca1943e6137c/cffi-2.0.0-cp313-cp313-win_arm64.whl", hash = "sha256:256f80b80ca3853f90c21b23ee78cd008713787b1b1e93eae9f3d6a7134abd91", size = 177780, upload-time = "2025-09-08T23:23:16.761Z" }, + { url = "https://files.pythonhosted.org/packages/92/c4/3ce07396253a83250ee98564f8d7e9789fab8e58858f35d07a9a2c78de9f/cffi-2.0.0-cp314-cp314-macosx_10_13_x86_64.whl", hash = "sha256:fc33c5141b55ed366cfaad382df24fe7dcbc686de5be719b207bb248e3053dc5", size = 185320, upload-time = "2025-09-08T23:23:18.087Z" }, + { url = "https://files.pythonhosted.org/packages/59/dd/27e9fa567a23931c838c6b02d0764611c62290062a6d4e8ff7863daf9730/cffi-2.0.0-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:c654de545946e0db659b3400168c9ad31b5d29593291482c43e3564effbcee13", size = 181487, upload-time = "2025-09-08T23:23:19.622Z" }, + { url = "https://files.pythonhosted.org/packages/d6/43/0e822876f87ea8a4ef95442c3d766a06a51fc5298823f884ef87aaad168c/cffi-2.0.0-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:24b6f81f1983e6df8db3adc38562c83f7d4a0c36162885ec7f7b77c7dcbec97b", size = 220049, upload-time = "2025-09-08T23:23:20.853Z" }, + { url = "https://files.pythonhosted.org/packages/b4/89/76799151d9c2d2d1ead63c2429da9ea9d7aac304603de0c6e8764e6e8e70/cffi-2.0.0-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:12873ca6cb9b0f0d3a0da705d6086fe911591737a59f28b7936bdfed27c0d47c", size = 207793, upload-time = "2025-09-08T23:23:22.08Z" }, + { url = "https://files.pythonhosted.org/packages/bb/dd/3465b14bb9e24ee24cb88c9e3730f6de63111fffe513492bf8c808a3547e/cffi-2.0.0-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:d9b97165e8aed9272a6bb17c01e3cc5871a594a446ebedc996e2397a1c1ea8ef", size = 206300, upload-time = "2025-09-08T23:23:23.314Z" }, + { url = "https://files.pythonhosted.org/packages/47/d9/d83e293854571c877a92da46fdec39158f8d7e68da75bf73581225d28e90/cffi-2.0.0-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:afb8db5439b81cf9c9d0c80404b60c3cc9c3add93e114dcae767f1477cb53775", size = 219244, upload-time = "2025-09-08T23:23:24.541Z" }, + { url = "https://files.pythonhosted.org/packages/2b/0f/1f177e3683aead2bb00f7679a16451d302c436b5cbf2505f0ea8146ef59e/cffi-2.0.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:737fe7d37e1a1bffe70bd5754ea763a62a066dc5913ca57e957824b72a85e205", size = 222828, upload-time = "2025-09-08T23:23:26.143Z" }, + { url = "https://files.pythonhosted.org/packages/c6/0f/cafacebd4b040e3119dcb32fed8bdef8dfe94da653155f9d0b9dc660166e/cffi-2.0.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:38100abb9d1b1435bc4cc340bb4489635dc2f0da7456590877030c9b3d40b0c1", size = 220926, upload-time = "2025-09-08T23:23:27.873Z" }, + { url = "https://files.pythonhosted.org/packages/3e/aa/df335faa45b395396fcbc03de2dfcab242cd61a9900e914fe682a59170b1/cffi-2.0.0-cp314-cp314-win32.whl", hash = "sha256:087067fa8953339c723661eda6b54bc98c5625757ea62e95eb4898ad5e776e9f", size = 175328, upload-time = "2025-09-08T23:23:44.61Z" }, + { url = "https://files.pythonhosted.org/packages/bb/92/882c2d30831744296ce713f0feb4c1cd30f346ef747b530b5318715cc367/cffi-2.0.0-cp314-cp314-win_amd64.whl", hash = "sha256:203a48d1fb583fc7d78a4c6655692963b860a417c0528492a6bc21f1aaefab25", size = 185650, upload-time = "2025-09-08T23:23:45.848Z" }, + { url = "https://files.pythonhosted.org/packages/9f/2c/98ece204b9d35a7366b5b2c6539c350313ca13932143e79dc133ba757104/cffi-2.0.0-cp314-cp314-win_arm64.whl", hash = "sha256:dbd5c7a25a7cb98f5ca55d258b103a2054f859a46ae11aaf23134f9cc0d356ad", size = 180687, upload-time = "2025-09-08T23:23:47.105Z" }, + { url = "https://files.pythonhosted.org/packages/3e/61/c768e4d548bfa607abcda77423448df8c471f25dbe64fb2ef6d555eae006/cffi-2.0.0-cp314-cp314t-macosx_10_13_x86_64.whl", hash = "sha256:9a67fc9e8eb39039280526379fb3a70023d77caec1852002b4da7e8b270c4dd9", size = 188773, upload-time = "2025-09-08T23:23:29.347Z" }, + { url = "https://files.pythonhosted.org/packages/2c/ea/5f76bce7cf6fcd0ab1a1058b5af899bfbef198bea4d5686da88471ea0336/cffi-2.0.0-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:7a66c7204d8869299919db4d5069a82f1561581af12b11b3c9f48c584eb8743d", size = 185013, upload-time = "2025-09-08T23:23:30.63Z" }, + { url = "https://files.pythonhosted.org/packages/be/b4/c56878d0d1755cf9caa54ba71e5d049479c52f9e4afc230f06822162ab2f/cffi-2.0.0-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:7cc09976e8b56f8cebd752f7113ad07752461f48a58cbba644139015ac24954c", size = 221593, upload-time = "2025-09-08T23:23:31.91Z" }, + { url = "https://files.pythonhosted.org/packages/e0/0d/eb704606dfe8033e7128df5e90fee946bbcb64a04fcdaa97321309004000/cffi-2.0.0-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:92b68146a71df78564e4ef48af17551a5ddd142e5190cdf2c5624d0c3ff5b2e8", size = 209354, upload-time = "2025-09-08T23:23:33.214Z" }, + { url = "https://files.pythonhosted.org/packages/d8/19/3c435d727b368ca475fb8742ab97c9cb13a0de600ce86f62eab7fa3eea60/cffi-2.0.0-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:b1e74d11748e7e98e2f426ab176d4ed720a64412b6a15054378afdb71e0f37dc", size = 208480, upload-time = "2025-09-08T23:23:34.495Z" }, + { url = "https://files.pythonhosted.org/packages/d0/44/681604464ed9541673e486521497406fadcc15b5217c3e326b061696899a/cffi-2.0.0-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:28a3a209b96630bca57cce802da70c266eb08c6e97e5afd61a75611ee6c64592", size = 221584, upload-time = "2025-09-08T23:23:36.096Z" }, + { url = "https://files.pythonhosted.org/packages/25/8e/342a504ff018a2825d395d44d63a767dd8ebc927ebda557fecdaca3ac33a/cffi-2.0.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:7553fb2090d71822f02c629afe6042c299edf91ba1bf94951165613553984512", size = 224443, upload-time = "2025-09-08T23:23:37.328Z" }, + { url = "https://files.pythonhosted.org/packages/e1/5e/b666bacbbc60fbf415ba9988324a132c9a7a0448a9a8f125074671c0f2c3/cffi-2.0.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:6c6c373cfc5c83a975506110d17457138c8c63016b563cc9ed6e056a82f13ce4", size = 223437, upload-time = "2025-09-08T23:23:38.945Z" }, + { url = "https://files.pythonhosted.org/packages/a0/1d/ec1a60bd1a10daa292d3cd6bb0b359a81607154fb8165f3ec95fe003b85c/cffi-2.0.0-cp314-cp314t-win32.whl", hash = "sha256:1fc9ea04857caf665289b7a75923f2c6ed559b8298a1b8c49e59f7dd95c8481e", size = 180487, upload-time = "2025-09-08T23:23:40.423Z" }, + { url = "https://files.pythonhosted.org/packages/bf/41/4c1168c74fac325c0c8156f04b6749c8b6a8f405bbf91413ba088359f60d/cffi-2.0.0-cp314-cp314t-win_amd64.whl", hash = "sha256:d68b6cef7827e8641e8ef16f4494edda8b36104d79773a334beaa1e3521430f6", size = 191726, upload-time = "2025-09-08T23:23:41.742Z" }, + { url = "https://files.pythonhosted.org/packages/ae/3a/dbeec9d1ee0844c679f6bb5d6ad4e9f198b1224f4e7a32825f47f6192b0c/cffi-2.0.0-cp314-cp314t-win_arm64.whl", hash = "sha256:0a1527a803f0a659de1af2e1fd700213caba79377e27e4693648c2923da066f9", size = 184195, upload-time = "2025-09-08T23:23:43.004Z" }, +] + +[[package]] +name = "charset-normalizer" +version = "3.4.7" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/e7/a1/67fe25fac3c7642725500a3f6cfe5821ad557c3abb11c9d20d12c7008d3e/charset_normalizer-3.4.7.tar.gz", hash = "sha256:ae89db9e5f98a11a4bf50407d4363e7b09b31e55bc117b4f7d80aab97ba009e5", size = 144271, upload-time = "2026-04-02T09:28:39.342Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0c/eb/4fc8d0a7110eb5fc9cc161723a34a8a6c200ce3b4fbf681bc86feee22308/charset_normalizer-3.4.7-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:eca9705049ad3c7345d574e3510665cb2cf844c2f2dcfe675332677f081cbd46", size = 311328, upload-time = "2026-04-02T09:26:24.331Z" }, + { url = "https://files.pythonhosted.org/packages/f8/e3/0fadc706008ac9d7b9b5be6dc767c05f9d3e5df51744ce4cc9605de7b9f4/charset_normalizer-3.4.7-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:6178f72c5508bfc5fd446a5905e698c6212932f25bcdd4b47a757a50605a90e2", size = 208061, upload-time = "2026-04-02T09:26:25.568Z" }, + { url = "https://files.pythonhosted.org/packages/42/f0/3dd1045c47f4a4604df85ec18ad093912ae1344ac706993aff91d38773a2/charset_normalizer-3.4.7-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:e1421b502d83040e6d7fb2fb18dff63957f720da3d77b2fbd3187ceb63755d7b", size = 229031, upload-time = "2026-04-02T09:26:26.865Z" }, + { url = "https://files.pythonhosted.org/packages/dc/67/675a46eb016118a2fbde5a277a5d15f4f69d5f3f5f338e5ee2f8948fcf43/charset_normalizer-3.4.7-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:edac0f1ab77644605be2cbba52e6b7f630731fc42b34cb0f634be1a6eface56a", size = 225239, upload-time = "2026-04-02T09:26:28.044Z" }, + { url = "https://files.pythonhosted.org/packages/4b/f8/d0118a2f5f23b02cd166fa385c60f9b0d4f9194f574e2b31cef350ad7223/charset_normalizer-3.4.7-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5649fd1c7bade02f320a462fdefd0b4bd3ce036065836d4f42e0de958038e116", size = 216589, upload-time = "2026-04-02T09:26:29.239Z" }, + { url = "https://files.pythonhosted.org/packages/b1/f1/6d2b0b261b6c4ceef0fcb0d17a01cc5bc53586c2d4796fa04b5c540bc13d/charset_normalizer-3.4.7-cp312-cp312-manylinux_2_31_armv7l.whl", hash = "sha256:203104ed3e428044fd943bc4bf45fa73c0730391f9621e37fe39ecf477b128cb", size = 202733, upload-time = "2026-04-02T09:26:30.5Z" }, + { url = "https://files.pythonhosted.org/packages/6f/c0/7b1f943f7e87cc3db9626ba17807d042c38645f0a1d4415c7a14afb5591f/charset_normalizer-3.4.7-cp312-cp312-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:298930cec56029e05497a76988377cbd7457ba864beeea92ad7e844fe74cd1f1", size = 212652, upload-time = "2026-04-02T09:26:31.709Z" }, + { url = "https://files.pythonhosted.org/packages/38/dd/5a9ab159fe45c6e72079398f277b7d2b523e7f716acc489726115a910097/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:708838739abf24b2ceb208d0e22403dd018faeef86ddac04319a62ae884c4f15", size = 211229, upload-time = "2026-04-02T09:26:33.282Z" }, + { url = "https://files.pythonhosted.org/packages/d5/ff/531a1cad5ca855d1c1a8b69cb71abfd6d85c0291580146fda7c82857caa1/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:0f7eb884681e3938906ed0434f20c63046eacd0111c4ba96f27b76084cd679f5", size = 203552, upload-time = "2026-04-02T09:26:34.845Z" }, + { url = "https://files.pythonhosted.org/packages/c1/4c/a5fb52d528a8ca41f7598cb619409ece30a169fbdf9cdce592e53b46c3a6/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:4dc1e73c36828f982bfe79fadf5919923f8a6f4df2860804db9a98c48824ce8d", size = 230806, upload-time = "2026-04-02T09:26:36.152Z" }, + { url = "https://files.pythonhosted.org/packages/59/7a/071feed8124111a32b316b33ae4de83d36923039ef8cf48120266844285b/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_riscv64.whl", hash = "sha256:aed52fea0513bac0ccde438c188c8a471c4e0f457c2dd20cdbf6ea7a450046c7", size = 212316, upload-time = "2026-04-02T09:26:37.672Z" }, + { url = "https://files.pythonhosted.org/packages/fd/35/f7dba3994312d7ba508e041eaac39a36b120f32d4c8662b8814dab876431/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:fea24543955a6a729c45a73fe90e08c743f0b3334bbf3201e6c4bc1b0c7fa464", size = 227274, upload-time = "2026-04-02T09:26:38.93Z" }, + { url = "https://files.pythonhosted.org/packages/8a/2d/a572df5c9204ab7688ec1edc895a73ebded3b023bb07364710b05dd1c9be/charset_normalizer-3.4.7-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:bb6d88045545b26da47aa879dd4a89a71d1dce0f0e549b1abcb31dfe4a8eac49", size = 218468, upload-time = "2026-04-02T09:26:40.17Z" }, + { url = "https://files.pythonhosted.org/packages/86/eb/890922a8b03a568ca2f336c36585a4713c55d4d67bf0f0c78924be6315ca/charset_normalizer-3.4.7-cp312-cp312-win32.whl", hash = "sha256:2257141f39fe65a3fdf38aeccae4b953e5f3b3324f4ff0daf9f15b8518666a2c", size = 148460, upload-time = "2026-04-02T09:26:41.416Z" }, + { url = "https://files.pythonhosted.org/packages/35/d9/0e7dffa06c5ab081f75b1b786f0aefc88365825dfcd0ac544bdb7b2b6853/charset_normalizer-3.4.7-cp312-cp312-win_amd64.whl", hash = "sha256:5ed6ab538499c8644b8a3e18debabcd7ce684f3fa91cf867521a7a0279cab2d6", size = 159330, upload-time = "2026-04-02T09:26:42.554Z" }, + { url = "https://files.pythonhosted.org/packages/9e/5d/481bcc2a7c88ea6b0878c299547843b2521ccbc40980cb406267088bc701/charset_normalizer-3.4.7-cp312-cp312-win_arm64.whl", hash = "sha256:56be790f86bfb2c98fb742ce566dfb4816e5a83384616ab59c49e0604d49c51d", size = 147828, upload-time = "2026-04-02T09:26:44.075Z" }, + { url = "https://files.pythonhosted.org/packages/c1/3b/66777e39d3ae1ddc77ee606be4ec6d8cbd4c801f65e5a1b6f2b11b8346dd/charset_normalizer-3.4.7-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:f496c9c3cc02230093d8330875c4c3cdfc3b73612a5fd921c65d39cbcef08063", size = 309627, upload-time = "2026-04-02T09:26:45.198Z" }, + { url = "https://files.pythonhosted.org/packages/2e/4e/b7f84e617b4854ade48a1b7915c8ccfadeba444d2a18c291f696e37f0d3b/charset_normalizer-3.4.7-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:0ea948db76d31190bf08bd371623927ee1339d5f2a0b4b1b4a4439a65298703c", size = 207008, upload-time = "2026-04-02T09:26:46.824Z" }, + { url = "https://files.pythonhosted.org/packages/c4/bb/ec73c0257c9e11b268f018f068f5d00aa0ef8c8b09f7753ebd5f2880e248/charset_normalizer-3.4.7-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:a277ab8928b9f299723bc1a2dabb1265911b1a76341f90a510368ca44ad9ab66", size = 228303, upload-time = "2026-04-02T09:26:48.397Z" }, + { url = "https://files.pythonhosted.org/packages/85/fb/32d1f5033484494619f701e719429c69b766bfc4dbc61aa9e9c8c166528b/charset_normalizer-3.4.7-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:3bec022aec2c514d9cf199522a802bd007cd588ab17ab2525f20f9c34d067c18", size = 224282, upload-time = "2026-04-02T09:26:49.684Z" }, + { url = "https://files.pythonhosted.org/packages/fa/07/330e3a0dda4c404d6da83b327270906e9654a24f6c546dc886a0eb0ffb23/charset_normalizer-3.4.7-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:e044c39e41b92c845bc815e5ae4230804e8e7bc29e399b0437d64222d92809dd", size = 215595, upload-time = "2026-04-02T09:26:50.915Z" }, + { url = "https://files.pythonhosted.org/packages/e3/7c/fc890655786e423f02556e0216d4b8c6bcb6bdfa890160dc66bf52dee468/charset_normalizer-3.4.7-cp313-cp313-manylinux_2_31_armv7l.whl", hash = "sha256:f495a1652cf3fbab2eb0639776dad966c2fb874d79d87ca07f9d5f059b8bd215", size = 201986, upload-time = "2026-04-02T09:26:52.197Z" }, + { url = "https://files.pythonhosted.org/packages/d8/97/bfb18b3db2aed3b90cf54dc292ad79fdd5ad65c4eae454099475cbeadd0d/charset_normalizer-3.4.7-cp313-cp313-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:e712b419df8ba5e42b226c510472b37bd57b38e897d3eca5e8cfd410a29fa859", size = 211711, upload-time = "2026-04-02T09:26:53.49Z" }, + { url = "https://files.pythonhosted.org/packages/6f/a5/a581c13798546a7fd557c82614a5c65a13df2157e9ad6373166d2a3e645d/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:7804338df6fcc08105c7745f1502ba68d900f45fd770d5bdd5288ddccb8a42d8", size = 210036, upload-time = "2026-04-02T09:26:54.975Z" }, + { url = "https://files.pythonhosted.org/packages/8c/bf/b3ab5bcb478e4193d517644b0fb2bf5497fbceeaa7a1bc0f4d5b50953861/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:481551899c856c704d58119b5025793fa6730adda3571971af568f66d2424bb5", size = 202998, upload-time = "2026-04-02T09:26:56.303Z" }, + { url = "https://files.pythonhosted.org/packages/e7/4e/23efd79b65d314fa320ec6017b4b5834d5c12a58ba4610aa353af2e2f577/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f59099f9b66f0d7145115e6f80dd8b1d847176df89b234a5a6b3f00437aa0832", size = 230056, upload-time = "2026-04-02T09:26:57.554Z" }, + { url = "https://files.pythonhosted.org/packages/b9/9f/1e1941bc3f0e01df116e68dc37a55c4d249df5e6fa77f008841aef68264f/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_riscv64.whl", hash = "sha256:f59ad4c0e8f6bba240a9bb85504faa1ab438237199d4cce5f622761507b8f6a6", size = 211537, upload-time = "2026-04-02T09:26:58.843Z" }, + { url = "https://files.pythonhosted.org/packages/80/0f/088cbb3020d44428964a6c97fe1edfb1b9550396bf6d278330281e8b709c/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:3dedcc22d73ec993f42055eff4fcfed9318d1eeb9a6606c55892a26964964e48", size = 226176, upload-time = "2026-04-02T09:27:00.437Z" }, + { url = "https://files.pythonhosted.org/packages/6a/9f/130394f9bbe06f4f63e22641d32fc9b202b7e251c9aef4db044324dac493/charset_normalizer-3.4.7-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:64f02c6841d7d83f832cd97ccf8eb8a906d06eb95d5276069175c696b024b60a", size = 217723, upload-time = "2026-04-02T09:27:02.021Z" }, + { url = "https://files.pythonhosted.org/packages/73/55/c469897448a06e49f8fa03f6caae97074fde823f432a98f979cc42b90e69/charset_normalizer-3.4.7-cp313-cp313-win32.whl", hash = "sha256:4042d5c8f957e15221d423ba781e85d553722fc4113f523f2feb7b188cc34c5e", size = 148085, upload-time = "2026-04-02T09:27:03.192Z" }, + { url = "https://files.pythonhosted.org/packages/5d/78/1b74c5bbb3f99b77a1715c91b3e0b5bdb6fe302d95ace4f5b1bec37b0167/charset_normalizer-3.4.7-cp313-cp313-win_amd64.whl", hash = "sha256:3946fa46a0cf3e4c8cb1cc52f56bb536310d34f25f01ca9b6c16afa767dab110", size = 158819, upload-time = "2026-04-02T09:27:04.454Z" }, + { url = "https://files.pythonhosted.org/packages/68/86/46bd42279d323deb8687c4a5a811fd548cb7d1de10cf6535d099877a9a9f/charset_normalizer-3.4.7-cp313-cp313-win_arm64.whl", hash = "sha256:80d04837f55fc81da168b98de4f4b797ef007fc8a79ab71c6ec9bc4dd662b15b", size = 147915, upload-time = "2026-04-02T09:27:05.971Z" }, + { url = "https://files.pythonhosted.org/packages/97/c8/c67cb8c70e19ef1960b97b22ed2a1567711de46c4ddf19799923adc836c2/charset_normalizer-3.4.7-cp314-cp314-macosx_10_15_universal2.whl", hash = "sha256:c36c333c39be2dbca264d7803333c896ab8fa7d4d6f0ab7edb7dfd7aea6e98c0", size = 309234, upload-time = "2026-04-02T09:27:07.194Z" }, + { url = "https://files.pythonhosted.org/packages/99/85/c091fdee33f20de70d6c8b522743b6f831a2f1cd3ff86de4c6a827c48a76/charset_normalizer-3.4.7-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:1c2aed2e5e41f24ea8ef1590b8e848a79b56f3a5564a65ceec43c9d692dc7d8a", size = 208042, upload-time = "2026-04-02T09:27:08.749Z" }, + { url = "https://files.pythonhosted.org/packages/87/1c/ab2ce611b984d2fd5d86a5a8a19c1ae26acac6bad967da4967562c75114d/charset_normalizer-3.4.7-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:54523e136b8948060c0fa0bc7b1b50c32c186f2fceee897a495406bb6e311d2b", size = 228706, upload-time = "2026-04-02T09:27:09.951Z" }, + { url = "https://files.pythonhosted.org/packages/a8/29/2b1d2cb00bf085f59d29eb773ce58ec2d325430f8c216804a0a5cd83cbca/charset_normalizer-3.4.7-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:715479b9a2802ecac752a3b0efa2b0b60285cf962ee38414211abdfccc233b41", size = 224727, upload-time = "2026-04-02T09:27:11.175Z" }, + { url = "https://files.pythonhosted.org/packages/47/5c/032c2d5a07fe4d4855fea851209cca2b6f03ebeb6d4e3afdb3358386a684/charset_normalizer-3.4.7-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:bd6c2a1c7573c64738d716488d2cdd3c00e340e4835707d8fdb8dc1a66ef164e", size = 215882, upload-time = "2026-04-02T09:27:12.446Z" }, + { url = "https://files.pythonhosted.org/packages/2c/c2/356065d5a8b78ed04499cae5f339f091946a6a74f91e03476c33f0ab7100/charset_normalizer-3.4.7-cp314-cp314-manylinux_2_31_armv7l.whl", hash = "sha256:c45e9440fb78f8ddabcf714b68f936737a121355bf59f3907f4e17721b9d1aae", size = 200860, upload-time = "2026-04-02T09:27:13.721Z" }, + { url = "https://files.pythonhosted.org/packages/0c/cd/a32a84217ced5039f53b29f460962abb2d4420def55afabe45b1c3c7483d/charset_normalizer-3.4.7-cp314-cp314-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:3534e7dcbdcf757da6b85a0bbf5b6868786d5982dd959b065e65481644817a18", size = 211564, upload-time = "2026-04-02T09:27:15.272Z" }, + { url = "https://files.pythonhosted.org/packages/44/86/58e6f13ce26cc3b8f4a36b94a0f22ae2f00a72534520f4ae6857c4b81f89/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:e8ac484bf18ce6975760921bb6148041faa8fef0547200386ea0b52b5d27bf7b", size = 211276, upload-time = "2026-04-02T09:27:16.834Z" }, + { url = "https://files.pythonhosted.org/packages/8f/fe/d17c32dc72e17e155e06883efa84514ca375f8a528ba2546bee73fc4df81/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:a5fe03b42827c13cdccd08e6c0247b6a6d4b5e3cdc53fd1749f5896adcdc2356", size = 201238, upload-time = "2026-04-02T09:27:18.229Z" }, + { url = "https://files.pythonhosted.org/packages/6a/29/f33daa50b06525a237451cdb6c69da366c381a3dadcd833fa5676bc468b3/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:2d6eb928e13016cea4f1f21d1e10c1cebd5a421bc57ddf5b1142ae3f86824fab", size = 230189, upload-time = "2026-04-02T09:27:19.445Z" }, + { url = "https://files.pythonhosted.org/packages/b6/6e/52c84015394a6a0bdcd435210a7e944c5f94ea1055f5cc5d56c5fe368e7b/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_riscv64.whl", hash = "sha256:e74327fb75de8986940def6e8dee4f127cc9752bee7355bb323cc5b2659b6d46", size = 211352, upload-time = "2026-04-02T09:27:20.79Z" }, + { url = "https://files.pythonhosted.org/packages/8c/d7/4353be581b373033fb9198bf1da3cf8f09c1082561e8e922aa7b39bf9fe8/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:d6038d37043bced98a66e68d3aa2b6a35505dc01328cd65217cefe82f25def44", size = 227024, upload-time = "2026-04-02T09:27:22.063Z" }, + { url = "https://files.pythonhosted.org/packages/30/45/99d18aa925bd1740098ccd3060e238e21115fffbfdcb8f3ece837d0ace6c/charset_normalizer-3.4.7-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:7579e913a5339fb8fa133f6bbcfd8e6749696206cf05acdbdca71a1b436d8e72", size = 217869, upload-time = "2026-04-02T09:27:23.486Z" }, + { url = "https://files.pythonhosted.org/packages/5c/05/5ee478aa53f4bb7996482153d4bfe1b89e0f087f0ab6b294fcf92d595873/charset_normalizer-3.4.7-cp314-cp314-win32.whl", hash = "sha256:5b77459df20e08151cd6f8b9ef8ef1f961ef73d85c21a555c7eed5b79410ec10", size = 148541, upload-time = "2026-04-02T09:27:25.146Z" }, + { url = "https://files.pythonhosted.org/packages/48/77/72dcb0921b2ce86420b2d79d454c7022bf5be40202a2a07906b9f2a35c97/charset_normalizer-3.4.7-cp314-cp314-win_amd64.whl", hash = "sha256:92a0a01ead5e668468e952e4238cccd7c537364eb7d851ab144ab6627dbbe12f", size = 159634, upload-time = "2026-04-02T09:27:26.642Z" }, + { url = "https://files.pythonhosted.org/packages/c6/a3/c2369911cd72f02386e4e340770f6e158c7980267da16af8f668217abaa0/charset_normalizer-3.4.7-cp314-cp314-win_arm64.whl", hash = "sha256:67f6279d125ca0046a7fd386d01b311c6363844deac3e5b069b514ba3e63c246", size = 148384, upload-time = "2026-04-02T09:27:28.271Z" }, + { url = "https://files.pythonhosted.org/packages/94/09/7e8a7f73d24dba1f0035fbbf014d2c36828fc1bf9c88f84093e57d315935/charset_normalizer-3.4.7-cp314-cp314t-macosx_10_15_universal2.whl", hash = "sha256:effc3f449787117233702311a1b7d8f59cba9ced946ba727bdc329ec69028e24", size = 330133, upload-time = "2026-04-02T09:27:29.474Z" }, + { url = "https://files.pythonhosted.org/packages/8d/da/96975ddb11f8e977f706f45cddd8540fd8242f71ecdb5d18a80723dcf62c/charset_normalizer-3.4.7-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:fbccdc05410c9ee21bbf16a35f4c1d16123dcdeb8a1d38f33654fa21d0234f79", size = 216257, upload-time = "2026-04-02T09:27:30.793Z" }, + { url = "https://files.pythonhosted.org/packages/e5/e8/1d63bf8ef2d388e95c64b2098f45f84758f6d102a087552da1485912637b/charset_normalizer-3.4.7-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:733784b6d6def852c814bce5f318d25da2ee65dd4839a0718641c696e09a2960", size = 234851, upload-time = "2026-04-02T09:27:32.44Z" }, + { url = "https://files.pythonhosted.org/packages/9b/40/e5ff04233e70da2681fa43969ad6f66ca5611d7e669be0246c4c7aaf6dc8/charset_normalizer-3.4.7-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:a89c23ef8d2c6b27fd200a42aa4ac72786e7c60d40efdc76e6011260b6e949c4", size = 233393, upload-time = "2026-04-02T09:27:34.03Z" }, + { url = "https://files.pythonhosted.org/packages/be/c1/06c6c49d5a5450f76899992f1ee40b41d076aee9279b49cf9974d2f313d5/charset_normalizer-3.4.7-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6c114670c45346afedc0d947faf3c7f701051d2518b943679c8ff88befe14f8e", size = 223251, upload-time = "2026-04-02T09:27:35.369Z" }, + { url = "https://files.pythonhosted.org/packages/2b/9f/f2ff16fb050946169e3e1f82134d107e5d4ae72647ec8a1b1446c148480f/charset_normalizer-3.4.7-cp314-cp314t-manylinux_2_31_armv7l.whl", hash = "sha256:a180c5e59792af262bf263b21a3c49353f25945d8d9f70628e73de370d55e1e1", size = 206609, upload-time = "2026-04-02T09:27:36.661Z" }, + { url = "https://files.pythonhosted.org/packages/69/d5/a527c0cd8d64d2eab7459784fb4169a0ac76e5a6fc5237337982fd61347e/charset_normalizer-3.4.7-cp314-cp314t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:3c9a494bc5ec77d43cea229c4f6db1e4d8fe7e1bbffa8b6f0f0032430ff8ab44", size = 220014, upload-time = "2026-04-02T09:27:38.019Z" }, + { url = "https://files.pythonhosted.org/packages/7e/80/8a7b8104a3e203074dc9aa2c613d4b726c0e136bad1cc734594b02867972/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:8d828b6667a32a728a1ad1d93957cdf37489c57b97ae6c4de2860fa749b8fc1e", size = 218979, upload-time = "2026-04-02T09:27:39.37Z" }, + { url = "https://files.pythonhosted.org/packages/02/9a/b759b503d507f375b2b5c153e4d2ee0a75aa215b7f2489cf314f4541f2c0/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:cf1493cd8607bec4d8a7b9b004e699fcf8f9103a9284cc94962cb73d20f9d4a3", size = 209238, upload-time = "2026-04-02T09:27:40.722Z" }, + { url = "https://files.pythonhosted.org/packages/c2/4e/0f3f5d47b86bdb79256e7290b26ac847a2832d9a4033f7eb2cd4bcf4bb5b/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:0c96c3b819b5c3e9e165495db84d41914d6894d55181d2d108cc1a69bfc9cce0", size = 236110, upload-time = "2026-04-02T09:27:42.33Z" }, + { url = "https://files.pythonhosted.org/packages/96/23/bce28734eb3ed2c91dcf93abeb8a5cf393a7b2749725030bb630e554fdd8/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_riscv64.whl", hash = "sha256:752a45dc4a6934060b3b0dab47e04edc3326575f82be64bc4fc293914566503e", size = 219824, upload-time = "2026-04-02T09:27:43.924Z" }, + { url = "https://files.pythonhosted.org/packages/2c/6f/6e897c6984cc4d41af319b077f2f600fc8214eb2fe2d6bcb79141b882400/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:8778f0c7a52e56f75d12dae53ae320fae900a8b9b4164b981b9c5ce059cd1fcb", size = 233103, upload-time = "2026-04-02T09:27:45.348Z" }, + { url = "https://files.pythonhosted.org/packages/76/22/ef7bd0fe480a0ae9b656189ec00744b60933f68b4f42a7bb06589f6f576a/charset_normalizer-3.4.7-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:ce3412fbe1e31eb81ea42f4169ed94861c56e643189e1e75f0041f3fe7020abe", size = 225194, upload-time = "2026-04-02T09:27:46.706Z" }, + { url = "https://files.pythonhosted.org/packages/c5/a7/0e0ab3e0b5bc1219bd80a6a0d4d72ca74d9250cb2382b7c699c147e06017/charset_normalizer-3.4.7-cp314-cp314t-win32.whl", hash = "sha256:c03a41a8784091e67a39648f70c5f97b5b6a37f216896d44d2cdcb82615339a0", size = 159827, upload-time = "2026-04-02T09:27:48.053Z" }, + { url = "https://files.pythonhosted.org/packages/7a/1d/29d32e0fb40864b1f878c7f5a0b343ae676c6e2b271a2d55cc3a152391da/charset_normalizer-3.4.7-cp314-cp314t-win_amd64.whl", hash = "sha256:03853ed82eeebbce3c2abfdbc98c96dc205f32a79627688ac9a27370ea61a49c", size = 174168, upload-time = "2026-04-02T09:27:49.795Z" }, + { url = "https://files.pythonhosted.org/packages/de/32/d92444ad05c7a6e41fb2036749777c163baf7a0301a040cb672d6b2b1ae9/charset_normalizer-3.4.7-cp314-cp314t-win_arm64.whl", hash = "sha256:c35abb8bfff0185efac5878da64c45dafd2b37fb0383add1be155a763c1f083d", size = 153018, upload-time = "2026-04-02T09:27:51.116Z" }, + { url = "https://files.pythonhosted.org/packages/db/8f/61959034484a4a7c527811f4721e75d02d653a35afb0b6054474d8185d4c/charset_normalizer-3.4.7-py3-none-any.whl", hash = "sha256:3dce51d0f5e7951f8bb4900c257dad282f49190fdbebecd4ba99bcc41fef404d", size = 61958, upload-time = "2026-04-02T09:28:37.794Z" }, +] + +[[package]] +name = "click" +version = "8.3.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/57/75/31212c6bf2503fdf920d87fee5d7a86a2e3bcf444984126f13d8e4016804/click-8.3.2.tar.gz", hash = "sha256:14162b8b3b3550a7d479eafa77dfd3c38d9dc8951f6f69c78913a8f9a7540fd5", size = 302856, upload-time = "2026-04-03T19:14:45.118Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e4/20/71885d8b97d4f3dde17b1fdb92dbd4908b00541c5a3379787137285f602e/click-8.3.2-py3-none-any.whl", hash = "sha256:1924d2c27c5653561cd2cae4548d1406039cb79b858b747cfea24924bbc1616d", size = 108379, upload-time = "2026-04-03T19:14:43.505Z" }, +] + +[[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, upload-time = "2022-10-25T02:36:22.414Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/d6/3965ed04c63042e047cb6a3e6ed1a63a35087b6a609aa3a15ed8ac56c221/colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6", size = 25335, upload-time = "2022-10-25T02:36:20.889Z" }, +] + +[[package]] +name = "coverage" +version = "7.13.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9d/e0/70553e3000e345daff267cec284ce4cbf3fc141b6da229ac52775b5428f1/coverage-7.13.5.tar.gz", hash = "sha256:c81f6515c4c40141f83f502b07bbfa5c240ba25bbe73da7b33f1e5b6120ff179", size = 915967, upload-time = "2026-03-17T10:33:18.341Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a0/c3/a396306ba7db865bf96fc1fb3b7fd29bcbf3d829df642e77b13555163cd6/coverage-7.13.5-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:460cf0114c5016fa841214ff5564aa4864f11948da9440bc97e21ad1f4ba1e01", size = 219554, upload-time = "2026-03-17T10:30:42.208Z" }, + { url = "https://files.pythonhosted.org/packages/a6/16/a68a19e5384e93f811dccc51034b1fd0b865841c390e3c931dcc4699e035/coverage-7.13.5-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0e223ce4b4ed47f065bfb123687686512e37629be25cc63728557ae7db261422", size = 219908, upload-time = "2026-03-17T10:30:43.906Z" }, + { url = "https://files.pythonhosted.org/packages/29/72/20b917c6793af3a5ceb7fb9c50033f3ec7865f2911a1416b34a7cfa0813b/coverage-7.13.5-cp312-cp312-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:6e3370441f4513c6252bf042b9c36d22491142385049243253c7e48398a15a9f", size = 251419, upload-time = "2026-03-17T10:30:45.545Z" }, + { url = "https://files.pythonhosted.org/packages/8c/49/cd14b789536ac6a4778c453c6a2338bc0a2fb60c5a5a41b4008328b9acc1/coverage-7.13.5-cp312-cp312-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:03ccc709a17a1de074fb1d11f217342fb0d2b1582ed544f554fc9fc3f07e95f5", size = 254159, upload-time = "2026-03-17T10:30:47.204Z" }, + { url = "https://files.pythonhosted.org/packages/9d/00/7b0edcfe64e2ed4c0340dac14a52ad0f4c9bd0b8b5e531af7d55b703db7c/coverage-7.13.5-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3f4818d065964db3c1c66dc0fbdac5ac692ecbc875555e13374fdbe7eedb4376", size = 255270, upload-time = "2026-03-17T10:30:48.812Z" }, + { url = "https://files.pythonhosted.org/packages/93/89/7ffc4ba0f5d0a55c1e84ea7cee39c9fc06af7b170513d83fbf3bbefce280/coverage-7.13.5-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:012d5319e66e9d5a218834642d6c35d265515a62f01157a45bcc036ecf947256", size = 257538, upload-time = "2026-03-17T10:30:50.77Z" }, + { url = "https://files.pythonhosted.org/packages/81/bd/73ddf85f93f7e6fa83e77ccecb6162d9415c79007b4bc124008a4995e4a7/coverage-7.13.5-cp312-cp312-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:8dd02af98971bdb956363e4827d34425cb3df19ee550ef92855b0acb9c7ce51c", size = 251821, upload-time = "2026-03-17T10:30:52.5Z" }, + { url = "https://files.pythonhosted.org/packages/a0/81/278aff4e8dec4926a0bcb9486320752811f543a3ce5b602cc7a29978d073/coverage-7.13.5-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:f08fd75c50a760c7eb068ae823777268daaf16a80b918fa58eea888f8e3919f5", size = 253191, upload-time = "2026-03-17T10:30:54.543Z" }, + { url = "https://files.pythonhosted.org/packages/70/ee/fe1621488e2e0a58d7e94c4800f0d96f79671553488d401a612bebae324b/coverage-7.13.5-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:843ea8643cf967d1ac7e8ecd4bb00c99135adf4816c0c0593fdcc47b597fcf09", size = 251337, upload-time = "2026-03-17T10:30:56.663Z" }, + { url = "https://files.pythonhosted.org/packages/37/a6/f79fb37aa104b562207cc23cb5711ab6793608e246cae1e93f26b2236ed9/coverage-7.13.5-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:9d44d7aa963820b1b971dbecd90bfe5fe8f81cff79787eb6cca15750bd2f79b9", size = 255404, upload-time = "2026-03-17T10:30:58.427Z" }, + { url = "https://files.pythonhosted.org/packages/75/f0/ed15262a58ec81ce457ceb717b7f78752a1713556b19081b76e90896e8d4/coverage-7.13.5-cp312-cp312-musllinux_1_2_riscv64.whl", hash = "sha256:7132bed4bd7b836200c591410ae7d97bf7ae8be6fc87d160b2bd881df929e7bf", size = 250903, upload-time = "2026-03-17T10:31:00.093Z" }, + { url = "https://files.pythonhosted.org/packages/0f/e9/9129958f20e7e9d4d56d51d42ccf708d15cac355ff4ac6e736e97a9393d2/coverage-7.13.5-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a698e363641b98843c517817db75373c83254781426e94ada3197cabbc2c919c", size = 252780, upload-time = "2026-03-17T10:31:01.916Z" }, + { url = "https://files.pythonhosted.org/packages/a4/d7/0ad9b15812d81272db94379fe4c6df8fd17781cc7671fdfa30c76ba5ff7b/coverage-7.13.5-cp312-cp312-win32.whl", hash = "sha256:bdba0a6b8812e8c7df002d908a9a2ea3c36e92611b5708633c50869e6d922fdf", size = 222093, upload-time = "2026-03-17T10:31:03.642Z" }, + { url = "https://files.pythonhosted.org/packages/29/3d/821a9a5799fac2556bcf0bd37a70d1d11fa9e49784b6d22e92e8b2f85f18/coverage-7.13.5-cp312-cp312-win_amd64.whl", hash = "sha256:d2c87e0c473a10bffe991502eac389220533024c8082ec1ce849f4218dded810", size = 222900, upload-time = "2026-03-17T10:31:05.651Z" }, + { url = "https://files.pythonhosted.org/packages/d4/fa/2238c2ad08e35cf4f020ea721f717e09ec3152aea75d191a7faf3ef009a8/coverage-7.13.5-cp312-cp312-win_arm64.whl", hash = "sha256:bf69236a9a81bdca3bff53796237aab096cdbf8d78a66ad61e992d9dac7eb2de", size = 221515, upload-time = "2026-03-17T10:31:07.293Z" }, + { url = "https://files.pythonhosted.org/packages/74/8c/74fedc9663dcf168b0a059d4ea756ecae4da77a489048f94b5f512a8d0b3/coverage-7.13.5-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:5ec4af212df513e399cf11610cc27063f1586419e814755ab362e50a85ea69c1", size = 219576, upload-time = "2026-03-17T10:31:09.045Z" }, + { url = "https://files.pythonhosted.org/packages/0c/c9/44fb661c55062f0818a6ffd2685c67aa30816200d5f2817543717d4b92eb/coverage-7.13.5-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:941617e518602e2d64942c88ec8499f7fbd49d3f6c4327d3a71d43a1973032f3", size = 219942, upload-time = "2026-03-17T10:31:10.708Z" }, + { url = "https://files.pythonhosted.org/packages/5f/13/93419671cee82b780bab7ea96b67c8ef448f5f295f36bf5031154ec9a790/coverage-7.13.5-cp313-cp313-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:da305e9937617ee95c2e39d8ff9f040e0487cbf1ac174f777ed5eddd7a7c1f26", size = 250935, upload-time = "2026-03-17T10:31:12.392Z" }, + { url = "https://files.pythonhosted.org/packages/ac/68/1666e3a4462f8202d836920114fa7a5ee9275d1fa45366d336c551a162dd/coverage-7.13.5-cp313-cp313-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:78e696e1cc714e57e8b25760b33a8b1026b7048d270140d25dafe1b0a1ee05a3", size = 253541, upload-time = "2026-03-17T10:31:14.247Z" }, + { url = "https://files.pythonhosted.org/packages/4e/5e/3ee3b835647be646dcf3c65a7c6c18f87c27326a858f72ab22c12730773d/coverage-7.13.5-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:02ca0eed225b2ff301c474aeeeae27d26e2537942aa0f87491d3e147e784a82b", size = 254780, upload-time = "2026-03-17T10:31:16.193Z" }, + { url = "https://files.pythonhosted.org/packages/44/b3/cb5bd1a04cfcc49ede6cd8409d80bee17661167686741e041abc7ee1b9a9/coverage-7.13.5-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:04690832cbea4e4663d9149e05dba142546ca05cb1848816760e7f58285c970a", size = 256912, upload-time = "2026-03-17T10:31:17.89Z" }, + { url = "https://files.pythonhosted.org/packages/1b/66/c1dceb7b9714473800b075f5c8a84f4588f887a90eb8645282031676e242/coverage-7.13.5-cp313-cp313-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:0590e44dd2745c696a778f7bab6aa95256de2cbc8b8cff4f7db8ff09813d6969", size = 251165, upload-time = "2026-03-17T10:31:19.605Z" }, + { url = "https://files.pythonhosted.org/packages/b7/62/5502b73b97aa2e53ea22a39cf8649ff44827bef76d90bf638777daa27a9d/coverage-7.13.5-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:d7cfad2d6d81dd298ab6b89fe72c3b7b05ec7544bdda3b707ddaecff8d25c161", size = 252908, upload-time = "2026-03-17T10:31:21.312Z" }, + { url = "https://files.pythonhosted.org/packages/7d/37/7792c2d69854397ca77a55c4646e5897c467928b0e27f2d235d83b5d08c6/coverage-7.13.5-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:e092b9499de38ae0fbfbc603a74660eb6ff3e869e507b50d85a13b6db9863e15", size = 250873, upload-time = "2026-03-17T10:31:23.565Z" }, + { url = "https://files.pythonhosted.org/packages/a3/23/bc866fb6163be52a8a9e5d708ba0d3b1283c12158cefca0a8bbb6e247a43/coverage-7.13.5-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:48c39bc4a04d983a54a705a6389512883d4a3b9862991b3617d547940e9f52b1", size = 255030, upload-time = "2026-03-17T10:31:25.58Z" }, + { url = "https://files.pythonhosted.org/packages/7d/8b/ef67e1c222ef49860701d346b8bbb70881bef283bd5f6cbba68a39a086c7/coverage-7.13.5-cp313-cp313-musllinux_1_2_riscv64.whl", hash = "sha256:2d3807015f138ffea1ed9afeeb8624fd781703f2858b62a8dd8da5a0994c57b6", size = 250694, upload-time = "2026-03-17T10:31:27.316Z" }, + { url = "https://files.pythonhosted.org/packages/46/0d/866d1f74f0acddbb906db212e096dee77a8e2158ca5e6bb44729f9d93298/coverage-7.13.5-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:ee2aa19e03161671ec964004fb74b2257805d9710bf14a5c704558b9d8dbaf17", size = 252469, upload-time = "2026-03-17T10:31:29.472Z" }, + { url = "https://files.pythonhosted.org/packages/7a/f5/be742fec31118f02ce42b21c6af187ad6a344fed546b56ca60caacc6a9a0/coverage-7.13.5-cp313-cp313-win32.whl", hash = "sha256:ce1998c0483007608c8382f4ff50164bfc5bd07a2246dd272aa4043b75e61e85", size = 222112, upload-time = "2026-03-17T10:31:31.526Z" }, + { url = "https://files.pythonhosted.org/packages/66/40/7732d648ab9d069a46e686043241f01206348e2bbf128daea85be4d6414b/coverage-7.13.5-cp313-cp313-win_amd64.whl", hash = "sha256:631efb83f01569670a5e866ceb80fe483e7c159fac6f167e6571522636104a0b", size = 222923, upload-time = "2026-03-17T10:31:33.633Z" }, + { url = "https://files.pythonhosted.org/packages/48/af/fea819c12a095781f6ccd504890aaddaf88b8fab263c4940e82c7b770124/coverage-7.13.5-cp313-cp313-win_arm64.whl", hash = "sha256:f4cd16206ad171cbc2470dbea9103cf9a7607d5fe8c242fdf1edf36174020664", size = 221540, upload-time = "2026-03-17T10:31:35.445Z" }, + { url = "https://files.pythonhosted.org/packages/23/d2/17879af479df7fbbd44bd528a31692a48f6b25055d16482fdf5cdb633805/coverage-7.13.5-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0428cbef5783ad91fe240f673cc1f76b25e74bbfe1a13115e4aa30d3f538162d", size = 220262, upload-time = "2026-03-17T10:31:37.184Z" }, + { url = "https://files.pythonhosted.org/packages/5b/4c/d20e554f988c8f91d6a02c5118f9abbbf73a8768a3048cb4962230d5743f/coverage-7.13.5-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e0b216a19534b2427cc201a26c25da4a48633f29a487c61258643e89d28200c0", size = 220617, upload-time = "2026-03-17T10:31:39.245Z" }, + { url = "https://files.pythonhosted.org/packages/29/9c/f9f5277b95184f764b24e7231e166dfdb5780a46d408a2ac665969416d61/coverage-7.13.5-cp313-cp313t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:972a9cd27894afe4bc2b1480107054e062df08e671df7c2f18c205e805ccd806", size = 261912, upload-time = "2026-03-17T10:31:41.324Z" }, + { url = "https://files.pythonhosted.org/packages/d5/f6/7f1ab39393eeb50cfe4747ae8ef0e4fc564b989225aa1152e13a180d74f8/coverage-7.13.5-cp313-cp313t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:4b59148601efcd2bac8c4dbf1f0ad6391693ccf7a74b8205781751637076aee3", size = 263987, upload-time = "2026-03-17T10:31:43.724Z" }, + { url = "https://files.pythonhosted.org/packages/a0/d7/62c084fb489ed9c6fbdf57e006752e7c516ea46fd690e5ed8b8617c7d52e/coverage-7.13.5-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:505d7083c8b0c87a8fa8c07370c285847c1f77739b22e299ad75a6af6c32c5c9", size = 266416, upload-time = "2026-03-17T10:31:45.769Z" }, + { url = "https://files.pythonhosted.org/packages/a9/f6/df63d8660e1a0bff6125947afda112a0502736f470d62ca68b288ea762d8/coverage-7.13.5-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:60365289c3741e4db327e7baff2a4aaacf22f788e80fa4683393891b70a89fbd", size = 267558, upload-time = "2026-03-17T10:31:48.293Z" }, + { url = "https://files.pythonhosted.org/packages/5b/02/353ca81d36779bd108f6d384425f7139ac3c58c750dcfaafe5d0bee6436b/coverage-7.13.5-cp313-cp313t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:1b88c69c8ef5d4b6fe7dea66d6636056a0f6a7527c440e890cf9259011f5e606", size = 261163, upload-time = "2026-03-17T10:31:50.125Z" }, + { url = "https://files.pythonhosted.org/packages/2c/16/2e79106d5749bcaf3aee6d309123548e3276517cd7851faa8da213bc61bf/coverage-7.13.5-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:5b13955d31d1633cf9376908089b7cebe7d15ddad7aeaabcbe969a595a97e95e", size = 263981, upload-time = "2026-03-17T10:31:51.961Z" }, + { url = "https://files.pythonhosted.org/packages/29/c7/c29e0c59ffa6942030ae6f50b88ae49988e7e8da06de7ecdbf49c6d4feae/coverage-7.13.5-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:f70c9ab2595c56f81a89620e22899eea8b212a4041bd728ac6f4a28bf5d3ddd0", size = 261604, upload-time = "2026-03-17T10:31:53.872Z" }, + { url = "https://files.pythonhosted.org/packages/40/48/097cdc3db342f34006a308ab41c3a7c11c3f0d84750d340f45d88a782e00/coverage-7.13.5-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:084b84a8c63e8d6fc7e3931b316a9bcafca1458d753c539db82d31ed20091a87", size = 265321, upload-time = "2026-03-17T10:31:55.997Z" }, + { url = "https://files.pythonhosted.org/packages/bb/1f/4994af354689e14fd03a75f8ec85a9a68d94e0188bbdab3fc1516b55e512/coverage-7.13.5-cp313-cp313t-musllinux_1_2_riscv64.whl", hash = "sha256:ad14385487393e386e2ea988b09d62dd42c397662ac2dabc3832d71253eee479", size = 260502, upload-time = "2026-03-17T10:31:58.308Z" }, + { url = "https://files.pythonhosted.org/packages/22/c6/9bb9ef55903e628033560885f5c31aa227e46878118b63ab15dc7ba87797/coverage-7.13.5-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:7f2c47b36fe7709a6e83bfadf4eefb90bd25fbe4014d715224c4316f808e59a2", size = 262688, upload-time = "2026-03-17T10:32:00.141Z" }, + { url = "https://files.pythonhosted.org/packages/14/4f/f5df9007e50b15e53e01edea486814783a7f019893733d9e4d6caad75557/coverage-7.13.5-cp313-cp313t-win32.whl", hash = "sha256:67e9bc5449801fad0e5dff329499fb090ba4c5800b86805c80617b4e29809b2a", size = 222788, upload-time = "2026-03-17T10:32:02.246Z" }, + { url = "https://files.pythonhosted.org/packages/e1/98/aa7fccaa97d0f3192bec013c4e6fd6d294a6ed44b640e6bb61f479e00ed5/coverage-7.13.5-cp313-cp313t-win_amd64.whl", hash = "sha256:da86cdcf10d2519e10cabb8ac2de03da1bcb6e4853790b7fbd48523332e3a819", size = 223851, upload-time = "2026-03-17T10:32:04.416Z" }, + { url = "https://files.pythonhosted.org/packages/3d/8b/e5c469f7352651e5f013198e9e21f97510b23de957dd06a84071683b4b60/coverage-7.13.5-cp313-cp313t-win_arm64.whl", hash = "sha256:0ecf12ecb326fe2c339d93fc131816f3a7367d223db37817208905c89bded911", size = 222104, upload-time = "2026-03-17T10:32:06.65Z" }, + { url = "https://files.pythonhosted.org/packages/8e/77/39703f0d1d4b478bfd30191d3c14f53caf596fac00efb3f8f6ee23646439/coverage-7.13.5-cp314-cp314-macosx_10_15_x86_64.whl", hash = "sha256:fbabfaceaeb587e16f7008f7795cd80d20ec548dc7f94fbb0d4ec2e038ce563f", size = 219621, upload-time = "2026-03-17T10:32:08.589Z" }, + { url = "https://files.pythonhosted.org/packages/e2/3e/51dff36d99ae14639a133d9b164d63e628532e2974d8b1edb99dd1ebc733/coverage-7.13.5-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:9bb2a28101a443669a423b665939381084412b81c3f8c0fcfbac57f4e30b5b8e", size = 219953, upload-time = "2026-03-17T10:32:10.507Z" }, + { url = "https://files.pythonhosted.org/packages/6a/6c/1f1917b01eb647c2f2adc9962bd66c79eb978951cab61bdc1acab3290c07/coverage-7.13.5-cp314-cp314-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:bd3a2fbc1c6cccb3c5106140d87cc6a8715110373ef42b63cf5aea29df8c217a", size = 250992, upload-time = "2026-03-17T10:32:12.41Z" }, + { url = "https://files.pythonhosted.org/packages/22/e5/06b1f88f42a5a99df42ce61208bdec3bddb3d261412874280a19796fc09c/coverage-7.13.5-cp314-cp314-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:6c36ddb64ed9d7e496028d1d00dfec3e428e0aabf4006583bb1839958d280510", size = 253503, upload-time = "2026-03-17T10:32:14.449Z" }, + { url = "https://files.pythonhosted.org/packages/80/28/2a148a51e5907e504fa7b85490277734e6771d8844ebcc48764a15e28155/coverage-7.13.5-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:380e8e9084d8eb38db3a9176a1a4f3c0082c3806fa0dc882d1d87abc3c789247", size = 254852, upload-time = "2026-03-17T10:32:16.56Z" }, + { url = "https://files.pythonhosted.org/packages/61/77/50e8d3d85cc0b7ebe09f30f151d670e302c7ff4a1bf6243f71dd8b0981fa/coverage-7.13.5-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:e808af52a0513762df4d945ea164a24b37f2f518cbe97e03deaa0ee66139b4d6", size = 257161, upload-time = "2026-03-17T10:32:19.004Z" }, + { url = "https://files.pythonhosted.org/packages/3b/c4/b5fd1d4b7bf8d0e75d997afd3925c59ba629fc8616f1b3aae7605132e256/coverage-7.13.5-cp314-cp314-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:e301d30dd7e95ae068671d746ba8c34e945a82682e62918e41b2679acd2051a0", size = 251021, upload-time = "2026-03-17T10:32:21.344Z" }, + { url = "https://files.pythonhosted.org/packages/f8/66/6ea21f910e92d69ef0b1c3346ea5922a51bad4446c9126db2ae96ee24c4c/coverage-7.13.5-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:800bc829053c80d240a687ceeb927a94fd108bbdc68dfbe505d0d75ab578a882", size = 252858, upload-time = "2026-03-17T10:32:23.506Z" }, + { url = "https://files.pythonhosted.org/packages/9e/ea/879c83cb5d61aa2a35fb80e72715e92672daef8191b84911a643f533840c/coverage-7.13.5-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:0b67af5492adb31940ee418a5a655c28e48165da5afab8c7fa6fd72a142f8740", size = 250823, upload-time = "2026-03-17T10:32:25.516Z" }, + { url = "https://files.pythonhosted.org/packages/8a/fb/616d95d3adb88b9803b275580bdeee8bd1b69a886d057652521f83d7322f/coverage-7.13.5-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:c9136ff29c3a91e25b1d1552b5308e53a1e0653a23e53b6366d7c2dcbbaf8a16", size = 255099, upload-time = "2026-03-17T10:32:27.944Z" }, + { url = "https://files.pythonhosted.org/packages/1c/93/25e6917c90ec1c9a56b0b26f6cad6408e5f13bb6b35d484a0d75c9cf000d/coverage-7.13.5-cp314-cp314-musllinux_1_2_riscv64.whl", hash = "sha256:cff784eef7f0b8f6cb28804fbddcfa99f89efe4cc35fb5627e3ac58f91ed3ac0", size = 250638, upload-time = "2026-03-17T10:32:29.914Z" }, + { url = "https://files.pythonhosted.org/packages/fc/7b/dc1776b0464145a929deed214aef9fb1493f159b59ff3c7eeeedf91eddd0/coverage-7.13.5-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:68a4953be99b17ac3c23b6efbc8a38330d99680c9458927491d18700ef23ded0", size = 252295, upload-time = "2026-03-17T10:32:31.981Z" }, + { url = "https://files.pythonhosted.org/packages/ea/fb/99cbbc56a26e07762a2740713f3c8f9f3f3106e3a3dd8cc4474954bccd34/coverage-7.13.5-cp314-cp314-win32.whl", hash = "sha256:35a31f2b1578185fbe6aa2e74cea1b1d0bbf4c552774247d9160d29b80ed56cc", size = 222360, upload-time = "2026-03-17T10:32:34.233Z" }, + { url = "https://files.pythonhosted.org/packages/8d/b7/4758d4f73fb536347cc5e4ad63662f9d60ba9118cb6785e9616b2ce5d7fa/coverage-7.13.5-cp314-cp314-win_amd64.whl", hash = "sha256:2aa055ae1857258f9e0045be26a6d62bdb47a72448b62d7b55f4820f361a2633", size = 223174, upload-time = "2026-03-17T10:32:36.369Z" }, + { url = "https://files.pythonhosted.org/packages/2c/f2/24d84e1dfe70f8ac9fdf30d338239860d0d1d5da0bda528959d0ebc9da28/coverage-7.13.5-cp314-cp314-win_arm64.whl", hash = "sha256:1b11eef33edeae9d142f9b4358edb76273b3bfd30bc3df9a4f95d0e49caf94e8", size = 221739, upload-time = "2026-03-17T10:32:38.736Z" }, + { url = "https://files.pythonhosted.org/packages/60/5b/4a168591057b3668c2428bff25dd3ebc21b629d666d90bcdfa0217940e84/coverage-7.13.5-cp314-cp314t-macosx_10_15_x86_64.whl", hash = "sha256:10a0c37f0b646eaff7cce1874c31d1f1ccb297688d4c747291f4f4c70741cc8b", size = 220351, upload-time = "2026-03-17T10:32:41.196Z" }, + { url = "https://files.pythonhosted.org/packages/f5/21/1fd5c4dbfe4a58b6b99649125635df46decdfd4a784c3cd6d410d303e370/coverage-7.13.5-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:b5db73ba3c41c7008037fa731ad5459fc3944cb7452fc0aa9f822ad3533c583c", size = 220612, upload-time = "2026-03-17T10:32:43.204Z" }, + { url = "https://files.pythonhosted.org/packages/d6/fe/2a924b3055a5e7e4512655a9d4609781b0d62334fa0140c3e742926834e2/coverage-7.13.5-cp314-cp314t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:750db93a81e3e5a9831b534be7b1229df848b2e125a604fe6651e48aa070e5f9", size = 261985, upload-time = "2026-03-17T10:32:45.514Z" }, + { url = "https://files.pythonhosted.org/packages/d7/0d/c8928f2bd518c45990fe1a2ab8db42e914ef9b726c975facc4282578c3eb/coverage-7.13.5-cp314-cp314t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:9ddb4f4a5479f2539644be484da179b653273bca1a323947d48ab107b3ed1f29", size = 264107, upload-time = "2026-03-17T10:32:47.971Z" }, + { url = "https://files.pythonhosted.org/packages/ef/ae/4ae35bbd9a0af9d820362751f0766582833c211224b38665c0f8de3d487f/coverage-7.13.5-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:d8a7a2049c14f413163e2bdabd37e41179b1d1ccb10ffc6ccc4b7a718429c607", size = 266513, upload-time = "2026-03-17T10:32:50.1Z" }, + { url = "https://files.pythonhosted.org/packages/9c/20/d326174c55af36f74eac6ae781612d9492f060ce8244b570bb9d50d9d609/coverage-7.13.5-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:e1c85e0b6c05c592ea6d8768a66a254bfb3874b53774b12d4c89c481eb78cb90", size = 267650, upload-time = "2026-03-17T10:32:52.391Z" }, + { url = "https://files.pythonhosted.org/packages/7a/5e/31484d62cbd0eabd3412e30d74386ece4a0837d4f6c3040a653878bfc019/coverage-7.13.5-cp314-cp314t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:777c4d1eff1b67876139d24288aaf1817f6c03d6bae9c5cc8d27b83bcfe38fe3", size = 261089, upload-time = "2026-03-17T10:32:54.544Z" }, + { url = "https://files.pythonhosted.org/packages/e9/d8/49a72d6de146eebb0b7e48cc0f4bc2c0dd858e3d4790ab2b39a2872b62bd/coverage-7.13.5-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:6697e29b93707167687543480a40f0db8f356e86d9f67ddf2e37e2dfd91a9dab", size = 263982, upload-time = "2026-03-17T10:32:56.803Z" }, + { url = "https://files.pythonhosted.org/packages/06/3b/0351f1bd566e6e4dd39e978efe7958bde1d32f879e85589de147654f57bb/coverage-7.13.5-cp314-cp314t-musllinux_1_2_i686.whl", hash = "sha256:8fdf453a942c3e4d99bd80088141c4c6960bb232c409d9c3558e2dbaa3998562", size = 261579, upload-time = "2026-03-17T10:32:59.466Z" }, + { url = "https://files.pythonhosted.org/packages/5d/ce/796a2a2f4017f554d7810f5c573449b35b1e46788424a548d4d19201b222/coverage-7.13.5-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:32ca0c0114c9834a43f045a87dcebd69d108d8ffb666957ea65aa132f50332e2", size = 265316, upload-time = "2026-03-17T10:33:01.847Z" }, + { url = "https://files.pythonhosted.org/packages/3d/16/d5ae91455541d1a78bc90abf495be600588aff8f6db5c8b0dae739fa39c9/coverage-7.13.5-cp314-cp314t-musllinux_1_2_riscv64.whl", hash = "sha256:8769751c10f339021e2638cd354e13adeac54004d1941119b2c96fe5276d45ea", size = 260427, upload-time = "2026-03-17T10:33:03.945Z" }, + { url = "https://files.pythonhosted.org/packages/48/11/07f413dba62db21fb3fad5d0de013a50e073cc4e2dc4306e770360f6dfc8/coverage-7.13.5-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:cec2d83125531bd153175354055cdb7a09987af08a9430bd173c937c6d0fba2a", size = 262745, upload-time = "2026-03-17T10:33:06.285Z" }, + { url = "https://files.pythonhosted.org/packages/91/15/d792371332eb4663115becf4bad47e047d16234b1aff687b1b18c58d60ae/coverage-7.13.5-cp314-cp314t-win32.whl", hash = "sha256:0cd9ed7a8b181775459296e402ca4fb27db1279740a24e93b3b41942ebe4b215", size = 223146, upload-time = "2026-03-17T10:33:08.756Z" }, + { url = "https://files.pythonhosted.org/packages/db/51/37221f59a111dca5e85be7dbf09696323b5b9f13ff65e0641d535ed06ea8/coverage-7.13.5-cp314-cp314t-win_amd64.whl", hash = "sha256:301e3b7dfefecaca37c9f1aa6f0049b7d4ab8dd933742b607765d757aca77d43", size = 224254, upload-time = "2026-03-17T10:33:11.174Z" }, + { url = "https://files.pythonhosted.org/packages/54/83/6acacc889de8987441aa7d5adfbdbf33d288dad28704a67e574f1df9bcbb/coverage-7.13.5-cp314-cp314t-win_arm64.whl", hash = "sha256:9dacc2ad679b292709e0f5fc1ac74a6d4d5562e424058962c7bb0c658ad25e45", size = 222276, upload-time = "2026-03-17T10:33:13.466Z" }, + { url = "https://files.pythonhosted.org/packages/9e/ee/a4cf96b8ce1e566ed238f0659ac2d3f007ed1d14b181bcb684e19561a69a/coverage-7.13.5-py3-none-any.whl", hash = "sha256:34b02417cf070e173989b3db962f7ed56d2f644307b2cf9d5a0f258e13084a61", size = 211346, upload-time = "2026-03-17T10:33:15.691Z" }, +] + +[[package]] +name = "cryptography" +version = "46.0.7" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cffi", marker = "platform_python_implementation != 'PyPy'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/47/93/ac8f3d5ff04d54bc814e961a43ae5b0b146154c89c61b47bb07557679b18/cryptography-46.0.7.tar.gz", hash = "sha256:e4cfd68c5f3e0bfdad0d38e023239b96a2fe84146481852dffbcca442c245aa5", size = 750652, upload-time = "2026-04-08T01:57:54.692Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0b/5d/4a8f770695d73be252331e60e526291e3df0c9b27556a90a6b47bccca4c2/cryptography-46.0.7-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:ea42cbe97209df307fdc3b155f1b6fa2577c0defa8f1f7d3be7d31d189108ad4", size = 7179869, upload-time = "2026-04-08T01:56:17.157Z" }, + { url = "https://files.pythonhosted.org/packages/5f/45/6d80dc379b0bbc1f9d1e429f42e4cb9e1d319c7a8201beffd967c516ea01/cryptography-46.0.7-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:b36a4695e29fe69215d75960b22577197aca3f7a25b9cf9d165dcfe9d80bc325", size = 4275492, upload-time = "2026-04-08T01:56:19.36Z" }, + { url = "https://files.pythonhosted.org/packages/4a/9a/1765afe9f572e239c3469f2cb429f3ba7b31878c893b246b4b2994ffe2fe/cryptography-46.0.7-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:5ad9ef796328c5e3c4ceed237a183f5d41d21150f972455a9d926593a1dcb308", size = 4426670, upload-time = "2026-04-08T01:56:21.415Z" }, + { url = "https://files.pythonhosted.org/packages/8f/3e/af9246aaf23cd4ee060699adab1e47ced3f5f7e7a8ffdd339f817b446462/cryptography-46.0.7-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:73510b83623e080a2c35c62c15298096e2a5dc8d51c3b4e1740211839d0dea77", size = 4280275, upload-time = "2026-04-08T01:56:23.539Z" }, + { url = "https://files.pythonhosted.org/packages/0f/54/6bbbfc5efe86f9d71041827b793c24811a017c6ac0fd12883e4caa86b8ed/cryptography-46.0.7-cp311-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:cbd5fb06b62bd0721e1170273d3f4d5a277044c47ca27ee257025146c34cbdd1", size = 4928402, upload-time = "2026-04-08T01:56:25.624Z" }, + { url = "https://files.pythonhosted.org/packages/2d/cf/054b9d8220f81509939599c8bdbc0c408dbd2bdd41688616a20731371fe0/cryptography-46.0.7-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:420b1e4109cc95f0e5700eed79908cef9268265c773d3a66f7af1eef53d409ef", size = 4459985, upload-time = "2026-04-08T01:56:27.309Z" }, + { url = "https://files.pythonhosted.org/packages/f9/46/4e4e9c6040fb01c7467d47217d2f882daddeb8828f7df800cb806d8a2288/cryptography-46.0.7-cp311-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:24402210aa54baae71d99441d15bb5a1919c195398a87b563df84468160a65de", size = 3990652, upload-time = "2026-04-08T01:56:29.095Z" }, + { url = "https://files.pythonhosted.org/packages/36/5f/313586c3be5a2fbe87e4c9a254207b860155a8e1f3cca99f9910008e7d08/cryptography-46.0.7-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:8a469028a86f12eb7d2fe97162d0634026d92a21f3ae0ac87ed1c4a447886c83", size = 4279805, upload-time = "2026-04-08T01:56:30.928Z" }, + { url = "https://files.pythonhosted.org/packages/69/33/60dfc4595f334a2082749673386a4d05e4f0cf4df8248e63b2c3437585f2/cryptography-46.0.7-cp311-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:9694078c5d44c157ef3162e3bf3946510b857df5a3955458381d1c7cfc143ddb", size = 4892883, upload-time = "2026-04-08T01:56:32.614Z" }, + { url = "https://files.pythonhosted.org/packages/c7/0b/333ddab4270c4f5b972f980adef4faa66951a4aaf646ca067af597f15563/cryptography-46.0.7-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:42a1e5f98abb6391717978baf9f90dc28a743b7d9be7f0751a6f56a75d14065b", size = 4459756, upload-time = "2026-04-08T01:56:34.306Z" }, + { url = "https://files.pythonhosted.org/packages/d2/14/633913398b43b75f1234834170947957c6b623d1701ffc7a9600da907e89/cryptography-46.0.7-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:91bbcb08347344f810cbe49065914fe048949648f6bd5c2519f34619142bbe85", size = 4410244, upload-time = "2026-04-08T01:56:35.977Z" }, + { url = "https://files.pythonhosted.org/packages/10/f2/19ceb3b3dc14009373432af0c13f46aa08e3ce334ec6eff13492e1812ccd/cryptography-46.0.7-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:5d1c02a14ceb9148cc7816249f64f623fbfee39e8c03b3650d842ad3f34d637e", size = 4674868, upload-time = "2026-04-08T01:56:38.034Z" }, + { url = "https://files.pythonhosted.org/packages/1a/bb/a5c213c19ee94b15dfccc48f363738633a493812687f5567addbcbba9f6f/cryptography-46.0.7-cp311-abi3-win32.whl", hash = "sha256:d23c8ca48e44ee015cd0a54aeccdf9f09004eba9fc96f38c911011d9ff1bd457", size = 3026504, upload-time = "2026-04-08T01:56:39.666Z" }, + { url = "https://files.pythonhosted.org/packages/2b/02/7788f9fefa1d060ca68717c3901ae7fffa21ee087a90b7f23c7a603c32ae/cryptography-46.0.7-cp311-abi3-win_amd64.whl", hash = "sha256:397655da831414d165029da9bc483bed2fe0e75dde6a1523ec2fe63f3c46046b", size = 3488363, upload-time = "2026-04-08T01:56:41.893Z" }, + { url = "https://files.pythonhosted.org/packages/7b/56/15619b210e689c5403bb0540e4cb7dbf11a6bf42e483b7644e471a2812b3/cryptography-46.0.7-cp314-cp314t-macosx_10_9_universal2.whl", hash = "sha256:d151173275e1728cf7839aaa80c34fe550c04ddb27b34f48c232193df8db5842", size = 7119671, upload-time = "2026-04-08T01:56:44Z" }, + { url = "https://files.pythonhosted.org/packages/74/66/e3ce040721b0b5599e175ba91ab08884c75928fbeb74597dd10ef13505d2/cryptography-46.0.7-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:db0f493b9181c7820c8134437eb8b0b4792085d37dbb24da050476ccb664e59c", size = 4268551, upload-time = "2026-04-08T01:56:46.071Z" }, + { url = "https://files.pythonhosted.org/packages/03/11/5e395f961d6868269835dee1bafec6a1ac176505a167f68b7d8818431068/cryptography-46.0.7-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ebd6daf519b9f189f85c479427bbd6e9c9037862cf8fe89ee35503bd209ed902", size = 4408887, upload-time = "2026-04-08T01:56:47.718Z" }, + { url = "https://files.pythonhosted.org/packages/40/53/8ed1cf4c3b9c8e611e7122fb56f1c32d09e1fff0f1d77e78d9ff7c82653e/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_aarch64.whl", hash = "sha256:b7b412817be92117ec5ed95f880defe9cf18a832e8cafacf0a22337dc1981b4d", size = 4271354, upload-time = "2026-04-08T01:56:49.312Z" }, + { url = "https://files.pythonhosted.org/packages/50/46/cf71e26025c2e767c5609162c866a78e8a2915bbcfa408b7ca495c6140c4/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_ppc64le.whl", hash = "sha256:fbfd0e5f273877695cb93baf14b185f4878128b250cc9f8e617ea0c025dfb022", size = 4905845, upload-time = "2026-04-08T01:56:50.916Z" }, + { url = "https://files.pythonhosted.org/packages/c0/ea/01276740375bac6249d0a971ebdf6b4dc9ead0ee0a34ef3b5a88c1a9b0d4/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_x86_64.whl", hash = "sha256:ffca7aa1d00cf7d6469b988c581598f2259e46215e0140af408966a24cf086ce", size = 4444641, upload-time = "2026-04-08T01:56:52.882Z" }, + { url = "https://files.pythonhosted.org/packages/3d/4c/7d258f169ae71230f25d9f3d06caabcff8c3baf0978e2b7d65e0acac3827/cryptography-46.0.7-cp314-cp314t-manylinux_2_31_armv7l.whl", hash = "sha256:60627cf07e0d9274338521205899337c5d18249db56865f943cbe753aa96f40f", size = 3967749, upload-time = "2026-04-08T01:56:54.597Z" }, + { url = "https://files.pythonhosted.org/packages/b5/2a/2ea0767cad19e71b3530e4cad9605d0b5e338b6a1e72c37c9c1ceb86c333/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_aarch64.whl", hash = "sha256:80406c3065e2c55d7f49a9550fe0c49b3f12e5bfff5dedb727e319e1afb9bf99", size = 4270942, upload-time = "2026-04-08T01:56:56.416Z" }, + { url = "https://files.pythonhosted.org/packages/41/3d/fe14df95a83319af25717677e956567a105bb6ab25641acaa093db79975d/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_ppc64le.whl", hash = "sha256:c5b1ccd1239f48b7151a65bc6dd54bcfcc15e028c8ac126d3fada09db0e07ef1", size = 4871079, upload-time = "2026-04-08T01:56:58.31Z" }, + { url = "https://files.pythonhosted.org/packages/9c/59/4a479e0f36f8f378d397f4eab4c850b4ffb79a2f0d58704b8fa0703ddc11/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_x86_64.whl", hash = "sha256:d5f7520159cd9c2154eb61eb67548ca05c5774d39e9c2c4339fd793fe7d097b2", size = 4443999, upload-time = "2026-04-08T01:57:00.508Z" }, + { url = "https://files.pythonhosted.org/packages/28/17/b59a741645822ec6d04732b43c5d35e4ef58be7bfa84a81e5ae6f05a1d33/cryptography-46.0.7-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:fcd8eac50d9138c1d7fc53a653ba60a2bee81a505f9f8850b6b2888555a45d0e", size = 4399191, upload-time = "2026-04-08T01:57:02.654Z" }, + { url = "https://files.pythonhosted.org/packages/59/6a/bb2e166d6d0e0955f1e9ff70f10ec4b2824c9cfcdb4da772c7dd69cc7d80/cryptography-46.0.7-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:65814c60f8cc400c63131584e3e1fad01235edba2614b61fbfbfa954082db0ee", size = 4655782, upload-time = "2026-04-08T01:57:04.592Z" }, + { url = "https://files.pythonhosted.org/packages/95/b6/3da51d48415bcb63b00dc17c2eff3a651b7c4fed484308d0f19b30e8cb2c/cryptography-46.0.7-cp314-cp314t-win32.whl", hash = "sha256:fdd1736fed309b4300346f88f74cd120c27c56852c3838cab416e7a166f67298", size = 3002227, upload-time = "2026-04-08T01:57:06.91Z" }, + { url = "https://files.pythonhosted.org/packages/32/a8/9f0e4ed57ec9cebe506e58db11ae472972ecb0c659e4d52bbaee80ca340a/cryptography-46.0.7-cp314-cp314t-win_amd64.whl", hash = "sha256:e06acf3c99be55aa3b516397fe42f5855597f430add9c17fa46bf2e0fb34c9bb", size = 3475332, upload-time = "2026-04-08T01:57:08.807Z" }, + { url = "https://files.pythonhosted.org/packages/a7/7f/cd42fc3614386bc0c12f0cb3c4ae1fc2bbca5c9662dfed031514911d513d/cryptography-46.0.7-cp38-abi3-macosx_10_9_universal2.whl", hash = "sha256:462ad5cb1c148a22b2e3bcc5ad52504dff325d17daf5df8d88c17dda1f75f2a4", size = 7165618, upload-time = "2026-04-08T01:57:10.645Z" }, + { url = "https://files.pythonhosted.org/packages/a5/d0/36a49f0262d2319139d2829f773f1b97ef8aef7f97e6e5bd21455e5a8fb5/cryptography-46.0.7-cp38-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:84d4cced91f0f159a7ddacad249cc077e63195c36aac40b4150e7a57e84fffe7", size = 4270628, upload-time = "2026-04-08T01:57:12.885Z" }, + { url = "https://files.pythonhosted.org/packages/8a/6c/1a42450f464dda6ffbe578a911f773e54dd48c10f9895a23a7e88b3e7db5/cryptography-46.0.7-cp38-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:128c5edfe5e5938b86b03941e94fac9ee793a94452ad1365c9fc3f4f62216832", size = 4415405, upload-time = "2026-04-08T01:57:14.923Z" }, + { url = "https://files.pythonhosted.org/packages/9a/92/4ed714dbe93a066dc1f4b4581a464d2d7dbec9046f7c8b7016f5286329e2/cryptography-46.0.7-cp38-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:5e51be372b26ef4ba3de3c167cd3d1022934bc838ae9eaad7e644986d2a3d163", size = 4272715, upload-time = "2026-04-08T01:57:16.638Z" }, + { url = "https://files.pythonhosted.org/packages/b7/e6/a26b84096eddd51494bba19111f8fffe976f6a09f132706f8f1bf03f51f7/cryptography-46.0.7-cp38-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:cdf1a610ef82abb396451862739e3fc93b071c844399e15b90726ef7470eeaf2", size = 4918400, upload-time = "2026-04-08T01:57:19.021Z" }, + { url = "https://files.pythonhosted.org/packages/c7/08/ffd537b605568a148543ac3c2b239708ae0bd635064bab41359252ef88ed/cryptography-46.0.7-cp38-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:1d25aee46d0c6f1a501adcddb2d2fee4b979381346a78558ed13e50aa8a59067", size = 4450634, upload-time = "2026-04-08T01:57:21.185Z" }, + { url = "https://files.pythonhosted.org/packages/16/01/0cd51dd86ab5b9befe0d031e276510491976c3a80e9f6e31810cce46c4ad/cryptography-46.0.7-cp38-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:cdfbe22376065ffcf8be74dc9a909f032df19bc58a699456a21712d6e5eabfd0", size = 3985233, upload-time = "2026-04-08T01:57:22.862Z" }, + { url = "https://files.pythonhosted.org/packages/92/49/819d6ed3a7d9349c2939f81b500a738cb733ab62fbecdbc1e38e83d45e12/cryptography-46.0.7-cp38-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:abad9dac36cbf55de6eb49badd4016806b3165d396f64925bf2999bcb67837ba", size = 4271955, upload-time = "2026-04-08T01:57:24.814Z" }, + { url = "https://files.pythonhosted.org/packages/80/07/ad9b3c56ebb95ed2473d46df0847357e01583f4c52a85754d1a55e29e4d0/cryptography-46.0.7-cp38-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:935ce7e3cfdb53e3536119a542b839bb94ec1ad081013e9ab9b7cfd478b05006", size = 4879888, upload-time = "2026-04-08T01:57:26.88Z" }, + { url = "https://files.pythonhosted.org/packages/b8/c7/201d3d58f30c4c2bdbe9b03844c291feb77c20511cc3586daf7edc12a47b/cryptography-46.0.7-cp38-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:35719dc79d4730d30f1c2b6474bd6acda36ae2dfae1e3c16f2051f215df33ce0", size = 4449961, upload-time = "2026-04-08T01:57:29.068Z" }, + { url = "https://files.pythonhosted.org/packages/a5/ef/649750cbf96f3033c3c976e112265c33906f8e462291a33d77f90356548c/cryptography-46.0.7-cp38-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:7bbc6ccf49d05ac8f7d7b5e2e2c33830d4fe2061def88210a126d130d7f71a85", size = 4401696, upload-time = "2026-04-08T01:57:31.029Z" }, + { url = "https://files.pythonhosted.org/packages/41/52/a8908dcb1a389a459a29008c29966c1d552588d4ae6d43f3a1a4512e0ebe/cryptography-46.0.7-cp38-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:a1529d614f44b863a7b480c6d000fe93b59acee9c82ffa027cfadc77521a9f5e", size = 4664256, upload-time = "2026-04-08T01:57:33.144Z" }, + { url = "https://files.pythonhosted.org/packages/4b/fa/f0ab06238e899cc3fb332623f337a7364f36f4bb3f2534c2bb95a35b132c/cryptography-46.0.7-cp38-abi3-win32.whl", hash = "sha256:f247c8c1a1fb45e12586afbb436ef21ff1e80670b2861a90353d9b025583d246", size = 3013001, upload-time = "2026-04-08T01:57:34.933Z" }, + { url = "https://files.pythonhosted.org/packages/d2/f1/00ce3bde3ca542d1acd8f8cfa38e446840945aa6363f9b74746394b14127/cryptography-46.0.7-cp38-abi3-win_amd64.whl", hash = "sha256:506c4ff91eff4f82bdac7633318a526b1d1309fc07ca76a3ad182cb5b686d6d3", size = 3472985, upload-time = "2026-04-08T01:57:36.714Z" }, +] + +[[package]] +name = "decorator" +version = "5.2.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/43/fa/6d96a0978d19e17b68d634497769987b16c8f4cd0a7a05048bec693caa6b/decorator-5.2.1.tar.gz", hash = "sha256:65f266143752f734b0a7cc83c46f4618af75b8c5911b00ccb61d0ac9b6da0360", size = 56711, upload-time = "2025-02-24T04:41:34.073Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4e/8c/f3147f5c4b73e7550fe5f9352eaa956ae838d5c51eb58e7a25b9f3e2643b/decorator-5.2.1-py3-none-any.whl", hash = "sha256:d316bb415a2d9e2d2b3abcc4084c6502fc09240e292cd76a76afc106a1c8e04a", size = 9190, upload-time = "2025-02-24T04:41:32.565Z" }, +] + +[[package]] +name = "execnet" +version = "2.1.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/bf/89/780e11f9588d9e7128a3f87788354c7946a9cbb1401ad38a48c4db9a4f07/execnet-2.1.2.tar.gz", hash = "sha256:63d83bfdd9a23e35b9c6a3261412324f964c2ec8dcd8d3c6916ee9373e0befcd", size = 166622, upload-time = "2025-11-12T09:56:37.75Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ab/84/02fc1827e8cdded4aa65baef11296a9bbe595c474f0d6d758af082d849fd/execnet-2.1.2-py3-none-any.whl", hash = "sha256:67fba928dd5a544b783f6056f449e5e3931a5c378b128bc18501f7ea79e296ec", size = 40708, upload-time = "2025-11-12T09:56:36.333Z" }, +] + +[[package]] +name = "executing" +version = "2.2.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cc/28/c14e053b6762b1044f34a13aab6859bbf40456d37d23aa286ac24cfd9a5d/executing-2.2.1.tar.gz", hash = "sha256:3632cc370565f6648cc328b32435bd120a1e4ebb20c77e3fdde9a13cd1e533c4", size = 1129488, upload-time = "2025-09-01T09:48:10.866Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c1/ea/53f2148663b321f21b5a606bd5f191517cf40b7072c0497d3c92c4a13b1e/executing-2.2.1-py2.py3-none-any.whl", hash = "sha256:760643d3452b4d777d295bb167ccc74c64a81df23fb5e08eff250c425a4b2017", size = 28317, upload-time = "2025-09-01T09:48:08.5Z" }, +] + +[[package]] +name = "filelock" +version = "3.28.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d6/17/6e8890271880903e3538660a21d63a6c1fea969ac71d0d6b608b78727fa9/filelock-3.28.0.tar.gz", hash = "sha256:4ed1010aae813c4ee8d9c660e4792475ee60c4a0ba76073ceaf862bd317e3ca6", size = 56474, upload-time = "2026-04-14T22:54:33.625Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3b/21/2f728888c45033d34a417bfcd248ea2564c9e08ab1bfd301377cf05d5586/filelock-3.28.0-py3-none-any.whl", hash = "sha256:de9af6712788e7171df1b28b15eba2446c69721433fa427a9bee07b17820a9db", size = 39189, upload-time = "2026-04-14T22:54:32.037Z" }, +] + +[[package]] +name = "frozenlist" +version = "1.8.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/2d/f5/c831fac6cc817d26fd54c7eaccd04ef7e0288806943f7cc5bbf69f3ac1f0/frozenlist-1.8.0.tar.gz", hash = "sha256:3ede829ed8d842f6cd48fc7081d7a41001a56f1f38603f9d49bf3020d59a31ad", size = 45875, upload-time = "2025-10-06T05:38:17.865Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/69/29/948b9aa87e75820a38650af445d2ef2b6b8a6fab1a23b6bb9e4ef0be2d59/frozenlist-1.8.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:78f7b9e5d6f2fdb88cdde9440dc147259b62b9d3b019924def9f6478be254ac1", size = 87782, upload-time = "2025-10-06T05:36:06.649Z" }, + { url = "https://files.pythonhosted.org/packages/64/80/4f6e318ee2a7c0750ed724fa33a4bdf1eacdc5a39a7a24e818a773cd91af/frozenlist-1.8.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:229bf37d2e4acdaf808fd3f06e854a4a7a3661e871b10dc1f8f1896a3b05f18b", size = 50594, upload-time = "2025-10-06T05:36:07.69Z" }, + { url = "https://files.pythonhosted.org/packages/2b/94/5c8a2b50a496b11dd519f4a24cb5496cf125681dd99e94c604ccdea9419a/frozenlist-1.8.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:f833670942247a14eafbb675458b4e61c82e002a148f49e68257b79296e865c4", size = 50448, upload-time = "2025-10-06T05:36:08.78Z" }, + { url = "https://files.pythonhosted.org/packages/6a/bd/d91c5e39f490a49df14320f4e8c80161cfcce09f1e2cde1edd16a551abb3/frozenlist-1.8.0-cp312-cp312-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:494a5952b1c597ba44e0e78113a7266e656b9794eec897b19ead706bd7074383", size = 242411, upload-time = "2025-10-06T05:36:09.801Z" }, + { url = "https://files.pythonhosted.org/packages/8f/83/f61505a05109ef3293dfb1ff594d13d64a2324ac3482be2cedc2be818256/frozenlist-1.8.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:96f423a119f4777a4a056b66ce11527366a8bb92f54e541ade21f2374433f6d4", size = 243014, upload-time = "2025-10-06T05:36:11.394Z" }, + { url = "https://files.pythonhosted.org/packages/d8/cb/cb6c7b0f7d4023ddda30cf56b8b17494eb3a79e3fda666bf735f63118b35/frozenlist-1.8.0-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:3462dd9475af2025c31cc61be6652dfa25cbfb56cbbf52f4ccfe029f38decaf8", size = 234909, upload-time = "2025-10-06T05:36:12.598Z" }, + { url = "https://files.pythonhosted.org/packages/31/c5/cd7a1f3b8b34af009fb17d4123c5a778b44ae2804e3ad6b86204255f9ec5/frozenlist-1.8.0-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c4c800524c9cd9bac5166cd6f55285957fcfc907db323e193f2afcd4d9abd69b", size = 250049, upload-time = "2025-10-06T05:36:14.065Z" }, + { url = "https://files.pythonhosted.org/packages/c0/01/2f95d3b416c584a1e7f0e1d6d31998c4a795f7544069ee2e0962a4b60740/frozenlist-1.8.0-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:d6a5df73acd3399d893dafc71663ad22534b5aa4f94e8a2fabfe856c3c1b6a52", size = 256485, upload-time = "2025-10-06T05:36:15.39Z" }, + { url = "https://files.pythonhosted.org/packages/ce/03/024bf7720b3abaebcff6d0793d73c154237b85bdf67b7ed55e5e9596dc9a/frozenlist-1.8.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:405e8fe955c2280ce66428b3ca55e12b3c4e9c336fb2103a4937e891c69a4a29", size = 237619, upload-time = "2025-10-06T05:36:16.558Z" }, + { url = "https://files.pythonhosted.org/packages/69/fa/f8abdfe7d76b731f5d8bd217827cf6764d4f1d9763407e42717b4bed50a0/frozenlist-1.8.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:908bd3f6439f2fef9e85031b59fd4f1297af54415fb60e4254a95f75b3cab3f3", size = 250320, upload-time = "2025-10-06T05:36:17.821Z" }, + { url = "https://files.pythonhosted.org/packages/f5/3c/b051329f718b463b22613e269ad72138cc256c540f78a6de89452803a47d/frozenlist-1.8.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:294e487f9ec720bd8ffcebc99d575f7eff3568a08a253d1ee1a0378754b74143", size = 246820, upload-time = "2025-10-06T05:36:19.046Z" }, + { url = "https://files.pythonhosted.org/packages/0f/ae/58282e8f98e444b3f4dd42448ff36fa38bef29e40d40f330b22e7108f565/frozenlist-1.8.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:74c51543498289c0c43656701be6b077f4b265868fa7f8a8859c197006efb608", size = 250518, upload-time = "2025-10-06T05:36:20.763Z" }, + { url = "https://files.pythonhosted.org/packages/8f/96/007e5944694d66123183845a106547a15944fbbb7154788cbf7272789536/frozenlist-1.8.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:776f352e8329135506a1d6bf16ac3f87bc25b28e765949282dcc627af36123aa", size = 239096, upload-time = "2025-10-06T05:36:22.129Z" }, + { url = "https://files.pythonhosted.org/packages/66/bb/852b9d6db2fa40be96f29c0d1205c306288f0684df8fd26ca1951d461a56/frozenlist-1.8.0-cp312-cp312-win32.whl", hash = "sha256:433403ae80709741ce34038da08511d4a77062aa924baf411ef73d1146e74faf", size = 39985, upload-time = "2025-10-06T05:36:23.661Z" }, + { url = "https://files.pythonhosted.org/packages/b8/af/38e51a553dd66eb064cdf193841f16f077585d4d28394c2fa6235cb41765/frozenlist-1.8.0-cp312-cp312-win_amd64.whl", hash = "sha256:34187385b08f866104f0c0617404c8eb08165ab1272e884abc89c112e9c00746", size = 44591, upload-time = "2025-10-06T05:36:24.958Z" }, + { url = "https://files.pythonhosted.org/packages/a7/06/1dc65480ab147339fecc70797e9c2f69d9cea9cf38934ce08df070fdb9cb/frozenlist-1.8.0-cp312-cp312-win_arm64.whl", hash = "sha256:fe3c58d2f5db5fbd18c2987cba06d51b0529f52bc3a6cdc33d3f4eab725104bd", size = 40102, upload-time = "2025-10-06T05:36:26.333Z" }, + { url = "https://files.pythonhosted.org/packages/2d/40/0832c31a37d60f60ed79e9dfb5a92e1e2af4f40a16a29abcc7992af9edff/frozenlist-1.8.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:8d92f1a84bb12d9e56f818b3a746f3efba93c1b63c8387a73dde655e1e42282a", size = 85717, upload-time = "2025-10-06T05:36:27.341Z" }, + { url = "https://files.pythonhosted.org/packages/30/ba/b0b3de23f40bc55a7057bd38434e25c34fa48e17f20ee273bbde5e0650f3/frozenlist-1.8.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:96153e77a591c8adc2ee805756c61f59fef4cf4073a9275ee86fe8cba41241f7", size = 49651, upload-time = "2025-10-06T05:36:28.855Z" }, + { url = "https://files.pythonhosted.org/packages/0c/ab/6e5080ee374f875296c4243c381bbdef97a9ac39c6e3ce1d5f7d42cb78d6/frozenlist-1.8.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f21f00a91358803399890ab167098c131ec2ddd5f8f5fd5fe9c9f2c6fcd91e40", size = 49417, upload-time = "2025-10-06T05:36:29.877Z" }, + { url = "https://files.pythonhosted.org/packages/d5/4e/e4691508f9477ce67da2015d8c00acd751e6287739123113a9fca6f1604e/frozenlist-1.8.0-cp313-cp313-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:fb30f9626572a76dfe4293c7194a09fb1fe93ba94c7d4f720dfae3b646b45027", size = 234391, upload-time = "2025-10-06T05:36:31.301Z" }, + { url = "https://files.pythonhosted.org/packages/40/76/c202df58e3acdf12969a7895fd6f3bc016c642e6726aa63bd3025e0fc71c/frozenlist-1.8.0-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:eaa352d7047a31d87dafcacbabe89df0aa506abb5b1b85a2fb91bc3faa02d822", size = 233048, upload-time = "2025-10-06T05:36:32.531Z" }, + { url = "https://files.pythonhosted.org/packages/f9/c0/8746afb90f17b73ca5979c7a3958116e105ff796e718575175319b5bb4ce/frozenlist-1.8.0-cp313-cp313-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:03ae967b4e297f58f8c774c7eabcce57fe3c2434817d4385c50661845a058121", size = 226549, upload-time = "2025-10-06T05:36:33.706Z" }, + { url = "https://files.pythonhosted.org/packages/7e/eb/4c7eefc718ff72f9b6c4893291abaae5fbc0c82226a32dcd8ef4f7a5dbef/frozenlist-1.8.0-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:f6292f1de555ffcc675941d65fffffb0a5bcd992905015f85d0592201793e0e5", size = 239833, upload-time = "2025-10-06T05:36:34.947Z" }, + { url = "https://files.pythonhosted.org/packages/c2/4e/e5c02187cf704224f8b21bee886f3d713ca379535f16893233b9d672ea71/frozenlist-1.8.0-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:29548f9b5b5e3460ce7378144c3010363d8035cea44bc0bf02d57f5a685e084e", size = 245363, upload-time = "2025-10-06T05:36:36.534Z" }, + { url = "https://files.pythonhosted.org/packages/1f/96/cb85ec608464472e82ad37a17f844889c36100eed57bea094518bf270692/frozenlist-1.8.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:ec3cc8c5d4084591b4237c0a272cc4f50a5b03396a47d9caaf76f5d7b38a4f11", size = 229314, upload-time = "2025-10-06T05:36:38.582Z" }, + { url = "https://files.pythonhosted.org/packages/5d/6f/4ae69c550e4cee66b57887daeebe006fe985917c01d0fff9caab9883f6d0/frozenlist-1.8.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:517279f58009d0b1f2e7c1b130b377a349405da3f7621ed6bfae50b10adf20c1", size = 243365, upload-time = "2025-10-06T05:36:40.152Z" }, + { url = "https://files.pythonhosted.org/packages/7a/58/afd56de246cf11780a40a2c28dc7cbabbf06337cc8ddb1c780a2d97e88d8/frozenlist-1.8.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:db1e72ede2d0d7ccb213f218df6a078a9c09a7de257c2fe8fcef16d5925230b1", size = 237763, upload-time = "2025-10-06T05:36:41.355Z" }, + { url = "https://files.pythonhosted.org/packages/cb/36/cdfaf6ed42e2644740d4a10452d8e97fa1c062e2a8006e4b09f1b5fd7d63/frozenlist-1.8.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:b4dec9482a65c54a5044486847b8a66bf10c9cb4926d42927ec4e8fd5db7fed8", size = 240110, upload-time = "2025-10-06T05:36:42.716Z" }, + { url = "https://files.pythonhosted.org/packages/03/a8/9ea226fbefad669f11b52e864c55f0bd57d3c8d7eb07e9f2e9a0b39502e1/frozenlist-1.8.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:21900c48ae04d13d416f0e1e0c4d81f7931f73a9dfa0b7a8746fb2fe7dd970ed", size = 233717, upload-time = "2025-10-06T05:36:44.251Z" }, + { url = "https://files.pythonhosted.org/packages/1e/0b/1b5531611e83ba7d13ccc9988967ea1b51186af64c42b7a7af465dcc9568/frozenlist-1.8.0-cp313-cp313-win32.whl", hash = "sha256:8b7b94a067d1c504ee0b16def57ad5738701e4ba10cec90529f13fa03c833496", size = 39628, upload-time = "2025-10-06T05:36:45.423Z" }, + { url = "https://files.pythonhosted.org/packages/d8/cf/174c91dbc9cc49bc7b7aab74d8b734e974d1faa8f191c74af9b7e80848e6/frozenlist-1.8.0-cp313-cp313-win_amd64.whl", hash = "sha256:878be833caa6a3821caf85eb39c5ba92d28e85df26d57afb06b35b2efd937231", size = 43882, upload-time = "2025-10-06T05:36:46.796Z" }, + { url = "https://files.pythonhosted.org/packages/c1/17/502cd212cbfa96eb1388614fe39a3fc9ab87dbbe042b66f97acb57474834/frozenlist-1.8.0-cp313-cp313-win_arm64.whl", hash = "sha256:44389d135b3ff43ba8cc89ff7f51f5a0bb6b63d829c8300f79a2fe4fe61bcc62", size = 39676, upload-time = "2025-10-06T05:36:47.8Z" }, + { url = "https://files.pythonhosted.org/packages/d2/5c/3bbfaa920dfab09e76946a5d2833a7cbdf7b9b4a91c714666ac4855b88b4/frozenlist-1.8.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:e25ac20a2ef37e91c1b39938b591457666a0fa835c7783c3a8f33ea42870db94", size = 89235, upload-time = "2025-10-06T05:36:48.78Z" }, + { url = "https://files.pythonhosted.org/packages/d2/d6/f03961ef72166cec1687e84e8925838442b615bd0b8854b54923ce5b7b8a/frozenlist-1.8.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:07cdca25a91a4386d2e76ad992916a85038a9b97561bf7a3fd12d5d9ce31870c", size = 50742, upload-time = "2025-10-06T05:36:49.837Z" }, + { url = "https://files.pythonhosted.org/packages/1e/bb/a6d12b7ba4c3337667d0e421f7181c82dda448ce4e7ad7ecd249a16fa806/frozenlist-1.8.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:4e0c11f2cc6717e0a741f84a527c52616140741cd812a50422f83dc31749fb52", size = 51725, upload-time = "2025-10-06T05:36:50.851Z" }, + { url = "https://files.pythonhosted.org/packages/bc/71/d1fed0ffe2c2ccd70b43714c6cab0f4188f09f8a67a7914a6b46ee30f274/frozenlist-1.8.0-cp313-cp313t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:b3210649ee28062ea6099cfda39e147fa1bc039583c8ee4481cb7811e2448c51", size = 284533, upload-time = "2025-10-06T05:36:51.898Z" }, + { url = "https://files.pythonhosted.org/packages/c9/1f/fb1685a7b009d89f9bf78a42d94461bc06581f6e718c39344754a5d9bada/frozenlist-1.8.0-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:581ef5194c48035a7de2aefc72ac6539823bb71508189e5de01d60c9dcd5fa65", size = 292506, upload-time = "2025-10-06T05:36:53.101Z" }, + { url = "https://files.pythonhosted.org/packages/e6/3b/b991fe1612703f7e0d05c0cf734c1b77aaf7c7d321df4572e8d36e7048c8/frozenlist-1.8.0-cp313-cp313t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:3ef2d026f16a2b1866e1d86fc4e1291e1ed8a387b2c333809419a2f8b3a77b82", size = 274161, upload-time = "2025-10-06T05:36:54.309Z" }, + { url = "https://files.pythonhosted.org/packages/ca/ec/c5c618767bcdf66e88945ec0157d7f6c4a1322f1473392319b7a2501ded7/frozenlist-1.8.0-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:5500ef82073f599ac84d888e3a8c1f77ac831183244bfd7f11eaa0289fb30714", size = 294676, upload-time = "2025-10-06T05:36:55.566Z" }, + { url = "https://files.pythonhosted.org/packages/7c/ce/3934758637d8f8a88d11f0585d6495ef54b2044ed6ec84492a91fa3b27aa/frozenlist-1.8.0-cp313-cp313t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:50066c3997d0091c411a66e710f4e11752251e6d2d73d70d8d5d4c76442a199d", size = 300638, upload-time = "2025-10-06T05:36:56.758Z" }, + { url = "https://files.pythonhosted.org/packages/fc/4f/a7e4d0d467298f42de4b41cbc7ddaf19d3cfeabaf9ff97c20c6c7ee409f9/frozenlist-1.8.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:5c1c8e78426e59b3f8005e9b19f6ff46e5845895adbde20ece9218319eca6506", size = 283067, upload-time = "2025-10-06T05:36:57.965Z" }, + { url = "https://files.pythonhosted.org/packages/dc/48/c7b163063d55a83772b268e6d1affb960771b0e203b632cfe09522d67ea5/frozenlist-1.8.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:eefdba20de0d938cec6a89bd4d70f346a03108a19b9df4248d3cf0d88f1b0f51", size = 292101, upload-time = "2025-10-06T05:36:59.237Z" }, + { url = "https://files.pythonhosted.org/packages/9f/d0/2366d3c4ecdc2fd391e0afa6e11500bfba0ea772764d631bbf82f0136c9d/frozenlist-1.8.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:cf253e0e1c3ceb4aaff6df637ce033ff6535fb8c70a764a8f46aafd3d6ab798e", size = 289901, upload-time = "2025-10-06T05:37:00.811Z" }, + { url = "https://files.pythonhosted.org/packages/b8/94/daff920e82c1b70e3618a2ac39fbc01ae3e2ff6124e80739ce5d71c9b920/frozenlist-1.8.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:032efa2674356903cd0261c4317a561a6850f3ac864a63fc1583147fb05a79b0", size = 289395, upload-time = "2025-10-06T05:37:02.115Z" }, + { url = "https://files.pythonhosted.org/packages/e3/20/bba307ab4235a09fdcd3cc5508dbabd17c4634a1af4b96e0f69bfe551ebd/frozenlist-1.8.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:6da155091429aeba16851ecb10a9104a108bcd32f6c1642867eadaee401c1c41", size = 283659, upload-time = "2025-10-06T05:37:03.711Z" }, + { url = "https://files.pythonhosted.org/packages/fd/00/04ca1c3a7a124b6de4f8a9a17cc2fcad138b4608e7a3fc5877804b8715d7/frozenlist-1.8.0-cp313-cp313t-win32.whl", hash = "sha256:0f96534f8bfebc1a394209427d0f8a63d343c9779cda6fc25e8e121b5fd8555b", size = 43492, upload-time = "2025-10-06T05:37:04.915Z" }, + { url = "https://files.pythonhosted.org/packages/59/5e/c69f733a86a94ab10f68e496dc6b7e8bc078ebb415281d5698313e3af3a1/frozenlist-1.8.0-cp313-cp313t-win_amd64.whl", hash = "sha256:5d63a068f978fc69421fb0e6eb91a9603187527c86b7cd3f534a5b77a592b888", size = 48034, upload-time = "2025-10-06T05:37:06.343Z" }, + { url = "https://files.pythonhosted.org/packages/16/6c/be9d79775d8abe79b05fa6d23da99ad6e7763a1d080fbae7290b286093fd/frozenlist-1.8.0-cp313-cp313t-win_arm64.whl", hash = "sha256:bf0a7e10b077bf5fb9380ad3ae8ce20ef919a6ad93b4552896419ac7e1d8e042", size = 41749, upload-time = "2025-10-06T05:37:07.431Z" }, + { url = "https://files.pythonhosted.org/packages/f1/c8/85da824b7e7b9b6e7f7705b2ecaf9591ba6f79c1177f324c2735e41d36a2/frozenlist-1.8.0-cp314-cp314-macosx_10_13_universal2.whl", hash = "sha256:cee686f1f4cadeb2136007ddedd0aaf928ab95216e7691c63e50a8ec066336d0", size = 86127, upload-time = "2025-10-06T05:37:08.438Z" }, + { url = "https://files.pythonhosted.org/packages/8e/e8/a1185e236ec66c20afd72399522f142c3724c785789255202d27ae992818/frozenlist-1.8.0-cp314-cp314-macosx_10_13_x86_64.whl", hash = "sha256:119fb2a1bd47307e899c2fac7f28e85b9a543864df47aa7ec9d3c1b4545f096f", size = 49698, upload-time = "2025-10-06T05:37:09.48Z" }, + { url = "https://files.pythonhosted.org/packages/a1/93/72b1736d68f03fda5fdf0f2180fb6caaae3894f1b854d006ac61ecc727ee/frozenlist-1.8.0-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:4970ece02dbc8c3a92fcc5228e36a3e933a01a999f7094ff7c23fbd2beeaa67c", size = 49749, upload-time = "2025-10-06T05:37:10.569Z" }, + { url = "https://files.pythonhosted.org/packages/a7/b2/fabede9fafd976b991e9f1b9c8c873ed86f202889b864756f240ce6dd855/frozenlist-1.8.0-cp314-cp314-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:cba69cb73723c3f329622e34bdbf5ce1f80c21c290ff04256cff1cd3c2036ed2", size = 231298, upload-time = "2025-10-06T05:37:11.993Z" }, + { url = "https://files.pythonhosted.org/packages/3a/3b/d9b1e0b0eed36e70477ffb8360c49c85c8ca8ef9700a4e6711f39a6e8b45/frozenlist-1.8.0-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:778a11b15673f6f1df23d9586f83c4846c471a8af693a22e066508b77d201ec8", size = 232015, upload-time = "2025-10-06T05:37:13.194Z" }, + { url = "https://files.pythonhosted.org/packages/dc/94/be719d2766c1138148564a3960fc2c06eb688da592bdc25adcf856101be7/frozenlist-1.8.0-cp314-cp314-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:0325024fe97f94c41c08872db482cf8ac4800d80e79222c6b0b7b162d5b13686", size = 225038, upload-time = "2025-10-06T05:37:14.577Z" }, + { url = "https://files.pythonhosted.org/packages/e4/09/6712b6c5465f083f52f50cf74167b92d4ea2f50e46a9eea0523d658454ae/frozenlist-1.8.0-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:97260ff46b207a82a7567b581ab4190bd4dfa09f4db8a8b49d1a958f6aa4940e", size = 240130, upload-time = "2025-10-06T05:37:15.781Z" }, + { url = "https://files.pythonhosted.org/packages/f8/d4/cd065cdcf21550b54f3ce6a22e143ac9e4836ca42a0de1022da8498eac89/frozenlist-1.8.0-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:54b2077180eb7f83dd52c40b2750d0a9f175e06a42e3213ce047219de902717a", size = 242845, upload-time = "2025-10-06T05:37:17.037Z" }, + { url = "https://files.pythonhosted.org/packages/62/c3/f57a5c8c70cd1ead3d5d5f776f89d33110b1addae0ab010ad774d9a44fb9/frozenlist-1.8.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:2f05983daecab868a31e1da44462873306d3cbfd76d1f0b5b69c473d21dbb128", size = 229131, upload-time = "2025-10-06T05:37:18.221Z" }, + { url = "https://files.pythonhosted.org/packages/6c/52/232476fe9cb64f0742f3fde2b7d26c1dac18b6d62071c74d4ded55e0ef94/frozenlist-1.8.0-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:33f48f51a446114bc5d251fb2954ab0164d5be02ad3382abcbfe07e2531d650f", size = 240542, upload-time = "2025-10-06T05:37:19.771Z" }, + { url = "https://files.pythonhosted.org/packages/5f/85/07bf3f5d0fb5414aee5f47d33c6f5c77bfe49aac680bfece33d4fdf6a246/frozenlist-1.8.0-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:154e55ec0655291b5dd1b8731c637ecdb50975a2ae70c606d100750a540082f7", size = 237308, upload-time = "2025-10-06T05:37:20.969Z" }, + { url = "https://files.pythonhosted.org/packages/11/99/ae3a33d5befd41ac0ca2cc7fd3aa707c9c324de2e89db0e0f45db9a64c26/frozenlist-1.8.0-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:4314debad13beb564b708b4a496020e5306c7333fa9a3ab90374169a20ffab30", size = 238210, upload-time = "2025-10-06T05:37:22.252Z" }, + { url = "https://files.pythonhosted.org/packages/b2/60/b1d2da22f4970e7a155f0adde9b1435712ece01b3cd45ba63702aea33938/frozenlist-1.8.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:073f8bf8becba60aa931eb3bc420b217bb7d5b8f4750e6f8b3be7f3da85d38b7", size = 231972, upload-time = "2025-10-06T05:37:23.5Z" }, + { url = "https://files.pythonhosted.org/packages/3f/ab/945b2f32de889993b9c9133216c068b7fcf257d8595a0ac420ac8677cab0/frozenlist-1.8.0-cp314-cp314-win32.whl", hash = "sha256:bac9c42ba2ac65ddc115d930c78d24ab8d4f465fd3fc473cdedfccadb9429806", size = 40536, upload-time = "2025-10-06T05:37:25.581Z" }, + { url = "https://files.pythonhosted.org/packages/59/ad/9caa9b9c836d9ad6f067157a531ac48b7d36499f5036d4141ce78c230b1b/frozenlist-1.8.0-cp314-cp314-win_amd64.whl", hash = "sha256:3e0761f4d1a44f1d1a47996511752cf3dcec5bbdd9cc2b4fe595caf97754b7a0", size = 44330, upload-time = "2025-10-06T05:37:26.928Z" }, + { url = "https://files.pythonhosted.org/packages/82/13/e6950121764f2676f43534c555249f57030150260aee9dcf7d64efda11dd/frozenlist-1.8.0-cp314-cp314-win_arm64.whl", hash = "sha256:d1eaff1d00c7751b7c6662e9c5ba6eb2c17a2306ba5e2a37f24ddf3cc953402b", size = 40627, upload-time = "2025-10-06T05:37:28.075Z" }, + { url = "https://files.pythonhosted.org/packages/c0/c7/43200656ecc4e02d3f8bc248df68256cd9572b3f0017f0a0c4e93440ae23/frozenlist-1.8.0-cp314-cp314t-macosx_10_13_universal2.whl", hash = "sha256:d3bb933317c52d7ea5004a1c442eef86f426886fba134ef8cf4226ea6ee1821d", size = 89238, upload-time = "2025-10-06T05:37:29.373Z" }, + { url = "https://files.pythonhosted.org/packages/d1/29/55c5f0689b9c0fb765055629f472c0de484dcaf0acee2f7707266ae3583c/frozenlist-1.8.0-cp314-cp314t-macosx_10_13_x86_64.whl", hash = "sha256:8009897cdef112072f93a0efdce29cd819e717fd2f649ee3016efd3cd885a7ed", size = 50738, upload-time = "2025-10-06T05:37:30.792Z" }, + { url = "https://files.pythonhosted.org/packages/ba/7d/b7282a445956506fa11da8c2db7d276adcbf2b17d8bb8407a47685263f90/frozenlist-1.8.0-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:2c5dcbbc55383e5883246d11fd179782a9d07a986c40f49abe89ddf865913930", size = 51739, upload-time = "2025-10-06T05:37:32.127Z" }, + { url = "https://files.pythonhosted.org/packages/62/1c/3d8622e60d0b767a5510d1d3cf21065b9db874696a51ea6d7a43180a259c/frozenlist-1.8.0-cp314-cp314t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:39ecbc32f1390387d2aa4f5a995e465e9e2f79ba3adcac92d68e3e0afae6657c", size = 284186, upload-time = "2025-10-06T05:37:33.21Z" }, + { url = "https://files.pythonhosted.org/packages/2d/14/aa36d5f85a89679a85a1d44cd7a6657e0b1c75f61e7cad987b203d2daca8/frozenlist-1.8.0-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:92db2bf818d5cc8d9c1f1fc56b897662e24ea5adb36ad1f1d82875bd64e03c24", size = 292196, upload-time = "2025-10-06T05:37:36.107Z" }, + { url = "https://files.pythonhosted.org/packages/05/23/6bde59eb55abd407d34f77d39a5126fb7b4f109a3f611d3929f14b700c66/frozenlist-1.8.0-cp314-cp314t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:2dc43a022e555de94c3b68a4ef0b11c4f747d12c024a520c7101709a2144fb37", size = 273830, upload-time = "2025-10-06T05:37:37.663Z" }, + { url = "https://files.pythonhosted.org/packages/d2/3f/22cff331bfad7a8afa616289000ba793347fcd7bc275f3b28ecea2a27909/frozenlist-1.8.0-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:cb89a7f2de3602cfed448095bab3f178399646ab7c61454315089787df07733a", size = 294289, upload-time = "2025-10-06T05:37:39.261Z" }, + { url = "https://files.pythonhosted.org/packages/a4/89/5b057c799de4838b6c69aa82b79705f2027615e01be996d2486a69ca99c4/frozenlist-1.8.0-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:33139dc858c580ea50e7e60a1b0ea003efa1fd42e6ec7fdbad78fff65fad2fd2", size = 300318, upload-time = "2025-10-06T05:37:43.213Z" }, + { url = "https://files.pythonhosted.org/packages/30/de/2c22ab3eb2a8af6d69dc799e48455813bab3690c760de58e1bf43b36da3e/frozenlist-1.8.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:168c0969a329b416119507ba30b9ea13688fafffac1b7822802537569a1cb0ef", size = 282814, upload-time = "2025-10-06T05:37:45.337Z" }, + { url = "https://files.pythonhosted.org/packages/59/f7/970141a6a8dbd7f556d94977858cfb36fa9b66e0892c6dd780d2219d8cd8/frozenlist-1.8.0-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:28bd570e8e189d7f7b001966435f9dac6718324b5be2990ac496cf1ea9ddb7fe", size = 291762, upload-time = "2025-10-06T05:37:46.657Z" }, + { url = "https://files.pythonhosted.org/packages/c1/15/ca1adae83a719f82df9116d66f5bb28bb95557b3951903d39135620ef157/frozenlist-1.8.0-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:b2a095d45c5d46e5e79ba1e5b9cb787f541a8dee0433836cea4b96a2c439dcd8", size = 289470, upload-time = "2025-10-06T05:37:47.946Z" }, + { url = "https://files.pythonhosted.org/packages/ac/83/dca6dc53bf657d371fbc88ddeb21b79891e747189c5de990b9dfff2ccba1/frozenlist-1.8.0-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:eab8145831a0d56ec9c4139b6c3e594c7a83c2c8be25d5bcf2d86136a532287a", size = 289042, upload-time = "2025-10-06T05:37:49.499Z" }, + { url = "https://files.pythonhosted.org/packages/96/52/abddd34ca99be142f354398700536c5bd315880ed0a213812bc491cff5e4/frozenlist-1.8.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:974b28cf63cc99dfb2188d8d222bc6843656188164848c4f679e63dae4b0708e", size = 283148, upload-time = "2025-10-06T05:37:50.745Z" }, + { url = "https://files.pythonhosted.org/packages/af/d3/76bd4ed4317e7119c2b7f57c3f6934aba26d277acc6309f873341640e21f/frozenlist-1.8.0-cp314-cp314t-win32.whl", hash = "sha256:342c97bf697ac5480c0a7ec73cd700ecfa5a8a40ac923bd035484616efecc2df", size = 44676, upload-time = "2025-10-06T05:37:52.222Z" }, + { url = "https://files.pythonhosted.org/packages/89/76/c615883b7b521ead2944bb3480398cbb07e12b7b4e4d073d3752eb721558/frozenlist-1.8.0-cp314-cp314t-win_amd64.whl", hash = "sha256:06be8f67f39c8b1dc671f5d83aaefd3358ae5cdcf8314552c57e7ed3e6475bdd", size = 49451, upload-time = "2025-10-06T05:37:53.425Z" }, + { url = "https://files.pythonhosted.org/packages/e0/a3/5982da14e113d07b325230f95060e2169f5311b1017ea8af2a29b374c289/frozenlist-1.8.0-cp314-cp314t-win_arm64.whl", hash = "sha256:102e6314ca4da683dca92e3b1355490fed5f313b768500084fbe6371fddfdb79", size = 42507, upload-time = "2025-10-06T05:37:54.513Z" }, + { url = "https://files.pythonhosted.org/packages/9a/9a/e35b4a917281c0b8419d4207f4334c8e8c5dbf4f3f5f9ada73958d937dcc/frozenlist-1.8.0-py3-none-any.whl", hash = "sha256:0c18a16eab41e82c295618a77502e17b195883241c563b00f0aa5106fc4eaa0d", size = 13409, upload-time = "2025-10-06T05:38:16.721Z" }, +] + +[[package]] +name = "icdiff" +version = "2.0.10" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/5b/6d/41c8ff13b67c30ceb9325398ff6bb9fc4b77208c6c1d79db8d5913840bc6/icdiff-2.0.10.tar.gz", hash = "sha256:75a3de5c9af35ab45fb0504df59770c514a12c0d2b2c99e5f9c5c2429957e133", size = 16378, upload-time = "2026-02-08T15:03:26.76Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/85/4b/bb94514476624bfff977599e5ef3ac9aea603abb693639ed305b22ed1b35/icdiff-2.0.10-py3-none-any.whl", hash = "sha256:fd8ce592d94261c435e2a410f668a41ec3be4527878f8df01100797021d8edb0", size = 17103, upload-time = "2026-02-08T15:03:25.723Z" }, +] + +[[package]] +name = "idna" +version = "3.11" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6f/6d/0703ccc57f3a7233505399edb88de3cbd678da106337b9fcde432b65ed60/idna-3.11.tar.gz", hash = "sha256:795dafcc9c04ed0c1fb032c2aa73654d8e8c5023a7df64a53f39190ada629902", size = 194582, upload-time = "2025-10-12T14:55:20.501Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0e/61/66938bbb5fc52dbdf84594873d5b51fb1f7c7794e9c0f5bd885f30bc507b/idna-3.11-py3-none-any.whl", hash = "sha256:771a87f49d9defaf64091e6e6fe9c18d4833f140bd19464795bc32d966ca37ea", size = 71008, upload-time = "2025-10-12T14:55:18.883Z" }, +] + +[[package]] +name = "iniconfig" +version = "2.3.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/34/14ca021ce8e5dfedc35312d08ba8bf51fdd999c576889fc2c24cb97f4f10/iniconfig-2.3.0.tar.gz", hash = "sha256:c76315c77db068650d49c5b56314774a7804df16fee4402c1f19d6d15d8c4730", size = 20503, upload-time = "2025-10-18T21:55:43.219Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/cb/b1/3846dd7f199d53cb17f49cba7e651e9ce294d8497c8c150530ed11865bb8/iniconfig-2.3.0-py3-none-any.whl", hash = "sha256:f631c04d2c48c52b84d0d0549c99ff3859c98df65b3101406327ecc7d53fbf12", size = 7484, upload-time = "2025-10-18T21:55:41.639Z" }, +] + +[[package]] +name = "ipdb" +version = "0.13.13" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "decorator" }, + { name = "ipython" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3d/1b/7e07e7b752017f7693a0f4d41c13e5ca29ce8cbcfdcc1fd6c4ad8c0a27a0/ipdb-0.13.13.tar.gz", hash = "sha256:e3ac6018ef05126d442af680aad863006ec19d02290561ac88b8b1c0b0cfc726", size = 17042, upload-time = "2023-03-09T15:40:57.487Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0c/4c/b075da0092003d9a55cf2ecc1cae9384a1ca4f650d51b00fc59875fe76f6/ipdb-0.13.13-py3-none-any.whl", hash = "sha256:45529994741c4ab6d2388bfa5d7b725c2cf7fe9deffabdb8a6113aa5ed449ed4", size = 12130, upload-time = "2023-03-09T15:40:55.021Z" }, +] + +[[package]] +name = "ipython" +version = "9.12.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "decorator" }, + { name = "ipython-pygments-lexers" }, + { name = "jedi" }, + { name = "matplotlib-inline" }, + { name = "pexpect", marker = "sys_platform != 'emscripten' and sys_platform != 'win32'" }, + { name = "prompt-toolkit" }, + { name = "pygments" }, + { name = "stack-data" }, + { name = "traitlets" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3a/73/7114f80a8f9cabdb13c27732dce24af945b2923dcab80723602f7c8bc2d8/ipython-9.12.0.tar.gz", hash = "sha256:01daa83f504b693ba523b5a407246cabde4eb4513285a3c6acaff11a66735ee4", size = 4428879, upload-time = "2026-03-27T09:42:45.312Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/59/22/906c8108974c673ebef6356c506cebb6870d48cedea3c41e949e2dd556bb/ipython-9.12.0-py3-none-any.whl", hash = "sha256:0f2701e8ee86e117e37f50563205d36feaa259d2e08d4a6bc6b6d74b18ce128d", size = 625661, upload-time = "2026-03-27T09:42:42.831Z" }, +] + +[[package]] +name = "ipython-pygments-lexers" +version = "1.1.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pygments" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ef/4c/5dd1d8af08107f88c7f741ead7a40854b8ac24ddf9ae850afbcf698aa552/ipython_pygments_lexers-1.1.1.tar.gz", hash = "sha256:09c0138009e56b6854f9535736f4171d855c8c08a563a0dcd8022f78355c7e81", size = 8393, upload-time = "2025-01-17T11:24:34.505Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d9/33/1f075bf72b0b747cb3288d011319aaf64083cf2efef8354174e3ed4540e2/ipython_pygments_lexers-1.1.1-py3-none-any.whl", hash = "sha256:a9462224a505ade19a605f71f8fa63c2048833ce50abc86768a0d81d876dc81c", size = 8074, upload-time = "2025-01-17T11:24:33.271Z" }, +] + +[[package]] +name = "jedi" +version = "0.19.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "parso" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/72/3a/79a912fbd4d8dd6fbb02bf69afd3bb72cf0c729bb3063c6f4498603db17a/jedi-0.19.2.tar.gz", hash = "sha256:4770dc3de41bde3966b02eb84fbcf557fb33cce26ad23da12c742fb50ecb11f0", size = 1231287, upload-time = "2024-11-11T01:41:42.873Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c0/5a/9cac0c82afec3d09ccd97c8b6502d48f165f9124db81b4bcb90b4af974ee/jedi-0.19.2-py2.py3-none-any.whl", hash = "sha256:a8ef22bde8490f57fe5c7681a3c83cb58874daf72b4784de3cce5b6ef6edb5b9", size = 1572278, upload-time = "2024-11-11T01:41:40.175Z" }, +] + +[[package]] +name = "jmespath" +version = "1.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d3/59/322338183ecda247fb5d1763a6cbe46eff7222eaeebafd9fa65d4bf5cb11/jmespath-1.1.0.tar.gz", hash = "sha256:472c87d80f36026ae83c6ddd0f1d05d4e510134ed462851fd5f754c8c3cbb88d", size = 27377, upload-time = "2026-01-22T16:35:26.279Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/14/2f/967ba146e6d58cf6a652da73885f52fc68001525b4197effc174321d70b4/jmespath-1.1.0-py3-none-any.whl", hash = "sha256:a5663118de4908c91729bea0acadca56526eb2698e83de10cd116ae0f4e97c64", size = 20419, upload-time = "2026-01-22T16:35:24.919Z" }, +] + +[[package]] +name = "kgb" +version = "7.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/e8/00/9e56dee65ec791a92348fb54e8ced08c4c4db494b0f58cfb34737d087fb4/kgb-7.3.tar.gz", hash = "sha256:b8af7e79cb8b0df5a2ec596010b8e5d014845cfaa9203577b85b99d4df192927", size = 62922, upload-time = "2025-12-11T23:56:24.911Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/eb/d6/1c81a1292fc50ad93d0b145f1c241ecb7d541fba4dcec7166e2e1d99f9cd/kgb-7.3-py2.py3-none-any.whl", hash = "sha256:0b300cd6d234a951f60e54ccda78c99a355393d6ae878d3d5925e726ae2f0450", size = 59662, upload-time = "2025-12-11T23:56:23.699Z" }, +] + +[[package]] +name = "markdown-it-py" +version = "4.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "mdurl" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/5b/f5/4ec618ed16cc4f8fb3b701563655a69816155e79e24a17b651541804721d/markdown_it_py-4.0.0.tar.gz", hash = "sha256:cb0a2b4aa34f932c007117b194e945bd74e0ec24133ceb5bac59009cda1cb9f3", size = 73070, upload-time = "2025-08-11T12:57:52.854Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/94/54/e7d793b573f298e1c9013b8c4dade17d481164aa517d1d7148619c2cedbf/markdown_it_py-4.0.0-py3-none-any.whl", hash = "sha256:87327c59b172c5011896038353a81343b6754500a08cd7a4973bb48c6d578147", size = 87321, upload-time = "2025-08-11T12:57:51.923Z" }, +] + +[[package]] +name = "matplotlib-inline" +version = "0.2.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "traitlets" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/c7/74/97e72a36efd4ae2bccb3463284300f8953f199b5ffbc04cbbb0ec78f74b1/matplotlib_inline-0.2.1.tar.gz", hash = "sha256:e1ee949c340d771fc39e241ea75683deb94762c8fa5f2927ec57c83c4dffa9fe", size = 8110, upload-time = "2025-10-23T09:00:22.126Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/af/33/ee4519fa02ed11a94aef9559552f3b17bb863f2ecfe1a35dc7f548cde231/matplotlib_inline-0.2.1-py3-none-any.whl", hash = "sha256:d56ce5156ba6085e00a9d54fead6ed29a9c47e215cd1bba2e976ef39f5710a76", size = 9516, upload-time = "2025-10-23T09:00:20.675Z" }, +] + +[[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, upload-time = "2022-08-14T12:40:10.846Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b3/38/89ba8ad64ae25be8de66a6d463314cf1eb366222074cfda9ee839c56a4b4/mdurl-0.1.2-py3-none-any.whl", hash = "sha256:84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8", size = 9979, upload-time = "2022-08-14T12:40:09.779Z" }, +] + +[[package]] +name = "multidict" +version = "6.7.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/1a/c2/c2d94cbe6ac1753f3fc980da97b3d930efe1da3af3c9f5125354436c073d/multidict-6.7.1.tar.gz", hash = "sha256:ec6652a1bee61c53a3e5776b6049172c53b6aaba34f18c9ad04f82712bac623d", size = 102010, upload-time = "2026-01-26T02:46:45.979Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8d/9c/f20e0e2cf80e4b2e4b1c365bf5fe104ee633c751a724246262db8f1a0b13/multidict-6.7.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:a90f75c956e32891a4eda3639ce6dd86e87105271f43d43442a3aedf3cddf172", size = 76893, upload-time = "2026-01-26T02:43:52.754Z" }, + { url = "https://files.pythonhosted.org/packages/fe/cf/18ef143a81610136d3da8193da9d80bfe1cb548a1e2d1c775f26b23d024a/multidict-6.7.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:3fccb473e87eaa1382689053e4a4618e7ba7b9b9b8d6adf2027ee474597128cd", size = 45456, upload-time = "2026-01-26T02:43:53.893Z" }, + { url = "https://files.pythonhosted.org/packages/a9/65/1caac9d4cd32e8433908683446eebc953e82d22b03d10d41a5f0fefe991b/multidict-6.7.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b0fa96985700739c4c7853a43c0b3e169360d6855780021bfc6d0f1ce7c123e7", size = 43872, upload-time = "2026-01-26T02:43:55.041Z" }, + { url = "https://files.pythonhosted.org/packages/cf/3b/d6bd75dc4f3ff7c73766e04e705b00ed6dbbaccf670d9e05a12b006f5a21/multidict-6.7.1-cp312-cp312-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:cb2a55f408c3043e42b40cc8eecd575afa27b7e0b956dfb190de0f8499a57a53", size = 251018, upload-time = "2026-01-26T02:43:56.198Z" }, + { url = "https://files.pythonhosted.org/packages/fd/80/c959c5933adedb9ac15152e4067c702a808ea183a8b64cf8f31af8ad3155/multidict-6.7.1-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:eb0ce7b2a32d09892b3dd6cc44877a0d02a33241fafca5f25c8b6b62374f8b75", size = 258883, upload-time = "2026-01-26T02:43:57.499Z" }, + { url = "https://files.pythonhosted.org/packages/86/85/7ed40adafea3d4f1c8b916e3b5cc3a8e07dfcdcb9cd72800f4ed3ca1b387/multidict-6.7.1-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c3a32d23520ee37bf327d1e1a656fec76a2edd5c038bf43eddfa0572ec49c60b", size = 242413, upload-time = "2026-01-26T02:43:58.755Z" }, + { url = "https://files.pythonhosted.org/packages/d2/57/b8565ff533e48595503c785f8361ff9a4fde4d67de25c207cd0ba3befd03/multidict-6.7.1-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:9c90fed18bffc0189ba814749fdcc102b536e83a9f738a9003e569acd540a733", size = 268404, upload-time = "2026-01-26T02:44:00.216Z" }, + { url = "https://files.pythonhosted.org/packages/e0/50/9810c5c29350f7258180dfdcb2e52783a0632862eb334c4896ac717cebcb/multidict-6.7.1-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:da62917e6076f512daccfbbde27f46fed1c98fee202f0559adec8ee0de67f71a", size = 269456, upload-time = "2026-01-26T02:44:02.202Z" }, + { url = "https://files.pythonhosted.org/packages/f3/8d/5e5be3ced1d12966fefb5c4ea3b2a5b480afcea36406559442c6e31d4a48/multidict-6.7.1-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:bfde23ef6ed9db7eaee6c37dcec08524cb43903c60b285b172b6c094711b3961", size = 256322, upload-time = "2026-01-26T02:44:03.56Z" }, + { url = "https://files.pythonhosted.org/packages/31/6e/d8a26d81ac166a5592782d208dd90dfdc0a7a218adaa52b45a672b46c122/multidict-6.7.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:3758692429e4e32f1ba0df23219cd0b4fc0a52f476726fff9337d1a57676a582", size = 253955, upload-time = "2026-01-26T02:44:04.845Z" }, + { url = "https://files.pythonhosted.org/packages/59/4c/7c672c8aad41534ba619bcd4ade7a0dc87ed6b8b5c06149b85d3dd03f0cd/multidict-6.7.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:398c1478926eca669f2fd6a5856b6de9c0acf23a2cb59a14c0ba5844fa38077e", size = 251254, upload-time = "2026-01-26T02:44:06.133Z" }, + { url = "https://files.pythonhosted.org/packages/7b/bd/84c24de512cbafbdbc39439f74e967f19570ce7924e3007174a29c348916/multidict-6.7.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:c102791b1c4f3ab36ce4101154549105a53dc828f016356b3e3bcae2e3a039d3", size = 252059, upload-time = "2026-01-26T02:44:07.518Z" }, + { url = "https://files.pythonhosted.org/packages/fa/ba/f5449385510825b73d01c2d4087bf6d2fccc20a2d42ac34df93191d3dd03/multidict-6.7.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:a088b62bd733e2ad12c50dad01b7d0166c30287c166e137433d3b410add807a6", size = 263588, upload-time = "2026-01-26T02:44:09.382Z" }, + { url = "https://files.pythonhosted.org/packages/d7/11/afc7c677f68f75c84a69fe37184f0f82fce13ce4b92f49f3db280b7e92b3/multidict-6.7.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:3d51ff4785d58d3f6c91bdbffcb5e1f7ddfda557727043aa20d20ec4f65e324a", size = 259642, upload-time = "2026-01-26T02:44:10.73Z" }, + { url = "https://files.pythonhosted.org/packages/2b/17/ebb9644da78c4ab36403739e0e6e0e30ebb135b9caf3440825001a0bddcb/multidict-6.7.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:fc5907494fccf3e7d3f94f95c91d6336b092b5fc83811720fae5e2765890dfba", size = 251377, upload-time = "2026-01-26T02:44:12.042Z" }, + { url = "https://files.pythonhosted.org/packages/ca/a4/840f5b97339e27846c46307f2530a2805d9d537d8b8bd416af031cad7fa0/multidict-6.7.1-cp312-cp312-win32.whl", hash = "sha256:28ca5ce2fd9716631133d0e9a9b9a745ad7f60bac2bccafb56aa380fc0b6c511", size = 41887, upload-time = "2026-01-26T02:44:14.245Z" }, + { url = "https://files.pythonhosted.org/packages/80/31/0b2517913687895f5904325c2069d6a3b78f66cc641a86a2baf75a05dcbb/multidict-6.7.1-cp312-cp312-win_amd64.whl", hash = "sha256:fcee94dfbd638784645b066074b338bc9cc155d4b4bffa4adce1615c5a426c19", size = 46053, upload-time = "2026-01-26T02:44:15.371Z" }, + { url = "https://files.pythonhosted.org/packages/0c/5b/aba28e4ee4006ae4c7df8d327d31025d760ffa992ea23812a601d226e682/multidict-6.7.1-cp312-cp312-win_arm64.whl", hash = "sha256:ba0a9fb644d0c1a2194cf7ffb043bd852cea63a57f66fbd33959f7dae18517bf", size = 43307, upload-time = "2026-01-26T02:44:16.852Z" }, + { url = "https://files.pythonhosted.org/packages/f2/22/929c141d6c0dba87d3e1d38fbdf1ba8baba86b7776469f2bc2d3227a1e67/multidict-6.7.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:2b41f5fed0ed563624f1c17630cb9941cf2309d4df00e494b551b5f3e3d67a23", size = 76174, upload-time = "2026-01-26T02:44:18.509Z" }, + { url = "https://files.pythonhosted.org/packages/c7/75/bc704ae15fee974f8fccd871305e254754167dce5f9e42d88a2def741a1d/multidict-6.7.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:84e61e3af5463c19b67ced91f6c634effb89ef8bfc5ca0267f954451ed4bb6a2", size = 45116, upload-time = "2026-01-26T02:44:19.745Z" }, + { url = "https://files.pythonhosted.org/packages/79/76/55cd7186f498ed080a18440c9013011eb548f77ae1b297206d030eb1180a/multidict-6.7.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:935434b9853c7c112eee7ac891bc4cb86455aa631269ae35442cb316790c1445", size = 43524, upload-time = "2026-01-26T02:44:21.571Z" }, + { url = "https://files.pythonhosted.org/packages/e9/3c/414842ef8d5a1628d68edee29ba0e5bcf235dbfb3ccd3ea303a7fe8c72ff/multidict-6.7.1-cp313-cp313-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:432feb25a1cb67fe82a9680b4d65fb542e4635cb3166cd9c01560651ad60f177", size = 249368, upload-time = "2026-01-26T02:44:22.803Z" }, + { url = "https://files.pythonhosted.org/packages/f6/32/befed7f74c458b4a525e60519fe8d87eef72bb1e99924fa2b0f9d97a221e/multidict-6.7.1-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:e82d14e3c948952a1a85503817e038cba5905a3352de76b9a465075d072fba23", size = 256952, upload-time = "2026-01-26T02:44:24.306Z" }, + { url = "https://files.pythonhosted.org/packages/03/d6/c878a44ba877f366630c860fdf74bfb203c33778f12b6ac274936853c451/multidict-6.7.1-cp313-cp313-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:4cfb48c6ea66c83bcaaf7e4dfa7ec1b6bbcf751b7db85a328902796dfde4c060", size = 240317, upload-time = "2026-01-26T02:44:25.772Z" }, + { url = "https://files.pythonhosted.org/packages/68/49/57421b4d7ad2e9e60e25922b08ceb37e077b90444bde6ead629095327a6f/multidict-6.7.1-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:1d540e51b7e8e170174555edecddbd5538105443754539193e3e1061864d444d", size = 267132, upload-time = "2026-01-26T02:44:27.648Z" }, + { url = "https://files.pythonhosted.org/packages/b7/fe/ec0edd52ddbcea2a2e89e174f0206444a61440b40f39704e64dc807a70bd/multidict-6.7.1-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:273d23f4b40f3dce4d6c8a821c741a86dec62cded82e1175ba3d99be128147ed", size = 268140, upload-time = "2026-01-26T02:44:29.588Z" }, + { url = "https://files.pythonhosted.org/packages/b0/73/6e1b01cbeb458807aa0831742232dbdd1fa92bfa33f52a3f176b4ff3dc11/multidict-6.7.1-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9d624335fd4fa1c08a53f8b4be7676ebde19cd092b3895c421045ca87895b429", size = 254277, upload-time = "2026-01-26T02:44:30.902Z" }, + { url = "https://files.pythonhosted.org/packages/6a/b2/5fb8c124d7561a4974c342bc8c778b471ebbeb3cc17df696f034a7e9afe7/multidict-6.7.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:12fad252f8b267cc75b66e8fc51b3079604e8d43a75428ffe193cd9e2195dfd6", size = 252291, upload-time = "2026-01-26T02:44:32.31Z" }, + { url = "https://files.pythonhosted.org/packages/5a/96/51d4e4e06bcce92577fcd488e22600bd38e4fd59c20cb49434d054903bd2/multidict-6.7.1-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:03ede2a6ffbe8ef936b92cb4529f27f42be7f56afcdab5ab739cd5f27fb1cbf9", size = 250156, upload-time = "2026-01-26T02:44:33.734Z" }, + { url = "https://files.pythonhosted.org/packages/db/6b/420e173eec5fba721a50e2a9f89eda89d9c98fded1124f8d5c675f7a0c0f/multidict-6.7.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:90efbcf47dbe33dcf643a1e400d67d59abeac5db07dc3f27d6bdeae497a2198c", size = 249742, upload-time = "2026-01-26T02:44:35.222Z" }, + { url = "https://files.pythonhosted.org/packages/44/a3/ec5b5bd98f306bc2aa297b8c6f11a46714a56b1e6ef5ebda50a4f5d7c5fb/multidict-6.7.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:5c4b9bfc148f5a91be9244d6264c53035c8a0dcd2f51f1c3c6e30e30ebaa1c84", size = 262221, upload-time = "2026-01-26T02:44:36.604Z" }, + { url = "https://files.pythonhosted.org/packages/cd/f7/e8c0d0da0cd1e28d10e624604e1a36bcc3353aaebdfdc3a43c72bc683a12/multidict-6.7.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:401c5a650f3add2472d1d288c26deebc540f99e2fb83e9525007a74cd2116f1d", size = 258664, upload-time = "2026-01-26T02:44:38.008Z" }, + { url = "https://files.pythonhosted.org/packages/52/da/151a44e8016dd33feed44f730bd856a66257c1ee7aed4f44b649fb7edeb3/multidict-6.7.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:97891f3b1b3ffbded884e2916cacf3c6fc87b66bb0dde46f7357404750559f33", size = 249490, upload-time = "2026-01-26T02:44:39.386Z" }, + { url = "https://files.pythonhosted.org/packages/87/af/a3b86bf9630b732897f6fc3f4c4714b90aa4361983ccbdcd6c0339b21b0c/multidict-6.7.1-cp313-cp313-win32.whl", hash = "sha256:e1c5988359516095535c4301af38d8a8838534158f649c05dd1050222321bcb3", size = 41695, upload-time = "2026-01-26T02:44:41.318Z" }, + { url = "https://files.pythonhosted.org/packages/b2/35/e994121b0e90e46134673422dd564623f93304614f5d11886b1b3e06f503/multidict-6.7.1-cp313-cp313-win_amd64.whl", hash = "sha256:960c83bf01a95b12b08fd54324a4eb1d5b52c88932b5cba5d6e712bb3ed12eb5", size = 45884, upload-time = "2026-01-26T02:44:42.488Z" }, + { url = "https://files.pythonhosted.org/packages/ca/61/42d3e5dbf661242a69c97ea363f2d7b46c567da8eadef8890022be6e2ab0/multidict-6.7.1-cp313-cp313-win_arm64.whl", hash = "sha256:563fe25c678aaba333d5399408f5ec3c383ca5b663e7f774dd179a520b8144df", size = 43122, upload-time = "2026-01-26T02:44:43.664Z" }, + { url = "https://files.pythonhosted.org/packages/6d/b3/e6b21c6c4f314bb956016b0b3ef2162590a529b84cb831c257519e7fde44/multidict-6.7.1-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:c76c4bec1538375dad9d452d246ca5368ad6e1c9039dadcf007ae59c70619ea1", size = 83175, upload-time = "2026-01-26T02:44:44.894Z" }, + { url = "https://files.pythonhosted.org/packages/fb/76/23ecd2abfe0957b234f6c960f4ade497f55f2c16aeb684d4ecdbf1c95791/multidict-6.7.1-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:57b46b24b5d5ebcc978da4ec23a819a9402b4228b8a90d9c656422b4bdd8a963", size = 48460, upload-time = "2026-01-26T02:44:46.106Z" }, + { url = "https://files.pythonhosted.org/packages/c4/57/a0ed92b23f3a042c36bc4227b72b97eca803f5f1801c1ab77c8a212d455e/multidict-6.7.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e954b24433c768ce78ab7929e84ccf3422e46deb45a4dc9f93438f8217fa2d34", size = 46930, upload-time = "2026-01-26T02:44:47.278Z" }, + { url = "https://files.pythonhosted.org/packages/b5/66/02ec7ace29162e447f6382c495dc95826bf931d3818799bbef11e8f7df1a/multidict-6.7.1-cp313-cp313t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:3bd231490fa7217cc832528e1cd8752a96f0125ddd2b5749390f7c3ec8721b65", size = 242582, upload-time = "2026-01-26T02:44:48.604Z" }, + { url = "https://files.pythonhosted.org/packages/58/18/64f5a795e7677670e872673aca234162514696274597b3708b2c0d276cce/multidict-6.7.1-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:253282d70d67885a15c8a7716f3a73edf2d635793ceda8173b9ecc21f2fb8292", size = 250031, upload-time = "2026-01-26T02:44:50.544Z" }, + { url = "https://files.pythonhosted.org/packages/c8/ed/e192291dbbe51a8290c5686f482084d31bcd9d09af24f63358c3d42fd284/multidict-6.7.1-cp313-cp313t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:0b4c48648d7649c9335cf1927a8b87fa692de3dcb15faa676c6a6f1f1aabda43", size = 228596, upload-time = "2026-01-26T02:44:51.951Z" }, + { url = "https://files.pythonhosted.org/packages/1e/7e/3562a15a60cf747397e7f2180b0a11dc0c38d9175a650e75fa1b4d325e15/multidict-6.7.1-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:98bc624954ec4d2c7cb074b8eefc2b5d0ce7d482e410df446414355d158fe4ca", size = 257492, upload-time = "2026-01-26T02:44:53.902Z" }, + { url = "https://files.pythonhosted.org/packages/24/02/7d0f9eae92b5249bb50ac1595b295f10e263dd0078ebb55115c31e0eaccd/multidict-6.7.1-cp313-cp313t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:1b99af4d9eec0b49927b4402bcbb58dea89d3e0db8806a4086117019939ad3dd", size = 255899, upload-time = "2026-01-26T02:44:55.316Z" }, + { url = "https://files.pythonhosted.org/packages/00/e3/9b60ed9e23e64c73a5cde95269ef1330678e9c6e34dd4eb6b431b85b5a10/multidict-6.7.1-cp313-cp313t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6aac4f16b472d5b7dc6f66a0d49dd57b0e0902090be16594dc9ebfd3d17c47e7", size = 247970, upload-time = "2026-01-26T02:44:56.783Z" }, + { url = "https://files.pythonhosted.org/packages/3e/06/538e58a63ed5cfb0bd4517e346b91da32fde409d839720f664e9a4ae4f9d/multidict-6.7.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:21f830fe223215dffd51f538e78c172ed7c7f60c9b96a2bf05c4848ad49921c3", size = 245060, upload-time = "2026-01-26T02:44:58.195Z" }, + { url = "https://files.pythonhosted.org/packages/b2/2f/d743a3045a97c895d401e9bd29aaa09b94f5cbdf1bd561609e5a6c431c70/multidict-6.7.1-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:f5dd81c45b05518b9aa4da4aa74e1c93d715efa234fd3e8a179df611cc85e5f4", size = 235888, upload-time = "2026-01-26T02:44:59.57Z" }, + { url = "https://files.pythonhosted.org/packages/38/83/5a325cac191ab28b63c52f14f1131f3b0a55ba3b9aa65a6d0bf2a9b921a0/multidict-6.7.1-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:eb304767bca2bb92fb9c5bd33cedc95baee5bb5f6c88e63706533a1c06ad08c8", size = 243554, upload-time = "2026-01-26T02:45:01.054Z" }, + { url = "https://files.pythonhosted.org/packages/20/1f/9d2327086bd15da2725ef6aae624208e2ef828ed99892b17f60c344e57ed/multidict-6.7.1-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:c9035dde0f916702850ef66460bc4239d89d08df4d02023a5926e7446724212c", size = 252341, upload-time = "2026-01-26T02:45:02.484Z" }, + { url = "https://files.pythonhosted.org/packages/e8/2c/2a1aa0280cf579d0f6eed8ee5211c4f1730bd7e06c636ba2ee6aafda302e/multidict-6.7.1-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:af959b9beeb66c822380f222f0e0a1889331597e81f1ded7f374f3ecb0fd6c52", size = 246391, upload-time = "2026-01-26T02:45:03.862Z" }, + { url = "https://files.pythonhosted.org/packages/e5/03/7ca022ffc36c5a3f6e03b179a5ceb829be9da5783e6fe395f347c0794680/multidict-6.7.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:41f2952231456154ee479651491e94118229844dd7226541788be783be2b5108", size = 243422, upload-time = "2026-01-26T02:45:05.296Z" }, + { url = "https://files.pythonhosted.org/packages/dc/1d/b31650eab6c5778aceed46ba735bd97f7c7d2f54b319fa916c0f96e7805b/multidict-6.7.1-cp313-cp313t-win32.whl", hash = "sha256:df9f19c28adcb40b6aae30bbaa1478c389efd50c28d541d76760199fc1037c32", size = 47770, upload-time = "2026-01-26T02:45:06.754Z" }, + { url = "https://files.pythonhosted.org/packages/ac/5b/2d2d1d522e51285bd61b1e20df8f47ae1a9d80839db0b24ea783b3832832/multidict-6.7.1-cp313-cp313t-win_amd64.whl", hash = "sha256:d54ecf9f301853f2c5e802da559604b3e95bb7a3b01a9c295c6ee591b9882de8", size = 53109, upload-time = "2026-01-26T02:45:08.044Z" }, + { url = "https://files.pythonhosted.org/packages/3d/a3/cc409ba012c83ca024a308516703cf339bdc4b696195644a7215a5164a24/multidict-6.7.1-cp313-cp313t-win_arm64.whl", hash = "sha256:5a37ca18e360377cfda1d62f5f382ff41f2b8c4ccb329ed974cc2e1643440118", size = 45573, upload-time = "2026-01-26T02:45:09.349Z" }, + { url = "https://files.pythonhosted.org/packages/91/cc/db74228a8be41884a567e88a62fd589a913708fcf180d029898c17a9a371/multidict-6.7.1-cp314-cp314-macosx_10_15_universal2.whl", hash = "sha256:8f333ec9c5eb1b7105e3b84b53141e66ca05a19a605368c55450b6ba208cb9ee", size = 75190, upload-time = "2026-01-26T02:45:10.651Z" }, + { url = "https://files.pythonhosted.org/packages/d5/22/492f2246bb5b534abd44804292e81eeaf835388901f0c574bac4eeec73c5/multidict-6.7.1-cp314-cp314-macosx_10_15_x86_64.whl", hash = "sha256:a407f13c188f804c759fc6a9f88286a565c242a76b27626594c133b82883b5c2", size = 44486, upload-time = "2026-01-26T02:45:11.938Z" }, + { url = "https://files.pythonhosted.org/packages/f1/4f/733c48f270565d78b4544f2baddc2fb2a245e5a8640254b12c36ac7ac68e/multidict-6.7.1-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:0e161ddf326db5577c3a4cc2d8648f81456e8a20d40415541587a71620d7a7d1", size = 43219, upload-time = "2026-01-26T02:45:14.346Z" }, + { url = "https://files.pythonhosted.org/packages/24/bb/2c0c2287963f4259c85e8bcbba9182ced8d7fca65c780c38e99e61629d11/multidict-6.7.1-cp314-cp314-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:1e3a8bb24342a8201d178c3b4984c26ba81a577c80d4d525727427460a50c22d", size = 245132, upload-time = "2026-01-26T02:45:15.712Z" }, + { url = "https://files.pythonhosted.org/packages/a7/f9/44d4b3064c65079d2467888794dea218d1601898ac50222ab8a9a8094460/multidict-6.7.1-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:97231140a50f5d447d3164f994b86a0bed7cd016e2682f8650d6a9158e14fd31", size = 252420, upload-time = "2026-01-26T02:45:17.293Z" }, + { url = "https://files.pythonhosted.org/packages/8b/13/78f7275e73fa17b24c9a51b0bd9d73ba64bb32d0ed51b02a746eb876abe7/multidict-6.7.1-cp314-cp314-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6b10359683bd8806a200fd2909e7c8ca3a7b24ec1d8132e483d58e791d881048", size = 233510, upload-time = "2026-01-26T02:45:19.356Z" }, + { url = "https://files.pythonhosted.org/packages/4b/25/8167187f62ae3cbd52da7893f58cb036b47ea3fb67138787c76800158982/multidict-6.7.1-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:283ddac99f7ac25a4acadbf004cb5ae34480bbeb063520f70ce397b281859362", size = 264094, upload-time = "2026-01-26T02:45:20.834Z" }, + { url = "https://files.pythonhosted.org/packages/a1/e7/69a3a83b7b030cf283fb06ce074a05a02322359783424d7edf0f15fe5022/multidict-6.7.1-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:538cec1e18c067d0e6103aa9a74f9e832904c957adc260e61cd9d8cf0c3b3d37", size = 260786, upload-time = "2026-01-26T02:45:22.818Z" }, + { url = "https://files.pythonhosted.org/packages/fe/3b/8ec5074bcfc450fe84273713b4b0a0dd47c0249358f5d82eb8104ffe2520/multidict-6.7.1-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7eee46ccb30ff48a1e35bb818cc90846c6be2b68240e42a78599166722cea709", size = 248483, upload-time = "2026-01-26T02:45:24.368Z" }, + { url = "https://files.pythonhosted.org/packages/48/5a/d5a99e3acbca0e29c5d9cba8f92ceb15dce78bab963b308ae692981e3a5d/multidict-6.7.1-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:fa263a02f4f2dd2d11a7b1bb4362aa7cb1049f84a9235d31adf63f30143469a0", size = 248403, upload-time = "2026-01-26T02:45:25.982Z" }, + { url = "https://files.pythonhosted.org/packages/35/48/e58cd31f6c7d5102f2a4bf89f96b9cf7e00b6c6f3d04ecc44417c00a5a3c/multidict-6.7.1-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:2e1425e2f99ec5bd36c15a01b690a1a2456209c5deed58f95469ffb46039ccbb", size = 240315, upload-time = "2026-01-26T02:45:27.487Z" }, + { url = "https://files.pythonhosted.org/packages/94/33/1cd210229559cb90b6786c30676bb0c58249ff42f942765f88793b41fdce/multidict-6.7.1-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:497394b3239fc6f0e13a78a3e1b61296e72bf1c5f94b4c4eb80b265c37a131cd", size = 245528, upload-time = "2026-01-26T02:45:28.991Z" }, + { url = "https://files.pythonhosted.org/packages/64/f2/6e1107d226278c876c783056b7db43d800bb64c6131cec9c8dfb6903698e/multidict-6.7.1-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:233b398c29d3f1b9676b4b6f75c518a06fcb2ea0b925119fb2c1bc35c05e1601", size = 258784, upload-time = "2026-01-26T02:45:30.503Z" }, + { url = "https://files.pythonhosted.org/packages/4d/c1/11f664f14d525e4a1b5327a82d4de61a1db604ab34c6603bb3c2cc63ad34/multidict-6.7.1-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:93b1818e4a6e0930454f0f2af7dfce69307ca03cdcfb3739bf4d91241967b6c1", size = 251980, upload-time = "2026-01-26T02:45:32.603Z" }, + { url = "https://files.pythonhosted.org/packages/e1/9f/75a9ac888121d0c5bbd4ecf4eead45668b1766f6baabfb3b7f66a410e231/multidict-6.7.1-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:f33dc2a3abe9249ea5d8360f969ec7f4142e7ac45ee7014d8f8d5acddf178b7b", size = 243602, upload-time = "2026-01-26T02:45:34.043Z" }, + { url = "https://files.pythonhosted.org/packages/9a/e7/50bf7b004cc8525d80dbbbedfdc7aed3e4c323810890be4413e589074032/multidict-6.7.1-cp314-cp314-win32.whl", hash = "sha256:3ab8b9d8b75aef9df299595d5388b14530839f6422333357af1339443cff777d", size = 40930, upload-time = "2026-01-26T02:45:36.278Z" }, + { url = "https://files.pythonhosted.org/packages/e0/bf/52f25716bbe93745595800f36fb17b73711f14da59ed0bb2eba141bc9f0f/multidict-6.7.1-cp314-cp314-win_amd64.whl", hash = "sha256:5e01429a929600e7dab7b166062d9bb54a5eed752384c7384c968c2afab8f50f", size = 45074, upload-time = "2026-01-26T02:45:37.546Z" }, + { url = "https://files.pythonhosted.org/packages/97/ab/22803b03285fa3a525f48217963da3a65ae40f6a1b6f6cf2768879e208f9/multidict-6.7.1-cp314-cp314-win_arm64.whl", hash = "sha256:4885cb0e817aef5d00a2e8451d4665c1808378dc27c2705f1bf4ef8505c0d2e5", size = 42471, upload-time = "2026-01-26T02:45:38.889Z" }, + { url = "https://files.pythonhosted.org/packages/e0/6d/f9293baa6146ba9507e360ea0292b6422b016907c393e2f63fc40ab7b7b5/multidict-6.7.1-cp314-cp314t-macosx_10_15_universal2.whl", hash = "sha256:0458c978acd8e6ea53c81eefaddbbee9c6c5e591f41b3f5e8e194780fe026581", size = 82401, upload-time = "2026-01-26T02:45:40.254Z" }, + { url = "https://files.pythonhosted.org/packages/7a/68/53b5494738d83558d87c3c71a486504d8373421c3e0dbb6d0db48ad42ee0/multidict-6.7.1-cp314-cp314t-macosx_10_15_x86_64.whl", hash = "sha256:c0abd12629b0af3cf590982c0b413b1e7395cd4ec026f30986818ab95bfaa94a", size = 48143, upload-time = "2026-01-26T02:45:41.635Z" }, + { url = "https://files.pythonhosted.org/packages/37/e8/5284c53310dcdc99ce5d66563f6e5773531a9b9fe9ec7a615e9bc306b05f/multidict-6.7.1-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:14525a5f61d7d0c94b368a42cff4c9a4e7ba2d52e2672a7b23d84dc86fb02b0c", size = 46507, upload-time = "2026-01-26T02:45:42.99Z" }, + { url = "https://files.pythonhosted.org/packages/e4/fc/6800d0e5b3875568b4083ecf5f310dcf91d86d52573160834fb4bfcf5e4f/multidict-6.7.1-cp314-cp314t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:17307b22c217b4cf05033dabefe68255a534d637c6c9b0cc8382718f87be4262", size = 239358, upload-time = "2026-01-26T02:45:44.376Z" }, + { url = "https://files.pythonhosted.org/packages/41/75/4ad0973179361cdf3a113905e6e088173198349131be2b390f9fa4da5fc6/multidict-6.7.1-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:7a7e590ff876a3eaf1c02a4dfe0724b6e69a9e9de6d8f556816f29c496046e59", size = 246884, upload-time = "2026-01-26T02:45:47.167Z" }, + { url = "https://files.pythonhosted.org/packages/c3/9c/095bb28b5da139bd41fb9a5d5caff412584f377914bd8787c2aa98717130/multidict-6.7.1-cp314-cp314t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:5fa6a95dfee63893d80a34758cd0e0c118a30b8dcb46372bf75106c591b77889", size = 225878, upload-time = "2026-01-26T02:45:48.698Z" }, + { url = "https://files.pythonhosted.org/packages/07/d0/c0a72000243756e8f5a277b6b514fa005f2c73d481b7d9e47cd4568aa2e4/multidict-6.7.1-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:a0543217a6a017692aa6ae5cc39adb75e587af0f3a82288b1492eb73dd6cc2a4", size = 253542, upload-time = "2026-01-26T02:45:50.164Z" }, + { url = "https://files.pythonhosted.org/packages/c0/6b/f69da15289e384ecf2a68837ec8b5ad8c33e973aa18b266f50fe55f24b8c/multidict-6.7.1-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:f99fe611c312b3c1c0ace793f92464d8cd263cc3b26b5721950d977b006b6c4d", size = 252403, upload-time = "2026-01-26T02:45:51.779Z" }, + { url = "https://files.pythonhosted.org/packages/a2/76/b9669547afa5a1a25cd93eaca91c0da1c095b06b6d2d8ec25b713588d3a1/multidict-6.7.1-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9004d8386d133b7e6135679424c91b0b854d2d164af6ea3f289f8f2761064609", size = 244889, upload-time = "2026-01-26T02:45:53.27Z" }, + { url = "https://files.pythonhosted.org/packages/7e/a9/a50d2669e506dad33cfc45b5d574a205587b7b8a5f426f2fbb2e90882588/multidict-6.7.1-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:e628ef0e6859ffd8273c69412a2465c4be4a9517d07261b33334b5ec6f3c7489", size = 241982, upload-time = "2026-01-26T02:45:54.919Z" }, + { url = "https://files.pythonhosted.org/packages/c5/bb/1609558ad8b456b4827d3c5a5b775c93b87878fd3117ed3db3423dfbce1b/multidict-6.7.1-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:841189848ba629c3552035a6a7f5bf3b02eb304e9fea7492ca220a8eda6b0e5c", size = 232415, upload-time = "2026-01-26T02:45:56.981Z" }, + { url = "https://files.pythonhosted.org/packages/d8/59/6f61039d2aa9261871e03ab9dc058a550d240f25859b05b67fd70f80d4b3/multidict-6.7.1-cp314-cp314t-musllinux_1_2_i686.whl", hash = "sha256:ce1bbd7d780bb5a0da032e095c951f7014d6b0a205f8318308140f1a6aba159e", size = 240337, upload-time = "2026-01-26T02:45:58.698Z" }, + { url = "https://files.pythonhosted.org/packages/a1/29/fdc6a43c203890dc2ae9249971ecd0c41deaedfe00d25cb6564b2edd99eb/multidict-6.7.1-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:b26684587228afed0d50cf804cc71062cc9c1cdf55051c4c6345d372947b268c", size = 248788, upload-time = "2026-01-26T02:46:00.862Z" }, + { url = "https://files.pythonhosted.org/packages/a9/14/a153a06101323e4cf086ecee3faadba52ff71633d471f9685c42e3736163/multidict-6.7.1-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:9f9af11306994335398293f9958071019e3ab95e9a707dc1383a35613f6abcb9", size = 242842, upload-time = "2026-01-26T02:46:02.824Z" }, + { url = "https://files.pythonhosted.org/packages/41/5f/604ae839e64a4a6efc80db94465348d3b328ee955e37acb24badbcd24d83/multidict-6.7.1-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:b4938326284c4f1224178a560987b6cf8b4d38458b113d9b8c1db1a836e640a2", size = 240237, upload-time = "2026-01-26T02:46:05.898Z" }, + { url = "https://files.pythonhosted.org/packages/5f/60/c3a5187bf66f6fb546ff4ab8fb5a077cbdd832d7b1908d4365c7f74a1917/multidict-6.7.1-cp314-cp314t-win32.whl", hash = "sha256:98655c737850c064a65e006a3df7c997cd3b220be4ec8fe26215760b9697d4d7", size = 48008, upload-time = "2026-01-26T02:46:07.468Z" }, + { url = "https://files.pythonhosted.org/packages/0c/f7/addf1087b860ac60e6f382240f64fb99f8bfb532bb06f7c542b83c29ca61/multidict-6.7.1-cp314-cp314t-win_amd64.whl", hash = "sha256:497bde6223c212ba11d462853cfa4f0ae6ef97465033e7dc9940cdb3ab5b48e5", size = 53542, upload-time = "2026-01-26T02:46:08.809Z" }, + { url = "https://files.pythonhosted.org/packages/4c/81/4629d0aa32302ef7b2ec65c75a728cc5ff4fa410c50096174c1632e70b3e/multidict-6.7.1-cp314-cp314t-win_arm64.whl", hash = "sha256:2bbd113e0d4af5db41d5ebfe9ccaff89de2120578164f86a5d17d5a576d1e5b2", size = 44719, upload-time = "2026-01-26T02:46:11.146Z" }, + { url = "https://files.pythonhosted.org/packages/81/08/7036c080d7117f28a4af526d794aab6a84463126db031b007717c1a6676e/multidict-6.7.1-py3-none-any.whl", hash = "sha256:55d97cc6dae627efa6a6e548885712d4864b81110ac76fa4e534c03819fa4a56", size = 12319, upload-time = "2026-01-26T02:46:44.004Z" }, +] + +[[package]] +name = "mypy" +version = "1.9.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "mypy-extensions" }, + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/72/1e/a587a862c766a755a58b62d8c00aed11b74a15dc415c1bf5da7b607b0efd/mypy-1.9.0.tar.gz", hash = "sha256:3cc5da0127e6a478cddd906068496a97a7618a21ce9b54bde5bf7e539c7af974", size = 2995901, upload-time = "2024-03-08T16:10:12.412Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/6e/96/40f0f605b1d4e2ad1fb11d21988ce3a3e205886c0fcbd35c9789a214de9a/mypy-1.9.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:aceb1db093b04db5cd390821464504111b8ec3e351eb85afd1433490163d60cd", size = 10725390, upload-time = "2024-03-08T16:10:01.099Z" }, + { url = "https://files.pythonhosted.org/packages/d7/d2/072e40384b53051106b4fcf03537fb88e2a6ad0757d2ab7f6c8c2f188a69/mypy-1.9.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0235391f1c6f6ce487b23b9dbd1327b4ec33bb93934aa986efe8a9563d9349e6", size = 9731292, upload-time = "2024-03-08T16:08:48.463Z" }, + { url = "https://files.pythonhosted.org/packages/85/a5/b7dc7eb69eda899fd07e71403b51b598a1f4df0f452d1da5844374082bcd/mypy-1.9.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d4d5ddc13421ba3e2e082a6c2d74c2ddb3979c39b582dacd53dd5d9431237185", size = 12455450, upload-time = "2024-03-08T16:08:57.375Z" }, + { url = "https://files.pythonhosted.org/packages/1c/1b/3e962a201d2f0f57c9fa1990e0dd6076f4f2f94954ab56e4a701ec3cc070/mypy-1.9.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:190da1ee69b427d7efa8aa0d5e5ccd67a4fb04038c380237a0d96829cb157913", size = 12530368, upload-time = "2024-03-08T16:09:17.061Z" }, + { url = "https://files.pythonhosted.org/packages/72/1f/8b214b69d08cc5e4bd8c3769ac55a43318f3529362ea55e5957774b69924/mypy-1.9.0-cp312-cp312-win_amd64.whl", hash = "sha256:fe28657de3bfec596bbeef01cb219833ad9d38dd5393fc649f4b366840baefe6", size = 9319112, upload-time = "2024-03-08T16:09:07.961Z" }, + { url = "https://files.pythonhosted.org/packages/60/db/0ba2eaedca52bf5276275e8489951c26206030b3d31bf06f00875ae75d5d/mypy-1.9.0-py3-none-any.whl", hash = "sha256:a260627a570559181a9ea5de61ac6297aa5af202f06fd7ab093ce74e7181e43e", size = 2555887, upload-time = "2024-03-08T16:09:48.584Z" }, +] + +[[package]] +name = "mypy-extensions" +version = "1.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a2/6e/371856a3fb9d31ca8dac321cda606860fa4548858c0cc45d9d1d4ca2628b/mypy_extensions-1.1.0.tar.gz", hash = "sha256:52e68efc3284861e772bbcd66823fde5ae21fd2fdb51c62a211403730b916558", size = 6343, upload-time = "2025-04-22T14:54:24.164Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/79/7b/2c79738432f5c924bef5071f933bcc9efd0473bac3b4aa584a6f7c1c8df8/mypy_extensions-1.1.0-py3-none-any.whl", hash = "sha256:1be4cccdb0f2482337c4743e60421de3a356cd97508abadd57d47403e94f5505", size = 4963, upload-time = "2025-04-22T14:54:22.983Z" }, +] + +[[package]] +name = "packaging" +version = "26.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/df/de/0d2b39fb4af88a0258f3bac87dfcbb48e73fbdea4a2ed0e2213f9a4c2f9a/packaging-26.1.tar.gz", hash = "sha256:f042152b681c4bfac5cae2742a55e103d27ab2ec0f3d88037136b6bfe7c9c5de", size = 215519, upload-time = "2026-04-14T21:12:49.362Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/7a/c2/920ef838e2f0028c8262f16101ec09ebd5969864e5a64c4c05fad0617c56/packaging-26.1-py3-none-any.whl", hash = "sha256:5d9c0669c6285e491e0ced2eee587eaf67b670d94a19e94e3984a481aba6802f", size = 95831, upload-time = "2026-04-14T21:12:47.56Z" }, +] + +[[package]] +name = "parso" +version = "0.8.6" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/81/76/a1e769043c0c0c9fe391b702539d594731a4362334cdf4dc25d0c09761e7/parso-0.8.6.tar.gz", hash = "sha256:2b9a0332696df97d454fa67b81618fd69c35a7b90327cbe6ba5c92d2c68a7bfd", size = 401621, upload-time = "2026-02-09T15:45:24.425Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b6/61/fae042894f4296ec49e3f193aff5d7c18440da9e48102c3315e1bc4519a7/parso-0.8.6-py2.py3-none-any.whl", hash = "sha256:2c549f800b70a5c4952197248825584cb00f033b29c692671d3bf08bf380baff", size = 106894, upload-time = "2026-02-09T15:45:21.391Z" }, +] + +[[package]] +name = "pathspec" +version = "1.0.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/fa/36/e27608899f9b8d4dff0617b2d9ab17ca5608956ca44461ac14ac48b44015/pathspec-1.0.4.tar.gz", hash = "sha256:0210e2ae8a21a9137c0d470578cb0e595af87edaa6ebf12ff176f14a02e0e645", size = 131200, upload-time = "2026-01-27T03:59:46.938Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ef/3c/2c197d226f9ea224a9ab8d197933f9da0ae0aac5b6e0f884e2b8d9c8e9f7/pathspec-1.0.4-py3-none-any.whl", hash = "sha256:fb6ae2fd4e7c921a165808a552060e722767cfa526f99ca5156ed2ce45a5c723", size = 55206, upload-time = "2026-01-27T03:59:45.137Z" }, +] + +[[package]] +name = "pdbr" +version = "0.9.7" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pyreadline3", marker = "sys_platform == 'win32'" }, + { name = "rich" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/37/47/1458f133732817d5e98e1323b454b032e70a07589c52562e8985e7add8e5/pdbr-0.9.7.tar.gz", hash = "sha256:8baba0b0dfe0ee2b758da4341507a71c20a6f6a3f92d9ee6bcd9650ae2e805bf", size = 15615, upload-time = "2026-01-05T08:36:11.166Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/84/9e/1ea57f0c7b13cc85f280eded3f24d0ee1f264709289ebc0804dc4608f34e/pdbr-0.9.7-py3-none-any.whl", hash = "sha256:b88c8df940897d01eb0944062b80cd4422e46903b2b910d281b215748029ba21", size = 16201, upload-time = "2026-01-05T08:36:09.942Z" }, +] + +[[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, upload-time = "2023-11-25T09:07:26.339Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772, upload-time = "2023-11-25T06:56:14.81Z" }, +] + +[[package]] +name = "platformdirs" +version = "4.9.6" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9f/4a/0883b8e3802965322523f0b200ecf33d31f10991d0401162f4b23c698b42/platformdirs-4.9.6.tar.gz", hash = "sha256:3bfa75b0ad0db84096ae777218481852c0ebc6c727b3168c1b9e0118e458cf0a", size = 29400, upload-time = "2026-04-09T00:04:10.812Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/75/a6/a0a304dc33b49145b21f4808d763822111e67d1c3a32b524a1baf947b6e1/platformdirs-4.9.6-py3-none-any.whl", hash = "sha256:e61adb1d5e5cb3441b4b7710bea7e4c12250ca49439228cc1021c00dcfac0917", size = 21348, upload-time = "2026-04-09T00:04:09.463Z" }, +] + +[[package]] +name = "pluggy" +version = "1.6.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f9/e2/3e91f31a7d2b083fe6ef3fa267035b518369d9511ffab804f839851d2779/pluggy-1.6.0.tar.gz", hash = "sha256:7dcc130b76258d33b90f61b658791dede3486c3e6bfb003ee5c9bfb396dd22f3", size = 69412, upload-time = "2025-05-15T12:30:07.975Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/54/20/4d324d65cc6d9205fabedc306948156824eb9f0ee1633355a8f7ec5c66bf/pluggy-1.6.0-py3-none-any.whl", hash = "sha256:e920276dd6813095e9377c0bc5566d94c932c33b27a3e3945d8389c374dd4746", size = 20538, upload-time = "2025-05-15T12:30:06.134Z" }, +] + +[[package]] +name = "pprintpp" +version = "0.4.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/06/1a/7737e7a0774da3c3824d654993cf57adc915cb04660212f03406334d8c0b/pprintpp-0.4.0.tar.gz", hash = "sha256:ea826108e2c7f49dc6d66c752973c3fc9749142a798d6b254e1e301cfdbc6403", size = 17995, upload-time = "2018-07-01T01:42:34.87Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4e/d1/e4ed95fdd3ef13b78630280d9e9e240aeb65cc7c544ec57106149c3942fb/pprintpp-0.4.0-py2.py3-none-any.whl", hash = "sha256:b6b4dcdd0c0c0d75e4d7b2f21a9e933e5b2ce62b26e1a54537f9651ae5a5c01d", size = 16952, upload-time = "2018-07-01T01:42:36.496Z" }, +] + +[[package]] +name = "prompt-toolkit" +version = "3.0.52" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "wcwidth" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a1/96/06e01a7b38dce6fe1db213e061a4602dd6032a8a97ef6c1a862537732421/prompt_toolkit-3.0.52.tar.gz", hash = "sha256:28cde192929c8e7321de85de1ddbe736f1375148b02f2e17edd840042b1be855", size = 434198, upload-time = "2025-08-27T15:24:02.057Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/84/03/0d3ce49e2505ae70cf43bc5bb3033955d2fc9f932163e84dc0779cc47f48/prompt_toolkit-3.0.52-py3-none-any.whl", hash = "sha256:9aac639a3bbd33284347de5ad8d68ecc044b91a762dc39b7c21095fcd6a19955", size = 391431, upload-time = "2025-08-27T15:23:59.498Z" }, +] + +[[package]] +name = "propcache" +version = "0.4.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9e/da/e9fc233cf63743258bff22b3dfa7ea5baef7b5bc324af47a0ad89b8ffc6f/propcache-0.4.1.tar.gz", hash = "sha256:f48107a8c637e80362555f37ecf49abe20370e557cc4ab374f04ec4423c97c3d", size = 46442, upload-time = "2025-10-08T19:49:02.291Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a2/0f/f17b1b2b221d5ca28b4b876e8bb046ac40466513960646bda8e1853cdfa2/propcache-0.4.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:e153e9cd40cc8945138822807139367f256f89c6810c2634a4f6902b52d3b4e2", size = 80061, upload-time = "2025-10-08T19:46:46.075Z" }, + { url = "https://files.pythonhosted.org/packages/76/47/8ccf75935f51448ba9a16a71b783eb7ef6b9ee60f5d14c7f8a8a79fbeed7/propcache-0.4.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:cd547953428f7abb73c5ad82cbb32109566204260d98e41e5dfdc682eb7f8403", size = 46037, upload-time = "2025-10-08T19:46:47.23Z" }, + { url = "https://files.pythonhosted.org/packages/0a/b6/5c9a0e42df4d00bfb4a3cbbe5cf9f54260300c88a0e9af1f47ca5ce17ac0/propcache-0.4.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:f048da1b4f243fc44f205dfd320933a951b8d89e0afd4c7cacc762a8b9165207", size = 47324, upload-time = "2025-10-08T19:46:48.384Z" }, + { url = "https://files.pythonhosted.org/packages/9e/d3/6c7ee328b39a81ee877c962469f1e795f9db87f925251efeb0545e0020d0/propcache-0.4.1-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ec17c65562a827bba85e3872ead335f95405ea1674860d96483a02f5c698fa72", size = 225505, upload-time = "2025-10-08T19:46:50.055Z" }, + { url = "https://files.pythonhosted.org/packages/01/5d/1c53f4563490b1d06a684742cc6076ef944bc6457df6051b7d1a877c057b/propcache-0.4.1-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:405aac25c6394ef275dee4c709be43745d36674b223ba4eb7144bf4d691b7367", size = 230242, upload-time = "2025-10-08T19:46:51.815Z" }, + { url = "https://files.pythonhosted.org/packages/20/e1/ce4620633b0e2422207c3cb774a0ee61cac13abc6217763a7b9e2e3f4a12/propcache-0.4.1-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:0013cb6f8dde4b2a2f66903b8ba740bdfe378c943c4377a200551ceb27f379e4", size = 238474, upload-time = "2025-10-08T19:46:53.208Z" }, + { url = "https://files.pythonhosted.org/packages/46/4b/3aae6835b8e5f44ea6a68348ad90f78134047b503765087be2f9912140ea/propcache-0.4.1-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:15932ab57837c3368b024473a525e25d316d8353016e7cc0e5ba9eb343fbb1cf", size = 221575, upload-time = "2025-10-08T19:46:54.511Z" }, + { url = "https://files.pythonhosted.org/packages/6e/a5/8a5e8678bcc9d3a1a15b9a29165640d64762d424a16af543f00629c87338/propcache-0.4.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:031dce78b9dc099f4c29785d9cf5577a3faf9ebf74ecbd3c856a7b92768c3df3", size = 216736, upload-time = "2025-10-08T19:46:56.212Z" }, + { url = "https://files.pythonhosted.org/packages/f1/63/b7b215eddeac83ca1c6b934f89d09a625aa9ee4ba158338854c87210cc36/propcache-0.4.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:ab08df6c9a035bee56e31af99be621526bd237bea9f32def431c656b29e41778", size = 213019, upload-time = "2025-10-08T19:46:57.595Z" }, + { url = "https://files.pythonhosted.org/packages/57/74/f580099a58c8af587cac7ba19ee7cb418506342fbbe2d4a4401661cca886/propcache-0.4.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:4d7af63f9f93fe593afbf104c21b3b15868efb2c21d07d8732c0c4287e66b6a6", size = 220376, upload-time = "2025-10-08T19:46:59.067Z" }, + { url = "https://files.pythonhosted.org/packages/c4/ee/542f1313aff7eaf19c2bb758c5d0560d2683dac001a1c96d0774af799843/propcache-0.4.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:cfc27c945f422e8b5071b6e93169679e4eb5bf73bbcbf1ba3ae3a83d2f78ebd9", size = 226988, upload-time = "2025-10-08T19:47:00.544Z" }, + { url = "https://files.pythonhosted.org/packages/8f/18/9c6b015dd9c6930f6ce2229e1f02fb35298b847f2087ea2b436a5bfa7287/propcache-0.4.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:35c3277624a080cc6ec6f847cbbbb5b49affa3598c4535a0a4682a697aaa5c75", size = 215615, upload-time = "2025-10-08T19:47:01.968Z" }, + { url = "https://files.pythonhosted.org/packages/80/9e/e7b85720b98c45a45e1fca6a177024934dc9bc5f4d5dd04207f216fc33ed/propcache-0.4.1-cp312-cp312-win32.whl", hash = "sha256:671538c2262dadb5ba6395e26c1731e1d52534bfe9ae56d0b5573ce539266aa8", size = 38066, upload-time = "2025-10-08T19:47:03.503Z" }, + { url = "https://files.pythonhosted.org/packages/54/09/d19cff2a5aaac632ec8fc03737b223597b1e347416934c1b3a7df079784c/propcache-0.4.1-cp312-cp312-win_amd64.whl", hash = "sha256:cb2d222e72399fcf5890d1d5cc1060857b9b236adff2792ff48ca2dfd46c81db", size = 41655, upload-time = "2025-10-08T19:47:04.973Z" }, + { url = "https://files.pythonhosted.org/packages/68/ab/6b5c191bb5de08036a8c697b265d4ca76148efb10fa162f14af14fb5f076/propcache-0.4.1-cp312-cp312-win_arm64.whl", hash = "sha256:204483131fb222bdaaeeea9f9e6c6ed0cac32731f75dfc1d4a567fc1926477c1", size = 37789, upload-time = "2025-10-08T19:47:06.077Z" }, + { url = "https://files.pythonhosted.org/packages/bf/df/6d9c1b6ac12b003837dde8a10231a7344512186e87b36e855bef32241942/propcache-0.4.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:43eedf29202c08550aac1d14e0ee619b0430aaef78f85864c1a892294fbc28cf", size = 77750, upload-time = "2025-10-08T19:47:07.648Z" }, + { url = "https://files.pythonhosted.org/packages/8b/e8/677a0025e8a2acf07d3418a2e7ba529c9c33caf09d3c1f25513023c1db56/propcache-0.4.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:d62cdfcfd89ccb8de04e0eda998535c406bf5e060ffd56be6c586cbcc05b3311", size = 44780, upload-time = "2025-10-08T19:47:08.851Z" }, + { url = "https://files.pythonhosted.org/packages/89/a4/92380f7ca60f99ebae761936bc48a72a639e8a47b29050615eef757cb2a7/propcache-0.4.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:cae65ad55793da34db5f54e4029b89d3b9b9490d8abe1b4c7ab5d4b8ec7ebf74", size = 46308, upload-time = "2025-10-08T19:47:09.982Z" }, + { url = "https://files.pythonhosted.org/packages/2d/48/c5ac64dee5262044348d1d78a5f85dd1a57464a60d30daee946699963eb3/propcache-0.4.1-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:333ddb9031d2704a301ee3e506dc46b1fe5f294ec198ed6435ad5b6a085facfe", size = 208182, upload-time = "2025-10-08T19:47:11.319Z" }, + { url = "https://files.pythonhosted.org/packages/c6/0c/cd762dd011a9287389a6a3eb43aa30207bde253610cca06824aeabfe9653/propcache-0.4.1-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:fd0858c20f078a32cf55f7e81473d96dcf3b93fd2ccdb3d40fdf54b8573df3af", size = 211215, upload-time = "2025-10-08T19:47:13.146Z" }, + { url = "https://files.pythonhosted.org/packages/30/3e/49861e90233ba36890ae0ca4c660e95df565b2cd15d4a68556ab5865974e/propcache-0.4.1-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:678ae89ebc632c5c204c794f8dab2837c5f159aeb59e6ed0539500400577298c", size = 218112, upload-time = "2025-10-08T19:47:14.913Z" }, + { url = "https://files.pythonhosted.org/packages/f1/8b/544bc867e24e1bd48f3118cecd3b05c694e160a168478fa28770f22fd094/propcache-0.4.1-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d472aeb4fbf9865e0c6d622d7f4d54a4e101a89715d8904282bb5f9a2f476c3f", size = 204442, upload-time = "2025-10-08T19:47:16.277Z" }, + { url = "https://files.pythonhosted.org/packages/50/a6/4282772fd016a76d3e5c0df58380a5ea64900afd836cec2c2f662d1b9bb3/propcache-0.4.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:4d3df5fa7e36b3225954fba85589da77a0fe6a53e3976de39caf04a0db4c36f1", size = 199398, upload-time = "2025-10-08T19:47:17.962Z" }, + { url = "https://files.pythonhosted.org/packages/3e/ec/d8a7cd406ee1ddb705db2139f8a10a8a427100347bd698e7014351c7af09/propcache-0.4.1-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:ee17f18d2498f2673e432faaa71698032b0127ebf23ae5974eeaf806c279df24", size = 196920, upload-time = "2025-10-08T19:47:19.355Z" }, + { url = "https://files.pythonhosted.org/packages/f6/6c/f38ab64af3764f431e359f8baf9e0a21013e24329e8b85d2da32e8ed07ca/propcache-0.4.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:580e97762b950f993ae618e167e7be9256b8353c2dcd8b99ec100eb50f5286aa", size = 203748, upload-time = "2025-10-08T19:47:21.338Z" }, + { url = "https://files.pythonhosted.org/packages/d6/e3/fa846bd70f6534d647886621388f0a265254d30e3ce47e5c8e6e27dbf153/propcache-0.4.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:501d20b891688eb8e7aa903021f0b72d5a55db40ffaab27edefd1027caaafa61", size = 205877, upload-time = "2025-10-08T19:47:23.059Z" }, + { url = "https://files.pythonhosted.org/packages/e2/39/8163fc6f3133fea7b5f2827e8eba2029a0277ab2c5beee6c1db7b10fc23d/propcache-0.4.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:9a0bd56e5b100aef69bd8562b74b46254e7c8812918d3baa700c8a8009b0af66", size = 199437, upload-time = "2025-10-08T19:47:24.445Z" }, + { url = "https://files.pythonhosted.org/packages/93/89/caa9089970ca49c7c01662bd0eeedfe85494e863e8043565aeb6472ce8fe/propcache-0.4.1-cp313-cp313-win32.whl", hash = "sha256:bcc9aaa5d80322bc2fb24bb7accb4a30f81e90ab8d6ba187aec0744bc302ad81", size = 37586, upload-time = "2025-10-08T19:47:25.736Z" }, + { url = "https://files.pythonhosted.org/packages/f5/ab/f76ec3c3627c883215b5c8080debb4394ef5a7a29be811f786415fc1e6fd/propcache-0.4.1-cp313-cp313-win_amd64.whl", hash = "sha256:381914df18634f5494334d201e98245c0596067504b9372d8cf93f4bb23e025e", size = 40790, upload-time = "2025-10-08T19:47:26.847Z" }, + { url = "https://files.pythonhosted.org/packages/59/1b/e71ae98235f8e2ba5004d8cb19765a74877abf189bc53fc0c80d799e56c3/propcache-0.4.1-cp313-cp313-win_arm64.whl", hash = "sha256:8873eb4460fd55333ea49b7d189749ecf6e55bf85080f11b1c4530ed3034cba1", size = 37158, upload-time = "2025-10-08T19:47:27.961Z" }, + { url = "https://files.pythonhosted.org/packages/83/ce/a31bbdfc24ee0dcbba458c8175ed26089cf109a55bbe7b7640ed2470cfe9/propcache-0.4.1-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:92d1935ee1f8d7442da9c0c4fa7ac20d07e94064184811b685f5c4fada64553b", size = 81451, upload-time = "2025-10-08T19:47:29.445Z" }, + { url = "https://files.pythonhosted.org/packages/25/9c/442a45a470a68456e710d96cacd3573ef26a1d0a60067e6a7d5e655621ed/propcache-0.4.1-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:473c61b39e1460d386479b9b2f337da492042447c9b685f28be4f74d3529e566", size = 46374, upload-time = "2025-10-08T19:47:30.579Z" }, + { url = "https://files.pythonhosted.org/packages/f4/bf/b1d5e21dbc3b2e889ea4327044fb16312a736d97640fb8b6aa3f9c7b3b65/propcache-0.4.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:c0ef0aaafc66fbd87842a3fe3902fd889825646bc21149eafe47be6072725835", size = 48396, upload-time = "2025-10-08T19:47:31.79Z" }, + { url = "https://files.pythonhosted.org/packages/f4/04/5b4c54a103d480e978d3c8a76073502b18db0c4bc17ab91b3cb5092ad949/propcache-0.4.1-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:f95393b4d66bfae908c3ca8d169d5f79cd65636ae15b5e7a4f6e67af675adb0e", size = 275950, upload-time = "2025-10-08T19:47:33.481Z" }, + { url = "https://files.pythonhosted.org/packages/b4/c1/86f846827fb969c4b78b0af79bba1d1ea2156492e1b83dea8b8a6ae27395/propcache-0.4.1-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c07fda85708bc48578467e85099645167a955ba093be0a2dcba962195676e859", size = 273856, upload-time = "2025-10-08T19:47:34.906Z" }, + { url = "https://files.pythonhosted.org/packages/36/1d/fc272a63c8d3bbad6878c336c7a7dea15e8f2d23a544bda43205dfa83ada/propcache-0.4.1-cp313-cp313t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:af223b406d6d000830c6f65f1e6431783fc3f713ba3e6cc8c024d5ee96170a4b", size = 280420, upload-time = "2025-10-08T19:47:36.338Z" }, + { url = "https://files.pythonhosted.org/packages/07/0c/01f2219d39f7e53d52e5173bcb09c976609ba30209912a0680adfb8c593a/propcache-0.4.1-cp313-cp313t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:a78372c932c90ee474559c5ddfffd718238e8673c340dc21fe45c5b8b54559a0", size = 263254, upload-time = "2025-10-08T19:47:37.692Z" }, + { url = "https://files.pythonhosted.org/packages/2d/18/cd28081658ce597898f0c4d174d4d0f3c5b6d4dc27ffafeef835c95eb359/propcache-0.4.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:564d9f0d4d9509e1a870c920a89b2fec951b44bf5ba7d537a9e7c1ccec2c18af", size = 261205, upload-time = "2025-10-08T19:47:39.659Z" }, + { url = "https://files.pythonhosted.org/packages/7a/71/1f9e22eb8b8316701c2a19fa1f388c8a3185082607da8e406a803c9b954e/propcache-0.4.1-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:17612831fda0138059cc5546f4d12a2aacfb9e47068c06af35c400ba58ba7393", size = 247873, upload-time = "2025-10-08T19:47:41.084Z" }, + { url = "https://files.pythonhosted.org/packages/4a/65/3d4b61f36af2b4eddba9def857959f1016a51066b4f1ce348e0cf7881f58/propcache-0.4.1-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:41a89040cb10bd345b3c1a873b2bf36413d48da1def52f268a055f7398514874", size = 262739, upload-time = "2025-10-08T19:47:42.51Z" }, + { url = "https://files.pythonhosted.org/packages/2a/42/26746ab087faa77c1c68079b228810436ccd9a5ce9ac85e2b7307195fd06/propcache-0.4.1-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:e35b88984e7fa64aacecea39236cee32dd9bd8c55f57ba8a75cf2399553f9bd7", size = 263514, upload-time = "2025-10-08T19:47:43.927Z" }, + { url = "https://files.pythonhosted.org/packages/94/13/630690fe201f5502d2403dd3cfd451ed8858fe3c738ee88d095ad2ff407b/propcache-0.4.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:6f8b465489f927b0df505cbe26ffbeed4d6d8a2bbc61ce90eb074ff129ef0ab1", size = 257781, upload-time = "2025-10-08T19:47:45.448Z" }, + { url = "https://files.pythonhosted.org/packages/92/f7/1d4ec5841505f423469efbfc381d64b7b467438cd5a4bbcbb063f3b73d27/propcache-0.4.1-cp313-cp313t-win32.whl", hash = "sha256:2ad890caa1d928c7c2965b48f3a3815c853180831d0e5503d35cf00c472f4717", size = 41396, upload-time = "2025-10-08T19:47:47.202Z" }, + { url = "https://files.pythonhosted.org/packages/48/f0/615c30622316496d2cbbc29f5985f7777d3ada70f23370608c1d3e081c1f/propcache-0.4.1-cp313-cp313t-win_amd64.whl", hash = "sha256:f7ee0e597f495cf415bcbd3da3caa3bd7e816b74d0d52b8145954c5e6fd3ff37", size = 44897, upload-time = "2025-10-08T19:47:48.336Z" }, + { url = "https://files.pythonhosted.org/packages/fd/ca/6002e46eccbe0e33dcd4069ef32f7f1c9e243736e07adca37ae8c4830ec3/propcache-0.4.1-cp313-cp313t-win_arm64.whl", hash = "sha256:929d7cbe1f01bb7baffb33dc14eb5691c95831450a26354cd210a8155170c93a", size = 39789, upload-time = "2025-10-08T19:47:49.876Z" }, + { url = "https://files.pythonhosted.org/packages/8e/5c/bca52d654a896f831b8256683457ceddd490ec18d9ec50e97dfd8fc726a8/propcache-0.4.1-cp314-cp314-macosx_10_13_universal2.whl", hash = "sha256:3f7124c9d820ba5548d431afb4632301acf965db49e666aa21c305cbe8c6de12", size = 78152, upload-time = "2025-10-08T19:47:51.051Z" }, + { url = "https://files.pythonhosted.org/packages/65/9b/03b04e7d82a5f54fb16113d839f5ea1ede58a61e90edf515f6577c66fa8f/propcache-0.4.1-cp314-cp314-macosx_10_13_x86_64.whl", hash = "sha256:c0d4b719b7da33599dfe3b22d3db1ef789210a0597bc650b7cee9c77c2be8c5c", size = 44869, upload-time = "2025-10-08T19:47:52.594Z" }, + { url = "https://files.pythonhosted.org/packages/b2/fa/89a8ef0468d5833a23fff277b143d0573897cf75bd56670a6d28126c7d68/propcache-0.4.1-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:9f302f4783709a78240ebc311b793f123328716a60911d667e0c036bc5dcbded", size = 46596, upload-time = "2025-10-08T19:47:54.073Z" }, + { url = "https://files.pythonhosted.org/packages/86/bd/47816020d337f4a746edc42fe8d53669965138f39ee117414c7d7a340cfe/propcache-0.4.1-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:c80ee5802e3fb9ea37938e7eecc307fb984837091d5fd262bb37238b1ae97641", size = 206981, upload-time = "2025-10-08T19:47:55.715Z" }, + { url = "https://files.pythonhosted.org/packages/df/f6/c5fa1357cc9748510ee55f37173eb31bfde6d94e98ccd9e6f033f2fc06e1/propcache-0.4.1-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:ed5a841e8bb29a55fb8159ed526b26adc5bdd7e8bd7bf793ce647cb08656cdf4", size = 211490, upload-time = "2025-10-08T19:47:57.499Z" }, + { url = "https://files.pythonhosted.org/packages/80/1e/e5889652a7c4a3846683401a48f0f2e5083ce0ec1a8a5221d8058fbd1adf/propcache-0.4.1-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:55c72fd6ea2da4c318e74ffdf93c4fe4e926051133657459131a95c846d16d44", size = 215371, upload-time = "2025-10-08T19:47:59.317Z" }, + { url = "https://files.pythonhosted.org/packages/b2/f2/889ad4b2408f72fe1a4f6a19491177b30ea7bf1a0fd5f17050ca08cfc882/propcache-0.4.1-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:8326e144341460402713f91df60ade3c999d601e7eb5ff8f6f7862d54de0610d", size = 201424, upload-time = "2025-10-08T19:48:00.67Z" }, + { url = "https://files.pythonhosted.org/packages/27/73/033d63069b57b0812c8bd19f311faebeceb6ba31b8f32b73432d12a0b826/propcache-0.4.1-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:060b16ae65bc098da7f6d25bf359f1f31f688384858204fe5d652979e0015e5b", size = 197566, upload-time = "2025-10-08T19:48:02.604Z" }, + { url = "https://files.pythonhosted.org/packages/dc/89/ce24f3dc182630b4e07aa6d15f0ff4b14ed4b9955fae95a0b54c58d66c05/propcache-0.4.1-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:89eb3fa9524f7bec9de6e83cf3faed9d79bffa560672c118a96a171a6f55831e", size = 193130, upload-time = "2025-10-08T19:48:04.499Z" }, + { url = "https://files.pythonhosted.org/packages/a9/24/ef0d5fd1a811fb5c609278d0209c9f10c35f20581fcc16f818da959fc5b4/propcache-0.4.1-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:dee69d7015dc235f526fe80a9c90d65eb0039103fe565776250881731f06349f", size = 202625, upload-time = "2025-10-08T19:48:06.213Z" }, + { url = "https://files.pythonhosted.org/packages/f5/02/98ec20ff5546f68d673df2f7a69e8c0d076b5abd05ca882dc7ee3a83653d/propcache-0.4.1-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:5558992a00dfd54ccbc64a32726a3357ec93825a418a401f5cc67df0ac5d9e49", size = 204209, upload-time = "2025-10-08T19:48:08.432Z" }, + { url = "https://files.pythonhosted.org/packages/a0/87/492694f76759b15f0467a2a93ab68d32859672b646aa8a04ce4864e7932d/propcache-0.4.1-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:c9b822a577f560fbd9554812526831712c1436d2c046cedee4c3796d3543b144", size = 197797, upload-time = "2025-10-08T19:48:09.968Z" }, + { url = "https://files.pythonhosted.org/packages/ee/36/66367de3575db1d2d3f3d177432bd14ee577a39d3f5d1b3d5df8afe3b6e2/propcache-0.4.1-cp314-cp314-win32.whl", hash = "sha256:ab4c29b49d560fe48b696cdcb127dd36e0bc2472548f3bf56cc5cb3da2b2984f", size = 38140, upload-time = "2025-10-08T19:48:11.232Z" }, + { url = "https://files.pythonhosted.org/packages/0c/2a/a758b47de253636e1b8aef181c0b4f4f204bf0dd964914fb2af90a95b49b/propcache-0.4.1-cp314-cp314-win_amd64.whl", hash = "sha256:5a103c3eb905fcea0ab98be99c3a9a5ab2de60228aa5aceedc614c0281cf6153", size = 41257, upload-time = "2025-10-08T19:48:12.707Z" }, + { url = "https://files.pythonhosted.org/packages/34/5e/63bd5896c3fec12edcbd6f12508d4890d23c265df28c74b175e1ef9f4f3b/propcache-0.4.1-cp314-cp314-win_arm64.whl", hash = "sha256:74c1fb26515153e482e00177a1ad654721bf9207da8a494a0c05e797ad27b992", size = 38097, upload-time = "2025-10-08T19:48:13.923Z" }, + { url = "https://files.pythonhosted.org/packages/99/85/9ff785d787ccf9bbb3f3106f79884a130951436f58392000231b4c737c80/propcache-0.4.1-cp314-cp314t-macosx_10_13_universal2.whl", hash = "sha256:824e908bce90fb2743bd6b59db36eb4f45cd350a39637c9f73b1c1ea66f5b75f", size = 81455, upload-time = "2025-10-08T19:48:15.16Z" }, + { url = "https://files.pythonhosted.org/packages/90/85/2431c10c8e7ddb1445c1f7c4b54d886e8ad20e3c6307e7218f05922cad67/propcache-0.4.1-cp314-cp314t-macosx_10_13_x86_64.whl", hash = "sha256:c2b5e7db5328427c57c8e8831abda175421b709672f6cfc3d630c3b7e2146393", size = 46372, upload-time = "2025-10-08T19:48:16.424Z" }, + { url = "https://files.pythonhosted.org/packages/01/20/b0972d902472da9bcb683fa595099911f4d2e86e5683bcc45de60dd05dc3/propcache-0.4.1-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:6f6ff873ed40292cd4969ef5310179afd5db59fdf055897e282485043fc80ad0", size = 48411, upload-time = "2025-10-08T19:48:17.577Z" }, + { url = "https://files.pythonhosted.org/packages/e2/e3/7dc89f4f21e8f99bad3d5ddb3a3389afcf9da4ac69e3deb2dcdc96e74169/propcache-0.4.1-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:49a2dc67c154db2c1463013594c458881a069fcf98940e61a0569016a583020a", size = 275712, upload-time = "2025-10-08T19:48:18.901Z" }, + { url = "https://files.pythonhosted.org/packages/20/67/89800c8352489b21a8047c773067644e3897f02ecbbd610f4d46b7f08612/propcache-0.4.1-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:005f08e6a0529984491e37d8dbc3dd86f84bd78a8ceb5fa9a021f4c48d4984be", size = 273557, upload-time = "2025-10-08T19:48:20.762Z" }, + { url = "https://files.pythonhosted.org/packages/e2/a1/b52b055c766a54ce6d9c16d9aca0cad8059acd9637cdf8aa0222f4a026ef/propcache-0.4.1-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:5c3310452e0d31390da9035c348633b43d7e7feb2e37be252be6da45abd1abcc", size = 280015, upload-time = "2025-10-08T19:48:22.592Z" }, + { url = "https://files.pythonhosted.org/packages/48/c8/33cee30bd890672c63743049f3c9e4be087e6780906bfc3ec58528be59c1/propcache-0.4.1-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:4c3c70630930447f9ef1caac7728c8ad1c56bc5015338b20fed0d08ea2480b3a", size = 262880, upload-time = "2025-10-08T19:48:23.947Z" }, + { url = "https://files.pythonhosted.org/packages/0c/b1/8f08a143b204b418285c88b83d00edbd61afbc2c6415ffafc8905da7038b/propcache-0.4.1-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:8e57061305815dfc910a3634dcf584f08168a8836e6999983569f51a8544cd89", size = 260938, upload-time = "2025-10-08T19:48:25.656Z" }, + { url = "https://files.pythonhosted.org/packages/cf/12/96e4664c82ca2f31e1c8dff86afb867348979eb78d3cb8546a680287a1e9/propcache-0.4.1-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:521a463429ef54143092c11a77e04056dd00636f72e8c45b70aaa3140d639726", size = 247641, upload-time = "2025-10-08T19:48:27.207Z" }, + { url = "https://files.pythonhosted.org/packages/18/ed/e7a9cfca28133386ba52278136d42209d3125db08d0a6395f0cba0c0285c/propcache-0.4.1-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:120c964da3fdc75e3731aa392527136d4ad35868cc556fd09bb6d09172d9a367", size = 262510, upload-time = "2025-10-08T19:48:28.65Z" }, + { url = "https://files.pythonhosted.org/packages/f5/76/16d8bf65e8845dd62b4e2b57444ab81f07f40caa5652b8969b87ddcf2ef6/propcache-0.4.1-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:d8f353eb14ee3441ee844ade4277d560cdd68288838673273b978e3d6d2c8f36", size = 263161, upload-time = "2025-10-08T19:48:30.133Z" }, + { url = "https://files.pythonhosted.org/packages/e7/70/c99e9edb5d91d5ad8a49fa3c1e8285ba64f1476782fed10ab251ff413ba1/propcache-0.4.1-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:ab2943be7c652f09638800905ee1bab2c544e537edb57d527997a24c13dc1455", size = 257393, upload-time = "2025-10-08T19:48:31.567Z" }, + { url = "https://files.pythonhosted.org/packages/08/02/87b25304249a35c0915d236575bc3574a323f60b47939a2262b77632a3ee/propcache-0.4.1-cp314-cp314t-win32.whl", hash = "sha256:05674a162469f31358c30bcaa8883cb7829fa3110bf9c0991fe27d7896c42d85", size = 42546, upload-time = "2025-10-08T19:48:32.872Z" }, + { url = "https://files.pythonhosted.org/packages/cb/ef/3c6ecf8b317aa982f309835e8f96987466123c6e596646d4e6a1dfcd080f/propcache-0.4.1-cp314-cp314t-win_amd64.whl", hash = "sha256:990f6b3e2a27d683cb7602ed6c86f15ee6b43b1194736f9baaeb93d0016633b1", size = 46259, upload-time = "2025-10-08T19:48:34.226Z" }, + { url = "https://files.pythonhosted.org/packages/c4/2d/346e946d4951f37eca1e4f55be0f0174c52cd70720f84029b02f296f4a38/propcache-0.4.1-cp314-cp314t-win_arm64.whl", hash = "sha256:ecef2343af4cc68e05131e45024ba34f6095821988a9d0a02aa7c73fcc448aa9", size = 40428, upload-time = "2025-10-08T19:48:35.441Z" }, + { url = "https://files.pythonhosted.org/packages/5b/5a/bc7b4a4ef808fa59a816c17b20c4bef6884daebbdf627ff2a161da67da19/propcache-0.4.1-py3-none-any.whl", hash = "sha256:af2a6052aeb6cf17d3e46ee169099044fd8224cbaf75c76a2ef596e8163e2237", size = 13305, upload-time = "2025-10-08T19:49:00.792Z" }, +] + +[[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, upload-time = "2020-12-28T15:15:30.155Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993, upload-time = "2020-12-28T15:15:28.35Z" }, +] + +[[package]] +name = "pure-eval" +version = "0.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cd/05/0a34433a064256a578f1783a10da6df098ceaa4a57bbeaa96a6c0352786b/pure_eval-0.2.3.tar.gz", hash = "sha256:5f4e983f40564c576c7c8635ae88db5956bb2229d7e9237d03b3c0b0190eaf42", size = 19752, upload-time = "2024-07-21T12:58:21.801Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8e/37/efad0257dc6e593a18957422533ff0f87ede7c9c6ea010a2177d738fb82f/pure_eval-0.2.3-py3-none-any.whl", hash = "sha256:1db8e35b67b3d218d818ae653e27f06c3aa420901fa7b081ca98cbedc874e0d0", size = 11842, upload-time = "2024-07-21T12:58:20.04Z" }, +] + +[[package]] +name = "pycparser" +version = "3.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/1b/7d/92392ff7815c21062bea51aa7b87d45576f649f16458d78b7cf94b9ab2e6/pycparser-3.0.tar.gz", hash = "sha256:600f49d217304a5902ac3c37e1281c9fe94e4d0489de643a9504c5cdfdfc6b29", size = 103492, upload-time = "2026-01-21T14:26:51.89Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0c/c3/44f3fbbfa403ea2a7c779186dc20772604442dde72947e7d01069cbe98e3/pycparser-3.0-py3-none-any.whl", hash = "sha256:b727414169a36b7d524c1c3e31839a521725078d7b2ff038656844266160a992", size = 48172, upload-time = "2026-01-21T14:26:50.693Z" }, +] + +[[package]] +name = "pydantic" +version = "2.13.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "annotated-types" }, + { name = "pydantic-core" }, + { name = "typing-extensions" }, + { name = "typing-inspection" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/f3/6b/1353beb3d1cd5cf61cdec5b6f87a9872399de3bc5cae0b7ce07ff4de2ab0/pydantic-2.13.1.tar.gz", hash = "sha256:a0f829b279ddd1e39291133fe2539d2aa46cc6b150c1706a270ff0879e3774d2", size = 843746, upload-time = "2026-04-15T14:57:19.398Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/81/5a/2225f4c176dbfed0d809e848b50ef08f70e61daa667b7fa14b0d311ae44d/pydantic-2.13.1-py3-none-any.whl", hash = "sha256:9557ecc2806faaf6037f85b1fbd963d01e30511c48085f0d573650fdeaad378a", size = 471917, upload-time = "2026-04-15T14:57:17.277Z" }, +] + +[[package]] +name = "pydantic-core" +version = "2.46.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a1/93/f97a86a7eb28faa1d038af2fd5d6166418b4433659108a4c311b57128b2d/pydantic_core-2.46.1.tar.gz", hash = "sha256:d408153772d9f298098fb5d620f045bdf0f017af0d5cb6e309ef8c205540caa4", size = 471230, upload-time = "2026-04-15T14:49:34.52Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ce/fb/caaa8ee23861c170f07dbd58fc2be3a2c02a32637693cbb23eef02e84808/pydantic_core-2.46.1-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:ae8c8c5eb4c796944f3166f2f0dab6c761c2c2cc5bd20e5f692128be8600b9a4", size = 2119472, upload-time = "2026-04-15T14:49:45.946Z" }, + { url = "https://files.pythonhosted.org/packages/fa/61/bcffaa52894489ff89e5e1cdde67429914bf083c0db7296bef153020f786/pydantic_core-2.46.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:daba6f5f5b986aa0682623a1a4f8d1ecb0ec00ce09cfa9ca71a3b742bc383e3a", size = 1951230, upload-time = "2026-04-15T14:52:27.646Z" }, + { url = "https://files.pythonhosted.org/packages/f8/95/80d2f43a2a1a1e3220fd329d614aa5a39e0a75d24353a3aaf226e605f1c2/pydantic_core-2.46.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0265f3a2460539ecc97817a80c7a23c458dd84191229b655522a2674f701f14e", size = 1976394, upload-time = "2026-04-15T14:50:32.742Z" }, + { url = "https://files.pythonhosted.org/packages/8d/31/2c5b1a207926b5fc1961a2d11da940129bc3841c36cc4df03014195b2966/pydantic_core-2.46.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:bb16c0156c4b4e94aa3719138cc43c53d30ff21126b6a3af63786dcc0757b56e", size = 2068455, upload-time = "2026-04-15T14:50:01.286Z" }, + { url = "https://files.pythonhosted.org/packages/7d/36/c6aa07274359a51ac62895895325ce90107e811c6cea39d2617a99ef10d7/pydantic_core-2.46.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1b42d80fad8e4b283e1e4138f1142f0d038c46d137aad2f9824ad9086080dd41", size = 2239049, upload-time = "2026-04-15T14:53:02.216Z" }, + { url = "https://files.pythonhosted.org/packages/0a/3f/77cdd0db8bddc714842dfd93f737c863751cf02001c993341504f6b0cd53/pydantic_core-2.46.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9cced85896d5b795293bc36b7e2fb0347a36c828551b50cbba510510d928548c", size = 2318681, upload-time = "2026-04-15T14:50:04.539Z" }, + { url = "https://files.pythonhosted.org/packages/a1/a3/09d929a40e6727274b0b500ad06e1b3f35d4f4665ae1c8ba65acbb17e9b5/pydantic_core-2.46.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a641cb1e74b44c418adaf9f5f450670dbec53511f030d8cde8d8accb66edc363", size = 2096527, upload-time = "2026-04-15T14:53:14.766Z" }, + { url = "https://files.pythonhosted.org/packages/89/ae/544c3a82456ebc254a9fcbe2715bab76c70acf9d291aaea24391147943e4/pydantic_core-2.46.1-cp312-cp312-manylinux_2_31_riscv64.whl", hash = "sha256:191e7a122ab14eb12415fe3f92610fc06c7f1d2b4b9101d24d490d447ac92506", size = 2170407, upload-time = "2026-04-15T14:51:27.138Z" }, + { url = "https://files.pythonhosted.org/packages/9d/ce/0dfd881c7af4c522f47b325707bd9a2cdcf4f40e4f2fd30df0e9a3e8d393/pydantic_core-2.46.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:4fe4ff660f7938b5d92f21529ce331b011aa35e481ab64b7cd03f52384e544bb", size = 2188578, upload-time = "2026-04-15T14:50:39.655Z" }, + { url = "https://files.pythonhosted.org/packages/a1/e9/980ea2a6d5114dd1a62ecc5f56feb3d34555f33bd11043f042e5f7f0724a/pydantic_core-2.46.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:18fcea085b3adc3868d8d19606da52d7a52d8bccd8e28652b0778dbe5e6a6660", size = 2188959, upload-time = "2026-04-15T14:52:42.243Z" }, + { url = "https://files.pythonhosted.org/packages/e7/f1/595e0f50f4bfc56cde2fe558f2b0978f29f2865da894c6226231e17464a5/pydantic_core-2.46.1-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:e8e589e7c9466e022d79e13c5764c2239b2e5a7993ba727822b021234f89b56b", size = 2339973, upload-time = "2026-04-15T14:52:10.642Z" }, + { url = "https://files.pythonhosted.org/packages/49/44/be9f979a6ab6b8c36865ccd92c3a38a760c66055e1f384665f35525134c4/pydantic_core-2.46.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:f78eb3d4027963bdc9baccd177f02a98bf8714bc51fe17153d8b51218918b5bc", size = 2385228, upload-time = "2026-04-15T14:51:00.77Z" }, + { url = "https://files.pythonhosted.org/packages/5b/d4/c826cd711787d240219f01d0d3ca116cb55516b8b95277820aa9c85e1882/pydantic_core-2.46.1-cp312-cp312-win32.whl", hash = "sha256:54fe30c20cab03844dc63bdc6ddca67f74a2eb8482df69c1e5f68396856241be", size = 1978828, upload-time = "2026-04-15T14:50:29.362Z" }, + { url = "https://files.pythonhosted.org/packages/22/05/8a1fcf8181be4c7a9cfc34e5fbf2d9c3866edc9dfd3c48d5401806e0a523/pydantic_core-2.46.1-cp312-cp312-win_amd64.whl", hash = "sha256:aea4e22ed4c53f2774221435e39969a54d2e783f4aee902cdd6c8011415de893", size = 2070015, upload-time = "2026-04-15T14:49:47.301Z" }, + { url = "https://files.pythonhosted.org/packages/61/d5/fea36ad2882b99c174ef4ffbc7ea6523f6abe26060fbc1f77d6441670232/pydantic_core-2.46.1-cp312-cp312-win_arm64.whl", hash = "sha256:f76fb49c34b4d66aa6e552ce9e852ea97a3a06301a9f01ae82f23e449e3a55f8", size = 2030176, upload-time = "2026-04-15T14:50:47.307Z" }, + { url = "https://files.pythonhosted.org/packages/ff/d2/bda39bad2f426cb5078e6ad28076614d3926704196efe0d7a2a19a99025d/pydantic_core-2.46.1-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:cdc8a5762a9c4b9d86e204d555444e3227507c92daba06259ee66595834de47a", size = 2119092, upload-time = "2026-04-15T14:49:50.392Z" }, + { url = "https://files.pythonhosted.org/packages/ee/f3/69631e64d69cb3481494b2bddefe0ddd07771209f74e9106d066f9138c2a/pydantic_core-2.46.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:ba381dfe9c85692c566ecb60fa5a77a697a2a8eebe274ec5e4d6ec15fafad799", size = 1951400, upload-time = "2026-04-15T14:51:06.588Z" }, + { url = "https://files.pythonhosted.org/packages/53/1c/21cb3db6ae997df31be8e91f213081f72ffa641cb45c89b8a1986832b1f9/pydantic_core-2.46.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1593d8de98207466dc070118322fef68307a0cc6a5625e7b386f6fdae57f9ab6", size = 1976864, upload-time = "2026-04-15T14:50:54.804Z" }, + { url = "https://files.pythonhosted.org/packages/91/9c/05c819f734318ce5a6ca24da300d93696c105af4adb90494ee571303afd8/pydantic_core-2.46.1-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8262c74a1af5b0fdf795f5537f7145785a63f9fbf9e15405f547440c30017ed8", size = 2066669, upload-time = "2026-04-15T14:51:42.346Z" }, + { url = "https://files.pythonhosted.org/packages/cb/23/fadddf1c7f2f517f58731aea9b35c914e6005250f08dac9b8e53904cdbaa/pydantic_core-2.46.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4b88949a24182e83fbbb3f7ca9b7858d0d37b735700ea91081434b7d37b3b444", size = 2238737, upload-time = "2026-04-15T14:50:45.558Z" }, + { url = "https://files.pythonhosted.org/packages/23/07/0cd4f95cb0359c8b1ec71e89c3777e7932c8dfeb9cd54740289f310aaead/pydantic_core-2.46.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8f3708cd55537aeaf3fd0ea55df0d68d0da51dcb07cbc8508745b34acc4c6e0", size = 2316258, upload-time = "2026-04-15T14:51:08.471Z" }, + { url = "https://files.pythonhosted.org/packages/0c/40/6fc24c3766a19c222a0d60d652b78f0283339d4cd4c173fab06b7ee76571/pydantic_core-2.46.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f79292435fff1d4f0c18d9cfaf214025cc88e4f5104bfaed53f173621da1c743", size = 2097474, upload-time = "2026-04-15T14:49:56.543Z" }, + { url = "https://files.pythonhosted.org/packages/4b/af/f39795d1ce549e35d0841382b9c616ae211caffb88863147369a8d74fba9/pydantic_core-2.46.1-cp313-cp313-manylinux_2_31_riscv64.whl", hash = "sha256:a2e607aeb59cf4575bb364470288db3b9a1f0e7415d053a322e3e154c1a0802e", size = 2168383, upload-time = "2026-04-15T14:51:29.269Z" }, + { url = "https://files.pythonhosted.org/packages/e6/32/0d563f74582795779df6cc270c3fc220f49f4daf7860d74a5a6cda8491ff/pydantic_core-2.46.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ec5ca190b75878a9f6ae1fc8f5eb678497934475aef3d93204c9fa01e97370b6", size = 2186182, upload-time = "2026-04-15T14:50:19.097Z" }, + { url = "https://files.pythonhosted.org/packages/5c/07/1c10d5ce312fc4cf86d1e50bdcdbb8ef248409597b099cab1b4bb3a093f7/pydantic_core-2.46.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:1f80535259dcdd517d7b8ca588d5ca24b4f337228e583bebedf7a3adcdf5f721", size = 2187859, upload-time = "2026-04-15T14:49:22.974Z" }, + { url = "https://files.pythonhosted.org/packages/92/01/e1f62d4cb39f0913dbf5c95b9b119ef30ddba9493dff8c2b012f0cdd67dc/pydantic_core-2.46.1-cp313-cp313-musllinux_1_1_armv7l.whl", hash = "sha256:24820b3c82c43df61eca30147e42853e6c127d8b868afdc0c162df829e011eb4", size = 2338372, upload-time = "2026-04-15T14:49:53.316Z" }, + { url = "https://files.pythonhosted.org/packages/44/ed/218dfeea6127fb1781a6ceca241ec6edf00e8a8933ff331af2215975a534/pydantic_core-2.46.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:f12794b1dd8ac9fb66619e0b3a0427189f5d5638e55a3de1385121a9b7bf9b39", size = 2384039, upload-time = "2026-04-15T14:53:04.929Z" }, + { url = "https://files.pythonhosted.org/packages/6c/1e/011e763cd059238249fbd5780e0f8d0b04b47f86c8925e22784f3e5fc977/pydantic_core-2.46.1-cp313-cp313-win32.whl", hash = "sha256:9bc09aed935cdf50f09e908923f9efbcca54e9244bd14a5a0e2a6c8d2c21b4e9", size = 1977943, upload-time = "2026-04-15T14:52:17.969Z" }, + { url = "https://files.pythonhosted.org/packages/8c/06/b559a490d3ed106e9b1777b8d5c8112dd8d31716243cd662616f66c1f8ea/pydantic_core-2.46.1-cp313-cp313-win_amd64.whl", hash = "sha256:fac2d6c8615b8b42bee14677861ba09d56ee076ba4a65cfb9c3c3d0cc89042f2", size = 2068729, upload-time = "2026-04-15T14:53:07.288Z" }, + { url = "https://files.pythonhosted.org/packages/9f/52/32a198946e2e19508532aa9da02a61419eb15bd2d96bab57f810f2713e31/pydantic_core-2.46.1-cp313-cp313-win_arm64.whl", hash = "sha256:f978329f12ace9f3cb814a5e44d98bbeced2e36f633132bafa06d2d71332e33e", size = 2029550, upload-time = "2026-04-15T14:52:22.707Z" }, + { url = "https://files.pythonhosted.org/packages/bd/2b/6793fe89ab66cb2d3d6e5768044eab80bba1d0fae8fd904d0a1574712e17/pydantic_core-2.46.1-cp314-cp314-macosx_10_12_x86_64.whl", hash = "sha256:9917cb61effac7ec0f448ef491ec7584526d2193be84ff981e85cbf18b68c42a", size = 2118110, upload-time = "2026-04-15T14:50:52.947Z" }, + { url = "https://files.pythonhosted.org/packages/d2/87/e9a905ddfcc2fd7bd862b340c02be6ab1f827922822d425513635d0ac774/pydantic_core-2.46.1-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:0e749679ca9f8a9d0bff95fb7f6b57bb53f2207fa42ffcc1ec86de7e0029ab89", size = 1948645, upload-time = "2026-04-15T14:51:55.577Z" }, + { url = "https://files.pythonhosted.org/packages/15/23/26e67f86ed62ac9d6f7f3091ee5220bf14b5ac36fb811851d601365ef896/pydantic_core-2.46.1-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f2ecacee70941e233a2dad23f7796a06f86cc10cc2fbd1c97c7dd5b5a79ffa4f", size = 1977576, upload-time = "2026-04-15T14:49:37.58Z" }, + { url = "https://files.pythonhosted.org/packages/b8/78/813c13c0de323d4de54ee2e6fdd69a0271c09ac8dd65a8a000931aa487a5/pydantic_core-2.46.1-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:647d0a2475b8ed471962eed92fa69145b864942f9c6daa10f95ac70676637ae7", size = 2060358, upload-time = "2026-04-15T14:51:40.087Z" }, + { url = "https://files.pythonhosted.org/packages/09/5e/4caf2a15149271fbd2b4d968899a450853c800b85152abcf54b11531417f/pydantic_core-2.46.1-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ac9cde61965b0697fce6e6cc372df9e1ad93734828aac36e9c1c42a22ad02897", size = 2235980, upload-time = "2026-04-15T14:50:34.535Z" }, + { url = "https://files.pythonhosted.org/packages/c2/c1/a2cdabb5da6f5cb63a3558bcafffc20f790fa14ccffbefbfb1370fadc93f/pydantic_core-2.46.1-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0a2eb0864085f8b641fb3f54a2fb35c58aff24b175b80bc8a945050fcde03204", size = 2316800, upload-time = "2026-04-15T14:52:46.999Z" }, + { url = "https://files.pythonhosted.org/packages/76/fd/19d711e4e9331f9d77f222bffc202bf30ea0d74f6419046376bb82f244c8/pydantic_core-2.46.1-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b83ce9fede4bc4fb649281d9857f06d30198b8f70168f18b987518d713111572", size = 2101762, upload-time = "2026-04-15T14:49:24.278Z" }, + { url = "https://files.pythonhosted.org/packages/dc/64/ce95625448e1a4e219390a2923fd594f3fa368599c6b42ac71a5df7238c9/pydantic_core-2.46.1-cp314-cp314-manylinux_2_31_riscv64.whl", hash = "sha256:cb33192753c60f269d2f4a1db8253c95b0df6e04f2989631a8cc1b0f4f6e2e92", size = 2167737, upload-time = "2026-04-15T14:50:41.637Z" }, + { url = "https://files.pythonhosted.org/packages/ad/31/413572d03ca3e73b408f00f54418b91a8be6401451bc791eaeff210328e5/pydantic_core-2.46.1-cp314-cp314-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:96611d51f953f87e1ae97637c01ee596a08b7f494ea00a5afb67ea6547b9f53b", size = 2185658, upload-time = "2026-04-15T14:51:46.799Z" }, + { url = "https://files.pythonhosted.org/packages/36/09/e4f581353bdf3f0c7de8a8b27afd14fc761da29d78146376315a6fedc487/pydantic_core-2.46.1-cp314-cp314-musllinux_1_1_aarch64.whl", hash = "sha256:9b176fa55f9107db5e6c86099aa5bfd934f1d3ba6a8b43f714ddeebaed3f42b7", size = 2184154, upload-time = "2026-04-15T14:52:49.629Z" }, + { url = "https://files.pythonhosted.org/packages/1a/a4/d0d52849933f5a4bf1ad9d8da612792f96469b37e286a269e3ee9c60bbb1/pydantic_core-2.46.1-cp314-cp314-musllinux_1_1_armv7l.whl", hash = "sha256:79a59f63a4ce4f3330e27e6f3ce281dd1099453b637350e97d7cf24c207cd120", size = 2332379, upload-time = "2026-04-15T14:49:55.009Z" }, + { url = "https://files.pythonhosted.org/packages/30/93/25bfb08fdbef419f73290e573899ce938a327628c34e8f3a4bafeea30126/pydantic_core-2.46.1-cp314-cp314-musllinux_1_1_x86_64.whl", hash = "sha256:f200fce071808a385a314b7343f5e3688d7c45746be3d64dc71ee2d3e2a13268", size = 2377964, upload-time = "2026-04-15T14:51:59.649Z" }, + { url = "https://files.pythonhosted.org/packages/15/36/b777766ff83fef1cf97473d64764cd44f38e0d8c269ed06faace9ae17666/pydantic_core-2.46.1-cp314-cp314-win32.whl", hash = "sha256:3a07eccc0559fb9acc26d55b16bf8ebecd7f237c74a9e2c5741367db4e6d8aff", size = 1976450, upload-time = "2026-04-15T14:51:57.665Z" }, + { url = "https://files.pythonhosted.org/packages/7b/4b/4cd19d2437acfc18ca166db5a2067040334991eb862c4ecf2db098c91fbf/pydantic_core-2.46.1-cp314-cp314-win_amd64.whl", hash = "sha256:1706d270309ac7d071ffe393988c471363705feb3d009186e55d17786ada9622", size = 2067750, upload-time = "2026-04-15T14:49:38.941Z" }, + { url = "https://files.pythonhosted.org/packages/7f/a0/490751c0ef8f5b27aae81731859aed1508e72c1a9b5774c6034269db773b/pydantic_core-2.46.1-cp314-cp314-win_arm64.whl", hash = "sha256:22d4e7457ade8af06528012f382bc994a97cc2ce6e119305a70b3deff1e409d6", size = 2021109, upload-time = "2026-04-15T14:50:27.728Z" }, + { url = "https://files.pythonhosted.org/packages/36/3a/2a018968245fffd25d5f1972714121ad309ff2de19d80019ad93494844f9/pydantic_core-2.46.1-cp314-cp314t-macosx_10_12_x86_64.whl", hash = "sha256:607ff9db0b7e2012e7eef78465e69f9a0d7d1c3e7c6a84cf0c4011db0fcc3feb", size = 2111548, upload-time = "2026-04-15T14:52:08.273Z" }, + { url = "https://files.pythonhosted.org/packages/77/5b/4103b6192213217e874e764e5467d2ff10d8873c1147d01fa432ac281880/pydantic_core-2.46.1-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:8cda3eacaea13bd02a1bea7e457cc9fc30b91c5a91245cef9b215140f80dd78c", size = 1926745, upload-time = "2026-04-15T14:50:03.045Z" }, + { url = "https://files.pythonhosted.org/packages/c3/70/602a667cf4be4bec6c3334512b12ae4ea79ce9bfe41dc51be1fd34434453/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b9493279cdc7997fe19e5ed9b41f30cbc3806bd4722adb402fedb6f6d41bd72a", size = 1965922, upload-time = "2026-04-15T14:51:12.555Z" }, + { url = "https://files.pythonhosted.org/packages/a9/24/06a89ce5323e755b7d2812189f9706b87aaebe49b34d247b380502f7992c/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3644e5e10059999202355b6c6616e624909e23773717d8f76deb8a6e2a72328c", size = 2043221, upload-time = "2026-04-15T14:51:18.995Z" }, + { url = "https://files.pythonhosted.org/packages/2c/6e/b1d9ad907d9d76964903903349fd2e33c87db4b993cc44713edcad0fc488/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4ad6c9de57683e26c92730991960c0c3571b8053263b042de2d3e105930b2767", size = 2243655, upload-time = "2026-04-15T14:50:10.718Z" }, + { url = "https://files.pythonhosted.org/packages/ef/73/787abfaad51174641abb04c8aa125322279b40ad7ce23c495f5a69f76554/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:557ebaa27c7617e7088002318c679a8ce685fa048523417cd1ca52b7f516d955", size = 2295976, upload-time = "2026-04-15T14:53:09.694Z" }, + { url = "https://files.pythonhosted.org/packages/56/0b/b7c5a631b6d5153d4a1ea4923b139aea256dc3bd99c8e6c7b312c7733146/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3cd37e39b22b796ba0298fe81e9421dd7b65f97acfbb0fb19b33ffdda7b9a7b4", size = 2103439, upload-time = "2026-04-15T14:50:08.32Z" }, + { url = "https://files.pythonhosted.org/packages/2a/3f/952ee470df69e5674cdec1cbde22331adf643b5cc2ff79f4292d80146ee4/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_31_riscv64.whl", hash = "sha256:6689443b59714992e67d62505cdd2f952d6cf1c14cc9fd9aeec6719befc6f23b", size = 2132871, upload-time = "2026-04-15T14:50:24.445Z" }, + { url = "https://files.pythonhosted.org/packages/e3/8b/1dea3b1e683c60c77a60f710215f90f486755962aa8939dbcb7c0f975ac3/pydantic_core-2.46.1-cp314-cp314t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6f32c41ca1e3456b5dd691827b7c1433c12d5f0058cc186afbb3615bc07d97b8", size = 2168658, upload-time = "2026-04-15T14:52:24.897Z" }, + { url = "https://files.pythonhosted.org/packages/67/97/32ae283810910d274d5ba9f48f856f5f2f612410b78b249f302d297816f5/pydantic_core-2.46.1-cp314-cp314t-musllinux_1_1_aarch64.whl", hash = "sha256:88cd1355578852db83954dc36e4f58f299646916da976147c20cf6892ba5dc43", size = 2171184, upload-time = "2026-04-15T14:52:34.854Z" }, + { url = "https://files.pythonhosted.org/packages/a2/57/c9a855527fe56c2072070640221f53095b0b19eaf651f3c77643c9cabbe3/pydantic_core-2.46.1-cp314-cp314t-musllinux_1_1_armv7l.whl", hash = "sha256:a170fefdb068279a473cc9d34848b85e61d68bfcc2668415b172c5dfc6f213bf", size = 2316573, upload-time = "2026-04-15T14:52:12.871Z" }, + { url = "https://files.pythonhosted.org/packages/37/b3/14c39ffc7399819c5448007c7bcb4e6da5669850cfb7dcbb727594290b48/pydantic_core-2.46.1-cp314-cp314t-musllinux_1_1_x86_64.whl", hash = "sha256:556a63ff1006934dba4eed7ea31b58274c227e29298ec398e4275eda4b905e95", size = 2378340, upload-time = "2026-04-15T14:51:02.619Z" }, + { url = "https://files.pythonhosted.org/packages/01/55/a37461fbb29c053ea4e62cfc5c2d56425cb5efbef8316e63f6d84ae45718/pydantic_core-2.46.1-cp314-cp314t-win32.whl", hash = "sha256:3b146d8336a995f7d7da6d36e4a779b7e7dff2719ac00a1eb8bd3ded00bec87b", size = 1960843, upload-time = "2026-04-15T14:52:06.103Z" }, + { url = "https://files.pythonhosted.org/packages/22/d7/97e1221197d17a27f768363f87ec061519eeeed15bbd315d2e9d1429ff03/pydantic_core-2.46.1-cp314-cp314t-win_amd64.whl", hash = "sha256:f1bc856c958e6fe9ec071e210afe6feb695f2e2e81fd8d2b102f558d364c4c17", size = 2048696, upload-time = "2026-04-15T14:52:52.154Z" }, + { url = "https://files.pythonhosted.org/packages/19/d5/4eac95255c7d35094b46a32ec1e4d80eac94729c694726ee1d69948bd5f0/pydantic_core-2.46.1-cp314-cp314t-win_arm64.whl", hash = "sha256:21a5bfd8a1aa4de60494cdf66b0c912b1495f26a8899896040021fbd6038d989", size = 2022343, upload-time = "2026-04-15T14:49:49.036Z" }, + { url = "https://files.pythonhosted.org/packages/f4/97/95de673a1356a88b2efdaa120eb6af357a81555c35f6809a7a1423ff7aef/pydantic_core-2.46.1-graalpy312-graalpy250_312_native-macosx_10_12_x86_64.whl", hash = "sha256:5f9107a24a4bc00293434dfa95cf8968751ad0dd703b26ea83a75a56f7326041", size = 2107564, upload-time = "2026-04-15T14:50:49.14Z" }, + { url = "https://files.pythonhosted.org/packages/00/fc/a7c16d85211ea9accddc693b7d049f20b0c06440d9264d1e1c074394ee6c/pydantic_core-2.46.1-graalpy312-graalpy250_312_native-macosx_11_0_arm64.whl", hash = "sha256:2b1801ba99876984d0a03362782819238141c4d0f3f67f69093663691332fc35", size = 1939925, upload-time = "2026-04-15T14:50:36.188Z" }, + { url = "https://files.pythonhosted.org/packages/2e/23/87841169d77820ddabeb81d82002c95dcb82163846666d74f5bdeeaec750/pydantic_core-2.46.1-graalpy312-graalpy250_312_native-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b7fd82a91a20ed6d54fa8c91e7a98255b1ff45bf09b051bfe7fe04eb411e232e", size = 1995313, upload-time = "2026-04-15T14:50:22.538Z" }, + { url = "https://files.pythonhosted.org/packages/ea/96/b46609359a354fa9cd336fc5d93334f1c358b756cc81e4b397347a88fa6f/pydantic_core-2.46.1-graalpy312-graalpy250_312_native-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0f135bf07c92c93def97008bc4496d16934da9efefd7204e5f22a2c92523cb1f", size = 2151197, upload-time = "2026-04-15T14:51:22.925Z" }, +] + +[[package]] +name = "pygments" +version = "2.20.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c3/b2/bc9c9196916376152d655522fdcebac55e66de6603a76a02bca1b6414f6c/pygments-2.20.0.tar.gz", hash = "sha256:6757cd03768053ff99f3039c1a36d6c0aa0b263438fcab17520b30a303a82b5f", size = 4955991, upload-time = "2026-03-29T13:29:33.898Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f4/7e/a72dd26f3b0f4f2bf1dd8923c85f7ceb43172af56d63c7383eb62b332364/pygments-2.20.0-py3-none-any.whl", hash = "sha256:81a9e26dd42fd28a23a2d169d86d7ac03b46e2f8b59ed4698fb4785f946d0176", size = 1231151, upload-time = "2026-03-29T13:29:30.038Z" }, +] + +[[package]] +name = "pyreadline3" +version = "3.5.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/0f/49/4cea918a08f02817aabae639e3d0ac046fef9f9180518a3ad394e22da148/pyreadline3-3.5.4.tar.gz", hash = "sha256:8d57d53039a1c75adba8e50dd3d992b28143480816187ea5efbd5c78e6c885b7", size = 99839, upload-time = "2024-09-19T02:40:10.062Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/5a/dc/491b7661614ab97483abf2056be1deee4dc2490ecbf7bff9ab5cdbac86e1/pyreadline3-3.5.4-py3-none-any.whl", hash = "sha256:eaf8e6cc3c49bcccf145fc6067ba8643d1df34d604a1ec0eccbf7a18e6d3fae6", size = 83178, upload-time = "2024-09-19T02:40:08.598Z" }, +] + +[[package]] +name = "pytest" +version = "8.4.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "iniconfig" }, + { name = "packaging" }, + { name = "pluggy" }, + { name = "pygments" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a3/5c/00a0e072241553e1a7496d638deababa67c5058571567b92a7eaa258397c/pytest-8.4.2.tar.gz", hash = "sha256:86c0d0b93306b961d58d62a4db4879f27fe25513d4b969df351abdddb3c30e01", size = 1519618, upload-time = "2025-09-04T14:34:22.711Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a8/a4/20da314d277121d6534b3a980b29035dcd51e6744bd79075a6ce8fa4eb8d/pytest-8.4.2-py3-none-any.whl", hash = "sha256:872f880de3fc3a5bdc88a11b39c9710c3497a547cfa9320bc3c5e62fbf272e79", size = 365750, upload-time = "2025-09-04T14:34:20.226Z" }, +] + +[[package]] +name = "pytest-asyncio" +version = "0.25.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/94/18/82fcb4ee47d66d99f6cd1efc0b11b2a25029f303c599a5afda7c1bca4254/pytest_asyncio-0.25.0.tar.gz", hash = "sha256:8c0610303c9e0442a5db8604505fc0f545456ba1528824842b37b4a626cbf609", size = 53298, upload-time = "2024-12-13T06:12:44.53Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/88/56/2ee0cab25c11d4e38738a2a98c645a8f002e2ecf7b5ed774c70d53b92bb1/pytest_asyncio-0.25.0-py3-none-any.whl", hash = "sha256:db5432d18eac6b7e28b46dcd9b69921b55c3b1086e85febfe04e70b18d9e81b3", size = 19245, upload-time = "2024-12-13T06:12:41.805Z" }, +] + +[[package]] +name = "pytest-cov" +version = "7.1.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "coverage" }, + { name = "pluggy" }, + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b1/51/a849f96e117386044471c8ec2bd6cfebacda285da9525c9106aeb28da671/pytest_cov-7.1.0.tar.gz", hash = "sha256:30674f2b5f6351aa09702a9c8c364f6a01c27aae0c1366ae8016160d1efc56b2", size = 55592, upload-time = "2026-03-21T20:11:16.284Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9d/7a/d968e294073affff457b041c2be9868a40c1c71f4a35fcc1e45e5493067b/pytest_cov-7.1.0-py3-none-any.whl", hash = "sha256:a0461110b7865f9a271aa1b51e516c9a95de9d696734a2f71e3e78f46e1d4678", size = 22876, upload-time = "2026-03-21T20:11:14.438Z" }, +] + +[[package]] +name = "pytest-custom-exit-code" +version = "0.3.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/92/9d/e1eb0af5e96a5c34f59b9aa69dfb680764420fe60f2ec28cfbc5339f99f8/pytest-custom_exit_code-0.3.0.tar.gz", hash = "sha256:51ffff0ee2c1ddcc1242e2ddb2a5fd02482717e33a2326ef330e3aa430244635", size = 3633, upload-time = "2019-08-07T09:45:15.781Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/35/a0/effb6cbbccfd1c106c572d3d619b3418d71093afb4cd4f91f51e6a1799d2/pytest_custom_exit_code-0.3.0-py3-none-any.whl", hash = "sha256:6e0ce6e57ce3a583cb7e5023f7d1021e19dfec22be41d9ad345bae2fc61caf3b", size = 4055, upload-time = "2019-08-07T09:45:13.767Z" }, +] + +[[package]] +name = "pytest-icdiff" +version = "0.9" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "icdiff" }, + { name = "pprintpp" }, + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/5a/0c/66e1e2590e98f4428e374a3b6448dc086a908d15b1e24b914539d13b7ac4/pytest-icdiff-0.9.tar.gz", hash = "sha256:13aede616202e57fcc882568b64589002ef85438046f012ac30a8d959dac8b75", size = 7110, upload-time = "2023-12-05T11:18:30.192Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e2/e1/cafe1edf7a30be6fa1bbbf43f7af12b34682eadcf19eb6e9f7352062c422/pytest_icdiff-0.9-py3-none-any.whl", hash = "sha256:efee0da3bd1b24ef2d923751c5c547fbb8df0a46795553fba08ef57c3ca03d82", size = 4994, upload-time = "2023-12-05T11:18:28.572Z" }, +] + +[[package]] +name = "pytest-instafail" +version = "0.5.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/86/bd/e0ba6c3cd20b9aa445f0af229f3a9582cce589f083537978a23e6f14e310/pytest-instafail-0.5.0.tar.gz", hash = "sha256:33a606f7e0c8e646dc3bfee0d5e3a4b7b78ef7c36168cfa1f3d93af7ca706c9e", size = 5849, upload-time = "2023-03-31T17:17:32.161Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e8/c0/c32dc39fc172e684fdb3d30169843efb65c067be1e12689af4345731126e/pytest_instafail-0.5.0-py3-none-any.whl", hash = "sha256:6855414487e9e4bb76a118ce952c3c27d3866af15487506c4ded92eb72387819", size = 4176, upload-time = "2023-03-31T17:17:30.065Z" }, +] + +[[package]] +name = "pytest-mock" +version = "3.15.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/68/14/eb014d26be205d38ad5ad20d9a80f7d201472e08167f0bb4361e251084a9/pytest_mock-3.15.1.tar.gz", hash = "sha256:1849a238f6f396da19762269de72cb1814ab44416fa73a8686deac10b0d87a0f", size = 34036, upload-time = "2025-09-16T16:37:27.081Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/5a/cc/06253936f4a7fa2e0f48dfe6d851d9c56df896a9ab09ac019d70b760619c/pytest_mock-3.15.1-py3-none-any.whl", hash = "sha256:0a25e2eb88fe5168d535041d09a4529a188176ae608a6d249ee65abc0949630d", size = 10095, upload-time = "2025-09-16T16:37:25.734Z" }, +] + +[[package]] +name = "pytest-rerunfailures" +version = "16.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "packaging" }, + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/de/04/71e9520551fc8fe2cf5c1a1842e4e600265b0815f2016b7c27ec85688682/pytest_rerunfailures-16.1.tar.gz", hash = "sha256:c38b266db8a808953ebd71ac25c381cb1981a78ff9340a14bcb9f1b9bff1899e", size = 30889, upload-time = "2025-10-10T07:06:01.238Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/77/54/60eabb34445e3db3d3d874dc1dfa72751bfec3265bd611cb13c8b290adea/pytest_rerunfailures-16.1-py3-none-any.whl", hash = "sha256:5d11b12c0ca9a1665b5054052fcc1084f8deadd9328962745ef6b04e26382e86", size = 14093, upload-time = "2025-10-10T07:06:00.019Z" }, +] + +[[package]] +name = "pytest-timeouts" +version = "1.2.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/cf/47/9a8d7fa18736d69d218e303345dfe21755e956fd863b0f043ef854ff84a6/pytest-timeouts-1.2.1.tar.gz", hash = "sha256:390351afc7ecb422ea0ec38081e0acd91cad416b383944a9a3358087de50c2fb", size = 5002, upload-time = "2019-09-21T06:03:25.167Z" } + +[[package]] +name = "pytest-unordered" +version = "0.7.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/bd/3e/6ec9ec74551804c9e005d5b3cbe1fd663f03ed3bd4bdb1ce764c3d334d8e/pytest_unordered-0.7.0.tar.gz", hash = "sha256:0f953a438db00a9f6f99a0f4727f2d75e72dd93319b3d548a97ec9db4903a44f", size = 7930, upload-time = "2025-06-03T12:56:04.289Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/5c/95/ae2875e19472797e9672b65412858ab6639d8e55defd9859241e5ff80d02/pytest_unordered-0.7.0-py3-none-any.whl", hash = "sha256:486b26d24a2d3b879a275c3d16d14eda1bd9c32aafddbb17b98ac755daba7584", size = 6210, upload-time = "2025-06-03T12:36:06.66Z" }, +] + +[[package]] +name = "pytest-xdist" +version = "3.8.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "execnet" }, + { name = "pytest" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/78/b4/439b179d1ff526791eb921115fca8e44e596a13efeda518b9d845a619450/pytest_xdist-3.8.0.tar.gz", hash = "sha256:7e578125ec9bc6050861aa93f2d59f1d8d085595d6551c2c90b6f4fad8d3a9f1", size = 88069, upload-time = "2025-07-01T13:30:59.346Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ca/31/d4e37e9e550c2b92a9cbc2e4d0b7420a27224968580b5a447f420847c975/pytest_xdist-3.8.0-py3-none-any.whl", hash = "sha256:202ca578cfeb7370784a8c33d6d05bc6e13b4f25b5053c30a152269fd10f0b88", size = 46396, upload-time = "2025-07-01T13:30:56.632Z" }, +] + +[[package]] +name = "pytokens" +version = "0.4.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b6/34/b4e015b99031667a7b960f888889c5bd34ef585c85e1cb56a594b92836ac/pytokens-0.4.1.tar.gz", hash = "sha256:292052fe80923aae2260c073f822ceba21f3872ced9a68bb7953b348e561179a", size = 23015, upload-time = "2026-01-30T01:03:45.924Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/41/5d/e44573011401fb82e9d51e97f1290ceb377800fb4eed650b96f4753b499c/pytokens-0.4.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:140709331e846b728475786df8aeb27d24f48cbcf7bcd449f8de75cae7a45083", size = 160663, upload-time = "2026-01-30T01:03:06.473Z" }, + { url = "https://files.pythonhosted.org/packages/f0/e6/5bbc3019f8e6f21d09c41f8b8654536117e5e211a85d89212d59cbdab381/pytokens-0.4.1-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:6d6c4268598f762bc8e91f5dbf2ab2f61f7b95bdc07953b602db879b3c8c18e1", size = 255626, upload-time = "2026-01-30T01:03:08.177Z" }, + { url = "https://files.pythonhosted.org/packages/bf/3c/2d5297d82286f6f3d92770289fd439956b201c0a4fc7e72efb9b2293758e/pytokens-0.4.1-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:24afde1f53d95348b5a0eb19488661147285ca4dd7ed752bbc3e1c6242a304d1", size = 269779, upload-time = "2026-01-30T01:03:09.756Z" }, + { url = "https://files.pythonhosted.org/packages/20/01/7436e9ad693cebda0551203e0bf28f7669976c60ad07d6402098208476de/pytokens-0.4.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:5ad948d085ed6c16413eb5fec6b3e02fa00dc29a2534f088d3302c47eb59adf9", size = 268076, upload-time = "2026-01-30T01:03:10.957Z" }, + { url = "https://files.pythonhosted.org/packages/2e/df/533c82a3c752ba13ae7ef238b7f8cdd272cf1475f03c63ac6cf3fcfb00b6/pytokens-0.4.1-cp312-cp312-win_amd64.whl", hash = "sha256:3f901fe783e06e48e8cbdc82d631fca8f118333798193e026a50ce1b3757ea68", size = 103552, upload-time = "2026-01-30T01:03:12.066Z" }, + { url = "https://files.pythonhosted.org/packages/cb/dc/08b1a080372afda3cceb4f3c0a7ba2bde9d6a5241f1edb02a22a019ee147/pytokens-0.4.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:8bdb9d0ce90cbf99c525e75a2fa415144fd570a1ba987380190e8b786bc6ef9b", size = 160720, upload-time = "2026-01-30T01:03:13.843Z" }, + { url = "https://files.pythonhosted.org/packages/64/0c/41ea22205da480837a700e395507e6a24425151dfb7ead73343d6e2d7ffe/pytokens-0.4.1-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5502408cab1cb18e128570f8d598981c68a50d0cbd7c61312a90507cd3a1276f", size = 254204, upload-time = "2026-01-30T01:03:14.886Z" }, + { url = "https://files.pythonhosted.org/packages/e0/d2/afe5c7f8607018beb99971489dbb846508f1b8f351fcefc225fcf4b2adc0/pytokens-0.4.1-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:29d1d8fb1030af4d231789959f21821ab6325e463f0503a61d204343c9b355d1", size = 268423, upload-time = "2026-01-30T01:03:15.936Z" }, + { url = "https://files.pythonhosted.org/packages/68/d4/00ffdbd370410c04e9591da9220a68dc1693ef7499173eb3e30d06e05ed1/pytokens-0.4.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:970b08dd6b86058b6dc07efe9e98414f5102974716232d10f32ff39701e841c4", size = 266859, upload-time = "2026-01-30T01:03:17.458Z" }, + { url = "https://files.pythonhosted.org/packages/a7/c9/c3161313b4ca0c601eeefabd3d3b576edaa9afdefd32da97210700e47652/pytokens-0.4.1-cp313-cp313-win_amd64.whl", hash = "sha256:9bd7d7f544d362576be74f9d5901a22f317efc20046efe2034dced238cbbfe78", size = 103520, upload-time = "2026-01-30T01:03:18.652Z" }, + { url = "https://files.pythonhosted.org/packages/8f/a7/b470f672e6fc5fee0a01d9e75005a0e617e162381974213a945fcd274843/pytokens-0.4.1-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:4a14d5f5fc78ce85e426aa159489e2d5961acf0e47575e08f35584009178e321", size = 160821, upload-time = "2026-01-30T01:03:19.684Z" }, + { url = "https://files.pythonhosted.org/packages/80/98/e83a36fe8d170c911f864bfded690d2542bfcfacb9c649d11a9e6eb9dc41/pytokens-0.4.1-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:97f50fd18543be72da51dd505e2ed20d2228c74e0464e4262e4899797803d7fa", size = 254263, upload-time = "2026-01-30T01:03:20.834Z" }, + { url = "https://files.pythonhosted.org/packages/0f/95/70d7041273890f9f97a24234c00b746e8da86df462620194cef1d411ddeb/pytokens-0.4.1-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:dc74c035f9bfca0255c1af77ddd2d6ae8419012805453e4b0e7513e17904545d", size = 268071, upload-time = "2026-01-30T01:03:21.888Z" }, + { url = "https://files.pythonhosted.org/packages/da/79/76e6d09ae19c99404656d7db9c35dfd20f2086f3eb6ecb496b5b31163bad/pytokens-0.4.1-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:f66a6bbe741bd431f6d741e617e0f39ec7257ca1f89089593479347cc4d13324", size = 271716, upload-time = "2026-01-30T01:03:23.633Z" }, + { url = "https://files.pythonhosted.org/packages/79/37/482e55fa1602e0a7ff012661d8c946bafdc05e480ea5a32f4f7e336d4aa9/pytokens-0.4.1-cp314-cp314-win_amd64.whl", hash = "sha256:b35d7e5ad269804f6697727702da3c517bb8a5228afa450ab0fa787732055fc9", size = 104539, upload-time = "2026-01-30T01:03:24.788Z" }, + { url = "https://files.pythonhosted.org/packages/30/e8/20e7db907c23f3d63b0be3b8a4fd1927f6da2395f5bcc7f72242bb963dfe/pytokens-0.4.1-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:8fcb9ba3709ff77e77f1c7022ff11d13553f3c30299a9fe246a166903e9091eb", size = 168474, upload-time = "2026-01-30T01:03:26.428Z" }, + { url = "https://files.pythonhosted.org/packages/d6/81/88a95ee9fafdd8f5f3452107748fd04c24930d500b9aba9738f3ade642cc/pytokens-0.4.1-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:79fc6b8699564e1f9b521582c35435f1bd32dd06822322ec44afdeba666d8cb3", size = 290473, upload-time = "2026-01-30T01:03:27.415Z" }, + { url = "https://files.pythonhosted.org/packages/cf/35/3aa899645e29b6375b4aed9f8d21df219e7c958c4c186b465e42ee0a06bf/pytokens-0.4.1-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d31b97b3de0f61571a124a00ffe9a81fb9939146c122c11060725bd5aea79975", size = 303485, upload-time = "2026-01-30T01:03:28.558Z" }, + { url = "https://files.pythonhosted.org/packages/52/a0/07907b6ff512674d9b201859f7d212298c44933633c946703a20c25e9d81/pytokens-0.4.1-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:967cf6e3fd4adf7de8fc73cd3043754ae79c36475c1c11d514fc72cf5490094a", size = 306698, upload-time = "2026-01-30T01:03:29.653Z" }, + { url = "https://files.pythonhosted.org/packages/39/2a/cbbf9250020a4a8dd53ba83a46c097b69e5eb49dd14e708f496f548c6612/pytokens-0.4.1-cp314-cp314t-win_amd64.whl", hash = "sha256:584c80c24b078eec1e227079d56dc22ff755e0ba8654d8383b2c549107528918", size = 116287, upload-time = "2026-01-30T01:03:30.912Z" }, + { url = "https://files.pythonhosted.org/packages/c6/78/397db326746f0a342855b81216ae1f0a32965deccfd7c830a2dbc66d2483/pytokens-0.4.1-py3-none-any.whl", hash = "sha256:26cef14744a8385f35d0e095dc8b3a7583f6c953c2e3d269c7f82484bf5ad2de", size = 13729, upload-time = "2026-01-30T01:03:45.029Z" }, +] + +[[package]] +name = "pyyaml" +version = "6.0.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/05/8e/961c0007c59b8dd7729d542c61a4d537767a59645b82a0b521206e1e25c2/pyyaml-6.0.3.tar.gz", hash = "sha256:d76623373421df22fb4cf8817020cbb7ef15c725b9d5e45f17e189bfc384190f", size = 130960, upload-time = "2025-09-25T21:33:16.546Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/33/422b98d2195232ca1826284a76852ad5a86fe23e31b009c9886b2d0fb8b2/pyyaml-6.0.3-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:7f047e29dcae44602496db43be01ad42fc6f1cc0d8cd6c83d342306c32270196", size = 182063, upload-time = "2025-09-25T21:32:11.445Z" }, + { url = "https://files.pythonhosted.org/packages/89/a0/6cf41a19a1f2f3feab0e9c0b74134aa2ce6849093d5517a0c550fe37a648/pyyaml-6.0.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:fc09d0aa354569bc501d4e787133afc08552722d3ab34836a80547331bb5d4a0", size = 173973, upload-time = "2025-09-25T21:32:12.492Z" }, + { url = "https://files.pythonhosted.org/packages/ed/23/7a778b6bd0b9a8039df8b1b1d80e2e2ad78aa04171592c8a5c43a56a6af4/pyyaml-6.0.3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:9149cad251584d5fb4981be1ecde53a1ca46c891a79788c0df828d2f166bda28", size = 775116, upload-time = "2025-09-25T21:32:13.652Z" }, + { url = "https://files.pythonhosted.org/packages/65/30/d7353c338e12baef4ecc1b09e877c1970bd3382789c159b4f89d6a70dc09/pyyaml-6.0.3-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:5fdec68f91a0c6739b380c83b951e2c72ac0197ace422360e6d5a959d8d97b2c", size = 844011, upload-time = "2025-09-25T21:32:15.21Z" }, + { url = "https://files.pythonhosted.org/packages/8b/9d/b3589d3877982d4f2329302ef98a8026e7f4443c765c46cfecc8858c6b4b/pyyaml-6.0.3-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:ba1cc08a7ccde2d2ec775841541641e4548226580ab850948cbfda66a1befcdc", size = 807870, upload-time = "2025-09-25T21:32:16.431Z" }, + { url = "https://files.pythonhosted.org/packages/05/c0/b3be26a015601b822b97d9149ff8cb5ead58c66f981e04fedf4e762f4bd4/pyyaml-6.0.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:8dc52c23056b9ddd46818a57b78404882310fb473d63f17b07d5c40421e47f8e", size = 761089, upload-time = "2025-09-25T21:32:17.56Z" }, + { url = "https://files.pythonhosted.org/packages/be/8e/98435a21d1d4b46590d5459a22d88128103f8da4c2d4cb8f14f2a96504e1/pyyaml-6.0.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:41715c910c881bc081f1e8872880d3c650acf13dfa8214bad49ed4cede7c34ea", size = 790181, upload-time = "2025-09-25T21:32:18.834Z" }, + { url = "https://files.pythonhosted.org/packages/74/93/7baea19427dcfbe1e5a372d81473250b379f04b1bd3c4c5ff825e2327202/pyyaml-6.0.3-cp312-cp312-win32.whl", hash = "sha256:96b533f0e99f6579b3d4d4995707cf36df9100d67e0c8303a0c55b27b5f99bc5", size = 137658, upload-time = "2025-09-25T21:32:20.209Z" }, + { url = "https://files.pythonhosted.org/packages/86/bf/899e81e4cce32febab4fb42bb97dcdf66bc135272882d1987881a4b519e9/pyyaml-6.0.3-cp312-cp312-win_amd64.whl", hash = "sha256:5fcd34e47f6e0b794d17de1b4ff496c00986e1c83f7ab2fb8fcfe9616ff7477b", size = 154003, upload-time = "2025-09-25T21:32:21.167Z" }, + { url = "https://files.pythonhosted.org/packages/1a/08/67bd04656199bbb51dbed1439b7f27601dfb576fb864099c7ef0c3e55531/pyyaml-6.0.3-cp312-cp312-win_arm64.whl", hash = "sha256:64386e5e707d03a7e172c0701abfb7e10f0fb753ee1d773128192742712a98fd", size = 140344, upload-time = "2025-09-25T21:32:22.617Z" }, + { url = "https://files.pythonhosted.org/packages/d1/11/0fd08f8192109f7169db964b5707a2f1e8b745d4e239b784a5a1dd80d1db/pyyaml-6.0.3-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:8da9669d359f02c0b91ccc01cac4a67f16afec0dac22c2ad09f46bee0697eba8", size = 181669, upload-time = "2025-09-25T21:32:23.673Z" }, + { url = "https://files.pythonhosted.org/packages/b1/16/95309993f1d3748cd644e02e38b75d50cbc0d9561d21f390a76242ce073f/pyyaml-6.0.3-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:2283a07e2c21a2aa78d9c4442724ec1eb15f5e42a723b99cb3d822d48f5f7ad1", size = 173252, upload-time = "2025-09-25T21:32:25.149Z" }, + { url = "https://files.pythonhosted.org/packages/50/31/b20f376d3f810b9b2371e72ef5adb33879b25edb7a6d072cb7ca0c486398/pyyaml-6.0.3-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ee2922902c45ae8ccada2c5b501ab86c36525b883eff4255313a253a3160861c", size = 767081, upload-time = "2025-09-25T21:32:26.575Z" }, + { url = "https://files.pythonhosted.org/packages/49/1e/a55ca81e949270d5d4432fbbd19dfea5321eda7c41a849d443dc92fd1ff7/pyyaml-6.0.3-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:a33284e20b78bd4a18c8c2282d549d10bc8408a2a7ff57653c0cf0b9be0afce5", size = 841159, upload-time = "2025-09-25T21:32:27.727Z" }, + { url = "https://files.pythonhosted.org/packages/74/27/e5b8f34d02d9995b80abcef563ea1f8b56d20134d8f4e5e81733b1feceb2/pyyaml-6.0.3-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0f29edc409a6392443abf94b9cf89ce99889a1dd5376d94316ae5145dfedd5d6", size = 801626, upload-time = "2025-09-25T21:32:28.878Z" }, + { url = "https://files.pythonhosted.org/packages/f9/11/ba845c23988798f40e52ba45f34849aa8a1f2d4af4b798588010792ebad6/pyyaml-6.0.3-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:f7057c9a337546edc7973c0d3ba84ddcdf0daa14533c2065749c9075001090e6", size = 753613, upload-time = "2025-09-25T21:32:30.178Z" }, + { url = "https://files.pythonhosted.org/packages/3d/e0/7966e1a7bfc0a45bf0a7fb6b98ea03fc9b8d84fa7f2229e9659680b69ee3/pyyaml-6.0.3-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:eda16858a3cab07b80edaf74336ece1f986ba330fdb8ee0d6c0d68fe82bc96be", size = 794115, upload-time = "2025-09-25T21:32:31.353Z" }, + { url = "https://files.pythonhosted.org/packages/de/94/980b50a6531b3019e45ddeada0626d45fa85cbe22300844a7983285bed3b/pyyaml-6.0.3-cp313-cp313-win32.whl", hash = "sha256:d0eae10f8159e8fdad514efdc92d74fd8d682c933a6dd088030f3834bc8e6b26", size = 137427, upload-time = "2025-09-25T21:32:32.58Z" }, + { url = "https://files.pythonhosted.org/packages/97/c9/39d5b874e8b28845e4ec2202b5da735d0199dbe5b8fb85f91398814a9a46/pyyaml-6.0.3-cp313-cp313-win_amd64.whl", hash = "sha256:79005a0d97d5ddabfeeea4cf676af11e647e41d81c9a7722a193022accdb6b7c", size = 154090, upload-time = "2025-09-25T21:32:33.659Z" }, + { url = "https://files.pythonhosted.org/packages/73/e8/2bdf3ca2090f68bb3d75b44da7bbc71843b19c9f2b9cb9b0f4ab7a5a4329/pyyaml-6.0.3-cp313-cp313-win_arm64.whl", hash = "sha256:5498cd1645aa724a7c71c8f378eb29ebe23da2fc0d7a08071d89469bf1d2defb", size = 140246, upload-time = "2025-09-25T21:32:34.663Z" }, + { url = "https://files.pythonhosted.org/packages/9d/8c/f4bd7f6465179953d3ac9bc44ac1a8a3e6122cf8ada906b4f96c60172d43/pyyaml-6.0.3-cp314-cp314-macosx_10_13_x86_64.whl", hash = "sha256:8d1fab6bb153a416f9aeb4b8763bc0f22a5586065f86f7664fc23339fc1c1fac", size = 181814, upload-time = "2025-09-25T21:32:35.712Z" }, + { url = "https://files.pythonhosted.org/packages/bd/9c/4d95bb87eb2063d20db7b60faa3840c1b18025517ae857371c4dd55a6b3a/pyyaml-6.0.3-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:34d5fcd24b8445fadc33f9cf348c1047101756fd760b4dacb5c3e99755703310", size = 173809, upload-time = "2025-09-25T21:32:36.789Z" }, + { url = "https://files.pythonhosted.org/packages/92/b5/47e807c2623074914e29dabd16cbbdd4bf5e9b2db9f8090fa64411fc5382/pyyaml-6.0.3-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:501a031947e3a9025ed4405a168e6ef5ae3126c59f90ce0cd6f2bfc477be31b7", size = 766454, upload-time = "2025-09-25T21:32:37.966Z" }, + { url = "https://files.pythonhosted.org/packages/02/9e/e5e9b168be58564121efb3de6859c452fccde0ab093d8438905899a3a483/pyyaml-6.0.3-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:b3bc83488de33889877a0f2543ade9f70c67d66d9ebb4ac959502e12de895788", size = 836355, upload-time = "2025-09-25T21:32:39.178Z" }, + { url = "https://files.pythonhosted.org/packages/88/f9/16491d7ed2a919954993e48aa941b200f38040928474c9e85ea9e64222c3/pyyaml-6.0.3-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c458b6d084f9b935061bc36216e8a69a7e293a2f1e68bf956dcd9e6cbcd143f5", size = 794175, upload-time = "2025-09-25T21:32:40.865Z" }, + { url = "https://files.pythonhosted.org/packages/dd/3f/5989debef34dc6397317802b527dbbafb2b4760878a53d4166579111411e/pyyaml-6.0.3-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:7c6610def4f163542a622a73fb39f534f8c101d690126992300bf3207eab9764", size = 755228, upload-time = "2025-09-25T21:32:42.084Z" }, + { url = "https://files.pythonhosted.org/packages/d7/ce/af88a49043cd2e265be63d083fc75b27b6ed062f5f9fd6cdc223ad62f03e/pyyaml-6.0.3-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:5190d403f121660ce8d1d2c1bb2ef1bd05b5f68533fc5c2ea899bd15f4399b35", size = 789194, upload-time = "2025-09-25T21:32:43.362Z" }, + { url = "https://files.pythonhosted.org/packages/23/20/bb6982b26a40bb43951265ba29d4c246ef0ff59c9fdcdf0ed04e0687de4d/pyyaml-6.0.3-cp314-cp314-win_amd64.whl", hash = "sha256:4a2e8cebe2ff6ab7d1050ecd59c25d4c8bd7e6f400f5f82b96557ac0abafd0ac", size = 156429, upload-time = "2025-09-25T21:32:57.844Z" }, + { url = "https://files.pythonhosted.org/packages/f4/f4/a4541072bb9422c8a883ab55255f918fa378ecf083f5b85e87fc2b4eda1b/pyyaml-6.0.3-cp314-cp314-win_arm64.whl", hash = "sha256:93dda82c9c22deb0a405ea4dc5f2d0cda384168e466364dec6255b293923b2f3", size = 143912, upload-time = "2025-09-25T21:32:59.247Z" }, + { url = "https://files.pythonhosted.org/packages/7c/f9/07dd09ae774e4616edf6cda684ee78f97777bdd15847253637a6f052a62f/pyyaml-6.0.3-cp314-cp314t-macosx_10_13_x86_64.whl", hash = "sha256:02893d100e99e03eda1c8fd5c441d8c60103fd175728e23e431db1b589cf5ab3", size = 189108, upload-time = "2025-09-25T21:32:44.377Z" }, + { url = "https://files.pythonhosted.org/packages/4e/78/8d08c9fb7ce09ad8c38ad533c1191cf27f7ae1effe5bb9400a46d9437fcf/pyyaml-6.0.3-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:c1ff362665ae507275af2853520967820d9124984e0f7466736aea23d8611fba", size = 183641, upload-time = "2025-09-25T21:32:45.407Z" }, + { url = "https://files.pythonhosted.org/packages/7b/5b/3babb19104a46945cf816d047db2788bcaf8c94527a805610b0289a01c6b/pyyaml-6.0.3-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:6adc77889b628398debc7b65c073bcb99c4a0237b248cacaf3fe8a557563ef6c", size = 831901, upload-time = "2025-09-25T21:32:48.83Z" }, + { url = "https://files.pythonhosted.org/packages/8b/cc/dff0684d8dc44da4d22a13f35f073d558c268780ce3c6ba1b87055bb0b87/pyyaml-6.0.3-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:a80cb027f6b349846a3bf6d73b5e95e782175e52f22108cfa17876aaeff93702", size = 861132, upload-time = "2025-09-25T21:32:50.149Z" }, + { url = "https://files.pythonhosted.org/packages/b1/5e/f77dc6b9036943e285ba76b49e118d9ea929885becb0a29ba8a7c75e29fe/pyyaml-6.0.3-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:00c4bdeba853cc34e7dd471f16b4114f4162dc03e6b7afcc2128711f0eca823c", size = 839261, upload-time = "2025-09-25T21:32:51.808Z" }, + { url = "https://files.pythonhosted.org/packages/ce/88/a9db1376aa2a228197c58b37302f284b5617f56a5d959fd1763fb1675ce6/pyyaml-6.0.3-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:66e1674c3ef6f541c35191caae2d429b967b99e02040f5ba928632d9a7f0f065", size = 805272, upload-time = "2025-09-25T21:32:52.941Z" }, + { url = "https://files.pythonhosted.org/packages/da/92/1446574745d74df0c92e6aa4a7b0b3130706a4142b2d1a5869f2eaa423c6/pyyaml-6.0.3-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:16249ee61e95f858e83976573de0f5b2893b3677ba71c9dd36b9cf8be9ac6d65", size = 829923, upload-time = "2025-09-25T21:32:54.537Z" }, + { url = "https://files.pythonhosted.org/packages/f0/7a/1c7270340330e575b92f397352af856a8c06f230aa3e76f86b39d01b416a/pyyaml-6.0.3-cp314-cp314t-win_amd64.whl", hash = "sha256:4ad1906908f2f5ae4e5a8ddfce73c320c2a1429ec52eafd27138b7f1cbe341c9", size = 174062, upload-time = "2025-09-25T21:32:55.767Z" }, + { url = "https://files.pythonhosted.org/packages/f1/12/de94a39c2ef588c7e6455cfbe7343d3b2dc9d6b6b2f40c4c6565744c873d/pyyaml-6.0.3-cp314-cp314t-win_arm64.whl", hash = "sha256:ebc55a14a21cb14062aa4162f906cd962b28e2e9ea38f9b4391244cd8de4ae0b", size = 149341, upload-time = "2025-09-25T21:32:56.828Z" }, +] + +[[package]] +name = "requests" +version = "2.33.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "certifi" }, + { name = "charset-normalizer" }, + { name = "idna" }, + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/5f/a4/98b9c7c6428a668bf7e42ebb7c79d576a1c3c1e3ae2d47e674b468388871/requests-2.33.1.tar.gz", hash = "sha256:18817f8c57c6263968bc123d237e3b8b08ac046f5456bd1e307ee8f4250d3517", size = 134120, upload-time = "2026-03-30T16:09:15.531Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d7/8e/7540e8a2036f79a125c1d2ebadf69ed7901608859186c856fa0388ef4197/requests-2.33.1-py3-none-any.whl", hash = "sha256:4e6d1ef462f3626a1f0a0a9c42dd93c63bad33f9f1c1937509b8c5c8718ab56a", size = 64947, upload-time = "2026-03-30T16:09:13.83Z" }, +] + +[[package]] +name = "requests-mock" +version = "1.12.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "requests" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/92/32/587625f91f9a0a3d84688bf9cfc4b2480a7e8ec327cefd0ff2ac891fd2cf/requests-mock-1.12.1.tar.gz", hash = "sha256:e9e12e333b525156e82a3c852f22016b9158220d2f47454de9cae8a77d371401", size = 60901, upload-time = "2024-03-29T03:54:29.446Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/97/ec/889fbc557727da0c34a33850950310240f2040f3b1955175fdb2b36a8910/requests_mock-1.12.1-py2.py3-none-any.whl", hash = "sha256:b1e37054004cdd5e56c84454cc7df12b25f90f382159087f4b6915aaeef39563", size = 27695, upload-time = "2024-03-29T03:54:27.64Z" }, +] + +[[package]] +name = "rich" +version = "15.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "markdown-it-py" }, + { name = "pygments" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/c0/8f/0722ca900cc807c13a6a0c696dacf35430f72e0ec571c4275d2371fca3e9/rich-15.0.0.tar.gz", hash = "sha256:edd07a4824c6b40189fb7ac9bc4c52536e9780fbbfbddf6f1e2502c31b068c36", size = 230680, upload-time = "2026-04-12T08:24:00.75Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/82/3b/64d4899d73f91ba49a8c18a8ff3f0ea8f1c1d75481760df8c68ef5235bf5/rich-15.0.0-py3-none-any.whl", hash = "sha256:33bd4ef74232fb73fe9279a257718407f169c09b78a87ad3d296f548e27de0bb", size = 310654, upload-time = "2026-04-12T08:24:02.83Z" }, +] + +[[package]] +name = "ruff" +version = "0.11.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/90/61/fb87430f040e4e577e784e325351186976516faef17d6fcd921fe28edfd7/ruff-0.11.2.tar.gz", hash = "sha256:ec47591497d5a1050175bdf4e1a4e6272cddff7da88a2ad595e1e326041d8d94", size = 3857511, upload-time = "2025-03-21T13:31:17.419Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/62/99/102578506f0f5fa29fd7e0df0a273864f79af044757aef73d1cae0afe6ad/ruff-0.11.2-py3-none-linux_armv6l.whl", hash = "sha256:c69e20ea49e973f3afec2c06376eb56045709f0212615c1adb0eda35e8a4e477", size = 10113146, upload-time = "2025-03-21T13:30:26.68Z" }, + { url = "https://files.pythonhosted.org/packages/74/ad/5cd4ba58ab602a579997a8494b96f10f316e874d7c435bcc1a92e6da1b12/ruff-0.11.2-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:2c5424cc1c4eb1d8ecabe6d4f1b70470b4f24a0c0171356290b1953ad8f0e272", size = 10867092, upload-time = "2025-03-21T13:30:37.949Z" }, + { url = "https://files.pythonhosted.org/packages/fc/3e/d3f13619e1d152c7b600a38c1a035e833e794c6625c9a6cea6f63dbf3af4/ruff-0.11.2-py3-none-macosx_11_0_arm64.whl", hash = "sha256:ecf20854cc73f42171eedb66f006a43d0a21bfb98a2523a809931cda569552d9", size = 10224082, upload-time = "2025-03-21T13:30:39.962Z" }, + { url = "https://files.pythonhosted.org/packages/90/06/f77b3d790d24a93f38e3806216f263974909888fd1e826717c3ec956bbcd/ruff-0.11.2-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0c543bf65d5d27240321604cee0633a70c6c25c9a2f2492efa9f6d4b8e4199bb", size = 10394818, upload-time = "2025-03-21T13:30:42.551Z" }, + { url = "https://files.pythonhosted.org/packages/99/7f/78aa431d3ddebfc2418cd95b786642557ba8b3cb578c075239da9ce97ff9/ruff-0.11.2-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:20967168cc21195db5830b9224be0e964cc9c8ecf3b5a9e3ce19876e8d3a96e3", size = 9952251, upload-time = "2025-03-21T13:30:45.196Z" }, + { url = "https://files.pythonhosted.org/packages/30/3e/f11186d1ddfaca438c3bbff73c6a2fdb5b60e6450cc466129c694b0ab7a2/ruff-0.11.2-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:955a9ce63483999d9f0b8f0b4a3ad669e53484232853054cc8b9d51ab4c5de74", size = 11563566, upload-time = "2025-03-21T13:30:47.516Z" }, + { url = "https://files.pythonhosted.org/packages/22/6c/6ca91befbc0a6539ee133d9a9ce60b1a354db12c3c5d11cfdbf77140f851/ruff-0.11.2-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:86b3a27c38b8fce73bcd262b0de32e9a6801b76d52cdb3ae4c914515f0cef608", size = 12208721, upload-time = "2025-03-21T13:30:49.56Z" }, + { url = "https://files.pythonhosted.org/packages/19/b0/24516a3b850d55b17c03fc399b681c6a549d06ce665915721dc5d6458a5c/ruff-0.11.2-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a3b66a03b248c9fcd9d64d445bafdf1589326bee6fc5c8e92d7562e58883e30f", size = 11662274, upload-time = "2025-03-21T13:30:52.055Z" }, + { url = "https://files.pythonhosted.org/packages/d7/65/76be06d28ecb7c6070280cef2bcb20c98fbf99ff60b1c57d2fb9b8771348/ruff-0.11.2-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0397c2672db015be5aa3d4dac54c69aa012429097ff219392c018e21f5085147", size = 13792284, upload-time = "2025-03-21T13:30:54.24Z" }, + { url = "https://files.pythonhosted.org/packages/ce/d2/4ceed7147e05852876f3b5f3fdc23f878ce2b7e0b90dd6e698bda3d20787/ruff-0.11.2-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:869bcf3f9abf6457fbe39b5a37333aa4eecc52a3b99c98827ccc371a8e5b6f1b", size = 11327861, upload-time = "2025-03-21T13:30:56.757Z" }, + { url = "https://files.pythonhosted.org/packages/c4/78/4935ecba13706fd60ebe0e3dc50371f2bdc3d9bc80e68adc32ff93914534/ruff-0.11.2-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:2a2b50ca35457ba785cd8c93ebbe529467594087b527a08d487cf0ee7b3087e9", size = 10276560, upload-time = "2025-03-21T13:30:58.881Z" }, + { url = "https://files.pythonhosted.org/packages/81/7f/1b2435c3f5245d410bb5dc80f13ec796454c21fbda12b77d7588d5cf4e29/ruff-0.11.2-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:7c69c74bf53ddcfbc22e6eb2f31211df7f65054bfc1f72288fc71e5f82db3eab", size = 9945091, upload-time = "2025-03-21T13:31:01.45Z" }, + { url = "https://files.pythonhosted.org/packages/39/c4/692284c07e6bf2b31d82bb8c32f8840f9d0627d92983edaac991a2b66c0a/ruff-0.11.2-py3-none-musllinux_1_2_i686.whl", hash = "sha256:6e8fb75e14560f7cf53b15bbc55baf5ecbe373dd5f3aab96ff7aa7777edd7630", size = 10977133, upload-time = "2025-03-21T13:31:04.013Z" }, + { url = "https://files.pythonhosted.org/packages/94/cf/8ab81cb7dd7a3b0a3960c2769825038f3adcd75faf46dd6376086df8b128/ruff-0.11.2-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:842a472d7b4d6f5924e9297aa38149e5dcb1e628773b70e6387ae2c97a63c58f", size = 11378514, upload-time = "2025-03-21T13:31:06.166Z" }, + { url = "https://files.pythonhosted.org/packages/d9/3a/a647fa4f316482dacf2fd68e8a386327a33d6eabd8eb2f9a0c3d291ec549/ruff-0.11.2-py3-none-win32.whl", hash = "sha256:aca01ccd0eb5eb7156b324cfaa088586f06a86d9e5314b0eb330cb48415097cc", size = 10319835, upload-time = "2025-03-21T13:31:10.7Z" }, + { url = "https://files.pythonhosted.org/packages/86/54/3c12d3af58012a5e2cd7ebdbe9983f4834af3f8cbea0e8a8c74fa1e23b2b/ruff-0.11.2-py3-none-win_amd64.whl", hash = "sha256:3170150172a8f994136c0c66f494edf199a0bbea7a409f649e4bc8f4d7084080", size = 11373713, upload-time = "2025-03-21T13:31:13.148Z" }, + { url = "https://files.pythonhosted.org/packages/d6/d4/dd813703af8a1e2ac33bf3feb27e8a5ad514c9f219df80c64d69807e7f71/ruff-0.11.2-py3-none-win_arm64.whl", hash = "sha256:52933095158ff328f4c77af3d74f0379e34fd52f175144cefc1b192e7ccd32b4", size = 10441990, upload-time = "2025-03-21T13:31:15.206Z" }, +] + +[[package]] +name = "semver" +version = "3.0.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/d1/d3159231aec234a59dd7d601e9dd9fe96f3afff15efd33c1070019b26132/semver-3.0.4.tar.gz", hash = "sha256:afc7d8c584a5ed0a11033af086e8af226a9c0b206f313e0301f8dd7b6b589602", size = 269730, upload-time = "2025-01-24T13:19:27.617Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a6/24/4d91e05817e92e3a61c8a21e08fd0f390f5301f1c448b137c57c4bc6e543/semver-3.0.4-py3-none-any.whl", hash = "sha256:9c824d87ba7f7ab4a1890799cec8596f15c1241cb473404ea1cb0c55e4b04746", size = 17912, upload-time = "2025-01-24T13:19:24.949Z" }, +] + +[[package]] +name = "stack-data" +version = "0.6.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "asttokens" }, + { name = "executing" }, + { name = "pure-eval" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/28/e3/55dcc2cfbc3ca9c29519eb6884dd1415ecb53b0e934862d3559ddcb7e20b/stack_data-0.6.3.tar.gz", hash = "sha256:836a778de4fec4dcd1dcd89ed8abff8a221f58308462e1c4aa2a3cf30148f0b9", size = 44707, upload-time = "2023-09-30T13:58:05.479Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f1/7b/ce1eafaf1a76852e2ec9b22edecf1daa58175c090266e9f6c64afcd81d91/stack_data-0.6.3-py3-none-any.whl", hash = "sha256:d5558e0c25a4cb0853cddad3d77da9891a08cb85dd9f9f91b9f8cd66e511e695", size = 24521, upload-time = "2023-09-30T13:58:03.53Z" }, +] + +[[package]] +name = "time-machine" +version = "3.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/02/fc/37b02f6094dbb1f851145330460532176ed2f1dc70511a35828166c41e52/time_machine-3.2.0.tar.gz", hash = "sha256:a4ddd1cea17b8950e462d1805a42b20c81eb9aafc8f66b392dd5ce997e037d79", size = 14804, upload-time = "2025-12-17T23:33:02.599Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/71/8b/080c8eedcd67921a52ba5bd0e075362062509ab63c86fc1a0442fad241a6/time_machine-3.2.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:cc4bee5b0214d7dc4ebc91f4a4c600f1a598e9b5606ac751f42cb6f6740b1dbb", size = 19255, upload-time = "2025-12-17T23:31:58.057Z" }, + { url = "https://files.pythonhosted.org/packages/66/17/0e5291e9eb705bf8a5a1305f826e979af307bbeb79def4ddbf4b3f9a81e0/time_machine-3.2.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:3ca036304b4460ae2fdc1b52dd8b1fa7cf1464daa427fc49567413c09aa839c1", size = 15360, upload-time = "2025-12-17T23:31:59.048Z" }, + { url = "https://files.pythonhosted.org/packages/8b/e8/9ab87b71d2e2b62463b9b058b7ae7ac09fb57f8fcd88729dec169d304340/time_machine-3.2.0-cp312-cp312-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:5442735b41d7a2abc2f04579b4ca6047ed4698a8338a4fec92c7c9423e7938cb", size = 33029, upload-time = "2025-12-17T23:32:00.413Z" }, + { url = "https://files.pythonhosted.org/packages/4b/26/b5ca19da6f25ea905b3e10a0ea95d697c1aeba0404803a43c68f1af253e6/time_machine-3.2.0-cp312-cp312-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:97da3e971e505cb637079fb07ab0bcd36e33279f8ecac888ff131f45ef1e4d8d", size = 34579, upload-time = "2025-12-17T23:32:01.431Z" }, + { url = "https://files.pythonhosted.org/packages/79/ca/6ac7ad5f10ea18cc1d9de49716ba38c32132c7b64532430d92ef240c116b/time_machine-3.2.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3cdda6dee4966e38aeb487309bb414c6cb23a81fc500291c77a8fcd3098832e7", size = 35961, upload-time = "2025-12-17T23:32:02.521Z" }, + { url = "https://files.pythonhosted.org/packages/33/67/390dd958bed395ab32d79a9fe61fe111825c0dd4ded54dbba7e867f171e6/time_machine-3.2.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:33d9efd302a6998bcc8baa4d84f259f8a4081105bd3d7f7af7f1d0abd3b1c8aa", size = 34668, upload-time = "2025-12-17T23:32:03.585Z" }, + { url = "https://files.pythonhosted.org/packages/da/57/c88fff034a4e9538b3ae7c68c9cfb283670b14d17522c5a8bc17d29f9a4b/time_machine-3.2.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:3a0b0a33971f14145853c9bd95a6ab0353cf7e0019fa2a7aa1ae9fddfe8eab50", size = 32891, upload-time = "2025-12-17T23:32:04.656Z" }, + { url = "https://files.pythonhosted.org/packages/2d/70/ebbb76022dba0fec8f9156540fc647e4beae1680c787c01b1b6200e56d70/time_machine-3.2.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2d0be9e5f22c38082d247a2cdcd8a936504e9db60b7b3606855fb39f299e9548", size = 34080, upload-time = "2025-12-17T23:32:06.146Z" }, + { url = "https://files.pythonhosted.org/packages/db/9a/2ca9e7af3df540dc1c79e3de588adeddb7dcc2107829248e6969c4f14167/time_machine-3.2.0-cp312-cp312-win32.whl", hash = "sha256:3f74623648b936fdce5f911caf386c0a0b579456410975de8c0dfeaaffece1d8", size = 17371, upload-time = "2025-12-17T23:32:07.164Z" }, + { url = "https://files.pythonhosted.org/packages/d8/ce/21d23efc9c2151939af1b7ee4e60d86d661b74ef32b8eaa148f6fe8c899c/time_machine-3.2.0-cp312-cp312-win_amd64.whl", hash = "sha256:34e26a41d994b5e4b205136a90e9578470386749cc9a2ecf51ca18f83ce25e23", size = 18132, upload-time = "2025-12-17T23:32:08.447Z" }, + { url = "https://files.pythonhosted.org/packages/2f/34/c2b70be483accf6db9e5d6c3139bce3c38fe51f898ccf64e8d3fe14fbf4d/time_machine-3.2.0-cp312-cp312-win_arm64.whl", hash = "sha256:0615d3d82c418d6293f271c348945c5091a71f37e37173653d5c26d0e74b13a8", size = 16930, upload-time = "2025-12-17T23:32:09.477Z" }, + { url = "https://files.pythonhosted.org/packages/ee/cd/43ad5efc88298af3c59b66769cea7f055567a85071579ed40536188530c1/time_machine-3.2.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:c421a8eb85a4418a7675a41bf8660224318c46cc62e4751c8f1ceca752059090", size = 19318, upload-time = "2025-12-17T23:32:10.518Z" }, + { url = "https://files.pythonhosted.org/packages/b0/f6/084010ef7f4a3f38b5a4900923d7c85b29e797655c4f6ee4ce54d903cca8/time_machine-3.2.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:8f4e758f7727d0058c4950c66b58200c187072122d6f7a98b610530a4233ea7b", size = 15390, upload-time = "2025-12-17T23:32:11.625Z" }, + { url = "https://files.pythonhosted.org/packages/25/aa/1cabb74134f492270dc6860cb7865859bf40ecf828be65972827646e91ad/time_machine-3.2.0-cp313-cp313-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:154bd3f75c81f70218b2585cc12b60762fb2665c507eec5ec5037d8756d9b4e0", size = 33115, upload-time = "2025-12-17T23:32:13.219Z" }, + { url = "https://files.pythonhosted.org/packages/5e/03/78c5d7dfa366924eb4dbfcc3fc917c39a4280ca234b12819cc1f16c03d88/time_machine-3.2.0-cp313-cp313-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:d50cfe5ebea422c896ad8d278af9648412b7533b8ea6adeeee698a3fd9b1d3b7", size = 34705, upload-time = "2025-12-17T23:32:14.29Z" }, + { url = "https://files.pythonhosted.org/packages/86/93/d5e877c24541f674c6869ff6e9c56833369796010190252e92c9d7ae5f0f/time_machine-3.2.0-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:636576501724bd6a9124e69d86e5aef263479e89ef739c5db361469f0463a0a1", size = 36104, upload-time = "2025-12-17T23:32:15.354Z" }, + { url = "https://files.pythonhosted.org/packages/22/1c/d4bae72f388f67efc9609f89b012e434bb19d9549c7a7b47d6c7d9e5c55d/time_machine-3.2.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:40e6f40c57197fcf7ec32d2c563f4df0a82c42cdcc3cab27f688e98f6060df10", size = 34765, upload-time = "2025-12-17T23:32:16.434Z" }, + { url = "https://files.pythonhosted.org/packages/1d/c3/ac378cf301d527d8dfad2f0db6bad0dfb1ab73212eaa56d6b96ee5d9d20b/time_machine-3.2.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:a1bcf0b846bbfc19a79bc19e3fa04d8c7b1e8101c1b70340ffdb689cd801ea53", size = 33010, upload-time = "2025-12-17T23:32:17.532Z" }, + { url = "https://files.pythonhosted.org/packages/06/35/7ce897319accda7a6970b288a9a8c52d25227342a7508505a2b3d235b649/time_machine-3.2.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:ae55a56c179f4fe7a62575ad5148b6ed82f6c7e5cf2f9a9ec65f2f5b067db5f5", size = 34185, upload-time = "2025-12-17T23:32:18.566Z" }, + { url = "https://files.pythonhosted.org/packages/bf/28/f922022269749cb02eee2b62919671153c4088994fa955a6b0e50327ff81/time_machine-3.2.0-cp313-cp313-win32.whl", hash = "sha256:a66fe55a107e46916007a391d4030479df8864ec6ad6f6a6528221befc5c886e", size = 17397, upload-time = "2025-12-17T23:32:19.605Z" }, + { url = "https://files.pythonhosted.org/packages/ee/dc/fd87cde397f4a7bea493152f0aca8fd569ec709cad9e0f2ca7011eb8c7f7/time_machine-3.2.0-cp313-cp313-win_amd64.whl", hash = "sha256:30c9ce57165df913e4f74e285a8ab829ff9b7aa3e5ec0973f88f642b9a7b3d15", size = 18139, upload-time = "2025-12-17T23:32:20.991Z" }, + { url = "https://files.pythonhosted.org/packages/75/81/b8ce58233addc5d7d54d2fabc49dcbc02d79e3f079d150aa1bec3d5275ef/time_machine-3.2.0-cp313-cp313-win_arm64.whl", hash = "sha256:89cad7e179e9bdcc84dcf09efe52af232c4cc7a01b3de868356bbd59d95bd9b8", size = 16964, upload-time = "2025-12-17T23:32:22.075Z" }, + { url = "https://files.pythonhosted.org/packages/67/e7/487f0ba5fe6c58186a5e1af2a118dfa2c160fedb37ef53a7e972d410408e/time_machine-3.2.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:59d71545e62525a4b85b6de9ab5c02ee3c61110fd7f636139914a2335dcbfc9c", size = 20000, upload-time = "2025-12-17T23:32:23.058Z" }, + { url = "https://files.pythonhosted.org/packages/e1/17/eb2c0054c8d44dd42df84ccd434539249a9c7d0b8eb53f799be2102500ab/time_machine-3.2.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:999672c621c35362bc28e03ca0c7df21500195540773c25993421fd8d6cc5003", size = 15657, upload-time = "2025-12-17T23:32:24.125Z" }, + { url = "https://files.pythonhosted.org/packages/43/21/93443b5d1dd850f8bb9442e90d817a9033dcce6bfbdd3aabbb9786251c80/time_machine-3.2.0-cp313-cp313t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:5faf7397f0580c7b9d67288522c8d7863e85f0cffadc0f1fccdb2c3dfce5783e", size = 39216, upload-time = "2025-12-17T23:32:25.542Z" }, + { url = "https://files.pythonhosted.org/packages/9f/9e/18544cf8acc72bb1dc03762231c82ecc259733f4bb6770a7bbe5cd138603/time_machine-3.2.0-cp313-cp313t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:d3dd886ec49f1fa5a00e844f5947e5c0f98ce574750c24b7424c6f77fc1c3e87", size = 40764, upload-time = "2025-12-17T23:32:26.643Z" }, + { url = "https://files.pythonhosted.org/packages/27/f7/9fe9ce2795636a3a7467307af6bdf38bb613ddb701a8a5cd50ec713beb5e/time_machine-3.2.0-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:da0ecd96bc7bbe450acaaabe569d84e81688f1be8ad58d1470e42371d145fb53", size = 43526, upload-time = "2025-12-17T23:32:27.693Z" }, + { url = "https://files.pythonhosted.org/packages/03/c1/a93e975ba9dec22e87ec92d18c28e67d36bd536f9119ffa439b2892b0c9c/time_machine-3.2.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:158220e946c1c4fb8265773a0282c88c35a7e3bb5d78e3561214e3b3231166f3", size = 41727, upload-time = "2025-12-17T23:32:28.985Z" }, + { url = "https://files.pythonhosted.org/packages/5f/fb/e3633e5a6bbed1c76bb2e9810dabc2f8467532ffcd29b9aed404b473061a/time_machine-3.2.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:8c1aee29bc54356f248d5d7dfdd131e12ca825e850a08c0ebdb022266d073013", size = 38952, upload-time = "2025-12-17T23:32:30.031Z" }, + { url = "https://files.pythonhosted.org/packages/82/3d/02e9fb2526b3d6b1b45bc8e4d912d95d1cd699d1a3f6df985817d37a0600/time_machine-3.2.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:c8ed2224f09d25b1c2fc98683613aca12f90f682a427eabb68fc824d27014e4a", size = 39829, upload-time = "2025-12-17T23:32:31.075Z" }, + { url = "https://files.pythonhosted.org/packages/85/c8/c14265212436da8e0814c45463987b3f57de3eca4de023cc2eabb0c62ef3/time_machine-3.2.0-cp313-cp313t-win32.whl", hash = "sha256:3498719f8dab51da76d29a20c1b5e52ee7db083dddf3056af7fa69c1b94e1fe6", size = 17852, upload-time = "2025-12-17T23:32:32.079Z" }, + { url = "https://files.pythonhosted.org/packages/1d/bc/8acb13cf6149f47508097b158a9a8bec9ec4530a70cb406124e8023581f5/time_machine-3.2.0-cp313-cp313t-win_amd64.whl", hash = "sha256:e0d90bee170b219e1d15e6a58164aa808f5170090e4f090bd0670303e34181b1", size = 18918, upload-time = "2025-12-17T23:32:33.106Z" }, + { url = "https://files.pythonhosted.org/packages/24/87/c443ee508c2708fd2514ccce9052f5e48888783ce690506919629ebc8eb0/time_machine-3.2.0-cp313-cp313t-win_arm64.whl", hash = "sha256:051de220fdb6e20d648111bbad423d9506fdbb2e44d4429cef3dc0382abf1fc2", size = 17261, upload-time = "2025-12-17T23:32:34.446Z" }, + { url = "https://files.pythonhosted.org/packages/61/70/b4b980d126ed155c78d1879c50d60c8dcbd47bd11cb14ee7be50e0dfc07f/time_machine-3.2.0-cp314-cp314-macosx_10_15_universal2.whl", hash = "sha256:1398980c017fe5744d66f419e0115ee48a53b00b146d738e1416c225eb610b82", size = 19303, upload-time = "2025-12-17T23:32:35.796Z" }, + { url = "https://files.pythonhosted.org/packages/73/73/eaa33603c69a68fe2b6f54f9dd75481693d62f1d29676531002be06e2d1c/time_machine-3.2.0-cp314-cp314-macosx_10_15_x86_64.whl", hash = "sha256:4f8f4e35f4191ef70c2ab8ff490761ee9051b891afce2bf86dde3918eb7b537b", size = 15431, upload-time = "2025-12-17T23:32:37.244Z" }, + { url = "https://files.pythonhosted.org/packages/76/10/b81e138e86cc7bab40cdb59d294b341e172201f4a6c84bb0ec080407977a/time_machine-3.2.0-cp314-cp314-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:6db498686ecf6163c5aa8cf0bcd57bbe0f4081184f247edf3ee49a2612b584f9", size = 33206, upload-time = "2025-12-17T23:32:38.713Z" }, + { url = "https://files.pythonhosted.org/packages/d3/72/4deab446b579e8bd5dca91de98595c5d6bd6a17ce162abf5c5f2ce40d3d8/time_machine-3.2.0-cp314-cp314-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:027c1807efb74d0cd58ad16524dec94212fbe900115d70b0123399883657ac0f", size = 34792, upload-time = "2025-12-17T23:32:40.223Z" }, + { url = "https://files.pythonhosted.org/packages/2c/39/439c6b587ddee76d533fe972289d0646e0a5520e14dc83d0a30aeb5565f7/time_machine-3.2.0-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:92432610c05676edd5e6946a073c6f0c926923123ce7caee1018dc10782c713d", size = 36187, upload-time = "2025-12-17T23:32:41.705Z" }, + { url = "https://files.pythonhosted.org/packages/4b/db/2da4368db15180989bab83746a857bde05ad16e78f326801c142bb747a06/time_machine-3.2.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:c25586b62480eb77ef3d953fba273209478e1ef49654592cd6a52a68dfe56a67", size = 34855, upload-time = "2025-12-17T23:32:42.817Z" }, + { url = "https://files.pythonhosted.org/packages/88/84/120a431fee50bc4c241425bee4d3a4910df4923b7ab5f7dff1bf0c772f08/time_machine-3.2.0-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:6bf3a2fa738d15e0b95d14469a0b8ea42635467408d8b490e263d5d45c9a177f", size = 33222, upload-time = "2025-12-17T23:32:43.94Z" }, + { url = "https://files.pythonhosted.org/packages/f9/ea/89cfda82bb8c57ff91bb9a26751aa234d6d90e9b4d5ab0ad9dce0f9f0329/time_machine-3.2.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:ce76b82276d7ad2a66cdc85dad4df19d1422b69183170a34e8fbc4c3f35502f7", size = 34270, upload-time = "2025-12-17T23:32:45.037Z" }, + { url = "https://files.pythonhosted.org/packages/8a/aa/235357da4f69a51a8d35fcbfcfa77cdc7dc24f62ae54025006570bda7e2d/time_machine-3.2.0-cp314-cp314-win32.whl", hash = "sha256:14d6778273c543441863dff712cd1d7803dee946b18de35921eb8df10714539d", size = 17544, upload-time = "2025-12-17T23:32:46.099Z" }, + { url = "https://files.pythonhosted.org/packages/7b/51/6c8405a7276be79693b792cff22ce41067ec05db26a7d02f2d5b06324434/time_machine-3.2.0-cp314-cp314-win_amd64.whl", hash = "sha256:cbf821da96dbc80d349fa9e7c36e670b41d68a878d28c8850057992fed430eef", size = 18423, upload-time = "2025-12-17T23:32:47.468Z" }, + { url = "https://files.pythonhosted.org/packages/d9/03/a3cf419e20c35fc203c6e4fed48b5b667c1a2b4da456d9971e605f73ecef/time_machine-3.2.0-cp314-cp314-win_arm64.whl", hash = "sha256:71c75d71f8e68abc8b669bca26ed2ddd558430a6c171e32b8620288565f18c0e", size = 17050, upload-time = "2025-12-17T23:32:48.91Z" }, + { url = "https://files.pythonhosted.org/packages/86/a1/142de946dc4393f910bf4564b5c3ba819906e1f49b06c9cb557519c849e4/time_machine-3.2.0-cp314-cp314t-macosx_10_15_universal2.whl", hash = "sha256:4e374779021446fc2b5c29d80457ec9a3b1a5df043dc2aae07d7c1415d52323c", size = 19991, upload-time = "2025-12-17T23:32:49.933Z" }, + { url = "https://files.pythonhosted.org/packages/ee/62/7f17def6289901f94726921811a16b9adce46e666362c75d45730c60274f/time_machine-3.2.0-cp314-cp314t-macosx_10_15_x86_64.whl", hash = "sha256:122310a6af9c36e9a636da32830e591e7923e8a07bdd0a43276c3a36c6821c90", size = 15707, upload-time = "2025-12-17T23:32:50.969Z" }, + { url = "https://files.pythonhosted.org/packages/5d/d3/3502fb9bd3acb159c18844b26c43220201a0d4a622c0c853785d07699a92/time_machine-3.2.0-cp314-cp314t-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:ba3eeb0f018cc362dd8128befa3426696a2e16dd223c3fb695fde184892d4d8c", size = 39207, upload-time = "2025-12-17T23:32:52.033Z" }, + { url = "https://files.pythonhosted.org/packages/5a/be/8b27f4aa296fda14a5a2ad7f588ddd450603c33415ab3f8e85b2f1a44678/time_machine-3.2.0-cp314-cp314t-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:77d38ba664b381a7793f8786efc13b5004f0d5f672dae814430445b8202a67a6", size = 40764, upload-time = "2025-12-17T23:32:53.167Z" }, + { url = "https://files.pythonhosted.org/packages/42/cd/fe4c4e5c8ab6d48fab3624c32be9116fb120173a35fe67e482e5cf68b3d2/time_machine-3.2.0-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:f09abeb8f03f044d72712207e0489a62098ad3ad16dac38927fcf80baca4d6a7", size = 43508, upload-time = "2025-12-17T23:32:54.597Z" }, + { url = "https://files.pythonhosted.org/packages/b4/28/5a3ba2fce85b97655a425d6bb20a441550acd2b304c96b2c19d3839f721a/time_machine-3.2.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:6b28367ce4f73987a55e230e1d30a57a3af85da8eb1a140074eb6e8c7e6ef19f", size = 41712, upload-time = "2025-12-17T23:32:55.781Z" }, + { url = "https://files.pythonhosted.org/packages/81/58/e38084be7fdabb4835db68a3a47e58c34182d79fc35df1ecbe0db2c5359f/time_machine-3.2.0-cp314-cp314t-musllinux_1_2_i686.whl", hash = "sha256:903c7751c904581da9f7861c3015bed7cdc40047321291d3694a3cdc783bbca3", size = 38939, upload-time = "2025-12-17T23:32:56.867Z" }, + { url = "https://files.pythonhosted.org/packages/40/d0/ad3feb0a392ef4e0c08bc32024950373ddc0669002cbdcbb9f3bf0c2d114/time_machine-3.2.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:528217cad85ede5f85c8bc78b0341868d3c3cfefc6ecb5b622e1cacb6c73247b", size = 39837, upload-time = "2025-12-17T23:32:58.283Z" }, + { url = "https://files.pythonhosted.org/packages/5b/9e/5f4b2ea63b267bd78f3245e76f5528836611b5f2d30b5e7300a722fe4428/time_machine-3.2.0-cp314-cp314t-win32.whl", hash = "sha256:75724762ffd517e7e80aaec1fad1ff5a7414bd84e2b3ee7a0bacfeb67c14926e", size = 18091, upload-time = "2025-12-17T23:32:59.403Z" }, + { url = "https://files.pythonhosted.org/packages/39/6f/456b1f4d2700ae02b19eba830f870596a4b89b74bac3b6c80666f1b108c5/time_machine-3.2.0-cp314-cp314t-win_amd64.whl", hash = "sha256:2526abbd053c5bca898d1b3e7898eec34626b12206718d8c7ce88fd12c1c9c5c", size = 19208, upload-time = "2025-12-17T23:33:00.488Z" }, + { url = "https://files.pythonhosted.org/packages/2f/22/8063101427ecd3d2652aada4d21d0876b07a3dc789125bca2ee858fec3ed/time_machine-3.2.0-cp314-cp314t-win_arm64.whl", hash = "sha256:7f2fb6784b414edbe2c0b558bfaab0c251955ba27edd62946cce4a01675a992c", size = 17359, upload-time = "2025-12-17T23:33:01.54Z" }, +] + +[[package]] +name = "traitlets" +version = "5.14.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/eb/79/72064e6a701c2183016abbbfedaba506d81e30e232a68c9f0d6f6fcd1574/traitlets-5.14.3.tar.gz", hash = "sha256:9ed0579d3502c94b4b3732ac120375cda96f923114522847de4b3bb98b96b6b7", size = 161621, upload-time = "2024-04-19T11:11:49.746Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/00/c0/8f5d070730d7836adc9c9b6408dec68c6ced86b304a9b26a14df072a6e8c/traitlets-5.14.3-py3-none-any.whl", hash = "sha256:b74e89e397b1ed28cc831db7aea759ba6640cb3de13090ca145426688ff1ac4f", size = 85359, upload-time = "2024-04-19T11:11:46.763Z" }, +] + +[[package]] +name = "types-aiofiles" +version = "25.1.0.20260409" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6c/66/9e62a2692792bc96c0f423f478149f4a7b84720704c546c8960b0a047c89/types_aiofiles-25.1.0.20260409.tar.gz", hash = "sha256:49e67d72bdcf9fe406f5815758a78dc34a1249bb5aa2adba78a80aec0a775435", size = 14812, upload-time = "2026-04-09T04:22:35.308Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/27/d0/28236f869ba4dfb223ecdbc267eb2bdb634b81a561dd992230a4f9ec48fa/types_aiofiles-25.1.0.20260409-py3-none-any.whl", hash = "sha256:923fedb532c772cc0f62e0ce4282725afa82ca5b41cabd9857f06b55e5eee8de", size = 14372, upload-time = "2026-04-09T04:22:34.328Z" }, +] + +[[package]] +name = "types-certifi" +version = "2021.10.8.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/52/68/943c3aeaf14624712a0357c4a67814dba5cea36d194f5c764dad7959a00c/types-certifi-2021.10.8.3.tar.gz", hash = "sha256:72cf7798d165bc0b76e1c10dd1ea3097c7063c42c21d664523b928e88b554a4f", size = 2095, upload-time = "2022-06-09T15:19:05.244Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b5/63/2463d89481e811f007b0e1cd0a91e52e141b47f9de724d20db7b861dcfec/types_certifi-2021.10.8.3-py3-none-any.whl", hash = "sha256:b2d1e325e69f71f7c78e5943d410e650b4707bb0ef32e4ddf3da37f54176e88a", size = 2136, upload-time = "2022-06-09T15:19:03.127Z" }, +] + +[[package]] +name = "types-cffi" +version = "2.0.0.20260408" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "types-setuptools" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/64/67/eb4ef3408fdc0b4e5af38b30c0e6ad4663b41bdae9fb85a9f09a8db61a99/types_cffi-2.0.0.20260408.tar.gz", hash = "sha256:aa8b9c456ab715c079fc655929811f21f331bfb940f4a821987c581bf4e36230", size = 17541, upload-time = "2026-04-08T04:36:03.918Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c3/a3/7fbd93ededcc7c77e9e5948b9794161733ebdbf618a27965b1bea0e728a4/types_cffi-2.0.0.20260408-py3-none-any.whl", hash = "sha256:68bd296742b4ff7c0afe3547f50bd0acc55416ecf322ffefd2b7344ef6388a42", size = 20101, upload-time = "2026-04-08T04:36:02.995Z" }, +] + +[[package]] +name = "types-croniter" +version = "6.2.2.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c5/e4/89a0101471d6fe4e912dad24c54ae7afd90a9eaa5c74adef2c81f383f8da/types_croniter-6.2.2.20260408.tar.gz", hash = "sha256:a28a18908db371654990d30a3fd99856adc5137e475a23dbda4b10dce85525da", size = 12040, upload-time = "2026-04-08T04:27:20.068Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c3/05/b32e67944ff33e83c181cadf5835858d63f4292a2f2ff5bf6a1edb7f6fed/types_croniter-6.2.2.20260408-py3-none-any.whl", hash = "sha256:242087a5b6e201b7004e55f71ed34f466951b74551c64ef1c6a8a08c47d3cc0d", size = 9732, upload-time = "2026-04-08T04:27:19.229Z" }, +] + +[[package]] +name = "types-deprecated" +version = "1.3.1.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/1a/db/076de3e81b106d3cec17aec9640ab1b2d02f29bad441de280459c161ce65/types_deprecated-1.3.1.20260408.tar.gz", hash = "sha256:62d6a86d0cc754c14bb2de31162d069b1c6a07ce11ee65e5258f8f75308eb3a3", size = 8524, upload-time = "2026-04-08T04:26:39.894Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/53/d0/d3258379deb749d949c3c72313981c9d2cceec518b87dcf506f022f5d49f/types_deprecated-1.3.1.20260408-py3-none-any.whl", hash = "sha256:b64e1eab560d4fa9394a27a3099211344b0e0f2f3ac8026d825c86e70d65cdd5", size = 9079, upload-time = "2026-04-08T04:26:38.752Z" }, +] + +[[package]] +name = "types-docutils" +version = "0.22.3.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/3c/49/48a386fe15539556de085b87a69568b028cca2fa4b92596a3d4f79ac6784/types_docutils-0.22.3.20260408.tar.gz", hash = "sha256:22d5d45e4e0d65a1bc8280987a73e28669bb1cc9d16b18d0afc91713d1be26da", size = 57383, upload-time = "2026-04-08T04:27:26.924Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/08/47/1667fda6e9fcb044f8fb797f6dc4367b88dc2ab40f1a035e387f5405e870/types_docutils-0.22.3.20260408-py3-none-any.whl", hash = "sha256:2545a86966022cdf1468d430b0007eba0837be77974a7f3fafa1b04a6815d531", size = 91981, upload-time = "2026-04-08T04:27:25.934Z" }, +] + +[[package]] +name = "types-markdown" +version = "3.10.2.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/dd/0e/a690840934c459aa50e0470e7550d7f151632eafa4a8e3c21d18009ad15c/types_markdown-3.10.2.20260408.tar.gz", hash = "sha256:d5cba15ed65a1420e80e31c17e3d4a2ad7208a3f3a4da97fd2c5f093caf523cd", size = 19784, upload-time = "2026-04-08T04:33:07.644Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/75/7e/265a8df257c8dced6ea89295f793a19f0a49ccbfeae1ed562368b2caf7a3/types_markdown-3.10.2.20260408-py3-none-any.whl", hash = "sha256:b0bbe8b7a8174db732067b86e391262898f5f536589ea81efec6d35ceb829331", size = 25857, upload-time = "2026-04-08T04:33:06.769Z" }, +] + +[[package]] +name = "types-paramiko" +version = "4.0.0.20260408" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cryptography" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/aa/f5/2a556b03ba264508b6bc6a65131500265f210ff3ebf5d76dbe51b53c3979/types_paramiko-4.0.0.20260408.tar.gz", hash = "sha256:978191a2e11064fa4c7f9ada0fccf49159a17beb98b780310dd2c2d2b4106063", size = 29116, upload-time = "2026-04-08T04:35:04.631Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d9/e2/cf451598a6a8820139d021b2be08a836b9b905d744bcc73b72172e7e10b3/types_paramiko-4.0.0.20260408-py3-none-any.whl", hash = "sha256:350bf53edb4eb88181be68854d598e1cc3a8764fe905d49913025b86e831adbc", size = 38816, upload-time = "2026-04-08T04:35:03.503Z" }, +] + +[[package]] +name = "types-protobuf" +version = "7.34.1.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/5b/b1/4521e68c2cc17703d80eb42796751345376dd4c706f84007ef5e7c707774/types_protobuf-7.34.1.20260408.tar.gz", hash = "sha256:e2c0a0430e08c75b52671a6f0035abfdcc791aad12af16274282de1b721758ab", size = 68835, upload-time = "2026-04-08T04:26:43.613Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ef/b5/0bc9874d89c58fb0ce851e150055ce732d254dbb10b06becbc7635d0d635/types_protobuf-7.34.1.20260408-py3-none-any.whl", hash = "sha256:ebbcd4e27b145aef6a59bc0cb6c013b3528151c1ba5e7f7337aeee355d276a5e", size = 86012, upload-time = "2026-04-08T04:26:42.566Z" }, +] + +[[package]] +name = "types-pymysql" +version = "1.1.0.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b3/04/c3570f05ebab083f28698c829dddf754ffefc30aae4e29915610848e44db/types_pymysql-1.1.0.20260408.tar.gz", hash = "sha256:b784dc37908479e3767e2d794ab507b3674adb1c686ca3d13fc9e2960dbcb9ec", size = 22344, upload-time = "2026-04-08T04:27:47.651Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/70/b3/15dee33878709705a4cc83bcc1bb30e00e95bbe038b472cb1207a15b50a1/types_pymysql-1.1.0.20260408-py3-none-any.whl", hash = "sha256:da630647eaaa7a926a3907794f4067f269cd245b2c202c74aa3c6a3bd660a9db", size = 23071, upload-time = "2026-04-08T04:27:46.735Z" }, +] + +[[package]] +name = "types-pyopenssl" +version = "24.1.0.20240722" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cryptography" }, + { name = "types-cffi" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/93/29/47a346550fd2020dac9a7a6d033ea03fccb92fa47c726056618cc889745e/types-pyOpenSSL-24.1.0.20240722.tar.gz", hash = "sha256:47913b4678a01d879f503a12044468221ed8576263c1540dcb0484ca21b08c39", size = 8458, upload-time = "2024-07-22T02:32:22.558Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/98/05/c868a850b6fbb79c26f5f299b768ee0adc1f9816d3461dcf4287916f655b/types_pyOpenSSL-24.1.0.20240722-py3-none-any.whl", hash = "sha256:6a7a5d2ec042537934cfb4c9d4deb0e16c4c6250b09358df1f083682fe6fda54", size = 7499, upload-time = "2024-07-22T02:32:21.232Z" }, +] + +[[package]] +name = "types-python-dateutil" +version = "2.9.0.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/88/f3/2427775f80cd5e19a0a71ba8e5ab7645a01a852f43a5fd0ffc24f66338e0/types_python_dateutil-2.9.0.20260408.tar.gz", hash = "sha256:8b056ec01568674235f64ecbcef928972a5fac412f5aab09c516dfa2acfbb582", size = 16981, upload-time = "2026-04-08T04:28:10.995Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/fd/c6/eeba37bfee282a6a97f889faef9352d6172c6a5088eb9a4daf570d9d748d/types_python_dateutil-2.9.0.20260408-py3-none-any.whl", hash = "sha256:473139d514a71c9d1fbd8bb328974bedcb1cc3dba57aad04ffa4157f483c216f", size = 18437, upload-time = "2026-04-08T04:28:10.095Z" }, +] + +[[package]] +name = "types-python-slugify" +version = "8.0.2.20240310" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d2/cb/7fdc1982b117d216a9ebbe4ecc6619690991c48f994d93dee7888f459976/types-python-slugify-8.0.2.20240310.tar.gz", hash = "sha256:5157b508c7fed587520c70d77f62aea0fafdc6620893c2ec8972f13a1faf5560", size = 3661, upload-time = "2024-03-10T02:19:03.582Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ce/6d/873300a60133e51b284a5287a618d3d948a65160e3954b90b3dc5e562667/types_python_slugify-8.0.2.20240310-py3-none-any.whl", hash = "sha256:0efec18b802c69ebd22dcee55c91afaeaa80e1e40ddd66ccabf69fd42ce87b74", size = 3566, upload-time = "2024-03-10T02:19:02.575Z" }, +] + +[[package]] +name = "types-pytz" +version = "2026.1.1.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f7/b7/33f5a4f29b1f285b99ff79a607751a7996194cbb98705e331dab7a2daa28/types_pytz-2026.1.1.20260408.tar.gz", hash = "sha256:89b6a34b9198ea2a4b98a9d15cbca987053f52a105fd44f7ce3789cae4349408", size = 10788, upload-time = "2026-04-08T04:28:14.54Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ae/90/12c059e6bb330a22d9cc97daf027ac7fb7f50fbf518e4d88185b4d39120e/types_pytz-2026.1.1.20260408-py3-none-any.whl", hash = "sha256:c7e4dec76221fb7d0c97b91ad8561d689bebe39b6bcb7b728387e7ffd8cde788", size = 10124, upload-time = "2026-04-08T04:28:13.353Z" }, +] + +[[package]] +name = "types-pyyaml" +version = "6.0.12.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/74/73/b759b1e413c31034cc01ecdfb96b38115d0ab4db55a752a3929f0cd449fd/types_pyyaml-6.0.12.20260408.tar.gz", hash = "sha256:92a73f2b8d7f39ef392a38131f76b970f8c66e4c42b3125ae872b7c93b556307", size = 17735, upload-time = "2026-04-08T04:30:50.974Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1c/f0/c391068b86abb708882c6d75a08cd7d25b2c7227dab527b3a3685a3c635b/types_pyyaml-6.0.12.20260408-py3-none-any.whl", hash = "sha256:fbc42037d12159d9c801ebfcc79ebd28335a7c13b08a4cfbc6916df78fee9384", size = 20339, upload-time = "2026-04-08T04:30:50.113Z" }, +] + +[[package]] +name = "types-redis" +version = "4.6.0.20241004" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "cryptography" }, + { name = "types-pyopenssl" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/3a/95/c054d3ac940e8bac4ca216470c80c26688a0e79e09f520a942bb27da3386/types-redis-4.6.0.20241004.tar.gz", hash = "sha256:5f17d2b3f9091ab75384153bfa276619ffa1cf6a38da60e10d5e6749cc5b902e", size = 49679, upload-time = "2024-10-04T02:43:59.224Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/55/82/7d25dce10aad92d2226b269bce2f85cfd843b4477cd50245d7d40ecf8f89/types_redis-4.6.0.20241004-py3-none-any.whl", hash = "sha256:ef5da68cb827e5f606c8f9c0b49eeee4c2669d6d97122f301d3a55dc6a63f6ed", size = 58737, upload-time = "2024-10-04T02:43:57.968Z" }, +] + +[[package]] +name = "types-requests" +version = "2.33.0.20260408" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/69/6a/749dc53a54a3f35842c1f8197b3ca6b54af6d7458a1bfc75f6629b6da666/types_requests-2.33.0.20260408.tar.gz", hash = "sha256:95b9a86376807a216b2fb412b47617b202091c3ea7c078f47cc358d5528ccb7b", size = 23882, upload-time = "2026-04-08T04:34:49.33Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/90/b8/78fd6c037de4788c040fdd323b3369804400351b7827473920f6c1d03c10/types_requests-2.33.0.20260408-py3-none-any.whl", hash = "sha256:81f31d5ea4acb39f03be7bc8bed569ba6d5a9c5d97e89f45ac43d819b68ca50f", size = 20739, upload-time = "2026-04-08T04:34:48.325Z" }, +] + +[[package]] +name = "types-setuptools" +version = "82.0.0.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c3/12/3464b410c50420dd4674fa5fe9d3880711c1dbe1a06f5fe4960ee9067b9e/types_setuptools-82.0.0.20260408.tar.gz", hash = "sha256:036c68caf7e672a699f5ebbf914708d40644c14e05298bc49f7272be91cf43d3", size = 44861, upload-time = "2026-04-08T04:29:33.292Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3d/e1/46a4fc3ef03aabf5d18bac9df5cf37c6b02c3bddf3e05c3533f4b4588331/types_setuptools-82.0.0.20260408-py3-none-any.whl", hash = "sha256:ece0a215cdfa6463a65fd6f68bd940f39e455729300ddfe61cab1147ed1d2462", size = 68428, upload-time = "2026-04-08T04:29:32.175Z" }, +] + +[[package]] +name = "types-tabulate" +version = "0.10.0.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/78/59/b563bfb6e216b8573052c09cb4abcbdca836487db4cfad9b7d492c327c0b/types_tabulate-0.10.0.20260408.tar.gz", hash = "sha256:903d62fdf7e5a0ff659fd5d629df716232f7658c6d30e98f0374488d06ffacf4", size = 8367, upload-time = "2026-04-08T04:30:00.482Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/38/d1/34e27f543dd944f51fc6b0013a1a41113079cede9cc3be0a5f426f2f8d9d/types_tabulate-0.10.0.20260408-py3-none-any.whl", hash = "sha256:2b19d193603d38c34645de53c0c1087e2364487d518d4a2f44268db2366723cc", size = 8139, upload-time = "2026-04-08T04:29:59.699Z" }, +] + +[[package]] +name = "types-toml" +version = "0.10.8.20260408" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/71/9b/887564a51a84c96ba08b715570e546f0ea793df6372b736bfbc596ca5536/types_toml-0.10.8.20260408.tar.gz", hash = "sha256:6b30b031235565a12febb1388900b129f1adeabfcfa594da46d0372b2ac107ad", size = 9341, upload-time = "2026-04-08T04:27:54.394Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/56/f1/942d95ba026779bc6e3064f8b094216588dc3276cc328cf8e03a0541918d/types_toml-0.10.8.20260408-py3-none-any.whl", hash = "sha256:e958d4c660385e548705a298f17dc162baf44c8b6d6aff79aeefe75f4f77ac87", size = 9677, upload-time = "2026-04-08T04:27:53.526Z" }, +] + +[[package]] +name = "typing-extensions" +version = "4.15.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/94/1a15dd82efb362ac84269196e94cf00f187f7ed21c242792a923cdb1c61f/typing_extensions-4.15.0.tar.gz", hash = "sha256:0cea48d173cc12fa28ecabc3b837ea3cf6f38c6d1136f85cbaaf598984861466", size = 109391, upload-time = "2025-08-25T13:49:26.313Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/18/67/36e9267722cc04a6b9f15c7f3441c2363321a3ea07da7ae0c0707beb2a9c/typing_extensions-4.15.0-py3-none-any.whl", hash = "sha256:f0fa19c6845758ab08074a0cfa8b7aecb71c999ca73d62883bc25cc018c4e548", size = 44614, upload-time = "2025-08-25T13:49:24.86Z" }, +] + +[[package]] +name = "typing-inspection" +version = "0.4.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/55/e3/70399cb7dd41c10ac53367ae42139cf4b1ca5f36bb3dc6c9d33acdb43655/typing_inspection-0.4.2.tar.gz", hash = "sha256:ba561c48a67c5958007083d386c3295464928b01faa735ab8547c5692e87f464", size = 75949, upload-time = "2025-10-01T02:14:41.687Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/dc/9b/47798a6c91d8bdb567fe2698fe81e0c6b7cb7ef4d13da4114b41d239f65d/typing_inspection-0.4.2-py3-none-any.whl", hash = "sha256:4ed1cacbdc298c220f1bd249ed5287caa16f34d44ef4e9c3d0cbad5b521545e7", size = 14611, upload-time = "2025-10-01T02:14:40.154Z" }, +] + +[[package]] +name = "urllib3" +version = "2.6.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c7/24/5f1b3bdffd70275f6661c76461e25f024d5a38a46f04aaca912426a2b1d3/urllib3-2.6.3.tar.gz", hash = "sha256:1b62b6884944a57dbe321509ab94fd4d3b307075e0c2eae991ac71ee15ad38ed", size = 435556, upload-time = "2026-01-07T16:24:43.925Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/39/08/aaaad47bc4e9dc8c725e68f9d04865dbcb2052843ff09c97b08904852d84/urllib3-2.6.3-py3-none-any.whl", hash = "sha256:bf272323e553dfb2e87d9bfd225ca7b0f467b919d7bbd355436d3fd37cb0acd4", size = 131584, upload-time = "2026-01-07T16:24:42.685Z" }, +] + +[[package]] +name = "wcwidth" +version = "0.6.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/35/a2/8e3becb46433538a38726c948d3399905a4c7cabd0df578ede5dc51f0ec2/wcwidth-0.6.0.tar.gz", hash = "sha256:cdc4e4262d6ef9a1a57e018384cbeb1208d8abbc64176027e2c2455c81313159", size = 159684, upload-time = "2026-02-06T19:19:40.919Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/68/5a/199c59e0a824a3db2b89c5d2dade7ab5f9624dbf6448dc291b46d5ec94d3/wcwidth-0.6.0-py3-none-any.whl", hash = "sha256:1a3a1e510b553315f8e146c54764f4fb6264ffad731b3d78088cdb1478ffbdad", size = 94189, upload-time = "2026-02-06T19:19:39.646Z" }, +] + +[[package]] +name = "wheel" +version = "0.46.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "packaging" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/89/24/a2eb353a6edac9a0303977c4cb048134959dd2a51b48a269dfc9dde00c8a/wheel-0.46.3.tar.gz", hash = "sha256:e3e79874b07d776c40bd6033f8ddf76a7dad46a7b8aa1b2787a83083519a1803", size = 60605, upload-time = "2026-01-22T12:39:49.136Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/87/22/b76d483683216dde3d67cba61fb2444be8d5be289bf628c13fc0fd90e5f9/wheel-0.46.3-py3-none-any.whl", hash = "sha256:4b399d56c9d9338230118d705d9737a2a468ccca63d5e813e2a4fc7815d8bc4d", size = 30557, upload-time = "2026-01-22T12:39:48.099Z" }, +] + +[[package]] +name = "yamllint" +version = "1.38.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pathspec" }, + { name = "pyyaml" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/28/a0/8fc2d68e132cf918f18273fdc8a1b8432b60d75ac12fdae4b0ef5c9d2e8d/yamllint-1.38.0.tar.gz", hash = "sha256:09e5f29531daab93366bb061e76019d5e91691ef0a40328f04c927387d1d364d", size = 142446, upload-time = "2026-01-13T07:47:53.276Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/05/92/aed08e68de6e6a3d7c2328ce7388072cd6affc26e2917197430b646aed02/yamllint-1.38.0-py3-none-any.whl", hash = "sha256:fc394a5b3be980a4062607b8fdddc0843f4fa394152b6da21722f5d59013c220", size = 68940, upload-time = "2026-01-13T07:47:51.343Z" }, +] + +[[package]] +name = "yarl" +version = "1.23.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "idna" }, + { name = "multidict" }, + { name = "propcache" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/23/6e/beb1beec874a72f23815c1434518bfc4ed2175065173fb138c3705f658d4/yarl-1.23.0.tar.gz", hash = "sha256:53b1ea6ca88ebd4420379c330aea57e258408dd0df9af0992e5de2078dc9f5d5", size = 194676, upload-time = "2026-03-01T22:07:53.373Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/88/8a/94615bc31022f711add374097ad4144d569e95ff3c38d39215d07ac153a0/yarl-1.23.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:1932b6b8bba8d0160a9d1078aae5838a66039e8832d41d2992daa9a3a08f7860", size = 124737, upload-time = "2026-03-01T22:05:12.897Z" }, + { url = "https://files.pythonhosted.org/packages/e3/6f/c6554045d59d64052698add01226bc867b52fe4a12373415d7991fdca95d/yarl-1.23.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:411225bae281f114067578891bc75534cfb3d92a3b4dfef7a6ca78ba354e6069", size = 87029, upload-time = "2026-03-01T22:05:14.376Z" }, + { url = "https://files.pythonhosted.org/packages/19/2a/725ecc166d53438bc88f76822ed4b1e3b10756e790bafd7b523fe97c322d/yarl-1.23.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:13a563739ae600a631c36ce096615fe307f131344588b0bc0daec108cdb47b25", size = 86310, upload-time = "2026-03-01T22:05:15.71Z" }, + { url = "https://files.pythonhosted.org/packages/99/30/58260ed98e6ff7f90ba84442c1ddd758c9170d70327394a6227b310cd60f/yarl-1.23.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:9cbf44c5cb4a7633d078788e1b56387e3d3cf2b8139a3be38040b22d6c3221c8", size = 97587, upload-time = "2026-03-01T22:05:17.384Z" }, + { url = "https://files.pythonhosted.org/packages/76/0a/8b08aac08b50682e65759f7f8dde98ae8168f72487e7357a5d684c581ef9/yarl-1.23.0-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:53ad387048f6f09a8969631e4de3f1bf70c50e93545d64af4f751b2498755072", size = 92528, upload-time = "2026-03-01T22:05:18.804Z" }, + { url = "https://files.pythonhosted.org/packages/52/07/0b7179101fe5f8385ec6c6bb5d0cb9f76bd9fb4a769591ab6fb5cdbfc69a/yarl-1.23.0-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:4a59ba56f340334766f3a4442e0efd0af895fae9e2b204741ef885c446b3a1a8", size = 105339, upload-time = "2026-03-01T22:05:20.235Z" }, + { url = "https://files.pythonhosted.org/packages/d3/8a/36d82869ab5ec829ca8574dfcb92b51286fcfb1e9c7a73659616362dc880/yarl-1.23.0-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:803a3c3ce4acc62eaf01eaca1208dcf0783025ef27572c3336502b9c232005e7", size = 105061, upload-time = "2026-03-01T22:05:22.268Z" }, + { url = "https://files.pythonhosted.org/packages/66/3e/868e5c3364b6cee19ff3e1a122194fa4ce51def02c61023970442162859e/yarl-1.23.0-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:a3d2bff8f37f8d0f96c7ec554d16945050d54462d6e95414babaa18bfafc7f51", size = 100132, upload-time = "2026-03-01T22:05:23.638Z" }, + { url = "https://files.pythonhosted.org/packages/cf/26/9c89acf82f08a52cb52d6d39454f8d18af15f9d386a23795389d1d423823/yarl-1.23.0-cp312-cp312-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:c75eb09e8d55bceb4367e83496ff8ef2bc7ea6960efb38e978e8073ea59ecb67", size = 99289, upload-time = "2026-03-01T22:05:25.749Z" }, + { url = "https://files.pythonhosted.org/packages/6f/54/5b0db00d2cb056922356104468019c0a132e89c8d3ab67d8ede9f4483d2a/yarl-1.23.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:877b0738624280e34c55680d6054a307aa94f7d52fa0e3034a9cc6e790871da7", size = 96950, upload-time = "2026-03-01T22:05:27.318Z" }, + { url = "https://files.pythonhosted.org/packages/f6/40/10fa93811fd439341fad7e0718a86aca0de9548023bbb403668d6555acab/yarl-1.23.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:b5405bb8f0e783a988172993cfc627e4d9d00432d6bbac65a923041edacf997d", size = 93960, upload-time = "2026-03-01T22:05:28.738Z" }, + { url = "https://files.pythonhosted.org/packages/bc/d2/8ae2e6cd77d0805f4526e30ec43b6f9a3dfc542d401ac4990d178e4bf0cf/yarl-1.23.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:1c3a3598a832590c5a3ce56ab5576361b5688c12cb1d39429cf5dba30b510760", size = 104703, upload-time = "2026-03-01T22:05:30.438Z" }, + { url = "https://files.pythonhosted.org/packages/2f/0c/b3ceacf82c3fe21183ce35fa2acf5320af003d52bc1fcf5915077681142e/yarl-1.23.0-cp312-cp312-musllinux_1_2_riscv64.whl", hash = "sha256:8419ebd326430d1cbb7efb5292330a2cf39114e82df5cc3d83c9a0d5ebeaf2f2", size = 98325, upload-time = "2026-03-01T22:05:31.835Z" }, + { url = "https://files.pythonhosted.org/packages/9d/e0/12900edd28bdab91a69bd2554b85ad7b151f64e8b521fe16f9ad2f56477a/yarl-1.23.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:be61f6fff406ca40e3b1d84716fde398fc08bc63dd96d15f3a14230a0973ed86", size = 105067, upload-time = "2026-03-01T22:05:33.358Z" }, + { url = "https://files.pythonhosted.org/packages/15/61/74bb1182cf79c9bbe4eb6b1f14a57a22d7a0be5e9cedf8e2d5c2086474c3/yarl-1.23.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:3ceb13c5c858d01321b5d9bb65e4cf37a92169ea470b70fec6f236b2c9dd7e34", size = 100285, upload-time = "2026-03-01T22:05:35.4Z" }, + { url = "https://files.pythonhosted.org/packages/69/7f/cd5ef733f2550de6241bd8bd8c3febc78158b9d75f197d9c7baa113436af/yarl-1.23.0-cp312-cp312-win32.whl", hash = "sha256:fffc45637bcd6538de8b85f51e3df3223e4ad89bccbfca0481c08c7fc8b7ed7d", size = 82359, upload-time = "2026-03-01T22:05:36.811Z" }, + { url = "https://files.pythonhosted.org/packages/f5/be/25216a49daeeb7af2bec0db22d5e7df08ed1d7c9f65d78b14f3b74fd72fc/yarl-1.23.0-cp312-cp312-win_amd64.whl", hash = "sha256:f69f57305656a4852f2a7203efc661d8c042e6cc67f7acd97d8667fb448a426e", size = 87674, upload-time = "2026-03-01T22:05:38.171Z" }, + { url = "https://files.pythonhosted.org/packages/d2/35/aeab955d6c425b227d5b7247eafb24f2653fedc32f95373a001af5dfeb9e/yarl-1.23.0-cp312-cp312-win_arm64.whl", hash = "sha256:6e87a6e8735b44816e7db0b2fbc9686932df473c826b0d9743148432e10bb9b9", size = 81879, upload-time = "2026-03-01T22:05:40.006Z" }, + { url = "https://files.pythonhosted.org/packages/9a/4b/a0a6e5d0ee8a2f3a373ddef8a4097d74ac901ac363eea1440464ccbe0898/yarl-1.23.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:16c6994ac35c3e74fb0ae93323bf8b9c2a9088d55946109489667c510a7d010e", size = 123796, upload-time = "2026-03-01T22:05:41.412Z" }, + { url = "https://files.pythonhosted.org/packages/67/b6/8925d68af039b835ae876db5838e82e76ec87b9782ecc97e192b809c4831/yarl-1.23.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:4a42e651629dafb64fd5b0286a3580613702b5809ad3f24934ea87595804f2c5", size = 86547, upload-time = "2026-03-01T22:05:42.841Z" }, + { url = "https://files.pythonhosted.org/packages/ae/50/06d511cc4b8e0360d3c94af051a768e84b755c5eb031b12adaaab6dec6e5/yarl-1.23.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:7c6b9461a2a8b47c65eef63bb1c76a4f1c119618ffa99ea79bc5bb1e46c5821b", size = 85854, upload-time = "2026-03-01T22:05:44.85Z" }, + { url = "https://files.pythonhosted.org/packages/c4/f4/4e30b250927ffdab4db70da08b9b8d2194d7c7b400167b8fbeca1e4701ca/yarl-1.23.0-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:2569b67d616eab450d262ca7cb9f9e19d2f718c70a8b88712859359d0ab17035", size = 98351, upload-time = "2026-03-01T22:05:46.836Z" }, + { url = "https://files.pythonhosted.org/packages/86/fc/4118c5671ea948208bdb1492d8b76bdf1453d3e73df051f939f563e7dcc5/yarl-1.23.0-cp313-cp313-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:e9d9a4d06d3481eab79803beb4d9bd6f6a8e781ec078ac70d7ef2dcc29d1bea5", size = 92711, upload-time = "2026-03-01T22:05:48.316Z" }, + { url = "https://files.pythonhosted.org/packages/56/11/1ed91d42bd9e73c13dc9e7eb0dd92298d75e7ac4dd7f046ad0c472e231cd/yarl-1.23.0-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:f514f6474e04179d3d33175ed3f3e31434d3130d42ec153540d5b157deefd735", size = 106014, upload-time = "2026-03-01T22:05:50.028Z" }, + { url = "https://files.pythonhosted.org/packages/ce/c9/74e44e056a23fbc33aca71779ef450ca648a5bc472bdad7a82339918f818/yarl-1.23.0-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:fda207c815b253e34f7e1909840fd14299567b1c0eb4908f8c2ce01a41265401", size = 105557, upload-time = "2026-03-01T22:05:51.416Z" }, + { url = "https://files.pythonhosted.org/packages/66/fe/b1e10b08d287f518994f1e2ff9b6d26f0adeecd8dd7d533b01bab29a3eda/yarl-1.23.0-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:34b6cf500e61c90f305094911f9acc9c86da1a05a7a3f5be9f68817043f486e4", size = 101559, upload-time = "2026-03-01T22:05:52.872Z" }, + { url = "https://files.pythonhosted.org/packages/72/59/c5b8d94b14e3d3c2a9c20cb100119fd534ab5a14b93673ab4cc4a4141ea5/yarl-1.23.0-cp313-cp313-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:d7504f2b476d21653e4d143f44a175f7f751cd41233525312696c76aa3dbb23f", size = 100502, upload-time = "2026-03-01T22:05:54.954Z" }, + { url = "https://files.pythonhosted.org/packages/77/4f/96976cb54cbfc5c9fd73ed4c51804f92f209481d1fb190981c0f8a07a1d7/yarl-1.23.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:578110dd426f0d209d1509244e6d4a3f1a3e9077655d98c5f22583d63252a08a", size = 98027, upload-time = "2026-03-01T22:05:56.409Z" }, + { url = "https://files.pythonhosted.org/packages/63/6e/904c4f476471afdbad6b7e5b70362fb5810e35cd7466529a97322b6f5556/yarl-1.23.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:609d3614d78d74ebe35f54953c5bbd2ac647a7ddb9c30a5d877580f5e86b22f2", size = 95369, upload-time = "2026-03-01T22:05:58.141Z" }, + { url = "https://files.pythonhosted.org/packages/9d/40/acfcdb3b5f9d68ef499e39e04d25e141fe90661f9d54114556cf83be8353/yarl-1.23.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:4966242ec68afc74c122f8459abd597afd7d8a60dc93d695c1334c5fd25f762f", size = 105565, upload-time = "2026-03-01T22:06:00.286Z" }, + { url = "https://files.pythonhosted.org/packages/5e/c6/31e28f3a6ba2869c43d124f37ea5260cac9c9281df803c354b31f4dd1f3c/yarl-1.23.0-cp313-cp313-musllinux_1_2_riscv64.whl", hash = "sha256:e0fd068364a6759bc794459f0a735ab151d11304346332489c7972bacbe9e72b", size = 99813, upload-time = "2026-03-01T22:06:01.712Z" }, + { url = "https://files.pythonhosted.org/packages/08/1f/6f65f59e72d54aa467119b63fc0b0b1762eff0232db1f4720cd89e2f4a17/yarl-1.23.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:39004f0ad156da43e86aa71f44e033de68a44e5a31fc53507b36dd253970054a", size = 105632, upload-time = "2026-03-01T22:06:03.188Z" }, + { url = "https://files.pythonhosted.org/packages/a3/c4/18b178a69935f9e7a338127d5b77d868fdc0f0e49becd286d51b3a18c61d/yarl-1.23.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:e5723c01a56c5028c807c701aa66722916d2747ad737a046853f6c46f4875543", size = 101895, upload-time = "2026-03-01T22:06:04.651Z" }, + { url = "https://files.pythonhosted.org/packages/8f/54/f5b870b5505663911dba950a8e4776a0dbd51c9c54c0ae88e823e4b874a0/yarl-1.23.0-cp313-cp313-win32.whl", hash = "sha256:1b6b572edd95b4fa8df75de10b04bc81acc87c1c7d16bcdd2035b09d30acc957", size = 82356, upload-time = "2026-03-01T22:06:06.04Z" }, + { url = "https://files.pythonhosted.org/packages/7a/84/266e8da36879c6edcd37b02b547e2d9ecdfea776be49598e75696e3316e1/yarl-1.23.0-cp313-cp313-win_amd64.whl", hash = "sha256:baaf55442359053c7d62f6f8413a62adba3205119bcb6f49594894d8be47e5e3", size = 87515, upload-time = "2026-03-01T22:06:08.107Z" }, + { url = "https://files.pythonhosted.org/packages/00/fd/7e1c66efad35e1649114fa13f17485f62881ad58edeeb7f49f8c5e748bf9/yarl-1.23.0-cp313-cp313-win_arm64.whl", hash = "sha256:fb4948814a2a98e3912505f09c9e7493b1506226afb1f881825368d6fb776ee3", size = 81785, upload-time = "2026-03-01T22:06:10.181Z" }, + { url = "https://files.pythonhosted.org/packages/9c/fc/119dd07004f17ea43bb91e3ece6587759edd7519d6b086d16bfbd3319982/yarl-1.23.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:aecfed0b41aa72b7881712c65cf764e39ce2ec352324f5e0837c7048d9e6daaa", size = 130719, upload-time = "2026-03-01T22:06:11.708Z" }, + { url = "https://files.pythonhosted.org/packages/e6/0d/9f2348502fbb3af409e8f47730282cd6bc80dec6630c1e06374d882d6eb2/yarl-1.23.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:a41bcf68efd19073376eb8cf948b8d9be0af26256403e512bb18f3966f1f9120", size = 89690, upload-time = "2026-03-01T22:06:13.429Z" }, + { url = "https://files.pythonhosted.org/packages/50/93/e88f3c80971b42cfc83f50a51b9d165a1dbf154b97005f2994a79f212a07/yarl-1.23.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:cde9a2ecd91668bcb7f077c4966d8ceddb60af01b52e6e3e2680e4cf00ad1a59", size = 89851, upload-time = "2026-03-01T22:06:15.53Z" }, + { url = "https://files.pythonhosted.org/packages/1c/07/61c9dd8ba8f86473263b4036f70fb594c09e99c0d9737a799dfd8bc85651/yarl-1.23.0-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5023346c4ee7992febc0068e7593de5fa2bf611848c08404b35ebbb76b1b0512", size = 95874, upload-time = "2026-03-01T22:06:17.553Z" }, + { url = "https://files.pythonhosted.org/packages/9e/e9/f9ff8ceefba599eac6abddcfb0b3bee9b9e636e96dbf54342a8577252379/yarl-1.23.0-cp313-cp313t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:d1009abedb49ae95b136a8904a3f71b342f849ffeced2d3747bf29caeda218c4", size = 88710, upload-time = "2026-03-01T22:06:19.004Z" }, + { url = "https://files.pythonhosted.org/packages/eb/78/0231bfcc5d4c8eec220bc2f9ef82cb4566192ea867a7c5b4148f44f6cbcd/yarl-1.23.0-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:a8d00f29b42f534cc8aa3931cfe773b13b23e561e10d2b26f27a8d309b0e82a1", size = 101033, upload-time = "2026-03-01T22:06:21.203Z" }, + { url = "https://files.pythonhosted.org/packages/cd/9b/30ea5239a61786f18fd25797151a17fbb3be176977187a48d541b5447dd4/yarl-1.23.0-cp313-cp313t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:95451e6ce06c3e104556d73b559f5da6c34a069b6b62946d3ad66afcd51642ea", size = 100817, upload-time = "2026-03-01T22:06:22.738Z" }, + { url = "https://files.pythonhosted.org/packages/62/e2/a4980481071791bc83bce2b7a1a1f7adcabfa366007518b4b845e92eeee3/yarl-1.23.0-cp313-cp313t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:531ef597132086b6cf96faa7c6c1dcd0361dd5f1694e5cc30375907b9b7d3ea9", size = 97482, upload-time = "2026-03-01T22:06:24.21Z" }, + { url = "https://files.pythonhosted.org/packages/e5/1e/304a00cf5f6100414c4b5a01fc7ff9ee724b62158a08df2f8170dfc72a2d/yarl-1.23.0-cp313-cp313t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:88f9fb0116fbfcefcab70f85cf4b74a2b6ce5d199c41345296f49d974ddb4123", size = 95949, upload-time = "2026-03-01T22:06:25.697Z" }, + { url = "https://files.pythonhosted.org/packages/68/03/093f4055ed4cae649ac53bca3d180bd37102e9e11d048588e9ab0c0108d0/yarl-1.23.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e7b0460976dc75cb87ad9cc1f9899a4b97751e7d4e77ab840fc9b6d377b8fd24", size = 95839, upload-time = "2026-03-01T22:06:27.309Z" }, + { url = "https://files.pythonhosted.org/packages/b9/28/4c75ebb108f322aa8f917ae10a8ffa4f07cae10a8a627b64e578617df6a0/yarl-1.23.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:115136c4a426f9da976187d238e84139ff6b51a20839aa6e3720cd1026d768de", size = 90696, upload-time = "2026-03-01T22:06:29.048Z" }, + { url = "https://files.pythonhosted.org/packages/23/9c/42c2e2dd91c1a570402f51bdf066bfdb1241c2240ba001967bad778e77b7/yarl-1.23.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:ead11956716a940c1abc816b7df3fa2b84d06eaed8832ca32f5c5e058c65506b", size = 100865, upload-time = "2026-03-01T22:06:30.525Z" }, + { url = "https://files.pythonhosted.org/packages/74/05/1bcd60a8a0a914d462c305137246b6f9d167628d73568505fce3f1cb2e65/yarl-1.23.0-cp313-cp313t-musllinux_1_2_riscv64.whl", hash = "sha256:fe8f8f5e70e6dbdfca9882cd9deaac058729bcf323cf7a58660901e55c9c94f6", size = 96234, upload-time = "2026-03-01T22:06:32.692Z" }, + { url = "https://files.pythonhosted.org/packages/90/b2/f52381aac396d6778ce516b7bc149c79e65bfc068b5de2857ab69eeea3b7/yarl-1.23.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:a0e317df055958a0c1e79e5d2aa5a5eaa4a6d05a20d4b0c9c3f48918139c9fc6", size = 100295, upload-time = "2026-03-01T22:06:34.268Z" }, + { url = "https://files.pythonhosted.org/packages/e5/e8/638bae5bbf1113a659b2435d8895474598afe38b4a837103764f603aba56/yarl-1.23.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:6f0fd84de0c957b2d280143522c4f91a73aada1923caee763e24a2b3fda9f8a5", size = 97784, upload-time = "2026-03-01T22:06:35.864Z" }, + { url = "https://files.pythonhosted.org/packages/80/25/a3892b46182c586c202629fc2159aa13975d3741d52ebd7347fd501d48d5/yarl-1.23.0-cp313-cp313t-win32.whl", hash = "sha256:93a784271881035ab4406a172edb0faecb6e7d00f4b53dc2f55919d6c9688595", size = 88313, upload-time = "2026-03-01T22:06:37.39Z" }, + { url = "https://files.pythonhosted.org/packages/43/68/8c5b36aa5178900b37387937bc2c2fe0e9505537f713495472dcf6f6fccc/yarl-1.23.0-cp313-cp313t-win_amd64.whl", hash = "sha256:dd00607bffbf30250fe108065f07453ec124dbf223420f57f5e749b04295e090", size = 94932, upload-time = "2026-03-01T22:06:39.579Z" }, + { url = "https://files.pythonhosted.org/packages/c6/cc/d79ba8292f51f81f4dc533a8ccfb9fc6992cabf0998ed3245de7589dc07c/yarl-1.23.0-cp313-cp313t-win_arm64.whl", hash = "sha256:ac09d42f48f80c9ee1635b2fcaa819496a44502737660d3c0f2ade7526d29144", size = 84786, upload-time = "2026-03-01T22:06:41.988Z" }, + { url = "https://files.pythonhosted.org/packages/90/98/b85a038d65d1b92c3903ab89444f48d3cee490a883477b716d7a24b1a78c/yarl-1.23.0-cp314-cp314-macosx_10_15_universal2.whl", hash = "sha256:21d1b7305a71a15b4794b5ff22e8eef96ff4a6d7f9657155e5aa419444b28912", size = 124455, upload-time = "2026-03-01T22:06:43.615Z" }, + { url = "https://files.pythonhosted.org/packages/39/54/bc2b45559f86543d163b6e294417a107bb87557609007c007ad889afec18/yarl-1.23.0-cp314-cp314-macosx_10_15_x86_64.whl", hash = "sha256:85610b4f27f69984932a7abbe52703688de3724d9f72bceb1cca667deff27474", size = 86752, upload-time = "2026-03-01T22:06:45.425Z" }, + { url = "https://files.pythonhosted.org/packages/24/f9/e8242b68362bffe6fb536c8db5076861466fc780f0f1b479fc4ffbebb128/yarl-1.23.0-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:23f371bd662cf44a7630d4d113101eafc0cfa7518a2760d20760b26021454719", size = 86291, upload-time = "2026-03-01T22:06:46.974Z" }, + { url = "https://files.pythonhosted.org/packages/ea/d8/d1cb2378c81dd729e98c716582b1ccb08357e8488e4c24714658cc6630e8/yarl-1.23.0-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:c4a80f77dc1acaaa61f0934176fccca7096d9b1ff08c8ba9cddf5ae034a24319", size = 99026, upload-time = "2026-03-01T22:06:48.459Z" }, + { url = "https://files.pythonhosted.org/packages/0a/ff/7196790538f31debe3341283b5b0707e7feb947620fc5e8236ef28d44f72/yarl-1.23.0-cp314-cp314-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:bd654fad46d8d9e823afbb4f87c79160b5a374ed1ff5bde24e542e6ba8f41434", size = 92355, upload-time = "2026-03-01T22:06:50.306Z" }, + { url = "https://files.pythonhosted.org/packages/c1/56/25d58c3eddde825890a5fe6aa1866228377354a3c39262235234ab5f616b/yarl-1.23.0-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:682bae25f0a0dd23a056739f23a134db9f52a63e2afd6bfb37ddc76292bbd723", size = 106417, upload-time = "2026-03-01T22:06:52.1Z" }, + { url = "https://files.pythonhosted.org/packages/51/8a/882c0e7bc8277eb895b31bce0138f51a1ba551fc2e1ec6753ffc1e7c1377/yarl-1.23.0-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:a82836cab5f197a0514235aaf7ffccdc886ccdaa2324bc0aafdd4ae898103039", size = 106422, upload-time = "2026-03-01T22:06:54.424Z" }, + { url = "https://files.pythonhosted.org/packages/42/2b/fef67d616931055bf3d6764885990a3ac647d68734a2d6a9e1d13de437a2/yarl-1.23.0-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1c57676bdedc94cd3bc37724cf6f8cd2779f02f6aba48de45feca073e714fe52", size = 101915, upload-time = "2026-03-01T22:06:55.895Z" }, + { url = "https://files.pythonhosted.org/packages/18/6a/530e16aebce27c5937920f3431c628a29a4b6b430fab3fd1c117b26ff3f6/yarl-1.23.0-cp314-cp314-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:c7f8dc16c498ff06497c015642333219871effba93e4a2e8604a06264aca5c5c", size = 100690, upload-time = "2026-03-01T22:06:58.21Z" }, + { url = "https://files.pythonhosted.org/packages/88/08/93749219179a45e27b036e03260fda05190b911de8e18225c294ac95bbc9/yarl-1.23.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:5ee586fb17ff8f90c91cf73c6108a434b02d69925f44f5f8e0d7f2f260607eae", size = 98750, upload-time = "2026-03-01T22:06:59.794Z" }, + { url = "https://files.pythonhosted.org/packages/d9/cf/ea424a004969f5d81a362110a6ac1496d79efdc6d50c2c4b2e3ea0fc2519/yarl-1.23.0-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:17235362f580149742739cc3828b80e24029d08cbb9c4bda0242c7b5bc610a8e", size = 94685, upload-time = "2026-03-01T22:07:01.375Z" }, + { url = "https://files.pythonhosted.org/packages/e2/b7/14341481fe568e2b0408bcf1484c652accafe06a0ade9387b5d3fd9df446/yarl-1.23.0-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:0793e2bd0cf14234983bbb371591e6bea9e876ddf6896cdcc93450996b0b5c85", size = 106009, upload-time = "2026-03-01T22:07:03.151Z" }, + { url = "https://files.pythonhosted.org/packages/0a/e6/5c744a9b54f4e8007ad35bce96fbc9218338e84812d36f3390cea616881a/yarl-1.23.0-cp314-cp314-musllinux_1_2_riscv64.whl", hash = "sha256:3650dc2480f94f7116c364096bc84b1d602f44224ef7d5c7208425915c0475dd", size = 100033, upload-time = "2026-03-01T22:07:04.701Z" }, + { url = "https://files.pythonhosted.org/packages/0c/23/e3bfc188d0b400f025bc49d99793d02c9abe15752138dcc27e4eaf0c4a9e/yarl-1.23.0-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:f40e782d49630ad384db66d4d8b73ff4f1b8955dc12e26b09a3e3af064b3b9d6", size = 106483, upload-time = "2026-03-01T22:07:06.231Z" }, + { url = "https://files.pythonhosted.org/packages/72/42/f0505f949a90b3f8b7a363d6cbdf398f6e6c58946d85c6d3a3bc70595b26/yarl-1.23.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:94f8575fbdf81749008d980c17796097e645574a3b8c28ee313931068dad14fe", size = 102175, upload-time = "2026-03-01T22:07:08.4Z" }, + { url = "https://files.pythonhosted.org/packages/aa/65/b39290f1d892a9dd671d1c722014ca062a9c35d60885d57e5375db0404b5/yarl-1.23.0-cp314-cp314-win32.whl", hash = "sha256:c8aa34a5c864db1087d911a0b902d60d203ea3607d91f615acd3f3108ac32169", size = 83871, upload-time = "2026-03-01T22:07:09.968Z" }, + { url = "https://files.pythonhosted.org/packages/a9/5b/9b92f54c784c26e2a422e55a8d2607ab15b7ea3349e28359282f84f01d43/yarl-1.23.0-cp314-cp314-win_amd64.whl", hash = "sha256:63e92247f383c85ab00dd0091e8c3fa331a96e865459f5ee80353c70a4a42d70", size = 89093, upload-time = "2026-03-01T22:07:11.501Z" }, + { url = "https://files.pythonhosted.org/packages/e0/7d/8a84dc9381fd4412d5e7ff04926f9865f6372b4c2fd91e10092e65d29eb8/yarl-1.23.0-cp314-cp314-win_arm64.whl", hash = "sha256:70efd20be968c76ece7baa8dafe04c5be06abc57f754d6f36f3741f7aa7a208e", size = 83384, upload-time = "2026-03-01T22:07:13.069Z" }, + { url = "https://files.pythonhosted.org/packages/dd/8d/d2fad34b1c08aa161b74394183daa7d800141aaaee207317e82c790b418d/yarl-1.23.0-cp314-cp314t-macosx_10_15_universal2.whl", hash = "sha256:9a18d6f9359e45722c064c97464ec883eb0e0366d33eda61cb19a244bf222679", size = 131019, upload-time = "2026-03-01T22:07:14.903Z" }, + { url = "https://files.pythonhosted.org/packages/19/ff/33009a39d3ccf4b94d7d7880dfe17fb5816c5a4fe0096d9b56abceea9ac7/yarl-1.23.0-cp314-cp314t-macosx_10_15_x86_64.whl", hash = "sha256:2803ed8b21ca47a43da80a6fd1ed3019d30061f7061daa35ac54f63933409412", size = 89894, upload-time = "2026-03-01T22:07:17.372Z" }, + { url = "https://files.pythonhosted.org/packages/0c/f1/dab7ac5e7306fb79c0190766a3c00b4cb8d09a1f390ded68c85a5934faf5/yarl-1.23.0-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:394906945aa8b19fc14a61cf69743a868bb8c465efe85eee687109cc540b98f4", size = 89979, upload-time = "2026-03-01T22:07:19.361Z" }, + { url = "https://files.pythonhosted.org/packages/aa/b1/08e95f3caee1fad6e65017b9f26c1d79877b502622d60e517de01e72f95d/yarl-1.23.0-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:71d006bee8397a4a89f469b8deb22469fe7508132d3c17fa6ed871e79832691c", size = 95943, upload-time = "2026-03-01T22:07:21.266Z" }, + { url = "https://files.pythonhosted.org/packages/c0/cc/6409f9018864a6aa186c61175b977131f373f1988e198e031236916e87e4/yarl-1.23.0-cp314-cp314t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:62694e275c93d54f7ccedcfef57d42761b2aad5234b6be1f3e3026cae4001cd4", size = 88786, upload-time = "2026-03-01T22:07:23.129Z" }, + { url = "https://files.pythonhosted.org/packages/76/40/cc22d1d7714b717fde2006fad2ced5efe5580606cb059ae42117542122f3/yarl-1.23.0-cp314-cp314t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:a31de1613658308efdb21ada98cbc86a97c181aa050ba22a808120bb5be3ab94", size = 101307, upload-time = "2026-03-01T22:07:24.689Z" }, + { url = "https://files.pythonhosted.org/packages/8f/0d/476c38e85ddb4c6ec6b20b815bdd779aa386a013f3d8b85516feee55c8dc/yarl-1.23.0-cp314-cp314t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:fb1e8b8d66c278b21d13b0a7ca22c41dd757a7c209c6b12c313e445c31dd3b28", size = 100904, upload-time = "2026-03-01T22:07:26.287Z" }, + { url = "https://files.pythonhosted.org/packages/72/32/0abe4a76d59adf2081dcb0397168553ece4616ada1c54d1c49d8936c74f8/yarl-1.23.0-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:50f9d8d531dfb767c565f348f33dd5139a6c43f5cbdf3f67da40d54241df93f6", size = 97728, upload-time = "2026-03-01T22:07:27.906Z" }, + { url = "https://files.pythonhosted.org/packages/b7/35/7b30f4810fba112f60f5a43237545867504e15b1c7647a785fbaf588fac2/yarl-1.23.0-cp314-cp314t-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:575aa4405a656e61a540f4a80eaa5260f2a38fff7bfdc4b5f611840d76e9e277", size = 95964, upload-time = "2026-03-01T22:07:30.198Z" }, + { url = "https://files.pythonhosted.org/packages/2d/86/ed7a73ab85ef00e8bb70b0cb5421d8a2a625b81a333941a469a6f4022828/yarl-1.23.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:041b1a4cefacf65840b4e295c6985f334ba83c30607441ae3cf206a0eed1a2e4", size = 95882, upload-time = "2026-03-01T22:07:32.132Z" }, + { url = "https://files.pythonhosted.org/packages/19/90/d56967f61a29d8498efb7afb651e0b2b422a1e9b47b0ab5f4e40a19b699b/yarl-1.23.0-cp314-cp314t-musllinux_1_2_armv7l.whl", hash = "sha256:d38c1e8231722c4ce40d7593f28d92b5fc72f3e9774fe73d7e800ec32299f63a", size = 90797, upload-time = "2026-03-01T22:07:34.404Z" }, + { url = "https://files.pythonhosted.org/packages/72/00/8b8f76909259f56647adb1011d7ed8b321bcf97e464515c65016a47ecdf0/yarl-1.23.0-cp314-cp314t-musllinux_1_2_ppc64le.whl", hash = "sha256:d53834e23c015ee83a99377db6e5e37d8484f333edb03bd15b4bc312cc7254fb", size = 101023, upload-time = "2026-03-01T22:07:35.953Z" }, + { url = "https://files.pythonhosted.org/packages/ac/e2/cab11b126fb7d440281b7df8e9ddbe4851e70a4dde47a202b6642586b8d9/yarl-1.23.0-cp314-cp314t-musllinux_1_2_riscv64.whl", hash = "sha256:2e27c8841126e017dd2a054a95771569e6070b9ee1b133366d8b31beb5018a41", size = 96227, upload-time = "2026-03-01T22:07:37.594Z" }, + { url = "https://files.pythonhosted.org/packages/c2/9b/2c893e16bfc50e6b2edf76c1a9eb6cb0c744346197e74c65e99ad8d634d0/yarl-1.23.0-cp314-cp314t-musllinux_1_2_s390x.whl", hash = "sha256:76855800ac56f878847a09ce6dba727c93ca2d89c9e9d63002d26b916810b0a2", size = 100302, upload-time = "2026-03-01T22:07:39.334Z" }, + { url = "https://files.pythonhosted.org/packages/28/ec/5498c4e3a6d5f1003beb23405671c2eb9cdbf3067d1c80f15eeafe301010/yarl-1.23.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:e09fd068c2e169a7070d83d3bde728a4d48de0549f975290be3c108c02e499b4", size = 98202, upload-time = "2026-03-01T22:07:41.717Z" }, + { url = "https://files.pythonhosted.org/packages/fe/c3/cd737e2d45e70717907f83e146f6949f20cc23cd4bf7b2688727763aa458/yarl-1.23.0-cp314-cp314t-win32.whl", hash = "sha256:73309162a6a571d4cbd3b6a1dcc703c7311843ae0d1578df6f09be4e98df38d4", size = 90558, upload-time = "2026-03-01T22:07:43.433Z" }, + { url = "https://files.pythonhosted.org/packages/e1/19/3774d162f6732d1cfb0b47b4140a942a35ca82bb19b6db1f80e9e7bdc8f8/yarl-1.23.0-cp314-cp314t-win_amd64.whl", hash = "sha256:4503053d296bc6e4cbd1fad61cf3b6e33b939886c4f249ba7c78b602214fabe2", size = 97610, upload-time = "2026-03-01T22:07:45.773Z" }, + { url = "https://files.pythonhosted.org/packages/51/47/3fa2286c3cb162c71cdb34c4224d5745a1ceceb391b2bd9b19b668a8d724/yarl-1.23.0-cp314-cp314t-win_arm64.whl", hash = "sha256:44bb7bef4ea409384e3f8bc36c063d77ea1b8d4a5b2706956c0d6695f07dcc25", size = 86041, upload-time = "2026-03-01T22:07:49.026Z" }, + { url = "https://files.pythonhosted.org/packages/69/68/c8739671f5699c7dc470580a4f821ef37c32c4cb0b047ce223a7f115757f/yarl-1.23.0-py3-none-any.whl", hash = "sha256:a2df6afe50dea8ae15fa34c9f824a3ee958d785fd5d089063d960bae1daa0a3f", size = 48288, upload-time = "2026-03-01T22:07:51.388Z" }, +] diff --git a/task-sdk/.pre-commit-config.yaml b/task-sdk/.pre-commit-config.yaml index 100a6e6490849..a7b79e8886ff1 100644 --- a/task-sdk/.pre-commit-config.yaml +++ b/task-sdk/.pre-commit-config.yaml @@ -43,11 +43,13 @@ repos: ^src/airflow/sdk/definitions/deadline\.py$| ^src/airflow/sdk/definitions/dag\.py$| ^src/airflow/sdk/definitions/_internal/types\.py$| + ^src/airflow/sdk/execution_time/coordinator\.py$| ^src/airflow/sdk/execution_time/execute_workload\.py$| ^src/airflow/sdk/execution_time/secrets_masker\.py$| ^src/airflow/sdk/execution_time/callback_supervisor\.py$| ^src/airflow/sdk/execution_time/supervisor\.py$| ^src/airflow/sdk/execution_time/task_runner\.py$| + ^src/airflow/sdk/execution_time/coordinator\.py$| ^src/airflow/sdk/serde/serializers/kubernetes\.py$| ^src/airflow/sdk/types.py$ - id: check-init-decorator-arguments diff --git a/task-sdk/pyproject.toml b/task-sdk/pyproject.toml index 6e4a0b1017d01..ced299cc41bd0 100644 --- a/task-sdk/pyproject.toml +++ b/task-sdk/pyproject.toml @@ -144,6 +144,7 @@ path = "src/airflow/sdk/__init__.py" "../shared/secrets_masker/src/airflow_shared/secrets_masker" = "src/airflow/sdk/_shared/secrets_masker" "../shared/serialization/src/airflow_shared/serialization" = "src/airflow/sdk/_shared/serialization" "../shared/timezones/src/airflow_shared/timezones" = "src/airflow/sdk/_shared/timezones" +"../shared/workloads/src/airflow_shared/workloads" = "src/airflow/sdk/_shared/workloads" "../shared/listeners/src/airflow_shared/listeners" = "src/airflow/sdk/_shared/listeners" "../shared/plugins_manager/src/airflow_shared/plugins_manager" = "src/airflow/sdk/_shared/plugins_manager" "../shared/providers_discovery/src/airflow_shared/providers_discovery" = "src/airflow/sdk/_shared/providers_discovery" @@ -317,6 +318,7 @@ shared_distributions = [ "apache-airflow-shared-secrets-masker", "apache-airflow-shared-serialization", "apache-airflow-shared-timezones", + "apache-airflow-shared-workloads", "apache-airflow-shared-observability", "apache-airflow-shared-plugins-manager", "apache-airflow-shared-providers-discovery", diff --git a/task-sdk/src/airflow/sdk/_shared/workloads b/task-sdk/src/airflow/sdk/_shared/workloads new file mode 120000 index 0000000000000..25036f1ff8e02 --- /dev/null +++ b/task-sdk/src/airflow/sdk/_shared/workloads @@ -0,0 +1 @@ +../../../../../shared/workloads/src/airflow_shared/workloads \ No newline at end of file diff --git a/task-sdk/src/airflow/sdk/execution_time/comms.py b/task-sdk/src/airflow/sdk/execution_time/comms.py index 1e11e9636e56f..93e7385b56348 100644 --- a/task-sdk/src/airflow/sdk/execution_time/comms.py +++ b/task-sdk/src/airflow/sdk/execution_time/comms.py @@ -64,6 +64,7 @@ import structlog from pydantic import AwareDatetime, BaseModel, ConfigDict, Field, JsonValue, TypeAdapter +from airflow.sdk._shared.workloads import TaskInstanceDTO # noqa: TC001 -- Pydantic needs this at runtime from airflow.sdk.api.datamodels._generated import ( AssetEventDagRunReference, AssetEventResponse, @@ -79,7 +80,6 @@ PreviousTIResponse, PrevSuccessfulDagRunResponse, TaskBreadcrumbsResponse, - TaskInstance, TaskInstanceState, TaskStatesResponse, TIDeferredStatePayload, @@ -332,7 +332,7 @@ def _get_response(self) -> ReceiveMsgType | None: class StartupDetails(BaseModel): model_config = ConfigDict(arbitrary_types_allowed=True) - ti: TaskInstance + ti: TaskInstanceDTO dag_rel_path: str bundle_info: BundleInfo start_date: datetime diff --git a/task-sdk/src/airflow/sdk/execution_time/coordinator.py b/task-sdk/src/airflow/sdk/execution_time/coordinator.py new file mode 100644 index 0000000000000..c3d0594bebe14 --- /dev/null +++ b/task-sdk/src/airflow/sdk/execution_time/coordinator.py @@ -0,0 +1,462 @@ +# +# 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. +""" +Runtime coordinator for non-Python DAG file processing and task execution. + +Provides :class:`BaseCoordinator`, the base class for +SDK-specific coordinators that bridge subprocess I/O between the +Airflow supervisor and an external-SDK runtime (Java, Go, Rust, etc.). + +The coordinator's :meth:`~BaseCoordinator.run_dag_parsing` method +handles the full lifecycle: + +1. Creates TCP servers for comm and logs channels. +2. Calls :meth:`~BaseCoordinator.dag_parsing_cmd` (provided + by the subclass) to obtain the subprocess command. +3. Spawns the subprocess and accepts TCP connections from it. +4. Runs a selector-based bridge that transparently forwards bytes + between fd 0 (supervisor) and the subprocess comm socket, and + re-emits the subprocess's log output through structlog. + +I/O multiplexing uses the same selector-based loop as +:class:`~airflow.sdk.execution_time.supervisor.WatchedSubprocess`, +driven by :func:`~airflow.sdk.execution_time.selector_loop.service_selector`. +""" + +from __future__ import annotations + +import contextlib +import os +import selectors +import socket +import subprocess +import time +from typing import TYPE_CHECKING, NamedTuple + +if TYPE_CHECKING: + from structlog.typing import FilteringBoundLogger + from typing_extensions import Self + + from airflow.sdk._shared.workloads import TaskInstanceDTO + from airflow.sdk.api.datamodels._generated import BundleInfo + from airflow.sdk.execution_time.comms import StartupDetails + + +def _start_server() -> socket.socket: + """Create a TCP server socket bound to a random port on localhost.""" + server = socket.socket() + server.bind(("127.0.0.1", 0)) + server.setblocking(True) + server.listen(1) + return server + + +def _send_startup_details(runtime_comm: socket.socket, startup_details: StartupDetails) -> None: + """ + Re-encode and send the ``StartupDetails`` frame to the runtime subprocess. + + In the task execution flow, ``task_runner.main()`` consumes the + ``StartupDetails`` message from fd 0 (to determine routing) before + delegating to the runtime coordinator. This function re-serializes + the message and writes it to the runtime subprocess's comm socket so + the subprocess receives it as if it came directly from the supervisor. + """ + from airflow.sdk.execution_time.comms import _ResponseFrame + + # Use mode="json" so that datetime, UUID, and other complex Python + # types are serialized as plain strings/numbers in msgpack — avoiding + # msgpack extension types (e.g. Timestamp) that non-Python decoders + # may not support. + frame = _ResponseFrame(id=0, body=startup_details.model_dump(mode="json")) + runtime_comm.sendall(frame.as_bytes()) + + +def _bridge( + supervisor_comm: socket.socket, + runtime_comm: socket.socket, + runtime_logs: socket.socket, + runtime_stderr: socket.socket, + proc: subprocess.Popen, + log: FilteringBoundLogger, +) -> None: + """ + Multiplex I/O between the supervisor and a runtime subprocess. + + Four channels are registered with the selector: + + - ``supervisor_comm`` -> ``runtime_comm`` (raw byte forwarding) + - ``runtime_comm`` -> ``supervisor_comm`` (raw byte forwarding) + - ``runtime_logs`` -> structlog (line-buffered JSON logs) + - ``runtime_stderr`` -> structlog (line-buffered stderr output) + + Uses the same ``(handler, on_close)`` callback contract as + :class:`~airflow.sdk.execution_time.supervisor.WatchedSubprocess`, + driven by :func:`~airflow.sdk.execution_time.selector_loop.service_selector`. + """ + from airflow.sdk.execution_time.selector_loop import ( + make_buffered_socket_reader, + make_raw_forwarder, + service_selector, + ) + from airflow.sdk.execution_time.supervisor import ( + forward_to_log, + process_log_messages_from_subprocess, + ) + + sel = selectors.DefaultSelector() + + def on_close(sock: socket.socket) -> None: + with contextlib.suppress(KeyError): + sel.unregister(sock) + + target_loggers = (log,) + + # Comm: bidirectional raw byte forwarding. + sel.register(supervisor_comm, selectors.EVENT_READ, make_raw_forwarder(runtime_comm, on_close)) + sel.register(runtime_comm, selectors.EVENT_READ, make_raw_forwarder(supervisor_comm, on_close)) + + # TCP logs channel: line-buffered JSON from the runtime SDK's LogSender, + # processed with the same handler as WatchedSubprocess (level mapping, + # timestamp parsing, exception extraction). + sel.register( + runtime_logs, + selectors.EVENT_READ, + make_buffered_socket_reader(process_log_messages_from_subprocess(target_loggers), on_close), + ) + # stderr: plain-text output from the runtime process's logging framework + # (e.g. SLF4J simple logger). Use forward_to_log which handles raw + # text lines, not process_log_messages_from_subprocess which expects JSON. + import logging + + sel.register( + runtime_stderr, + selectors.EVENT_READ, + make_buffered_socket_reader( + forward_to_log(target_loggers, logger="task.stderr", level=logging.ERROR), on_close + ), + ) + + # Event loop -- runs until the subprocess exits and all sockets are drained. + while sel.get_map(): + service_selector(sel, timeout=1.0) + if proc.poll() is not None: + # Subprocess has exited -- drain remaining data with a short deadline. + deadline = time.monotonic() + 5.0 + while sel.get_map() and time.monotonic() < deadline: + service_selector(sel, timeout=0.5) + break + + sel.close() + for sock in (supervisor_comm, runtime_comm, runtime_logs, runtime_stderr): + with contextlib.suppress(OSError): + sock.close() + + +class BaseCoordinator: + """ + Base coordinator for runtime-specific DAG file processing and task execution. + + Providers register subclasses in their ``provider.yaml`` under + ``coordinators``. Both :class:`ProvidersManager` (airflow-core) + and :class:`ProvidersManagerTaskRuntime` (task-sdk) discover registered + coordinators through this single extension point. + + Subclasses represent a specific SDK runtime (Java, Go, etc.) and + only need to implement :meth:`can_handle_dag_file`, + :meth:`dag_parsing_cmd` and :meth:`task_execution_cmd`. + The base class owns the entire bridge lifecycle: TCP servers, + subprocess management, selector-based I/O loop, and cleanup. + """ + + sdk: str + file_extension: str + + class DagParsingInfo(NamedTuple): + """Information needed for runtime Dag parsing.""" + + dag_file_path: str + bundle_name: str + bundle_path: str + mode: str = "dag-parsing" + + class TaskExecutionInfo(NamedTuple): + """Information needed for runtime task execution.""" + + what: TaskInstanceDTO + dag_rel_path: str | os.PathLike[str] + bundle_info: BundleInfo + startup_details: StartupDetails + mode: str = "task-execution" + + @classmethod + def can_handle_dag_file(cls, bundle_name: str, path: str | os.PathLike[str]) -> bool: + """ + Return ``True`` if this coordinator should handle DAG-file parsing for *path*. + + Called by :meth:`DagFileProcessorProcess._resolve_processor_target` to + decide whether to delegate parsing to this coordinator's + :meth:`run_dag_parsing` instead of the default Python entrypoint. + + The default implementation returns ``False``; subclasses must override. + """ + return False + + @classmethod + def get_code_from_file(cls, fileloc: str) -> str: + """ + Return the human-readable source code for a DAG file managed by this coordinator. + + Called by :class:`~airflow.models.dagcode.DagCode` when persisting DAG + source to the metadata database. The default Python path reads ``.py`` + files directly; runtime coordinators must override this to extract source + from their native packaging format (e.g. reading an embedded ``.java`` + file from a JAR bundle). + + :param fileloc: Absolute path to the DAG file (e.g. a ``/path/to/example.jar``). + :return: The source code as a string. + :raises FileNotFoundError: If source code cannot be retrieved from *fileloc*. + """ + raise NotImplementedError + + @classmethod + def dag_parsing_cmd( + cls, + *, + dag_file_path: str, + bundle_name: str, + bundle_path: str, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """ + Return the subprocess command for DAG file parsing. + + :param dag_file_path: Absolute path to the DAG file to parse. + :param bundle_name: Name of the DAG bundle. + :param bundle_path: Root path of the DAG bundle. + :param comm_addr: ``host:port`` the subprocess must connect to + for the bidirectional msgpack comm channel. + :param logs_addr: ``host:port`` the subprocess must connect to + for the structured JSON log channel. + :returns: Full command list (e.g. ``["java", "-cp", "...", ...]`` based on each runtime). + """ + raise NotImplementedError + + @classmethod + def task_execution_cmd( + cls, + *, + what: TaskInstanceDTO, + dag_file_path: str, + bundle_path: str, + bundle_info: BundleInfo, + comm_addr: str, + logs_addr: str, + ) -> list[str]: + """ + Return the subprocess command for task execution. + + :param what: The task instance to execute. + :param dag_file_path: Absolute path to the DAG file. + :param bundle_path: Root path of the DAG bundle. + :param bundle_info: Bundle metadata. + :param comm_addr: ``host:port`` the subprocess must connect to + for the bidirectional msgpack comm channel. + :param logs_addr: ``host:port`` the subprocess must connect to + for the structured JSON log channel. + :returns: Full command list. + """ + raise NotImplementedError + + @classmethod + def run_dag_parsing(cls, *, path: str, bundle_name: str, bundle_path: str) -> None: + """Entry point for running runtime-specific Dag File Processing.""" + cls._runtime_subprocess_entrypoint( + cls.DagParsingInfo( + dag_file_path=path, + bundle_name=bundle_name, + bundle_path=bundle_path, + ) + ) + + @classmethod + def run_task_execution( + cls, + *, + what: TaskInstanceDTO, + dag_rel_path: str | os.PathLike[str], + bundle_info: BundleInfo, + startup_details: StartupDetails, + ) -> None: + cls._runtime_subprocess_entrypoint( + cls.TaskExecutionInfo( + what=what, + dag_rel_path=dag_rel_path, + bundle_info=bundle_info, + startup_details=startup_details, + ) + ) + + @classmethod + def _runtime_subprocess_entrypoint(cls, entrypoint_info: DagParsingInfo | TaskExecutionInfo) -> None: + """ + Spawn the runtime subprocess and bridge I/O with the supervisor. + + This is called inside the forked child process where fd 0 is the + bidirectional comms socket to the supervisor. The method: + + 1. Creates TCP servers for comm and logs. + 2. Calls :meth:`dag_parsing_cmd` or :meth:`task_execution_cmd` to get the command. + 3. Spawns the subprocess with ``stdin=/dev/null`` and stderr + captured via a socketpair. + 4. Runs the selector-based bridge until the subprocess exits. + + fd layout (set up by ``_reopen_std_io_handles`` before this runs): + + - fd 0 -- bidirectional comms socket to the supervisor + (``DagFileParseRequest`` <-> ``DagFileParsingResult``, + length-prefixed msgpack frames) + - fd 1 -- stdout socket to the supervisor + - fd 2 -- stderr socket to the supervisor + - fd N -- structured JSON log channel (``log_fd``, configured by + ``_configure_logs_over_json_channel`` -> structlog) + """ + os.environ["_AIRFLOW_PROCESS_CONTEXT"] = "client" + + import structlog + + log = structlog.get_logger(logger_name="task") + log.info( + "Starting runtime subprocess", + sdk=cls.sdk, + mode=entrypoint_info.mode, + ) + + # TCP servers for the runtime subprocess to connect to. + comm_server = _start_server() + logs_server = _start_server() + comm_host, comm_port = comm_server.getsockname() + logs_host, logs_port = logs_server.getsockname() + + comm_addr = f"{comm_host}:{comm_port}" + logs_addr = f"{logs_host}:{logs_port}" + + # stderr uses a socketpair (instead of ``subprocess.PIPE``) so it + # is a real socket compatible with ``make_buffered_socket_reader``. + child_stderr, read_stderr = socket.socketpair() + + # For task execution, hold a BundleVersionLock for the entire + # subprocess lifetime to prevent the bundle version from being + # garbage-collected while the runtime process is still running. + bundle_version_lock: contextlib.AbstractContextManager = contextlib.nullcontext() + + if isinstance(entrypoint_info, cls.DagParsingInfo): + cmd = cls.dag_parsing_cmd( + dag_file_path=entrypoint_info.dag_file_path, + bundle_name=entrypoint_info.bundle_name, + bundle_path=entrypoint_info.bundle_path, + comm_addr=comm_addr, + logs_addr=logs_addr, + ) + elif isinstance(entrypoint_info, cls.TaskExecutionInfo): + from airflow.dag_processing.bundles.base import BundleVersionLock + from airflow.sdk.execution_time.task_runner import resolve_bundle + + bundle_instance = resolve_bundle(entrypoint_info.bundle_info, log) + resolved_dag_file_path = bundle_instance.path / entrypoint_info.dag_rel_path + + cmd = cls.task_execution_cmd( + what=entrypoint_info.what, + dag_file_path=os.fspath(resolved_dag_file_path), + bundle_path=os.fspath(bundle_instance.path), + bundle_info=entrypoint_info.bundle_info, + comm_addr=comm_addr, + logs_addr=logs_addr, + ) + bundle_version_lock = BundleVersionLock( + bundle_name=entrypoint_info.bundle_info.name, + bundle_version=entrypoint_info.bundle_info.version, + ) + else: + raise ValueError(f"Unknown entrypoint_info type: {type(entrypoint_info)}") + + with bundle_version_lock: + # stdin redirected to /dev/null so the subprocess does not inherit + # fd 0 (the comms socket). + proc = subprocess.Popen( + cmd, + stdin=subprocess.DEVNULL, + stderr=child_stderr.fileno(), + ) + child_stderr.close() + + # Wait for the subprocess to connect to both servers. + runtime_comm, _ = comm_server.accept() + runtime_logs, _ = logs_server.accept() + comm_server.close() + logs_server.close() + + # For task execution the supervisor already sent ``StartupDetails`` + # on fd 0 and ``task_runner.main()`` consumed it before delegating + # here. Re-encode and forward it to the runtime subprocess so it + # knows which task to execute. + if isinstance(entrypoint_info, cls.TaskExecutionInfo): + _send_startup_details(runtime_comm, entrypoint_info.startup_details) + + # fd 0 is the bidirectional comms socket to the supervisor. + supervisor_comm = socket.socket(fileno=os.dup(0)) + + _bridge(supervisor_comm, runtime_comm, runtime_logs, read_stderr, proc, log) + + +class QueueToCoordinatorMapper: + """ + Map queue names to coordinator names. + + Users often use queues as environment/isolation identifiers (e.g. ``"java-11"``, + ``"java-12"``). This mapper lets them reuse existing queue assignments to route + tasks to the correct coordinator. + + The mapping is read from the ``[sdk] queue_to_sdk`` + configuration option, which is a JSON dict of ``queue -> sdk``. + + Example configuration:: + + [sdk] + queue_to_sdk = {"java-11": "java", "java-12": "java"} + """ + + def __init__(self, mapping: dict[str, str]) -> None: + self._mapping = mapping + + @classmethod + def from_config(cls) -> Self: + """Load the queue-to-runtime mapping from airflow configuration.""" + from airflow.sdk.configuration import conf + + mapping = conf.getjson("sdk", "queue_to_sdk", fallback={}) + if not isinstance(mapping, dict): + return cls({}) + return cls(mapping) + + def resolve(self, queue: str) -> str | None: + """Return the runtime coordinator name for *queue*, or ``None`` if unmapped.""" + return self._mapping.get(queue) + + +__all__ = ["BaseCoordinator", "QueueToCoordinatorMapper"] diff --git a/task-sdk/src/airflow/sdk/execution_time/selector_loop.py b/task-sdk/src/airflow/sdk/execution_time/selector_loop.py new file mode 100644 index 0000000000000..d67014ad1b418 --- /dev/null +++ b/task-sdk/src/airflow/sdk/execution_time/selector_loop.py @@ -0,0 +1,159 @@ +# +# 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. +""" +Selector-based I/O loop utilities shared across subprocess monitors. + +Both :class:`~airflow.sdk.execution_time.supervisor.WatchedSubprocess` +(supervisor-side) and provider-registered bridges such as the Locale DagFileProcessor (child-side) use these building blocks to multiplex +socket I/O without threads. + +The common contract for every callback registered with the selector: + +* The selector stores a ``(handler, on_close)`` tuple as ``key.data``. +* ``handler(fileobj) -> bool`` — read available data and return + ``True`` to keep listening, ``False`` on EOF / error. +* ``on_close(fileobj)`` — called when the handler returns ``False``; + must unregister the fileobj from the selector. +* :func:`service_selector` drives one iteration of this protocol. +""" + +from __future__ import annotations + +import selectors +from contextlib import suppress +from typing import TYPE_CHECKING + +if TYPE_CHECKING: + from collections.abc import Callable, Generator + from socket import socket + + # (handler, on_close) — stored as ``selector.register(..., data=cb)`` + SelectorCallback = tuple[Callable[[socket], bool], Callable[[socket], None]] + + +# Sockets, even the `.makefile()` function don't correctly do line buffering on reading. If a chunk is read +# and it doesn't contain a new line character, `.readline()` will just return the chunk as is. +# +# This returns a callback suitable for attaching to a `selector` that reads in to a buffer, and yields lines +# to a (sync) generator +def make_buffered_socket_reader( + gen: Generator[None, bytes | bytearray, None], + on_close: Callable[[socket], None], + buffer_size: int = 4096, +) -> SelectorCallback: + """ + Create a selector callback that line-buffers socket data into a generator. + + Bytes are accumulated until a newline is found; each + complete line is sent to *gen* via ``gen.send(line)``. On EOF the + remainder of the buffer (if any) is flushed. + + Returns a ``(handler, on_close)`` tuple suitable for + ``selector.register(..., data=...)``. + """ + buffer = bytearray() # This will hold our accumulated binary data + read_buffer = bytearray(buffer_size) # Temporary buffer for each read + + # We need to start up the generator to get it to the point it's at waiting on the yield + next(gen) + + def cb(sock: socket): + nonlocal buffer, read_buffer + # Read up to `buffer_size` bytes of data from the socket + n_received = sock.recv_into(read_buffer) + + if not n_received: + # If no data is returned, the connection is closed. Return whatever is left in the buffer + if len(buffer): + with suppress(StopIteration): + gen.send(buffer) + return False + + buffer.extend(read_buffer[:n_received]) + + # We could have read multiple lines in one go, yield them all + while (newline_pos := buffer.find(b"\n")) != -1: + line = buffer[: newline_pos + 1] + try: + gen.send(line) + except StopIteration: + return False + buffer = buffer[newline_pos + 1 :] # Update the buffer with remaining data + + return True + + return cb, on_close + + +def make_raw_forwarder( + dest: socket, + on_close: Callable[[socket], None], +) -> SelectorCallback: + """ + Create a selector callback that forwards raw bytes to *dest*. + + Used for transparent protocol bridges where bytes must be shuttled + between two sockets without interpretation (e.g. length-prefixed + msgpack frames between a supervisor and a Java subprocess). + """ + + def cb(sock: socket) -> bool: + data = sock.recv(65536) + if not data: + return False + try: + dest.sendall(data) + except (BrokenPipeError, ConnectionResetError, OSError): + return False + return True + + return cb, on_close + + +def service_selector(selector: selectors.BaseSelector, timeout: float = 1.0) -> None: + """ + Process one round of selector events. + + For each ready socket whose handler returns ``False`` (EOF / error), + the socket's *on_close* callback is invoked and the socket is closed. + """ + # Ensure minimum timeout to prevent CPU spike with tight loop when timeout is 0 or negative + timeout = max(0.01, timeout) + events = selector.select(timeout=timeout) + for key, _ in events: + # Retrieve the handler responsible for processing this file object (e.g., stdout, stderr) + socket_handler, on_close = key.data + + # Example of handler behavior: + # If the subprocess writes "Hello, World!" to stdout: + # - `socket_handler` reads and processes the message. + # - If EOF is reached, the handler returns False to signal no more reads are expected. + # - BrokenPipeError should be caught and treated as if the handler returned false, similar + # to EOF case + try: + need_more = socket_handler(key.fileobj) + except (BrokenPipeError, ConnectionResetError): + need_more = False + + # If the handler signals that the file object is no longer needed (EOF, closed, etc.) + # unregister it from the selector to stop monitoring; `wait()` blocks until all selectors + # are removed. + if not need_more: + sock: socket = key.fileobj # type: ignore[assignment] + on_close(sock) + sock.close() diff --git a/task-sdk/src/airflow/sdk/execution_time/supervisor.py b/task-sdk/src/airflow/sdk/execution_time/supervisor.py index cd25d9279571b..fabb2db246911 100644 --- a/task-sdk/src/airflow/sdk/execution_time/supervisor.py +++ b/task-sdk/src/airflow/sdk/execution_time/supervisor.py @@ -126,6 +126,7 @@ handle_get_variable, handle_mask_secret, ) +from airflow.sdk.execution_time.selector_loop import make_buffered_socket_reader, service_selector try: from socket import send_fds @@ -141,9 +142,11 @@ from structlog.typing import FilteringBoundLogger, WrappedLogger from typing_extensions import Self + from airflow._shared.workloads import TaskInstanceDTO from airflow.executors.workloads import BundleInfo from airflow.sdk.bases.secrets_backend import BaseSecretsBackend from airflow.sdk.definitions.connection import Connection + from airflow.sdk.execution_time.selector_loop import SelectorCallback from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI __all__ = ["ActivitySubprocess", "WatchedSubprocess", "supervise", "supervise_task"] @@ -695,7 +698,7 @@ def _get_target_loggers(self) -> tuple[FilteringBoundLogger, ...]: target_loggers += (log,) return target_loggers - def _create_log_forwarder(self, loggers, name, log_level=logging.INFO) -> Callable[[socket], bool]: + def _create_log_forwarder(self, loggers, name, log_level=logging.INFO) -> SelectorCallback: """Create a socket handler that forwards logs to a logger.""" loggers = tuple( reconfigure_logger( @@ -888,41 +891,15 @@ def _service_subprocess( """ Service subprocess events by processing socket activity and checking for process exit. - This method: - - Waits for activity on the registered file objects (via `self.selector.select`). - - Processes any events triggered on these file objects. - - Checks if the subprocess has exited during the wait. + Delegates the selector event loop to :func:`service_selector` (shared + with provider-registered bridges), then checks the subprocess status. :param max_wait_time: Maximum time to block while waiting for events, in seconds. :param raise_on_timeout: If True, raise an exception if the subprocess does not exit within the timeout. :param expect_signal: Signal not to log if the task exits with this code. :returns: The process exit code, or None if it's still alive """ - # Ensure minimum timeout to prevent CPU spike with tight loop when timeout is 0 or negative - timeout = max(0.01, max_wait_time) - events = self.selector.select(timeout=timeout) - for key, _ in events: - # Retrieve the handler responsible for processing this file object (e.g., stdout, stderr) - socket_handler, on_close = key.data - - # Example of handler behavior: - # If the subprocess writes "Hello, World!" to stdout: - # - `socket_handler` reads and processes the message. - # - If EOF is reached, the handler returns False to signal no more reads are expected. - # - BrokenPipeError should be caught and treated as if the handler returned false, similar - # to EOF case - try: - need_more = socket_handler(key.fileobj) - except (BrokenPipeError, ConnectionResetError): - need_more = False - - # If the handler signals that the file object is no longer needed (EOF, closed, etc.) - # unregister it from the selector to stop monitoring; `wait()` blocks until all selectors - # are removed. - if not need_more: - sock: socket = key.fileobj # type: ignore[assignment] - on_close(sock) - sock.close() + service_selector(self.selector, timeout=max_wait_time) # Check if the subprocess has exited return self._check_subprocess_exit(raise_on_timeout=raise_on_timeout, expect_signal=expect_signal) @@ -1132,7 +1109,7 @@ class ActivitySubprocess(WatchedSubprocess): def start( # type: ignore[override] cls, *, - what: TaskInstance, + what: TaskInstanceDTO, dag_rel_path: str | os.PathLike[str], bundle_info, client: Client, @@ -1161,7 +1138,7 @@ def start( # type: ignore[override] def _on_child_started( self, *, - ti: TaskInstance, + ti: TaskInstanceDTO, dag_rel_path: str | os.PathLike[str], bundle_info, sentry_integration: str, @@ -1932,50 +1909,6 @@ def run_task_in_process(ti: TaskInstance, task) -> TaskRunResult: return InProcessTestSupervisor.start(what=ti, task=task) -# Sockets, even the `.makefile()` function don't correctly do line buffering on reading. If a chunk is read -# and it doesn't contain a new line character, `.readline()` will just return the chunk as is. -# -# This returns a callback suitable for attaching to a `selector` that reads in to a buffer, and yields lines -# to a (sync) generator -def make_buffered_socket_reader( - gen: Generator[None, bytes | bytearray, None], - on_close: Callable[[socket], None], - buffer_size: int = 4096, -): - buffer = bytearray() # This will hold our accumulated binary data - read_buffer = bytearray(buffer_size) # Temporary buffer for each read - - # We need to start up the generator to get it to the point it's at waiting on the yield - next(gen) - - def cb(sock: socket): - nonlocal buffer, read_buffer - # Read up to `buffer_size` bytes of data from the socket - n_received = sock.recv_into(read_buffer) - - if not n_received: - # If no data is returned, the connection is closed. Return whatever is left in the buffer - if len(buffer): - with suppress(StopIteration): - gen.send(buffer) - return False - - buffer.extend(read_buffer[:n_received]) - - # We could have read multiple lines in one go, yield them all - while (newline_pos := buffer.find(b"\n")) != -1: - line = buffer[: newline_pos + 1] - try: - gen.send(line) - except StopIteration: - return False - buffer = buffer[newline_pos + 1 :] # Update the buffer with remaining data - - return True - - return cb, on_close - - def length_prefixed_frame_reader( gen: Generator[None, _RequestFrame, None], on_close: Callable[[socket], None] ): @@ -2150,7 +2083,7 @@ def _configure_logging(log_path: str, client: Client) -> tuple[FilteringBoundLog def supervise_task( *, - ti: TaskInstance, + ti: TaskInstanceDTO, bundle_info: BundleInfo, dag_rel_path: str | os.PathLike[str], token: str, diff --git a/task-sdk/src/airflow/sdk/execution_time/task_runner.py b/task-sdk/src/airflow/sdk/execution_time/task_runner.py index 56ba8343c648b..68a5240386af0 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -47,6 +47,7 @@ from airflow.sdk.api.client import get_hostname, getuser from airflow.sdk.api.datamodels._generated import ( AssetProfile, + BundleInfo, DagRun, PreviousTIResponse, TaskInstance, @@ -778,12 +779,7 @@ def parse(what: StartupDetails, log: Logger) -> RuntimeTaskInstance: bundle_info = what.bundle_info bundle_prepare_start = time.monotonic() - bundle_instance = DagBundlesManager().get_bundle( - name=bundle_info.name, - version=bundle_info.version, - ) - bundle_instance.initialize() - _verify_bundle_access(bundle_instance, log) + bundle_instance = resolve_bundle(bundle_info, log) bundle_prepare_ms = int((time.monotonic() - bundle_prepare_start) * 1000) dag_absolute_path = os.fspath(Path(bundle_instance.path, what.dag_rel_path)) @@ -909,6 +905,22 @@ def _verify_bundle_access(bundle_instance: BaseDagBundle, log: Logger) -> None: ) +def resolve_bundle(bundle_info: BundleInfo, log: Logger) -> BaseDagBundle: + """ + Resolve, initialize, and verify access to a DAG bundle. + + Used by both the standard Python task execution path and locale + coordinators (Java, Go, etc.) to obtain a ready-to-use bundle instance. + """ + bundle_instance = DagBundlesManager().get_bundle( + name=bundle_info.name, + version=bundle_info.version, + ) + bundle_instance.initialize() + _verify_bundle_access(bundle_instance, log) + return bundle_instance + + def get_startup_details() -> StartupDetails: # The parent sends us a StartupDetails message un-prompted. After this, every single message is only sent # in response to us sending a request. @@ -1954,6 +1966,96 @@ def flush_spans(): provider.force_flush(timeout_millis=timeout_millis) +def _resolve_runtime_entrypoint(startup_details: StartupDetails, log: Logger) -> Callable[[], None] | None: + """ + Check provider-registered runtime coordinators for a runtime-specific entrypoint. + + Resolution order: + + 1. **Queue mapping** -- the ``[sdk] queue_to_sdk`` config maps + the task's ``queue`` to a runtime coordinator name (e.g. ``"java-queue" -> "java"``). + Used by the python-stub pattern where users set ``queue="java-queue"`` explicitly. + 2. **DAG file extension** -- if no queue mapping matches, the DAG file's extension + (e.g. ``.jar``) is compared against each coordinator's ``file_extension`` attribute. + Used by the pure-Java (or pure-) pattern where the entire DAG is authored + in a non-Python language. + + Returns a no-arg callable that bridges fd 0 to the runtime subprocess, + or ``None`` to fall through to the standard Python execution path. + """ + import functools + + from airflow.sdk.execution_time.coordinator import QueueToCoordinatorMapper + from airflow.sdk.providers_manager_runtime import ProvidersManagerTaskRuntime + + coordinators = ProvidersManagerTaskRuntime().coordinators + + # Step 1: queue-to-runtime mapping. + queue = startup_details.ti.queue + if (sdk := QueueToCoordinatorMapper.from_config().resolve(queue)) is not None: + for coordinator_cls in coordinators: + if not hasattr(coordinator_cls, "run_task_execution"): + continue + if getattr(coordinator_cls, "sdk", None) != sdk: + continue + + log.debug( + "Resolved sdk-specific entrypoint for task via queue mapping", + coordinator=coordinator_cls, + sdk=sdk, + queue=queue, + task_id=startup_details.ti.task_id, + ) + return functools.partial( + coordinator_cls.run_task_execution, + what=startup_details.ti, + dag_rel_path=startup_details.dag_rel_path, + bundle_info=startup_details.bundle_info, + startup_details=startup_details, + ) + + log.warning( + "No coordinator found for sdk", + sdk=sdk, + queue=queue, + task_id=startup_details.ti.task_id, + ) + return None + + # Step 2: DAG file extension fallback (pure- DAGs). + dag_rel_path = startup_details.dag_rel_path + for coordinator_cls in coordinators: + # TODO: Use `can_handle_dag_file` method instead of file_extension attribute for better maintainability. + ext = getattr(coordinator_cls, "file_extension", None) + if not ext or not dag_rel_path.endswith(ext): + continue + if not hasattr(coordinator_cls, "run_task_execution"): + continue + + log.debug( + "Resolved runtime-specific entrypoint for task via DAG file extension", + coordinator=coordinator_cls, + sdk=getattr(coordinator_cls, "sdk", None), + dag_rel_path=dag_rel_path, + task_id=startup_details.ti.task_id, + ) + return functools.partial( + coordinator_cls.run_task_execution, + what=startup_details.ti, + dag_rel_path=startup_details.dag_rel_path, + bundle_info=startup_details.bundle_info, + startup_details=startup_details, + ) + + log.debug( + "No runtime coordinator matched, using standard Python execution path", + queue=queue, + dag_rel_path=dag_rel_path, + task_id=startup_details.ti.task_id, + ) + return None + + @flush_spans() def main(): log = structlog.get_logger(logger_name="task") @@ -1980,6 +2082,14 @@ def main(): # startup message as a ResendLoggingFD response. if os.environ.pop("_AIRFLOW_FORK_EXEC", None) == "1": reinit_supervisor_comms() + # Check if a provider-registered runtime coordinator should + # handle this task (e.g. Java, Go) instead of the standard + # Python execution path. + log.debug("Checking for runtime-specific entrypoint") + runtime_entrypoint = _resolve_runtime_entrypoint(startup_details, log) + if runtime_entrypoint is not None: + runtime_entrypoint() + return span = _make_task_span(msg=startup_details) stack.enter_context(span) ti, context, log = startup(msg=startup_details) diff --git a/task-sdk/src/airflow/sdk/providers_manager_runtime.py b/task-sdk/src/airflow/sdk/providers_manager_runtime.py index e28ed3fe14a83..63c8c97f816ef 100644 --- a/task-sdk/src/airflow/sdk/providers_manager_runtime.py +++ b/task-sdk/src/airflow/sdk/providers_manager_runtime.py @@ -51,6 +51,7 @@ from airflow.sdk import BaseHook from airflow.sdk.bases.decorator import TaskDecorator from airflow.sdk.definitions.asset import Asset + from airflow.sdk.execution_time.coordinator import BaseCoordinator log = structlog.getLogger(__name__) @@ -150,6 +151,7 @@ def __init__(self): # Keeps dict of hooks keyed by connection type. They are lazy evaluated at access time self._hooks_lazy_dict: LazyDictWithCache[str, HookInfo | Callable] = LazyDictWithCache() self._plugins_set: set[PluginInfo] = set() + self._coordinators: list[type[BaseCoordinator]] = [] self._provider_schema_validator = _create_provider_info_schema_validator() self._init_airflow_core_hooks() # Populated by initialize_provider_configs(); holds provider-contributed config sections. @@ -220,6 +222,12 @@ def initialize_providers_taskflow_decorator(self): self.initialize_providers_list() self._discover_taskflow_decorators() + @provider_info_cache("coordinators") + def initialize_providers_coordinators(self): + """Lazy initialization of providers runtime coordinators.""" + self.initialize_providers_list() + self._discover_coordinators() + @provider_info_cache("provider_configs") def initialize_provider_configs(self): """Lazy initialization of provider configuration metadata and merge it into SDK ``conf``.""" @@ -464,6 +472,19 @@ def _import_hook( connection_testable=hasattr(hook_class, "test_connection"), ) + def _discover_coordinators(self) -> None: + """Retrieve and pre-load all coordinators defined in the providers.""" + seen: set[str] = set() + for provider_package, provider in self._provider_dict.items(): + for coordinator_class_path in provider.data.get("coordinators", []): + if coordinator_class_path in seen: + continue + coordinator_cls = _correctness_check(provider_package, coordinator_class_path, provider) + if coordinator_cls: + seen.add(coordinator_class_path) + self._coordinators.append(coordinator_cls) + self._coordinators = sorted(self._coordinators, key=lambda c: c.__qualname__) + def _discover_filesystems(self) -> None: """Retrieve all filesystems defined in the providers.""" for provider_package, provider in self._provider_dict.items(): @@ -611,6 +632,12 @@ def plugins(self) -> list[PluginInfo]: self.initialize_providers_plugins() return sorted(self._plugins_set, key=lambda x: x.plugin_class) + @property + def coordinators(self) -> list[type[BaseCoordinator]]: + """Returns pre-loaded runtime coordinator classes available in providers.""" + self.initialize_providers_coordinators() + return self._coordinators + @property def provider_configs(self) -> list[tuple[str, dict[str, Any]]]: self.initialize_provider_configs() @@ -643,6 +670,7 @@ def _cleanup(self): self._asset_uri_handlers.clear() self._asset_factories.clear() self._asset_to_openlineage_converters.clear() + self._coordinators.clear() self._provider_configs.clear() # Imported lazily to preserve SDK conf lazy initialization and avoid a configuration/runtime cycle. diff --git a/task-sdk/tests/task_sdk/execution_time/test_coordinator.py b/task-sdk/tests/task_sdk/execution_time/test_coordinator.py new file mode 100644 index 0000000000000..783b46afad23e --- /dev/null +++ b/task-sdk/tests/task_sdk/execution_time/test_coordinator.py @@ -0,0 +1,586 @@ +# +# 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 +import socket +import subprocess +from unittest.mock import MagicMock, patch + +import pytest + +from airflow.sdk.execution_time.coordinator import ( + BaseCoordinator, + _bridge, + _send_startup_details, + _start_server, +) + + +class TestStartServer: + def test_binds_to_localhost(self): + server = _start_server() + try: + host, port = server.getsockname() + assert host == "127.0.0.1" + assert port > 0 + finally: + server.close() + + def test_assigns_random_port(self): + s1 = _start_server() + s2 = _start_server() + try: + _, port1 = s1.getsockname() + _, port2 = s2.getsockname() + # Two servers should get different ports + assert port1 != port2 + finally: + s1.close() + s2.close() + + def test_accepts_connection(self): + server = _start_server() + try: + addr = server.getsockname() + client = socket.socket() + client.connect(addr) + conn, _ = server.accept() + conn.sendall(b"ping") + assert client.recv(4) == b"ping" + conn.close() + client.close() + finally: + server.close() + + +class TestSendStartupDetails: + def test_sends_frame_bytes_to_socket(self): + """Verify _send_startup_details calls sendall with a length-prefixed msgpack frame.""" + mock_startup = MagicMock() + mock_startup.model_dump.return_value = {"type": "StartupDetails", "ti": {}} + + mock_socket = MagicMock(spec=socket.socket) + + _send_startup_details(mock_socket, mock_startup) + + mock_startup.model_dump.assert_called_once_with(mode="json") + mock_socket.sendall.assert_called_once() + + sent_bytes = mock_socket.sendall.call_args[0][0] + # First 4 bytes are the big-endian length prefix + assert len(sent_bytes) > 4 + length = int.from_bytes(sent_bytes[:4], "big") + assert length == len(sent_bytes) - 4 + + def test_frame_contains_response_id_zero(self): + """The frame should use id=0.""" + import msgpack + + mock_startup = MagicMock() + mock_startup.model_dump.return_value = {"type": "StartupDetails"} + + mock_socket = MagicMock(spec=socket.socket) + + _send_startup_details(mock_socket, mock_startup) + + sent_bytes = mock_socket.sendall.call_args[0][0] + # Frame is encoded as [id, body, error] + frame = msgpack.unpackb(sent_bytes[4:]) + assert frame[0] == 0 + + def test_frame_body_matches_model_dump(self): + """The frame body should be the model_dump(mode='json') output.""" + import msgpack + + body = {"type": "StartupDetails", "ti": {"task_id": "t1"}, "dag_rel_path": "test.jar"} + mock_startup = MagicMock() + mock_startup.model_dump.return_value = body + + mock_socket = MagicMock(spec=socket.socket) + + _send_startup_details(mock_socket, mock_startup) + + sent_bytes = mock_socket.sendall.call_args[0][0] + # Frame is encoded as [id, body, error] + frame = msgpack.unpackb(sent_bytes[4:]) + assert frame[1] == body + + def test_real_socket_roundtrip(self): + """Send through real sockets and verify the frame is receivable.""" + import msgpack + + server = socket.socket() + server.bind(("127.0.0.1", 0)) + server.listen(1) + addr = server.getsockname() + + client = socket.socket() + client.connect(addr) + conn, _ = server.accept() + + try: + body = {"type": "StartupDetails", "value": 42} + mock_startup = MagicMock() + mock_startup.model_dump.return_value = body + + _send_startup_details(conn, mock_startup) + + # Read the length prefix + length_bytes = client.recv(4) + length = int.from_bytes(length_bytes, "big") + + # Read the payload — frame is [id, body, error] + data = client.recv(length) + frame = msgpack.unpackb(data) + assert frame[0] == 0 + assert frame[1] == body + finally: + conn.close() + client.close() + server.close() + + +class TestBaseCoordinatorDefaults: + def test_can_handle_dag_file_returns_false(self): + assert BaseCoordinator.can_handle_dag_file("bundle", "/path/to/dag.py") is False + + def test_get_code_from_file_raises_not_implemented(self): + with pytest.raises(NotImplementedError): + BaseCoordinator.get_code_from_file("/path/to/dag.jar") + + def test_dag_parsing_cmd_raises_not_implemented(self): + with pytest.raises(NotImplementedError): + BaseCoordinator.dag_parsing_cmd( + dag_file_path="/dag.jar", + bundle_name="b", + bundle_path="/path", + comm_addr="127.0.0.1:1234", + logs_addr="127.0.0.1:1235", + ) + + def test_task_execution_cmd_raises_not_implemented(self): + with pytest.raises(NotImplementedError): + BaseCoordinator.task_execution_cmd( + what=MagicMock(), + dag_file_path="/dag.jar", + bundle_path="/path", + bundle_info=MagicMock(), + comm_addr="127.0.0.1:1234", + logs_addr="127.0.0.1:1235", + ) + + +class TestCoordinatorNamedTuples: + def test_dag_parsing_info_defaults(self): + info = BaseCoordinator.DagParsingInfo( + dag_file_path="/dag.jar", + bundle_name="my-bundle", + bundle_path="/bundles/my-bundle", + ) + assert info.mode == "dag-parsing" + assert info.dag_file_path == "/dag.jar" + assert info.bundle_name == "my-bundle" + assert info.bundle_path == "/bundles/my-bundle" + + def test_task_execution_info_defaults(self): + mock_ti = MagicMock() + mock_bundle = MagicMock() + mock_startup = MagicMock() + info = BaseCoordinator.TaskExecutionInfo( + what=mock_ti, + dag_rel_path="dags/example.jar", + bundle_info=mock_bundle, + startup_details=mock_startup, + ) + assert info.mode == "task-execution" + assert info.what is mock_ti + assert info.dag_rel_path == "dags/example.jar" + + +class TestBridge: + def test_bridge_forwards_comm_bidirectionally(self): + """Verify _bridge sets up bidirectional forwarding and processes all channels.""" + # Use real socketpairs for the 4 channels + sup_send, sup_recv = socket.socketpair() + rt_send, rt_recv = socket.socketpair() + log_send, log_recv = socket.socketpair() + stderr_send, stderr_recv = socket.socketpair() + + mock_proc = MagicMock(spec=subprocess.Popen) + # Make the process "exit" immediately so the bridge drains and stops + mock_proc.poll.return_value = 0 + mock_log = MagicMock() + + try: + # Send data before starting the bridge + sup_send.sendall(b"from_supervisor") + rt_send.sendall(b"from_runtime") + log_send.sendall(b'{"event":"hello","level":"info"}\n') + stderr_send.sendall(b"stderr line\n") + + # Close sending sides so the bridge will see EOF + sup_send.close() + rt_send.close() + log_send.close() + stderr_send.close() + + _bridge(sup_recv, rt_recv, log_recv, stderr_recv, mock_proc, mock_log) + + # If we got here without hanging, the bridge correctly processed all channels + finally: + for s in (sup_send, rt_send, log_send, stderr_send, sup_recv, rt_recv, log_recv, stderr_recv): + try: + s.close() + except OSError: + pass + + def test_bridge_drains_after_process_exit(self): + """Verify _bridge drains remaining data after the subprocess exits.""" + sup_local, sup_remote = socket.socketpair() + rt_local, rt_remote = socket.socketpair() + log_local, log_remote = socket.socketpair() + stderr_local, stderr_remote = socket.socketpair() + + mock_proc = MagicMock(spec=subprocess.Popen) + # First poll: still running; subsequent: exited + mock_proc.poll.side_effect = [None, 0, 0, 0, 0, 0, 0, 0, 0, 0] + mock_log = MagicMock() + + try: + # Send data after bridge starts its first iteration + stderr_local.sendall(b"error output\n") + stderr_local.close() + sup_local.close() + rt_local.close() + log_local.close() + + _bridge(sup_remote, rt_remote, log_remote, stderr_remote, mock_proc, mock_log) + finally: + for s in ( + sup_local, + sup_remote, + rt_local, + rt_remote, + log_local, + log_remote, + stderr_local, + stderr_remote, + ): + try: + s.close() + except OSError: + pass + + def test_bridge_closes_all_sockets(self): + """Verify _bridge closes all four sockets when done.""" + sup = MagicMock(spec=socket.socket) + rt = MagicMock(spec=socket.socket) + logs = MagicMock(spec=socket.socket) + stderr = MagicMock(spec=socket.socket) + + mock_proc = MagicMock(spec=subprocess.Popen) + mock_proc.poll.return_value = 0 + mock_log = MagicMock() + + # Patch the selector to avoid real I/O; service_selector is imported inside + # _bridge so we patch it on the selector_loop module + with ( + patch("airflow.sdk.execution_time.coordinator.selectors.DefaultSelector") as mock_sel_cls, + patch("airflow.sdk.execution_time.selector_loop.service_selector"), + ): + mock_sel = MagicMock() + mock_sel_cls.return_value = mock_sel + # Empty selector map so the while loop exits immediately + mock_sel.get_map.return_value = {} + + _bridge(sup, rt, logs, stderr, mock_proc, mock_log) + + sup.close.assert_called() + rt.close.assert_called() + logs.close.assert_called() + stderr.close.assert_called() + mock_sel.close.assert_called_once() + + +class TestRunDagParsing: + @patch.object(BaseCoordinator, "_runtime_subprocess_entrypoint") + def test_run_dag_parsing_creates_dag_parsing_info(self, mock_entrypoint): + BaseCoordinator.run_dag_parsing( + path="/bundles/my-bundle/dags/example.jar", + bundle_name="my-bundle", + bundle_path="/bundles/my-bundle", + ) + + mock_entrypoint.assert_called_once() + info = mock_entrypoint.call_args[0][0] + assert isinstance(info, BaseCoordinator.DagParsingInfo) + assert info.dag_file_path == "/bundles/my-bundle/dags/example.jar" + assert info.bundle_name == "my-bundle" + assert info.bundle_path == "/bundles/my-bundle" + assert info.mode == "dag-parsing" + + +class TestRunTaskExecution: + @patch.object(BaseCoordinator, "_runtime_subprocess_entrypoint") + def test_run_task_execution_creates_task_execution_info(self, mock_entrypoint): + mock_ti = MagicMock() + mock_bundle_info = MagicMock() + mock_startup = MagicMock() + + BaseCoordinator.run_task_execution( + what=mock_ti, + dag_rel_path="dags/example.jar", + bundle_info=mock_bundle_info, + startup_details=mock_startup, + ) + + mock_entrypoint.assert_called_once() + info = mock_entrypoint.call_args[0][0] + assert isinstance(info, BaseCoordinator.TaskExecutionInfo) + assert info.what is mock_ti + assert info.dag_rel_path == "dags/example.jar" + assert info.bundle_info is mock_bundle_info + assert info.startup_details is mock_startup + assert info.mode == "task-execution" + + +class TestRuntimeSubprocessEntrypoint: + def test_unknown_entrypoint_info_type_raises(self): + class TestCoordinator(BaseCoordinator): + sdk = "test" + file_extension = ".test" + + # Needs a 'mode' attribute (accessed during logging) but must not be + # an instance of DagParsingInfo or TaskExecutionInfo. + fake_info = MagicMock() + fake_info.mode = "unknown" + + with pytest.raises(ValueError, match="Unknown entrypoint_info type"): + TestCoordinator._runtime_subprocess_entrypoint(fake_info) # type: ignore[arg-type] + + @patch("airflow.sdk.execution_time.coordinator._bridge") + @patch("airflow.sdk.execution_time.coordinator._send_startup_details") + @patch("subprocess.Popen", autospec=True) + @patch("airflow.sdk.execution_time.coordinator._start_server") + @patch("os.dup", return_value=99) + def test_dag_parsing_flow(self, mock_dup, mock_start_server, mock_popen, mock_send_startup, mock_bridge): + """Verify the dag-parsing entrypoint wires up servers, spawns subprocess, and bridges.""" + # Set up mock servers + comm_server = MagicMock(spec=socket.socket) + comm_server.getsockname.return_value = ("127.0.0.1", 5000) + logs_server = MagicMock(spec=socket.socket) + logs_server.getsockname.return_value = ("127.0.0.1", 5001) + mock_start_server.side_effect = [comm_server, logs_server] + + # The runtime connects back + runtime_comm = MagicMock(spec=socket.socket) + runtime_logs = MagicMock(spec=socket.socket) + comm_server.accept.return_value = (runtime_comm, ("127.0.0.1", 9000)) + logs_server.accept.return_value = (runtime_logs, ("127.0.0.1", 9001)) + + # Mock socketpair for stderr + child_stderr = MagicMock(spec=socket.socket) + read_stderr = MagicMock(spec=socket.socket) + child_stderr.fileno.return_value = 10 + + # Mock supervisor_comm created from os.dup(0) + supervisor_comm = MagicMock(spec=socket.socket) + + class TestCoordinator(BaseCoordinator): + sdk = "test" + file_extension = ".test" + + @classmethod + def dag_parsing_cmd(cls, **kwargs): + return ["test-runtime", "--parse", kwargs["dag_file_path"]] + + info = BaseCoordinator.DagParsingInfo( + dag_file_path="/dag.test", + bundle_name="test-bundle", + bundle_path="/bundles/test-bundle", + ) + + with ( + patch("socket.socketpair", return_value=(child_stderr, read_stderr)), + patch("airflow.sdk.execution_time.coordinator.socket.socket", return_value=supervisor_comm), + ): + TestCoordinator._runtime_subprocess_entrypoint(info) + + # Subprocess spawned + mock_popen.assert_called_once() + cmd = mock_popen.call_args[0][0] + assert cmd == ["test-runtime", "--parse", "/dag.test"] + + # Servers accepted and closed + comm_server.accept.assert_called_once() + logs_server.accept.assert_called_once() + comm_server.close.assert_called_once() + logs_server.close.assert_called_once() + + # stderr child side closed after Popen + child_stderr.close.assert_called_once() + + # _send_startup_details NOT called for dag parsing + mock_send_startup.assert_not_called() + + # _bridge called with the supervisor_comm socket + mock_bridge.assert_called_once() + assert mock_bridge.call_args[0][0] is supervisor_comm + + @patch("airflow.sdk.execution_time.coordinator._bridge") + @patch("airflow.sdk.execution_time.coordinator._send_startup_details") + @patch("subprocess.Popen", autospec=True) + @patch("airflow.sdk.execution_time.coordinator._start_server") + @patch("os.dup", return_value=99) + @patch("airflow.sdk.execution_time.task_runner.resolve_bundle") + @patch("airflow.dag_processing.bundles.base.BundleVersionLock", autospec=True) + def test_task_execution_flow( + self, + mock_bundle_lock, + mock_resolve_bundle, + mock_dup, + mock_start_server, + mock_popen, + mock_send_startup, + mock_bridge, + ): + """Verify the task-execution entrypoint resolves bundle, sends startup details, and bridges.""" + # Mock servers + comm_server = MagicMock(spec=socket.socket) + comm_server.getsockname.return_value = ("127.0.0.1", 6000) + logs_server = MagicMock(spec=socket.socket) + logs_server.getsockname.return_value = ("127.0.0.1", 6001) + mock_start_server.side_effect = [comm_server, logs_server] + + runtime_comm = MagicMock(spec=socket.socket) + runtime_logs = MagicMock(spec=socket.socket) + comm_server.accept.return_value = (runtime_comm, ("127.0.0.1", 9000)) + logs_server.accept.return_value = (runtime_logs, ("127.0.0.1", 9001)) + + child_stderr = MagicMock(spec=socket.socket) + read_stderr = MagicMock(spec=socket.socket) + child_stderr.fileno.return_value = 10 + + # Mock resolved bundle + mock_bundle_instance = MagicMock() + mock_bundle_instance.path = "/resolved/bundles/test-bundle" + mock_resolve_bundle.return_value = mock_bundle_instance + + # BundleVersionLock as context manager + mock_lock_instance = MagicMock() + mock_bundle_lock.return_value = mock_lock_instance + mock_lock_instance.__enter__ = MagicMock(return_value=mock_lock_instance) + mock_lock_instance.__exit__ = MagicMock(return_value=False) + + mock_ti = MagicMock() + mock_bundle_info = MagicMock() + mock_bundle_info.name = "test-bundle" + mock_bundle_info.version = "v1" + mock_startup = MagicMock() + + class TestCoordinator(BaseCoordinator): + sdk = "test" + file_extension = ".test" + + @classmethod + def task_execution_cmd(cls, **kwargs): + return ["test-runtime", "--execute", kwargs["dag_file_path"]] + + info = BaseCoordinator.TaskExecutionInfo( + what=mock_ti, + dag_rel_path="dags/example.test", + bundle_info=mock_bundle_info, + startup_details=mock_startup, + ) + + supervisor_comm = MagicMock(spec=socket.socket) + + with ( + patch("socket.socketpair", return_value=(child_stderr, read_stderr)), + patch("airflow.sdk.execution_time.coordinator.socket.socket", return_value=supervisor_comm), + ): + TestCoordinator._runtime_subprocess_entrypoint(info) + + # Bundle resolved + mock_resolve_bundle.assert_called_once() + + # BundleVersionLock used + mock_bundle_lock.assert_called_once_with(bundle_name="test-bundle", bundle_version="v1") + + # Subprocess spawned with resolved path + mock_popen.assert_called_once() + cmd = mock_popen.call_args[0][0] + assert cmd == ["test-runtime", "--execute", "/resolved/bundles/test-bundle/dags/example.test"] + + # StartupDetails forwarded to the runtime subprocess + mock_send_startup.assert_called_once_with(runtime_comm, mock_startup) + + # _bridge called + mock_bridge.assert_called_once() + + @patch("airflow.sdk.execution_time.coordinator._bridge") + @patch("subprocess.Popen", autospec=True) + @patch("airflow.sdk.execution_time.coordinator._start_server") + @patch("os.dup", return_value=99) + def test_sets_process_context_env_var(self, mock_dup, mock_start_server, mock_popen, mock_bridge): + """Verify _AIRFLOW_PROCESS_CONTEXT is set to 'client'.""" + comm_server = MagicMock(spec=socket.socket) + comm_server.getsockname.return_value = ("127.0.0.1", 7000) + logs_server = MagicMock(spec=socket.socket) + logs_server.getsockname.return_value = ("127.0.0.1", 7001) + mock_start_server.side_effect = [comm_server, logs_server] + + runtime_comm = MagicMock(spec=socket.socket) + runtime_logs = MagicMock(spec=socket.socket) + comm_server.accept.return_value = (runtime_comm, ("127.0.0.1", 9000)) + logs_server.accept.return_value = (runtime_logs, ("127.0.0.1", 9001)) + + child_stderr = MagicMock(spec=socket.socket) + read_stderr = MagicMock(spec=socket.socket) + child_stderr.fileno.return_value = 10 + + class TestCoordinator(BaseCoordinator): + sdk = "test" + file_extension = ".test" + + @classmethod + def dag_parsing_cmd(cls, **kwargs): + return ["echo", "test"] + + info = BaseCoordinator.DagParsingInfo( + dag_file_path="/dag.test", + bundle_name="b", + bundle_path="/path", + ) + + supervisor_comm = MagicMock(spec=socket.socket) + + old_val = os.environ.get("_AIRFLOW_PROCESS_CONTEXT") + try: + with ( + patch("socket.socketpair", return_value=(child_stderr, read_stderr)), + patch("airflow.sdk.execution_time.coordinator.socket.socket", return_value=supervisor_comm), + ): + TestCoordinator._runtime_subprocess_entrypoint(info) + assert os.environ["_AIRFLOW_PROCESS_CONTEXT"] == "client" + finally: + if old_val is None: + os.environ.pop("_AIRFLOW_PROCESS_CONTEXT", None) + else: + os.environ["_AIRFLOW_PROCESS_CONTEXT"] = old_val diff --git a/task-sdk/tests/task_sdk/execution_time/test_selector_loop.py b/task-sdk/tests/task_sdk/execution_time/test_selector_loop.py new file mode 100644 index 0000000000000..efbfa83adecf8 --- /dev/null +++ b/task-sdk/tests/task_sdk/execution_time/test_selector_loop.py @@ -0,0 +1,479 @@ +# +# 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 selectors +import socket +from unittest.mock import MagicMock + +import pytest + +from airflow.sdk.execution_time.selector_loop import ( + make_buffered_socket_reader, + make_raw_forwarder, + service_selector, +) + + +def _make_generator(): + """Return a generator that collects sent lines into a list.""" + received: list[bytes | bytearray] = [] + + def gen(): + while True: + line = yield + received.append(bytes(line)) + + g = gen() + return g, received + + +def _make_socket_pair(): + """Create a connected TCP socket pair on localhost.""" + server = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + server.bind(("127.0.0.1", 0)) + server.listen(1) + addr = server.getsockname() + + client = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + client.connect(addr) + conn, _ = server.accept() + server.close() + return client, conn + + +class TestMakeBufferedSocketReader: + def test_single_complete_line(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, returned_on_close = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + # recv_into writes data and returns count + data = b"hello world\n" + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, data) + + result = handler(sock) + + assert result is True + assert received == [b"hello world\n"] + assert returned_on_close is on_close + + def test_multiple_lines_in_single_recv(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + data = b"line1\nline2\nline3\n" + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, data) + + result = handler(sock) + + assert result is True + assert received == [b"line1\n", b"line2\n", b"line3\n"] + + def test_partial_line_accumulated_across_calls(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + + # First call: partial line (no newline) + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"hell") + result = handler(sock) + assert result is True + assert received == [] + + # Second call: rest of the line + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"o\n") + result = handler(sock) + assert result is True + assert received == [b"hello\n"] + + def test_eof_flushes_remaining_buffer(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + + # Send partial data (no newline) + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"leftover") + handler(sock) + assert received == [] + + # EOF (recv_into returns 0) — clear side_effect so return_value takes effect + sock.recv_into.side_effect = None + sock.recv_into.return_value = 0 + result = handler(sock) + + assert result is False + assert received == [b"leftover"] + + def test_eof_with_empty_buffer(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + sock.recv_into.return_value = 0 + + result = handler(sock) + + assert result is False + assert received == [] + + def test_generator_stop_iteration_returns_false(self): + """If the generator is exhausted, handler returns False.""" + + def limited_gen(): + yield # startup + yield # receive one line, then stop + + gen = limited_gen() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + # First line succeeds + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"line1\n") + result = handler(sock) + assert result is True + + # Second line triggers StopIteration in the generator + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"line2\n") + result = handler(sock) + assert result is False + + def test_mixed_complete_and_partial_lines(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close) + + sock = MagicMock(spec=socket.socket) + # Data contains one complete line and a partial line + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"complete\npart") + handler(sock) + assert received == [b"complete\n"] + + # Finish the partial line + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, b"ial\n") + handler(sock) + assert received == [b"complete\n", b"partial\n"] + + def test_custom_buffer_size(self): + gen, received = _make_generator() + on_close = MagicMock() + handler, _ = make_buffered_socket_reader(gen, on_close, buffer_size=8) + + sock = MagicMock(spec=socket.socket) + # Data larger than buffer_size — recv_into only reads buffer_size bytes + full_data = b"abcdefghijklmnop\n" + # Simulate chunked reads + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, full_data[: len(buf)]) + handler(sock) + # Only first 8 bytes read, no newline yet + assert received == [] + + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, full_data[8:16]) + handler(sock) + assert received == [] + + sock.recv_into.side_effect = lambda buf: _fill_buffer(buf, full_data[16:]) + handler(sock) + assert received == [b"abcdefghijklmnop\n"] + + +def _fill_buffer(buf: bytearray, data: bytes) -> int: + """Helper to simulate socket.recv_into by filling the buffer.""" + n = min(len(data), len(buf)) + buf[:n] = data[:n] + return n + + +class TestMakeRawForwarder: + def test_forwards_data_to_dest(self): + on_close = MagicMock() + dest = MagicMock(spec=socket.socket) + handler, returned_on_close = make_raw_forwarder(dest, on_close) + + src = MagicMock(spec=socket.socket) + src.recv.return_value = b"hello" + + result = handler(src) + + assert result is True + dest.sendall.assert_called_once_with(b"hello") + assert returned_on_close is on_close + + def test_eof_returns_false(self): + on_close = MagicMock() + dest = MagicMock(spec=socket.socket) + handler, _ = make_raw_forwarder(dest, on_close) + + src = MagicMock(spec=socket.socket) + src.recv.return_value = b"" + + result = handler(src) + + assert result is False + dest.sendall.assert_not_called() + + @pytest.mark.parametrize( + "exception", + [BrokenPipeError, ConnectionResetError, OSError], + ids=["broken_pipe", "connection_reset", "os_error"], + ) + def test_sendall_exception_returns_false(self, exception): + on_close = MagicMock() + dest = MagicMock(spec=socket.socket) + dest.sendall.side_effect = exception + handler, _ = make_raw_forwarder(dest, on_close) + + src = MagicMock(spec=socket.socket) + src.recv.return_value = b"data" + + result = handler(src) + + assert result is False + + def test_multiple_forwards(self): + on_close = MagicMock() + dest = MagicMock(spec=socket.socket) + handler, _ = make_raw_forwarder(dest, on_close) + + src = MagicMock(spec=socket.socket) + + for chunk in [b"chunk1", b"chunk2", b"chunk3"]: + src.recv.return_value = chunk + assert handler(src) is True + + assert dest.sendall.call_count == 3 + + +class TestServiceSelector: + def test_calls_handler_for_ready_sockets(self): + sel = MagicMock(spec=selectors.DefaultSelector) + handler = MagicMock(return_value=True) + on_close = MagicMock() + sock = MagicMock(spec=socket.socket) + + key = MagicMock() + key.data = (handler, on_close) + key.fileobj = sock + + sel.select.return_value = [(key, selectors.EVENT_READ)] + + service_selector(sel, timeout=1.0) + + handler.assert_called_once_with(sock) + on_close.assert_not_called() + sock.close.assert_not_called() + + def test_on_close_and_sock_close_when_handler_returns_false(self): + sel = MagicMock(spec=selectors.DefaultSelector) + handler = MagicMock(return_value=False) + on_close = MagicMock() + sock = MagicMock(spec=socket.socket) + + key = MagicMock() + key.data = (handler, on_close) + key.fileobj = sock + + sel.select.return_value = [(key, selectors.EVENT_READ)] + + service_selector(sel, timeout=1.0) + + handler.assert_called_once_with(sock) + on_close.assert_called_once_with(sock) + sock.close.assert_called_once() + + @pytest.mark.parametrize( + "exception", + [BrokenPipeError, ConnectionResetError], + ids=["broken_pipe", "connection_reset"], + ) + def test_pipe_errors_treated_as_eof(self, exception): + sel = MagicMock(spec=selectors.DefaultSelector) + handler = MagicMock(side_effect=exception) + on_close = MagicMock() + sock = MagicMock(spec=socket.socket) + + key = MagicMock() + key.data = (handler, on_close) + key.fileobj = sock + + sel.select.return_value = [(key, selectors.EVENT_READ)] + + service_selector(sel, timeout=1.0) + + on_close.assert_called_once_with(sock) + sock.close.assert_called_once() + + def test_empty_selector_no_events(self): + sel = MagicMock(spec=selectors.DefaultSelector) + sel.select.return_value = [] + + # Should not raise + service_selector(sel, timeout=1.0) + + @pytest.mark.parametrize( + ("input_timeout", "expected_min"), + [ + (0.0, 0.01), + (-1.0, 0.01), + (-100.0, 0.01), + (0.5, 0.5), + (2.0, 2.0), + ], + ids=["zero", "negative", "very_negative", "positive_half", "positive_two"], + ) + def test_timeout_clamped_to_minimum(self, input_timeout, expected_min): + sel = MagicMock(spec=selectors.DefaultSelector) + sel.select.return_value = [] + + service_selector(sel, timeout=input_timeout) + + sel.select.assert_called_once() + actual_timeout = sel.select.call_args[1].get("timeout") or sel.select.call_args[0][0] + assert actual_timeout == pytest.approx(expected_min) + + def test_multiple_ready_sockets(self): + sel = MagicMock(spec=selectors.DefaultSelector) + + handler1 = MagicMock(return_value=True) + on_close1 = MagicMock() + sock1 = MagicMock(spec=socket.socket) + key1 = MagicMock() + key1.data = (handler1, on_close1) + key1.fileobj = sock1 + + handler2 = MagicMock(return_value=False) + on_close2 = MagicMock() + sock2 = MagicMock(spec=socket.socket) + key2 = MagicMock() + key2.data = (handler2, on_close2) + key2.fileobj = sock2 + + sel.select.return_value = [(key1, selectors.EVENT_READ), (key2, selectors.EVENT_READ)] + + service_selector(sel, timeout=1.0) + + # First socket: handler returns True, stays open + handler1.assert_called_once_with(sock1) + on_close1.assert_not_called() + sock1.close.assert_not_called() + + # Second socket: handler returns False, closed + handler2.assert_called_once_with(sock2) + on_close2.assert_called_once_with(sock2) + sock2.close.assert_called_once() + + +class TestSelectorLoopIntegration: + def test_buffered_reader_with_real_sockets(self): + """End-to-end: send lines through real sockets and verify buffered reading.""" + gen, received = _make_generator() + sender, reader = _make_socket_pair() + try: + sel = selectors.DefaultSelector() + + def on_close(sock): + sel.unregister(sock) + + sel.register(reader, selectors.EVENT_READ, make_buffered_socket_reader(gen, on_close)) + + sender.sendall(b"first line\nsecond line\n") + + service_selector(sel, timeout=1.0) + + assert b"first line\n" in received + assert b"second line\n" in received + + # Close sender, then drain + sender.close() + sender = None + + service_selector(sel, timeout=0.5) + + sel.close() + finally: + if sender: + sender.close() + reader.close() + + def test_raw_forwarder_with_real_sockets(self): + """End-to-end: forward raw bytes between real socket pairs.""" + src_send, src_recv = _make_socket_pair() + # Use socketpair for the destination so reads/writes are symmetric + dst_write, dst_read = socket.socketpair() + try: + sel = selectors.DefaultSelector() + + def on_close(sock): + sel.unregister(sock) + + sel.register(src_recv, selectors.EVENT_READ, make_raw_forwarder(dst_write, on_close)) + + src_send.sendall(b"raw data payload") + + service_selector(sel, timeout=1.0) + + dst_read.setblocking(False) + forwarded = dst_read.recv(4096) + + assert forwarded == b"raw data payload" + + sel.close() + finally: + for s in (src_send, src_recv, dst_write, dst_read): + s.close() + + def test_eof_triggers_on_close_with_real_sockets(self): + """When the sender closes, the selector callback chain fires on_close.""" + gen, received = _make_generator() + sender, reader = _make_socket_pair() + closed_sockets: list[socket.socket] = [] + try: + sel = selectors.DefaultSelector() + + def on_close(sock): + sel.unregister(sock) + closed_sockets.append(sock) + + sel.register(reader, selectors.EVENT_READ, make_buffered_socket_reader(gen, on_close)) + + # Send data then close + sender.sendall(b"final\n") + service_selector(sel, timeout=1.0) + assert received == [b"final\n"] + + sender.close() + sender = None + service_selector(sel, timeout=0.5) + + # on_close should have been called, and socket closed by service_selector + assert len(closed_sockets) == 1 + + sel.close() + finally: + if sender: + sender.close() + reader.close() diff --git a/task-sdk/tests/task_sdk/execution_time/test_supervisor.py b/task-sdk/tests/task_sdk/execution_time/test_supervisor.py index f0d8e1a0b65d6..1c0b9602fc715 100644 --- a/task-sdk/tests/task_sdk/execution_time/test_supervisor.py +++ b/task-sdk/tests/task_sdk/execution_time/test_supervisor.py @@ -52,6 +52,7 @@ from task_sdk import FAKE_BUNDLE, make_client from uuid6 import uuid7 +from airflow._shared.workloads import TaskInstanceDTO from airflow.executors.workloads import BundleInfo from airflow.sdk import BaseOperator, timezone from airflow.sdk.api import client as sdk_client @@ -622,8 +623,16 @@ def subprocess_main(): proc = ActivitySubprocess.start( dag_rel_path=os.devnull, bundle_info=FAKE_BUNDLE, - what=TaskInstance( - id=ti_id, task_id="b", dag_id="c", run_id="d", try_number=1, dag_version_id=uuid7() + what=TaskInstanceDTO( + id=ti_id, + task_id="b", + dag_id="c", + run_id="d", + try_number=1, + dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), client=sdk_client.Client(base_url="", dry_run=True, token=""), target=subprocess_main, @@ -659,8 +668,16 @@ def _on_child_started(self, *args, **kwargs): proc = ActivitySubprocess.start( dag_rel_path=os.devnull, bundle_info=FAKE_BUNDLE, - what=TaskInstance( - id=ti_id, task_id="b", dag_id="c", run_id="d", try_number=1, dag_version_id=uuid7() + what=TaskInstanceDTO( + id=ti_id, + task_id="b", + dag_id="c", + run_id="d", + try_number=1, + dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), client=sdk_client.Client(base_url="", dry_run=True, token=""), target=subprocess_main, diff --git a/task-sdk/tests/task_sdk/execution_time/test_task_runner.py b/task-sdk/tests/task_sdk/execution_time/test_task_runner.py index 630aff9094ed1..7a9fd76811eec 100644 --- a/task-sdk/tests/task_sdk/execution_time/test_task_runner.py +++ b/task-sdk/tests/task_sdk/execution_time/test_task_runner.py @@ -55,6 +55,7 @@ timezone, ) from airflow.sdk._shared.observability.metrics.base_stats_logger import StatsLogger +from airflow.sdk._shared.workloads import TaskInstanceDTO from airflow.sdk.api.datamodels._generated import ( AssetProfile, AssetResponse, @@ -177,13 +178,16 @@ def execute(self, context): def test_parse(test_dags_dir: Path, make_ti_context): """Test that checks parsing of a basic dag with an un-mocked parse.""" what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="a", dag_id="super_basic", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="super_basic.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -224,13 +228,16 @@ def test_parse_dag_bag(mock_dagbag, test_dags_dir: Path, make_ti_context): mock_dag.task_dict = {"a": mock_task} what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="a", dag_id="super_basic", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="super_basic.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -284,13 +291,16 @@ def test_parse_dag_bag(mock_dagbag, test_dags_dir: Path, make_ti_context): def test_parse_not_found(test_dags_dir: Path, make_ti_context, dag_id, task_id, expected_error): """Check for nice error messages on dag not found.""" what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id=task_id, dag_id=dag_id, run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="super_basic.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -330,13 +340,16 @@ def test_parse_not_found_does_not_reschedule_when_max_attempts_reached(test_dags and should surface as a hard failure (SystemExit in the task runner process). """ what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="a", dag_id="madeup_dag_id", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="super_basic.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -391,13 +404,16 @@ def test_main_sends_reschedule_task_when_startup_reschedules( mock_comms_instance.socket = None mock_comms_decoder_cls.__getitem__.return_value.return_value = mock_comms_instance what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="my_task", dag_id="test_dag", run_id="test_run", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, context_carrier={}, ), dag_rel_path="", @@ -484,13 +500,16 @@ def test_task_span_is_child_of_dag_run_span(make_ti_context): # Step 3: build StartupDetails with ti.context_carrier = ti_carrier. what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="my_task", dag_id="test_dag", run_id="test_run", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, context_carrier=ti_carrier, ), dag_rel_path="", @@ -552,13 +571,16 @@ def test_task_span_no_parent_when_no_context_carrier(make_ti_context): provider.add_span_processor(SimpleSpanProcessor(in_mem_exporter)) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="standalone_task", dag_id="test_dag", run_id="test_run", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, context_carrier=None, ), dag_rel_path="", @@ -593,13 +615,16 @@ def test_parse_module_in_bundle_root(tmp_path: Path, make_ti_context): dag1_path.write_text(textwrap.dedent(dag1_code)) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="a", dag_id="dag_name", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="path_test.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -1040,13 +1065,16 @@ def test_basic_templated_dag(mocked_parse, make_ti_context, mock_supervisor_comm ) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="templated_task", dag_id="basic_templated_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), bundle_info=FAKE_BUNDLE, dag_rel_path="", @@ -1156,13 +1184,16 @@ def execute(self, context): instant = timezone.datetime(2024, 12, 3, 10, 0) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="templated_task", dag_id="basic_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=FAKE_BUNDLE, @@ -1204,13 +1235,16 @@ def execute(self, context): instant = timezone.datetime(2024, 12, 3, 10, 0) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="impersonation_task", dag_id="basic_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=FAKE_BUNDLE, @@ -1252,13 +1286,16 @@ def execute(self, context): instant = timezone.datetime(2024, 12, 3, 10, 0) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="impersonation_task", dag_id="basic_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=FAKE_BUNDLE, @@ -1292,13 +1329,16 @@ def execute(self, context): instant = timezone.datetime(2024, 12, 3, 10, 0) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="impersonation_task", dag_id="basic_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=FAKE_BUNDLE, @@ -1465,8 +1505,16 @@ def test_dag_parsing_context(make_ti_context, mock_supervisor_comms, monkeypatch task_id = "conditional_task" what = StartupDetails( - ti=TaskInstance( - id=uuid7(), task_id=task_id, dag_id=dag_id, run_id="c", try_number=1, dag_version_id=uuid7() + ti=TaskInstanceDTO( + id=uuid7(), + task_id=task_id, + dag_id=dag_id, + run_id="c", + try_number=1, + dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="dag_parsing_context.py", bundle_info=BundleInfo(name="my-bundle", version=None), @@ -3883,13 +3931,16 @@ def execute(self, context): task_id="test_task_runner_calls_listeners", do_xcom_push=True, multiple_outputs=True ) what = StartupDetails( - ti=TaskInstance( + ti=TaskInstanceDTO( id=uuid7(), task_id="templated_task", dag_id="basic_dag", run_id="c", try_number=1, dag_version_id=uuid7(), + pool_slots=1, + queue="default", + priority_weight=1, ), dag_rel_path="", bundle_info=FAKE_BUNDLE, diff --git a/task-sdk/tests/task_sdk/test_providers_manager_runtime.py b/task-sdk/tests/task_sdk/test_providers_manager_runtime.py index 1cae21d53c764..6e775f790be89 100644 --- a/task-sdk/tests/task_sdk/test_providers_manager_runtime.py +++ b/task-sdk/tests/task_sdk/test_providers_manager_runtime.py @@ -243,6 +243,33 @@ def test_already_initialized_provider_configs_emits_deprecation_warning(self): with pytest.warns(DeprecationWarning, match="already_initialized_provider_configs.*deprecated"): pm.already_initialized_provider_configs + @patch("airflow.sdk.providers_manager_runtime.import_string") + def test_coordinators(self, mock_import_string): + class ACoordinator: + pass + + class ZCoordinator: + pass + + mock_import_string.side_effect = lambda path: { + "airflow.providers.sdk.java.coordinator.ACoordinator": ACoordinator, + "airflow.providers.sdk.java.coordinator.ZCoordinator": ZCoordinator, + }[path] + providers_manager = ProvidersManagerTaskRuntime() + providers_manager._provider_dict["apache-airflow-providers-sdk-java"] = ProviderInfo( + version="0.0.1", + data={ + "coordinators": [ + "airflow.providers.sdk.java.coordinator.ZCoordinator", + "airflow.providers.sdk.java.coordinator.ACoordinator", + "airflow.providers.sdk.java.coordinator.ZCoordinator", + ] + }, + ) + + with patch.object(providers_manager, "initialize_providers_list"): + assert providers_manager.coordinators == [ACoordinator, ZCoordinator] + def test_initialize_provider_configs_can_reload_sdk_conf(self): from airflow.sdk.configuration import conf diff --git a/uv.lock b/uv.lock index e0084ec1cb511..1c94d9ad53d91 100644 --- a/uv.lock +++ b/uv.lock @@ -80,8 +80,8 @@ apache-airflow-providers-salesforce = false apache-airflow-providers-ssh = false apache-airflow-providers-papermill = false apache-airflow-providers-google = false -apache-airflow-providers-microsoft-psrp = false apache-airflow-providers-vertica = false +apache-airflow-providers-microsoft-psrp = false apache-airflow-providers-apache-hdfs = false apache-airflow-shared-template-rendering = false apache-airflow-mypy = false @@ -97,6 +97,7 @@ apache-airflow-shared-serialization = false apache-airflow-scripts = false apache-airflow-providers-exasol = false apache-airflow-providers-mongo = false +apache-airflow-shared-workloads = false apache-airflow-providers-apprise = false apache-airflow-providers-apache-impala = false apache-airflow-ctl = false @@ -147,11 +148,13 @@ apache-airflow-providers-atlassian-jira = false apache-airflow-providers-odbc = false apache-airflow-providers-postgres = false apache-airflow-providers-openai = false -apache-airflow-task-sdk-integration-tests = false +apache-airflow-providers-sdk-executable = false apache-airflow-providers-smtp = false +apache-airflow-task-sdk-integration-tests = false apache-airflow-providers-dingding = false apache-airflow-providers-apache-kylin = false apache-airflow-providers-cloudant = false +apache-airflow-providers-sdk-java = false apache-aurflow-docker-stack = false [manifest] @@ -249,6 +252,8 @@ members = [ "apache-airflow-providers-redis", "apache-airflow-providers-salesforce", "apache-airflow-providers-samba", + "apache-airflow-providers-sdk-executable", + "apache-airflow-providers-sdk-java", "apache-airflow-providers-segment", "apache-airflow-providers-sendgrid", "apache-airflow-providers-sftp", @@ -285,6 +290,7 @@ members = [ "apache-airflow-shared-state", "apache-airflow-shared-template-rendering", "apache-airflow-shared-timezones", + "apache-airflow-shared-workloads", "apache-airflow-task-sdk", "apache-airflow-task-sdk-integration-tests", "docker-stack", @@ -1025,6 +1031,8 @@ all = [ { name = "apache-airflow-providers-redis" }, { name = "apache-airflow-providers-salesforce" }, { name = "apache-airflow-providers-samba" }, + { name = "apache-airflow-providers-sdk-executable" }, + { name = "apache-airflow-providers-sdk-java" }, { name = "apache-airflow-providers-segment" }, { name = "apache-airflow-providers-sendgrid" }, { name = "apache-airflow-providers-sftp" }, @@ -1344,6 +1352,12 @@ salesforce = [ samba = [ { name = "apache-airflow-providers-samba" }, ] +sdk-executable = [ + { name = "apache-airflow-providers-sdk-executable" }, +] +sdk-java = [ + { name = "apache-airflow-providers-sdk-java" }, +] segment = [ { name = "apache-airflow-providers-segment" }, ] @@ -1440,6 +1454,7 @@ dev = [ { name = "apache-airflow-shared-state" }, { name = "apache-airflow-shared-template-rendering" }, { name = "apache-airflow-shared-timezones" }, + { name = "apache-airflow-shared-workloads" }, { name = "apache-airflow-task-sdk", extra = ["all"] }, { name = "apache-airflow-task-sdk-integration-tests" }, ] @@ -1635,6 +1650,10 @@ requires-dist = [ { name = "apache-airflow-providers-salesforce", marker = "extra == 'salesforce'", editable = "providers/salesforce" }, { name = "apache-airflow-providers-samba", marker = "extra == 'all'", editable = "providers/samba" }, { name = "apache-airflow-providers-samba", marker = "extra == 'samba'", editable = "providers/samba" }, + { name = "apache-airflow-providers-sdk-executable", marker = "extra == 'all'", editable = "providers/sdk/executable" }, + { name = "apache-airflow-providers-sdk-executable", marker = "extra == 'sdk-executable'", editable = "providers/sdk/executable" }, + { name = "apache-airflow-providers-sdk-java", marker = "extra == 'all'", editable = "providers/sdk/java" }, + { name = "apache-airflow-providers-sdk-java", marker = "extra == 'sdk-java'", editable = "providers/sdk/java" }, { name = "apache-airflow-providers-segment", marker = "extra == 'all'", editable = "providers/segment" }, { name = "apache-airflow-providers-segment", marker = "extra == 'segment'", editable = "providers/segment" }, { name = "apache-airflow-providers-sendgrid", marker = "extra == 'all'", editable = "providers/sendgrid" }, @@ -1685,7 +1704,7 @@ requires-dist = [ { name = "sentry-sdk", marker = "extra == 'sentry'", specifier = ">=2.30.0" }, { name = "uv", marker = "extra == 'uv'", specifier = ">=0.11.8" }, ] -provides-extras = ["all-core", "async", "graphviz", "gunicorn", "kerberos", "memray", "otel", "statsd", "all-task-sdk", "airbyte", "akeyless", "alibaba", "amazon", "apache-cassandra", "apache-drill", "apache-druid", "apache-flink", "apache-hdfs", "apache-hive", "apache-iceberg", "apache-impala", "apache-kafka", "apache-kylin", "apache-livy", "apache-pig", "apache-pinot", "apache-spark", "apache-tinkerpop", "apprise", "arangodb", "asana", "atlassian-jira", "celery", "cloudant", "cncf-kubernetes", "cohere", "common-ai", "common-compat", "common-io", "common-messaging", "common-sql", "databricks", "datadog", "dbt-cloud", "dingding", "discord", "docker", "edge3", "elasticsearch", "exasol", "fab", "facebook", "ftp", "git", "github", "google", "grpc", "hashicorp", "http", "imap", "influxdb", "informatica", "jdbc", "jenkins", "keycloak", "microsoft-azure", "microsoft-mssql", "microsoft-psrp", "microsoft-winrm", "mongo", "mysql", "neo4j", "odbc", "openai", "openfaas", "openlineage", "opensearch", "opsgenie", "oracle", "pagerduty", "papermill", "pgvector", "pinecone", "postgres", "presto", "qdrant", "redis", "salesforce", "samba", "segment", "sendgrid", "sftp", "singularity", "slack", "smtp", "snowflake", "sqlite", "ssh", "standard", "tableau", "telegram", "teradata", "trino", "vertica", "vespa", "weaviate", "yandex", "ydb", "zendesk", "all", "aiobotocore", "apache-atlas", "apache-webhdfs", "amazon-aws-auth", "cloudpickle", "github-enterprise", "google-auth", "ldap", "pandas", "polars", "rabbitmq", "sentry", "s3fs", "uv"] +provides-extras = ["all-core", "async", "graphviz", "gunicorn", "kerberos", "memray", "otel", "statsd", "all-task-sdk", "airbyte", "akeyless", "alibaba", "amazon", "apache-cassandra", "apache-drill", "apache-druid", "apache-flink", "apache-hdfs", "apache-hive", "apache-iceberg", "apache-impala", "apache-kafka", "apache-kylin", "apache-livy", "apache-pig", "apache-pinot", "apache-spark", "apache-tinkerpop", "apprise", "arangodb", "asana", "atlassian-jira", "celery", "cloudant", "cncf-kubernetes", "cohere", "common-ai", "common-compat", "common-io", "common-messaging", "common-sql", "databricks", "datadog", "dbt-cloud", "dingding", "discord", "docker", "edge3", "elasticsearch", "exasol", "fab", "facebook", "ftp", "git", "github", "google", "grpc", "hashicorp", "http", "imap", "influxdb", "informatica", "jdbc", "jenkins", "keycloak", "microsoft-azure", "microsoft-mssql", "microsoft-psrp", "microsoft-winrm", "mongo", "mysql", "neo4j", "odbc", "openai", "openfaas", "openlineage", "opensearch", "opsgenie", "oracle", "pagerduty", "papermill", "pgvector", "pinecone", "postgres", "presto", "qdrant", "redis", "salesforce", "samba", "sdk-executable", "sdk-java", "segment", "sendgrid", "sftp", "singularity", "slack", "smtp", "snowflake", "sqlite", "ssh", "standard", "tableau", "telegram", "teradata", "trino", "vertica", "vespa", "weaviate", "yandex", "ydb", "zendesk", "all", "aiobotocore", "apache-atlas", "apache-webhdfs", "amazon-aws-auth", "cloudpickle", "github-enterprise", "google-auth", "ldap", "pandas", "polars", "rabbitmq", "sentry", "s3fs", "uv"] [package.metadata.requires-dev] dev = [ @@ -1713,6 +1732,7 @@ dev = [ { name = "apache-airflow-shared-state", editable = "shared/state" }, { name = "apache-airflow-shared-template-rendering", editable = "shared/template_rendering" }, { name = "apache-airflow-shared-timezones", editable = "shared/timezones" }, + { name = "apache-airflow-shared-workloads", editable = "shared/workloads" }, { name = "apache-airflow-task-sdk", extras = ["all"], editable = "task-sdk" }, { name = "apache-airflow-task-sdk-integration-tests", editable = "task-sdk-integration-tests" }, ] @@ -2735,7 +2755,7 @@ packages = [] [[package]] name = "apache-airflow-providers-airbyte" -version = "5.4.1" +version = "5.4.2" source = { editable = "providers/airbyte" } dependencies = [ { name = "airbyte-api" }, @@ -2862,7 +2882,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-amazon" -version = "9.27.0" +version = "9.28.0" source = { editable = "providers/amazon" } dependencies = [ { name = "apache-airflow" }, @@ -4025,7 +4045,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-cncf-kubernetes" -version = "10.16.1" +version = "10.17.0" source = { editable = "providers/cncf/kubernetes" } dependencies = [ { name = "aiofiles" }, @@ -4113,7 +4133,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-common-ai" -version = "0.1.1" +version = "0.2.0" source = { editable = "providers/common/ai" } dependencies = [ { name = "apache-airflow" }, @@ -4341,7 +4361,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-common-sql" -version = "1.35.0" +version = "1.36.0" source = { editable = "providers/common/sql" } dependencies = [ { name = "apache-airflow" }, @@ -4442,7 +4462,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-databricks" -version = "7.13.0" +version = "7.14.0" source = { editable = "providers/databricks" } dependencies = [ { name = "aiohttp" }, @@ -4584,7 +4604,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-dbt-cloud" -version = "4.8.1" +version = "4.8.2" source = { editable = "providers/dbt/cloud" } dependencies = [ { name = "aiohttp" }, @@ -4678,7 +4698,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-discord" -version = "3.12.2" +version = "3.12.3" source = { editable = "providers/discord" } dependencies = [ { name = "apache-airflow" }, @@ -4756,7 +4776,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-edge3" -version = "3.5.0" +version = "3.6.0" source = { editable = "providers/edge3" } dependencies = [ { name = "aiofiles" }, @@ -4895,7 +4915,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-fab" -version = "3.6.2" +version = "3.6.3" source = { editable = "providers/fab" } dependencies = [ { name = "apache-airflow" }, @@ -5059,7 +5079,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-git" -version = "0.3.1" +version = "0.3.2" source = { editable = "providers/git" } dependencies = [ { name = "apache-airflow" }, @@ -5133,7 +5153,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-google" -version = "21.2.0" +version = "21.3.0" source = { editable = "providers/google" } dependencies = [ { name = "apache-airflow" }, @@ -5815,7 +5835,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-microsoft-azure" -version = "13.1.2" +version = "13.2.0" source = { editable = "providers/microsoft/azure" } dependencies = [ { name = "adlfs" }, @@ -6103,7 +6123,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-mysql" -version = "6.5.2" +version = "6.5.3" source = { editable = "providers/mysql" } dependencies = [ { name = "aiomysql" }, @@ -6333,7 +6353,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-openlineage" -version = "2.15.0" +version = "2.16.0" source = { editable = "providers/openlineage" } dependencies = [ { name = "apache-airflow" }, @@ -7027,6 +7047,88 @@ dev = [ ] docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "devel-common" }] +[[package]] +name = "apache-airflow-providers-sdk-executable" +version = "0.1.0" +source = { editable = "providers/sdk/executable" } +dependencies = [ + { name = "apache-airflow" }, + { name = "pyyaml" }, +] + +[package.optional-dependencies] +common-compat = [ + { name = "apache-airflow-providers-common-compat" }, +] + +[package.dev-dependencies] +dev = [ + { name = "apache-airflow" }, + { name = "apache-airflow-devel-common" }, + { name = "apache-airflow-providers-common-compat" }, + { name = "apache-airflow-task-sdk" }, +] +docs = [ + { name = "apache-airflow-devel-common", extra = ["docs"] }, +] + +[package.metadata] +requires-dist = [ + { name = "apache-airflow", editable = "." }, + { name = "apache-airflow-providers-common-compat", marker = "extra == 'common-compat'", editable = "providers/common/compat" }, + { name = "pyyaml" }, +] +provides-extras = ["common-compat"] + +[package.metadata.requires-dev] +dev = [ + { name = "apache-airflow", editable = "." }, + { name = "apache-airflow-devel-common", editable = "devel-common" }, + { name = "apache-airflow-providers-common-compat", editable = "providers/common/compat" }, + { name = "apache-airflow-task-sdk", editable = "task-sdk" }, +] +docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "devel-common" }] + +[[package]] +name = "apache-airflow-providers-sdk-java" +version = "0.1.0" +source = { editable = "providers/sdk/java" } +dependencies = [ + { name = "apache-airflow" }, +] + +[package.optional-dependencies] +common-compat = [ + { name = "apache-airflow-providers-common-compat" }, +] + +[package.dev-dependencies] +dev = [ + { name = "apache-airflow" }, + { name = "apache-airflow-devel-common" }, + { name = "apache-airflow-providers-common-compat" }, + { name = "apache-airflow-task-sdk" }, +] +docs = [ + { name = "apache-airflow-devel-common", extra = ["docs"] }, +] + +[package.metadata] +requires-dist = [ + { name = "apache-airflow", editable = "." }, + { name = "apache-airflow-providers-common-compat", marker = "extra == 'common-compat'", editable = "providers/common/compat" }, +] +provides-extras = ["common-compat"] + +[package.metadata.requires-dev] +dev = [ + { name = "apache-airflow", editable = "." }, + { name = "apache-airflow-devel-common", editable = "devel-common" }, + { name = "apache-airflow-providers-common-compat", editable = "providers/common/compat" }, + { name = "apache-airflow-task-sdk", editable = "task-sdk" }, +] +docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "devel-common" }] + [[package]] name = "apache-airflow-providers-segment" version = "3.9.4" @@ -7431,7 +7533,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-standard" -version = "1.12.3" +version = "1.13.0" source = { editable = "providers/standard" } dependencies = [ { name = "apache-airflow" }, @@ -7813,7 +7915,7 @@ docs = [{ name = "apache-airflow-devel-common", extras = ["docs"], editable = "d [[package]] name = "apache-airflow-providers-yandex" -version = "4.4.2" +version = "4.5.0" source = { editable = "providers/yandex" } dependencies = [ { name = "apache-airflow" }, @@ -8426,6 +8528,25 @@ requires-dist = [ dev = [{ name = "apache-airflow-devel-common", editable = "devel-common" }] mypy = [{ name = "apache-airflow-devel-common", extras = ["mypy"], editable = "devel-common" }] +[[package]] +name = "apache-airflow-shared-workloads" +version = "0.0" +source = { editable = "shared/workloads" } +dependencies = [ + { name = "pydantic" }, +] + +[package.dev-dependencies] +dev = [ + { name = "apache-airflow-devel-common" }, +] + +[package.metadata] +requires-dist = [{ name = "pydantic", specifier = ">=2.10.0" }] + +[package.metadata.requires-dev] +dev = [{ name = "apache-airflow-devel-common", editable = "devel-common" }] + [[package]] name = "apache-airflow-task-sdk" source = { editable = "task-sdk" }