From b716ec018d3f7ff341f0632c02160fbc055352f2 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 09:26:15 +0200 Subject: [PATCH 001/338] refactor: Refactored task map expansion using a job runner, also allow ExpandInput's and scheduler XCom's to resolve values on server side (like it was in the past) --- .../airflow/jobs/expand_task_job_runner.py | 152 ++++++++++ airflow-core/src/airflow/jobs/job.py | 20 ++ .../src/airflow/models/expandinput.py | 59 +++- .../src/airflow/models/taskinstance.py | 25 ++ airflow-core/src/airflow/models/taskmap.py | 268 ++++++++++-------- airflow-core/src/airflow/models/xcom_arg.py | 75 ++++- 6 files changed, 481 insertions(+), 118 deletions(-) create mode 100644 airflow-core/src/airflow/jobs/expand_task_job_runner.py diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py new file mode 100644 index 0000000000000..e218bbdcdd2b0 --- /dev/null +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -0,0 +1,152 @@ +from __future__ import annotations + +from collections.abc import Iterator +from typing import TYPE_CHECKING + +from airflow.configuration import conf +from airflow.exceptions import AirflowException +from airflow.jobs.base_job_runner import BaseJobRunner +from airflow.jobs.job import Job +from airflow.models import DagRun +from airflow.models.dag_version import DagVersion +from airflow.models.expandinput import SchedulerDictOfListsExpandInput +from airflow.models.taskinstance import TaskInstance, get_current_max_mapping, get_task_instance +from airflow.models.xcom_arg import SchedulerPlainXComArg +from airflow.policies import task_instance_mutation_hook +from airflow.sdk.bases.operator import BaseOperator +from airflow.sdk.definitions._internal.mixins import ResolveMixin +from airflow.sdk.definitions._internal.types import NOTSET +from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.sdk.execution_time.comms import XComResult +from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.module_loading import import_string +from airflow.utils.session import create_session +from airflow.utils.state import DagRunState +from sqlalchemy import select + +if TYPE_CHECKING: + from sqlalchemy.orm import Session + +task_expansion_batch_size = conf.getint("scheduler", "task_expansion_batch_size", fallback=10) + + +def get_dag_run(dag_id: str, run_id: str, session: Session) -> DagRun | None: + """ + Returns the TaskInstance for the task that is being expanded. + """ + return session.scalars( + select(DagRun) + .where( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + ).one_or_none() + + +class TaskExpansionJobRunner(BaseJobRunner, LoggingMixin): + job_type = "TaskExpansionJob" + + def __init__( + self, + job: Job, + task: MappedOperator, + run_id: str, + dag_version_id: DagVersion, + ) -> None: + super().__init__(job) + self.task = task + self.task.operator_class = import_string(f"{task._task_module}.{task._task_type}") + self.run_id = run_id + self.dag_version_id = dag_version_id + + @property + def dag_id(self) -> str: + return self.task.dag_id + + @property + def task_id(self) -> str: + return self.task.task_id + + def expand_input(self, session: Session) -> Iterator[dict]: + self.log.info("expand_input: %s", self.task.expand_input) + context = { + "task": self.task, + "run_id": self.run_id, + } + return iter(self.task.expand_input.resolve(context, session)) + + def expand_task(self, task_instance: TaskInstance, mapped_kwargs) -> TaskInstance: + self.log.info("expand task: %s", task_instance.map_index) + self.log.debug("unmap (%s): %s", self.task.operator_class, mapped_kwargs) + + operator = self.task.unmap(mapped_kwargs) + + self.log.info("operator (%s): %s", type(operator), operator) + + task_instance.task = operator + task_instance_mutation_hook(task_instance) + + self.log.info("creating ti %s: (%s) %s", task_instance.map_index, task_instance.id, task_instance) + + return task_instance + + def _check_dag_run_state(self, dag_run: DagRun) -> None: + self.log.info("dag_run_state: %s", dag_run.state) + + if dag_run.state == DagRunState.FAILED: + self.log.info("DagRun %s for dag %s has failed, stopping expansion", self.run_id, self.dag_id) + + raise AirflowException(f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure.") + + def _persist_task_instances(self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session) -> None: + if task_instances: + self.log.info("Persisting %d new task instances", len(task_instances)) + dag_run.task_instances.extend(task_instances) + session.merge(dag_run) + session.flush() + session.commit() + task_instances.clear() + + def expand_tasks(self, session: Session) -> int: + """ + Expands the task using the provided expand_input. + """ + counter = 0 + max_map_index = get_current_max_mapping( + dag_id=self.dag_id, + task_id=self.task_id, + run_id=self.run_id, + session=session, + ) + dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + + self.log.info("expand_tasks: %s", session) + self.log.info("max_map_index: %s", max_map_index) + self.log.info("dag_version_id: %s", self.dag_version_id) + self.log.info("dag_run: %s", dag_run) + + task_instances_batch = [] + + for map_index, mapped_kwargs in enumerate(self.expand_input(session=session)): + if map_index > max_map_index: + task_instance = TaskInstance( + task=self.task, + run_id=self.run_id, + map_index=map_index, + dag_version_id=self.dag_version_id, + ) + task_instances_batch.append(self.expand_task(task_instance, mapped_kwargs)) + counter += 1 + + if len(task_instances_batch) == task_expansion_batch_size: + dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + self._check_dag_run_state(dag_run) + self._persist_task_instances(dag_run, task_instances_batch, session=session) + + self._persist_task_instances(dag_run, task_instances_batch, session=session) + + return counter + + def _execute(self) -> int | None: + with create_session() as session: + return self.expand_tasks(session=session) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 9c06552abca21..dd17e3c817c8f 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -18,6 +18,7 @@ from __future__ import annotations from functools import cached_property, lru_cache +from multiprocessing import Process from time import sleep from typing import TYPE_CHECKING, Callable, NoReturn @@ -356,6 +357,25 @@ def run_job( job.complete_execution(session=session) +def run_job_async( + job: Job, execute_callable: Callable[[], int | None], session: Session +) -> None: + """ + Run the job asynchronously. + + The Job is always an ORM object and setting the state is happening within the + same DB session and the session is kept open throughout the whole execution. + + :meta private: + """ + job.prepare_for_execution(session=session) + try: + process = Process(target=execute_job, args=(job, execute_callable)) + process.start() + finally: + job.complete_execution(session=session) + + def execute_job(job: Job, execute_callable: Callable[[], int | None]) -> int | None: """ Execute the job. diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index b126c6f24b07f..4dd74a4b668d2 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -19,10 +19,13 @@ import functools import operator -from collections.abc import Iterable, Sized +from collections.abc import Iterable, Sized, Mapping, Generator from typing import TYPE_CHECKING, Any, ClassVar, Union import attrs +from airflow.utils.log.logging_mixin import LoggingMixin + +from airflow.models.taskmap import update_task_map_length if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -57,8 +60,13 @@ def _needs_run_time_resolution(v: OperatorExpandArgument) -> TypeGuard[MappedArg return isinstance(v, (MappedArgument, SchedulerXComArg)) +class ExpandInput(LoggingMixin): + def resolve(self, context: Mapping[str, Any], session: Session): + raise NotImplementedError() + + @attrs.define -class SchedulerDictOfListsExpandInput: +class SchedulerDictOfListsExpandInput(ExpandInput): value: dict EXPAND_INPUT_TYPE: ClassVar[str] = "dict-of-lists" @@ -110,9 +118,30 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: lengths = self._get_map_lengths(run_id, session=session) return functools.reduce(operator.mul, (lengths[name] for name in self.value), 1) + def resolve(self, context: Mapping[str, Any], session: Session) -> Generator[ + dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: + + self.log.info("expand_dict: %s", self.value) + + value = self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + + self.log.info("resolved value: %s", value) + + for key, item in value.items(): + result = item.resolve(context, session) + + self.log.info("resolved value %s: %s", key, result) + + for index, sub_item in enumerate(result): + yield {key: sub_item} + + update_task_map_length(index, item, context["run_id"], session) + + return [] + @attrs.define -class SchedulerListOfDictsExpandInput: +class SchedulerListOfDictsExpandInput(ExpandInput): value: list EXPAND_INPUT_TYPE: ClassVar[str] = "list-of-dicts" @@ -132,6 +161,30 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: raise NotFullyPopulated({"expand_kwargs() argument"}) return length + def resolve(self, context: Mapping[str, Any], session: Session) -> Generator[ + dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: + + self.log.info("expand_list: %s", self.value) + + value = self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + + self.log.info("resolved value: %s", value) + + for index, entry in enumerate(value): + self.log.info("entry: %s", entry) + + for key, item in entry.items(): + if isinstance(item, SchedulerXComArg): + entry[key] = item.resolve(context, session) + + self.log.info("resolved entry: %s", entry) + + yield entry + + update_task_map_length(index, self.value, context["run_id"], session) + + return [] + _EXPAND_INPUT_TYPES: dict[str, type[SchedulerExpandInput]] = { "dict-of-lists": SchedulerDictOfListsExpandInput, diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index 21e20b32b9627..fc81d6b5164e9 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -493,6 +493,31 @@ def uuid7() -> str: return str(uuid6.uuid7()) +def get_task_instance(dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION) -> TaskInstance | None: + @cache + def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): + return session.scalars( + select(TaskInstance) + .where( + TaskInstance.dag_id == _dag_id, + TaskInstance.task_id == _task_id, + TaskInstance.run_id == _run_id, + TaskInstance.map_index == _map_index, + ) + ).one_or_none() + return find_task_instance(dag_id, task_id, run_id, map_index) + + +def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Session) -> int: + return max(session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == dag_id, + TaskInstance.task_id == task_id, + TaskInstance.run_id == run_id, + ) + ), 0) + + class TaskInstance(Base, LoggingMixin): """ Task instances store the state of a task instance. diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index f0fd4c0231b70..dc3c6a19b0f6e 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -21,10 +21,13 @@ import collections.abc import enum +import logging from collections.abc import Collection, Iterable, Sequence from typing import TYPE_CHECKING, Any -from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select +from airflow.jobs.job import Job, run_job_async +from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select, \ + update from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion @@ -32,11 +35,31 @@ from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks from airflow.utils.state import State, TaskInstanceState +from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner + if TYPE_CHECKING: from sqlalchemy.orm import Session from airflow.models.dag import DAG as SchedulerDAG - from airflow.models.taskinstance import TaskInstance + from airflow.models.taskinstance import TaskInstance, get_task_instance, get_current_max_mapping + + +def update_task_map_length(index, item, run_id, session): + try: + length = index + 1 + logging.info("Persisting TaskMap length: %s", length) + session.execute( + update(TaskMap) + .where( + TaskMap.dag_id == item.operator.dag_id, + TaskMap.task_id == item.operator.task_id, + TaskMap.run_id == run_id, + TaskMap.map_index == -1, + ) + .values(length=length) + ) + except: + logging.exception("Persisting TaskMap length failed for task %s", item.operator.task_id) class TaskMapVariant(enum.Enum): @@ -132,8 +155,8 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq """ from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated - from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator + from airflow.sdk.definitions._internal.types import NOTSET from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.settings import task_instance_mutation_hook @@ -142,87 +165,9 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" ) - try: - total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) - except NotFullyPopulated as e: - if not task.dag or not task.dag.partial: - task.log.error( - "Cannot expand %r for run %s; missing upstream values: %s", - task, - run_id, - sorted(e.missing), - ) - total_length = None + unmapped_ti = get_task_instance(dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session) - state: TaskInstanceState | None = None - unmapped_ti: TaskInstance | None = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index == -1, - or_(TaskInstance.state.in_(State.unfinished), TaskInstance.state.is_(None)), - ) - ).one_or_none() - - all_expanded_tis: list[TaskInstance] = [] - - if unmapped_ti: - if TYPE_CHECKING: - assert task.dag is None or isinstance(task.dag, SchedulerDAG) - - # The unmapped task instance still exists and is unfinished, i.e. we - # haven't tried to run it before. - if total_length is None: - # If the DAG is partial, it's likely that the upstream tasks - # are not done yet, so the task can't fail yet. - if not task.dag or not task.dag.partial: - unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED - elif total_length < 1: - # If the upstream maps this to a zero-length value, simply mark - # the unmapped task instance as SKIPPED (if needed). - task.log.info( - "Marking %s as SKIPPED since the map has %d values to expand", - unmapped_ti, - total_length, - ) - unmapped_ti.state = TaskInstanceState.SKIPPED - else: - zero_index_ti_exists = exists_query( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index == 0, - session=session, - ) - if not zero_index_ti_exists: - # Otherwise convert this into the first mapped index, and create - # TaskInstance for other indexes. - unmapped_ti.map_index = 0 - task.log.debug("Updated in place to become %s", unmapped_ti) - all_expanded_tis.append(unmapped_ti) - # execute hook for task instance map index 0 - task_instance_mutation_hook(unmapped_ti) - session.flush() - else: - task.log.debug("Deleting the original task instance: %s", unmapped_ti) - session.delete(unmapped_ti) - state = unmapped_ti.state - dag_version_id = unmapped_ti.dag_version_id - - if total_length is None or total_length < 1: - # Nothing to fixup. - indexes_to_map: Iterable[int] = () - else: - # Only create "missing" ones. - current_max_mapping = session.scalar( - select(func.max(TaskInstance.map_index)).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - ) - ) - indexes_to_map = range(current_max_mapping + 1, total_length) + task.log.info("unmapped_ti: %s", unmapped_ti) if unmapped_ti: dag_version_id = unmapped_ti.dag_version_id @@ -231,36 +176,133 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq else: dag_version_id = None - for index in indexes_to_map: - # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. - ti = TaskInstance( - task, - run_id=run_id, - map_index=index, - state=state, - dag_version_id=dag_version_id, + task.log.info("dag_version_id: %s", dag_version_id) + + all_expanded_tis: list[TaskInstance] = [] + total_expanded_ti_count = 0 + + if isinstance(task, MappedOperator): + if unmapped_ti: + job = Job() + job_runner = TaskExpansionJobRunner( + job=job, + task=task, + run_id=run_id, + dag_version_id=dag_version_id, + ) + mapped_kwargs = next(job_runner.expand_input(session=session), None) + unmapped_ti.map_index = 0 + unmapped_ti = job_runner.expand_task(unmapped_ti, mapped_kwargs) + task_instance_mutation_hook(unmapped_ti) + session.merge(unmapped_ti) + session.flush() + all_expanded_tis.append(unmapped_ti) + total_expanded_ti_count = len(all_expanded_tis) + + task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) + + run_job_async(job=job, execute_callable=job_runner._execute, session=session) + else: + try: + total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) + except NotFullyPopulated as e: + if not task.dag or not task.dag.partial: + task.log.error( + "Cannot expand %r for run %s; missing upstream values: %s", + task, + run_id, + sorted(e.missing), + ) + total_length = None + + task.log.info("total_length: %s", total_length) + + state: TaskInstanceState | None = None + + if unmapped_ti: + if TYPE_CHECKING: + assert task.dag is None or isinstance(task.dag, SchedulerDAG) + + # The unmapped task instance still exists and is unfinished, i.e. we + # haven't tried to run it before. + if total_length is None: + # If the DAG is partial, it's likely that the upstream tasks + # are not done yet, so the task can't fail yet. + if not task.dag or not task.dag.partial: + unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED + elif total_length < 1: + # If the upstream maps this to a zero-length value, simply mark + # the unmapped task instance as SKIPPED (if needed). + task.log.info( + "Marking %s as SKIPPED since the map has %d values to expand", + unmapped_ti, + total_length, + ) + unmapped_ti.state = TaskInstanceState.SKIPPED + else: + zero_index_ti_exists = exists_query( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index == 0, + session=session, + ) + if not zero_index_ti_exists: + # Otherwise convert this into the first mapped index, and create + # TaskInstance for other indexes. + unmapped_ti.map_index = 0 + task.log.debug("Updated in place to become %s", unmapped_ti) + all_expanded_tis.append(unmapped_ti) + # execute hook for task instance map index 0 + task_instance_mutation_hook(unmapped_ti) + session.flush() + else: + task.log.debug("Deleting the original task instance: %s", unmapped_ti) + session.delete(unmapped_ti) + state = unmapped_ti.state + + if total_length is None or total_length < 1: + # Nothing to fixup. + indexes_to_map: Iterable[int] = () + else: + # Only create "missing" ones. + current_max_mapping = get_current_max_mapping(dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session) + indexes_to_map = range(current_max_mapping + 1, total_length) + + task.log.info("indexes_to_map: %s", indexes_to_map) + + for index in indexes_to_map: + # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. + ti = TaskInstance( + task, + run_id=run_id, + map_index=index, + state=state, + dag_version_id=dag_version_id, + ) + task.log.debug("Expanding TIs upserted %s", ti) + task_instance_mutation_hook(ti) + ti = session.merge(ti) + ti.refresh_from_task(task) # session.merge() loses task information. + all_expanded_tis.append(ti) + + # Coerce the None case to 0 -- these two are almost treated identically, + # except the unmapped ti (if exists) is marked to different states. + total_expanded_ti_count = total_length or 0 + + task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) + + # Any (old) task instances with inapplicable indexes (>= the total + # number we need) are set to "REMOVED". + query = select(TaskInstance).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index >= total_expanded_ti_count, ) - task.log.debug("Expanding TIs upserted %s", ti) - task_instance_mutation_hook(ti) - ti = session.merge(ti) - ti.refresh_from_task(task) # session.merge() loses task information. - all_expanded_tis.append(ti) - - # Coerce the None case to 0 -- these two are almost treated identically, - # except the unmapped ti (if exists) is marked to different states. - total_expanded_ti_count = total_length or 0 - - # Any (old) task instances with inapplicable indexes (>= the total - # number we need) are set to "REMOVED". - query = select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index >= total_expanded_ti_count, - ) - query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) - to_update = session.scalars(query) - for ti in to_update: - ti.state = TaskInstanceState.REMOVED + query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) + to_update = session.scalars(query) + for ti in to_update: + ti.state = TaskInstanceState.REMOVED session.flush() return all_expanded_tis, total_expanded_ti_count - 1 diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index cfda9295cec26..6cf901b6769c3 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -17,25 +17,32 @@ from __future__ import annotations -from collections.abc import Sequence +from collections.abc import Mapping, Sequence from functools import singledispatch from typing import TYPE_CHECKING, Any import attrs +from airflow.models.xcom import BaseXCom +from airflow.utils.log.logging_mixin import LoggingMixin from sqlalchemy import func, or_, select from sqlalchemy.orm import Session +from airflow.models.taskinstance import get_task_instance from airflow.sdk.definitions._internal.types import ArgNotSet from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.xcom_arg import ( XComArg, ) +from airflow.sdk.definitions._internal.mixins import ResolveMixin +from airflow.sdk.execution_time.comms import XComResult +from airflow.sdk.execution_time.xcom import resolve_xcom_backend from airflow.utils.db import exists_query from airflow.utils.state import State from airflow.utils.types import NOTSET from airflow.utils.xcom import XCOM_RETURN_KEY __all__ = ["XComArg", "get_task_map_length"] +xcom_backend: BaseXCom = resolve_xcom_backend() if TYPE_CHECKING: from airflow.models.dag import DAG as SchedulerDAG @@ -44,7 +51,7 @@ @attrs.define -class SchedulerXComArg: +class SchedulerXComArg(LoggingMixin): @classmethod def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: """ @@ -59,6 +66,9 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: """ raise NotImplementedError() + def resolve(self, context: Mapping[str, Any], session: Session): + raise NotImplementedError() + @attrs.define class SchedulerPlainXComArg(SchedulerXComArg): @@ -69,18 +79,79 @@ class SchedulerPlainXComArg(SchedulerXComArg): def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: return cls(dag.get_task(data["task_id"]), data["key"]) + def resolve(self, context: Mapping[str, Any], session: Session) -> Any: + task_instance = get_task_instance( + dag_id=self.operator.dag_id, + task_id=self.operator.task_id, + run_id=context["run_id"], + session=session, + ) + + context = { + **context, **{ + "task_instance": task_instance, + "ti": task_instance, + } + } + + self.log.info("XCom task_instance: %s", task_instance) + + value = task_instance.xcom_pull( + task_ids=self.operator.task_id, + key=self.operator.output.key, + map_indexes=task_instance.map_index, + session=session, + ) + + self.log.info("value: %s", value) + self.log.debug("xcom_backend: %s", xcom_backend) + + deserialized_value = xcom_backend.deserialize_value( + XComResult(key=self.operator.output.key, value=value)) + + self.log.info("deserialized_value: %s", deserialized_value) + + if isinstance(deserialized_value, ResolveMixin): + self.log.info("context: %s", context) + deserialized_value = deserialized_value.resolve(context) + self.log.info("resolved_value: %s", deserialized_value) + + return deserialized_value + @attrs.define class SchedulerMapXComArg(SchedulerXComArg): arg: SchedulerXComArg callables: Sequence[str] + @property + def operator(self) -> Operator: + """Return the operator that this XComArg is associated with.""" + return self.arg.operator if isinstance(self.arg, SchedulerPlainXComArg) else self.arg + @classmethod def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: # We are deliberately NOT deserializing the callables. These are shown # in the UI, and displaying a function object is useless. return cls(deserialize_xcom_arg(data["arg"], dag), data["callables"]) + def resolve(self, context: Mapping[str, Any], session: Session) -> Any: + self.log.info("arg: %s", self.arg) + resolved_arg = self.arg.resolve(context, session) + self.log.info("resolved_arg: %s", resolved_arg) + + def apply(arg: Any): + for index, _callable in enumerate(self.callables): + if isinstance(_callable, str): + _callable = eval(_callable) + self.callables[index] = _callable + self.log.debug("arg: %s", arg) + arg = _callable(arg) + self.log.debug("apply: %s", arg) + return arg + + return map(apply, resolved_arg) + @attrs.define class SchedulerConcatXComArg(SchedulerXComArg): From dc4132922d97bde651ef5ef0e0cdbabe074b408f Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 09:46:45 +0200 Subject: [PATCH 002/338] refactor: Moved helper methods for TaskInstance and TakMap after class declaration --- .../src/airflow/models/taskinstance.py | 50 +++++++++---------- airflow-core/src/airflow/models/taskmap.py | 36 ++++++------- 2 files changed, 43 insertions(+), 43 deletions(-) diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index fc81d6b5164e9..bbd06a96bb04b 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -493,31 +493,6 @@ def uuid7() -> str: return str(uuid6.uuid7()) -def get_task_instance(dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION) -> TaskInstance | None: - @cache - def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): - return session.scalars( - select(TaskInstance) - .where( - TaskInstance.dag_id == _dag_id, - TaskInstance.task_id == _task_id, - TaskInstance.run_id == _run_id, - TaskInstance.map_index == _map_index, - ) - ).one_or_none() - return find_task_instance(dag_id, task_id, run_id, map_index) - - -def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Session) -> int: - return max(session.scalar( - select(func.max(TaskInstance.map_index)).where( - TaskInstance.dag_id == dag_id, - TaskInstance.task_id == task_id, - TaskInstance.run_id == run_id, - ) - ), 0) - - class TaskInstance(Base, LoggingMixin): """ Task instances store the state of a task instance. @@ -2511,6 +2486,31 @@ def duration_expression_update( ) +def get_task_instance(dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION) -> TaskInstance | None: + @cache + def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): + return session.scalars( + select(TaskInstance) + .where( + TaskInstance.dag_id == _dag_id, + TaskInstance.task_id == _task_id, + TaskInstance.run_id == _run_id, + TaskInstance.map_index == _map_index, + ) + ).one_or_none() + return find_task_instance(dag_id, task_id, run_id, map_index) + + +def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Session) -> int: + return max(session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == dag_id, + TaskInstance.task_id == task_id, + TaskInstance.run_id == run_id, + ) + ), 0) + + def _find_common_ancestor_mapped_group(node1: Operator, node2: Operator) -> MappedTaskGroup | None: """Given two operators, find their innermost common mapped task group.""" if node1.dag is None or node2.dag is None or node1.dag_id != node2.dag_id: diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index dc3c6a19b0f6e..5b7bdf0e3b468 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -44,24 +44,6 @@ from airflow.models.taskinstance import TaskInstance, get_task_instance, get_current_max_mapping -def update_task_map_length(index, item, run_id, session): - try: - length = index + 1 - logging.info("Persisting TaskMap length: %s", length) - session.execute( - update(TaskMap) - .where( - TaskMap.dag_id == item.operator.dag_id, - TaskMap.task_id == item.operator.task_id, - TaskMap.run_id == run_id, - TaskMap.map_index == -1, - ) - .values(length=length) - ) - except: - logging.exception("Persisting TaskMap length failed for task %s", item.operator.task_id) - - class TaskMapVariant(enum.Enum): """ Task map variant. @@ -306,3 +288,21 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq ti.state = TaskInstanceState.REMOVED session.flush() return all_expanded_tis, total_expanded_ti_count - 1 + + +def update_task_map_length(index, item, run_id, session): + try: + length = index + 1 + logging.info("Persisting TaskMap length: %s", length) + session.execute( + update(TaskMap) + .where( + TaskMap.dag_id == item.operator.dag_id, + TaskMap.task_id == item.operator.task_id, + TaskMap.run_id == run_id, + TaskMap.map_index == -1, + ) + .values(length=length) + ) + except: + logging.exception("Persisting TaskMap length failed for task %s", item.operator.task_id) From b7b3be28016a2fcf5f447df7bde1368070506a82 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 10:27:01 +0200 Subject: [PATCH 003/338] refactor: Check if DagRun exist --- .../src/airflow/jobs/expand_task_job_runner.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index e218bbdcdd2b0..08c308ad22b3a 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -91,15 +91,16 @@ def expand_task(self, task_instance: TaskInstance, mapped_kwargs) -> TaskInstanc return task_instance def _check_dag_run_state(self, dag_run: DagRun) -> None: - self.log.info("dag_run_state: %s", dag_run.state) + if dag_run: + self.log.info("dag_run_state: %s", dag_run.state) - if dag_run.state == DagRunState.FAILED: - self.log.info("DagRun %s for dag %s has failed, stopping expansion", self.run_id, self.dag_id) + if dag_run.state == DagRunState.FAILED: + self.log.info("DagRun %s for dag %s has failed, stopping expansion", self.run_id, self.dag_id) - raise AirflowException(f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure.") + raise AirflowException(f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure.") def _persist_task_instances(self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session) -> None: - if task_instances: + if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) session.merge(dag_run) From 8ddfb7281c9da7c4e184edaa1a613265edb10c5d Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 10:50:56 +0200 Subject: [PATCH 004/338] refactor: Fixed some static checks --- .../src/airflow/models/expandinput.py | 22 +++--- .../src/airflow/models/taskinstance.py | 23 +++--- airflow-core/src/airflow/models/taskmap.py | 72 ++++++++++--------- airflow-core/src/airflow/models/xcom_arg.py | 12 ++-- 4 files changed, 75 insertions(+), 54 deletions(-) diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 4dd74a4b668d2..4b82c77fc7891 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -19,13 +19,13 @@ import functools import operator -from collections.abc import Iterable, Sized, Mapping, Generator +from collections.abc import Generator, Iterable, Mapping, Sized from typing import TYPE_CHECKING, Any, ClassVar, Union import attrs -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.models.taskmap import update_task_map_length +from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -118,12 +118,15 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: lengths = self._get_map_lengths(run_id, session=session) return functools.reduce(operator.mul, (lengths[name] for name in self.value), 1) - def resolve(self, context: Mapping[str, Any], session: Session) -> Generator[ - dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: + def resolve( + self, context: Mapping[str, Any], session: Session + ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: self.log.info("expand_dict: %s", self.value) - value = self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + value = ( + self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + ) self.log.info("resolved value: %s", value) @@ -161,12 +164,15 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: raise NotFullyPopulated({"expand_kwargs() argument"}) return length - def resolve(self, context: Mapping[str, Any], session: Session) -> Generator[ - dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: + def resolve( + self, context: Mapping[str, Any], session: Session + ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: self.log.info("expand_list: %s", self.value) - value = self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + value = ( + self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + ) self.log.info("resolved value: %s", value) diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index bbd06a96bb04b..dcded7825eafb 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -2486,29 +2486,34 @@ def duration_expression_update( ) -def get_task_instance(dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION) -> TaskInstance | None: +def get_task_instance( + dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION +) -> TaskInstance | None: @cache def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): return session.scalars( - select(TaskInstance) - .where( + select(TaskInstance).where( TaskInstance.dag_id == _dag_id, TaskInstance.task_id == _task_id, TaskInstance.run_id == _run_id, TaskInstance.map_index == _map_index, ) ).one_or_none() + return find_task_instance(dag_id, task_id, run_id, map_index) def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Session) -> int: - return max(session.scalar( - select(func.max(TaskInstance.map_index)).where( - TaskInstance.dag_id == dag_id, - TaskInstance.task_id == task_id, - TaskInstance.run_id == run_id, + return max( + session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == dag_id, + TaskInstance.task_id == task_id, + TaskInstance.run_id == run_id, + ) + ), + 0, ) - ), 0) def _find_common_ancestor_mapped_group(node1: Operator, node2: Operator) -> MappedTaskGroup | None: diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 5b7bdf0e3b468..07df84b6cfd09 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -25,23 +25,30 @@ from collections.abc import Collection, Iterable, Sequence from typing import TYPE_CHECKING, Any -from airflow.jobs.job import Job, run_job_async -from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select, \ - update +from sqlalchemy import ( + CheckConstraint, + Column, + ForeignKeyConstraint, + Integer, + String, + select, + update, +) +from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner +from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion +from airflow.models.taskinstance import get_task_instance, get_current_max_mapping from airflow.utils.db import exists_query from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks -from airflow.utils.state import State, TaskInstanceState - -from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner +from airflow.utils.state import TaskInstanceState if TYPE_CHECKING: from sqlalchemy.orm import Session from airflow.models.dag import DAG as SchedulerDAG - from airflow.models.taskinstance import TaskInstance, get_task_instance, get_current_max_mapping + from airflow.models.taskinstance import TaskInstance class TaskMapVariant(enum.Enum): @@ -138,7 +145,6 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated from airflow.sdk.bases.operator import BaseOperator - from airflow.sdk.definitions._internal.types import NOTSET from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.settings import task_instance_mutation_hook @@ -147,7 +153,9 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" ) - unmapped_ti = get_task_instance(dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session) + unmapped_ti = get_task_instance( + dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session + ) task.log.info("unmapped_ti: %s", unmapped_ti) @@ -161,29 +169,27 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq task.log.info("dag_version_id: %s", dag_version_id) all_expanded_tis: list[TaskInstance] = [] - total_expanded_ti_count = 0 - if isinstance(task, MappedOperator): - if unmapped_ti: - job = Job() - job_runner = TaskExpansionJobRunner( - job=job, - task=task, - run_id=run_id, - dag_version_id=dag_version_id, - ) - mapped_kwargs = next(job_runner.expand_input(session=session), None) - unmapped_ti.map_index = 0 - unmapped_ti = job_runner.expand_task(unmapped_ti, mapped_kwargs) - task_instance_mutation_hook(unmapped_ti) - session.merge(unmapped_ti) - session.flush() - all_expanded_tis.append(unmapped_ti) - total_expanded_ti_count = len(all_expanded_tis) - - task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) - - run_job_async(job=job, execute_callable=job_runner._execute, session=session) + if isinstance(task, MappedOperator) and unmapped_ti: + job = Job() + job_runner = TaskExpansionJobRunner( + job=job, + task=task, + run_id=run_id, + dag_version_id=dag_version_id, + ) + mapped_kwargs = next(job_runner.expand_input(session=session), None) + unmapped_ti.map_index = 0 + unmapped_ti = job_runner.expand_task(unmapped_ti, mapped_kwargs) + task_instance_mutation_hook(unmapped_ti) + session.merge(unmapped_ti) + session.flush() + all_expanded_tis.append(unmapped_ti) + total_expanded_ti_count = len(all_expanded_tis) + + task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) + + run_job_async(job=job, execute_callable=job_runner._execute, session=session) else: try: total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) @@ -248,7 +254,9 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq indexes_to_map: Iterable[int] = () else: # Only create "missing" ones. - current_max_mapping = get_current_max_mapping(dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session) + current_max_mapping = get_current_max_mapping( + dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session + ) indexes_to_map = range(current_max_mapping + 1, total_length) task.log.info("indexes_to_map: %s", indexes_to_map) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index 6cf901b6769c3..4ccf8c890158d 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -22,12 +22,11 @@ from typing import TYPE_CHECKING, Any import attrs -from airflow.models.xcom import BaseXCom -from airflow.utils.log.logging_mixin import LoggingMixin from sqlalchemy import func, or_, select from sqlalchemy.orm import Session from airflow.models.taskinstance import get_task_instance +from airflow.models.xcom import BaseXCom from airflow.sdk.definitions._internal.types import ArgNotSet from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.xcom_arg import ( @@ -37,6 +36,7 @@ from airflow.sdk.execution_time.comms import XComResult from airflow.sdk.execution_time.xcom import resolve_xcom_backend from airflow.utils.db import exists_query +from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State from airflow.utils.types import NOTSET from airflow.utils.xcom import XCOM_RETURN_KEY @@ -88,10 +88,11 @@ def resolve(self, context: Mapping[str, Any], session: Session) -> Any: ) context = { - **context, **{ + **context, + **{ "task_instance": task_instance, "ti": task_instance, - } + }, } self.log.info("XCom task_instance: %s", task_instance) @@ -107,7 +108,8 @@ def resolve(self, context: Mapping[str, Any], session: Session) -> Any: self.log.debug("xcom_backend: %s", xcom_backend) deserialized_value = xcom_backend.deserialize_value( - XComResult(key=self.operator.output.key, value=value)) + XComResult(key=self.operator.output.key, value=value) + ) self.log.info("deserialized_value: %s", deserialized_value) From c668a8af615e5d99ad71d22fa942f17e11494b93 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 11:31:33 +0200 Subject: [PATCH 005/338] refactor: Added DeferredIterable --- .../src/airflow/models/deferred_iterable.py | 208 ++++++++++++++++++ 1 file changed, 208 insertions(+) create mode 100644 airflow-core/src/airflow/models/deferred_iterable.py diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py new file mode 100644 index 0000000000000..cc2e65ce05879 --- /dev/null +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -0,0 +1,208 @@ +# +# 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 asyncio +from collections.abc import Iterable, Iterator, Sequence, Sized +from contextlib import contextmanager, suppress +from typing import TYPE_CHECKING, Any, Generator + +from airflow.exceptions import AirflowException +from airflow.serialization import serde +from airflow.sdk.bases.operator import BaseOperator as Operator +from airflow.sdk.definitions._internal.mixins import ResolveMixin +from airflow.sdk.definitions.context import Context +from airflow.sdk.definitions.xcom_arg import MapXComArg # noqa: F401 +from airflow.models.xcom_arg import SchedulerXComArg +from airflow.triggers.base import BaseTrigger, TriggerEvent +from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.module_loading import import_string +from airflow.utils.xcom import XCOM_RETURN_KEY +from sqlalchemy.orm import Session +from wrapt import synchronized + +if TYPE_CHECKING: + from asyncio import AbstractEventLoop + + +@contextmanager +def event_loop() -> Generator[AbstractEventLoop, None, None]: + new_event_loop = False + loop = None + try: + try: + loop = asyncio.get_event_loop() + if loop.is_closed(): + raise RuntimeError + except RuntimeError: + loop = asyncio.new_event_loop() + asyncio.set_event_loop(loop) + new_event_loop = True + yield loop + finally: + if new_event_loop and loop is not None: + with suppress(AttributeError): + loop.close() + + +# TODO: Check _run_inline_trigger method from DAG, could be refactored so it uses this method +@synchronized +async def run_trigger(trigger: BaseTrigger) -> TriggerEvent | None: + events = [] + async for event in trigger.run(): + events.append(event) + return next(iter(events), None) + + +class DeferredIterable(Iterator, Sequence, Sized, ResolveMixin, LoggingMixin): + """An iterable that lazily fetches XCom values one by one instead of loading all at once.""" + + def __init__( + self, + results: list[Any] | Any, + trigger: BaseTrigger, + operator: Operator, + next_method: str, + context: Context | None = None, + ): + super().__init__() + self.results = results.copy() if isinstance(results, list) else [results] + self.trigger = trigger + self.operator = operator + self.next_method = next_method + self.context = context + self.index = 0 + + def iter_references(self) -> Iterable[tuple[Operator, str]]: + yield self.operator, XCOM_RETURN_KEY + + def resolve(self, context: Context, session: Session = None, *, include_xcom: bool = True) -> Any: + self.log.info("resolve: %s", self) + return DeferredIterable( + results=self.results, + trigger=self.trigger, + operator=self.operator, + next_method=self.next_method, + context=context, + ) + + def __iter__(self) -> Iterator: + return self + + def __next__(self): + if self.index < len(self.results): + result = self.results[self.index] + self.index += 1 + return result + + if not self.trigger: + raise StopIteration + + self.log.info("No more results. Running trigger: %s", self.trigger) + + if not self.context: + raise AirflowException("Context is required to run the trigger.") + + results = self._execute_trigger() + + if isinstance(results, (list, set)): + self.results.extend(results) + else: + self.results.append(results) + + self.index += 1 + return self.results[-1] + + def _execute_trigger(self): + try: + with event_loop() as loop: + self.log.info("Running trigger: %s", self.trigger) + event = loop.run_until_complete(run_trigger(self.trigger)) + self.operator.render_template_fields(context=self.context) + next_method = getattr(self.operator, self.next_method) + self.log.info("Triggering next method: %s", self.next_method) + results = next_method(self.context, event.payload) + except Exception as e: + self.log.exception(e) + raise AirflowException from e + + if isinstance(results, DeferredIterable): + self.trigger = results.trigger + return results.results + + self.trigger = None + return results + + def __len__(self): + self.log.info("__len__: %s", self) + # TODO: maybe we should raise an exception here as you can't know the total length of an iterable in advance, but won't atm to keep Airflow happy + return len(self.results) + + def __getitem__(self, index: int): + if not (0 <= index < len(self)): + raise IndexError + + return self.results[index] + + def serialize(self): + """Ensure the object is JSON serializable.""" + return { + "results": self.results, + "trigger": self.trigger.serialize() if self.trigger else None, + "dag_fileloc": self.operator.dag.fileloc, + "dag_id": self.operator.dag_id, + "task_id": self.operator.task_id, + "next_method": self.next_method, + } + + @classmethod + def get_operator_from_dag( + cls, dag_fileloc: str, dag_id: str, task_id: str + ) -> Operator: + """Loads a DAG using DagBag and gets the operator by task_id.""" + + from airflow.models import DagBag + + dag_bag = DagBag(collect_dags=False) # Avoid loading all DAGs + dag_bag.process_file(dag_fileloc) + cls.logger().info("dag_bag: %s", dag_bag) + cls.logger().info("dags: %s", dag_bag.dags) + return dag_bag.dags[dag_id].get_task(task_id) + + @classmethod + def deserialize(cls, data: dict, version: int) -> DeferredIterable: + """Ensure the object is JSON deserializable.""" + trigger_class = import_string(data["trigger"][0]) + trigger = trigger_class(**data["trigger"][1]) + operator = cls.get_operator_from_dag( + data["dag_fileloc"], data["dag_id"], data["task_id"] + ) + cls.logger().info("deserialize: %s", operator) + return DeferredIterable( + results=data["results"], + trigger=trigger, + operator=operator, + next_method=data["next_method"], + ) + + +serde._extra_allowed = serde._extra_allowed.union( + { + "infrabel.operators.iterable.DeferredIterable", + } +) From 540a3401757bfe077a655b78c615154d9808fe45 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 11:31:54 +0200 Subject: [PATCH 006/338] refactor: Added license header to TaskExpansionJobRunner --- .../src/airflow/jobs/expand_task_job_runner.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 08c308ad22b3a..188ed441b0898 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from __future__ import annotations from collections.abc import Iterator From f8d62990ca44e223d64d9a9a40050a2cb09637fe Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 4 Jun 2025 17:17:12 +0200 Subject: [PATCH 007/338] refactor: Fixed additional static checks --- .../airflow/jobs/expand_task_job_runner.py | 24 +++++++++---------- airflow-core/src/airflow/jobs/job.py | 4 +--- .../src/airflow/models/deferred_iterable.py | 20 +++++++--------- .../src/airflow/models/expandinput.py | 2 -- .../src/airflow/models/taskinstance.py | 8 +++---- airflow-core/src/airflow/models/taskmap.py | 2 +- airflow-core/src/airflow/models/xcom_arg.py | 2 +- 7 files changed, 26 insertions(+), 36 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 188ed441b0898..9ef7c477e9cde 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -20,26 +20,21 @@ from collections.abc import Iterator from typing import TYPE_CHECKING +from sqlalchemy import select + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import Job from airflow.models import DagRun from airflow.models.dag_version import DagVersion -from airflow.models.expandinput import SchedulerDictOfListsExpandInput -from airflow.models.taskinstance import TaskInstance, get_current_max_mapping, get_task_instance -from airflow.models.xcom_arg import SchedulerPlainXComArg +from airflow.models.taskinstance import TaskInstance, get_current_max_mapping from airflow.policies import task_instance_mutation_hook -from airflow.sdk.bases.operator import BaseOperator -from airflow.sdk.definitions._internal.mixins import ResolveMixin -from airflow.sdk.definitions._internal.types import NOTSET from airflow.sdk.definitions.mappedoperator import MappedOperator -from airflow.sdk.execution_time.comms import XComResult from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string from airflow.utils.session import create_session from airflow.utils.state import DagRunState -from sqlalchemy import select if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -52,8 +47,7 @@ def get_dag_run(dag_id: str, run_id: str, session: Session) -> DagRun | None: Returns the TaskInstance for the task that is being expanded. """ return session.scalars( - select(DagRun) - .where( + select(DagRun).where( DagRun.dag_id == dag_id, DagRun.run_id == run_id, ) @@ -114,9 +108,13 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: if dag_run.state == DagRunState.FAILED: self.log.info("DagRun %s for dag %s has failed, stopping expansion", self.run_id, self.dag_id) - raise AirflowException(f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure.") + raise AirflowException( + f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure." + ) - def _persist_task_instances(self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session) -> None: + def _persist_task_instances( + self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session + ) -> None: if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) @@ -156,7 +154,7 @@ def expand_tasks(self, session: Session) -> int: task_instances_batch.append(self.expand_task(task_instance, mapped_kwargs)) counter += 1 - if len(task_instances_batch) == task_expansion_batch_size: + if len(task_instances_batch) == task_expansion_batch_size: dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index dd17e3c817c8f..221794b3bde11 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -357,9 +357,7 @@ def run_job( job.complete_execution(session=session) -def run_job_async( - job: Job, execute_callable: Callable[[], int | None], session: Session -) -> None: +def run_job_async(job: Job, execute_callable: Callable[[], int | None], session: Session) -> None: """ Run the job asynchronously. diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index cc2e65ce05879..99261386521bb 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -18,23 +18,23 @@ from __future__ import annotations import asyncio -from collections.abc import Iterable, Iterator, Sequence, Sized +from collections.abc import Generator, Iterable, Iterator, Sequence, Sized from contextlib import contextmanager, suppress -from typing import TYPE_CHECKING, Any, Generator +from typing import TYPE_CHECKING, Any + +from sqlalchemy.orm import Session +from wrapt import synchronized from airflow.exceptions import AirflowException -from airflow.serialization import serde from airflow.sdk.bases.operator import BaseOperator as Operator from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions.context import Context from airflow.sdk.definitions.xcom_arg import MapXComArg # noqa: F401 -from airflow.models.xcom_arg import SchedulerXComArg +from airflow.serialization import serde from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string from airflow.utils.xcom import XCOM_RETURN_KEY -from sqlalchemy.orm import Session -from wrapt import synchronized if TYPE_CHECKING: from asyncio import AbstractEventLoop @@ -171,9 +171,7 @@ def serialize(self): } @classmethod - def get_operator_from_dag( - cls, dag_fileloc: str, dag_id: str, task_id: str - ) -> Operator: + def get_operator_from_dag(cls, dag_fileloc: str, dag_id: str, task_id: str) -> Operator: """Loads a DAG using DagBag and gets the operator by task_id.""" from airflow.models import DagBag @@ -189,9 +187,7 @@ def deserialize(cls, data: dict, version: int) -> DeferredIterable: """Ensure the object is JSON deserializable.""" trigger_class = import_string(data["trigger"][0]) trigger = trigger_class(**data["trigger"][1]) - operator = cls.get_operator_from_dag( - data["dag_fileloc"], data["dag_id"], data["task_id"] - ) + operator = cls.get_operator_from_dag(data["dag_fileloc"], data["dag_id"], data["task_id"]) cls.logger().info("deserialize: %s", operator) return DeferredIterable( results=data["results"], diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 4b82c77fc7891..49dc599f919ae 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -121,7 +121,6 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: def resolve( self, context: Mapping[str, Any], session: Session ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - self.log.info("expand_dict: %s", self.value) value = ( @@ -167,7 +166,6 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: def resolve( self, context: Mapping[str, Any], session: Session ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - self.log.info("expand_list: %s", self.value) value = ( diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index dcded7825eafb..8cf511b042cc2 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -2510,10 +2510,10 @@ def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Ses TaskInstance.dag_id == dag_id, TaskInstance.task_id == task_id, TaskInstance.run_id == run_id, - ) - ), - 0, - ) + ) + ), + 0, + ) def _find_common_ancestor_mapped_group(node1: Operator, node2: Operator) -> MappedTaskGroup | None: diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 07df84b6cfd09..8c2bb67dfa392 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -39,7 +39,7 @@ from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion -from airflow.models.taskinstance import get_task_instance, get_current_max_mapping +from airflow.models.taskinstance import get_current_max_mapping, get_task_instance from airflow.utils.db import exists_query from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks from airflow.utils.state import TaskInstanceState diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index 4ccf8c890158d..172790f1e53b2 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -27,12 +27,12 @@ from airflow.models.taskinstance import get_task_instance from airflow.models.xcom import BaseXCom +from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions._internal.types import ArgNotSet from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.xcom_arg import ( XComArg, ) -from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.execution_time.comms import XComResult from airflow.sdk.execution_time.xcom import resolve_xcom_backend from airflow.utils.db import exists_query From e3756a5d979996bc6f6bbff34376f1496d019ab2 Mon Sep 17 00:00:00 2001 From: David Blain Date: Thu, 12 Jun 2025 09:22:20 +0200 Subject: [PATCH 008/338] refactor: Refactored TaskExpansionJobRunner as TaskMap length is now updated within the expand_tasks method instead of ExpandInput --- .../airflow/jobs/expand_task_job_runner.py | 51 ++++++++++++++----- airflow-core/src/airflow/jobs/job.py | 13 +++-- .../src/airflow/models/expandinput.py | 25 ++------- airflow-core/src/airflow/models/taskmap.py | 30 +++++------ airflow-core/src/airflow/models/xcom_arg.py | 14 +---- 5 files changed, 69 insertions(+), 64 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 9ef7c477e9cde..ddd92acf481b2 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -65,11 +65,17 @@ def __init__( dag_version_id: DagVersion, ) -> None: super().__init__(job) + self.job.dag_id = task.dag_id + self.job.job_type = self.job_type self.task = task self.task.operator_class = import_string(f"{task._task_module}.{task._task_type}") self.run_id = run_id self.dag_version_id = dag_version_id + @property + def job_id(self) -> str: + return self.job.id + @property def dag_id(self) -> str: return self.task.dag_id @@ -115,19 +121,29 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: def _persist_task_instances( self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session ) -> None: + from airflow.models.taskmap import update_task_map_length + if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) session.merge(dag_run) + update_task_map_length( + length=task_instances[-1].map_index + 1, + dag_id=self.dag_id, + task_id=self.task_id, + run_id=dag_run.run_id, + session=session, + ) session.flush() session.commit() task_instances.clear() - def expand_tasks(self, session: Session) -> int: + def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, session: Session = NEW_SESSION) -> list[TaskInstance]: """ Expands the task using the provided expand_input. """ - counter = 0 + from airflow.models.taskinstance import get_task_instance + max_map_index = get_current_max_mapping( dag_id=self.dag_id, task_id=self.task_id, @@ -135,24 +151,33 @@ def expand_tasks(self, session: Session) -> int: session=session, ) dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + unmapped_ti = get_task_instance(dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session) self.log.info("expand_tasks: %s", session) self.log.info("max_map_index: %s", max_map_index) self.log.info("dag_version_id: %s", self.dag_version_id) self.log.info("dag_run: %s", dag_run) + task_instances = [] task_instances_batch = [] - for map_index, mapped_kwargs in enumerate(self.expand_input(session=session)): + for map_index, mapped_kwargs in enumerate(expand_input): if map_index > max_map_index: - task_instance = TaskInstance( - task=self.task, - run_id=self.run_id, - map_index=map_index, - dag_version_id=self.dag_version_id, - ) - task_instances_batch.append(self.expand_task(task_instance, mapped_kwargs)) - counter += 1 + if map_index == 0 and unmapped_ti: + task_instance = unmapped_ti + task_instance.map_index = map_index + else: + task_instance = TaskInstance( + task=self.task, + run_id=self.run_id, + map_index=map_index, + dag_version_id=self.dag_version_id, + ) + if job_id: + task_instance.queued_by_job_id = job_id + task_instance = self.expand_task(task_instance, mapped_kwargs) + task_instances.append(task_instance) + task_instances_batch.append(task_instance) if len(task_instances_batch) == task_expansion_batch_size: dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) @@ -161,8 +186,8 @@ def expand_tasks(self, session: Session) -> int: self._persist_task_instances(dag_run, task_instances_batch, session=session) - return counter + return task_instances def _execute(self) -> int | None: with create_session() as session: - return self.expand_tasks(session=session) + return len(self.expand_tasks(expand_input=self.expand_input(session=session), job_id=self.job_id, session=session)) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 221794b3bde11..45077104d4332 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import asyncio from functools import cached_property, lru_cache from multiprocessing import Process from time import sleep @@ -356,8 +357,10 @@ def run_job( finally: job.complete_execution(session=session) - -def run_job_async(job: Job, execute_callable: Callable[[], int | None], session: Session) -> None: +@provide_session +def run_job_async( + job: Job, execute_callable: Callable[[], int | None], session: Session = NEW_SESSION +) -> int | None: """ Run the job asynchronously. @@ -366,9 +369,13 @@ def run_job_async(job: Job, execute_callable: Callable[[], int | None], session: :meta private: """ + def execute_async_job() -> int | None: + asyncio.set_event_loop(asyncio.new_event_loop()) + return execute_job(job, execute_callable) + job.prepare_for_execution(session=session) try: - process = Process(target=execute_job, args=(job, execute_callable)) + process = Process(target=execute_async_job) process.start() finally: job.complete_execution(session=session) diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 49dc599f919ae..e9c2d65b81e5d 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -24,7 +24,6 @@ import attrs -from airflow.models.taskmap import update_task_map_length from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: @@ -121,24 +120,18 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: def resolve( self, context: Mapping[str, Any], session: Session ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - self.log.info("expand_dict: %s", self.value) - value = ( - self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value + self.value.resolve(context, session) if _needs_run_time_resolution(self.value) else self.value ) - self.log.info("resolved value: %s", value) + self.log.debug("resolved value: %s", value) for key, item in value.items(): - result = item.resolve(context, session) - - self.log.info("resolved value %s: %s", key, result) + result = item.resolve(context, session) if _needs_run_time_resolution(item) else item for index, sub_item in enumerate(result): yield {key: sub_item} - update_task_map_length(index, item, context["run_id"], session) - return [] @@ -166,27 +159,19 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: def resolve( self, context: Mapping[str, Any], session: Session ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - self.log.info("expand_list: %s", self.value) - value = ( self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value ) - self.log.info("resolved value: %s", value) + self.log.debug("resolved value: %s", value) for index, entry in enumerate(value): - self.log.info("entry: %s", entry) - for key, item in entry.items(): - if isinstance(item, SchedulerXComArg): + if _needs_run_time_resolution(item): entry[key] = item.resolve(context, session) - self.log.info("resolved entry: %s", entry) - yield entry - update_task_map_length(index, self.value, context["run_id"], session) - return [] diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 8c2bb67dfa392..a546372d1b4af 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -297,20 +297,18 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq session.flush() return all_expanded_tis, total_expanded_ti_count - 1 - -def update_task_map_length(index, item, run_id, session): - try: - length = index + 1 - logging.info("Persisting TaskMap length: %s", length) - session.execute( - update(TaskMap) - .where( - TaskMap.dag_id == item.operator.dag_id, - TaskMap.task_id == item.operator.task_id, - TaskMap.run_id == run_id, - TaskMap.map_index == -1, - ) - .values(length=length) +# TODO: ths is a hack to update the TaskMap length after some task instances have been created while some of +# of them are already being executed and other one are still being expanded asynchronously. This is to prevent +# for already expanded tasks to be removed by the scheduler. +def update_task_map_length(length: int, dag_id: str, task_id: str, run_id: str, session: Session): + logging.info("Persisting TaskMap length: %s", length) + session.execute( + update(TaskMap) + .where( + TaskMap.dag_id == dag_id, + TaskMap.task_id == task_id, + TaskMap.run_id == run_id, + TaskMap.map_index == -1, ) - except: - logging.exception("Persisting TaskMap length failed for task %s", item.operator.task_id) + .values(length=length) + ) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index 172790f1e53b2..29c008d28daa2 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -95,8 +95,6 @@ def resolve(self, context: Mapping[str, Any], session: Session) -> Any: }, } - self.log.info("XCom task_instance: %s", task_instance) - value = task_instance.xcom_pull( task_ids=self.operator.task_id, key=self.operator.output.key, @@ -104,19 +102,15 @@ def resolve(self, context: Mapping[str, Any], session: Session) -> Any: session=session, ) - self.log.info("value: %s", value) - self.log.debug("xcom_backend: %s", xcom_backend) - deserialized_value = xcom_backend.deserialize_value( XComResult(key=self.operator.output.key, value=value) ) - self.log.info("deserialized_value: %s", deserialized_value) + self.log.debug("deserialized_value: %s", deserialized_value) if isinstance(deserialized_value, ResolveMixin): - self.log.info("context: %s", context) deserialized_value = deserialized_value.resolve(context) - self.log.info("resolved_value: %s", deserialized_value) + self.log.debug("resolved_value: %s", deserialized_value) return deserialized_value @@ -138,18 +132,14 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: return cls(deserialize_xcom_arg(data["arg"], dag), data["callables"]) def resolve(self, context: Mapping[str, Any], session: Session) -> Any: - self.log.info("arg: %s", self.arg) resolved_arg = self.arg.resolve(context, session) - self.log.info("resolved_arg: %s", resolved_arg) def apply(arg: Any): for index, _callable in enumerate(self.callables): if isinstance(_callable, str): _callable = eval(_callable) self.callables[index] = _callable - self.log.debug("arg: %s", arg) arg = _callable(arg) - self.log.debug("apply: %s", arg) return arg return map(apply, resolved_arg) From 3176939d8986264f94095ac74f95ea9bcfee1fad Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 08:32:49 +0200 Subject: [PATCH 009/338] refactor: Remove some logging statements and changed logging level for others in DeferredIterable --- airflow-core/src/airflow/models/deferred_iterable.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index 99261386521bb..784b4354cfb5b 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -92,7 +92,7 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: yield self.operator, XCOM_RETURN_KEY def resolve(self, context: Context, session: Session = None, *, include_xcom: bool = True) -> Any: - self.log.info("resolve: %s", self) + self.log.debug("resolve: %s", self) return DeferredIterable( results=self.results, trigger=self.trigger, @@ -131,11 +131,11 @@ def __next__(self): def _execute_trigger(self): try: with event_loop() as loop: - self.log.info("Running trigger: %s", self.trigger) + self.log.debug("Running trigger: %s", self.trigger) event = loop.run_until_complete(run_trigger(self.trigger)) self.operator.render_template_fields(context=self.context) next_method = getattr(self.operator, self.next_method) - self.log.info("Triggering next method: %s", self.next_method) + self.log.debug("Triggering next method: %s", self.next_method) results = next_method(self.context, event.payload) except Exception as e: self.log.exception(e) @@ -178,8 +178,6 @@ def get_operator_from_dag(cls, dag_fileloc: str, dag_id: str, task_id: str) -> O dag_bag = DagBag(collect_dags=False) # Avoid loading all DAGs dag_bag.process_file(dag_fileloc) - cls.logger().info("dag_bag: %s", dag_bag) - cls.logger().info("dags: %s", dag_bag.dags) return dag_bag.dags[dag_id].get_task(task_id) @classmethod From 441eb69132717fc732514659e6c0fb76ff583a11 Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 08:37:27 +0200 Subject: [PATCH 010/338] refactor: Removed commit statement and added missing import for NEW_SESSION in TaskExpansionJobRunner --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index ddd92acf481b2..59bc55d4c14f8 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -33,7 +33,7 @@ from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string -from airflow.utils.session import create_session +from airflow.utils.session import create_session, NEW_SESSION from airflow.utils.state import DagRunState if TYPE_CHECKING: @@ -135,7 +135,6 @@ def _persist_task_instances( session=session, ) session.flush() - session.commit() task_instances.clear() def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, session: Session = NEW_SESSION) -> list[TaskInstance]: From e0ec3754fb12320f2b6bb44ee0972e72fa959e1d Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 09:49:25 +0200 Subject: [PATCH 011/338] refactor: Refactored new TaskInstance and TaskMap methods as class methods --- .../airflow/jobs/expand_task_job_runner.py | 15 +++--- .../src/airflow/models/taskinstance.py | 54 +++++++++---------- airflow-core/src/airflow/models/taskmap.py | 35 ++++++------ 3 files changed, 53 insertions(+), 51 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 59bc55d4c14f8..99e7ba47a3181 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -28,7 +28,7 @@ from airflow.jobs.job import Job from airflow.models import DagRun from airflow.models.dag_version import DagVersion -from airflow.models.taskinstance import TaskInstance, get_current_max_mapping +from airflow.models.taskinstance import TaskInstance from airflow.policies import task_instance_mutation_hook from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.utils.log.logging_mixin import LoggingMixin @@ -121,13 +121,16 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: def _persist_task_instances( self, dag_run: DagRun, task_instances: list[TaskInstance], session: Session ) -> None: - from airflow.models.taskmap import update_task_map_length + """ + Expands the task using the provided expand_input. + """ + from airflow.models.taskmap import TaskMap if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) session.merge(dag_run) - update_task_map_length( + TaskMap.update_task_map_length( length=task_instances[-1].map_index + 1, dag_id=self.dag_id, task_id=self.task_id, @@ -141,16 +144,14 @@ def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, """ Expands the task using the provided expand_input. """ - from airflow.models.taskinstance import get_task_instance - - max_map_index = get_current_max_mapping( + max_map_index = TaskInstance.get_current_max_mapping( dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session, ) dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) - unmapped_ti = get_task_instance(dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session) + unmapped_ti = TaskInstance.get_task_instance(dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session) self.log.info("expand_tasks: %s", session) self.log.info("max_map_index: %s", max_map_index) diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index 8cf511b042cc2..a2fe2aee3e59f 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -2485,35 +2485,35 @@ def duration_expression_update( } ) + @classmethod + def get_task_instance( + cls, dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION + ) -> TaskInstance | None: + @cache + def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): + return session.scalars( + select(TaskInstance).where( + TaskInstance.dag_id == _dag_id, + TaskInstance.task_id == _task_id, + TaskInstance.run_id == _run_id, + TaskInstance.map_index == _map_index, + ) + ).one_or_none() -def get_task_instance( - dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION -) -> TaskInstance | None: - @cache - def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): - return session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == _dag_id, - TaskInstance.task_id == _task_id, - TaskInstance.run_id == _run_id, - TaskInstance.map_index == _map_index, - ) - ).one_or_none() - - return find_task_instance(dag_id, task_id, run_id, map_index) - + return find_task_instance(dag_id, task_id, run_id, map_index) -def get_current_max_mapping(dag_id: str, task_id: str, run_id: str, session: Session) -> int: - return max( - session.scalar( - select(func.max(TaskInstance.map_index)).where( - TaskInstance.dag_id == dag_id, - TaskInstance.task_id == task_id, - TaskInstance.run_id == run_id, - ) - ), - 0, - ) + @classmethod + def get_current_max_mapping(cls, dag_id: str, task_id: str, run_id: str, session: Session) -> int: + return max( + session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == dag_id, + TaskInstance.task_id == task_id, + TaskInstance.run_id == run_id, + ) + ), + 0, + ) def _find_common_ancestor_mapped_group(node1: Operator, node2: Operator) -> MappedTaskGroup | None: diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index a546372d1b4af..8362bf8d2ce32 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -39,7 +39,7 @@ from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion -from airflow.models.taskinstance import get_current_max_mapping, get_task_instance +from airflow.models.taskinstance import TaskInstance from airflow.utils.db import exists_query from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks from airflow.utils.state import TaskInstanceState @@ -153,7 +153,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" ) - unmapped_ti = get_task_instance( + unmapped_ti = TaskInstance.get_task_instance( dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session ) @@ -254,7 +254,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq indexes_to_map: Iterable[int] = () else: # Only create "missing" ones. - current_max_mapping = get_current_max_mapping( + current_max_mapping = TaskInstance.get_current_max_mapping( dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session ) indexes_to_map = range(current_max_mapping + 1, total_length) @@ -297,18 +297,19 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq session.flush() return all_expanded_tis, total_expanded_ti_count - 1 -# TODO: ths is a hack to update the TaskMap length after some task instances have been created while some of -# of them are already being executed and other one are still being expanded asynchronously. This is to prevent -# for already expanded tasks to be removed by the scheduler. -def update_task_map_length(length: int, dag_id: str, task_id: str, run_id: str, session: Session): - logging.info("Persisting TaskMap length: %s", length) - session.execute( - update(TaskMap) - .where( - TaskMap.dag_id == dag_id, - TaskMap.task_id == task_id, - TaskMap.run_id == run_id, - TaskMap.map_index == -1, + # TODO: ths is a hack to update the TaskMap length after some task instances have been created while some of + # of them are already being executed and other one are still being expanded asynchronously. This is to prevent + # for already expanded tasks to be removed by the scheduler. + @classmethod + def update_task_map_length(cls, length: int, dag_id: str, task_id: str, run_id: str, session: Session): + logging.info("Persisting TaskMap length: %s", length) + session.execute( + update(TaskMap) + .where( + TaskMap.dag_id == dag_id, + TaskMap.task_id == task_id, + TaskMap.run_id == run_id, + TaskMap.map_index == -1, + ) + .values(length=length) ) - .values(length=length) - ) From 3873e80c6bdebeafb8fe6b69e4ea4492abe41322 Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 09:59:19 +0200 Subject: [PATCH 012/338] refactor: Refactored get_run_dag as classmethod of DagRun --- .../src/airflow/jobs/expand_task_job_runner.py | 18 ++---------------- airflow-core/src/airflow/models/dagrun.py | 9 +++++++++ 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 99e7ba47a3181..9e88f930f11bd 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -20,8 +20,6 @@ from collections.abc import Iterator from typing import TYPE_CHECKING -from sqlalchemy import select - from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.jobs.base_job_runner import BaseJobRunner @@ -42,18 +40,6 @@ task_expansion_batch_size = conf.getint("scheduler", "task_expansion_batch_size", fallback=10) -def get_dag_run(dag_id: str, run_id: str, session: Session) -> DagRun | None: - """ - Returns the TaskInstance for the task that is being expanded. - """ - return session.scalars( - select(DagRun).where( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - ).one_or_none() - - class TaskExpansionJobRunner(BaseJobRunner, LoggingMixin): job_type = "TaskExpansionJob" @@ -150,7 +136,7 @@ def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, run_id=self.run_id, session=session, ) - dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + dag_run = DagRun.get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) unmapped_ti = TaskInstance.get_task_instance(dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session) self.log.info("expand_tasks: %s", session) @@ -180,7 +166,7 @@ def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, task_instances_batch.append(task_instance) if len(task_instances_batch) == task_expansion_batch_size: - dag_run = get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + dag_run = DagRun.get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index 8dcd00fef0cbc..b07c19083d9e3 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -2009,6 +2009,15 @@ def _get_log_template(log_template_id: int | None, session: Session = NEW_SESSIO def _get_partial_task_ids(dag: DAG | None) -> list[str] | None: return dag.task_ids if dag and dag.partial else None + @classmethod + def get_dag_run(cls, dag_id: str, run_id: str, session: Session) -> DagRun | None: + return session.scalars( + select(DagRun).where( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + ).one_or_none() + class DagRunNote(Base): """For storage of arbitrary notes concerning the dagrun instance.""" From 5876443fde6e454c74f5374f2d8613f86605c289 Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 12:19:34 +0200 Subject: [PATCH 013/338] refactor: Removed duplicate get_task_instance method --- .../src/airflow/jobs/expand_task_job_runner.py | 14 ++++++++------ airflow-core/src/airflow/models/taskinstance.py | 17 ----------------- 2 files changed, 8 insertions(+), 23 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 9e88f930f11bd..d7cb0c3c44b2e 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -126,18 +126,20 @@ def _persist_task_instances( session.flush() task_instances.clear() - def expand_tasks(self, expand_input: Iterator[dict], job_id: str | None = None, session: Session = NEW_SESSION) -> list[TaskInstance]: + def expand_tasks( + self, expand_input: Iterator[dict], job_id: str | None = None, session: Session = NEW_SESSION + ) -> list[TaskInstance]: """ Expands the task using the provided expand_input. """ + max_map_index = TaskInstance.get_current_max_mapping( - dag_id=self.dag_id, - task_id=self.task_id, - run_id=self.run_id, - session=session, + dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session ) dag_run = DagRun.get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) - unmapped_ti = TaskInstance.get_task_instance(dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session) + unmapped_ti = TaskInstance.get_task_instance( + dag_id=self.dag_id, run_id=self.run_id, task_id=self.task_id, map_index=-1, session=session + ) self.log.info("expand_tasks: %s", session) self.log.info("max_map_index: %s", max_map_index) diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index a2fe2aee3e59f..4b2ba33239f3b 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -2485,23 +2485,6 @@ def duration_expression_update( } ) - @classmethod - def get_task_instance( - cls, dag_id: str, task_id: str, run_id: str, map_index: int = -1, session: Session = NEW_SESSION - ) -> TaskInstance | None: - @cache - def find_task_instance(_dag_id: str, _task_id: str, _run_id: str, _map_index: int): - return session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == _dag_id, - TaskInstance.task_id == _task_id, - TaskInstance.run_id == _run_id, - TaskInstance.map_index == _map_index, - ) - ).one_or_none() - - return find_task_instance(dag_id, task_id, run_id, map_index) - @classmethod def get_current_max_mapping(cls, dag_id: str, task_id: str, run_id: str, session: Session) -> int: return max( From f2c3e9f22dddf064101e321071ec9069d135a32e Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 13:51:48 +0200 Subject: [PATCH 014/338] refactor: Moved import TaskInstance in expand_tasks method and type checking block --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index d7cb0c3c44b2e..78cb99651215b 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -26,7 +26,6 @@ from airflow.jobs.job import Job from airflow.models import DagRun from airflow.models.dag_version import DagVersion -from airflow.models.taskinstance import TaskInstance from airflow.policies import task_instance_mutation_hook from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.utils.log.logging_mixin import LoggingMixin @@ -37,6 +36,8 @@ if TYPE_CHECKING: from sqlalchemy.orm import Session + from airflow.models.taskinstance import TaskInstance + task_expansion_batch_size = conf.getint("scheduler", "task_expansion_batch_size", fallback=10) @@ -133,6 +134,8 @@ def expand_tasks( Expands the task using the provided expand_input. """ + from airflow.models.taskinstance import TaskInstance + max_map_index = TaskInstance.get_current_max_mapping( dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session ) From 23658a287838ec4db909b860e61d9b15e5aa5307 Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 13:56:21 +0200 Subject: [PATCH 015/338] refactor: Moved import TaskInstance in expand_mapped_task method and type checking block --- airflow-core/src/airflow/models/taskmap.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 8362bf8d2ce32..396cc978c859a 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -39,7 +39,6 @@ from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion -from airflow.models.taskinstance import TaskInstance from airflow.utils.db import exists_query from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks from airflow.utils.state import TaskInstanceState @@ -144,6 +143,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq """ from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated + from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.settings import task_instance_mutation_hook From 2c7f1b0ce0f36dca9dc3658d467cfd852cbf4b67 Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 14:50:07 +0200 Subject: [PATCH 016/338] refactor: Moved import TaskExpansionJobRunner in expand_mapped_task method to try avoiding circular import TaskInstance --- airflow-core/src/airflow/models/taskmap.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 396cc978c859a..31dd7a094314a 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -35,7 +35,6 @@ update, ) -from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion @@ -141,6 +140,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq :return: The newly created mapped task instances (if any) in ascending order by map index, and the maximum map index value. """ + from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance From 5fc723e62e3f4d21b9fe45bc088415534f97616e Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 13 Jun 2025 15:12:51 +0200 Subject: [PATCH 017/338] refactor: Fixed import of get_task_instance in xcom_arg module --- airflow-core/src/airflow/models/xcom_arg.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index 29c008d28daa2..e558668df498c 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -25,7 +25,7 @@ from sqlalchemy import func, or_, select from sqlalchemy.orm import Session -from airflow.models.taskinstance import get_task_instance +from airflow.models.taskinstance import TaskInstance from airflow.models.xcom import BaseXCom from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions._internal.types import ArgNotSet @@ -80,7 +80,7 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: return cls(dag.get_task(data["task_id"]), data["key"]) def resolve(self, context: Mapping[str, Any], session: Session) -> Any: - task_instance = get_task_instance( + task_instance = TaskInstance.get_task_instance( dag_id=self.operator.dag_id, task_id=self.operator.task_id, run_id=context["run_id"], From 56655b6d35c2b34bf54367e194ceed59c46c5b00 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 18 Jun 2025 14:02:23 +0200 Subject: [PATCH 018/338] refactor: Make sure TaskExpansionJobRunner uses a non scoped session to avoid CommitProhibitorGuard as it has to run in a separate process outside the scheduler --- .../src/airflow/jobs/base_job_runner.py | 2 +- .../airflow/jobs/expand_task_job_runner.py | 31 ++++++++++-------- airflow-core/src/airflow/jobs/job.py | 32 +++++++++++-------- airflow-core/src/airflow/models/taskmap.py | 19 ++++++----- airflow-core/src/airflow/models/xcom_arg.py | 24 ++------------ .../microsoft/azure/operators/msgraph.py | 17 ++++++++-- task-sdk/src/airflow/sdk/bases/operator.py | 3 ++ 7 files changed, 67 insertions(+), 61 deletions(-) diff --git a/airflow-core/src/airflow/jobs/base_job_runner.py b/airflow-core/src/airflow/jobs/base_job_runner.py index 05671e2050a2f..4c5c3cddcee10 100644 --- a/airflow-core/src/airflow/jobs/base_job_runner.py +++ b/airflow-core/src/airflow/jobs/base_job_runner.py @@ -42,7 +42,7 @@ def __init__(self, job: Job) -> None: job.job_type = self.job_type self.job: Job = job - def _execute(self) -> int | None: + def _execute(self, session: Session = NEW_SESSION) -> int | None: """ Execute the logic connected to the runner. diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 78cb99651215b..211e2ee6263d0 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -28,6 +28,7 @@ from airflow.models.dag_version import DagVersion from airflow.policies import task_instance_mutation_hook from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.utils.context import Context from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string from airflow.utils.session import create_session, NEW_SESSION @@ -48,15 +49,14 @@ def __init__( self, job: Job, task: MappedOperator, - run_id: str, + context: Context, dag_version_id: DagVersion, ) -> None: super().__init__(job) - self.job.dag_id = task.dag_id + task.operator_class = import_string(f"{task._task_module}.{task._task_type}") + self.context = {**context, **{"task": task}} + self.job.dag_id = self.dag_id self.job.job_type = self.job_type - self.task = task - self.task.operator_class = import_string(f"{task._task_module}.{task._task_type}") - self.run_id = run_id self.dag_version_id = dag_version_id @property @@ -65,19 +65,23 @@ def job_id(self) -> str: @property def dag_id(self) -> str: - return self.task.dag_id + return self.context["dag"].dag_id + + @property + def task(self) -> MappedOperator: + return self.context["task"] @property def task_id(self) -> str: return self.task.task_id + @property + def run_id(self) -> str: + return self.context["run_id"] + def expand_input(self, session: Session) -> Iterator[dict]: self.log.info("expand_input: %s", self.task.expand_input) - context = { - "task": self.task, - "run_id": self.run_id, - } - return iter(self.task.expand_input.resolve(context, session)) + return iter(self.task.expand_input.resolve(self.context, session)) def expand_task(self, task_instance: TaskInstance, mapped_kwargs) -> TaskInstance: self.log.info("expand task: %s", task_instance.map_index) @@ -179,6 +183,5 @@ def expand_tasks( return task_instances - def _execute(self) -> int | None: - with create_session() as session: - return len(self.expand_tasks(expand_input=self.expand_input(session=session), job_id=self.job_id, session=session)) + def _execute(self, session: Session) -> int | None: + return len(self.expand_tasks(expand_input=self.expand_input(session=session), job_id=self.job_id, session=session)) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 45077104d4332..3efddb9b65126 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -21,7 +21,7 @@ from functools import cached_property, lru_cache from multiprocessing import Process from time import sleep -from typing import TYPE_CHECKING, Callable, NoReturn +from typing import TYPE_CHECKING, Callable, NoReturn, Union from sqlalchemy import Column, Index, Integer, String, case, select from sqlalchemy.exc import OperationalError @@ -357,31 +357,35 @@ def run_job( finally: job.complete_execution(session=session) -@provide_session + def run_job_async( - job: Job, execute_callable: Callable[[], int | None], session: Session = NEW_SESSION + job: Job, execute_callable: Union[Callable[[], int | None],Callable[[Session], int | None]] ) -> int | None: """ Run the job asynchronously. The Job is always an ORM object and setting the state is happening within the - same DB session and the session is kept open throughout the whole execution. + same DB session and the session is kept open throughout the whole asynchronous execution. :meta private: """ def execute_async_job() -> int | None: asyncio.set_event_loop(asyncio.new_event_loop()) - return execute_job(job, execute_callable) - job.prepare_for_execution(session=session) - try: - process = Process(target=execute_async_job) - process.start() - finally: - job.complete_execution(session=session) + with create_session(scoped=False) as session: + job.prepare_for_execution(session=session) + try: + return execute_job(job, execute_callable, session) + finally: + job.complete_execution(session=session) + process = Process(target=execute_async_job) + process.start() -def execute_job(job: Job, execute_callable: Callable[[], int | None]) -> int | None: + +def execute_job( + job: Job, execute_callable: Union[Callable[[], int | None],Callable[[Session], int | None]], session: Session | None = None +) -> int | None: """ Execute the job. @@ -401,11 +405,13 @@ def execute_job(job: Job, execute_callable: Callable[[], int | None]) -> int | N :param execute_callable: callable to execute when running the job. + :param session: Optional session to use (new since AIP-88). + :meta private: """ ret = None try: - ret = execute_callable() + ret = execute_callable(session) if session else execute_callable() # In case of max runs or max duration job.state = JobState.SUCCESS except SystemExit: diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 31dd7a094314a..47153ccac9ff0 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -23,6 +23,7 @@ import enum import logging from collections.abc import Collection, Iterable, Sequence +from itertools import islice from typing import TYPE_CHECKING, Any from sqlalchemy import ( @@ -140,7 +141,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq :return: The newly created mapped task instances (if any) in ascending order by map index, and the maximum map index value. """ - from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner + from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner, task_expansion_batch_size from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance @@ -171,25 +172,23 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq all_expanded_tis: list[TaskInstance] = [] if isinstance(task, MappedOperator) and unmapped_ti: + context = unmapped_ti.get_template_context(session=session) job = Job() job_runner = TaskExpansionJobRunner( job=job, task=task, - run_id=run_id, + context=context, dag_version_id=dag_version_id, ) - mapped_kwargs = next(job_runner.expand_input(session=session), None) - unmapped_ti.map_index = 0 - unmapped_ti = job_runner.expand_task(unmapped_ti, mapped_kwargs) - task_instance_mutation_hook(unmapped_ti) - session.merge(unmapped_ti) - session.flush() - all_expanded_tis.append(unmapped_ti) + all_expanded_tis.extend(job_runner.expand_tasks( + expand_input=islice(job_runner.expand_input(session=session), task_expansion_batch_size), + session=session) + ) total_expanded_ti_count = len(all_expanded_tis) task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) - run_job_async(job=job, execute_callable=job_runner._execute, session=session) + run_job_async(job=job, execute_callable=job_runner._execute) else: try: total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index e558668df498c..ef41123f97b33 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -22,10 +22,6 @@ from typing import TYPE_CHECKING, Any import attrs -from sqlalchemy import func, or_, select -from sqlalchemy.orm import Session - -from airflow.models.taskinstance import TaskInstance from airflow.models.xcom import BaseXCom from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions._internal.types import ArgNotSet @@ -40,6 +36,8 @@ from airflow.utils.state import State from airflow.utils.types import NOTSET from airflow.utils.xcom import XCOM_RETURN_KEY +from sqlalchemy import func, or_, select +from sqlalchemy.orm import Session __all__ = ["XComArg", "get_task_map_length"] xcom_backend: BaseXCom = resolve_xcom_backend() @@ -80,25 +78,9 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: return cls(dag.get_task(data["task_id"]), data["key"]) def resolve(self, context: Mapping[str, Any], session: Session) -> Any: - task_instance = TaskInstance.get_task_instance( - dag_id=self.operator.dag_id, - task_id=self.operator.task_id, - run_id=context["run_id"], - session=session, - ) - - context = { - **context, - **{ - "task_instance": task_instance, - "ti": task_instance, - }, - } - - value = task_instance.xcom_pull( + value = context["ti"].xcom_pull( task_ids=self.operator.task_id, key=self.operator.output.key, - map_indexes=task_instance.map_index, session=session, ) diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py index 31d342ccc143a..a118cdbba1555 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py @@ -34,6 +34,8 @@ MSGraphTrigger, ResponseSerializer, ) + +from airflow.models.deferred_iterable import DeferredIterable from airflow.utils.xcom import XCOM_RETURN_KEY if TYPE_CHECKING: @@ -230,14 +232,25 @@ def execute_complete( self.trigger_next_link( response=response, method_name=self.execute_complete.__name__, context=context ) - except TaskDeferred as exception: + except TaskDeferred as task_deferred: + self.log.debug("streaming: %s", self.streaming) + + if self.streaming: + return DeferredIterable( + results=result, + trigger=task_deferred.trigger, + operator=self, + next_method=self.execute_complete.__name__, + context=context, + ) + self.append_result( results=results, result=result, append_result_as_list_if_absent=True, ) self.push_xcom(context=context, value=results) - raise exception + raise task_deferred if not results: return result diff --git a/task-sdk/src/airflow/sdk/bases/operator.py b/task-sdk/src/airflow/sdk/bases/operator.py index 3f74694aaa7c4..fb4130302ea23 100644 --- a/task-sdk/src/airflow/sdk/bases/operator.py +++ b/task-sdk/src/airflow/sdk/bases/operator.py @@ -857,6 +857,7 @@ def say_hello_world(**context): _task_display_name: str | None = None logger_name: str | None = None allow_nested_operators: bool = True + streaming: bool = False is_setup: bool = False is_teardown: bool = False @@ -1010,6 +1011,7 @@ def __init__( task_display_name: str | None = None, logger_name: str | None = None, allow_nested_operators: bool = True, + streaming: bool = False, **kwargs: Any, ): # Note: Metaclass handles passing in the DAG/TaskGroup from active context manager, if any @@ -1140,6 +1142,7 @@ def __init__( self._task_display_name = task_display_name self.allow_nested_operators = allow_nested_operators + self.streaming = streaming self._logger_name = logger_name From 1c5eb869775a29f93cc02f3e6b579ccba5bc20e2 Mon Sep 17 00:00:00 2001 From: David Blain Date: Thu, 19 Jun 2025 09:43:40 +0200 Subject: [PATCH 019/338] refactor: Fixed static checks --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 9 ++++++--- airflow-core/src/airflow/jobs/job.py | 6 ++++-- airflow-core/src/airflow/models/expandinput.py | 4 +--- airflow-core/src/airflow/models/taskmap.py | 8 +++++--- airflow-core/src/airflow/models/xcom_arg.py | 5 +++-- .../providers/microsoft/azure/operators/msgraph.py | 3 +-- 6 files changed, 20 insertions(+), 15 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 211e2ee6263d0..27ec5745bedc1 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -31,7 +31,7 @@ from airflow.utils.context import Context from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string -from airflow.utils.session import create_session, NEW_SESSION +from airflow.utils.session import NEW_SESSION from airflow.utils.state import DagRunState if TYPE_CHECKING: @@ -137,7 +137,6 @@ def expand_tasks( """ Expands the task using the provided expand_input. """ - from airflow.models.taskinstance import TaskInstance max_map_index = TaskInstance.get_current_max_mapping( @@ -184,4 +183,8 @@ def expand_tasks( return task_instances def _execute(self, session: Session) -> int | None: - return len(self.expand_tasks(expand_input=self.expand_input(session=session), job_id=self.job_id, session=session)) + return len( + self.expand_tasks( + expand_input=self.expand_input(session=session), job_id = self.job_id, session = session + ) + ) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 3efddb9b65126..438aead8cbab0 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -359,7 +359,7 @@ def run_job( def run_job_async( - job: Job, execute_callable: Union[Callable[[], int | None],Callable[[Session], int | None]] + job: Job, execute_callable: Union[Callable[[], int | None], Callable[[Session], int | None]] ) -> int | None: """ Run the job asynchronously. @@ -384,7 +384,9 @@ def execute_async_job() -> int | None: def execute_job( - job: Job, execute_callable: Union[Callable[[], int | None],Callable[[Session], int | None]], session: Session | None = None + job: Job, + execute_callable: Union[Callable[[], int | None], Callable[[Session], int | None]], + session: Session | None = None, ) -> int | None: """ Execute the job. diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index e9c2d65b81e5d..2cf6e2d0ffd41 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -159,9 +159,7 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: def resolve( self, context: Mapping[str, Any], session: Session ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - value = ( - self.value.resolve(context, session) if isinstance(self.value, SchedulerXComArg) else self.value - ) + value = self.value.resolve(context, session) if _needs_run_time_resolution(self.value) else self.value self.log.debug("resolved value: %s", value) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 47153ccac9ff0..aca3d0619c32d 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -180,9 +180,11 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq context=context, dag_version_id=dag_version_id, ) - all_expanded_tis.extend(job_runner.expand_tasks( - expand_input=islice(job_runner.expand_input(session=session), task_expansion_batch_size), - session=session) + all_expanded_tis.extend( + job_runner.expand_tasks( + expand_input=islice(job_runner.expand_input(session=session), task_expansion_batch_size), + session=session, + ) ) total_expanded_ti_count = len(all_expanded_tis) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index ef41123f97b33..6f0705e1ff1d4 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -22,6 +22,9 @@ from typing import TYPE_CHECKING, Any import attrs +from sqlalchemy import func, or_, select +from sqlalchemy.orm import Session + from airflow.models.xcom import BaseXCom from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions._internal.types import ArgNotSet @@ -36,8 +39,6 @@ from airflow.utils.state import State from airflow.utils.types import NOTSET from airflow.utils.xcom import XCOM_RETURN_KEY -from sqlalchemy import func, or_, select -from sqlalchemy.orm import Session __all__ = ["XComArg", "get_task_map_length"] xcom_backend: BaseXCom = resolve_xcom_backend() diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py index a118cdbba1555..315cedcd5d484 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/msgraph.py @@ -29,13 +29,12 @@ from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, TaskDeferred from airflow.models import BaseOperator +from airflow.models.deferred_iterable import DeferredIterable from airflow.providers.microsoft.azure.hooks.msgraph import KiotaRequestAdapterHook from airflow.providers.microsoft.azure.triggers.msgraph import ( MSGraphTrigger, ResponseSerializer, ) - -from airflow.models.deferred_iterable import DeferredIterable from airflow.utils.xcom import XCOM_RETURN_KEY if TYPE_CHECKING: From 8b393f01c48c643c7340850eaf6cd850f2c99af2 Mon Sep 17 00:00:00 2001 From: David Blain Date: Thu, 19 Jun 2025 15:50:37 +0200 Subject: [PATCH 020/338] refactor: Task expansion is now fully done in subprocess --- .../airflow/jobs/expand_task_job_runner.py | 12 +++++++++- airflow-core/src/airflow/jobs/job.py | 4 ++-- airflow-core/src/airflow/models/taskmap.py | 23 ++++--------------- 3 files changed, 17 insertions(+), 22 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 27ec5745bedc1..bad98725d687a 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -23,7 +23,7 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.jobs.base_job_runner import BaseJobRunner -from airflow.jobs.job import Job +from airflow.jobs.job import Job, run_job_async from airflow.models import DagRun from airflow.models.dag_version import DagVersion from airflow.policies import task_instance_mutation_hook @@ -188,3 +188,13 @@ def _execute(self, session: Session) -> int | None: expand_input=self.expand_input(session=session), job_id = self.job_id, session = session ) ) + + +def _run_task_expansion_job(args) -> None: + job_runner = TaskExpansionJobRunner( + job=Job(), + task=args.task, + context=args.context, + dag_version_id=args.dag_version_id, + ) + run_job_async(job=job_runner.job, execute_callable=job_runner._execute) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 438aead8cbab0..014f83c5b07b6 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -353,7 +353,7 @@ def run_job( """ job.prepare_for_execution(session=session) try: - return execute_job(job, execute_callable=execute_callable) + return execute_job(job, execute_callable=execute_callable, session=session) finally: job.complete_execution(session=session) @@ -375,7 +375,7 @@ def execute_async_job() -> int | None: with create_session(scoped=False) as session: job.prepare_for_execution(session=session) try: - return execute_job(job, execute_callable, session) + return execute_job(job, execute_callable=execute_callable, session=session) finally: job.complete_execution(session=session) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index aca3d0619c32d..0d207e9d3bafd 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -36,6 +36,7 @@ update, ) +from airflow.jobs.expand_task_job_runner import _run_task_expansion_job from airflow.jobs.job import Job, run_job_async from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion @@ -141,7 +142,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq :return: The newly created mapped task instances (if any) in ascending order by map index, and the maximum map index value. """ - from airflow.jobs.expand_task_job_runner import TaskExpansionJobRunner, task_expansion_batch_size + from argparse import Namespace from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance @@ -173,24 +174,8 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq if isinstance(task, MappedOperator) and unmapped_ti: context = unmapped_ti.get_template_context(session=session) - job = Job() - job_runner = TaskExpansionJobRunner( - job=job, - task=task, - context=context, - dag_version_id=dag_version_id, - ) - all_expanded_tis.extend( - job_runner.expand_tasks( - expand_input=islice(job_runner.expand_input(session=session), task_expansion_batch_size), - session=session, - ) - ) - total_expanded_ti_count = len(all_expanded_tis) - - task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) - - run_job_async(job=job, execute_callable=job_runner._execute) + task.log.info("_run_task_expansion_job: %s", context) + _run_task_expansion_job(Namespace(task=task, context=context, dag_version_id=dag_version_id)) else: try: total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) From 5f380eae0494cdb4f3aaa59a3d29bacd1e9d185e Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 08:09:04 +0200 Subject: [PATCH 021/338] refactor: Don't raise UnmappableXComTypePushed when XCom isn't a mappable type anymore (e.g. iteable), as we don't need to calculate length in advance anymore --- task-sdk/src/airflow/sdk/execution_time/task_runner.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 2374520e63f21..f41a399be0bd0 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -1228,11 +1228,9 @@ def _push_xcom_if_needed(result: Any, ti: RuntimeTaskInstance, log: Logger): mapped_length: int | None = None if not ti.is_mapped and has_mapped_dep: from airflow.sdk.definitions.mappedoperator import is_mappable_value - from airflow.sdk.exceptions import UnmappableXComTypePushed - if not is_mappable_value(xcom_value): - raise UnmappableXComTypePushed(xcom_value) - mapped_length = len(xcom_value) + if is_mappable_value(xcom_value): + mapped_length = len(xcom_value) log.info("Pushing xcom", ti=ti) From acd2d7183b32caf22b38bb944b7046aa09474096 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 08:15:07 +0200 Subject: [PATCH 022/338] refactor: Added values method in both expand input types so it can be used in generic way in task expansion job runner --- .../src/airflow/sdk/definitions/_internal/expandinput.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py index b1c0c6ee5f979..68605d558b1be 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py @@ -193,6 +193,9 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() + def values(self) -> Iterable[Any]: + return self.value.values() + def resolve(self, context: Mapping[str, Any]) -> tuple[Mapping[str, Any], set[int]]: map_index: int | None = context["ti"].map_index if map_index is None or map_index < 0: @@ -247,6 +250,9 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() + def values(self) -> Iterable[Any]: + return self.value + def resolve(self, context: Mapping[str, Any]) -> tuple[Mapping[str, Any], set[int]]: map_index = context["ti"].map_index if map_index < 0: From ad9c65ba0af4b8cbefb69a02e5cc0cec2d599f7d Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 08:17:55 +0200 Subject: [PATCH 023/338] refactor: Reverted expand_mapped_task method of TaskMap back to original implementation as we probably won't need it anymore once AIP-88 is fully implemented --- airflow-core/src/airflow/models/taskmap.py | 256 +++++++++------------ 1 file changed, 111 insertions(+), 145 deletions(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 0d207e9d3bafd..2798d60eb5188 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -21,28 +21,16 @@ import collections.abc import enum -import logging from collections.abc import Collection, Iterable, Sequence -from itertools import islice from typing import TYPE_CHECKING, Any -from sqlalchemy import ( - CheckConstraint, - Column, - ForeignKeyConstraint, - Integer, - String, - select, - update, -) - -from airflow.jobs.expand_task_job_runner import _run_task_expansion_job -from airflow.jobs.job import Job, run_job_async +from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select + from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion from airflow.utils.db import exists_query from airflow.utils.sqlalchemy import ExtendedJSON, with_row_locks -from airflow.utils.state import TaskInstanceState +from airflow.utils.state import State, TaskInstanceState if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -155,147 +143,125 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" ) - unmapped_ti = TaskInstance.get_task_instance( - dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session - ) - - task.log.info("unmapped_ti: %s", unmapped_ti) - - if unmapped_ti: - dag_version_id = unmapped_ti.dag_version_id - elif dag_version := DagVersion.get_latest_version(task.dag_id, session=session): - dag_version_id = dag_version.id - else: - dag_version_id = None + try: + total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) + except NotFullyPopulated as e: + if not task.dag or not task.dag.partial: + task.log.error( + "Cannot expand %r for run %s; missing upstream values: %s", + task, + run_id, + sorted(e.missing), + ) + total_length = None - task.log.info("dag_version_id: %s", dag_version_id) + state: TaskInstanceState | None = None + unmapped_ti: TaskInstance | None = session.scalars( + select(TaskInstance).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index == -1, + or_(TaskInstance.state.in_(State.unfinished), TaskInstance.state.is_(None)), + ) + ).one_or_none() all_expanded_tis: list[TaskInstance] = [] - if isinstance(task, MappedOperator) and unmapped_ti: - context = unmapped_ti.get_template_context(session=session) - task.log.info("_run_task_expansion_job: %s", context) - _run_task_expansion_job(Namespace(task=task, context=context, dag_version_id=dag_version_id)) - else: - try: - total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) - except NotFullyPopulated as e: + if unmapped_ti: + if TYPE_CHECKING: + assert task.dag is None or isinstance(task.dag, SchedulerDAG) + + # The unmapped task instance still exists and is unfinished, i.e. we + # haven't tried to run it before. + if total_length is None: + # If the DAG is partial, it's likely that the upstream tasks + # are not done yet, so the task can't fail yet. if not task.dag or not task.dag.partial: - task.log.error( - "Cannot expand %r for run %s; missing upstream values: %s", - task, - run_id, - sorted(e.missing), - ) - total_length = None - - task.log.info("total_length: %s", total_length) - - state: TaskInstanceState | None = None - - if unmapped_ti: - if TYPE_CHECKING: - assert task.dag is None or isinstance(task.dag, SchedulerDAG) - - # The unmapped task instance still exists and is unfinished, i.e. we - # haven't tried to run it before. - if total_length is None: - # If the DAG is partial, it's likely that the upstream tasks - # are not done yet, so the task can't fail yet. - if not task.dag or not task.dag.partial: - unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED - elif total_length < 1: - # If the upstream maps this to a zero-length value, simply mark - # the unmapped task instance as SKIPPED (if needed). - task.log.info( - "Marking %s as SKIPPED since the map has %d values to expand", - unmapped_ti, - total_length, - ) - unmapped_ti.state = TaskInstanceState.SKIPPED - else: - zero_index_ti_exists = exists_query( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index == 0, - session=session, - ) - if not zero_index_ti_exists: - # Otherwise convert this into the first mapped index, and create - # TaskInstance for other indexes. - unmapped_ti.map_index = 0 - task.log.debug("Updated in place to become %s", unmapped_ti) - all_expanded_tis.append(unmapped_ti) - # execute hook for task instance map index 0 - task_instance_mutation_hook(unmapped_ti) - session.flush() - else: - task.log.debug("Deleting the original task instance: %s", unmapped_ti) - session.delete(unmapped_ti) - state = unmapped_ti.state - - if total_length is None or total_length < 1: - # Nothing to fixup. - indexes_to_map: Iterable[int] = () + unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED + elif total_length < 1: + # If the upstream maps this to a zero-length value, simply mark + # the unmapped task instance as SKIPPED (if needed). + task.log.info( + "Marking %s as SKIPPED since the map has %d values to expand", + unmapped_ti, + total_length, + ) + unmapped_ti.state = TaskInstanceState.SKIPPED else: - # Only create "missing" ones. - current_max_mapping = TaskInstance.get_current_max_mapping( - dag_id=task.dag_id, task_id=task.task_id, run_id=run_id, session=session + zero_index_ti_exists = exists_query( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index == 0, + session=session, ) - indexes_to_map = range(current_max_mapping + 1, total_length) - - task.log.info("indexes_to_map: %s", indexes_to_map) + if not zero_index_ti_exists: + # Otherwise convert this into the first mapped index, and create + # TaskInstance for other indexes. + unmapped_ti.map_index = 0 + task.log.debug("Updated in place to become %s", unmapped_ti) + all_expanded_tis.append(unmapped_ti) + # execute hook for task instance map index 0 + task_instance_mutation_hook(unmapped_ti) + session.flush() + else: + task.log.debug("Deleting the original task instance: %s", unmapped_ti) + session.delete(unmapped_ti) + state = unmapped_ti.state + dag_version_id = unmapped_ti.dag_version_id - for index in indexes_to_map: - # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. - ti = TaskInstance( - task, - run_id=run_id, - map_index=index, - state=state, - dag_version_id=dag_version_id, + if total_length is None or total_length < 1: + # Nothing to fixup. + indexes_to_map: Iterable[int] = () + else: + # Only create "missing" ones. + current_max_mapping = session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, ) - task.log.debug("Expanding TIs upserted %s", ti) - task_instance_mutation_hook(ti) - ti = session.merge(ti) - ti.refresh_from_task(task) # session.merge() loses task information. - all_expanded_tis.append(ti) - - # Coerce the None case to 0 -- these two are almost treated identically, - # except the unmapped ti (if exists) is marked to different states. - total_expanded_ti_count = total_length or 0 + ) + indexes_to_map = range(current_max_mapping + 1, total_length) - task.log.info("total_expanded_ti_count: %s", total_expanded_ti_count) + if unmapped_ti: + dag_version_id = unmapped_ti.dag_version_id + elif dag_version := DagVersion.get_latest_version(task.dag_id, session=session): + dag_version_id = dag_version.id + else: + dag_version_id = None - # Any (old) task instances with inapplicable indexes (>= the total - # number we need) are set to "REMOVED". - query = select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index >= total_expanded_ti_count, + for index in indexes_to_map: + # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. + ti = TaskInstance( + task, + run_id=run_id, + map_index=index, + state=state, + dag_version_id=dag_version_id, ) - query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) - to_update = session.scalars(query) - for ti in to_update: - ti.state = TaskInstanceState.REMOVED + task.log.debug("Expanding TIs upserted %s", ti) + task_instance_mutation_hook(ti) + ti = session.merge(ti) + ti.refresh_from_task(task) # session.merge() loses task information. + all_expanded_tis.append(ti) + + # Coerce the None case to 0 -- these two are almost treated identically, + # except the unmapped ti (if exists) is marked to different states. + total_expanded_ti_count = total_length or 0 + + # Any (old) task instances with inapplicable indexes (>= the total + # number we need) are set to "REMOVED". + query = select(TaskInstance).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index >= total_expanded_ti_count, + ) + query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) + to_update = session.scalars(query) + for ti in to_update: + ti.state = TaskInstanceState.REMOVED session.flush() return all_expanded_tis, total_expanded_ti_count - 1 - - # TODO: ths is a hack to update the TaskMap length after some task instances have been created while some of - # of them are already being executed and other one are still being expanded asynchronously. This is to prevent - # for already expanded tasks to be removed by the scheduler. - @classmethod - def update_task_map_length(cls, length: int, dag_id: str, task_id: str, run_id: str, session: Session): - logging.info("Persisting TaskMap length: %s", length) - session.execute( - update(TaskMap) - .where( - TaskMap.dag_id == dag_id, - TaskMap.task_id == task_id, - TaskMap.run_id == run_id, - TaskMap.map_index == -1, - ) - .values(length=length) - ) From 2b097225cfef1365abe9321ab630d53f3526b180 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 08:20:24 +0200 Subject: [PATCH 024/338] refactor: Remove custom resolve methods on scheduler XCom's but kept operator property on SchedulerMapXCom so operators can be accessed in generic way identically of which type it is --- airflow-core/src/airflow/models/xcom_arg.py | 45 +-------------------- 1 file changed, 2 insertions(+), 43 deletions(-) diff --git a/airflow-core/src/airflow/models/xcom_arg.py b/airflow-core/src/airflow/models/xcom_arg.py index 6f0705e1ff1d4..1c2bc7e9fdeed 100644 --- a/airflow-core/src/airflow/models/xcom_arg.py +++ b/airflow-core/src/airflow/models/xcom_arg.py @@ -17,7 +17,7 @@ from __future__ import annotations -from collections.abc import Mapping, Sequence +from collections.abc import Sequence from functools import singledispatch from typing import TYPE_CHECKING, Any @@ -25,23 +25,17 @@ from sqlalchemy import func, or_, select from sqlalchemy.orm import Session -from airflow.models.xcom import BaseXCom -from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions._internal.types import ArgNotSet from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.xcom_arg import ( XComArg, ) -from airflow.sdk.execution_time.comms import XComResult -from airflow.sdk.execution_time.xcom import resolve_xcom_backend from airflow.utils.db import exists_query -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State from airflow.utils.types import NOTSET from airflow.utils.xcom import XCOM_RETURN_KEY __all__ = ["XComArg", "get_task_map_length"] -xcom_backend: BaseXCom = resolve_xcom_backend() if TYPE_CHECKING: from airflow.models.dag import DAG as SchedulerDAG @@ -50,7 +44,7 @@ @attrs.define -class SchedulerXComArg(LoggingMixin): +class SchedulerXComArg: @classmethod def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: """ @@ -65,9 +59,6 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: """ raise NotImplementedError() - def resolve(self, context: Mapping[str, Any], session: Session): - raise NotImplementedError() - @attrs.define class SchedulerPlainXComArg(SchedulerXComArg): @@ -78,25 +69,6 @@ class SchedulerPlainXComArg(SchedulerXComArg): def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: return cls(dag.get_task(data["task_id"]), data["key"]) - def resolve(self, context: Mapping[str, Any], session: Session) -> Any: - value = context["ti"].xcom_pull( - task_ids=self.operator.task_id, - key=self.operator.output.key, - session=session, - ) - - deserialized_value = xcom_backend.deserialize_value( - XComResult(key=self.operator.output.key, value=value) - ) - - self.log.debug("deserialized_value: %s", deserialized_value) - - if isinstance(deserialized_value, ResolveMixin): - deserialized_value = deserialized_value.resolve(context) - self.log.debug("resolved_value: %s", deserialized_value) - - return deserialized_value - @attrs.define class SchedulerMapXComArg(SchedulerXComArg): @@ -114,19 +86,6 @@ def _deserialize(cls, data: dict[str, Any], dag: SchedulerDAG) -> Self: # in the UI, and displaying a function object is useless. return cls(deserialize_xcom_arg(data["arg"], dag), data["callables"]) - def resolve(self, context: Mapping[str, Any], session: Session) -> Any: - resolved_arg = self.arg.resolve(context, session) - - def apply(arg: Any): - for index, _callable in enumerate(self.callables): - if isinstance(_callable, str): - _callable = eval(_callable) - self.callables[index] = _callable - arg = _callable(arg) - return arg - - return map(apply, resolved_arg) - @attrs.define class SchedulerConcatXComArg(SchedulerXComArg): From 37e1b1d765b24caef5b3ca71d4ca75eb02a463e3 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 08:21:47 +0200 Subject: [PATCH 025/338] refactor: Refactored TaskExpansionJobRunner so it can be run from the triggerer instead of the scheduler as the scheduler cannot resolve XCom's --- .../airflow/jobs/expand_task_job_runner.py | 169 ++++++++---------- 1 file changed, 75 insertions(+), 94 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index bad98725d687a..da2754e15f605 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -17,77 +17,45 @@ # under the License. from __future__ import annotations -from collections.abc import Iterator +import time from typing import TYPE_CHECKING from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import Job, run_job_async -from airflow.models import DagRun -from airflow.models.dag_version import DagVersion +from airflow.models import DagRun, DagBag from airflow.policies import task_instance_mutation_hook +from airflow.sdk import XComArg from airflow.sdk.definitions.mappedoperator import MappedOperator -from airflow.utils.context import Context from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.module_loading import import_string -from airflow.utils.session import NEW_SESSION +from airflow.utils.session import NEW_SESSION, create_session from airflow.utils.state import DagRunState if TYPE_CHECKING: from sqlalchemy.orm import Session + from airflow.dag import DAG + from airflow.jobs.triggerer_job_runner import TriggerRunnerSupervisor from airflow.models.taskinstance import TaskInstance task_expansion_batch_size = conf.getint("scheduler", "task_expansion_batch_size", fallback=10) class TaskExpansionJobRunner(BaseJobRunner, LoggingMixin): - job_type = "TaskExpansionJob" - def __init__( self, job: Job, - task: MappedOperator, - context: Context, - dag_version_id: DagVersion, - ) -> None: + trigger_runner: TriggerRunnerSupervisor, + ): super().__init__(job) - task.operator_class = import_string(f"{task._task_module}.{task._task_type}") - self.context = {**context, **{"task": task}} - self.job.dag_id = self.dag_id - self.job.job_type = self.job_type - self.dag_version_id = dag_version_id - - @property - def job_id(self) -> str: - return self.job.id - - @property - def dag_id(self) -> str: - return self.context["dag"].dag_id - - @property - def task(self) -> MappedOperator: - return self.context["task"] - - @property - def task_id(self) -> str: - return self.task.task_id - - @property - def run_id(self) -> str: - return self.context["run_id"] - - def expand_input(self, session: Session) -> Iterator[dict]: - self.log.info("expand_input: %s", self.task.expand_input) - return iter(self.task.expand_input.resolve(self.context, session)) + self.trigger_runner = trigger_runner def expand_task(self, task_instance: TaskInstance, mapped_kwargs) -> TaskInstance: self.log.info("expand task: %s", task_instance.map_index) - self.log.debug("unmap (%s): %s", self.task.operator_class, mapped_kwargs) + self.log.debug("unmap (%s): %s", task_instance.task.operator_class, mapped_kwargs) - operator = self.task.unmap(mapped_kwargs) + operator = task_instance.task.unmap(mapped_kwargs) self.log.info("operator (%s): %s", type(operator), operator) @@ -103,10 +71,10 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: self.log.info("dag_run_state: %s", dag_run.state) if dag_run.state == DagRunState.FAILED: - self.log.info("DagRun %s for dag %s has failed, stopping expansion", self.run_id, self.dag_id) + self.log.info("DagRun %s for dag %s has failed, stopping expansion", dag_run.run_id, dag_run.dag_id) raise AirflowException( - f"Stopping expansion of tasks for DagRun {self.run_id} of DAG {self.dag_id} due to failure." + f"Stopping expansion of tasks for DagRun {dag_run.run_id} of DAG {dag_run.dag_id} due to failure." ) def _persist_task_instances( @@ -115,86 +83,99 @@ def _persist_task_instances( """ Expands the task using the provided expand_input. """ - from airflow.models.taskmap import TaskMap if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) session.merge(dag_run) - TaskMap.update_task_map_length( - length=task_instances[-1].map_index + 1, - dag_id=self.dag_id, - task_id=self.task_id, - run_id=dag_run.run_id, - session=session, - ) session.flush() task_instances.clear() - def expand_tasks( - self, expand_input: Iterator[dict], job_id: str | None = None, session: Session = NEW_SESSION - ) -> list[TaskInstance]: + def expand_unmapped_task_instance( + self, dag_run: DagRun, unmapped_ti: TaskInstance, session: Session = NEW_SESSION + ) -> None: """ Expands the task using the provided expand_input. """ from airflow.models.taskinstance import TaskInstance - max_map_index = TaskInstance.get_current_max_mapping( - dag_id=self.dag_id, task_id=self.task_id, run_id=self.run_id, session=session - ) - dag_run = DagRun.get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) - unmapped_ti = TaskInstance.get_task_instance( - dag_id=self.dag_id, run_id=self.run_id, task_id=self.task_id, map_index=-1, session=session - ) - + self.log.info("task: %s", unmapped_ti.task) self.log.info("expand_tasks: %s", session) - self.log.info("max_map_index: %s", max_map_index) - self.log.info("dag_version_id: %s", self.dag_version_id) + self.log.info("dag_version_id: %s", unmapped_ti.dag_version_id) self.log.info("dag_run: %s", dag_run) + self.log.info("unmapped_ti state: %s", unmapped_ti.state) - task_instances = [] - task_instances_batch = [] + try: + task_instances_batch = [] - for map_index, mapped_kwargs in enumerate(expand_input): - if map_index > max_map_index: - if map_index == 0 and unmapped_ti: + context = unmapped_ti.get_template_context(session=session) + expand_input = unmapped_ti.task.expand_input.values() + + if isinstance(expand_input, XComArg): + expand_input = expand_input.resolve(context) + + for map_index, mapped_kwargs in enumerate(expand_input): + if map_index > 40: + self.log.warning("Stop expanding tasks over %s!", map_index) + break + if map_index == 0: task_instance = unmapped_ti task_instance.map_index = map_index else: task_instance = TaskInstance( - task=self.task, - run_id=self.run_id, + task=unmapped_ti.task, + run_id=dag_run.run_id, map_index=map_index, - dag_version_id=self.dag_version_id, + dag_version_id=unmapped_ti.dag_version_id, ) - if job_id: - task_instance.queued_by_job_id = job_id + + if isinstance(mapped_kwargs, XComArg): + context = task_instance.get_template_context(session=session) + self.log.info("context: %s", context) + mapped_kwargs = mapped_kwargs.resolve(context) + self.log.info("mapped_kwargs: %s", mapped_kwargs) task_instance = self.expand_task(task_instance, mapped_kwargs) - task_instances.append(task_instance) task_instances_batch.append(task_instance) if len(task_instances_batch) == task_expansion_batch_size: - dag_run = DagRun.get_dag_run(dag_id=self.dag_id, run_id=self.run_id, session=session) + dag_run = DagRun.get_dag_run(dag_id=unmapped_ti.dag_id, run_id=dag_run.run_id, session=session) self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) - self._persist_task_instances(dag_run, task_instances_batch, session=session) + self._persist_task_instances(dag_run, task_instances_batch, session=session) + except Exception: + self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) + + @staticmethod + def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: + task_instance.task = dag.get_task(task_instance.task_id) + return task_instance + + @staticmethod + def has_mapped_operator(task_instance: TaskInstance) -> bool: + return isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 + + def expand_tasks(self, session: Session): + dag_bag = DagBag() + dag_runs = DagRun.get_running_dag_runs_to_examine(session=session) + + for dag_run in dag_runs: + dag = dag_bag.get_dag(dag_run.dag_id) + self.log.info("Checking for unmapped task instances on: %s", dag_run) + for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): + self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) + + def _execute(self, session: Session = NEW_SESSION) -> int | None: + self.log.info("TaskExpansionJobRunner started") - return task_instances + while self.trigger_runner.is_alive(): + with create_session(scoped=False) as session: + self.expand_tasks(session=session) + time.sleep(self.job.heartrate) - def _execute(self, session: Session) -> int | None: - return len( - self.expand_tasks( - expand_input=self.expand_input(session=session), job_id = self.job_id, session = session - ) - ) + self.log.info("TaskExpansionJobRunner stopped") -def _run_task_expansion_job(args) -> None: - job_runner = TaskExpansionJobRunner( - job=Job(), - task=args.task, - context=args.context, - dag_version_id=args.dag_version_id, - ) - run_job_async(job=job_runner.job, execute_callable=job_runner._execute) +def task_expansion_run(triggerer_heartrate: float, trigger_runner: TriggerRunnerSupervisor): + task_expansion_job_runner = TaskExpansionJobRunner(job=Job(heartrate=triggerer_heartrate), trigger_runner=trigger_runner) + run_job_async(job=task_expansion_job_runner.job, execute_callable=task_expansion_job_runner._execute) From 63d5615d5137e54e6ec36aa7d4e6d0e5fdf870ec Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 10:11:32 +0200 Subject: [PATCH 026/338] refactor: Removed magic len and getitem methods from DeferredIterable --- .../src/airflow/models/deferred_iterable.py | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index 784b4354cfb5b..28cac250c4561 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -69,7 +69,7 @@ async def run_trigger(trigger: BaseTrigger) -> TriggerEvent | None: return next(iter(events), None) -class DeferredIterable(Iterator, Sequence, Sized, ResolveMixin, LoggingMixin): +class DeferredIterable(Iterator, ResolveMixin, LoggingMixin): """An iterable that lazily fetches XCom values one by one instead of loading all at once.""" def __init__( @@ -148,17 +148,6 @@ def _execute_trigger(self): self.trigger = None return results - def __len__(self): - self.log.info("__len__: %s", self) - # TODO: maybe we should raise an exception here as you can't know the total length of an iterable in advance, but won't atm to keep Airflow happy - return len(self.results) - - def __getitem__(self, index: int): - if not (0 <= index < len(self)): - raise IndexError - - return self.results[index] - def serialize(self): """Ensure the object is JSON serializable.""" return { From 93bddc48e526ada6dbf02cef11eb5dadb78f6b54 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 11:43:28 +0200 Subject: [PATCH 027/338] refactor: Removed unused import --- airflow-core/src/airflow/models/taskmap.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 2798d60eb5188..f0fd4c0231b70 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -130,7 +130,6 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq :return: The newly created mapped task instances (if any) in ascending order by map index, and the maximum map index value. """ - from argparse import Namespace from airflow.models.baseoperator import BaseOperator as DBBaseOperator from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance From 1c3aa805cfa7aa32a211068687f762773b58f27c Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 11:51:19 +0200 Subject: [PATCH 028/338] refactor: Don't do task expansion in DagRun anymore --- .../src/airflow/jobs/triggerer_job_runner.py | 4 + airflow-core/src/airflow/models/dagrun.py | 78 +------------------ 2 files changed, 7 insertions(+), 75 deletions(-) diff --git a/airflow-core/src/airflow/jobs/triggerer_job_runner.py b/airflow-core/src/airflow/jobs/triggerer_job_runner.py index 5df6a03261523..178e8d792fe9b 100644 --- a/airflow-core/src/airflow/jobs/triggerer_job_runner.py +++ b/airflow-core/src/airflow/jobs/triggerer_job_runner.py @@ -150,6 +150,8 @@ def _exit_gracefully(self, signum, frame) -> None: sys.exit(os.EX_SOFTWARE) def _execute(self) -> int | None: + from airflow.jobs.expand_task_job_runner import task_expansion_run + self.log.info("Starting the triggerer") try: # Kick off runner sub-process without DB access @@ -157,6 +159,8 @@ def _execute(self) -> int | None: job=self.job, capacity=self.capacity, logger=log ) + task_expansion_run(self.job.heartrate, self.trigger_runner) # TODO: job shouldn't be started here + # Run the main DB comms loop in this process self.trigger_runner.run() return self.trigger_runner._exit_code diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index b07c19083d9e3..4103f2a1a8a92 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -1311,19 +1311,11 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: if unfinished_tis: schedulable_tis = [ut for ut in unfinished_tis if ut.state in SCHEDULEABLE_STATES] self.log.debug("number of scheduleable tasks for %s: %s task(s)", self, len(schedulable_tis)) - schedulable_tis, changed_tis, expansion_happened = self._get_ready_tis( + schedulable_tis, changed_tis = self._get_ready_tis( schedulable_tis, finished_tis, session=session, ) - - # During expansion, we may change some tis into non-schedulable - # states, so we need to re-compute. - if expansion_happened: - changed_tis = True - new_unfinished_tis = [t for t in unfinished_tis if t.state in State.unfinished] - finished_tis.extend(t for t in unfinished_tis if t.state in State.finished) - unfinished_tis = new_unfinished_tis else: schedulable_tis = [] changed_tis = False @@ -1381,7 +1373,7 @@ def _get_ready_tis( schedulable_tis: list[TI], finished_tis: list[TI], session: Session, - ) -> tuple[list[TI], bool, bool]: + ) -> tuple[list[TI], bool]: old_states = {} ready_tis: list[TI] = [] changed_tis = False @@ -1398,77 +1390,13 @@ def _get_ready_tis( finished_tis=finished_tis, ) - def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: - """ - Try to expand the ti, if needed. - - If the ti needs expansion, newly created task instances are - returned as well as the original ti. - The original ti is also modified in-place and assigned the - ``map_index`` of 0. - - If the ti does not need expansion, either because the task is not - mapped, or has already been expanded, *None* is returned. - """ - if TYPE_CHECKING: - assert ti.task - - if ti.map_index >= 0: # Already expanded, we're good. - return None - - from airflow.sdk.definitions.mappedoperator import MappedOperator as TaskSDKMappedOperator - - if isinstance(ti.task, TaskSDKMappedOperator): - # If we get here, it could be that we are moving from non-mapped to mapped - # after task instance clearing or this ti is not yet expanded. Safe to clear - # the db references. - ti.clear_db_references(session=session) - try: - expanded_tis, _ = TaskMap.expand_mapped_task(ti.task, self.run_id, session=session) - except NotMapped: # Not a mapped task, nothing needed. - return None - if expanded_tis: - return expanded_tis - return () - - # Check dependencies. - expansion_happened = False - # Set of task ids for which was already done _revise_map_indexes_if_mapped - revised_map_index_task_ids = set() - for schedulable in itertools.chain(schedulable_tis, additional_tis): - if TYPE_CHECKING: - assert isinstance(schedulable.task, BaseOperator) - old_state = schedulable.state - if not schedulable.are_dependencies_met(session=session, dep_context=dep_context): - old_states[schedulable.key] = old_state - continue - # If schedulable is not yet expanded, try doing it now. This is - # called in two places: First and ideally in the mini scheduler at - # the end of LocalTaskJob, and then as an "expansion of last resort" - # in the scheduler to ensure that the mapped task is correctly - # expanded before executed. Also see _revise_map_indexes_if_mapped - # docstring for additional information. - new_tis = None - if schedulable.map_index < 0: - new_tis = _expand_mapped_task_if_needed(schedulable) - if new_tis is not None: - additional_tis.extend(new_tis) - expansion_happened = True - if new_tis is None and schedulable.state in SCHEDULEABLE_STATES: - # It's enough to revise map index once per task id, - # checking the map index for each mapped task significantly slows down scheduling - if schedulable.task.task_id not in revised_map_index_task_ids: - ready_tis.extend(self._revise_map_indexes_if_mapped(schedulable.task, session=session)) - revised_map_index_task_ids.add(schedulable.task.task_id) - ready_tis.append(schedulable) - # Check if any ti changed state tis_filter = TI.filter_for_tis(old_states) if tis_filter is not None: fresh_tis = session.scalars(select(TI).where(tis_filter)).all() changed_tis = any(ti.state != old_states[ti.key] for ti in fresh_tis) - return ready_tis, changed_tis, expansion_happened + return ready_tis, changed_tis def _are_premature_tis( self, From 41b0200f084f3c743b87a212d90cb16b7e3bc72b Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 11:58:13 +0200 Subject: [PATCH 029/338] refactor: Removed resolve method from SchedulerExpandInput --- .../src/airflow/models/expandinput.py | 46 +++---------------- 1 file changed, 7 insertions(+), 39 deletions(-) diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 2cf6e2d0ffd41..2078635b573c8 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -59,13 +59,16 @@ def _needs_run_time_resolution(v: OperatorExpandArgument) -> TypeGuard[MappedArg return isinstance(v, (MappedArgument, SchedulerXComArg)) -class ExpandInput(LoggingMixin): - def resolve(self, context: Mapping[str, Any], session: Session): +class SchedulerExpandInput(LoggingMixin): + def get_parse_time_mapped_ti_count(self) -> int: + raise NotImplementedError() + + def get_total_map_length(self, run_id: str, *, session: Session) -> int: raise NotImplementedError() @attrs.define -class SchedulerDictOfListsExpandInput(ExpandInput): +class SchedulerDictOfListsExpandInput(SchedulerExpandInput): value: dict EXPAND_INPUT_TYPE: ClassVar[str] = "dict-of-lists" @@ -117,26 +120,9 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: lengths = self._get_map_lengths(run_id, session=session) return functools.reduce(operator.mul, (lengths[name] for name in self.value), 1) - def resolve( - self, context: Mapping[str, Any], session: Session - ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - value = ( - self.value.resolve(context, session) if _needs_run_time_resolution(self.value) else self.value - ) - - self.log.debug("resolved value: %s", value) - - for key, item in value.items(): - result = item.resolve(context, session) if _needs_run_time_resolution(item) else item - - for index, sub_item in enumerate(result): - yield {key: sub_item} - - return [] - @attrs.define -class SchedulerListOfDictsExpandInput(ExpandInput): +class SchedulerListOfDictsExpandInput(SchedulerExpandInput): value: list EXPAND_INPUT_TYPE: ClassVar[str] = "list-of-dicts" @@ -156,30 +142,12 @@ def get_total_map_length(self, run_id: str, *, session: Session) -> int: raise NotFullyPopulated({"expand_kwargs() argument"}) return length - def resolve( - self, context: Mapping[str, Any], session: Session - ) -> Generator[dict[Any, str | Any] | dict[Any, Any], None, list[Any]]: - value = self.value.resolve(context, session) if _needs_run_time_resolution(self.value) else self.value - - self.log.debug("resolved value: %s", value) - - for index, entry in enumerate(value): - for key, item in entry.items(): - if _needs_run_time_resolution(item): - entry[key] = item.resolve(context, session) - - yield entry - - return [] - _EXPAND_INPUT_TYPES: dict[str, type[SchedulerExpandInput]] = { "dict-of-lists": SchedulerDictOfListsExpandInput, "list-of-dicts": SchedulerListOfDictsExpandInput, } -SchedulerExpandInput = Union[SchedulerDictOfListsExpandInput, SchedulerListOfDictsExpandInput] - def create_expand_input(kind: str, value: Any) -> SchedulerExpandInput: return _EXPAND_INPUT_TYPES[kind](value) From 05b3f062df06737c22c54d38e9ef69c86f805c00 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 12:32:47 +0200 Subject: [PATCH 030/338] refactor: Make sure dependencies are met before expanding unmapped task instance --- .../airflow/jobs/expand_task_job_runner.py | 82 +++++++++---------- 1 file changed, 41 insertions(+), 41 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index da2754e15f605..a07fa5b61e3c9 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -30,7 +30,7 @@ from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.session import NEW_SESSION, create_session -from airflow.utils.state import DagRunState +from airflow.utils.state import DagRunState, State if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -105,46 +105,40 @@ def expand_unmapped_task_instance( self.log.info("dag_run: %s", dag_run) self.log.info("unmapped_ti state: %s", unmapped_ti.state) - try: - task_instances_batch = [] - - context = unmapped_ti.get_template_context(session=session) - expand_input = unmapped_ti.task.expand_input.values() - - if isinstance(expand_input, XComArg): - expand_input = expand_input.resolve(context) - - for map_index, mapped_kwargs in enumerate(expand_input): - if map_index > 40: - self.log.warning("Stop expanding tasks over %s!", map_index) - break - if map_index == 0: - task_instance = unmapped_ti - task_instance.map_index = map_index - else: - task_instance = TaskInstance( - task=unmapped_ti.task, - run_id=dag_run.run_id, - map_index=map_index, - dag_version_id=unmapped_ti.dag_version_id, - ) - - if isinstance(mapped_kwargs, XComArg): - context = task_instance.get_template_context(session=session) - self.log.info("context: %s", context) - mapped_kwargs = mapped_kwargs.resolve(context) - self.log.info("mapped_kwargs: %s", mapped_kwargs) - task_instance = self.expand_task(task_instance, mapped_kwargs) - task_instances_batch.append(task_instance) - - if len(task_instances_batch) == task_expansion_batch_size: - dag_run = DagRun.get_dag_run(dag_id=unmapped_ti.dag_id, run_id=dag_run.run_id, session=session) - self._check_dag_run_state(dag_run) - self._persist_task_instances(dag_run, task_instances_batch, session=session) + task_instances_batch = [] + + context = unmapped_ti.get_template_context(session=session) + expand_input = unmapped_ti.task.expand_input.values() + + if isinstance(expand_input, XComArg): + expand_input = expand_input.resolve(context) + + for map_index, mapped_kwargs in enumerate(expand_input): + if map_index == 0: + task_instance = unmapped_ti + task_instance.map_index = map_index + else: + task_instance = TaskInstance( + task=unmapped_ti.task, + run_id=dag_run.run_id, + map_index=map_index, + dag_version_id=unmapped_ti.dag_version_id, + ) + + if isinstance(mapped_kwargs, XComArg): + context = task_instance.get_template_context(session=session) + self.log.info("context: %s", context) + mapped_kwargs = mapped_kwargs.resolve(context) + self.log.info("mapped_kwargs: %s", mapped_kwargs) + task_instance = self.expand_task(task_instance, mapped_kwargs) + task_instances_batch.append(task_instance) + + if len(task_instances_batch) == task_expansion_batch_size: + dag_run = DagRun.get_dag_run(dag_id=unmapped_ti.dag_id, run_id=dag_run.run_id, session=session) + self._check_dag_run_state(dag_run) + self._persist_task_instances(dag_run, task_instances_batch, session=session) self._persist_task_instances(dag_run, task_instances_batch, session=session) - except Exception: - self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) @staticmethod def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @@ -153,7 +147,7 @@ def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @staticmethod def has_mapped_operator(task_instance: TaskInstance) -> bool: - return isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 + return isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 and task_instance.state in State.unfinished def expand_tasks(self, session: Session): dag_bag = DagBag() @@ -163,7 +157,13 @@ def expand_tasks(self, session: Session): dag = dag_bag.get_dag(dag_run.dag_id) self.log.info("Checking for unmapped task instances on: %s", dag_run) for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): - self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) + try: + are_dependencies_met = unmapped_ti.are_dependencies_met(session=session, verbose=True) + self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) + if are_dependencies_met: + self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) + except Exception: + self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) def _execute(self, session: Session = NEW_SESSION) -> int | None: self.log.info("TaskExpansionJobRunner started") From 466c60698271bd2ac4b82312e199b031d91a9b01 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 24 Jun 2025 19:26:08 +0200 Subject: [PATCH 031/338] refactor: Implemented _LazyMapResult in xcom args --- .../src/airflow/sdk/definitions/xcom_arg.py | 84 +++++++++++++++++-- 1 file changed, 75 insertions(+), 9 deletions(-) diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 2a93585304cb0..61f56705fa782 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -41,6 +41,7 @@ # Callable objects contained by MapXComArg. We only accept callables from # the user, but deserialize them into strings in a serialized XComArg for # safety (those callables are arbitrary user code). +FilterCallables = Sequence[Callable[[Any], bool]] MapCallables = Sequence[Callable[[Any], Any]] @@ -377,22 +378,85 @@ def _get_callable_name(f: Callable | str) -> str: return "" -class _MapResult(Sequence): - def __init__(self, value: Sequence | dict, callables: MapCallables) -> None: - self.value = value +class _MappableResult(Sequence): + def __init__( + self, value: Sequence | dict, callables: FilterCallables | MapCallables + ) -> None: + self.value = self._convert(value) self.callables = callables def __getitem__(self, index: Any) -> Any: - value = self.value[index] + raise NotImplementedError - for f in self.callables: - value = f(value) + def __len__(self) -> int: + raise NotImplementedError + + @staticmethod + def _convert(value: Sequence | dict) -> list: + if isinstance(value, (dict, set)): + return list(value) + if isinstance(value, list): + return value + raise ValueError( + f"XCom filter expects sequence or dict, not {type(value).__name__}" + ) + + def _apply_callables(self, value) -> Any: + for func in self.callables: + value = func(value) + return value + + def __str__(self): + return f"{self.__class__.__name__}({self.value}, {self.callables})" + + +class _MapResult(_MappableResult): + def __getitem__(self, index: Any) -> Any: + value = self._apply_callables(self.value[index]) return value def __len__(self) -> int: return len(self.value) +class _LazyMapResult(_MappableResult): + def __init__(self, value: Iterable, callables: MapCallables) -> None: + super().__init__([], callables) + self._iterator = iter(value) + + def __next__(self) -> Any: + value = self._apply_callables(next(self._iterator)) + self.value.append(value) + return value + + def __getitem__(self, index: Any) -> Any: + if index < 0: + raise IndexError + + while len(self.value) <= index: + try: + next(self) + except StopIteration: + raise IndexError + return self.value[index] + + def __len__(self) -> int: + while True: + try: + next(self) + except StopIteration: + break + return len(self.value) + + def __iter__(self) -> Iterator: + yield from self.value + while True: + try: + yield next(self) + except StopIteration: + break + + class MapXComArg(XComArg): """ An XCom reference with ``map()`` call(s) applied. @@ -429,9 +493,11 @@ def map(self, f: Callable[[Any], Any]) -> MapXComArg: def resolve(self, context: Mapping[str, Any]) -> Any: value = self.arg.resolve(context) - if not isinstance(value, (Sequence, dict)): - raise ValueError(f"XCom map expects sequence or dict, not {type(value).__name__}") - return _MapResult(value, self.callables) + if isinstance(value, (Sequence, dict)): + return _MapResult(value, self.callables) + if isinstance(value, Iterable): + return _LazyMapResult(value, self.callables) + raise ValueError(f"XCom map expects sequence or dict, not {type(value).__name__}") class _ZipResult(Sequence): From e2724b6735f57153e07d2a85d36cb89651426d05 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 25 Jun 2025 09:35:17 +0200 Subject: [PATCH 032/338] refactor: Only try to deserialize XCom value if it's not from a MappedOperator --- .../src/airflow/sdk/definitions/xcom_arg.py | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 61f56705fa782..417e144917f8e 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -28,7 +28,11 @@ from airflow.sdk.definitions._internal.abstractoperator import AbstractOperator from airflow.sdk.definitions._internal.mixins import DependencyMixin, ResolveMixin from airflow.sdk.definitions._internal.types import NOTSET, ArgNotSet +from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.sdk.execution_time.comms import XComResult from airflow.sdk.execution_time.lazy_sequence import LazyXComSequence +from airflow.sdk.execution_time.xcom import XCom +from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.setup_teardown import SetupTeardownContext from airflow.utils.trigger_rule import TriggerRule from airflow.utils.xcom import XCOM_RETURN_KEY @@ -45,7 +49,7 @@ MapCallables = Sequence[Callable[[Any], Any]] -class XComArg(ResolveMixin, DependencyMixin): +class XComArg(LoggingMixin, ResolveMixin, DependencyMixin): """ Reference to an XCom value pushed from another operator. @@ -333,7 +337,7 @@ def concat(self, *others: XComArg) -> ConcatXComArg: raise ValueError("cannot concatenate non-return XCom") return super().concat(*others) - def resolve(self, context: Mapping[str, Any]) -> Any: + def _resolve(self, context: Mapping[str, Any]) -> Any: ti = context["ti"] task_id = self.operator.task_id @@ -364,6 +368,23 @@ def resolve(self, context: Mapping[str, Any]) -> Any: return None raise XComNotFound(ti.dag_id, task_id, self.key) + def resolve(self, context: Mapping[str, Any]) -> Any: + value = self._resolve(context) + + self.log.debug("value: %s", value) + + if not isinstance(self.operator, MappedOperator): + deserialized_value = XCom.deserialize_value( + XComResult(key=self.operator.output.key, value=value) + ) + + self.log.debug("deserialized_value: %s", deserialized_value) + + if isinstance(deserialized_value, ResolveMixin): + deserialized_value = deserialized_value.resolve(context) + return deserialized_value + return value + def _get_callable_name(f: Callable | str) -> str: """Try to "describe" a callable by getting its name.""" From 4308ad82bc5d400e7a24c4fd3d84ce5a25aaa2b4 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 25 Jun 2025 20:09:37 +0200 Subject: [PATCH 033/338] refactor: Refactored TaskExpansionTrigger and make sure unmapped tasks aren't executed by the scheduler anymore and also added feature bit to allow enabling AIP-88 --- .../airflow/jobs/expand_task_job_runner.py | 57 ++-- .../src/airflow/jobs/triggerer_job_runner.py | 4 +- airflow-core/src/airflow/models/dagrun.py | 86 ++++++- airflow-core/src/airflow/models/taskmap.py | 243 +++++++++--------- .../src/airflow/sdk/definitions/xcom_arg.py | 5 +- .../airflow/sdk/execution_time/task_runner.py | 4 + 6 files changed, 251 insertions(+), 148 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index a07fa5b61e3c9..839945b62c747 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -28,6 +28,7 @@ from airflow.policies import task_instance_mutation_hook from airflow.sdk import XComArg from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.ti_deps.dep_context import DepContext from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.session import NEW_SESSION, create_session from airflow.utils.state import DagRunState, State @@ -51,20 +52,9 @@ def __init__( super().__init__(job) self.trigger_runner = trigger_runner - def expand_task(self, task_instance: TaskInstance, mapped_kwargs) -> TaskInstance: - self.log.info("expand task: %s", task_instance.map_index) - self.log.debug("unmap (%s): %s", task_instance.task.operator_class, mapped_kwargs) - - operator = task_instance.task.unmap(mapped_kwargs) - - self.log.info("operator (%s): %s", type(operator), operator) - - task_instance.task = operator - task_instance_mutation_hook(task_instance) - - self.log.info("creating ti %s: (%s) %s", task_instance.map_index, task_instance.id, task_instance) - - return task_instance + @property + def job_id(self) -> str: + return self.job.id def _check_dag_run_state(self, dag_run: DagRun) -> None: if dag_run: @@ -108,29 +98,43 @@ def expand_unmapped_task_instance( task_instances_batch = [] context = unmapped_ti.get_template_context(session=session) + unmapped_task = unmapped_ti.task expand_input = unmapped_ti.task.expand_input.values() + self.log.info("expand_input: %s", expand_input) + if isinstance(expand_input, XComArg): expand_input = expand_input.resolve(context) for map_index, mapped_kwargs in enumerate(expand_input): + self.log.info("map_index: %s", map_index) + + if isinstance(mapped_kwargs, XComArg): + context = unmapped_ti.get_template_context(session=session) + self.log.info("context: %s", context) + mapped_kwargs = mapped_kwargs.resolve(context) + + self.log.info("mapped_kwargs: %s", mapped_kwargs) + + task = unmapped_task.unmap(mapped_kwargs) + + self.log.info("task: %s", type(task)) + if map_index == 0: task_instance = unmapped_ti + task_instance.task = task task_instance.map_index = map_index + task_instance.refresh_from_task(task) else: task_instance = TaskInstance( - task=unmapped_ti.task, + task=task, run_id=dag_run.run_id, map_index=map_index, dag_version_id=unmapped_ti.dag_version_id, ) - if isinstance(mapped_kwargs, XComArg): - context = task_instance.get_template_context(session=session) - self.log.info("context: %s", context) - mapped_kwargs = mapped_kwargs.resolve(context) - self.log.info("mapped_kwargs: %s", mapped_kwargs) - task_instance = self.expand_task(task_instance, mapped_kwargs) + task_instance.queued_by_job_id = self.job_id + task_instance_mutation_hook(task_instance) task_instances_batch.append(task_instance) if len(task_instances_batch) == task_expansion_batch_size: @@ -138,7 +142,7 @@ def expand_unmapped_task_instance( self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) - self._persist_task_instances(dag_run, task_instances_batch, session=session) + self._persist_task_instances(dag_run, task_instances_batch, session=session) @staticmethod def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @@ -158,7 +162,14 @@ def expand_tasks(self, session: Session): self.log.info("Checking for unmapped task instances on: %s", dag_run) for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): try: - are_dependencies_met = unmapped_ti.are_dependencies_met(session=session, verbose=True) + finished_tis = list(map(lambda task: self.get_task(dag, task), filter(lambda ti: ti.state in State.finished, dag_run.task_instances))) + dep_context = DepContext( + flag_upstream_failed=True, + ignore_unmapped_tasks=True, # Ignore this Dep, as we will expand it if we can. + finished_tis=finished_tis, + ) + self.log.info("Unmapped task state on: %s", unmapped_ti.state) + are_dependencies_met = unmapped_ti.are_dependencies_met(dep_context=dep_context, session=session, verbose=True) self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) if are_dependencies_met: self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) diff --git a/airflow-core/src/airflow/jobs/triggerer_job_runner.py b/airflow-core/src/airflow/jobs/triggerer_job_runner.py index 178e8d792fe9b..0ed3ae4a2d31d 100644 --- a/airflow-core/src/airflow/jobs/triggerer_job_runner.py +++ b/airflow-core/src/airflow/jobs/triggerer_job_runner.py @@ -41,6 +41,7 @@ from airflow.executors import workloads from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import perform_heartbeat +from airflow.models.taskmap import enable_lazy_task_expansion from airflow.models.trigger import Trigger from airflow.sdk.execution_time.comms import ( ConnectionResult, @@ -159,7 +160,8 @@ def _execute(self) -> int | None: job=self.job, capacity=self.capacity, logger=log ) - task_expansion_run(self.job.heartrate, self.trigger_runner) # TODO: job shouldn't be started here + if enable_lazy_task_expansion: + task_expansion_run(self.job.heartrate, self.trigger_runner) # TODO: job shouldn't be started here # Run the main DB comms loop in this process self.trigger_runner.run() diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index 4103f2a1a8a92..e6ab2c694af55 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -73,7 +73,8 @@ from airflow.models.taskinstance import TaskInstance as TI from airflow.models.taskinstancehistory import TaskInstanceHistory as TIH from airflow.models.tasklog import LogTemplate -from airflow.models.taskmap import TaskMap +from airflow.models.taskmap import TaskMap, enable_lazy_task_expansion +from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext @@ -1311,11 +1312,19 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: if unfinished_tis: schedulable_tis = [ut for ut in unfinished_tis if ut.state in SCHEDULEABLE_STATES] self.log.debug("number of scheduleable tasks for %s: %s task(s)", self, len(schedulable_tis)) - schedulable_tis, changed_tis = self._get_ready_tis( + schedulable_tis, changed_tis, expansion_happened = self._get_ready_tis( schedulable_tis, finished_tis, session=session, ) + + # During expansion, we may change some tis into non-schedulable + # states, so we need to re-compute. + if expansion_happened: + changed_tis = True + new_unfinished_tis = [t for t in unfinished_tis if t.state in State.unfinished] + finished_tis.extend(t for t in unfinished_tis if t.state in State.finished) + unfinished_tis = new_unfinished_tis else: schedulable_tis = [] changed_tis = False @@ -1373,7 +1382,7 @@ def _get_ready_tis( schedulable_tis: list[TI], finished_tis: list[TI], session: Session, - ) -> tuple[list[TI], bool]: + ) -> tuple[list[TI], bool, bool]: old_states = {} ready_tis: list[TI] = [] changed_tis = False @@ -1390,13 +1399,82 @@ def _get_ready_tis( finished_tis=finished_tis, ) + def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: + """ + Try to expand the ti, if needed. + + If the ti needs expansion, newly created task instances are + returned as well as the original ti. + The original ti is also modified in-place and assigned the + ``map_index`` of 0. + + If the ti does not need expansion, either because the task is not + mapped, or has already been expanded, *None* is returned. + """ + if TYPE_CHECKING: + assert ti.task + + if ti.map_index >= 0: # Already expanded, we're good. + return None + + from airflow.sdk.definitions.mappedoperator import MappedOperator as TaskSDKMappedOperator + + if isinstance(ti.task, TaskSDKMappedOperator): + # If we get here, it could be that we are moving from non-mapped to mapped + # after task instance clearing or this ti is not yet expanded. Safe to clear + # the db references. + ti.clear_db_references(session=session) + try: + expanded_tis, _ = TaskMap.expand_mapped_task(ti.task, self.run_id, session=session) + except NotMapped: # Not a mapped task, nothing needed. + return None + if expanded_tis: + return expanded_tis + return () + + def is_unmapped_task(ti: TI) -> bool: + # TODO: check why task is still MappedOperator even when not an unmapped task anymore + return isinstance(schedulable.task, MappedOperator) and schedulable.map_index == -1 + + # Check dependencies. + expansion_happened = False + # Set of task ids for which was already done _revise_map_indexes_if_mapped + revised_map_index_task_ids = set() + for schedulable in itertools.chain(schedulable_tis, additional_tis): + if TYPE_CHECKING: + assert isinstance(schedulable.task, BaseOperator) + old_state = schedulable.state + if not schedulable.are_dependencies_met(session=session, dep_context=dep_context): + old_states[schedulable.key] = old_state + continue + # If schedulable is not yet expanded, try doing it now. This is + # called in two places: First and ideally in the mini scheduler at + # the end of LocalTaskJob, and then as an "expansion of last resort" + # in the scheduler to ensure that the mapped task is correctly + # expanded before executed. Also see _revise_map_indexes_if_mapped + # docstring for additional information. + new_tis = None + if schedulable.map_index < 0: + new_tis = _expand_mapped_task_if_needed(schedulable) + if new_tis is not None: + additional_tis.extend(new_tis) + expansion_happened = True + if new_tis is None and schedulable.state in SCHEDULEABLE_STATES: + # It's enough to revise map index once per task id, + # checking the map index for each mapped task significantly slows down scheduling + if schedulable.task.task_id not in revised_map_index_task_ids: + ready_tis.extend(self._revise_map_indexes_if_mapped(schedulable.task, session=session)) + revised_map_index_task_ids.add(schedulable.task.task_id) + if not enable_lazy_task_expansion or not is_unmapped_task(schedulable): + ready_tis.append(schedulable) + # Check if any ti changed state tis_filter = TI.filter_for_tis(old_states) if tis_filter is not None: fresh_tis = session.scalars(select(TI).where(tis_filter)).all() changed_tis = any(ti.state != old_states[ti.key] for ti in fresh_tis) - return ready_tis, changed_tis + return ready_tis, changed_tis, expansion_happened def _are_premature_tis( self, diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index f0fd4c0231b70..a9c296eb4cb92 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -26,6 +26,7 @@ from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select +from airflow.configuration import conf from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion from airflow.utils.db import exists_query @@ -39,6 +40,10 @@ from airflow.models.taskinstance import TaskInstance +# AIP-88: https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+AIP-88%3A+Lazy+task+expansion +enable_lazy_task_expansion = conf.getboolean("scheduler", "enable_lazy_task_expansion", fallback=False) + + class TaskMapVariant(enum.Enum): """ Task map variant. @@ -135,132 +140,136 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator from airflow.sdk.definitions.mappedoperator import MappedOperator + from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.settings import task_instance_mutation_hook - if not isinstance(task, (BaseOperator, MappedOperator)): - raise RuntimeError( - f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" - ) - - try: - total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) - except NotFullyPopulated as e: - if not task.dag or not task.dag.partial: - task.log.error( - "Cannot expand %r for run %s; missing upstream values: %s", - task, - run_id, - sorted(e.missing), + if not enable_lazy_task_expansion: + if not isinstance(task, (BaseOperator, MappedOperator)): + raise RuntimeError( + f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" ) - total_length = None - - state: TaskInstanceState | None = None - unmapped_ti: TaskInstance | None = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index == -1, - or_(TaskInstance.state.in_(State.unfinished), TaskInstance.state.is_(None)), - ) - ).one_or_none() - - all_expanded_tis: list[TaskInstance] = [] - if unmapped_ti: - if TYPE_CHECKING: - assert task.dag is None or isinstance(task.dag, SchedulerDAG) - - # The unmapped task instance still exists and is unfinished, i.e. we - # haven't tried to run it before. - if total_length is None: - # If the DAG is partial, it's likely that the upstream tasks - # are not done yet, so the task can't fail yet. + try: + total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) + except NotFullyPopulated as e: if not task.dag or not task.dag.partial: - unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED - elif total_length < 1: - # If the upstream maps this to a zero-length value, simply mark - # the unmapped task instance as SKIPPED (if needed). - task.log.info( - "Marking %s as SKIPPED since the map has %d values to expand", - unmapped_ti, - total_length, - ) - unmapped_ti.state = TaskInstanceState.SKIPPED - else: - zero_index_ti_exists = exists_query( + task.log.error( + "Cannot expand %r for run %s; missing upstream values: %s", + task, + run_id, + sorted(e.missing), + ) + total_length = None + + state: TaskInstanceState | None = None + unmapped_ti: TaskInstance | None = session.scalars( + select(TaskInstance).where( TaskInstance.dag_id == task.dag_id, TaskInstance.task_id == task.task_id, TaskInstance.run_id == run_id, - TaskInstance.map_index == 0, - session=session, + TaskInstance.map_index == -1, + or_(TaskInstance.state.in_(State.unfinished), TaskInstance.state.is_(None)), ) - if not zero_index_ti_exists: - # Otherwise convert this into the first mapped index, and create - # TaskInstance for other indexes. - unmapped_ti.map_index = 0 - task.log.debug("Updated in place to become %s", unmapped_ti) - all_expanded_tis.append(unmapped_ti) - # execute hook for task instance map index 0 - task_instance_mutation_hook(unmapped_ti) - session.flush() + ).one_or_none() + + all_expanded_tis: list[TaskInstance] = [] + + if unmapped_ti: + if TYPE_CHECKING: + assert task.dag is None or isinstance(task.dag, SchedulerDAG) + + # The unmapped task instance still exists and is unfinished, i.e. we + # haven't tried to run it before. + if total_length is None: + # If the DAG is partial, it's likely that the upstream tasks + # are not done yet, so the task can't fail yet. + if not task.dag or not task.dag.partial: + unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED + elif total_length < 1: + # If the upstream maps this to a zero-length value, simply mark + # the unmapped task instance as SKIPPED (if needed). + task.log.info( + "Marking %s as SKIPPED since the map has %d values to expand", + unmapped_ti, + total_length, + ) + unmapped_ti.state = TaskInstanceState.SKIPPED else: - task.log.debug("Deleting the original task instance: %s", unmapped_ti) - session.delete(unmapped_ti) - state = unmapped_ti.state - dag_version_id = unmapped_ti.dag_version_id - - if total_length is None or total_length < 1: - # Nothing to fixup. - indexes_to_map: Iterable[int] = () - else: - # Only create "missing" ones. - current_max_mapping = session.scalar( - select(func.max(TaskInstance.map_index)).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, + zero_index_ti_exists = exists_query( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index == 0, + session=session, + ) + if not zero_index_ti_exists: + # Otherwise convert this into the first mapped index, and create + # TaskInstance for other indexes. + unmapped_ti.map_index = 0 + task.log.debug("Updated in place to become %s", unmapped_ti) + all_expanded_tis.append(unmapped_ti) + # execute hook for task instance map index 0 + task_instance_mutation_hook(unmapped_ti) + session.flush() + else: + task.log.debug("Deleting the original task instance: %s", unmapped_ti) + session.delete(unmapped_ti) + state = unmapped_ti.state + dag_version_id = unmapped_ti.dag_version_id + + if total_length is None or total_length < 1: + # Nothing to fixup. + indexes_to_map: Iterable[int] = () + else: + # Only create "missing" ones. + current_max_mapping = session.scalar( + select(func.max(TaskInstance.map_index)).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + ) ) + indexes_to_map = range(current_max_mapping + 1, total_length) + + if unmapped_ti: + dag_version_id = unmapped_ti.dag_version_id + elif dag_version := DagVersion.get_latest_version(task.dag_id, session=session): + dag_version_id = dag_version.id + else: + dag_version_id = None + + for index in indexes_to_map: + # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. + ti = TaskInstance( + task, + run_id=run_id, + map_index=index, + state=state, + dag_version_id=dag_version_id, + ) + task.log.debug("Expanding TIs upserted %s", ti) + task_instance_mutation_hook(ti) + ti = session.merge(ti) + ti.refresh_from_task(task) # session.merge() loses task information. + all_expanded_tis.append(ti) + + # Coerce the None case to 0 -- these two are almost treated identically, + # except the unmapped ti (if exists) is marked to different states. + total_expanded_ti_count = total_length or 0 + + # Any (old) task instances with inapplicable indexes (>= the total + # number we need) are set to "REMOVED". + query = select(TaskInstance).where( + TaskInstance.dag_id == task.dag_id, + TaskInstance.task_id == task.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index >= total_expanded_ti_count, ) - indexes_to_map = range(current_max_mapping + 1, total_length) - - if unmapped_ti: - dag_version_id = unmapped_ti.dag_version_id - elif dag_version := DagVersion.get_latest_version(task.dag_id, session=session): - dag_version_id = dag_version.id - else: - dag_version_id = None - - for index in indexes_to_map: - # TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings. - ti = TaskInstance( - task, - run_id=run_id, - map_index=index, - state=state, - dag_version_id=dag_version_id, - ) - task.log.debug("Expanding TIs upserted %s", ti) - task_instance_mutation_hook(ti) - ti = session.merge(ti) - ti.refresh_from_task(task) # session.merge() loses task information. - all_expanded_tis.append(ti) - - # Coerce the None case to 0 -- these two are almost treated identically, - # except the unmapped ti (if exists) is marked to different states. - total_expanded_ti_count = total_length or 0 - - # Any (old) task instances with inapplicable indexes (>= the total - # number we need) are set to "REMOVED". - query = select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index >= total_expanded_ti_count, - ) - query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) - to_update = session.scalars(query) - for ti in to_update: - ti.state = TaskInstanceState.REMOVED - session.flush() - return all_expanded_tis, total_expanded_ti_count - 1 + query = with_row_locks(query, of=TaskInstance, session=session, skip_locked=True) + to_update = session.scalars(query) + for ti in to_update: + ti.state = TaskInstanceState.REMOVED + session.flush() + return all_expanded_tis, total_expanded_ti_count - 1 + + raise NotMapped diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 417e144917f8e..2c7063b2783fb 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -373,13 +373,12 @@ def resolve(self, context: Mapping[str, Any]) -> Any: self.log.debug("value: %s", value) - if not isinstance(self.operator, MappedOperator): + # TODO: check why this is needed? + if isinstance(value, str): deserialized_value = XCom.deserialize_value( XComResult(key=self.operator.output.key, value=value) ) - self.log.debug("deserialized_value: %s", deserialized_value) - if isinstance(deserialized_value, ResolveMixin): deserialized_value = deserialized_value.resolve(context) return deserialized_value 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 f41a399be0bd0..e681eeabdfcba 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -43,6 +43,7 @@ from airflow.dag_processing.bundles.manager import DagBundlesManager from airflow.exceptions import AirflowInactiveAssetInInletOrOutletException from airflow.listeners.listener import get_listener_manager +from airflow.models.taskmap import enable_lazy_task_expansion from airflow.sdk.api.datamodels._generated import ( AssetProfile, TaskInstance, @@ -1228,9 +1229,12 @@ def _push_xcom_if_needed(result: Any, ti: RuntimeTaskInstance, log: Logger): mapped_length: int | None = None if not ti.is_mapped and has_mapped_dep: from airflow.sdk.definitions.mappedoperator import is_mappable_value + from airflow.sdk.exceptions import UnmappableXComTypePushed if is_mappable_value(xcom_value): mapped_length = len(xcom_value) + elif not enable_lazy_task_expansion: + raise UnmappableXComTypePushed(xcom_value) log.info("Pushing xcom", ti=ti) From 07f088042dbf12c1f8b2d53e62fcf39dd4faac2d Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 25 Jun 2025 20:22:43 +0200 Subject: [PATCH 034/338] refactor: We don't unmap operator in TaskExpansionTrigger as it has to be done by the executor when running the expanded task --- .../src/airflow/jobs/expand_task_job_runner.py | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 839945b62c747..a27f1bc3ab476 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -98,7 +98,6 @@ def expand_unmapped_task_instance( task_instances_batch = [] context = unmapped_ti.get_template_context(session=session) - unmapped_task = unmapped_ti.task expand_input = unmapped_ti.task.expand_input.values() self.log.info("expand_input: %s", expand_input) @@ -106,28 +105,15 @@ def expand_unmapped_task_instance( if isinstance(expand_input, XComArg): expand_input = expand_input.resolve(context) - for map_index, mapped_kwargs in enumerate(expand_input): + for map_index, _ in enumerate(expand_input): self.log.info("map_index: %s", map_index) - if isinstance(mapped_kwargs, XComArg): - context = unmapped_ti.get_template_context(session=session) - self.log.info("context: %s", context) - mapped_kwargs = mapped_kwargs.resolve(context) - - self.log.info("mapped_kwargs: %s", mapped_kwargs) - - task = unmapped_task.unmap(mapped_kwargs) - - self.log.info("task: %s", type(task)) - if map_index == 0: task_instance = unmapped_ti - task_instance.task = task task_instance.map_index = map_index - task_instance.refresh_from_task(task) else: task_instance = TaskInstance( - task=task, + task=unmapped_ti.task, run_id=dag_run.run_id, map_index=map_index, dag_version_id=unmapped_ti.dag_version_id, From 9f87c04ffd260a89487402fb547df67e904b1ea9 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 25 Jun 2025 20:42:29 +0200 Subject: [PATCH 035/338] refactor: Make sure expanded task are committed in batch in TaskExpansionJobRunner so the scheduler can pick those up while expansion is still running --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index a27f1bc3ab476..5ba8dc47f740d 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -78,7 +78,7 @@ def _persist_task_instances( self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) session.merge(dag_run) - session.flush() + session.commit() task_instances.clear() def expand_unmapped_task_instance( @@ -106,6 +106,10 @@ def expand_unmapped_task_instance( expand_input = expand_input.resolve(context) for map_index, _ in enumerate(expand_input): + # if map_index > 40: + # self.log.warning("Stop expanding tasks over %s!", map_index) + # break + self.log.info("map_index: %s", map_index) if map_index == 0: From b94f270352ad93dc80e61dce651524d6ca27d65c Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 25 Jun 2025 20:43:36 +0200 Subject: [PATCH 036/338] refactor: Push newly fetched results back to XCom in DeferredIterable so we don't need to fetch those again when we re-iterate over it --- airflow-core/src/airflow/models/deferred_iterable.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index 28cac250c4561..0715c9cb009fb 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -26,6 +26,7 @@ from wrapt import synchronized from airflow.exceptions import AirflowException +from airflow.models.xcom import XComModel from airflow.sdk.bases.operator import BaseOperator as Operator from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions.context import Context @@ -125,6 +126,16 @@ def __next__(self): else: self.results.append(results) + # We push appended results back to existing XCom + XComModel.set( + key=self.operator.output.key, + value=self.results, + task_id=self.operator.task_id, + dag_id=self.operator.dag_id, + run_id=self.context["run_id"], + # map_index=self.context["ti"].map_index, + ) + self.index += 1 return self.results[-1] From 1b0fbd667d8fa0ad8984405a9eb33f63bf9b0294 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 30 Jun 2025 09:25:39 +0200 Subject: [PATCH 037/338] refactor: Moved enable_lazy_task_expansion to MappedOperator in Task SDK definitions --- airflow-core/src/airflow/jobs/triggerer_job_runner.py | 2 +- airflow-core/src/airflow/models/dagrun.py | 4 ++-- airflow-core/src/airflow/models/taskmap.py | 8 ++------ task-sdk/src/airflow/sdk/definitions/mappedoperator.py | 4 ++++ task-sdk/src/airflow/sdk/definitions/xcom_arg.py | 4 ++-- task-sdk/src/airflow/sdk/execution_time/task_runner.py | 3 +-- 6 files changed, 12 insertions(+), 13 deletions(-) diff --git a/airflow-core/src/airflow/jobs/triggerer_job_runner.py b/airflow-core/src/airflow/jobs/triggerer_job_runner.py index fa986b6e35643..d6dfdf7deb746 100644 --- a/airflow-core/src/airflow/jobs/triggerer_job_runner.py +++ b/airflow-core/src/airflow/jobs/triggerer_job_runner.py @@ -42,8 +42,8 @@ from airflow.executors import workloads from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import perform_heartbeat -from airflow.models.taskmap import enable_lazy_task_expansion from airflow.models.trigger import Trigger +from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion from airflow.sdk.execution_time.comms import ( CommsDecoder, ConnectionResult, diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index d2edd1a62c37a..f79ffdfc16711 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -70,8 +70,8 @@ from airflow.models.taskinstance import TaskInstance as TI from airflow.models.taskinstancehistory import TaskInstanceHistory as TIH from airflow.models.tasklog import LogTemplate -from airflow.models.taskmap import TaskMap, enable_lazy_task_expansion -from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.models.taskmap import TaskMap +from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 670f88ef3f6e8..15ec1c32b251e 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -26,7 +26,6 @@ from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select -from airflow.configuration import conf from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion from airflow.utils.db import exists_query @@ -40,10 +39,6 @@ from airflow.models.taskinstance import TaskInstance -# AIP-88: https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+AIP-88%3A+Lazy+task+expansion -enable_lazy_task_expansion = conf.getboolean("scheduler", "enable_lazy_task_expansion", fallback=False) - - class TaskMapVariant(enum.Enum): """ Task map variant. @@ -139,7 +134,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator - from airflow.sdk.definitions.mappedoperator import MappedOperator + from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.settings import task_instance_mutation_hook @@ -147,6 +142,7 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq if not isinstance(task, (BaseOperator, MappedOperator)): raise RuntimeError( f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" + ) try: total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) diff --git a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py index 9f2cc18604354..eebf9cb5d3e28 100644 --- a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py +++ b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py @@ -26,6 +26,7 @@ import attrs import methodtools +from airflow.configuration import conf from airflow.models.abstractoperator import TaskStateChangeCallback from airflow.sdk.definitions._internal.abstractoperator import ( DEFAULT_EXECUTOR, @@ -82,6 +83,9 @@ TaskStateChangeCallbackAttrType = TaskStateChangeCallback | list[TaskStateChangeCallback] | None ValidationSource = Literal["expand"] | Literal["partial"] +# AIP-88: https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+AIP-88%3A+Lazy+task+expansion +enable_lazy_task_expansion = conf.getboolean("scheduler", "enable_lazy_task_expansion", fallback=False) + def validate_mapping_kwargs(op: type[BaseOperator], func: ValidationSource, value: dict[str, Any]) -> None: # use a dict so order of args is same as code order diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 48ddc534b9a4b..0da42e003dff7 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -25,10 +25,10 @@ from typing import TYPE_CHECKING, Any, overload from airflow.exceptions import AirflowException, XComNotFound +from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion from airflow.sdk.definitions._internal.abstractoperator import AbstractOperator from airflow.sdk.definitions._internal.mixins import DependencyMixin, ResolveMixin from airflow.sdk.definitions._internal.types import NOTSET, ArgNotSet -from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.execution_time.comms import XComResult from airflow.sdk.execution_time.lazy_sequence import LazyXComSequence from airflow.sdk.execution_time.xcom import XCom @@ -516,7 +516,7 @@ def resolve(self, context: Mapping[str, Any]) -> Any: if isinstance(value, (Sequence, dict)): return _MapResult(value, self.callables) - if isinstance(value, Iterable): + if isinstance(value, Iterable) and enable_lazy_task_expansion: return _LazyMapResult(value, self.callables) raise ValueError(f"XCom map expects sequence or dict, not {type(value).__name__}") 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 09cd8d57cd248..49ac801480464 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -42,7 +42,6 @@ from airflow.dag_processing.bundles.manager import DagBundlesManager from airflow.exceptions import AirflowInactiveAssetInInletOrOutletException from airflow.listeners.listener import get_listener_manager -from airflow.models.taskmap import enable_lazy_task_expansion from airflow.sdk.api.datamodels._generated import ( AssetProfile, TaskInstance, @@ -53,7 +52,7 @@ from airflow.sdk.definitions._internal.dag_parsing_context import _airflow_parsing_context_manager from airflow.sdk.definitions._internal.types import NOTSET, ArgNotSet from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetNameRef, AssetUniqueKey, AssetUriRef -from airflow.sdk.definitions.mappedoperator import MappedOperator +from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion from airflow.sdk.definitions.param import process_params from airflow.sdk.exceptions import AirflowRuntimeError, ErrorType from airflow.sdk.execution_time.callback_runner import create_executable_runner From 9a44fb0a1a0546988b89f6a44a9e99ab0cee6408 Mon Sep 17 00:00:00 2001 From: David Blain Date: Thu, 3 Jul 2025 16:27:01 +0200 Subject: [PATCH 038/338] refactor: Refactored expand inputs resolve methods to support both resolving methods when AIP-88 is enabled or not and removed values method --- .../airflow/jobs/expand_task_job_runner.py | 7 +-- .../src/airflow/models/deferred_iterable.py | 4 +- .../src/airflow/models/taskinstance.py | 3 +- .../sdk/definitions/_internal/expandinput.py | 56 ++++++++++++------- .../airflow/sdk/definitions/mappedoperator.py | 11 +++- .../src/airflow/sdk/definitions/xcom_arg.py | 10 ++-- 6 files changed, 55 insertions(+), 36 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 5ba8dc47f740d..c4a15b02b2461 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -98,15 +98,12 @@ def expand_unmapped_task_instance( task_instances_batch = [] context = unmapped_ti.get_template_context(session=session) - expand_input = unmapped_ti.task.expand_input.values() + expand_input = unmapped_ti.task.expand_input.resolve(context) self.log.info("expand_input: %s", expand_input) - if isinstance(expand_input, XComArg): - expand_input = expand_input.resolve(context) - for map_index, _ in enumerate(expand_input): - # if map_index > 40: + # if map_index > 200: # self.log.warning("Stop expanding tasks over %s!", map_index) # break diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index 0715c9cb009fb..842f97ab7b26f 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -18,7 +18,7 @@ from __future__ import annotations import asyncio -from collections.abc import Generator, Iterable, Iterator, Sequence, Sized +from collections.abc import Generator, Iterable, Iterator from contextlib import contextmanager, suppress from typing import TYPE_CHECKING, Any @@ -26,11 +26,11 @@ from wrapt import synchronized from airflow.exceptions import AirflowException -from airflow.models.xcom import XComModel from airflow.sdk.bases.operator import BaseOperator as Operator from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions.context import Context from airflow.sdk.definitions.xcom_arg import MapXComArg # noqa: F401 +from airflow.sdk.execution_time.xcom import XCom from airflow.serialization import serde from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index f10ce95909c66..1df3dce8b57d6 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -85,6 +85,7 @@ from airflow.models.taskreschedule import TaskReschedule from airflow.models.xcom import LazyXComSelectSequence, XComModel from airflow.plugins_manager import integrate_macros_plugins +from airflow.sdk.definitions._internal.expandinput import NotFullyPopulated from airflow.settings import task_instance_mutation_hook from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext @@ -2036,7 +2037,7 @@ def get_triggering_events() -> dict[str, list[AssetEvent]]: for upstream in task.upstream_list }, ) - except NotMapped: + except (NotMapped, NotFullyPopulated): pass return context diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py index 3fdff131ecd66..1e0f01283060d 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py @@ -19,12 +19,14 @@ import functools import operator -from collections.abc import Iterable, Mapping, Sequence, Sized +from collections.abc import Iterable, Generator, Mapping, Sequence, Sized from typing import TYPE_CHECKING, Any, ClassVar, Union import attrs +from airflow.sdk import XComArg from airflow.sdk.definitions._internal.mixins import ResolveMixin +from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion if TYPE_CHECKING: from airflow.sdk.definitions.xcom_arg import XComArg @@ -193,28 +195,38 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() - def values(self) -> Iterable[Any]: - return self.value.values() + def resolve(self, context: Mapping[str, Any]) -> Generator[Mapping[str, Any], tuple[Mapping[str, Any], set[int]]]: + if enable_lazy_task_expansion: + for key in self.value: + value = self.value[key] - def resolve(self, context: Mapping[str, Any]) -> tuple[Mapping[str, Any], set[int]]: - map_index: int | None = context["ti"].map_index - if map_index is None or map_index < 0: - raise RuntimeError("can't resolve task-mapping argument without expanding") + if _needs_run_time_resolution(value): + self.value[key] = value = value.resolve(context=context) - upstream_map_indexes = getattr(context["ti"], "_upstream_map_indexes", {}) + if isinstance(value, (Iterable, Sequence)) and not isinstance(value, str): + for item in value: + yield {key: item} + else: + yield {key: value} + else: + map_index: int | None = context["ti"].map_index + if map_index is None or map_index < 0: + raise RuntimeError("can't resolve task-mapping argument without expanding") - # TODO: This initiates one API call for each XComArg. Would it be - # more efficient to do one single call and unpack the value here? - resolved = { - k: v.resolve(context) if _needs_run_time_resolution(v) else v for k, v in self.value.items() - } + upstream_map_indexes = getattr(context["ti"], "_upstream_map_indexes", {}) + + # TODO: This initiates one API call for each XComArg. Would it be + # more efficient to do one single call and unpack the value here? + resolved = { + k: v.resolve(context) if _needs_run_time_resolution(v) else v for k, v in self.value.items() + } - all_lengths = self._get_map_lengths(resolved, upstream_map_indexes) + all_lengths = self._get_map_lengths(resolved, upstream_map_indexes) - data = {k: self._expand_mapped_field(k, v, map_index, all_lengths) for k, v in resolved.items()} - literal_keys = {k for k, _ in self._iter_parse_time_resolved_kwargs()} - resolved_oids = {id(v) for k, v in data.items() if k not in literal_keys} - return data, resolved_oids + data = {k: self._expand_mapped_field(k, v, map_index, all_lengths) for k, v in resolved.items()} + literal_keys = {k for k, _ in self._iter_parse_time_resolved_kwargs()} + resolved_oids = {id(v) for k, v in data.items() if k not in literal_keys} + return data, resolved_oids def _describe_type(value: Any) -> str: @@ -250,10 +262,12 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() - def values(self) -> Iterable[Any]: - return self.value + def resolve(self, context: Mapping[str, Any]) -> Sequence[XComArg | Mapping[str, Any]] | tuple[Mapping[str, Any], set[int]]: + if enable_lazy_task_expansion: + if _needs_run_time_resolution(self.value): + self.value = self.value.resolve(context) + return self.value - def resolve(self, context: Mapping[str, Any]) -> tuple[Mapping[str, Any], set[int]]: map_index = context["ti"].map_index if map_index < 0: raise RuntimeError("can't resolve task-mapping argument without expanding") diff --git a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py index eebf9cb5d3e28..107c75fa7ffb0 100644 --- a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py +++ b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py @@ -21,6 +21,7 @@ import copy import warnings from collections.abc import Collection, Iterable, Iterator, Mapping, Sequence +from itertools import islice from typing import TYPE_CHECKING, Any, ClassVar import attrs @@ -694,7 +695,15 @@ def _expand_mapped_kwargs(self, context: Mapping[str, Any]) -> tuple[Mapping[str This exists because taskflow operators expand against op_kwargs, not the entire operator kwargs dict. """ - return self._get_specified_expand_input().resolve(context) + if not enable_lazy_task_expansion: + return self._get_specified_expand_input().resolve(context) + + map_index = context["ti"].map_index + self.log.debug("map_index: %s", map_index) + iterator = self._get_specified_expand_input().resolve(context) + kwargs = next(iterator) if map_index == 0 else next(islice(iterator, map_index, map_index + 1)) + self.log.debug("kwargs: %s", kwargs) + return kwargs, {id(kwargs)} def _get_unmap_kwargs(self, mapped_kwargs: Mapping[str, Any], *, strict: bool) -> dict[str, Any]: """ diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 0da42e003dff7..104ccc42a779c 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -373,15 +373,13 @@ def resolve(self, context: Mapping[str, Any]) -> Any: self.log.debug("value: %s", value) - # TODO: check why this is needed? + # TODO: check why this is needed when resolving from TaskExpansionJobRunner? if isinstance(value, str): - deserialized_value = XCom.deserialize_value( + value = XCom.deserialize_value( XComResult(key=self.operator.output.key, value=value) ) - self.log.debug("deserialized_value: %s", deserialized_value) - if isinstance(deserialized_value, ResolveMixin): - deserialized_value = deserialized_value.resolve(context) - return deserialized_value + if isinstance(value, ResolveMixin): + value = value.resolve(context) return value From 1a7fe5e5ece4ff19aa628aab169c3ac64be2474e Mon Sep 17 00:00:00 2001 From: David Blain Date: Fri, 4 Jul 2025 15:14:51 +0200 Subject: [PATCH 039/338] refactor: Refactored AIP-88 to be Aiflow 3.0.2 compatible --- .../airflow/jobs/expand_task_job_runner.py | 67 +++++++++---------- airflow-core/src/airflow/jobs/job.py | 26 ++----- .../airflow/sdk/definitions/mappedoperator.py | 5 +- .../src/airflow/sdk/definitions/xcom_arg.py | 26 ++++--- 4 files changed, 59 insertions(+), 65 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index c4a15b02b2461..add0dffcf2489 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -26,11 +26,10 @@ from airflow.jobs.job import Job, run_job_async from airflow.models import DagRun, DagBag from airflow.policies import task_instance_mutation_hook -from airflow.sdk import XComArg from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.ti_deps.dep_context import DepContext from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.session import NEW_SESSION, create_session +from airflow.utils.session import create_session from airflow.utils.state import DagRunState, State if TYPE_CHECKING: @@ -82,7 +81,7 @@ def _persist_task_instances( task_instances.clear() def expand_unmapped_task_instance( - self, dag_run: DagRun, unmapped_ti: TaskInstance, session: Session = NEW_SESSION + self, dag_run: DagRun, unmapped_ti: TaskInstance, session: Session ) -> None: """ Expands the task using the provided expand_input. @@ -100,14 +99,14 @@ def expand_unmapped_task_instance( context = unmapped_ti.get_template_context(session=session) expand_input = unmapped_ti.task.expand_input.resolve(context) - self.log.info("expand_input: %s", expand_input) + self.log.debug("expand_input: %s", expand_input) for map_index, _ in enumerate(expand_input): - # if map_index > 200: - # self.log.warning("Stop expanding tasks over %s!", map_index) - # break + if map_index > 200: + self.log.warning("Stop expanding tasks over %s!", map_index) + break - self.log.info("map_index: %s", map_index) + self.log.debug("map_index: %s", map_index) if map_index == 0: task_instance = unmapped_ti @@ -140,35 +139,35 @@ def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: def has_mapped_operator(task_instance: TaskInstance) -> bool: return isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 and task_instance.state in State.unfinished - def expand_tasks(self, session: Session): - dag_bag = DagBag() - dag_runs = DagRun.get_running_dag_runs_to_examine(session=session) - - for dag_run in dag_runs: - dag = dag_bag.get_dag(dag_run.dag_id) - self.log.info("Checking for unmapped task instances on: %s", dag_run) - for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): - try: - finished_tis = list(map(lambda task: self.get_task(dag, task), filter(lambda ti: ti.state in State.finished, dag_run.task_instances))) - dep_context = DepContext( - flag_upstream_failed=True, - ignore_unmapped_tasks=True, # Ignore this Dep, as we will expand it if we can. - finished_tis=finished_tis, - ) - self.log.info("Unmapped task state on: %s", unmapped_ti.state) - are_dependencies_met = unmapped_ti.are_dependencies_met(dep_context=dep_context, session=session, verbose=True) - self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) - if are_dependencies_met: - self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) - except Exception: - self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) - - def _execute(self, session: Session = NEW_SESSION) -> int | None: + def expand_tasks(self): + with create_session(scoped=False) as session: + dag_bag = DagBag() + dag_runs = DagRun.get_running_dag_runs_to_examine(session=session) + + for dag_run in dag_runs: + dag = dag_bag.get_dag(dag_run.dag_id) + self.log.info("Checking for unmapped task instances on: %s", dag_run) + for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): + try: + finished_tis = list(map(lambda task: self.get_task(dag, task), filter(lambda ti: ti.state in State.finished, dag_run.task_instances))) + dep_context = DepContext( + flag_upstream_failed=True, + ignore_unmapped_tasks=True, # Ignore this Dep, as we will expand it if we can. + finished_tis=finished_tis, + ) + self.log.info("Unmapped task state on: %s", unmapped_ti.state) + are_dependencies_met = unmapped_ti.are_dependencies_met(dep_context=dep_context, session=session, verbose=True) + self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) + if are_dependencies_met: + self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) + except Exception: + self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) + + def _execute(self) -> int | None: self.log.info("TaskExpansionJobRunner started") while self.trigger_runner.is_alive(): - with create_session(scoped=False) as session: - self.expand_tasks(session=session) + self.expand_tasks() time.sleep(self.job.heartrate) self.log.info("TaskExpansionJobRunner stopped") diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index d493b94dfe243..7d89a16353e28 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -22,7 +22,7 @@ from functools import cached_property, lru_cache from multiprocessing import Process from time import sleep -from typing import TYPE_CHECKING, Callable, NoReturn, Union +from typing import TYPE_CHECKING, NoReturn, Union from sqlalchemy import Column, Index, Integer, String, case, select from sqlalchemy.exc import OperationalError @@ -354,19 +354,17 @@ def run_job( """ job.prepare_for_execution(session=session) try: - return execute_job(job, execute_callable=execute_callable, session=session) + return execute_job(job, execute_callable=execute_callable) finally: job.complete_execution(session=session) -def run_job_async( - job: Job, execute_callable: Union[Callable[[], int | None], Callable[[Session], int | None]] -) -> int | None: +def run_job_async(job: Job, execute_callable: Callable[[], int | None]) -> int | None: """ Run the job asynchronously. The Job is always an ORM object and setting the state is happening within the - same DB session and the session is kept open throughout the whole asynchronous execution. + same DB session and the session is kept open throughout the whole execution. :meta private: """ @@ -374,21 +372,13 @@ def execute_async_job() -> int | None: asyncio.set_event_loop(asyncio.new_event_loop()) with create_session(scoped=False) as session: - job.prepare_for_execution(session=session) - try: - return execute_job(job, execute_callable=execute_callable, session=session) - finally: - job.complete_execution(session=session) + run_job(job=job, execute_callable=execute_callable, session=session) process = Process(target=execute_async_job) process.start() -def execute_job( - job: Job, - execute_callable: Union[Callable[[], int | None], Callable[[Session], int | None]], - session: Session | None = None, -) -> int | None: +def execute_job(job: Job, execute_callable: Callable[[], int | None]) -> int | None: """ Execute the job. @@ -408,13 +398,11 @@ def execute_job( :param execute_callable: callable to execute when running the job. - :param session: Optional session to use (new since AIP-88). - :meta private: """ ret = None try: - ret = execute_callable(session) if session else execute_callable() + ret = execute_callable() # In case of max runs or max duration job.state = JobState.SUCCESS except SystemExit: diff --git a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py index e6e42b97e0219..19677f543357d 100644 --- a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py +++ b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py @@ -691,8 +691,9 @@ def _expand_mapped_kwargs(self, context: Mapping[str, Any]) -> tuple[Mapping[str map_index = context["ti"].map_index self.log.debug("map_index: %s", map_index) - iterator = self._get_specified_expand_input().resolve(context) - kwargs = next(iterator) if map_index == 0 else next(islice(iterator, map_index, map_index + 1)) + expand_input = self._get_specified_expand_input().resolve(context) + self.log.debug("expand_input: %s", expand_input) + kwargs = next(expand_input) if map_index == 0 else next(islice(expand_input, map_index, map_index + 1)) self.log.debug("kwargs: %s", kwargs) return kwargs, {id(kwargs)} diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 921f33032bbb5..b9172201ccd86 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -25,6 +25,7 @@ from typing import TYPE_CHECKING, Any, overload from airflow.exceptions import AirflowException, XComNotFound +from airflow.models.xcom import XComModel from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion from airflow.sdk.definitions._internal.abstractoperator import AbstractOperator from airflow.sdk.definitions._internal.mixins import DependencyMixin, ResolveMixin @@ -371,15 +372,20 @@ def _resolve(self, context: Mapping[str, Any]) -> Any: def resolve(self, context: Mapping[str, Any]) -> Any: value = self._resolve(context) - self.log.debug("value: %s", value) + self.log.debug("value (%s): %s", type(value), value) # TODO: check why this is needed when resolving from TaskExpansionJobRunner? if isinstance(value, str): - value = XCom.deserialize_value( - XComResult(key=self.operator.output.key, value=value) - ) - if isinstance(value, ResolveMixin): + result = XComResult(key=self.operator.output.key, value=value) + value = XCom.deserialize_value(result) + if isinstance(value, str): + value = XComModel.deserialize_value(result) + + self.log.debug("deserialized value (%s): %s", type(value), value) + + if isinstance(value, ResolveMixin): # Only needed for DeferredIterable value = value.resolve(context) + return value @@ -396,7 +402,7 @@ def _get_callable_name(f: Callable | str) -> str: return "" -class _MappableResult(Sequence): +class _MappableResult(Iterable): def __init__( self, value: Sequence | dict, callables: FilterCallables | MapCallables ) -> None: @@ -409,14 +415,14 @@ def __getitem__(self, index: Any) -> Any: def __len__(self) -> int: raise NotImplementedError - @staticmethod - def _convert(value: Sequence | dict) -> list: + @classmethod + def _convert(cls, value: Sequence | dict) -> list: if isinstance(value, (dict, set)): return list(value) if isinstance(value, list): return value raise ValueError( - f"XCom filter expects sequence or dict, not {type(value).__name__}" + f"{cls.__name__} expects sequence or dict, not {type(value).__name__}" ) def _apply_callables(self, value) -> Any: @@ -428,7 +434,7 @@ def __str__(self): return f"{self.__class__.__name__}({self.value}, {self.callables})" -class _MapResult(_MappableResult): +class _MapResult(_MappableResult, Sequence): def __getitem__(self, index: Any) -> Any: value = self._apply_callables(self.value[index]) return value From 3dd0a16b3288c4ebff8164310862d152f36bc8a0 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 14:25:55 +0200 Subject: [PATCH 040/338] refactor: Fixed inner import of BaseSensorOperator in mappedoperator --- task-sdk/src/airflow/sdk/definitions/mappedoperator.py | 1 - 1 file changed, 1 deletion(-) diff --git a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py index 19677f543357d..98d306b0e001f 100644 --- a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py +++ b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py @@ -28,7 +28,6 @@ import methodtools from airflow.configuration import conf -from airflow.models.abstractoperator import TaskStateChangeCallback from airflow.sdk.bases.xcom import BaseXCom from airflow.sdk.definitions._internal.abstractoperator import ( DEFAULT_EXECUTOR, From c25a918fbebafbfb4d5c14bfa37f9cfb2b53845c Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 14:33:33 +0200 Subject: [PATCH 041/338] refactor: Moved enable_lazy_task_expansion flag to root of sdk definitions module --- airflow-core/src/airflow/jobs/triggerer_job_runner.py | 1 + airflow-core/src/airflow/models/dagrun.py | 3 ++- airflow-core/src/airflow/models/taskmap.py | 3 ++- task-sdk/src/airflow/sdk/definitions/__init__.py | 8 ++++++++ .../src/airflow/sdk/definitions/_internal/expandinput.py | 2 +- task-sdk/src/airflow/sdk/definitions/xcom_arg.py | 2 +- task-sdk/src/airflow/sdk/execution_time/task_runner.py | 3 ++- 7 files changed, 17 insertions(+), 5 deletions(-) diff --git a/airflow-core/src/airflow/jobs/triggerer_job_runner.py b/airflow-core/src/airflow/jobs/triggerer_job_runner.py index 771f112ecedc7..8bb4b8fdb0a0a 100644 --- a/airflow-core/src/airflow/jobs/triggerer_job_runner.py +++ b/airflow-core/src/airflow/jobs/triggerer_job_runner.py @@ -46,6 +46,7 @@ from airflow.models.trigger import Trigger from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion from airflow.sdk.api.datamodels._generated import HITLDetailResponse +from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.execution_time.comms import ( CommsDecoder, ConnectionResult, diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index d81d69c2b1fd2..802b3d9fc91b5 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -73,7 +73,8 @@ from airflow.models.taskinstancehistory import TaskInstanceHistory as TIH from airflow.models.tasklog import LogTemplate from airflow.models.taskmap import TaskMap -from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion +from airflow.sdk.definitions import enable_lazy_task_expansion +from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.sdk.definitions.deadline import DeadlineReference from airflow.stats import Stats diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 04e923af98f09..3ef96ccd62cc8 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -134,8 +134,9 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq from airflow.models.mappedoperator import get_mapped_ti_count from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator - from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion from airflow.serialization.serialized_objects import SerializedBaseOperator + from airflow.sdk.definitions import enable_lazy_task_expansion + from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.settings import task_instance_mutation_hook diff --git a/task-sdk/src/airflow/sdk/definitions/__init__.py b/task-sdk/src/airflow/sdk/definitions/__init__.py index 13a83393a9124..9852abf18f019 100644 --- a/task-sdk/src/airflow/sdk/definitions/__init__.py +++ b/task-sdk/src/airflow/sdk/definitions/__init__.py @@ -14,3 +14,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + +from __future__ import annotations + +from airflow.configuration import conf + + +# AIP-88: https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+AIP-88%3A+Lazy+task+expansion +enable_lazy_task_expansion = conf.getboolean("scheduler", "enable_lazy_task_expansion", fallback=False) diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py index 3b8cf857efe09..cd90172137ac4 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py @@ -25,8 +25,8 @@ import attrs from airflow.sdk import XComArg +from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.definitions._internal.mixins import ResolveMixin -from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion if TYPE_CHECKING: from typing import TypeGuard diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index b9172201ccd86..6539a8b647f74 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -26,7 +26,7 @@ from airflow.exceptions import AirflowException, XComNotFound from airflow.models.xcom import XComModel -from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion +from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.definitions._internal.abstractoperator import AbstractOperator from airflow.sdk.definitions._internal.mixins import DependencyMixin, ResolveMixin from airflow.sdk.definitions._internal.setup_teardown import SetupTeardownContext 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 1e3b9fed9993d..948d40b76a140 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -54,8 +54,9 @@ from airflow.sdk.bases.xcom import BaseXCom from airflow.sdk.definitions._internal.dag_parsing_context import _airflow_parsing_context_manager from airflow.sdk.definitions._internal.types import NOTSET, ArgNotSet +from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetNameRef, AssetUniqueKey, AssetUriRef -from airflow.sdk.definitions.mappedoperator import MappedOperator, enable_lazy_task_expansion +from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.param import process_params from airflow.sdk.exceptions import AirflowRuntimeError, ErrorType from airflow.sdk.execution_time.callback_runner import create_executable_runner From 2253358bf810861dc6224f546610c4a42f4a7bf0 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 15:10:19 +0200 Subject: [PATCH 042/338] refactor: Fixed expand_mapped_task method in TaskMap --- airflow-core/src/airflow/models/taskmap.py | 31 +--------------------- 1 file changed, 1 insertion(+), 30 deletions(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index 3ef96ccd62cc8..e80e7a21f920a 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -144,38 +144,10 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq if not isinstance(task, (BaseOperator, MappedOperator, SerializedBaseOperator)): raise RuntimeError( f"cannot expand unrecognized operator type {type(task).__module__}.{type(task).__name__}" - - try: - total_length: int | None = get_mapped_ti_count(task, run_id, session=session) - except NotFullyPopulated as e: - if not task.dag or not task.dag.partial: - task.log.error( - "Cannot expand %r for run %s; missing upstream values: %s", - task, - run_id, - sorted(e.missing), ) - total_length = None - - state: TaskInstanceState | None = None - unmapped_ti: TaskInstance | None = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == task.dag_id, - TaskInstance.task_id == task.task_id, - TaskInstance.run_id == run_id, - TaskInstance.map_index == -1, - or_(TaskInstance.state.in_(State.unfinished), TaskInstance.state.is_(None)), - ) - ).one_or_none() - - all_expanded_tis: list[TaskInstance] = [] - - if unmapped_ti: - if TYPE_CHECKING: - assert task.dag is None or isinstance(task.dag, SchedulerDAG) try: - total_length: int | None = DBBaseOperator.get_mapped_ti_count(task, run_id, session=session) + total_length: int | None = get_mapped_ti_count(task, run_id, session=session) except NotFullyPopulated as e: if not task.dag or not task.dag.partial: task.log.error( @@ -240,7 +212,6 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq task.log.debug("Deleting the original task instance: %s", unmapped_ti) session.delete(unmapped_ti) state = unmapped_ti.state - dag_version_id = unmapped_ti.dag_version_id if total_length is None or total_length < 1: # Nothing to fixup. From 710eedace8abf7259a8a1cc45367e115dc8da14e Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 15:53:20 +0200 Subject: [PATCH 043/338] refactor: Fixed static checks --- task-sdk/src/airflow/sdk/definitions/__init__.py | 1 - .../airflow/sdk/definitions/_internal/expandinput.py | 11 ++++++++--- .../src/airflow/sdk/definitions/mappedoperator.py | 4 +++- task-sdk/src/airflow/sdk/definitions/xcom_arg.py | 8 ++------ .../src/airflow/sdk/execution_time/task_runner.py | 2 +- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/task-sdk/src/airflow/sdk/definitions/__init__.py b/task-sdk/src/airflow/sdk/definitions/__init__.py index 9852abf18f019..de7b2cff16416 100644 --- a/task-sdk/src/airflow/sdk/definitions/__init__.py +++ b/task-sdk/src/airflow/sdk/definitions/__init__.py @@ -19,6 +19,5 @@ from airflow.configuration import conf - # AIP-88: https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+AIP-88%3A+Lazy+task+expansion enable_lazy_task_expansion = conf.getboolean("scheduler", "enable_lazy_task_expansion", fallback=False) diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py index cd90172137ac4..6532d8eae2a59 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py @@ -19,7 +19,7 @@ import functools import operator -from collections.abc import Iterable, Generator, Mapping, Sequence, Sized +from collections.abc import Generator, Iterable, Mapping, Sequence, Sized from typing import TYPE_CHECKING, Any, ClassVar, Union import attrs @@ -196,7 +196,9 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() - def resolve(self, context: Mapping[str, Any]) -> Generator[Mapping[str, Any], tuple[Mapping[str, Any], set[int]]]: + def resolve( + self, context: Mapping[str, Any] + ) -> Generator[Mapping[str, Any], tuple[Mapping[str, Any], set[int]]]: if enable_lazy_task_expansion: for key in self.value: value = self.value[key] @@ -265,7 +267,10 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: if isinstance(x, XComArg): yield from x.iter_references() - def resolve(self, context: Mapping[str, Any]) -> Sequence[XComArg | Mapping[str, Any]] | tuple[Mapping[str, Any], set[int]]: + def resolve( + self, context: Mapping[str, Any] + ) -> Sequence[XComArg | Mapping[str, Any]] | tuple[ + Mapping[str, Any], set[int]]: if enable_lazy_task_expansion: if _needs_run_time_resolution(self.value): self.value = self.value.resolve(context) diff --git a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py index 98d306b0e001f..7ee0990405bda 100644 --- a/task-sdk/src/airflow/sdk/definitions/mappedoperator.py +++ b/task-sdk/src/airflow/sdk/definitions/mappedoperator.py @@ -692,7 +692,9 @@ def _expand_mapped_kwargs(self, context: Mapping[str, Any]) -> tuple[Mapping[str self.log.debug("map_index: %s", map_index) expand_input = self._get_specified_expand_input().resolve(context) self.log.debug("expand_input: %s", expand_input) - kwargs = next(expand_input) if map_index == 0 else next(islice(expand_input, map_index, map_index + 1)) + kwargs = ( + next(expand_input) if map_index == 0 else next(islice(expand_input, map_index, map_index + 1)) + ) self.log.debug("kwargs: %s", kwargs) return kwargs, {id(kwargs)} diff --git a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py index 6539a8b647f74..8990da3208dd8 100644 --- a/task-sdk/src/airflow/sdk/definitions/xcom_arg.py +++ b/task-sdk/src/airflow/sdk/definitions/xcom_arg.py @@ -403,9 +403,7 @@ def _get_callable_name(f: Callable | str) -> str: class _MappableResult(Iterable): - def __init__( - self, value: Sequence | dict, callables: FilterCallables | MapCallables - ) -> None: + def __init__(self, value: Sequence | dict, callables: FilterCallables | MapCallables) -> None: self.value = self._convert(value) self.callables = callables @@ -421,9 +419,7 @@ def _convert(cls, value: Sequence | dict) -> list: return list(value) if isinstance(value, list): return value - raise ValueError( - f"{cls.__name__} expects sequence or dict, not {type(value).__name__}" - ) + raise ValueError(f"{cls.__name__} expects sequence or dict, not {type(value).__name__}") def _apply_callables(self, value) -> Any: for func in self.callables: 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 948d40b76a140..75652e16a2184 100644 --- a/task-sdk/src/airflow/sdk/execution_time/task_runner.py +++ b/task-sdk/src/airflow/sdk/execution_time/task_runner.py @@ -52,9 +52,9 @@ ) from airflow.sdk.bases.operator import BaseOperator, ExecutorSafeguard from airflow.sdk.bases.xcom import BaseXCom +from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.definitions._internal.dag_parsing_context import _airflow_parsing_context_manager from airflow.sdk.definitions._internal.types import NOTSET, ArgNotSet -from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetNameRef, AssetUniqueKey, AssetUriRef from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions.param import process_params From b7ecb9640df7b2332762aeb3b19f2b2895a2a9f8 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:52:27 +0200 Subject: [PATCH 044/338] refactor: Reformatted resolve method of ListOfDictsExpandInput --- task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py index 6532d8eae2a59..101bc4fb38245 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/expandinput.py @@ -269,8 +269,7 @@ def iter_references(self) -> Iterable[tuple[Operator, str]]: def resolve( self, context: Mapping[str, Any] - ) -> Sequence[XComArg | Mapping[str, Any]] | tuple[ - Mapping[str, Any], set[int]]: + ) -> Sequence[XComArg | Mapping[str, Any]] | tuple[Mapping[str, Any], set[int]]: if enable_lazy_task_expansion: if _needs_run_time_resolution(self.value): self.value = self.value.resolve(context) From f095297ad9b9e38f2eb37341dc9ec66fd332803e Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:53:16 +0200 Subject: [PATCH 045/338] refactor: Reorganized imports in expand_mapped_task method of TaskMap --- airflow-core/src/airflow/models/taskmap.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index e80e7a21f920a..c705b52e4ff98 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -134,10 +134,10 @@ def expand_mapped_task(cls, task, run_id: str, *, session: Session) -> tuple[Seq from airflow.models.mappedoperator import get_mapped_ti_count from airflow.models.taskinstance import TaskInstance from airflow.sdk.bases.operator import BaseOperator - from airflow.serialization.serialized_objects import SerializedBaseOperator from airflow.sdk.definitions import enable_lazy_task_expansion - from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions._internal.abstractoperator import NotMapped + from airflow.sdk.definitions.mappedoperator import MappedOperator + from airflow.serialization.serialized_objects import SerializedBaseOperator from airflow.settings import task_instance_mutation_hook if not enable_lazy_task_expansion: From d2e7cb37b5fdda396f02c5bffeefe8c3888643fa Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:55:14 +0200 Subject: [PATCH 046/338] refactor: Removed unused import in TaskInstance --- airflow-core/src/airflow/models/taskinstance.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow-core/src/airflow/models/taskinstance.py b/airflow-core/src/airflow/models/taskinstance.py index d79125db2ac61..1bb55d99523b7 100644 --- a/airflow-core/src/airflow/models/taskinstance.py +++ b/airflow-core/src/airflow/models/taskinstance.py @@ -89,7 +89,6 @@ from airflow.models.taskmap import TaskMap from airflow.models.taskreschedule import TaskReschedule from airflow.models.xcom import XCOM_RETURN_KEY, LazyXComSelectSequence, XComModel -from airflow.plugins_manager import integrate_macros_plugins from airflow.sdk.definitions._internal.expandinput import NotFullyPopulated from airflow.settings import task_instance_mutation_hook from airflow.stats import Stats From 2468e8fd4ddd23107df87925a06f864e67f64ae1 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:56:05 +0200 Subject: [PATCH 047/338] refactor: Reorganized imports in ExpandInput --- airflow-core/src/airflow/models/expandinput.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 9f68e559bfa6c..803b0ee2566a8 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -19,8 +19,8 @@ import functools import operator -from collections.abc import Generator, Iterable, Mapping, Sized -from typing import TYPE_CHECKING, Any, ClassVar, Union +from collections.abc import Iterable, Sized +from typing import TYPE_CHECKING, Any, ClassVar import attrs from airflow.utils.log.logging_mixin import LoggingMixin From 9da6ef9a975f6cb4a57a5accac1114ae38e75a3b Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:57:38 +0200 Subject: [PATCH 048/338] refactor: Removed unused import in DeferredIterable --- airflow-core/src/airflow/models/deferred_iterable.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index 842f97ab7b26f..d314a9d43b40e 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -26,11 +26,11 @@ from wrapt import synchronized from airflow.exceptions import AirflowException +from airflow.models.xcom import XComModel from airflow.sdk.bases.operator import BaseOperator as Operator from airflow.sdk.definitions._internal.mixins import ResolveMixin from airflow.sdk.definitions.context import Context from airflow.sdk.definitions.xcom_arg import MapXComArg # noqa: F401 -from airflow.sdk.execution_time.xcom import XCom from airflow.serialization import serde from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.utils.log.logging_mixin import LoggingMixin From a953e05174b440ba3760822d412beb628174adf8 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 20:58:43 +0200 Subject: [PATCH 049/338] refactor: Reorganized imports in DagRun --- airflow-core/src/airflow/models/dagrun.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index 802b3d9fc91b5..c6c00cc42bb84 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -74,9 +74,9 @@ from airflow.models.tasklog import LogTemplate from airflow.models.taskmap import TaskMap from airflow.sdk.definitions import enable_lazy_task_expansion -from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.sdk.definitions._internal.abstractoperator import NotMapped from airflow.sdk.definitions.deadline import DeadlineReference +from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_states import SCHEDULEABLE_STATES From 728146377ff7b495165d543b30663ef2aac4cccd Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 21:00:32 +0200 Subject: [PATCH 050/338] refactor: Removed duplicate import in TriggererJobRunner --- airflow-core/src/airflow/jobs/triggerer_job_runner.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/airflow-core/src/airflow/jobs/triggerer_job_runner.py b/airflow-core/src/airflow/jobs/triggerer_job_runner.py index 8bb4b8fdb0a0a..1a14e77944e34 100644 --- a/airflow-core/src/airflow/jobs/triggerer_job_runner.py +++ b/airflow-core/src/airflow/jobs/triggerer_job_runner.py @@ -44,7 +44,6 @@ from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import perform_heartbeat from airflow.models.trigger import Trigger -from airflow.sdk.definitions.mappedoperator import enable_lazy_task_expansion from airflow.sdk.api.datamodels._generated import HITLDetailResponse from airflow.sdk.definitions import enable_lazy_task_expansion from airflow.sdk.execution_time.comms import ( @@ -166,7 +165,9 @@ def _execute(self) -> int | None: ) if enable_lazy_task_expansion: - task_expansion_run(self.job.heartrate, self.trigger_runner) # TODO: job shouldn't be started here + task_expansion_run( + self.job.heartrate, self.trigger_runner + ) # TODO: job shouldn't be started here # Run the main DB comms loop in this process self.trigger_runner.run() From be93d9317d0f2a86c0652c6eda2daf33f0e80521 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 21:02:25 +0200 Subject: [PATCH 051/338] refactor: Reformatted Job --- airflow-core/src/airflow/jobs/job.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/job.py b/airflow-core/src/airflow/jobs/job.py index 7d89a16353e28..130f360070606 100644 --- a/airflow-core/src/airflow/jobs/job.py +++ b/airflow-core/src/airflow/jobs/job.py @@ -22,7 +22,7 @@ from functools import cached_property, lru_cache from multiprocessing import Process from time import sleep -from typing import TYPE_CHECKING, NoReturn, Union +from typing import TYPE_CHECKING, NoReturn from sqlalchemy import Column, Index, Integer, String, case, select from sqlalchemy.exc import OperationalError @@ -368,6 +368,7 @@ def run_job_async(job: Job, execute_callable: Callable[[], int | None]) -> int | :meta private: """ + def execute_async_job() -> int | None: asyncio.set_event_loop(asyncio.new_event_loop()) From f32a55cc3269c54a7fa6c638649c94697fec9b1b Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 21:12:50 +0200 Subject: [PATCH 052/338] refactor: Reformatted TaskExpansionJobRunner --- .../airflow/jobs/expand_task_job_runner.py | 45 ++++++++++++++----- 1 file changed, 34 insertions(+), 11 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index add0dffcf2489..625ff0f3a7973 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -24,7 +24,7 @@ from airflow.exceptions import AirflowException from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import Job, run_job_async -from airflow.models import DagRun, DagBag +from airflow.models import DagBag, DagRun from airflow.policies import task_instance_mutation_hook from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.ti_deps.dep_context import DepContext @@ -39,8 +39,6 @@ from airflow.jobs.triggerer_job_runner import TriggerRunnerSupervisor from airflow.models.taskinstance import TaskInstance -task_expansion_batch_size = conf.getint("scheduler", "task_expansion_batch_size", fallback=10) - class TaskExpansionJobRunner(BaseJobRunner, LoggingMixin): def __init__( @@ -60,7 +58,10 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: self.log.info("dag_run_state: %s", dag_run.state) if dag_run.state == DagRunState.FAILED: - self.log.info("DagRun %s for dag %s has failed, stopping expansion", dag_run.run_id, dag_run.dag_id) + self.log.info( + "DagRun %s for dag %s has failed, stopping expansion", + dag_run.run_id, dag_run.dag_id + ) raise AirflowException( f"Stopping expansion of tasks for DagRun {dag_run.run_id} of DAG {dag_run.dag_id} due to failure." @@ -96,6 +97,7 @@ def expand_unmapped_task_instance( task_instances_batch = [] + task_expansion_batch_size = conf.getint("core", "parallelism") context = unmapped_ti.get_template_context(session=session) expand_input = unmapped_ti.task.expand_input.resolve(context) @@ -124,7 +126,9 @@ def expand_unmapped_task_instance( task_instances_batch.append(task_instance) if len(task_instances_batch) == task_expansion_batch_size: - dag_run = DagRun.get_dag_run(dag_id=unmapped_ti.dag_id, run_id=dag_run.run_id, session=session) + dag_run = DagRun.get_dag_run( + dag_id = unmapped_ti.dag_id, run_id = dag_run.run_id, session = session + ) self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) @@ -137,7 +141,11 @@ def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @staticmethod def has_mapped_operator(task_instance: TaskInstance) -> bool: - return isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 and task_instance.state in State.unfinished + return ( + isinstance(task_instance.task, MappedOperator) + and task_instance.map_index == -1 + and task_instance.state in State.unfinished + ) def expand_tasks(self): with create_session(scoped=False) as session: @@ -147,21 +155,34 @@ def expand_tasks(self): for dag_run in dag_runs: dag = dag_bag.get_dag(dag_run.dag_id) self.log.info("Checking for unmapped task instances on: %s", dag_run) - for unmapped_ti in filter(self.has_mapped_operator, map(lambda task: self.get_task(dag, task), dag_run.task_instances)): + for unmapped_ti in filter( + self.has_mapped_operator, + map(lambda task: self.get_task(dag, task), dag_run.task_instances), + ): try: - finished_tis = list(map(lambda task: self.get_task(dag, task), filter(lambda ti: ti.state in State.finished, dag_run.task_instances))) + finished_tis = list( + map( + lambda task: self.get_task(dag, task), + filter(lambda ti: ti.state in State.finished, dag_run.task_instances), + ) + ) dep_context = DepContext( flag_upstream_failed=True, ignore_unmapped_tasks=True, # Ignore this Dep, as we will expand it if we can. finished_tis=finished_tis, ) self.log.info("Unmapped task state on: %s", unmapped_ti.state) - are_dependencies_met = unmapped_ti.are_dependencies_met(dep_context=dep_context, session=session, verbose=True) + are_dependencies_met = unmapped_ti.are_dependencies_met( + dep_context = dep_context, session = session, verbose = True + ) self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) if are_dependencies_met: self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) except Exception: - self.log.exception("Unexpected error occurred during task expansion of %s", unmapped_ti) + self.log.exception( + "Unexpected error occurred during task expansion of %s", + unmapped_ti + ) def _execute(self) -> int | None: self.log.info("TaskExpansionJobRunner started") @@ -174,5 +195,7 @@ def _execute(self) -> int | None: def task_expansion_run(triggerer_heartrate: float, trigger_runner: TriggerRunnerSupervisor): - task_expansion_job_runner = TaskExpansionJobRunner(job=Job(heartrate=triggerer_heartrate), trigger_runner=trigger_runner) + task_expansion_job_runner = TaskExpansionJobRunner( + job=Job(heartrate=triggerer_heartrate), trigger_runner=trigger_runner + ) run_job_async(job=task_expansion_job_runner.job, execute_callable=task_expansion_job_runner._execute) From 9fa7968b6ce57ca394e3ff94087b4e2c3e22c543 Mon Sep 17 00:00:00 2001 From: David Blain Date: Mon, 4 Aug 2025 21:14:53 +0200 Subject: [PATCH 053/338] refactor: Moved import of MappedOperator under has_mapped_operator method --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 625ff0f3a7973..b837f856b0006 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -26,7 +26,6 @@ from airflow.jobs.job import Job, run_job_async from airflow.models import DagBag, DagRun from airflow.policies import task_instance_mutation_hook -from airflow.sdk.definitions.mappedoperator import MappedOperator from airflow.ti_deps.dep_context import DepContext from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.session import create_session @@ -141,6 +140,8 @@ def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @staticmethod def has_mapped_operator(task_instance: TaskInstance) -> bool: + from airflow.sdk.definitions.mappedoperator import MappedOperator + return ( isinstance(task_instance.task, MappedOperator) and task_instance.map_index == -1 From 5cb26a2981b82ced3c37258d37aafb11fad8d130 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 5 Aug 2025 08:43:13 +0200 Subject: [PATCH 054/338] refactor: Changed import of MappedOperator in TaskExpansionJobRunner --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index b837f856b0006..2140f92d6bf92 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -26,6 +26,7 @@ from airflow.jobs.job import Job, run_job_async from airflow.models import DagBag, DagRun from airflow.policies import task_instance_mutation_hook +from airflow.sdk.definitions.mappedoperator import MappedOperator as TaskSDKMappedOperator from airflow.ti_deps.dep_context import DepContext from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.session import create_session @@ -140,10 +141,8 @@ def get_task(dag: DAG, task_instance: TaskInstance) -> TaskInstance: @staticmethod def has_mapped_operator(task_instance: TaskInstance) -> bool: - from airflow.sdk.definitions.mappedoperator import MappedOperator - return ( - isinstance(task_instance.task, MappedOperator) + isinstance(task_instance.task, TaskSDKMappedOperator) and task_instance.map_index == -1 and task_instance.state in State.unfinished ) From cc45de8e427b0b591c01300a4d17544c694726c3 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 5 Aug 2025 08:45:54 +0200 Subject: [PATCH 055/338] refactor: Reformatted TaskExpansionJobRunner --- .../src/airflow/jobs/expand_task_job_runner.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 2140f92d6bf92..d811156ded72f 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -59,8 +59,7 @@ def _check_dag_run_state(self, dag_run: DagRun) -> None: if dag_run.state == DagRunState.FAILED: self.log.info( - "DagRun %s for dag %s has failed, stopping expansion", - dag_run.run_id, dag_run.dag_id + "DagRun %s for dag %s has failed, stopping expansion", dag_run.run_id, dag_run.dag_id ) raise AirflowException( @@ -127,7 +126,7 @@ def expand_unmapped_task_instance( if len(task_instances_batch) == task_expansion_batch_size: dag_run = DagRun.get_dag_run( - dag_id = unmapped_ti.dag_id, run_id = dag_run.run_id, session = session + dag_id=unmapped_ti.dag_id, run_id=dag_run.run_id, session=session ) self._check_dag_run_state(dag_run) self._persist_task_instances(dag_run, task_instances_batch, session=session) @@ -173,15 +172,14 @@ def expand_tasks(self): ) self.log.info("Unmapped task state on: %s", unmapped_ti.state) are_dependencies_met = unmapped_ti.are_dependencies_met( - dep_context = dep_context, session = session, verbose = True + dep_context=dep_context, session=session, verbose=True ) self.log.info("Are dependencies met on %s: %s", unmapped_ti, are_dependencies_met) if are_dependencies_met: self.expand_unmapped_task_instance(dag_run, unmapped_ti, session=session) except Exception: self.log.exception( - "Unexpected error occurred during task expansion of %s", - unmapped_ti + "Unexpected error occurred during task expansion of %s", unmapped_ti ) def _execute(self) -> int | None: From 4b9601ed58b983186ff910667765e78a74184404 Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 5 Aug 2025 08:47:39 +0200 Subject: [PATCH 056/338] refactor: Reformatted DeferredIterable --- airflow-core/src/airflow/models/deferred_iterable.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow-core/src/airflow/models/deferred_iterable.py b/airflow-core/src/airflow/models/deferred_iterable.py index d314a9d43b40e..f68779156e410 100644 --- a/airflow-core/src/airflow/models/deferred_iterable.py +++ b/airflow-core/src/airflow/models/deferred_iterable.py @@ -173,7 +173,6 @@ def serialize(self): @classmethod def get_operator_from_dag(cls, dag_fileloc: str, dag_id: str, task_id: str) -> Operator: """Loads a DAG using DagBag and gets the operator by task_id.""" - from airflow.models import DagBag dag_bag = DagBag(collect_dags=False) # Avoid loading all DAGs From ce08ec7e1eb25a3b1eb4c3500c49405c41dfb8ef Mon Sep 17 00:00:00 2001 From: David Blain Date: Tue, 5 Aug 2025 08:48:11 +0200 Subject: [PATCH 057/338] refactor: Reformatted imports ExpandInput --- airflow-core/src/airflow/models/expandinput.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow-core/src/airflow/models/expandinput.py b/airflow-core/src/airflow/models/expandinput.py index 803b0ee2566a8..9e590a449d764 100644 --- a/airflow-core/src/airflow/models/expandinput.py +++ b/airflow-core/src/airflow/models/expandinput.py @@ -21,6 +21,7 @@ import operator from collections.abc import Iterable, Sized from typing import TYPE_CHECKING, Any, ClassVar + import attrs from airflow.utils.log.logging_mixin import LoggingMixin From dcd2db31f53b0d079ee404ff30a6647937e0bdcd Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 6 Aug 2025 10:25:16 +0200 Subject: [PATCH 058/338] refactor: Removed white line in _persist_task_instances method --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index d811156ded72f..5e58ddf7c4569 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -72,7 +72,6 @@ def _persist_task_instances( """ Expands the task using the provided expand_input. """ - if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) dag_run.task_instances.extend(task_instances) From 3cd19230a34c1eae544bebde5752750018231b90 Mon Sep 17 00:00:00 2001 From: David Blain Date: Wed, 6 Aug 2025 12:09:57 +0200 Subject: [PATCH 059/338] refactor: Update task_map length when persisting task instances while expanding to prevent scheduler to stop earlier --- .../src/airflow/jobs/expand_task_job_runner.py | 1 + airflow-core/src/airflow/models/taskmap.py | 16 +++++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 5e58ddf7c4569..92d52ac847dec 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -74,6 +74,7 @@ def _persist_task_instances( """ if dag_run and task_instances: self.log.info("Persisting %d new task instances", len(task_instances)) + update_task_map_length(task=task_instances[-1], session=session) dag_run.task_instances.extend(task_instances) session.merge(dag_run) session.commit() diff --git a/airflow-core/src/airflow/models/taskmap.py b/airflow-core/src/airflow/models/taskmap.py index f6bedd3636fd5..e3ff9d0730340 100644 --- a/airflow-core/src/airflow/models/taskmap.py +++ b/airflow-core/src/airflow/models/taskmap.py @@ -24,7 +24,7 @@ from collections.abc import Collection, Iterable, Sequence from typing import TYPE_CHECKING, Any -from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select +from sqlalchemy import CheckConstraint, Column, ForeignKeyConstraint, Integer, String, func, or_, select, update from airflow.models.base import COLLATION_ARGS, ID_LEN, TaskInstanceDependencies from airflow.models.dag_version import DagVersion @@ -275,3 +275,17 @@ def expand_mapped_task( return all_expanded_tis, total_expanded_ti_count - 1 raise NotMapped + + +def update_task_map_length(task: TaskInstance, session): + length = task.map_index + 1 + session.execute( + update(TaskMap) + .where( + TaskMap.dag_id == task.dag_id, + TaskMap.task_id == task.task_id, + TaskMap.run_id == task.run_id, + TaskMap.map_index == -1, + ) + .values(length=length) + ) From 65b28bd92dd0fe806b0f60e906926b58f2b4e63e Mon Sep 17 00:00:00 2001 From: David Blain Date: Sat, 13 Sep 2025 19:39:31 +0200 Subject: [PATCH 060/338] refactor: Added missing import update_task_map_length --- airflow-core/src/airflow/jobs/expand_task_job_runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow-core/src/airflow/jobs/expand_task_job_runner.py b/airflow-core/src/airflow/jobs/expand_task_job_runner.py index 92d52ac847dec..ded865fbe5780 100644 --- a/airflow-core/src/airflow/jobs/expand_task_job_runner.py +++ b/airflow-core/src/airflow/jobs/expand_task_job_runner.py @@ -25,6 +25,7 @@ from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import Job, run_job_async from airflow.models import DagBag, DagRun +from airflow.models.taskmap import update_task_map_length from airflow.policies import task_instance_mutation_hook from airflow.sdk.definitions.mappedoperator import MappedOperator as TaskSDKMappedOperator from airflow.ti_deps.dep_context import DepContext From e77a2a53fe1e97d71181f6261f35ab2b21141f2e Mon Sep 17 00:00:00 2001 From: "Zhen-Lun (Kevin) Hong" Date: Thu, 25 Sep 2025 01:34:22 +0530 Subject: [PATCH 061/338] Fix test_schedulable_task_exist_when_rerun_removed_upstream_mapped_task (#56065) --- airflow-core/tests/unit/models/test_dagrun.py | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/airflow-core/tests/unit/models/test_dagrun.py b/airflow-core/tests/unit/models/test_dagrun.py index 3ec2eccef0167..3813f89b3a37b 100644 --- a/airflow-core/tests/unit/models/test_dagrun.py +++ b/airflow-core/tests/unit/models/test_dagrun.py @@ -2169,14 +2169,20 @@ def do_something_else(i): dr = dag_maker.create_dagrun() - ti = dr.get_task_instance("do_something_else", session=session) - ti.map_index = 0 - task = ti.task - for map_index in range(1, 5): - ti = TI(task, run_id=dr.run_id, map_index=map_index, dag_version_id=ti.dag_version_id) - session.add(ti) - ti.dag_run = dr + tis = dr.get_task_instances() + for ti in tis: + if ti.task_id == "do_something_else": + ti.map_index = 0 + task = ti.task + for map_index in range(1, 5): + ti = TI(task, run_id=dr.run_id, map_index=map_index, dag_version_id=ti.dag_version_id) + session.add(ti) + ti.dag_run = dr + else: + # run tasks "do_something" to get XCOMs for correct downstream length + ti.run() session.flush() + tis = dr.get_task_instances() for ti in tis: if ti.task_id == "do_something": @@ -2188,7 +2194,7 @@ def do_something_else(i): session.commit() # The Upstream is done with 2 removed tis and 3 success tis (tis, _) = dr.update_state() - assert len(tis) + assert len(tis) == 3 assert dr.state != DagRunState.FAILED From 92dce7563449493bec8bb249d3f7df923581555e Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 24 Sep 2025 21:15:05 +0100 Subject: [PATCH 062/338] Remove defunct ossrank.com badge (#56033) The site appears dead now --- README.md | 2 +- generated/PYPI_README.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c2779295a9ba0..c27640a6d4bd6 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ | License | [![License](https://img.shields.io/:license-Apache%202-blue.svg)](https://www.apache.org/licenses/LICENSE-2.0.txt) | | PyPI | [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) [![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) [![PyPI - Downloads](https://img.shields.io/pypi/dm/apache-airflow)](https://pypi.org/project/apache-airflow/) | | Containers | [![Docker Pulls](https://img.shields.io/docker/pulls/apache/airflow.svg)](https://hub.docker.com/r/apache/airflow) [![Docker Stars](https://img.shields.io/docker/stars/apache/airflow.svg)](https://hub.docker.com/r/apache/airflow) [![Artifact HUB](https://img.shields.io/endpoint?url=https://artifacthub.io/badge/repository/apache-airflow)](https://artifacthub.io/packages/search?repo=apache-airflow) | -| Community | [![Contributors](https://img.shields.io/github/contributors/apache/airflow)](https://github.com/apache/airflow/graphs/contributors) [![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack) ![Commit Activity](https://img.shields.io/github/commit-activity/m/apache/airflow) [![OSSRank](https://shields.io/endpoint?url=https://ossrank.com/shield/6)](https://ossrank.com/p/6) [![LFX Health Score](https://insights.linuxfoundation.org/api/badge/health-score?project=apache-airflow)](https://insights.linuxfoundation.org/project/apache-airflow) | +| Community | [![Contributors](https://img.shields.io/github/contributors/apache/airflow)](https://github.com/apache/airflow/graphs/contributors) [![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack) ![Commit Activity](https://img.shields.io/github/commit-activity/m/apache/airflow) [![LFX Health Score](https://insights.linuxfoundation.org/api/badge/health-score?project=apache-airflow)](https://insights.linuxfoundation.org/project/apache-airflow) | diff --git a/generated/PYPI_README.md b/generated/PYPI_README.md index 6c2283dab21fa..29568c1fd38e8 100644 --- a/generated/PYPI_README.md +++ b/generated/PYPI_README.md @@ -27,7 +27,7 @@ PROJECT BY THE `generate-pypi-readme` PREK HOOK. YOUR CHANGES HERE WILL BE AUTOM | License | [![License](https://img.shields.io/:license-Apache%202-blue.svg)](https://www.apache.org/licenses/LICENSE-2.0.txt) | | PyPI | [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) [![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) [![PyPI - Downloads](https://img.shields.io/pypi/dm/apache-airflow)](https://pypi.org/project/apache-airflow/) | | Containers | [![Docker Pulls](https://img.shields.io/docker/pulls/apache/airflow.svg)](https://hub.docker.com/r/apache/airflow) [![Docker Stars](https://img.shields.io/docker/stars/apache/airflow.svg)](https://hub.docker.com/r/apache/airflow) [![Artifact HUB](https://img.shields.io/endpoint?url=https://artifacthub.io/badge/repository/apache-airflow)](https://artifacthub.io/packages/search?repo=apache-airflow) | -| Community | [![Contributors](https://img.shields.io/github/contributors/apache/airflow)](https://github.com/apache/airflow/graphs/contributors) [![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack) ![Commit Activity](https://img.shields.io/github/commit-activity/m/apache/airflow) [![OSSRank](https://shields.io/endpoint?url=https://ossrank.com/shield/6)](https://ossrank.com/p/6) [![LFX Health Score](https://insights.linuxfoundation.org/api/badge/health-score?project=apache-airflow)](https://insights.linuxfoundation.org/project/apache-airflow) | +| Community | [![Contributors](https://img.shields.io/github/contributors/apache/airflow)](https://github.com/apache/airflow/graphs/contributors) [![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack) ![Commit Activity](https://img.shields.io/github/commit-activity/m/apache/airflow) [![LFX Health Score](https://insights.linuxfoundation.org/api/badge/health-score?project=apache-airflow)](https://insights.linuxfoundation.org/project/apache-airflow) | From af6943fdbb60cb945a954c0165c3e5061fa0a048 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 24 Sep 2025 22:21:23 +0100 Subject: [PATCH 063/338] Correct version added for new Namespaced log levels feature (#55990) The PR that added this was opened before 3.1.0rc, but it didn't get merged until after the branch point, so given it's a new feature, not a bug fix, this won't be released until 3.2.0 --- airflow-core/src/airflow/config_templates/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/config_templates/config.yml b/airflow-core/src/airflow/config_templates/config.yml index c4f10ac9624e9..d9ab15799f6da 100644 --- a/airflow-core/src/airflow/config_templates/config.yml +++ b/airflow-core/src/airflow/config_templates/config.yml @@ -812,7 +812,7 @@ logging: The logger names are viewable in task logs (as the "source" attribute), or in server components by including ``%(name)s`` in your format string, or in the between ``[]`` after the message in the default format. - version_added: 3.1.0 + version_added: 3.2.0 type: string example: "sqlalchemy=INFO sqlalchemy.engine=DEBUG, botocor" default: ~ From 9468cb03aa3f3aac9bc2f587150926f71ce65eff Mon Sep 17 00:00:00 2001 From: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> Date: Wed, 24 Sep 2025 23:59:19 +0200 Subject: [PATCH 064/338] Bump Edge UI packages after dependabot triaging (#56016) * Bump Edge UI packages after dependabot triaging * Fix problems by generated code from outdated fastapi --- .../edge3/plugins/www/dist/main.umd.cjs | 56 +- .../providers/edge3/plugins/www/package.json | 40 +- .../edge3/plugins/www/pnpm-lock.yaml | 2697 ++++++++--------- .../plugins/www/src/components/ui/Alert.tsx | 1 - .../providers/edge3/plugins/www/src/theme.ts | 1 - providers/edge3/www-hash.txt | 2 +- 6 files changed, 1370 insertions(+), 1427 deletions(-) diff --git a/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs b/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs index fab4e130f3c82..8677681e23078 100644 --- a/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs +++ b/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs @@ -1,4 +1,4 @@ -(function(P,Tt){typeof exports=="object"&&typeof module<"u"?module.exports=Tt(require("react"),require("react-dom")):typeof define=="function"&&define.amd?define(["react","react-dom"],Tt):(P=typeof globalThis<"u"?globalThis:P||self,P.AirflowPlugin=Tt(P.React,P.ReactDOM))})(this,(function(P,Tt){"use strict";function gp(e){const t=Object.create(null,{[Symbol.toStringTag]:{value:"Module"}});if(e){for(const n in e)if(n!=="default"){const r=Object.getOwnPropertyDescriptor(e,n);Object.defineProperty(t,n,r.get?r:{enumerable:!0,get:()=>e[n]})}}return t.default=e,Object.freeze(t)}const O=gp(P);function pp(e){return e&&e.__esModule&&Object.prototype.hasOwnProperty.call(e,"default")?e.default:e}var Wi={exports:{}},ur={};/** +(function(P,Ft){typeof exports=="object"&&typeof module<"u"?module.exports=Ft(require("react"),require("react-dom")):typeof define=="function"&&define.amd?define(["react","react-dom"],Ft):(P=typeof globalThis<"u"?globalThis:P||self,P.AirflowPlugin=Ft(P.React,P.ReactDOM))})(this,(function(P,Ft){"use strict";function Sp(e){const t=Object.create(null,{[Symbol.toStringTag]:{value:"Module"}});if(e){for(const n in e)if(n!=="default"){const r=Object.getOwnPropertyDescriptor(e,n);Object.defineProperty(t,n,r.get?r:{enumerable:!0,get:()=>e[n]})}}return t.default=e,Object.freeze(t)}const C=Sp(P);function Ep(e){return e&&e.__esModule&&Object.prototype.hasOwnProperty.call(e,"default")?e.default:e}var qi={exports:{}},hr={};/** * @license React * react-jsx-runtime.production.js * @@ -6,28 +6,28 @@ * * This source code is licensed under the MIT license found in the * LICENSE file in the root directory of this source tree. - */var al;function mp(){if(al)return ur;al=1;var e=Symbol.for("react.transitional.element"),t=Symbol.for("react.fragment");function n(r,o,i){var s=null;if(i!==void 0&&(s=""+i),o.key!==void 0&&(s=""+o.key),"key"in o){i={};for(var a in o)a!=="key"&&(i[a]=o[a])}else i=o;return o=i.ref,{$$typeof:e,type:r,key:s,ref:o!==void 0?o:null,props:i}}return ur.Fragment=t,ur.jsx=n,ur.jsxs=n,ur}var ll;function vp(){return ll||(ll=1,Wi.exports=mp()),Wi.exports}var f=vp();function cl(e){var t=Object.create(null);return function(n){return t[n]===void 0&&(t[n]=e(n)),t[n]}}var bp=/^((children|dangerouslySetInnerHTML|key|ref|autoFocus|defaultValue|defaultChecked|innerHTML|suppressContentEditableWarning|suppressHydrationWarning|valueLink|abbr|accept|acceptCharset|accessKey|action|allow|allowUserMedia|allowPaymentRequest|allowFullScreen|allowTransparency|alt|async|autoComplete|autoPlay|capture|cellPadding|cellSpacing|challenge|charSet|checked|cite|classID|className|cols|colSpan|content|contentEditable|contextMenu|controls|controlsList|coords|crossOrigin|data|dateTime|decoding|default|defer|dir|disabled|disablePictureInPicture|disableRemotePlayback|download|draggable|encType|enterKeyHint|fetchpriority|fetchPriority|form|formAction|formEncType|formMethod|formNoValidate|formTarget|frameBorder|headers|height|hidden|high|href|hrefLang|htmlFor|httpEquiv|id|inputMode|integrity|is|keyParams|keyType|kind|label|lang|list|loading|loop|low|marginHeight|marginWidth|max|maxLength|media|mediaGroup|method|min|minLength|multiple|muted|name|nonce|noValidate|open|optimum|pattern|placeholder|playsInline|popover|popoverTarget|popoverTargetAction|poster|preload|profile|radioGroup|readOnly|referrerPolicy|rel|required|reversed|role|rows|rowSpan|sandbox|scope|scoped|scrolling|seamless|selected|shape|size|sizes|slot|span|spellCheck|src|srcDoc|srcLang|srcSet|start|step|style|summary|tabIndex|target|title|translate|type|useMap|value|width|wmode|wrap|about|datatype|inlist|prefix|property|resource|typeof|vocab|autoCapitalize|autoCorrect|autoSave|color|incremental|fallback|inert|itemProp|itemScope|itemType|itemID|itemRef|on|option|results|security|unselectable|accentHeight|accumulate|additive|alignmentBaseline|allowReorder|alphabetic|amplitude|arabicForm|ascent|attributeName|attributeType|autoReverse|azimuth|baseFrequency|baselineShift|baseProfile|bbox|begin|bias|by|calcMode|capHeight|clip|clipPathUnits|clipPath|clipRule|colorInterpolation|colorInterpolationFilters|colorProfile|colorRendering|contentScriptType|contentStyleType|cursor|cx|cy|d|decelerate|descent|diffuseConstant|direction|display|divisor|dominantBaseline|dur|dx|dy|edgeMode|elevation|enableBackground|end|exponent|externalResourcesRequired|fill|fillOpacity|fillRule|filter|filterRes|filterUnits|floodColor|floodOpacity|focusable|fontFamily|fontSize|fontSizeAdjust|fontStretch|fontStyle|fontVariant|fontWeight|format|from|fr|fx|fy|g1|g2|glyphName|glyphOrientationHorizontal|glyphOrientationVertical|glyphRef|gradientTransform|gradientUnits|hanging|horizAdvX|horizOriginX|ideographic|imageRendering|in|in2|intercept|k|k1|k2|k3|k4|kernelMatrix|kernelUnitLength|kerning|keyPoints|keySplines|keyTimes|lengthAdjust|letterSpacing|lightingColor|limitingConeAngle|local|markerEnd|markerMid|markerStart|markerHeight|markerUnits|markerWidth|mask|maskContentUnits|maskUnits|mathematical|mode|numOctaves|offset|opacity|operator|order|orient|orientation|origin|overflow|overlinePosition|overlineThickness|panose1|paintOrder|pathLength|patternContentUnits|patternTransform|patternUnits|pointerEvents|points|pointsAtX|pointsAtY|pointsAtZ|preserveAlpha|preserveAspectRatio|primitiveUnits|r|radius|refX|refY|renderingIntent|repeatCount|repeatDur|requiredExtensions|requiredFeatures|restart|result|rotate|rx|ry|scale|seed|shapeRendering|slope|spacing|specularConstant|specularExponent|speed|spreadMethod|startOffset|stdDeviation|stemh|stemv|stitchTiles|stopColor|stopOpacity|strikethroughPosition|strikethroughThickness|string|stroke|strokeDasharray|strokeDashoffset|strokeLinecap|strokeLinejoin|strokeMiterlimit|strokeOpacity|strokeWidth|surfaceScale|systemLanguage|tableValues|targetX|targetY|textAnchor|textDecoration|textRendering|textLength|to|transform|u1|u2|underlinePosition|underlineThickness|unicode|unicodeBidi|unicodeRange|unitsPerEm|vAlphabetic|vHanging|vIdeographic|vMathematical|values|vectorEffect|version|vertAdvY|vertOriginX|vertOriginY|viewBox|viewTarget|visibility|widths|wordSpacing|writingMode|x|xHeight|x1|x2|xChannelSelector|xlinkActuate|xlinkArcrole|xlinkHref|xlinkRole|xlinkShow|xlinkTitle|xlinkType|xmlBase|xmlns|xmlnsXlink|xmlLang|xmlSpace|y|y1|y2|yChannelSelector|z|zoomAndPan|for|class|autofocus)|(([Dd][Aa][Tt][Aa]|[Aa][Rr][Ii][Aa]|x)-.*))$/,yp=cl(function(e){return bp.test(e)||e.charCodeAt(0)===111&&e.charCodeAt(1)===110&&e.charCodeAt(2)<91});function xp(e){if(e.sheet)return e.sheet;for(var t=0;t0?Se(Nn,--Ae):0,Tn--,ue===10&&(Tn=1,ro--),ue}function je(){return ue=Ae2||gr(ue)>3?"":" "}function _p(e,t){for(;--t&&je()&&!(ue<48||ue>102||ue>57&&ue<65||ue>70&&ue<97););return fr(e,io()+(t<6&&st()==32&&je()==32))}function Ki(e){for(;je();)switch(ue){case e:return Ae;case 34:case 39:e!==34&&e!==39&&Ki(ue);break;case 40:e===41&&Ki(e);break;case 92:je();break}return Ae}function Vp(e,t){for(;je()&&e+ue!==57;)if(e+ue===84&&st()===47)break;return"/*"+fr(t,Ae-1)+"*"+to(e===47?e:je())}function Fp(e){for(;!gr(st());)je();return fr(e,Ae)}function Lp(e){return pl(ao("",null,null,null,[""],e=gl(e),0,[0],e))}function ao(e,t,n,r,o,i,s,a,l){for(var c=0,u=0,d=s,h=0,m=0,g=0,p=1,v=1,y=1,x=0,b="",C=o,k=i,E=r,I=b;v;)switch(g=x,x=je()){case 40:if(g!=108&&Se(I,d-1)==58){Gi(I+=X(so(x),"&","&\f"),"&\f")!=-1&&(y=-1);break}case 34:case 39:case 91:I+=so(x);break;case 9:case 10:case 13:case 32:I+=Ap(g);break;case 92:I+=_p(io()-1,7);continue;case 47:switch(st()){case 42:case 47:no(Dp(Vp(je(),io()),t,n),l);break;default:I+="/"}break;case 123*p:a[c++]=it(I)*y;case 125*p:case 59:case 0:switch(x){case 0:case 125:v=0;case 59+u:y==-1&&(I=X(I,/\f/g,"")),m>0&&it(I)-d&&no(m>32?vl(I+";",r,n,d-1):vl(X(I," ","")+";",r,n,d-2),l);break;case 59:I+=";";default:if(no(E=ml(I,t,n,c,u,o,a,b,C=[],k=[],d),i),x===123)if(u===0)ao(I,t,E,E,C,i,d,a,k);else switch(h===99&&Se(I,3)===110?100:h){case 100:case 108:case 109:case 115:ao(e,E,E,r&&no(ml(e,E,E,0,0,o,a,b,o,C=[],d),k),o,k,d,a,r?C:k);break;default:ao(I,E,E,E,[""],k,0,a,k)}}c=u=m=0,p=y=1,b=I="",d=s;break;case 58:d=1+it(I),m=g;default:if(p<1){if(x==123)--p;else if(x==125&&p++==0&&Np()==125)continue}switch(I+=to(x),x*p){case 38:y=u>0?1:(I+="\f",-1);break;case 44:a[c++]=(it(I)-1)*y,y=1;break;case 64:st()===45&&(I+=so(je())),h=st(),u=d=it(b=I+=Fp(io())),x++;break;case 45:g===45&&it(I)==2&&(p=0)}}return i}function ml(e,t,n,r,o,i,s,a,l,c,u){for(var d=o-1,h=o===0?i:[""],m=qi(h),g=0,p=0,v=0;g0?h[y]+" "+x:X(x,/&\f/g,h[y])))&&(l[v++]=b);return oo(e,t,n,o===0?Hi:a,l,c,u)}function Dp(e,t,n){return oo(e,t,n,ul,to(Tp()),dr(e,2,-2),0)}function vl(e,t,n,r){return oo(e,t,n,Ui,dr(e,0,r),dr(e,r+1,-1),r)}function An(e,t){for(var n="",r=qi(e),o=0;o6)switch(Se(e,t+1)){case 109:if(Se(e,t+4)!==45)break;case 102:return X(e,/(.+:)(.+)-([^]+)/,"$1"+Y+"$2-$3$1"+eo+(Se(e,t+3)==108?"$3":"$2-$3"))+e;case 115:return~Gi(e,"stretch")?yl(X(e,"stretch","fill-available"),t)+e:e}break;case 4949:if(Se(e,t+1)!==115)break;case 6444:switch(Se(e,it(e)-3-(~Gi(e,"!important")&&10))){case 107:return X(e,":",":"+Y)+e;case 101:return X(e,/(.+:)([^;!]+)(;|!.+)?/,"$1"+Y+(Se(e,14)===45?"inline-":"")+"box$3$1"+Y+"$2$3$1"+Pe+"$2box$3")+e}break;case 5936:switch(Se(e,t+11)){case 114:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb")+e;case 108:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb-rl")+e;case 45:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"lr")+e}return Y+e+Pe+e+e}return e}var Gp=function(t,n,r,o){if(t.length>-1&&!t.return)switch(t.type){case Ui:t.return=yl(t.value,t.length);break;case dl:return An([hr(t,{value:X(t.value,"@","@"+Y)})],o);case Hi:if(t.length)return Rp(t.props,function(i){switch(Ip(i,/(::plac\w+|:read-\w+)/)){case":read-only":case":read-write":return An([hr(t,{props:[X(i,/:(read-\w+)/,":"+eo+"$1")]})],o);case"::placeholder":return An([hr(t,{props:[X(i,/:(plac\w+)/,":"+Y+"input-$1")]}),hr(t,{props:[X(i,/:(plac\w+)/,":"+eo+"$1")]}),hr(t,{props:[X(i,/:(plac\w+)/,Pe+"input-$1")]})],o)}return""})}},qp=[Gp],Kp=function(t){var n=t.key;if(n==="css"){var r=document.querySelectorAll("style[data-emotion]:not([data-s])");Array.prototype.forEach.call(r,function(p){var v=p.getAttribute("data-emotion");v.indexOf(" ")!==-1&&(document.head.appendChild(p),p.setAttribute("data-s",""))})}var o=t.stylisPlugins||qp,i={},s,a=[];s=t.container||document.head,Array.prototype.forEach.call(document.querySelectorAll('style[data-emotion^="'+n+' "]'),function(p){for(var v=p.getAttribute("data-emotion").split(" "),y=1;y0?we(_n,--Ae):0,An--,ue===10&&(An=1,so--),ue}function je(){return ue=Ae2||mr(ue)>3?"":" "}function Wp(e,t){for(;--t&&je()&&!(ue<48||ue>102||ue>57&&ue<65||ue>70&&ue<97););return pr(e,lo()+(t<6&&st()==32&&je()==32))}function Ji(e){for(;je();)switch(ue){case e:return Ae;case 34:case 39:e!==34&&e!==39&&Ji(ue);break;case 40:e===41&&Ji(e);break;case 92:je();break}return Ae}function Hp(e,t){for(;je()&&e+ue!==57;)if(e+ue===84&&st()===47)break;return"/*"+pr(t,Ae-1)+"*"+oo(e===47?e:je())}function Up(e){for(;!mr(st());)je();return pr(e,Ae)}function Gp(e){return Sl(uo("",null,null,null,[""],e=wl(e),0,[0],e))}function uo(e,t,n,r,o,i,s,a,l){for(var c=0,u=0,h=s,d=0,m=0,f=0,p=1,v=1,b=1,y=0,x="",w=o,k=i,O=r,R=x;v;)switch(f=y,y=je()){case 40:if(f!=108&&we(R,h-1)==58){Xi(R+=X(co(y),"&","&\f"),"&\f")!=-1&&(b=-1);break}case 34:case 39:case 91:R+=co(y);break;case 9:case 10:case 13:case 32:R+=Bp(f);break;case 92:R+=Wp(lo()-1,7);continue;case 47:switch(st()){case 42:case 47:io(qp(Hp(je(),lo()),t,n),l);break;default:R+="/"}break;case 123*p:a[c++]=it(R)*b;case 125*p:case 59:case 0:switch(y){case 0:case 125:v=0;case 59+u:b==-1&&(R=X(R,/\f/g,"")),m>0&&it(R)-h&&io(m>32?Ol(R+";",r,n,h-1):Ol(X(R," ","")+";",r,n,h-2),l);break;case 59:R+=";";default:if(io(O=El(R,t,n,c,u,o,a,x,w=[],k=[],h),i),y===123)if(u===0)uo(R,t,O,O,w,i,h,a,k);else switch(d===99&&we(R,3)===110?100:d){case 100:case 108:case 109:case 115:uo(e,O,O,r&&io(El(e,O,O,0,0,o,a,x,o,w=[],h),k),o,k,h,a,r?w:k);break;default:uo(R,O,O,O,[""],k,0,a,k)}}c=u=m=0,p=b=1,x=R="",h=s;break;case 58:h=1+it(R),m=f;default:if(p<1){if(y==123)--p;else if(y==125&&p++==0&&$p()==125)continue}switch(R+=oo(y),y*p){case 38:b=u>0?1:(R+="\f",-1);break;case 44:a[c++]=(it(R)-1)*b,b=1;break;case 64:st()===45&&(R+=co(je())),d=st(),u=h=it(x=R+=Up(lo())),y++;break;case 45:f===45&&it(R)==2&&(p=0)}}return i}function El(e,t,n,r,o,i,s,a,l,c,u){for(var h=o-1,d=o===0?i:[""],m=Qi(d),f=0,p=0,v=0;f0?d[b]+" "+y:X(y,/&\f/g,d[b])))&&(l[v++]=x);return ao(e,t,n,o===0?Ki:a,l,c,u)}function qp(e,t,n){return ao(e,t,n,yl,oo(jp()),fr(e,2,-2),0)}function Ol(e,t,n,r){return ao(e,t,n,Yi,fr(e,0,r),fr(e,r+1,-1),r)}function Vn(e,t){for(var n="",r=Qi(e),o=0;o6)switch(we(e,t+1)){case 109:if(we(e,t+4)!==45)break;case 102:return X(e,/(.+:)(.+)-([^]+)/,"$1"+Y+"$2-$3$1"+ro+(we(e,t+3)==108?"$3":"$2-$3"))+e;case 115:return~Xi(e,"stretch")?Rl(X(e,"stretch","fill-available"),t)+e:e}break;case 4949:if(we(e,t+1)!==115)break;case 6444:switch(we(e,it(e)-3-(~Xi(e,"!important")&&10))){case 107:return X(e,":",":"+Y)+e;case 101:return X(e,/(.+:)([^;!]+)(;|!.+)?/,"$1"+Y+(we(e,14)===45?"inline-":"")+"box$3$1"+Y+"$2$3$1"+Pe+"$2box$3")+e}break;case 5936:switch(we(e,t+11)){case 114:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb")+e;case 108:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb-rl")+e;case 45:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"lr")+e}return Y+e+Pe+e+e}return e}var nm=function(t,n,r,o){if(t.length>-1&&!t.return)switch(t.type){case Yi:t.return=Rl(t.value,t.length);break;case xl:return Vn([gr(t,{value:X(t.value,"@","@"+Y)})],o);case Ki:if(t.length)return Mp(t.props,function(i){switch(zp(i,/(::plac\w+|:read-\w+)/)){case":read-only":case":read-write":return Vn([gr(t,{props:[X(i,/:(read-\w+)/,":"+ro+"$1")]})],o);case"::placeholder":return Vn([gr(t,{props:[X(i,/:(plac\w+)/,":"+Y+"input-$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,":"+ro+"$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,Pe+"input-$1")]})],o)}return""})}},rm=[nm],om=function(t){var n=t.key;if(n==="css"){var r=document.querySelectorAll("style[data-emotion]:not([data-s])");Array.prototype.forEach.call(r,function(p){var v=p.getAttribute("data-emotion");v.indexOf(" ")!==-1&&(document.head.appendChild(p),p.setAttribute("data-s",""))})}var o=t.stylisPlugins||rm,i={},s,a=[];s=t.container||document.head,Array.prototype.forEach.call(document.querySelectorAll('style[data-emotion^="'+n+' "]'),function(p){for(var v=p.getAttribute("data-emotion").split(" "),b=1;b=4;++r,o-=4)n=e.charCodeAt(r)&255|(e.charCodeAt(++r)&255)<<8|(e.charCodeAt(++r)&255)<<16|(e.charCodeAt(++r)&255)<<24,n=(n&65535)*1540483477+((n>>>16)*59797<<16),n^=n>>>24,t=(n&65535)*1540483477+((n>>>16)*59797<<16)^(t&65535)*1540483477+((t>>>16)*59797<<16);switch(o){case 3:t^=(e.charCodeAt(r+2)&255)<<16;case 2:t^=(e.charCodeAt(r+1)&255)<<8;case 1:t^=e.charCodeAt(r)&255,t=(t&65535)*1540483477+((t>>>16)*59797<<16)}return t^=t>>>13,t=(t&65535)*1540483477+((t>>>16)*59797<<16),((t^t>>>15)>>>0).toString(36)}var em={animationIterationCount:1,aspectRatio:1,borderImageOutset:1,borderImageSlice:1,borderImageWidth:1,boxFlex:1,boxFlexGroup:1,boxOrdinalGroup:1,columnCount:1,columns:1,flex:1,flexGrow:1,flexPositive:1,flexShrink:1,flexNegative:1,flexOrder:1,gridRow:1,gridRowEnd:1,gridRowSpan:1,gridRowStart:1,gridColumn:1,gridColumnEnd:1,gridColumnSpan:1,gridColumnStart:1,msGridRow:1,msGridRowSpan:1,msGridColumn:1,msGridColumnSpan:1,fontWeight:1,lineHeight:1,opacity:1,order:1,orphans:1,scale:1,tabSize:1,widows:1,zIndex:1,zoom:1,WebkitLineClamp:1,fillOpacity:1,floodOpacity:1,stopOpacity:1,strokeDasharray:1,strokeDashoffset:1,strokeMiterlimit:1,strokeOpacity:1,strokeWidth:1},tm=/[A-Z]|^ms/g,nm=/_EMO_([^_]+?)_([^]*?)_EMO_/g,wl=function(t){return t.charCodeAt(1)===45},El=function(t){return t!=null&&typeof t!="boolean"},Zi=cl(function(e){return wl(e)?e:e.replace(tm,"-$&").toLowerCase()}),Ol=function(t,n){switch(t){case"animation":case"animationName":if(typeof n=="string")return n.replace(nm,function(r,o,i){return at={name:o,styles:i,next:at},o})}return em[t]!==1&&!wl(t)&&typeof n=="number"&&n!==0?n+"px":n};function pr(e,t,n){if(n==null)return"";var r=n;if(r.__emotion_styles!==void 0)return r;switch(typeof n){case"boolean":return"";case"object":{var o=n;if(o.anim===1)return at={name:o.name,styles:o.styles,next:at},o.name;var i=n;if(i.styles!==void 0){var s=i.next;if(s!==void 0)for(;s!==void 0;)at={name:s.name,styles:s.styles,next:at},s=s.next;var a=i.styles+";";return a}return rm(e,t,n)}case"function":{if(e!==void 0){var l=at,c=n(e);return at=l,pr(e,t,c)}break}}var u=n;if(t==null)return u;var d=t[u];return d!==void 0?d:u}function rm(e,t,n){var r="";if(Array.isArray(n))for(var o=0;or?.(...n))}}const um=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),dm=/^on[A-Z]/;function mr(...e){let t={};for(let n of e){for(let r in t){if(dm.test(r)&&typeof t[r]=="function"&&typeof n[r]=="function"){t[r]=cm(t[r],n[r]);continue}if(r==="className"||r==="class"){t[r]=um(t[r],n[r]);continue}if(r==="style"){t[r]=Object.assign({},t[r]??{},n[r]??{});continue}t[r]=n[r]!==void 0?n[r]:t[r]}for(let r in n)t[r]===void 0&&(t[r]=n[r])}return t}function hm(e,t){if(e!=null){if(typeof e=="function"){e(t);return}try{e.current=t}catch{throw new Error(`Cannot assign value '${t}' to ref '${e}'`)}}}function fm(...e){return t=>{e.forEach(n=>{hm(n,t)})}}function vr(e){const t=Object.assign({},e);for(let n in t)t[n]===void 0&&delete t[n];return t}const Ge=(...e)=>e.filter(Boolean).map(t=>t.trim()).join(" ");function gm(e){return e.default||e}const _e=e=>e!=null&&typeof e=="object"&&!Array.isArray(e),lt=e=>typeof e=="string",is=e=>typeof e=="function";function pm(e){const t=O.version;return!lt(t)||t.startsWith("18.")?e?.ref:e?.props?.ref}const Vl=(...e)=>{const t=e.reduce((n,r)=>(r?.forEach(o=>n.add(o)),n),new Set([]));return Array.from(t)};function mm(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function _n(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??mm(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[vm,lo]=_n({name:"ChakraContext",strict:!0,providerName:""});function bm(e){const{value:t,children:n}=e;return f.jsxs(vm,{value:t,children:[!t._config.disableLayers&&f.jsx(_l,{styles:t.layers.atRule}),f.jsx(_l,{styles:t._global}),n]})}const ym=(e,t)=>{const n={},r={},o=Object.keys(e);for(const i of o)t(i)?r[i]=e[i]:n[i]=e[i];return[r,n]},Vn=(e,t)=>{const n=is(t)?t:r=>t.includes(r);return ym(e,n)},xm=new Set(["htmlWidth","htmlHeight","htmlSize","htmlTranslate"]);function km(e){return typeof e=="string"&&xm.has(e)}function Cm(e,t,n){const{css:r,isValidProperty:o}=lo(),{children:i,...s}=e,a=P.useMemo(()=>{const[h,m]=Vn(s,x=>n(x,t.variantKeys)),[g,p]=Vn(m,t.variantKeys),[v,y]=Vn(p,o);return{forwardedProps:h,variantProps:g,styleProps:v,elementProps:y}},[t.variantKeys,n,s,o]),{css:l,...c}=a.styleProps,u=P.useMemo(()=>{const h={...a.variantProps};return t.variantKeys.includes("colorPalette")||(h.colorPalette=s.colorPalette),t.variantKeys.includes("orientation")||(h.orientation=s.orientation),t(h)},[t,a.variantProps,s.colorPalette,s.orientation]);return{styles:P.useMemo(()=>r(u,...Sm(l),c),[r,u,l,c]),props:{...a.forwardedProps,...a.elementProps,children:i}}}const Sm=e=>(Array.isArray(e)?e:[e]).filter(Boolean).flat(),wm=gm(yp),Em=e=>e!=="theme",Om=(e,t,n)=>{let r;if(t){const o=t.shouldForwardProp;r=e.__emotion_forwardProp&&o?i=>e.__emotion_forwardProp(i)&&o(i):o}return typeof r!="function"&&n&&(r=e.__emotion_forwardProp),r};let Pm=typeof document<"u";const Fl=({cache:e,serialized:t,isStringTag:n})=>{Qi(e,t,n);const r=Rl(()=>Ji(e,t,n));if(!Pm&&r!==void 0){let o=t.name,i=t.next;for(;i!==void 0;)o=Ge(o,i.name),i=i.next;return f.jsx("style",{"data-emotion":Ge(e.key,o),dangerouslySetInnerHTML:{__html:r},nonce:e.sheet.nonce})}return null},Ll={path:["d"],text:["x","y"],circle:["cx","cy","r"],rect:["width","height","x","y","rx","ry"],ellipse:["cx","cy","rx","ry"],g:["transform"],stop:["offset","stopOpacity"]},Im=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),ss=((e,t={},n={})=>{if(Im(Ll,e)){n.forwardProps||(n.forwardProps=[]);const c=Ll[e];n.forwardProps=Vl([...n.forwardProps,...c])}const r=e.__emotion_real===e,o=r&&e.__emotion_base||e;let i,s;n!==void 0&&(i=n.label,s=n.target);let a=[];const l=ts((c,u,d)=>{const{cva:h,isValidProperty:m}=lo(),g=t.__cva__?t:h(t),p=Rm(e.__emotion_cva,g),v=z=>(W,J)=>z.includes(W)?!0:!J?.includes(W)&&!m(W);!n.shouldForwardProp&&n.forwardProps&&(n.shouldForwardProp=v(n.forwardProps));const y=(z,W)=>{const J=typeof e=="string"&&e.charCodeAt(0)>96?wm:Em,ee=!W?.includes(z)&&!m(z);return J(z)&&ee},x=Om(e,n,r)||y,b=O.useMemo(()=>Object.assign({},n.defaultProps,vr(c)),[c]),{props:C,styles:k}=Cm(b,p,x);let E="",I=[k],R=C;if(C.theme==null){R={};for(let z in C)R[z]=C[z];R.theme=O.useContext(ns)}typeof C.className=="string"?E=Sl(u.registered,I,C.className):C.className!=null&&(E=Ge(E,C.className));const _=es(a.concat(I),u.registered,R);_.styles&&(E=Ge(E,`${u.key}-${_.name}`)),s!==void 0&&(E=Ge(E,s));const A=!x("as");let T=A&&C.as||o,S={};for(let z in C)if(!(A&&z==="as")){if(km(z)){const W=z.replace("html","").toLowerCase();S[W]=C[z];continue}x(z)&&(S[z]=C[z])}let N=E.trim();N?S.className=N:Reflect.deleteProperty(S,"className"),S.ref=d;const D=n.forwardAsChild||n.forwardProps?.includes("asChild");if(C.asChild&&!D){const z=O.isValidElement(C.children)?O.Children.only(C.children):O.Children.toArray(C.children).find(O.isValidElement);if(!z)throw new Error("[chakra-ui > factory] No valid child found");T=z.type,S.children=null,Reflect.deleteProperty(S,"asChild"),S=mr(S,z.props),S.ref=fm(d,pm(z))}return S.as&&D?(S.as=void 0,f.jsxs(O.Fragment,{children:[f.jsx(Fl,{cache:u,serialized:_,isStringTag:typeof T=="string"}),f.jsx(T,{asChild:!0,...S,children:f.jsx(C.as,{children:S.children})})]})):f.jsxs(O.Fragment,{children:[f.jsx(Fl,{cache:u,serialized:_,isStringTag:typeof T=="string"}),f.jsx(T,{...S})]})});return l.displayName=i!==void 0?i:`chakra(${typeof o=="string"?o:o.displayName||o.name||"Component"})`,l.__emotion_real=l,l.__emotion_base=o,l.__emotion_forwardProp=n.shouldForwardProp,l.__emotion_cva=t,Object.defineProperty(l,"toString",{value(){return`.${s}`}}),l}).bind(),as=new Map,ve=new Proxy(ss,{apply(e,t,n){return ss(...n)},get(e,t){return as.has(t)||as.set(t,ss(t)),as.get(t)}}),Rm=(e,t)=>e&&!t?e:!e&&t?t:e.merge(t),br=ve("div");br.displayName="Box";const Tm=Object.freeze({}),Nm=Object.freeze({});function Am(e){const{key:t,recipe:n}=e,r=lo();return P.useMemo(()=>{const o=n||(t!=null?r.getRecipe(t):{});return r.cva(structuredClone(o))},[t,n,r])}const _m=e=>e.charAt(0).toUpperCase()+e.slice(1);function Nt(e){const{key:t,recipe:n}=e,r=_m(t||n.className||"Component"),[o,i]=_n({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`});function s(c){const{unstyled:u,...d}=c,h=Am({key:t,recipe:d.recipe||n}),[m,g]=P.useMemo(()=>h.splitVariantProps(d),[h,d]);return{styles:u?Tm:h(m),className:h.className,props:g}}const a=(c,u)=>{const d=ve(c,{},u),h=P.forwardRef((m,g)=>{const p=i(),v=P.useMemo(()=>mr(p,m),[m,p]),{styles:y,className:x,props:b}=s(v);return f.jsx(d,{...b,ref:g,css:[y,v.css],className:Ge(x,v.className)})});return h.displayName=c.displayName||c.name,h};function l(){return o}return{withContext:a,PropsProvider:o,withPropsProvider:l,usePropsContext:i,useRecipeResult:s}}function co(e){return e==null?[]:Array.isArray(e)?e:[e]}var yr=e=>e[0],ls=e=>e[e.length-1],Vm=(e,t)=>e.indexOf(t)!==-1,Gt=(e,...t)=>e.concat(t),qt=(e,...t)=>e.filter(n=>!t.includes(n)),Fn=e=>Array.from(new Set(e)),cs=(e,t)=>{const n=new Set(t);return e.filter(r=>!n.has(r))},Ln=(e,t)=>Vm(e,t)?qt(e,t):Gt(e,t);function Dl(e,t,n={}){const{step:r=1,loop:o=!0}=n,i=t+r,s=e.length,a=s-1;return t===-1?r>0?0:a:i<0?o?a:0:i>=s?o?0:t>s?s:t:i}function Fm(e,t,n={}){return e[Dl(e,t,n)]}function Lm(e,t,n={}){const{step:r=1,loop:o=!0}=n;return Dl(e,t,{step:-r,loop:o})}function Dm(e,t,n={}){return e[Lm(e,t,n)]}function zl(e,t){return e.reduce(([n,r],o)=>(t(o)?n.push(o):r.push(o),[n,r]),[[],[]])}var Ml=e=>e?.constructor.name==="Array",zm=(e,t)=>{if(e.length!==t.length)return!1;for(let n=0;n{if(Object.is(e,t))return!0;if(e==null&&t!=null||e!=null&&t==null)return!1;if(typeof e?.isEqual=="function"&&typeof t?.isEqual=="function")return e.isEqual(t);if(typeof e=="function"&&typeof t=="function")return e.toString()===t.toString();if(Ml(e)&&Ml(t))return zm(Array.from(e),Array.from(t));if(typeof e!="object"||typeof t!="object")return!1;const n=Object.keys(t??Object.create(null)),r=n.length;for(let o=0;oArray.isArray(e),Mm=e=>e===!0||e===!1,jl=e=>e!=null&&typeof e=="object",Kt=e=>jl(e)&&!xr(e),uo=e=>typeof e=="string",Yt=e=>typeof e=="function",jm=e=>e==null,At=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),$m=e=>Object.prototype.toString.call(e),$l=Function.prototype.toString,Bm=$l.call(Object),Wm=e=>{if(!jl(e)||$m(e)!="[object Object]"||Gm(e))return!1;const t=Object.getPrototypeOf(e);if(t===null)return!0;const n=At(t,"constructor")&&t.constructor;return typeof n=="function"&&n instanceof n&&$l.call(n)==Bm},Hm=e=>typeof e=="object"&&e!==null&&"$$typeof"in e&&"props"in e,Um=e=>typeof e=="object"&&e!==null&&"__v_isVNode"in e,Gm=e=>Hm(e)||Um(e),ho=(e,...t)=>(typeof e=="function"?e(...t):e)??void 0,qm=e=>e(),Km=()=>{},fo=(...e)=>(...t)=>{e.forEach(function(n){n?.(...t)})},Ym=(()=>{let e=0;return()=>(e++,e.toString(36))})();function bt(e,t,...n){if(e in t){const o=t[e];return Yt(o)?o(...n):o}const r=new Error(`No matching key: ${JSON.stringify(e)} in ${JSON.stringify(Object.keys(t))}`);throw Error.captureStackTrace?.(r,bt),r}var Bl=(e,t)=>{try{return e()}catch(n){return n instanceof Error&&Error.captureStackTrace?.(n,Bl),t?.()}},{floor:Wl,abs:Hl,round:go,min:Xm,max:Qm,pow:Jm,sign:Zm}=Math,us=e=>Number.isNaN(e),_t=e=>us(e)?0:e,Ul=(e,t)=>(e%t+t)%t,e0=(e,t)=>(e%t+t)%t,t0=(e,t)=>_t(e)>=t,n0=(e,t)=>_t(e)<=t,r0=(e,t,n)=>{const r=_t(e),o=t==null||r>=t,i=n==null||r<=n;return o&&i},o0=(e,t,n)=>go((_t(e)-t)/n)*n+t,Ve=(e,t,n)=>Xm(Qm(_t(e),t),n),i0=(e,t,n)=>(_t(e)-t)/(n-t),s0=(e,t,n,r)=>Ve(o0(e*(n-t)+t,t,r),t,n),Gl=(e,t)=>{let n=e,r=t.toString(),o=r.indexOf("."),i=o>=0?r.length-o:0;if(i>0){let s=Jm(10,i);n=go(n*s)/s}return n},ds=(e,t)=>typeof t=="number"?Wl(e*t+.5)/t:go(e),ql=(e,t,n,r)=>{const o=t!=null?Number(t):0,i=Number(n),s=(e-o)%r;let a=Hl(s)*2>=r?e+Zm(s)*(r-Hl(s)):e-s;if(a=Gl(a,r),!us(o)&&ai){const l=Wl((i-o)/r),c=o+l*r;a=l<=0||c{const r=Math.pow(n,t);return go(e*r)/r},Kl=e=>{if(!Number.isFinite(e))return 0;let t=1,n=0;for(;Math.round(e*t)/t!==e;)t*=10,n+=1;return n},Yl=(e,t,n)=>{let r=t==="+"?e+n:e-n;if(e%1!==0||n%1!==0){const o=10**Math.max(Kl(e),Kl(n));e=Math.round(e*o),n=Math.round(n*o),r=t==="+"?e+n:e-n,r/=o}return r},a0=(e,t)=>Yl(_t(e),"+",t),l0=(e,t)=>Yl(_t(e),"-",t);function po(e){if(!Wm(e)||e===void 0)return e;const t=Reflect.ownKeys(e).filter(r=>typeof r=="string"),n={};for(const r of t){const o=e[r];o!==void 0&&(n[r]=po(o))}return n}function c0(e,t=Object.is){let n={...e};const r=new Set,o=u=>(r.add(u),()=>r.delete(u)),i=()=>{r.forEach(u=>u())};return{subscribe:o,get:u=>n[u],set:(u,d)=>{t(n[u],d)||(n[u]=d,i())},update:u=>{let d=!1;for(const h in u){const m=u[h];m!==void 0&&!t(n[h],m)&&(n[h]=m,d=!0)}d&&i()},snapshot:()=>({...n})}}function kr(...e){e.length===1?e[0]:e[1],e.length===2&&e[0]}function Xl(e,t){if(e==null)throw new Error(t())}function u0(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Dn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??u0(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[qR,Ql]=Dn({name:"EnvironmentContext",hookName:"useEnvironmentContext",providerName:"",strict:!1,defaultValue:{getRootNode:()=>document,getDocument:()=>document,getWindow:()=>window}});function d0(e){if(!e)return;const t=e.selectionStart??0,n=e.selectionEnd??0;Math.abs(n-t)===0&&t===0&&e.setSelectionRange(e.value.length,e.value.length)}var Jl=e=>Math.max(0,Math.min(1,e)),h0=(e,t)=>e.map((n,r)=>e[(Math.max(t,0)+r)%e.length]),Zl=()=>{},mo=e=>typeof e=="object"&&e!==null,f0=2147483647,g0=1,p0=9,m0=11,Te=e=>mo(e)&&e.nodeType===g0&&typeof e.nodeName=="string",hs=e=>mo(e)&&e.nodeType===p0,v0=e=>mo(e)&&e===e.window,ec=e=>Te(e)?e.localName||"":"#document";function b0(e){return["html","body","#document"].includes(ec(e))}var y0=e=>mo(e)&&e.nodeType!==void 0,zn=e=>y0(e)&&e.nodeType===m0&&"host"in e,x0=e=>Te(e)&&e.localName==="input",k0=e=>!!e?.matches("a[href]"),C0=e=>Te(e)?e.offsetWidth>0||e.offsetHeight>0||e.getClientRects().length>0:!1;function S0(e){if(!e)return!1;const t=e.getRootNode();return Cr(t)===e}var w0=/(textarea|select)/;function tc(e){if(e==null||!Te(e))return!1;try{return x0(e)&&e.selectionStart!=null||w0.test(e.localName)||e.isContentEditable||e.getAttribute("contenteditable")==="true"||e.getAttribute("contenteditable")===""}catch{return!1}}function Xt(e,t){if(!e||!t||!Te(e)||!Te(t))return!1;const n=t.getRootNode?.();if(e===t||e.contains(t))return!0;if(n&&zn(n)){let r=t;for(;r;){if(e===r)return!0;r=r.parentNode||r.host}}return!1}function Fe(e){return hs(e)?e:v0(e)?e.document:e?.ownerDocument??document}function E0(e){return Fe(e).documentElement}function be(e){return zn(e)?be(e.host):hs(e)?e.defaultView??window:Te(e)?e.ownerDocument?.defaultView??window:window}function Cr(e){let t=e.activeElement;for(;t?.shadowRoot;){const n=t.shadowRoot.activeElement;if(!n||n===t)break;t=n}return t}function O0(e){if(ec(e)==="html")return e;const t=e.assignedSlot||e.parentNode||zn(e)&&e.host||E0(e);return zn(t)?t.host:t}var fs=new WeakMap;function vo(e){return fs.has(e)||fs.set(e,be(e).getComputedStyle(e)),fs.get(e)}var bo=()=>typeof document<"u";function P0(){return navigator.userAgentData?.platform??navigator.platform}function I0(){const e=navigator.userAgentData;return e&&Array.isArray(e.brands)?e.brands.map(({brand:t,version:n})=>`${t}/${n}`).join(" "):navigator.userAgent}var gs=e=>bo()&&e.test(P0()),nc=e=>bo()&&e.test(I0()),R0=e=>bo()&&e.test(navigator.vendor),rc=()=>bo()&&!!navigator.maxTouchPoints,T0=()=>gs(/^iPhone/i),N0=()=>gs(/^iPad/i)||xo()&&navigator.maxTouchPoints>1,yo=()=>T0()||N0(),A0=()=>xo()||yo(),xo=()=>gs(/^Mac/i),oc=()=>A0()&&R0(/apple/i),_0=()=>nc(/Firefox/i),V0=()=>nc(/Android/i);function F0(e){return e.composedPath?.()??e.nativeEvent?.composedPath?.()}function $e(e){return F0(e)?.[0]??e.target}function L0(e){return j0(e).isComposing||e.keyCode===229}function D0(e){return e.pointerType===""&&e.isTrusted?!0:V0()&&e.pointerType?e.type==="click"&&e.buttons===1:e.detail===0&&!e.pointerType}var z0=e=>e.button===2||xo()&&e.ctrlKey&&e.button===0,M0=e=>"touches"in e&&e.touches.length>0;function j0(e){return e.nativeEvent??e}function ic(e,t="client"){const n=M0(e)?e.touches[0]||e.changedTouches[0]:e;return{x:n[`${t}X`],y:n[`${t}Y`]}}var se=(e,t,n,r)=>{const o=typeof e=="function"?e():e;return o?.addEventListener(t,n,r),()=>{o?.removeEventListener(t,n,r)}};function $0(e,t){const{type:n="HTMLInputElement",property:r="value"}=t,o=be(e)[n].prototype;return Object.getOwnPropertyDescriptor(o,r)??{}}function B0(e){if(e.localName==="input")return"HTMLInputElement";if(e.localName==="textarea")return"HTMLTextAreaElement";if(e.localName==="select")return"HTMLSelectElement"}function ko(e,t,n="value"){if(!e)return;const r=B0(e);r&&$0(e,{type:r,property:n}).set?.call(e,t),e.setAttribute(n,t)}function W0(e,t){const{value:n,bubbles:r=!0}=t;if(!e)return;const o=be(e);e instanceof o.HTMLInputElement&&(ko(e,`${n}`),e.dispatchEvent(new o.Event("input",{bubbles:r})))}function H0(e){return U0(e)?e.form:e.closest("form")}function U0(e){return e.matches("textarea, input, select, button")}function G0(e,t){if(!e)return;const n=H0(e),r=o=>{o.defaultPrevented||t()};return n?.addEventListener("reset",r,{passive:!0}),()=>n?.removeEventListener("reset",r)}function q0(e,t){const n=e?.closest("fieldset");if(!n)return;t(n.disabled);const r=be(n),o=new r.MutationObserver(()=>t(n.disabled));return o.observe(n,{attributes:!0,attributeFilter:["disabled"]}),()=>o.disconnect()}function ps(e,t){if(!e)return;const{onFieldsetDisabledChange:n,onFormReset:r}=t,o=[G0(e,r),q0(e,n)];return()=>o.forEach(i=>i?.())}var sc=e=>Te(e)&&e.tagName==="IFRAME",K0=e=>!Number.isNaN(parseInt(e.getAttribute("tabindex")||"0",10)),Y0=e=>parseInt(e.getAttribute("tabindex")||"0",10)<0,ms="input:not([type='hidden']):not([disabled]), select:not([disabled]), textarea:not([disabled]), a[href], button:not([disabled]), [tabindex], iframe, object, embed, area[href], audio[controls], video[controls], [contenteditable]:not([contenteditable='false']), details > summary:first-of-type",ac=(e,t=!1)=>{if(!e)return[];const n=Array.from(e.querySelectorAll(ms));(t==!0||t=="if-empty"&&n.length===0)&&Te(e)&&Vt(e)&&n.unshift(e);const o=n.filter(Vt);return o.forEach((i,s)=>{if(sc(i)&&i.contentDocument){const a=i.contentDocument.body;o.splice(s,1,...ac(a))}}),o};function Vt(e){return!e||e.closest("[inert]")?!1:e.matches(ms)&&C0(e)}function vs(e,t){if(!e)return[];const r=Array.from(e.querySelectorAll(ms)).filter(Qt);return r.forEach((o,i)=>{if(sc(o)&&o.contentDocument){const s=o.contentDocument.body,a=vs(s);r.splice(i,1,...a)}}),r.length,r}function Qt(e){return e!=null&&e.tabIndex>0?!0:Vt(e)&&!Y0(e)}function Sr(e){return e.tabIndex<0&&(/^(audio|video|details)$/.test(e.localName)||tc(e))&&!K0(e)?0:e.tabIndex}function bs(e){const{root:t,getInitialEl:n,filter:r,enabled:o=!0}=e;if(!o)return;let i=null;if(i||(i=typeof n=="function"?n():n),i||(i=t?.querySelector("[data-autofocus],[autofocus]")),!i){const s=vs(t);i=r?s.filter(r)[0]:s[0]}return i||t||void 0}function ys(e){const t=new Set;function n(r){const o=globalThis.requestAnimationFrame(r);t.add(()=>globalThis.cancelAnimationFrame(o))}return n(()=>n(e)),function(){t.forEach(o=>o())}}function G(e){let t;const n=globalThis.requestAnimationFrame(()=>{t=e()});return()=>{globalThis.cancelAnimationFrame(n),t?.()}}function X0(e,t,n){const r=G(()=>{e.removeEventListener(t,o,!0),n()}),o=()=>{r(),n()};return e.addEventListener(t,o,{once:!0,capture:!0}),r}function Q0(e,t){if(!e)return;const{attributes:n,callback:r}=t,o=e.ownerDocument.defaultView||window,i=new o.MutationObserver(s=>{for(const a of s)a.type==="attributes"&&a.attributeName&&n.includes(a.attributeName)&&r(a)});return i.observe(e,{attributes:!0,attributeFilter:n}),()=>i.disconnect()}function Co(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(Q0(i,t))})),()=>{o.forEach(i=>i?.())}}function lc(e){const t=()=>{const n=be(e);e.dispatchEvent(new n.MouseEvent("click"))};_0()?X0(e,"keyup",t):queueMicrotask(t)}function So(e){const t=O0(e);return b0(t)?Fe(t).body:Te(t)&&xs(t)?t:So(t)}function cc(e,t=[]){const n=So(e),r=n===e.ownerDocument.body,o=be(n);return r?t.concat(o,o.visualViewport||[],xs(n)?n:[]):t.concat(n,cc(n,[]))}var J0=/auto|scroll|overlay|hidden|clip/,Z0=new Set(["inline","contents"]);function xs(e){const t=be(e),{overflow:n,overflowX:r,overflowY:o,display:i}=t.getComputedStyle(e);return J0.test(n+o+r)&&!Z0.has(i)}function ev(e){return e.scrollHeight>e.clientHeight||e.scrollWidth>e.clientWidth}function wo(e,t){const{rootEl:n,...r}=t||{};!e||!n||!xs(n)||!ev(n)||e.scrollIntoView(r)}function uc(e,t){const{left:n,top:r,width:o,height:i}=t.getBoundingClientRect(),s={x:e.x-n,y:e.y-r},a={x:Jl(s.x/o),y:Jl(s.y/i)};function l(c={}){const{dir:u="ltr",orientation:d="horizontal",inverted:h}=c,m=typeof h=="object"?h.x:h,g=typeof h=="object"?h.y:h;return d==="horizontal"?u==="rtl"||m?1-a.x:a.x:g?1-a.y:a.y}return{offset:s,percent:a,getPercentValue:l}}function tv(e,t){const n=e.body,r="pointerLockElement"in e||"mozPointerLockElement"in e,o=()=>!!e.pointerLockElement;function i(){}function s(l){o(),console.error("PointerLock error occurred:",l),e.exitPointerLock()}if(!r)return;try{n.requestPointerLock()}catch{}const a=[se(e,"pointerlockchange",i,!1),se(e,"pointerlockerror",s,!1)];return()=>{a.forEach(l=>l()),e.exitPointerLock()}}var Mn="default",ks="",Eo=new WeakMap;function nv(e={}){const{target:t,doc:n}=e,r=n??document,o=r.documentElement;return yo()?(Mn==="default"&&(ks=o.style.webkitUserSelect,o.style.webkitUserSelect="none"),Mn="disabled"):t&&(Eo.set(t,t.style.userSelect),t.style.userSelect="none"),()=>rv({target:t,doc:r})}function rv(e={}){const{target:t,doc:n}=e,o=(n??document).documentElement;if(yo()){if(Mn!=="disabled")return;Mn="restoring",setTimeout(()=>{ys(()=>{Mn==="restoring"&&(o.style.webkitUserSelect==="none"&&(o.style.webkitUserSelect=ks||""),ks="",Mn="default")})},300)}else if(t&&Eo.has(t)){const i=Eo.get(t);t.style.userSelect==="none"&&(t.style.userSelect=i??""),t.getAttribute("style")===""&&t.removeAttribute("style"),Eo.delete(t)}}function dc(e={}){const{defer:t,target:n,...r}=e,o=t?G:s=>s(),i=[];return i.push(o(()=>{const s=typeof n=="function"?n():n;i.push(nv({...r,target:s}))})),()=>{i.forEach(s=>s?.())}}function ov(e,t){const{onPointerMove:n,onPointerUp:r}=t,o=a=>{const l=ic(a),c=Math.sqrt(l.x**2+l.y**2),u=a.pointerType==="touch"?10:5;if(!(c{const l=ic(a);r({point:l,event:a})},s=[se(e,"pointermove",o,!1),se(e,"pointerup",i,!1),se(e,"pointercancel",i,!1),se(e,"contextmenu",i,!1),dc({doc:e})];return()=>{s.forEach(a=>a())}}function Oo(e,t){return Array.from(e?.querySelectorAll(t)??[])}function iv(e,t){return e?.querySelector(t)??null}var Cs=e=>e.id;function sv(e,t,n=Cs){return e.find(r=>n(r)===t)}function Ss(e,t,n=Cs){const r=sv(e,t,n);return r?e.indexOf(r):-1}function av(e,t,n=!0){let r=Ss(e,t);return r=n?(r+1)%e.length:Math.min(r+1,e.length-1),e[r]}function lv(e,t,n=!0){let r=Ss(e,t);return r===-1?n?e[e.length-1]:null:(r=n?(r-1+e.length)%e.length:Math.max(0,r-1),e[r])}var cv=e=>e.split("").map(t=>{const n=t.charCodeAt(0);return n>0&&n<128?t:n>=128&&n<=255?`/x${n.toString(16)}`.replace("/","\\"):""}).join("").trim(),uv=e=>cv(e.dataset?.valuetext??e.textContent??""),dv=(e,t)=>e.trim().toLowerCase().startsWith(t.toLowerCase());function hv(e,t,n,r=Cs){const o=n?Ss(e,n,r):-1;let i=n?h0(e,o):e;return t.length===1&&(i=i.filter(a=>r(a)!==n)),i.find(a=>dv(uv(a),t))}function Po(e,t){if(!e)return Zl;const n=Object.keys(t).reduce((r,o)=>(r[o]=e.style.getPropertyValue(o),r),{});return Object.assign(e.style,t),()=>{Object.assign(e.style,n),e.style.length===0&&e.removeAttribute("style")}}function fv(e,t,n){if(!e)return Zl;const r=e.style.getPropertyValue(t);return e.style.setProperty(t,n),()=>{e.style.setProperty(t,r),e.style.length===0&&e.removeAttribute("style")}}function gv(e,t){const{state:n,activeId:r,key:o,timeout:i=350,itemToId:s}=t,a=n.keysSoFar+o,c=a.length>1&&Array.from(a).every(g=>g===a[0])?a[0]:a;let u=e.slice();const d=hv(u,c,r,s);function h(){clearTimeout(n.timer),n.timer=-1}function m(g){n.keysSoFar=g,h(),g!==""&&(n.timer=+setTimeout(()=>{m(""),h()},i))}return m(a),d}var wr=Object.assign(gv,{defaultOptions:{keysSoFar:"",timer:-1},isValidEvent:pv});function pv(e){return e.key.length===1&&!e.ctrlKey&&!e.metaKey}function mv(e,t,n){const{signal:r}=t;return[new Promise((s,a)=>{const l=setTimeout(()=>{a(new Error(`Timeout of ${n}ms exceeded`))},n);r.addEventListener("abort",()=>{clearTimeout(l),a(new Error("Promise aborted"))}),e.then(c=>{r.aborted||(clearTimeout(l),s(c))}).catch(c=>{r.aborted||(clearTimeout(l),a(c))})}),()=>t.abort()]}function vv(e,t){const{timeout:n,rootNode:r}=t,o=be(r),i=Fe(r),s=new o.AbortController;return mv(new Promise(a=>{const l=e();if(l){a(l);return}const c=new o.MutationObserver(()=>{const u=e();u&&u.isConnected&&(c.disconnect(),a(u))});c.observe(i.body,{childList:!0,subtree:!0})}),s,n)}var bv=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),yv=/((?:--)?(?:\w+-?)+)\s*:\s*([^;]*)/g,hc=e=>{const t={};let n;for(;n=yv.exec(e);)t[n[1]]=n[2];return t},xv=(e,t)=>{if(uo(e)){if(uo(t))return`${e};${t}`;e=hc(e)}else uo(t)&&(t=hc(t));return Object.assign({},e??{},t??{})};function qe(...e){let t={};for(let n of e){if(!n)continue;for(let o in t){if(o.startsWith("on")&&typeof t[o]=="function"&&typeof n[o]=="function"){t[o]=fo(n[o],t[o]);continue}if(o==="className"||o==="class"){t[o]=bv(t[o],n[o]);continue}if(o==="style"){t[o]=xv(t[o],n[o]);continue}t[o]=n[o]!==void 0?n[o]:t[o]}for(let o in n)t[o]===void 0&&(t[o]=n[o]);const r=Object.getOwnPropertySymbols(n);for(let o of r)t[o]=n[o]}return t}function fc(e,t,n){let r=[],o;return i=>{const s=e(i);return(s.length!==r.length||s.some((l,c)=>!Ze(r[c],l)))&&(r=s,o=t(s,i)),o}}function Ft(){return{and:(...e)=>function(n){return e.every(r=>n.guard(r))},or:(...e)=>function(n){return e.some(r=>n.guard(r))},not:e=>function(n){return!n.guard(e)}}}function KR(e){return e}function gc(){return{guards:Ft(),createMachine:e=>e,choose:e=>function({choose:n}){return n(e)?.actions}}}var jn=(e=>(e.NotStarted="Not Started",e.Started="Started",e.Stopped="Stopped",e))(jn||{}),ws="__init__";function kv(e){const t=()=>e.getRootNode?.()??document,n=()=>Fe(t());return{...e,getRootNode:t,getDoc:n,getWin:()=>n().defaultView??window,getActiveElement:()=>Cr(t()),isActiveElement:S0,getById:s=>t().getElementById(s)}}function Io(...e){return t=>{const n=[];for(const r of e)if(typeof r=="function"){const o=r(t);typeof o=="function"&&n.push(o)}else r&&(r.current=t);if(n.length)return()=>{for(const r of n)r()}}}function Cv(e){let t=Object.getOwnPropertyDescriptor(e.props,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning;return n?e.ref:(t=Object.getOwnPropertyDescriptor(e,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning,n?e.props.ref:e.props.ref||e.ref)}const Es=e=>{const t=P.memo(P.forwardRef((n,r)=>{const{asChild:o,children:i,...s}=n;if(!o)return P.createElement(e,{...s,ref:r},i);if(!P.isValidElement(i))return null;const a=P.Children.only(i),l=Cv(a);return P.cloneElement(a,{...qe(s,a.props),ref:r?Io(r,l):l})}));return t.displayName=e.displayName||e.name,t},yt=(()=>{const e=new Map;return new Proxy(Es,{apply(t,n,r){return Es(r[0])},get(t,n){const r=n;return e.has(r)||e.set(r,Es(r)),e.get(r)}})})(),[XR,Sv]=Dn({name:"LocaleContext",hookName:"useLocaleContext",providerName:"",strict:!1,defaultValue:{dir:"ltr",locale:"en-US"}}),pc=()=>(e,t)=>t.reduce((n,r)=>{const[o,i]=n,s=r;return i[s]!==void 0&&(o[s]=i[s]),delete i[s],[o,i]},[{},{...e}]),mc=e=>pc()(e,["immediate","lazyMount","onExitComplete","present","skipAnimationOnMount","unmountOnExit"]);function wv(e){return new Proxy({},{get(t,n){return n==="style"?r=>e({style:r}).style:e}})}var M=()=>e=>Array.from(new Set(e));function Ev(e,t){const{state:n,send:r,context:o}=e,i=n.matches("mounted","unmountSuspended");return{skip:!o.get("initial"),present:i,setNode(s){s&&r({type:"NODE.SET",node:s})},unmount(){r({type:"UNMOUNT"})}}}var Ov={props({props:e}){return{...e,present:!!e.present}},initialState({prop:e}){return e("present")?"mounted":"unmounted"},refs(){return{node:null,styles:null}},context({bindable:e}){return{unmountAnimationName:e(()=>({defaultValue:null})),prevAnimationName:e(()=>({defaultValue:null})),present:e(()=>({defaultValue:!1})),initial:e(()=>({sync:!0,defaultValue:!1}))}},exit:["clearInitial","cleanupNode"],watch({track:e,prop:t,send:n}){e([()=>t("present")],()=>{n({type:"PRESENCE.CHANGED"})})},on:{"NODE.SET":{actions:["setupNode"]},"PRESENCE.CHANGED":{actions:["setInitial","syncPresence"]}},states:{mounted:{on:{UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]},"UNMOUNT.SUSPEND":{target:"unmountSuspended"}}},unmountSuspended:{effects:["trackAnimationEvents"],on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]},UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]}}},unmounted:{on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]}}}},implementations:{actions:{setInitial:({context:e})=>{e.get("initial")||queueMicrotask(()=>{e.set("initial",!0)})},clearInitial:({context:e})=>{e.set("initial",!1)},invokeOnExitComplete:({prop:e})=>{e("onExitComplete")?.()},setupNode:({refs:e,event:t})=>{e.get("node")!==t.node&&(e.set("node",t.node),e.set("styles",vo(t.node)))},cleanupNode:({refs:e})=>{e.set("node",null),e.set("styles",null)},syncPresence:({context:e,refs:t,send:n,prop:r})=>{const o=r("present");if(o)return n({type:"MOUNT",src:"presence.changed"});const i=t.get("node");if(!o&&i?.ownerDocument.visibilityState==="hidden")return n({type:"UNMOUNT",src:"visibilitychange"});G(()=>{const s=Ro(t.get("styles"));e.set("unmountAnimationName",s),s==="none"||s===e.get("prevAnimationName")||t.get("styles")?.display==="none"||t.get("styles")?.animationDuration==="0s"?n({type:"UNMOUNT",src:"presence.changed"}):n({type:"UNMOUNT.SUSPEND"})})},setPrevAnimationName:({context:e,refs:t})=>{G(()=>{e.set("prevAnimationName",Ro(t.get("styles")))})},clearPrevAnimationName:({context:e})=>{e.set("prevAnimationName",null)}},effects:{trackAnimationEvents:({context:e,refs:t,send:n})=>{const r=t.get("node");if(!r)return;const o=a=>{(a.composedPath?.()?.[0]??a.target)===r&&e.set("prevAnimationName",Ro(t.get("styles")))},i=a=>{const l=Ro(t.get("styles"));$e(a)===r&&l===e.get("unmountAnimationName")&&n({type:"UNMOUNT",src:"animationend"})};r.addEventListener("animationstart",o),r.addEventListener("animationcancel",i),r.addEventListener("animationend",i);const s=Po(r,{animationFillMode:"forwards"});return()=>{r.removeEventListener("animationstart",o),r.removeEventListener("animationcancel",i),r.removeEventListener("animationend",i),ys(()=>s())}}}}};function Ro(e){return e?.animationName||"none"}M()(["onExitComplete","present","immediate"]);var vc=typeof globalThis.document<"u"?P.useLayoutEffect:P.useEffect;function To(e){const t=e().value??e().defaultValue,n=e().isEqual??Object.is,[r]=P.useState(t),[o,i]=P.useState(r),s=e().value!==void 0,a=P.useRef(o);a.current=s?e().value:o;const l=P.useRef(a.current);vc(()=>{l.current=a.current},[o,e().value]);const c=d=>{const h=l.current,m=Yt(d)?d(h):d;e().debug&&console.log(`[bindable > ${e().debug}] setValue`,{next:m,prev:h}),s||i(m),n(m,h)||e().onChange?.(m,h)};function u(){return s?e().value:o}return{initial:r,ref:a,get:u,set(d){(e().sync?Tt.flushSync:qm)(()=>c(d))},invoke(d,h){e().onChange?.(d,h)},hash(d){return e().hash?.(d)??String(d)}}}To.cleanup=e=>{P.useEffect(()=>e,[])},To.ref=e=>{const t=P.useRef(e);return{get:()=>t.current,set:n=>{t.current=n}}};function Pv(e){const t=P.useRef(e);return{get(n){return t.current[n]},set(n,r){t.current[n]=r}}}var Iv=(e,t)=>{const n=P.useRef(!1),r=P.useRef(!1);P.useEffect(()=>{if(n.current&&r.current)return t();r.current=!0},[...(e??[]).map(o=>typeof o=="function"?o():o)]),P.useEffect(()=>(n.current=!0,()=>{n.current=!1}),[])};function bc(e,t={}){const n=P.useMemo(()=>{const{id:T,ids:S,getRootNode:N}=t;return kv({id:T,ids:S,getRootNode:N})},[t]),r=(...T)=>{e.debug&&console.log(...T)},o=e.props?.({props:po(t),scope:n})??t,i=Rv(o),s=e.context?.({prop:i,bindable:To,scope:n,flush:xc,getContext(){return l},getComputed(){return k},getRefs(){return p},getEvent(){return m()}}),a=yc(s),l={get(T){return a.current?.[T].ref.current},set(T,S){a.current?.[T].set(S)},initial(T){return a.current?.[T].initial},hash(T){const S=a.current?.[T].get();return a.current?.[T].hash(S)}},c=P.useRef(new Map),u=P.useRef(null),d=P.useRef(null),h=P.useRef({type:""}),m=()=>({...h.current,current(){return h.current},previous(){return d.current}}),g=()=>({...E,matches(...T){return T.includes(E.ref.current)},hasTag(T){return!!e.states[E.ref.current]?.tags?.includes(T)}}),p=Pv(e.refs?.({prop:i,context:l})??{}),v=()=>({state:g(),context:l,event:m(),prop:i,send:A,action:y,guard:x,track:Iv,refs:p,computed:k,flush:xc,scope:n,choose:C}),y=T=>{const S=Yt(T)?T(v()):T;if(!S)return;const N=S.map(D=>{const z=e.implementations?.actions?.[D];return z||kr(`[zag-js] No implementation found for action "${JSON.stringify(D)}"`),z});for(const D of N)D?.(v())},x=T=>Yt(T)?T(v()):e.implementations?.guards?.[T](v()),b=T=>{const S=Yt(T)?T(v()):T;if(!S)return;const N=S.map(z=>{const W=e.implementations?.effects?.[z];return W||kr(`[zag-js] No implementation found for effect "${JSON.stringify(z)}"`),W}),D=[];for(const z of N){const W=z?.(v());W&&D.push(W)}return()=>D.forEach(z=>z?.())},C=T=>co(T).find(S=>{let N=!S.guard;return uo(S.guard)?N=!!x(S.guard):Yt(S.guard)&&(N=S.guard(v())),N}),k=T=>{Xl(e.computed,()=>"[zag-js] No computed object found on machine");const S=e.computed[T];return S({context:l,event:m(),prop:i,refs:p,scope:n,computed:k})},E=To(()=>({defaultValue:e.initialState({prop:i}),onChange(T,S){S&&(c.current.get(S)?.(),c.current.delete(S)),S&&y(e.states[S]?.exit),y(u.current?.actions);const N=b(e.states[T]?.effects);if(N&&c.current.set(T,N),S===ws){y(e.entry);const D=b(e.effects);D&&c.current.set(ws,D)}y(e.states[T]?.entry)}})),I=P.useRef(void 0),R=P.useRef(jn.NotStarted);vc(()=>{queueMicrotask(()=>{const N=R.current===jn.Started;R.current=jn.Started,r(N?"rehydrating...":"initializing...");const D=I.current??E.initial;E.invoke(D,N?E.get():ws)});const T=c.current,S=E.ref.current;return()=>{r("unmounting..."),I.current=S,R.current=jn.Stopped,T.forEach(N=>N?.()),c.current=new Map,u.current=null,queueMicrotask(()=>{y(e.exit)})}},[]);const _=()=>"ref"in E?E.ref.current:E.get(),A=T=>{queueMicrotask(()=>{if(R.current!==jn.Started)return;d.current=h.current,h.current=T;let S=_();const N=e.states[S].on?.[T.type]??e.on?.[T.type],D=C(N);if(!D)return;u.current=D;const z=D.target??S;r("transition",T.type,D.target||S,`(${D.actions})`);const W=z!==S;W?Tt.flushSync(()=>E.set(z)):D.reenter&&!W?E.invoke(S,S):y(D.actions??[])})};return e.watch?.(v()),{state:g(),send:A,context:l,prop:i,scope:n,refs:p,computed:k,event:m(),getStatus:()=>R.current}}function yc(e){const t=P.useRef(e);return t.current=e,t}function Rv(e){const t=yc(e);return function(r){return t.current[r]}}function xc(e){queueMicrotask(()=>{Tt.flushSync(()=>e())})}var Tv=wv(e=>e);function Nv(e,t={}){const{sync:n=!1}=t,r=Av(e);return P.useCallback((...o)=>n?queueMicrotask(()=>r.current?.(...o)):r.current?.(...o),[n,r])}function Av(e){const t=P.useRef(e);return t.current=e,t}const Os=(e={})=>{const{lazyMount:t,unmountOnExit:n,present:r,skipAnimationOnMount:o=!1,...i}=e,s=P.useRef(!1),a={...i,present:r,onExitComplete:Nv(e.onExitComplete)},l=bc(Ov,a),c=Ev(l);c.present&&(s.current=!0);const u=!c.present&&!s.current&&t||n&&!c.present&&s.current,d=()=>({"data-state":c.skip&&o?void 0:r?"open":"closed",hidden:!c.present});return{ref:c.setNode,getPresenceProps:d,present:c.present,unmounted:u}},[kc,Ps]=Dn({name:"PresenceContext",hookName:"usePresenceContext",providerName:""}),No=ve("span"),{withContext:_v}=Nt({key:"text"}),Le=_v("p");function Cc(e,t=[]){const n=P.useRef(()=>{throw new Error("Cannot call an event handler while rendering.")});return P.useInsertionEffect(()=>{n.current=e}),P.useCallback((...r)=>n.current?.(...r),t)}function Jt(e={}){const t=Cc(e.onOpen),n=Cc(e.onClose),[r,o]=P.useState(e.defaultOpen||!1),i=e.open!==void 0?e.open:r,s=e.open!==void 0,a=P.useCallback(()=>{s||o(!1),n?.()},[s,n]),l=P.useCallback(()=>{s||o(!0),t?.()},[s,t]),c=P.useCallback(()=>{i?a():l()},[i,l,a]);return{open:i,onOpen:l,onClose:a,onToggle:c,setOpen:o}}var j=(e,t=[])=>({parts:(...n)=>{if(Vv(t))return j(e,n);throw new Error("createAnatomy().parts(...) should only be called once. Did you mean to use .extendWith(...) ?")},extendWith:(...n)=>j(e,[...t,...n]),omit:(...n)=>j(e,t.filter(r=>!n.includes(r))),rename:n=>j(n,t),keys:()=>t,build:()=>[...new Set(t)].reduce((n,r)=>Object.assign(n,{[r]:{selector:[`&[data-scope="${$n(e)}"][data-part="${$n(r)}"]`,`& [data-scope="${$n(e)}"][data-part="${$n(r)}"]`].join(", "),attrs:{"data-scope":$n(e),"data-part":$n(r)}}}),{})}),$n=e=>e.replace(/([A-Z])([A-Z])/g,"$1-$2").replace(/([a-z])([A-Z])/g,"$1-$2").replace(/[\s_]+/g,"-").toLowerCase(),Vv=e=>e.length===0,Sc=j("collapsible").parts("root","trigger","content","indicator");Sc.build(),M()(["dir","disabled","getRootNode","id","ids","onExitComplete","onOpenChange","defaultOpen","open"]);var Fv=Object.defineProperty,Lv=(e,t,n)=>t in e?Fv(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,Is=(e,t,n)=>Lv(e,t+"",n),Dv=(e,t)=>{if(Object.keys(e).length!==Object.keys(t).length)return!1;for(let n in e)if(e[n]!==t[n])return!1;return!0},Rs=class{toHexInt(){return this.toFormat("rgba").toHexInt()}getChannelValue(e){if(e in this)return this[e];throw new Error("Unsupported color channel: "+e)}getChannelValuePercent(e,t){const n=t??this.getChannelValue(e),{minValue:r,maxValue:o}=this.getChannelRange(e);return i0(n,r,o)}getChannelPercentValue(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=s0(t,n,r,o);return ql(i,n,r,o)}withChannelValue(e,t){const{minValue:n,maxValue:r}=this.getChannelRange(e);if(e in this){let o=this.clone();return o[e]=Ve(t,n,r),o}throw new Error("Unsupported color channel: "+e)}getColorAxes(e){let{xChannel:t,yChannel:n}=e,r=t||this.getChannels().find(s=>s!==n),o=n||this.getChannels().find(s=>s!==r),i=this.getChannels().find(s=>s!==r&&s!==o);return{xChannel:r,yChannel:o,zChannel:i}}incrementChannel(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=ql(Ve(this.getChannelValue(e)+t,n,r),n,r,o);return this.withChannelValue(e,i)}decrementChannel(e,t){return this.incrementChannel(e,-t)}isEqual(e){return Dv(this.toJSON(),e.toJSON())&&this.getChannelValue("alpha")===e.getChannelValue("alpha")}},zv=/^#[\da-f]+$/i,Mv=/^rgba?\((.*)\)$/,jv=/[^#]/gi,wc=class ji extends Rs{constructor(t,n,r,o){super(),this.red=t,this.green=n,this.blue=r,this.alpha=o}static parse(t){let n=[];if(zv.test(t)&&[4,5,7,9].includes(t.length)){const o=(t.length<6?t.replace(jv,"$&$&"):t).slice(1).split("");for(;o.length>0;)n.push(parseInt(o.splice(0,2).join(""),16));n[3]=n[3]!==void 0?n[3]/255:void 0}const r=t.match(Mv);return r?.[1]&&(n=r[1].split(",").map(o=>Number(o.trim())).map((o,i)=>Ve(o,0,i<3?255:1))),n.length<3?void 0:new ji(n[0],n[1],n[2],n[3]??1)}toString(t){switch(t){case"hex":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")).toUpperCase();case"hexa":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")+Math.round(this.alpha*255).toString(16).padStart(2,"0")).toUpperCase();case"rgb":return`rgb(${this.red}, ${this.green}, ${this.blue})`;case"css":case"rgba":return`rgba(${this.red}, ${this.green}, ${this.blue}, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"hsb":return this.toHSB().toString("hsb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"rgba":return this;case"hsba":return this.toHSB();case"hsla":return this.toHSL();default:throw new Error("Unsupported color conversion: rgb -> "+t)}}toHexInt(){return this.red<<16|this.green<<8|this.blue}toHSB(){const t=this.red/255,n=this.green/255,r=this.blue/255,o=Math.min(t,n,r),i=Math.max(t,n,r),s=i-o,a=i===0?0:s/i;let l=0;if(s!==0){switch(i){case t:l=(n-r)/s+(nNumber(a.trim().replace("%","")));return new $i(Ul(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsl":return`hsl(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%)`;case"css":case"hsla":return`hsla(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%, ${this.alpha})`;case"hsb":return this.toHSB().toString("hsb");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsla":return this;case"hsba":return this.toHSB();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsl -> "+t)}}toHSB(){let t=this.saturation/100,n=this.lightness/100,r=n+t*Math.min(n,1-n);return t=r===0?0:2*(1-n/r),new As(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.lightness/100,o=n*Math.min(r,1-r),i=(s,a=(s+t/30)%12)=>r-o*Math.max(Math.min(a-3,9-a,1),-1);return new Ts(Math.round(i(0)*255),Math.round(i(8)*255),Math.round(i(4)*255),oe(this.alpha,2))}clone(){return new $i(this.hue,this.saturation,this.lightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"lightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="lightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"lightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,l:this.lightness,a:this.alpha}}getFormat(){return"hsla"}getChannels(){return $i.colorChannels}};Is(Ec,"colorChannels",["hue","saturation","lightness"]);var Ns=Ec,Bv=/hsb\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%)\)|hsba\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d(.\d+)?)\)/,Oc=class Bi extends Rs{constructor(t,n,r,o){super(),this.hue=t,this.saturation=n,this.brightness=r,this.alpha=o}static parse(t){let n;if(n=t.match(Bv)){const[r,o,i,s]=(n[1]??n[2]).split(",").map(a=>Number(a.trim().replace("%","")));return new Bi(Ul(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"css":return this.toHSL().toString("css");case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsb":return`hsb(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%)`;case"hsba":return`hsba(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsba":return this;case"hsla":return this.toHSL();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsb -> "+t)}}toHSL(){let t=this.saturation/100,n=this.brightness/100,r=n*(1-t/2);return t=r===0||r===1?0:(n-r)/Math.min(r,1-r),new Ns(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.brightness/100,o=(i,s=(i+t/60)%6)=>r-n*r*Math.max(Math.min(s,4-s,1),0);return new Ts(Math.round(o(5)*255),Math.round(o(3)*255),Math.round(o(1)*255),oe(this.alpha,2))}clone(){return new Bi(this.hue,this.saturation,this.brightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"brightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="brightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"brightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,b:this.brightness,a:this.alpha}}getFormat(){return"hsba"}getChannels(){return Bi.colorChannels}};Is(Oc,"colorChannels",["hue","saturation","brightness"]);var As=Oc,Wv="aliceblue:f0f8ff,antiquewhite:faebd7,aqua:00ffff,aquamarine:7fffd4,azure:f0ffff,beige:f5f5dc,bisque:ffe4c4,black:000000,blanchedalmond:ffebcd,blue:0000ff,blueviolet:8a2be2,brown:a52a2a,burlywood:deb887,cadetblue:5f9ea0,chartreuse:7fff00,chocolate:d2691e,coral:ff7f50,cornflowerblue:6495ed,cornsilk:fff8dc,crimson:dc143c,cyan:00ffff,darkblue:00008b,darkcyan:008b8b,darkgoldenrod:b8860b,darkgray:a9a9a9,darkgreen:006400,darkkhaki:bdb76b,darkmagenta:8b008b,darkolivegreen:556b2f,darkorange:ff8c00,darkorchid:9932cc,darkred:8b0000,darksalmon:e9967a,darkseagreen:8fbc8f,darkslateblue:483d8b,darkslategray:2f4f4f,darkturquoise:00ced1,darkviolet:9400d3,deeppink:ff1493,deepskyblue:00bfff,dimgray:696969,dodgerblue:1e90ff,firebrick:b22222,floralwhite:fffaf0,forestgreen:228b22,fuchsia:ff00ff,gainsboro:dcdcdc,ghostwhite:f8f8ff,gold:ffd700,goldenrod:daa520,gray:808080,green:008000,greenyellow:adff2f,honeydew:f0fff0,hotpink:ff69b4,indianred:cd5c5c,indigo:4b0082,ivory:fffff0,khaki:f0e68c,lavender:e6e6fa,lavenderblush:fff0f5,lawngreen:7cfc00,lemonchiffon:fffacd,lightblue:add8e6,lightcoral:f08080,lightcyan:e0ffff,lightgoldenrodyellow:fafad2,lightgrey:d3d3d3,lightgreen:90ee90,lightpink:ffb6c1,lightsalmon:ffa07a,lightseagreen:20b2aa,lightskyblue:87cefa,lightslategray:778899,lightsteelblue:b0c4de,lightyellow:ffffe0,lime:00ff00,limegreen:32cd32,linen:faf0e6,magenta:ff00ff,maroon:800000,mediumaquamarine:66cdaa,mediumblue:0000cd,mediumorchid:ba55d3,mediumpurple:9370d8,mediumseagreen:3cb371,mediumslateblue:7b68ee,mediumspringgreen:00fa9a,mediumturquoise:48d1cc,mediumvioletred:c71585,midnightblue:191970,mintcream:f5fffa,mistyrose:ffe4e1,moccasin:ffe4b5,navajowhite:ffdead,navy:000080,oldlace:fdf5e6,olive:808000,olivedrab:6b8e23,orange:ffa500,orangered:ff4500,orchid:da70d6,palegoldenrod:eee8aa,palegreen:98fb98,paleturquoise:afeeee,palevioletred:d87093,papayawhip:ffefd5,peachpuff:ffdab9,peru:cd853f,pink:ffc0cb,plum:dda0dd,powderblue:b0e0e6,purple:800080,rebeccapurple:663399,red:ff0000,rosybrown:bc8f8f,royalblue:4169e1,saddlebrown:8b4513,salmon:fa8072,sandybrown:f4a460,seagreen:2e8b57,seashell:fff5ee,sienna:a0522d,silver:c0c0c0,skyblue:87ceeb,slateblue:6a5acd,slategray:708090,snow:fffafa,springgreen:00ff7f,steelblue:4682b4,tan:d2b48c,teal:008080,thistle:d8bfd8,tomato:ff6347,turquoise:40e0d0,violet:ee82ee,wheat:f5deb3,white:ffffff,whitesmoke:f5f5f5,yellow:ffff00,yellowgreen:9acd32",Hv=e=>{const t=new Map,n=e.split(",");for(let r=0;r{if(Pc.has(e))return Ao(Pc.get(e));const t=Ts.parse(e)||As.parse(e)||Ns.parse(e);if(!t){const n=new Error("Invalid color value: "+e);throw Error.captureStackTrace?.(n,Ao),n}return t};const Uv=["top","right","bottom","left"],Lt=Math.min,Be=Math.max,_o=Math.round,Vo=Math.floor,ct=e=>({x:e,y:e}),Gv={left:"right",right:"left",bottom:"top",top:"bottom"},qv={start:"end",end:"start"};function _s(e,t,n){return Be(e,Lt(t,n))}function xt(e,t){return typeof e=="function"?e(t):e}function kt(e){return e.split("-")[0]}function Bn(e){return e.split("-")[1]}function Vs(e){return e==="x"?"y":"x"}function Fs(e){return e==="y"?"height":"width"}const Kv=new Set(["top","bottom"]);function ut(e){return Kv.has(kt(e))?"y":"x"}function Ls(e){return Vs(ut(e))}function Yv(e,t,n){n===void 0&&(n=!1);const r=Bn(e),o=Ls(e),i=Fs(o);let s=o==="x"?r===(n?"end":"start")?"right":"left":r==="start"?"bottom":"top";return t.reference[i]>t.floating[i]&&(s=Fo(s)),[s,Fo(s)]}function Xv(e){const t=Fo(e);return[Ds(e),t,Ds(t)]}function Ds(e){return e.replace(/start|end/g,t=>qv[t])}const Ic=["left","right"],Rc=["right","left"],Qv=["top","bottom"],Jv=["bottom","top"];function Zv(e,t,n){switch(e){case"top":case"bottom":return n?t?Rc:Ic:t?Ic:Rc;case"left":case"right":return t?Qv:Jv;default:return[]}}function eb(e,t,n,r){const o=Bn(e);let i=Zv(kt(e),n==="start",r);return o&&(i=i.map(s=>s+"-"+o),t&&(i=i.concat(i.map(Ds)))),i}function Fo(e){return e.replace(/left|right|bottom|top/g,t=>Gv[t])}function tb(e){return{top:0,right:0,bottom:0,left:0,...e}}function Tc(e){return typeof e!="number"?tb(e):{top:e,right:e,bottom:e,left:e}}function Lo(e){const{x:t,y:n,width:r,height:o}=e;return{width:r,height:o,top:n,left:t,right:t+r,bottom:n+o,x:t,y:n}}function Nc(e,t,n){let{reference:r,floating:o}=e;const i=ut(t),s=Ls(t),a=Fs(s),l=kt(t),c=i==="y",u=r.x+r.width/2-o.width/2,d=r.y+r.height/2-o.height/2,h=r[a]/2-o[a]/2;let m;switch(l){case"top":m={x:u,y:r.y-o.height};break;case"bottom":m={x:u,y:r.y+r.height};break;case"right":m={x:r.x+r.width,y:d};break;case"left":m={x:r.x-o.width,y:d};break;default:m={x:r.x,y:r.y}}switch(Bn(t)){case"start":m[s]-=h*(n&&c?-1:1);break;case"end":m[s]+=h*(n&&c?-1:1);break}return m}const nb=async(e,t,n)=>{const{placement:r="bottom",strategy:o="absolute",middleware:i=[],platform:s}=n,a=i.filter(Boolean),l=await(s.isRTL==null?void 0:s.isRTL(t));let c=await s.getElementRects({reference:e,floating:t,strategy:o}),{x:u,y:d}=Nc(c,r,l),h=r,m={},g=0;for(let p=0;p({name:"arrow",options:e,async fn(t){const{x:n,y:r,placement:o,rects:i,platform:s,elements:a,middlewareData:l}=t,{element:c,padding:u=0}=xt(e,t)||{};if(c==null)return{};const d=Tc(u),h={x:n,y:r},m=Ls(o),g=Fs(m),p=await s.getDimensions(c),v=m==="y",y=v?"top":"left",x=v?"bottom":"right",b=v?"clientHeight":"clientWidth",C=i.reference[g]+i.reference[m]-h[m]-i.floating[g],k=h[m]-i.reference[m],E=await(s.getOffsetParent==null?void 0:s.getOffsetParent(c));let I=E?E[b]:0;(!I||!await(s.isElement==null?void 0:s.isElement(E)))&&(I=a.floating[b]||i.floating[g]);const R=C/2-k/2,_=I/2-p[g]/2-1,A=Lt(d[y],_),T=Lt(d[x],_),S=A,N=I-p[g]-T,D=I/2-p[g]/2+R,z=_s(S,D,N),W=!l.arrow&&Bn(o)!=null&&D!==z&&i.reference[g]/2-(DD<=0)){var T,S;const D=(((T=i.flip)==null?void 0:T.index)||0)+1,z=I[D];if(z&&(!(d==="alignment"?x!==ut(z):!1)||A.every(ee=>ut(ee.placement)===x?ee.overflows[0]>0:!0)))return{data:{index:D,overflows:A},reset:{placement:z}};let W=(S=A.filter(J=>J.overflows[0]<=0).sort((J,ee)=>J.overflows[1]-ee.overflows[1])[0])==null?void 0:S.placement;if(!W)switch(m){case"bestFit":{var N;const J=(N=A.filter(ee=>{if(E){const $=ut(ee.placement);return $===x||$==="y"}return!0}).map(ee=>[ee.placement,ee.overflows.filter($=>$>0).reduce(($,q)=>$+q,0)]).sort((ee,$)=>ee[1]-$[1])[0])==null?void 0:N[0];J&&(W=J);break}case"initialPlacement":W=a;break}if(o!==W)return{reset:{placement:W}}}return{}}}};function Ac(e,t){return{top:e.top-t.height,right:e.right-t.width,bottom:e.bottom-t.height,left:e.left-t.width}}function _c(e){return Uv.some(t=>e[t]>=0)}const ib=function(e){return e===void 0&&(e={}),{name:"hide",options:e,async fn(t){const{rects:n}=t,{strategy:r="referenceHidden",...o}=xt(e,t);switch(r){case"referenceHidden":{const i=await Er(t,{...o,elementContext:"reference"}),s=Ac(i,n.reference);return{data:{referenceHiddenOffsets:s,referenceHidden:_c(s)}}}case"escaped":{const i=await Er(t,{...o,altBoundary:!0}),s=Ac(i,n.floating);return{data:{escapedOffsets:s,escaped:_c(s)}}}default:return{}}}}},Vc=new Set(["left","top"]);async function sb(e,t){const{placement:n,platform:r,elements:o}=e,i=await(r.isRTL==null?void 0:r.isRTL(o.floating)),s=kt(n),a=Bn(n),l=ut(n)==="y",c=Vc.has(s)?-1:1,u=i&&l?-1:1,d=xt(t,e);let{mainAxis:h,crossAxis:m,alignmentAxis:g}=typeof d=="number"?{mainAxis:d,crossAxis:0,alignmentAxis:null}:{mainAxis:d.mainAxis||0,crossAxis:d.crossAxis||0,alignmentAxis:d.alignmentAxis};return a&&typeof g=="number"&&(m=a==="end"?g*-1:g),l?{x:m*u,y:h*c}:{x:h*c,y:m*u}}const ab=function(e){return e===void 0&&(e=0),{name:"offset",options:e,async fn(t){var n,r;const{x:o,y:i,placement:s,middlewareData:a}=t,l=await sb(t,e);return s===((n=a.offset)==null?void 0:n.placement)&&(r=a.arrow)!=null&&r.alignmentOffset?{}:{x:o+l.x,y:i+l.y,data:{...l,placement:s}}}}},lb=function(e){return e===void 0&&(e={}),{name:"shift",options:e,async fn(t){const{x:n,y:r,placement:o}=t,{mainAxis:i=!0,crossAxis:s=!1,limiter:a={fn:v=>{let{x:y,y:x}=v;return{x:y,y:x}}},...l}=xt(e,t),c={x:n,y:r},u=await Er(t,l),d=ut(kt(o)),h=Vs(d);let m=c[h],g=c[d];if(i){const v=h==="y"?"top":"left",y=h==="y"?"bottom":"right",x=m+u[v],b=m-u[y];m=_s(x,m,b)}if(s){const v=d==="y"?"top":"left",y=d==="y"?"bottom":"right",x=g+u[v],b=g-u[y];g=_s(x,g,b)}const p=a.fn({...t,[h]:m,[d]:g});return{...p,data:{x:p.x-n,y:p.y-r,enabled:{[h]:i,[d]:s}}}}}},cb=function(e){return e===void 0&&(e={}),{options:e,fn(t){const{x:n,y:r,placement:o,rects:i,middlewareData:s}=t,{offset:a=0,mainAxis:l=!0,crossAxis:c=!0}=xt(e,t),u={x:n,y:r},d=ut(o),h=Vs(d);let m=u[h],g=u[d];const p=xt(a,t),v=typeof p=="number"?{mainAxis:p,crossAxis:0}:{mainAxis:0,crossAxis:0,...p};if(l){const b=h==="y"?"height":"width",C=i.reference[h]-i.floating[b]+v.mainAxis,k=i.reference[h]+i.reference[b]-v.mainAxis;mk&&(m=k)}if(c){var y,x;const b=h==="y"?"width":"height",C=Vc.has(kt(o)),k=i.reference[d]-i.floating[b]+(C&&((y=s.offset)==null?void 0:y[d])||0)+(C?0:v.crossAxis),E=i.reference[d]+i.reference[b]+(C?0:((x=s.offset)==null?void 0:x[d])||0)-(C?v.crossAxis:0);gE&&(g=E)}return{[h]:m,[d]:g}}}},ub=function(e){return e===void 0&&(e={}),{name:"size",options:e,async fn(t){var n,r;const{placement:o,rects:i,platform:s,elements:a}=t,{apply:l=()=>{},...c}=xt(e,t),u=await Er(t,c),d=kt(o),h=Bn(o),m=ut(o)==="y",{width:g,height:p}=i.floating;let v,y;d==="top"||d==="bottom"?(v=d,y=h===(await(s.isRTL==null?void 0:s.isRTL(a.floating))?"start":"end")?"left":"right"):(y=d,v=h==="end"?"top":"bottom");const x=p-u.top-u.bottom,b=g-u.left-u.right,C=Lt(p-u[v],x),k=Lt(g-u[y],b),E=!t.middlewareData.shift;let I=C,R=k;if((n=t.middlewareData.shift)!=null&&n.enabled.x&&(R=b),(r=t.middlewareData.shift)!=null&&r.enabled.y&&(I=x),E&&!h){const A=Be(u.left,0),T=Be(u.right,0),S=Be(u.top,0),N=Be(u.bottom,0);m?R=g-2*(A!==0||T!==0?A+T:Be(u.left,u.right)):I=p-2*(S!==0||N!==0?S+N:Be(u.top,u.bottom))}await l({...t,availableWidth:R,availableHeight:I});const _=await s.getDimensions(a.floating);return g!==_.width||p!==_.height?{reset:{rects:!0}}:{}}}};function Do(){return typeof window<"u"}function Wn(e){return Fc(e)?(e.nodeName||"").toLowerCase():"#document"}function We(e){var t;return(e==null||(t=e.ownerDocument)==null?void 0:t.defaultView)||window}function dt(e){var t;return(t=(Fc(e)?e.ownerDocument:e.document)||window.document)==null?void 0:t.documentElement}function Fc(e){return Do()?e instanceof Node||e instanceof We(e).Node:!1}function et(e){return Do()?e instanceof Element||e instanceof We(e).Element:!1}function ht(e){return Do()?e instanceof HTMLElement||e instanceof We(e).HTMLElement:!1}function Lc(e){return!Do()||typeof ShadowRoot>"u"?!1:e instanceof ShadowRoot||e instanceof We(e).ShadowRoot}const db=new Set(["inline","contents"]);function Or(e){const{overflow:t,overflowX:n,overflowY:r,display:o}=tt(e);return/auto|scroll|overlay|hidden|clip/.test(t+r+n)&&!db.has(o)}const hb=new Set(["table","td","th"]);function fb(e){return hb.has(Wn(e))}const gb=[":popover-open",":modal"];function zo(e){return gb.some(t=>{try{return e.matches(t)}catch{return!1}})}const pb=["transform","translate","scale","rotate","perspective"],mb=["transform","translate","scale","rotate","perspective","filter"],vb=["paint","layout","strict","content"];function zs(e){const t=Ms(),n=et(e)?tt(e):e;return pb.some(r=>n[r]?n[r]!=="none":!1)||(n.containerType?n.containerType!=="normal":!1)||!t&&(n.backdropFilter?n.backdropFilter!=="none":!1)||!t&&(n.filter?n.filter!=="none":!1)||mb.some(r=>(n.willChange||"").includes(r))||vb.some(r=>(n.contain||"").includes(r))}function bb(e){let t=Dt(e);for(;ht(t)&&!Hn(t);){if(zs(t))return t;if(zo(t))return null;t=Dt(t)}return null}function Ms(){return typeof CSS>"u"||!CSS.supports?!1:CSS.supports("-webkit-backdrop-filter","none")}const yb=new Set(["html","body","#document"]);function Hn(e){return yb.has(Wn(e))}function tt(e){return We(e).getComputedStyle(e)}function Mo(e){return et(e)?{scrollLeft:e.scrollLeft,scrollTop:e.scrollTop}:{scrollLeft:e.scrollX,scrollTop:e.scrollY}}function Dt(e){if(Wn(e)==="html")return e;const t=e.assignedSlot||e.parentNode||Lc(e)&&e.host||dt(e);return Lc(t)?t.host:t}function Dc(e){const t=Dt(e);return Hn(t)?e.ownerDocument?e.ownerDocument.body:e.body:ht(t)&&Or(t)?t:Dc(t)}function Pr(e,t,n){var r;t===void 0&&(t=[]),n===void 0&&(n=!0);const o=Dc(e),i=o===((r=e.ownerDocument)==null?void 0:r.body),s=We(o);if(i){const a=js(s);return t.concat(s,s.visualViewport||[],Or(o)?o:[],a&&n?Pr(a):[])}return t.concat(o,Pr(o,[],n))}function js(e){return e.parent&&Object.getPrototypeOf(e.parent)?e.frameElement:null}function zc(e){const t=tt(e);let n=parseFloat(t.width)||0,r=parseFloat(t.height)||0;const o=ht(e),i=o?e.offsetWidth:n,s=o?e.offsetHeight:r,a=_o(n)!==i||_o(r)!==s;return a&&(n=i,r=s),{width:n,height:r,$:a}}function $s(e){return et(e)?e:e.contextElement}function Un(e){const t=$s(e);if(!ht(t))return ct(1);const n=t.getBoundingClientRect(),{width:r,height:o,$:i}=zc(t);let s=(i?_o(n.width):n.width)/r,a=(i?_o(n.height):n.height)/o;return(!s||!Number.isFinite(s))&&(s=1),(!a||!Number.isFinite(a))&&(a=1),{x:s,y:a}}const xb=ct(0);function Mc(e){const t=We(e);return!Ms()||!t.visualViewport?xb:{x:t.visualViewport.offsetLeft,y:t.visualViewport.offsetTop}}function kb(e,t,n){return t===void 0&&(t=!1),!n||t&&n!==We(e)?!1:t}function Zt(e,t,n,r){t===void 0&&(t=!1),n===void 0&&(n=!1);const o=e.getBoundingClientRect(),i=$s(e);let s=ct(1);t&&(r?et(r)&&(s=Un(r)):s=Un(e));const a=kb(i,n,r)?Mc(i):ct(0);let l=(o.left+a.x)/s.x,c=(o.top+a.y)/s.y,u=o.width/s.x,d=o.height/s.y;if(i){const h=We(i),m=r&&et(r)?We(r):r;let g=h,p=js(g);for(;p&&r&&m!==g;){const v=Un(p),y=p.getBoundingClientRect(),x=tt(p),b=y.left+(p.clientLeft+parseFloat(x.paddingLeft))*v.x,C=y.top+(p.clientTop+parseFloat(x.paddingTop))*v.y;l*=v.x,c*=v.y,u*=v.x,d*=v.y,l+=b,c+=C,g=We(p),p=js(g)}}return Lo({width:u,height:d,x:l,y:c})}function jo(e,t){const n=Mo(e).scrollLeft;return t?t.left+n:Zt(dt(e)).left+n}function jc(e,t){const n=e.getBoundingClientRect(),r=n.left+t.scrollLeft-jo(e,n),o=n.top+t.scrollTop;return{x:r,y:o}}function Cb(e){let{elements:t,rect:n,offsetParent:r,strategy:o}=e;const i=o==="fixed",s=dt(r),a=t?zo(t.floating):!1;if(r===s||a&&i)return n;let l={scrollLeft:0,scrollTop:0},c=ct(1);const u=ct(0),d=ht(r);if((d||!d&&!i)&&((Wn(r)!=="body"||Or(s))&&(l=Mo(r)),ht(r))){const m=Zt(r);c=Un(r),u.x=m.x+r.clientLeft,u.y=m.y+r.clientTop}const h=s&&!d&&!i?jc(s,l):ct(0);return{width:n.width*c.x,height:n.height*c.y,x:n.x*c.x-l.scrollLeft*c.x+u.x+h.x,y:n.y*c.y-l.scrollTop*c.y+u.y+h.y}}function Sb(e){return Array.from(e.getClientRects())}function wb(e){const t=dt(e),n=Mo(e),r=e.ownerDocument.body,o=Be(t.scrollWidth,t.clientWidth,r.scrollWidth,r.clientWidth),i=Be(t.scrollHeight,t.clientHeight,r.scrollHeight,r.clientHeight);let s=-n.scrollLeft+jo(e);const a=-n.scrollTop;return tt(r).direction==="rtl"&&(s+=Be(t.clientWidth,r.clientWidth)-o),{width:o,height:i,x:s,y:a}}const $c=25;function Eb(e,t){const n=We(e),r=dt(e),o=n.visualViewport;let i=r.clientWidth,s=r.clientHeight,a=0,l=0;if(o){i=o.width,s=o.height;const u=Ms();(!u||u&&t==="fixed")&&(a=o.offsetLeft,l=o.offsetTop)}const c=jo(r);if(c<=0){const u=r.ownerDocument,d=u.body,h=getComputedStyle(d),m=u.compatMode==="CSS1Compat"&&parseFloat(h.marginLeft)+parseFloat(h.marginRight)||0,g=Math.abs(r.clientWidth-d.clientWidth-m);g<=$c&&(i-=g)}else c<=$c&&(i+=c);return{width:i,height:s,x:a,y:l}}const Ob=new Set(["absolute","fixed"]);function Pb(e,t){const n=Zt(e,!0,t==="fixed"),r=n.top+e.clientTop,o=n.left+e.clientLeft,i=ht(e)?Un(e):ct(1),s=e.clientWidth*i.x,a=e.clientHeight*i.y,l=o*i.x,c=r*i.y;return{width:s,height:a,x:l,y:c}}function Bc(e,t,n){let r;if(t==="viewport")r=Eb(e,n);else if(t==="document")r=wb(dt(e));else if(et(t))r=Pb(t,n);else{const o=Mc(e);r={x:t.x-o.x,y:t.y-o.y,width:t.width,height:t.height}}return Lo(r)}function Wc(e,t){const n=Dt(e);return n===t||!et(n)||Hn(n)?!1:tt(n).position==="fixed"||Wc(n,t)}function Ib(e,t){const n=t.get(e);if(n)return n;let r=Pr(e,[],!1).filter(a=>et(a)&&Wn(a)!=="body"),o=null;const i=tt(e).position==="fixed";let s=i?Dt(e):e;for(;et(s)&&!Hn(s);){const a=tt(s),l=zs(s);!l&&a.position==="fixed"&&(o=null),(i?!l&&!o:!l&&a.position==="static"&&!!o&&Ob.has(o.position)||Or(s)&&!l&&Wc(e,s))?r=r.filter(u=>u!==s):o=a,s=Dt(s)}return t.set(e,r),r}function Rb(e){let{element:t,boundary:n,rootBoundary:r,strategy:o}=e;const s=[...n==="clippingAncestors"?zo(t)?[]:Ib(t,this._c):[].concat(n),r],a=s[0],l=s.reduce((c,u)=>{const d=Bc(t,u,o);return c.top=Be(d.top,c.top),c.right=Lt(d.right,c.right),c.bottom=Lt(d.bottom,c.bottom),c.left=Be(d.left,c.left),c},Bc(t,a,o));return{width:l.right-l.left,height:l.bottom-l.top,x:l.left,y:l.top}}function Tb(e){const{width:t,height:n}=zc(e);return{width:t,height:n}}function Nb(e,t,n){const r=ht(t),o=dt(t),i=n==="fixed",s=Zt(e,!0,i,t);let a={scrollLeft:0,scrollTop:0};const l=ct(0);function c(){l.x=jo(o)}if(r||!r&&!i)if((Wn(t)!=="body"||Or(o))&&(a=Mo(t)),r){const m=Zt(t,!0,i,t);l.x=m.x+t.clientLeft,l.y=m.y+t.clientTop}else o&&c();i&&!r&&o&&c();const u=o&&!r&&!i?jc(o,a):ct(0),d=s.left+a.scrollLeft-l.x-u.x,h=s.top+a.scrollTop-l.y-u.y;return{x:d,y:h,width:s.width,height:s.height}}function Bs(e){return tt(e).position==="static"}function Hc(e,t){if(!ht(e)||tt(e).position==="fixed")return null;if(t)return t(e);let n=e.offsetParent;return dt(e)===n&&(n=n.ownerDocument.body),n}function Uc(e,t){const n=We(e);if(zo(e))return n;if(!ht(e)){let o=Dt(e);for(;o&&!Hn(o);){if(et(o)&&!Bs(o))return o;o=Dt(o)}return n}let r=Hc(e,t);for(;r&&fb(r)&&Bs(r);)r=Hc(r,t);return r&&Hn(r)&&Bs(r)&&!zs(r)?n:r||bb(e)||n}const Ab=async function(e){const t=this.getOffsetParent||Uc,n=this.getDimensions,r=await n(e.floating);return{reference:Nb(e.reference,await t(e.floating),e.strategy),floating:{x:0,y:0,width:r.width,height:r.height}}};function _b(e){return tt(e).direction==="rtl"}const Vb={convertOffsetParentRelativeRectToViewportRelativeRect:Cb,getDocumentElement:dt,getClippingRect:Rb,getOffsetParent:Uc,getElementRects:Ab,getClientRects:Sb,getDimensions:Tb,getScale:Un,isElement:et,isRTL:_b};function Gc(e,t){return e.x===t.x&&e.y===t.y&&e.width===t.width&&e.height===t.height}function Fb(e,t){let n=null,r;const o=dt(e);function i(){var a;clearTimeout(r),(a=n)==null||a.disconnect(),n=null}function s(a,l){a===void 0&&(a=!1),l===void 0&&(l=1),i();const c=e.getBoundingClientRect(),{left:u,top:d,width:h,height:m}=c;if(a||t(),!h||!m)return;const g=Vo(d),p=Vo(o.clientWidth-(u+h)),v=Vo(o.clientHeight-(d+m)),y=Vo(u),b={rootMargin:-g+"px "+-p+"px "+-v+"px "+-y+"px",threshold:Be(0,Lt(1,l))||1};let C=!0;function k(E){const I=E[0].intersectionRatio;if(I!==l){if(!C)return s();I?s(!1,I):r=setTimeout(()=>{s(!1,1e-7)},1e3)}I===1&&!Gc(c,e.getBoundingClientRect())&&s(),C=!1}try{n=new IntersectionObserver(k,{...b,root:o.ownerDocument})}catch{n=new IntersectionObserver(k,b)}n.observe(e)}return s(!0),i}function Lb(e,t,n,r){r===void 0&&(r={});const{ancestorScroll:o=!0,ancestorResize:i=!0,elementResize:s=typeof ResizeObserver=="function",layoutShift:a=typeof IntersectionObserver=="function",animationFrame:l=!1}=r,c=$s(e),u=o||i?[...c?Pr(c):[],...Pr(t)]:[];u.forEach(y=>{o&&y.addEventListener("scroll",n,{passive:!0}),i&&y.addEventListener("resize",n)});const d=c&&a?Fb(c,n):null;let h=-1,m=null;s&&(m=new ResizeObserver(y=>{let[x]=y;x&&x.target===c&&m&&(m.unobserve(t),cancelAnimationFrame(h),h=requestAnimationFrame(()=>{var b;(b=m)==null||b.observe(t)})),n()}),c&&!l&&m.observe(c),m.observe(t));let g,p=l?Zt(e):null;l&&v();function v(){const y=Zt(e);p&&!Gc(p,y)&&n(),p=y,g=requestAnimationFrame(v)}return n(),()=>{var y;u.forEach(x=>{o&&x.removeEventListener("scroll",n),i&&x.removeEventListener("resize",n)}),d?.(),(y=m)==null||y.disconnect(),m=null,l&&cancelAnimationFrame(g)}}const Db=ab,zb=lb,Mb=ob,jb=ub,$b=ib,Bb=rb,Wb=cb,Hb=(e,t,n)=>{const r=new Map,o={platform:Vb,...n},i={...o.platform,_c:r};return nb(e,t,{...o,platform:i})};function qc(e=0,t=0,n=0,r=0){if(typeof DOMRect=="function")return new DOMRect(e,t,n,r);const o={x:e,y:t,width:n,height:r,top:t,right:e+n,bottom:t+r,left:e};return{...o,toJSON:()=>o}}function Ub(e){if(!e)return qc();const{x:t,y:n,width:r,height:o}=e;return qc(t,n,r,o)}function Gb(e,t){return{contextElement:Te(e)?e:void 0,getBoundingClientRect:()=>{const n=e,r=t?.(n);return r||!n?Ub(r):n.getBoundingClientRect()}}}var Kc=e=>({variable:e,reference:`var(${e})`}),Yc={transformOrigin:Kc("--transform-origin"),arrowOffset:Kc("--arrow-offset")},qb=e=>e==="top"||e==="bottom"?"y":"x";function Kb(e,t){return{name:"transformOrigin",fn(n){const{elements:r,middlewareData:o,placement:i,rects:s,y:a}=n,l=i.split("-")[0],c=qb(l),u=o.arrow?.x||0,d=o.arrow?.y||0,h=t?.clientWidth||0,m=t?.clientHeight||0,g=u+h/2,p=d+m/2,v=Math.abs(o.shift?.y||0),y=s.reference.height/2,x=m/2,b=e.offset?.mainAxis??e.gutter,C=typeof b=="number"?b+x:b??x,k=v>C,E={top:`${g}px calc(100% + ${C}px)`,bottom:`${g}px ${-C}px`,left:`calc(100% + ${C}px) ${p}px`,right:`${-C}px ${p}px`}[l],I=`${g}px ${s.reference.y+y-a}px`,R=!!e.overlap&&c==="y"&&k;return r.floating.style.setProperty(Yc.transformOrigin.variable,R?I:E),{data:{transformOrigin:R?I:E}}}}}var Yb={name:"rects",fn({rects:e}){return{data:e}}},Xb=e=>{if(e)return{name:"shiftArrow",fn({placement:t,middlewareData:n}){if(!n.arrow)return{};const{x:r,y:o}=n.arrow,i=t.split("-")[0];return Object.assign(e.style,{left:r!=null?`${r}px`:"",top:o!=null?`${o}px`:"",[i]:`calc(100% + ${Yc.arrowOffset.reference})`}),{}}}};function Qb(e){const[t,n]=e.split("-");return{side:t,align:n,hasAlign:n!=null}}function Jb(e){return e.split("-")[0]}var Zb={strategy:"absolute",placement:"bottom",listeners:!0,gutter:8,flip:!0,slide:!0,overlap:!1,sameWidth:!1,fitViewport:!1,overflowPadding:8,arrowPadding:4};function Xc(e,t){const n=e.devicePixelRatio||1;return Math.round(t*n)/n}function Ws(e){return typeof e=="function"?e():e==="clipping-ancestors"?"clippingAncestors":e}function ey(e,t,n){const r=e||t.createElement("div");return Bb({element:r,padding:n.arrowPadding})}function ty(e,t){if(!jm(t.offset??t.gutter))return Db(({placement:n})=>{const r=(e?.clientHeight||0)/2,o=t.offset?.mainAxis??t.gutter,i=typeof o=="number"?o+r:o??r,{hasAlign:s}=Qb(n),a=s?void 0:t.shift,l=t.offset?.crossAxis??a;return po({crossAxis:l,mainAxis:i,alignmentAxis:t.shift})})}function ny(e){if(!e.flip)return;const t=Ws(e.boundary);return Mb({...t?{boundary:t}:void 0,padding:e.overflowPadding,fallbackPlacements:e.flip===!0?void 0:e.flip})}function ry(e){if(!e.slide&&!e.overlap)return;const t=Ws(e.boundary);return zb({...t?{boundary:t}:void 0,mainAxis:e.slide,crossAxis:e.overlap,padding:e.overflowPadding,limiter:Wb()})}function oy(e){return jb({padding:e.overflowPadding,apply({elements:t,rects:n,availableHeight:r,availableWidth:o}){const i=t.floating,s=Math.round(n.reference.width),a=Math.round(n.reference.height);o=Math.floor(o),r=Math.floor(r),i.style.setProperty("--reference-width",`${s}px`),i.style.setProperty("--reference-height",`${a}px`),i.style.setProperty("--available-width",`${o}px`),i.style.setProperty("--available-height",`${r}px`)}})}function iy(e){if(e.hideWhenDetached)return $b({strategy:"referenceHidden",boundary:Ws(e.boundary)??"clippingAncestors"})}function sy(e){return e?e===!0?{ancestorResize:!0,ancestorScroll:!0,elementResize:!0,layoutShift:!0}:e:{}}function ay(e,t,n={}){const r=Gb(e,n.getAnchorRect);if(!t||!r)return;const o=Object.assign({},Zb,n),i=t.querySelector("[data-part=arrow]"),s=[ty(i,o),ny(o),ry(o),ey(i,t.ownerDocument,o),Xb(i),Kb({gutter:o.gutter,offset:o.offset,overlap:o.overlap},i),oy(o),iy(o),Yb],{placement:a,strategy:l,onComplete:c,onPositioned:u}=o,d=async()=>{if(!r||!t)return;const p=await Hb(r,t,{placement:a,middleware:s,strategy:l});c?.(p),u?.({placed:!0});const v=be(t),y=Xc(v,p.x),x=Xc(v,p.y);t.style.setProperty("--x",`${y}px`),t.style.setProperty("--y",`${x}px`),o.hideWhenDetached&&(p.middlewareData.hide?.referenceHidden?(t.style.setProperty("visibility","hidden"),t.style.setProperty("pointer-events","none")):(t.style.removeProperty("visibility"),t.style.removeProperty("pointer-events")));const b=t.firstElementChild;if(b){const C=vo(b);t.style.setProperty("--z-index",C.zIndex)}},h=async()=>{n.updatePosition?(await n.updatePosition({updatePosition:d,floatingElement:t}),u?.({placed:!0})):await d()},m=sy(o.listeners),g=o.listeners?Lb(r,t,h,m):Km;return h(),()=>{g?.(),u?.({placed:!1})}}function nt(e,t,n={}){const{defer:r,...o}=n,i=r?G:a=>a(),s=[];return s.push(i(()=>{const a=typeof e=="function"?e():e,l=typeof t=="function"?t():t;s.push(ay(a,l,o))})),()=>{s.forEach(a=>a?.())}}function ly(e){const t={each(n){for(let r=0;r{try{i.document.addEventListener(n,r,o)}catch{}}),()=>{try{t.removeEventListener(n,r,o)}catch{}}},removeEventListener(n,r,o){t.each(i=>{try{i.document.removeEventListener(n,r,o)}catch{}})}};return t}function cy(e){const t=e.frameElement!=null?e.parent:null;return{addEventListener:(n,r,o)=>{try{t?.addEventListener(n,r,o)}catch{}return()=>{try{t?.removeEventListener(n,r,o)}catch{}}},removeEventListener:(n,r,o)=>{try{t?.removeEventListener(n,r,o)}catch{}}}}var Qc="pointerdown.outside",Jc="focus.outside";function uy(e){for(const t of e)if(Te(t)&&Vt(t))return!0;return!1}var Zc=e=>"clientY"in e;function dy(e,t){if(!Zc(t)||!e)return!1;const n=e.getBoundingClientRect();return n.width===0||n.height===0?!1:n.top<=t.clientY&&t.clientY<=n.top+n.height&&n.left<=t.clientX&&t.clientX<=n.left+n.width}function hy(e,t){return e.y<=t.y&&t.y<=e.y+e.height&&e.x<=t.x&&t.x<=e.x+e.width}function eu(e,t){if(!t||!Zc(e))return!1;const n=t.scrollHeight>t.clientHeight,r=n&&e.clientX>t.offsetLeft+t.clientWidth,o=t.scrollWidth>t.clientWidth,i=o&&e.clientY>t.offsetTop+t.clientHeight,s={x:t.offsetLeft,y:t.offsetTop,width:t.clientWidth+(n?16:0),height:t.clientHeight+(o?16:0)},a={x:e.clientX,y:e.clientY};return hy(s,a)?r||i:!1}function fy(e,t){const{exclude:n,onFocusOutside:r,onPointerDownOutside:o,onInteractOutside:i,defer:s}=t;if(!e)return;const a=Fe(e),l=be(e),c=ly(l),u=cy(l);function d(x,b){if(!Te(b)||!b.isConnected||Xt(e,b)||dy(e,x))return!1;const C=a.querySelector(`[aria-controls="${e.id}"]`);if(C){const E=So(C);if(eu(x,E))return!1}const k=So(e);return eu(x,k)?!1:!n?.(b)}const h=new Set,m=zn(e?.getRootNode());function g(x){function b(C){const k=s&&!rc()?G:R=>R(),E=C??x,I=E?.composedPath?.()??[E?.target];k(()=>{const R=m?I[0]:$e(x);if(!(!e||!d(x,R))){if(o||i){const _=fo(o,i);e.addEventListener(Qc,_,{once:!0})}tu(e,Qc,{bubbles:!1,cancelable:!0,detail:{originalEvent:E,contextmenu:z0(E),focusable:uy(I),target:R}})}})}x.pointerType==="touch"?(h.forEach(C=>C()),h.add(se(a,"click",b,{once:!0})),h.add(u.addEventListener("click",b,{once:!0})),h.add(c.addEventListener("click",b,{once:!0}))):b()}const p=new Set,v=setTimeout(()=>{p.add(se(a,"pointerdown",g,!0)),p.add(u.addEventListener("pointerdown",g,!0)),p.add(c.addEventListener("pointerdown",g,!0))},0);function y(x){(s?G:C=>C())(()=>{const C=$e(x);if(!(!e||!d(x,C))){if(r||i){const k=fo(r,i);e.addEventListener(Jc,k,{once:!0})}tu(e,Jc,{bubbles:!1,cancelable:!0,detail:{originalEvent:x,contextmenu:!1,focusable:Vt(C),target:C}})}})}return rc()||(p.add(se(a,"focusin",y,!0)),p.add(u.addEventListener("focusin",y,!0)),p.add(c.addEventListener("focusin",y,!0))),()=>{clearTimeout(v),h.forEach(x=>x()),p.forEach(x=>x())}}function gy(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(fy(i,t))})),()=>{o.forEach(i=>i?.())}}function tu(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,n);return e.dispatchEvent(o)}function py(e,t){const n=r=>{r.key==="Escape"&&(r.isComposing||t?.(r))};return se(Fe(e),"keydown",n,{capture:!0})}var nu="layer:request-dismiss",Ke={layers:[],branches:[],count(){return this.layers.length},pointerBlockingLayers(){return this.layers.filter(e=>e.pointerBlocking)},topMostPointerBlockingLayer(){return[...this.pointerBlockingLayers()].slice(-1)[0]},hasPointerBlockingLayer(){return this.pointerBlockingLayers().length>0},isBelowPointerBlockingLayer(e){const t=this.indexOf(e),n=this.topMostPointerBlockingLayer()?this.indexOf(this.topMostPointerBlockingLayer()?.node):-1;return tt.type===e)},getNestedLayersByType(e,t){const n=this.indexOf(e);return n===-1?[]:this.layers.slice(n+1).filter(r=>r.type===t)},getParentLayerOfType(e,t){const n=this.indexOf(e);if(!(n<=0))return this.layers.slice(0,n).reverse().find(r=>r.type===t)},countNestedLayersOfType(e,t){return this.getNestedLayersByType(e,t).length},isInNestedLayer(e,t){return this.getNestedLayers(e).some(n=>Xt(n.node,t))},isInBranch(e){return Array.from(this.branches).some(t=>Xt(t,e))},add(e){this.layers.push(e),this.syncLayers()},addBranch(e){this.branches.push(e)},remove(e){const t=this.indexOf(e);t<0||(tKe.dismiss(r.node,e)),this.layers.splice(t,1),this.syncLayers())},removeBranch(e){const t=this.branches.indexOf(e);t>=0&&this.branches.splice(t,1)},syncLayers(){this.layers.forEach((e,t)=>{e.node.style.setProperty("--layer-index",`${t}`),e.node.removeAttribute("data-nested"),e.node.removeAttribute("data-has-nested"),this.getParentLayerOfType(e.node,e.type)&&e.node.setAttribute("data-nested",e.type);const r=this.countNestedLayersOfType(e.node,e.type);r>0&&e.node.setAttribute("data-has-nested",e.type),e.node.style.setProperty("--nested-layer-count",`${r}`)})},indexOf(e){return this.layers.findIndex(t=>t.node===e)},dismiss(e,t){const n=this.indexOf(e);if(n===-1)return;const r=this.layers[n];vy(e,nu,o=>{r.requestDismiss?.(o),o.defaultPrevented||r?.dismiss()}),my(e,nu,{originalLayer:e,targetLayer:t,originalIndex:n,targetIndex:t?this.indexOf(t):-1}),this.syncLayers()},clear(){this.remove(this.layers[0].node)}};function my(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,{cancelable:!0,bubbles:!0,detail:n});return e.dispatchEvent(o)}function vy(e,t,n){e.addEventListener(t,n,{once:!0})}var ru;function ou(){Ke.layers.forEach(({node:e})=>{e.style.pointerEvents=Ke.isBelowPointerBlockingLayer(e)?"none":"auto"})}function by(e){e.style.pointerEvents=""}function yy(e,t){const n=Fe(e),r=[];return Ke.hasPointerBlockingLayer()&&!n.body.hasAttribute("data-inert")&&(ru=document.body.style.pointerEvents,queueMicrotask(()=>{n.body.style.pointerEvents="none",n.body.setAttribute("data-inert","")})),t?.forEach(o=>{const[i,s]=vv(()=>{const a=o();return Te(a)?a:null},{timeout:1e3});i.then(a=>r.push(Po(a,{pointerEvents:"auto"}))),r.push(s)}),()=>{Ke.hasPointerBlockingLayer()||(queueMicrotask(()=>{n.body.style.pointerEvents=ru,n.body.removeAttribute("data-inert"),n.body.style.length===0&&n.body.removeAttribute("style")}),r.forEach(o=>o()))}}function xy(e,t){const{warnOnMissingNode:n=!0}=t;if(n&&!e){kr("[@zag-js/dismissable] node is `null` or `undefined`");return}if(!e)return;const{onDismiss:r,onRequestDismiss:o,pointerBlocking:i,exclude:s,debug:a,type:l="dialog"}=t,c={dismiss:r,node:e,type:l,pointerBlocking:i,requestDismiss:o};Ke.add(c),ou();function u(p){const v=$e(p.detail.originalEvent);Ke.isBelowPointerBlockingLayer(e)||Ke.isInBranch(v)||(t.onPointerDownOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onPointerDownOutside:",p.detail.originalEvent),r?.()))}function d(p){const v=$e(p.detail.originalEvent);Ke.isInBranch(v)||(t.onFocusOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onFocusOutside:",p.detail.originalEvent),r?.()))}function h(p){Ke.isTopMost(e)&&(t.onEscapeKeyDown?.(p),!p.defaultPrevented&&r&&(p.preventDefault(),r()))}function m(p){if(!e)return!1;const v=typeof s=="function"?s():s,y=Array.isArray(v)?v:[v],x=t.persistentElements?.map(b=>b()).filter(Te);return x&&y.push(...x),y.some(b=>Xt(b,p))||Ke.isInNestedLayer(e,p)}const g=[i?yy(e,t.persistentElements):void 0,py(e,h),gy(e,{exclude:m,onFocusOutside:d,onPointerDownOutside:u,defer:t.defer})];return()=>{Ke.remove(e),ou(),by(e),g.forEach(p=>p?.())}}function Gn(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=Yt(e)?e():e;o.push(xy(i,t))})),()=>{o.forEach(i=>i?.())}}var iu=j("color-picker",["root","label","control","trigger","positioner","content","area","areaThumb","valueText","areaBackground","channelSlider","channelSliderLabel","channelSliderTrack","channelSliderThumb","channelSliderValueText","channelInput","transparencyGrid","swatchGroup","swatchTrigger","swatchIndicator","swatch","eyeDropperTrigger","formatTrigger","formatSelect"]);iu.build();var ky=e=>e.ids?.hiddenInput??`color-picker:${e.id}:hidden-input`,Cy=e=>e.ids?.control??`color-picker:${e.id}:control`,Sy=e=>e.ids?.trigger??`color-picker:${e.id}:trigger`,wy=e=>e.ids?.content??`color-picker:${e.id}:content`,Ey=e=>e.ids?.positioner??`color-picker:${e.id}:positioner`,Oy=e=>e.ids?.formatSelect??`color-picker:${e.id}:format-select`,Py=e=>e.ids?.area??`color-picker:${e.id}:area`,Iy=e=>e.ids?.areaThumb??`color-picker:${e.id}:area-thumb`,Ry=(e,t)=>e.ids?.channelSliderTrack?.(t)??`color-picker:${e.id}:slider-track:${t}`,Ty=(e,t)=>e.ids?.channelSliderThumb?.(t)??`color-picker:${e.id}:slider-thumb:${t}`,$o=e=>e.getById(wy(e)),Ny=e=>e.getById(Iy(e)),Ay=(e,t)=>e.getById(Ty(e,t)),_y=e=>e.getById(Oy(e)),su=e=>e.getById(ky(e)),Vy=e=>e.getById(Py(e)),Fy=(e,t,n)=>{const r=Vy(e);if(!r)return;const{getPercentValue:o}=uc(t,r);return{x:o({dir:n,orientation:"horizontal"}),y:o({orientation:"vertical"})}},Ly=e=>e.getById(Cy(e)),Hs=e=>e.getById(Sy(e)),Dy=e=>e.getById(Ey(e)),zy=(e,t)=>e.getById(Ry(e,t)),My=(e,t,n,r)=>{const o=zy(e,n);if(!o)return;const{getPercentValue:i}=uc(t,o);return{x:i({dir:r,orientation:"horizontal"}),y:i({orientation:"vertical"})}},jy=e=>[...Oo($o(e),"input[data-channel]"),...Oo(Ly(e),"input[data-channel]")];function $y(e,t){if(t==null)return"";if(t==="hex")return e.toString("hex");if(t==="css")return e.toString("css");if(t in e)return e.getChannelValue(t).toString();const n=e.getFormat()==="hsla";switch(t){case"hue":return n?e.toFormat("hsla").getChannelValue("hue").toString():e.toFormat("hsba").getChannelValue("hue").toString();case"saturation":return n?e.toFormat("hsla").getChannelValue("saturation").toString():e.toFormat("hsba").getChannelValue("saturation").toString();case"lightness":return e.toFormat("hsla").getChannelValue("lightness").toString();case"brightness":return e.toFormat("hsba").getChannelValue("brightness").toString();case"red":case"green":case"blue":return e.toFormat("rgba").getChannelValue(t).toString();default:return e.getChannelValue(t).toString()}}var au=e=>Ao(e),By=/^[0-9a-fA-F]{3,8}$/;function Wy(e){return By.test(e)}function Hy(e){return e.startsWith("#")?e:Wy(e)?`#${e}`:e}var{and:Uy}=Ft();Uy("isOpenControlled","closeOnSelect");function lu(e,t,n){const r=jy(e);G(()=>{r.forEach(o=>{const i=o.dataset.channel;ko(o,$y(n||t,i))})})}function Gy(e,t){const n=_y(e);n&&G(()=>ko(n,t))}M()(["closeOnSelect","dir","disabled","format","defaultFormat","getRootNode","id","ids","initialFocusEl","inline","name","positioning","onFocusOutside","onFormatChange","onInteractOutside","onOpenChange","onPointerDownOutside","onValueChange","onValueChangeEnd","defaultOpen","open","positioning","required","readOnly","value","defaultValue","invalid","openAutoFocus"]),M()(["xChannel","yChannel"]),M()(["channel","orientation"]),M()(["value","disabled"]),M()(["value","respectAlpha"]),M()(["size"]);const[cu,qy]=Dn({name:"RenderStrategyContext",hookName:"useRenderStrategyContext",providerName:""}),uu=e=>pc()(e,["lazyMount","unmountOnExit"]);var du=j("accordion").parts("root","item","itemTrigger","itemContent","itemIndicator");du.build();var hu=e=>e.ids?.root??`accordion:${e.id}`,fu=(e,t)=>e.ids?.itemTrigger?.(t)??`accordion:${e.id}:trigger:${t}`,Ky=e=>e.getById(hu(e)),Bo=e=>{const n=`[aria-controls][data-ownedby='${CSS.escape(hu(e))}']:not([disabled])`;return Oo(Ky(e),n)},Yy=e=>yr(Bo(e)),Xy=e=>ls(Bo(e)),Qy=(e,t)=>av(Bo(e),fu(e,t)),Jy=(e,t)=>lv(Bo(e),fu(e,t)),{and:Zy,not:ex}=Ft();Zy("isExpanded","canToggle"),ex("isExpanded"),M()(["collapsible","dir","disabled","getRootNode","id","ids","multiple","onFocusChange","onValueChange","orientation","value","defaultValue"]),M()(["value","disabled"]);var Ir=(e,t)=>({x:e,y:t});function tx(e){const{x:t,y:n,width:r,height:o}=e,i=t+r/2,s=n+o/2;return{x:t,y:n,width:r,height:o,minX:t,minY:n,maxX:t+r,maxY:n+o,midX:i,midY:s,center:Ir(i,s)}}function nx(e){const t=Ir(e.minX,e.minY),n=Ir(e.maxX,e.minY),r=Ir(e.maxX,e.maxY),o=Ir(e.minX,e.maxY);return{top:t,right:n,bottom:r,left:o}}function rx(e,t){const n=tx(e),{top:r,right:o,left:i,bottom:s}=nx(n),[a]=t.split("-");return{top:[i,r,o,s],right:[r,o,s,i],bottom:[r,i,s,o],left:[o,r,i,s]}[a]}function ox(e,t){const{x:n,y:r}=t;let o=!1;for(let i=0,s=e.length-1;ir!=u>r&&n<(c-a)*(r-l)/(u-l)+a&&(o=!o)}return o}var gu=j("avatar").parts("root","image","fallback");gu.build(),M()(["dir","id","ids","onStatusChange","getRootNode"]);function ix(e){return!(e.metaKey||!xo()&&e.altKey||e.ctrlKey||e.key==="Control"||e.key==="Shift"||e.key==="Meta")}var sx=new Set(["checkbox","radio","range","color","file","image","button","submit","reset"]);function ax(e,t,n){const r=n?$e(n):null,o=be(r);return e=e||r instanceof o.HTMLInputElement&&!sx.has(r?.type)||r instanceof o.HTMLTextAreaElement||r instanceof o.HTMLElement&&r.isContentEditable,!(e&&t==="keyboard"&&n instanceof o.KeyboardEvent&&!Reflect.has(lx,n.key))}var en=null,Us=new Set,Rr=new Map,tn=!1,Gs=!1,lx={Tab:!0,Escape:!0};function Wo(e,t){for(let n of Us)n(e,t)}function Ho(e){tn=!0,ix(e)&&(en="keyboard",Wo("keyboard",e))}function Ye(e){en="pointer",(e.type==="mousedown"||e.type==="pointerdown")&&(tn=!0,Wo("pointer",e))}function pu(e){D0(e)&&(tn=!0,en="virtual")}function mu(e){const t=$e(e);t===be(t)||t===Fe(t)||(!tn&&!Gs&&(en="virtual",Wo("virtual",e)),tn=!1,Gs=!1)}function vu(){tn=!1,Gs=!0}function cx(e){if(typeof window>"u"||Rr.get(be(e)))return;const t=be(e),n=Fe(e);let r=t.HTMLElement.prototype.focus;function o(){en="virtual",Wo("virtual",null),tn=!0,r.apply(this,arguments)}Object.defineProperty(t.HTMLElement.prototype,"focus",{configurable:!0,value:o}),n.addEventListener("keydown",Ho,!0),n.addEventListener("keyup",Ho,!0),n.addEventListener("click",pu,!0),t.addEventListener("focus",mu,!0),t.addEventListener("blur",vu,!1),typeof t.PointerEvent<"u"?(n.addEventListener("pointerdown",Ye,!0),n.addEventListener("pointermove",Ye,!0),n.addEventListener("pointerup",Ye,!0)):(n.addEventListener("mousedown",Ye,!0),n.addEventListener("mousemove",Ye,!0),n.addEventListener("mouseup",Ye,!0)),t.addEventListener("beforeunload",()=>{ux(e)},{once:!0}),Rr.set(t,{focus:r})}var ux=(e,t)=>{const n=be(e),r=Fe(e);Rr.has(n)&&(n.HTMLElement.prototype.focus=Rr.get(n).focus,r.removeEventListener("keydown",Ho,!0),r.removeEventListener("keyup",Ho,!0),r.removeEventListener("click",pu,!0),n.removeEventListener("focus",mu,!0),n.removeEventListener("blur",vu,!1),typeof n.PointerEvent<"u"?(r.removeEventListener("pointerdown",Ye,!0),r.removeEventListener("pointermove",Ye,!0),r.removeEventListener("pointerup",Ye,!0)):(r.removeEventListener("mousedown",Ye,!0),r.removeEventListener("mousemove",Ye,!0),r.removeEventListener("mouseup",Ye,!0)),Rr.delete(n))};function bu(){return en==="keyboard"}function dx(e={}){const{isTextInput:t,autoFocus:n,onChange:r,root:o}=e;cx(o),r?.({isFocusVisible:n||bu(),modality:en});const i=(s,a)=>{ax(!!t,s,a)&&r?.({isFocusVisible:bu(),modality:s})};return Us.add(i),()=>{Us.delete(i)}}var yu=j("checkbox").parts("root","label","control","indicator");yu.build(),M()(["defaultChecked","checked","dir","disabled","form","getRootNode","id","ids","invalid","name","onCheckedChange","readOnly","required","value"]);const hx=yu.extendWith("group"),[QR,xu]=Dn({name:"FieldContext",hookName:"useFieldContext",providerName:"",strict:!1});var ku=j("clipboard").parts("root","control","trigger","indicator","input","label");ku.build(),M()(["getRootNode","id","ids","value","defaultValue","timeout","onStatusChange","onValueChange"]),M()(["copied"]);const fx=iu.extendWith("view");var gx=Object.defineProperty,px=(e,t,n)=>t in e?gx(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,V=(e,t,n)=>px(e,typeof t!="symbol"?t+"":t,n),Uo={itemToValue(e){return typeof e=="string"?e:Kt(e)&&At(e,"value")?e.value:""},itemToString(e){return typeof e=="string"?e:Kt(e)&&At(e,"label")?e.label:Uo.itemToValue(e)},isItemDisabled(e){return Kt(e)&&At(e,"disabled")?!!e.disabled:!1}},Go=class dp{constructor(t){this.options=t,V(this,"items"),V(this,"indexMap",null),V(this,"copy",n=>new dp({...this.options,items:n??[...this.items]})),V(this,"isEqual",n=>Ze(this.items,n.items)),V(this,"setItems",n=>this.copy(n)),V(this,"getValues",(n=this.items)=>{const r=[];for(const o of n){const i=this.getItemValue(o);i!=null&&r.push(i)}return r}),V(this,"find",n=>{if(n==null)return null;const r=this.indexOf(n);return r!==-1?this.at(r):null}),V(this,"findMany",n=>{const r=[];for(const o of n){const i=this.find(o);i!=null&&r.push(i)}return r}),V(this,"at",n=>{if(!this.options.groupBy&&!this.options.groupSort)return this.items[n]??null;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){if(r===n)return s;r++}return null}),V(this,"sortFn",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return(o??0)-(i??0)}),V(this,"sort",n=>[...n].sort(this.sortFn.bind(this))),V(this,"getItemValue",n=>n==null?null:this.options.itemToValue?.(n)??Uo.itemToValue(n)),V(this,"getItemDisabled",n=>n==null?!1:this.options.isItemDisabled?.(n)??Uo.isItemDisabled(n)),V(this,"stringifyItem",n=>n==null?null:this.options.itemToString?.(n)??Uo.itemToString(n)),V(this,"stringify",n=>n==null?null:this.stringifyItem(this.find(n))),V(this,"stringifyItems",(n,r=", ")=>{const o=[];for(const i of n){const s=this.stringifyItem(i);s!=null&&o.push(s)}return o.join(r)}),V(this,"stringifyMany",(n,r)=>this.stringifyItems(this.findMany(n),r)),V(this,"has",n=>this.indexOf(n)!==-1),V(this,"hasItem",n=>n==null?!1:this.has(this.getItemValue(n))),V(this,"group",()=>{const{groupBy:n,groupSort:r}=this.options;if(!n)return[["",[...this.items]]];const o=new Map;this.items.forEach((s,a)=>{const l=n(s,a);o.has(l)||o.set(l,[]),o.get(l).push(s)});let i=Array.from(o.entries());return r&&i.sort(([s],[a])=>{if(typeof r=="function")return r(s,a);if(Array.isArray(r)){const l=r.indexOf(s),c=r.indexOf(a);return l===-1?1:c===-1?-1:l-c}return r==="asc"?s.localeCompare(a):r==="desc"?a.localeCompare(s):0}),i}),V(this,"getNextValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.min(i+r,this.size-1):i+r;i<=this.size&&this.getItemDisabled(this.at(i));)i++;return this.getItemValue(this.at(i))}),V(this,"getPreviousValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.max(i-r,0):i-r;i>=0&&this.getItemDisabled(this.at(i));)i--;return this.getItemValue(this.at(i))}),V(this,"indexOf",n=>{if(n==null)return-1;if(!this.options.groupBy&&!this.options.groupSort)return this.items.findIndex(r=>this.getItemValue(r)===n);if(!this.indexMap){this.indexMap=new Map;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){const a=this.getItemValue(s);a!=null&&this.indexMap.set(a,r),r++}}return this.indexMap.get(n)??-1}),V(this,"getByText",(n,r)=>{const o=r!=null?this.indexOf(r):-1,i=n.length===1;for(let s=0;s{const{state:o,currentValue:i,timeout:s=350}=r,a=o.keysSoFar+n,c=a.length>1&&Array.from(a).every(g=>g===a[0])?a[0]:a,u=this.getByText(c,i),d=this.getItemValue(u);function h(){clearTimeout(o.timer),o.timer=-1}function m(g){o.keysSoFar=g,h(),g!==""&&(o.timer=+setTimeout(()=>{m(""),h()},s))}return m(a),d}),V(this,"update",(n,r)=>{let o=this.indexOf(n);return o===-1?this:this.copy([...this.items.slice(0,o),r,...this.items.slice(o+1)])}),V(this,"upsert",(n,r,o="append")=>{let i=this.indexOf(n);return i===-1?(o==="append"?this.append:this.prepend)(r):this.copy([...this.items.slice(0,i),r,...this.items.slice(i+1)])}),V(this,"insert",(n,...r)=>this.copy(Tr(this.items,n,...r))),V(this,"insertBefore",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Tr(this.items,o,...r))}),V(this,"insertAfter",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Tr(this.items,o+1,...r))}),V(this,"prepend",(...n)=>this.copy(Tr(this.items,0,...n))),V(this,"append",(...n)=>this.copy(Tr(this.items,this.items.length,...n))),V(this,"filter",n=>{const r=this.items.filter((o,i)=>n(this.stringifyItem(o),i,o));return this.copy(r)}),V(this,"remove",(...n)=>{const r=n.map(o=>typeof o=="string"?o:this.getItemValue(o));return this.copy(this.items.filter(o=>{const i=this.getItemValue(o);return i==null?!1:!r.includes(i)}))}),V(this,"move",(n,r)=>{const o=this.indexOf(n);return o===-1?this:this.copy(qo(this.items,[o],r))}),V(this,"moveBefore",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(qo(this.items,i,o))}),V(this,"moveAfter",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(qo(this.items,i,o+1))}),V(this,"reorder",(n,r)=>this.copy(qo(this.items,[n],r))),V(this,"compareValue",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return oi?1:0}),V(this,"range",(n,r)=>{let o=[],i=n;for(;i!=null;){if(this.find(i)&&o.push(i),i===r)return o;i=this.getNextValue(i)}return[]}),V(this,"getValueRange",(n,r)=>n&&r?this.compareValue(n,r)<=0?this.range(n,r):this.range(r,n):[]),V(this,"toString",()=>{let n="";for(const r of this.items){const o=this.getItemValue(r),i=this.stringifyItem(r),s=this.getItemDisabled(r),a=[o,i,s].filter(Boolean).join(":");n+=a+","}return n}),V(this,"toJSON",()=>({size:this.size,first:this.firstValue,last:this.lastValue})),this.items=[...t.items]}get size(){return this.items.length}get firstValue(){let t=0;for(;this.getItemDisabled(this.at(t));)t++;return this.getItemValue(this.at(t))}get lastValue(){let t=this.size-1;for(;this.getItemDisabled(this.at(t));)t--;return this.getItemValue(this.at(t))}*[Symbol.iterator](){yield*this.items}},mx=(e,t)=>!!e?.toLowerCase().startsWith(t.toLowerCase());function Tr(e,t,...n){return[...e.slice(0,t),...n,...e.slice(t)]}function qo(e,t,n){t=[...t].sort((o,i)=>o-i);const r=t.map(o=>e[o]);for(let o=t.length-1;o>=0;o--)e=[...e.slice(0,t[o]),...e.slice(t[o]+1)];return n=Math.max(0,n-t.filter(o=>ot[n])return 1}return e.length-t.length}function bx(e){return e.sort(Su)}function yx(e,t){let n;return He(e,{...t,onEnter:(r,o)=>{if(t.predicate(r,o))return n=r,"stop"}}),n}function xx(e,t){const n=[];return He(e,{onEnter:(r,o)=>{t.predicate(r,o)&&n.push(r)},getChildren:t.getChildren}),n}function wu(e,t){let n;return He(e,{onEnter:(r,o)=>{if(t.predicate(r,o))return n=[...o],"stop"},getChildren:t.getChildren}),n}function kx(e,t){let n=t.initialResult;return He(e,{...t,onEnter:(r,o)=>{n=t.nextResult(n,r,o)}}),n}function Cx(e,t){return kx(e,{...t,initialResult:[],nextResult:(n,r,o)=>(n.push(...t.transform(r,o)),n)})}function Sx(e,t){const{predicate:n,create:r,getChildren:o}=t,i=(s,a)=>{const l=o(s,a),c=[];l.forEach((m,g)=>{const p=[...a,g],v=i(m,p);v&&c.push(v)});const u=a.length===0,d=n(s,a),h=c.length>0;return u||d||h?r(s,c,a):null};return i(e,[])||r(e,[],[])}function wx(e,t){const n=[];let r=0;const o=new Map,i=new Map;return He(e,{getChildren:t.getChildren,onEnter:(s,a)=>{o.has(s)||o.set(s,r++);const l=t.getChildren(s,a);l.forEach(m=>{i.has(m)||i.set(m,s),o.has(m)||o.set(m,r++)});const c=l.length>0?l.map(m=>o.get(m)):void 0,u=i.get(s),d=u?o.get(u):void 0,h=o.get(s);n.push({...s,_children:c,_parent:d,_index:h})}}),n}function Ex(e,t){return{type:"insert",index:e,nodes:t}}function Ox(e){return{type:"remove",indexes:e}}function qs(){return{type:"replace"}}function Eu(e){return[e.slice(0,-1),e[e.length-1]]}function Ou(e,t,n=new Map){const[r,o]=Eu(e);for(let s=r.length-1;s>=0;s--){const a=r.slice(0,s).join();switch(n.get(a)?.type){case"remove":continue}n.set(a,qs())}const i=n.get(r.join());switch(i?.type){case"remove":n.set(r.join(),{type:"removeThenInsert",removeIndexes:i.indexes,insertIndex:o,insertNodes:t});break;default:n.set(r.join(),Ex(o,t))}return n}function Pu(e){const t=new Map,n=new Map;for(const r of e){const o=r.slice(0,-1).join(),i=n.get(o)??[];i.push(r[r.length-1]),n.set(o,i.sort((s,a)=>s-a))}for(const r of e)for(let o=r.length-2;o>=0;o--){const i=r.slice(0,o).join();t.has(i)||t.set(i,qs())}for(const[r,o]of n)t.set(r,Ox(o));return t}function Px(e,t){const n=new Map,[r,o]=Eu(e);for(let i=r.length-1;i>=0;i--){const s=r.slice(0,i).join();n.set(s,qs())}return n.set(r.join(),{type:"removeThenInsert",removeIndexes:[o],insertIndex:o,insertNodes:[t]}),n}function Ko(e,t,n){return Ix(e,{...n,getChildren:(r,o)=>{const i=o.join();switch(t.get(i)?.type){case"replace":case"remove":case"removeThenInsert":case"insert":return n.getChildren(r,o);default:return[]}},transform:(r,o,i)=>{const s=i.join(),a=t.get(s);switch(a?.type){case"remove":return n.create(r,o.filter((u,d)=>!a.indexes.includes(d)),i);case"removeThenInsert":const l=o.filter((u,d)=>!a.removeIndexes.includes(d)),c=a.removeIndexes.reduce((u,d)=>d{const i=[0,...o],s=i.join(),a=t.transform(r,n[s]??[],o),l=i.slice(0,-1).join(),c=n[l]??[];c.push(a),n[l]=c}}),n[""][0]}function Rx(e,t){const{nodes:n,at:r}=t;if(r.length===0)throw new Error("Can't insert nodes at the root");const o=Ou(r,n);return Ko(e,o,t)}function Tx(e,t){if(t.at.length===0)return t.node;const n=Px(t.at,t.node);return Ko(e,n,t)}function Nx(e,t){if(t.indexPaths.length===0)return e;for(const r of t.indexPaths)if(r.length===0)throw new Error("Can't remove the root node");const n=Pu(t.indexPaths);return Ko(e,n,t)}function Ax(e,t){if(t.indexPaths.length===0)return e;for(const i of t.indexPaths)if(i.length===0)throw new Error("Can't move the root node");if(t.to.length===0)throw new Error("Can't move nodes to the root");const n=vx(t.indexPaths),r=n.map(i=>Cu(e,i,t)),o=Ou(t.to,r,Pu(n));return Ko(e,o,t)}function He(e,t){const{onEnter:n,onLeave:r,getChildren:o}=t;let i=[],s=[{node:e}];const a=t.reuseIndexPath?()=>i:()=>i.slice();for(;s.length>0;){let l=s[s.length-1];if(l.state===void 0){const u=n?.(l.node,a());if(u==="stop")return;l.state=u==="skip"?-1:0}const c=l.children||o(l.node,a());if(l.children||(l.children=c),l.state!==-1){if(l.stateZe(this.rootNode,n.rootNode)),V(this,"getNodeChildren",n=>this.options.nodeToChildren?.(n)??qn.nodeToChildren(n)??[]),V(this,"resolveIndexPath",n=>typeof n=="string"?this.getIndexPath(n):n),V(this,"resolveNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(r):void 0}),V(this,"getNodeChildrenCount",n=>this.options.nodeToChildrenCount?.(n)??qn.nodeToChildrenCount(n)),V(this,"getNodeValue",n=>this.options.nodeToValue?.(n)??qn.nodeToValue(n)),V(this,"getNodeDisabled",n=>this.options.isNodeDisabled?.(n)??qn.isNodeDisabled(n)),V(this,"stringify",n=>{const r=this.findNode(n);return r?this.stringifyNode(r):null}),V(this,"stringifyNode",n=>this.options.nodeToString?.(n)??qn.nodeToString(n)),V(this,"getFirstNode",(n=this.rootNode)=>{let r;return He(n,{getChildren:this.getNodeChildren,onEnter:(o,i)=>{if(!r&&i.length>0&&!this.getNodeDisabled(o))return r=o,"stop"}}),r}),V(this,"getLastNode",(n=this.rootNode,r={})=>{let o;return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isSameNode(i,n)){if(r.skip?.({value:this.getNodeValue(i),node:i,indexPath:s}))return"skip";s.length>0&&!this.getNodeDisabled(i)&&(o=i)}}}),o}),V(this,"at",n=>Cu(this.rootNode,n,{getChildren:this.getNodeChildren})),V(this,"findNode",(n,r=this.rootNode)=>yx(r,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})),V(this,"findNodes",(n,r=this.rootNode)=>{const o=new Set(n.filter(i=>i!=null));return xx(r,{getChildren:this.getNodeChildren,predicate:i=>o.has(this.getNodeValue(i))})}),V(this,"sort",n=>n.reduce((r,o)=>{const i=this.getIndexPath(o);return i&&r.push({value:o,indexPath:i}),r},[]).sort((r,o)=>Su(r.indexPath,o.indexPath)).map(({value:r})=>r)),V(this,"getIndexPath",n=>wu(this.rootNode,{getChildren:this.getNodeChildren,predicate:r=>this.getNodeValue(r)===n})),V(this,"getValue",n=>{const r=this.at(n);return r?this.getNodeValue(r):void 0}),V(this,"getValuePath",n=>{if(!n)return[];const r=[];let o=[...n];for(;o.length>0;){const i=this.at(o);i&&r.unshift(this.getNodeValue(i)),o.pop()}return r}),V(this,"getDepth",n=>wu(this.rootNode,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})?.length??0),V(this,"isSameNode",(n,r)=>this.getNodeValue(n)===this.getNodeValue(r)),V(this,"isRootNode",n=>this.isSameNode(n,this.rootNode)),V(this,"contains",(n,r)=>!n||!r?!1:r.slice(0,n.length).every((o,i)=>n[i]===r[i])),V(this,"getNextNode",(n,r={})=>{let o=!1,i;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return l===n&&(o=!0),"skip";if(o&&!this.getNodeDisabled(s))return i=s,"stop";l===n&&(o=!0)}}),i}),V(this,"getPreviousNode",(n,r={})=>{let o,i=!1;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return"skip";if(l===n)return i=!0,"stop";this.getNodeDisabled(s)||(o=s)}}),i?o:void 0}),V(this,"getParentNodes",n=>{const r=this.resolveIndexPath(n)?.slice();if(!r)return[];const o=[];for(;r.length>0;){r.pop();const i=this.at(r);i&&!this.isRootNode(i)&&o.unshift(i)}return o}),V(this,"getDescendantNodes",(n,r)=>{const o=this.resolveNode(n);if(!o)return[];const i=[];return He(o,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{a.length!==0&&(!r?.withBranch&&this.isBranchNode(s)||i.push(s))}}),i}),V(this,"getDescendantValues",(n,r)=>this.getDescendantNodes(n,r).map(i=>this.getNodeValue(i))),V(this,"getParentIndexPath",n=>n.slice(0,-1)),V(this,"getParentNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(this.getParentIndexPath(r)):void 0}),V(this,"visit",n=>{const{skip:r,...o}=n;He(this.rootNode,{...o,getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isRootNode(i))return r?.({value:this.getNodeValue(i),node:i,indexPath:s})?"skip":o.onEnter?.(i,s)}})}),V(this,"getPreviousSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;--i>=0;){const s=o[i];if(!this.getNodeDisabled(s))return s}}),V(this,"getNextSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;++i{const r=this.getParentNode(n);return r?this.getNodeChildren(r):[]}),V(this,"getValues",(n=this.rootNode)=>Cx(n,{getChildren:this.getNodeChildren,transform:o=>[this.getNodeValue(o)]}).slice(1)),V(this,"isValidDepth",(n,r)=>r==null?!0:typeof r=="function"?r(n.length):n.length===r),V(this,"isBranchNode",n=>this.getNodeChildren(n).length>0||this.getNodeChildrenCount(n)!=null),V(this,"getBranchValues",(n=this.rootNode,r={})=>{let o=[];return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(s.length===0)return;const a=this.getNodeValue(i);if(r.skip?.({value:a,node:i,indexPath:s}))return"skip";this.isBranchNode(i)&&this.isValidDepth(s,r.depth)&&o.push(this.getNodeValue(i))}}),o}),V(this,"flatten",(n=this.rootNode)=>wx(n,{getChildren:this.getNodeChildren})),V(this,"_create",(n,r)=>this.getNodeChildren(n).length>0||r.length>0?{...n,children:r}:{...n}),V(this,"_insert",(n,r,o)=>this.copy(Rx(n,{at:r,nodes:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"copy",n=>new hp({...this.options,rootNode:n})),V(this,"_replace",(n,r,o)=>this.copy(Tx(n,{at:r,node:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_move",(n,r,o)=>this.copy(Ax(n,{indexPaths:r,to:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_remove",(n,r)=>this.copy(Nx(n,{indexPaths:r,getChildren:this.getNodeChildren,create:this._create}))),V(this,"replace",(n,r)=>this._replace(this.rootNode,n,r)),V(this,"remove",n=>this._remove(this.rootNode,n)),V(this,"insertBefore",(n,r)=>this.getParentNode(n)?this._insert(this.rootNode,n,r):void 0),V(this,"insertAfter",(n,r)=>{if(!this.getParentNode(n))return;const i=[...n.slice(0,-1),n[n.length-1]+1];return this._insert(this.rootNode,i,r)}),V(this,"move",(n,r)=>this._move(this.rootNode,n,r)),V(this,"filter",n=>{const r=Sx(this.rootNode,{predicate:n,getChildren:this.getNodeChildren,create:this._create});return this.copy(r)}),V(this,"toJSON",()=>this.getValues(this.rootNode)),this.rootNode=t.rootNode}},qn={nodeToValue(e){return typeof e=="string"?e:Kt(e)&&At(e,"value")?e.value:""},nodeToString(e){return typeof e=="string"?e:Kt(e)&&At(e,"label")?e.label:qn.nodeToValue(e)},isNodeDisabled(e){return Kt(e)&&At(e,"disabled")?!!e.disabled:!1},nodeToChildren(e){return e.children},nodeToChildrenCount(e){if(Kt(e)&&At(e,"childrenCount"))return e.childrenCount}},Kn=new WeakMap,Yo=new WeakMap,Xo={},Ks=0,Tu=e=>e&&(e.host||Tu(e.parentNode)),_x=(e,t)=>t.map(n=>{if(e.contains(n))return n;const r=Tu(n);return r&&e.contains(r)?r:(console.error("[zag-js > ariaHidden] target",n,"in not contained inside",e,". Doing nothing"),null)}).filter(n=>!!n),Vx=new Set(["script","output","status","next-route-announcer"]),Fx=e=>Vx.has(e.localName)||e.role==="status"||e.hasAttribute("aria-live")?!0:e.matches("[data-live-announcer]"),Lx=(e,t)=>{const{parentNode:n,markerName:r,controlAttribute:o}=t,i=_x(n,Array.isArray(e)?e:[e]);Xo[r]||(Xo[r]=new WeakMap);const s=Xo[r],a=[],l=new Set,c=new Set(i),u=h=>{!h||l.has(h)||(l.add(h),u(h.parentNode))};i.forEach(u);const d=h=>{!h||c.has(h)||Array.prototype.forEach.call(h.children,m=>{if(l.has(m))d(m);else try{if(Fx(m))return;const p=m.getAttribute(o)==="true",v=(Kn.get(m)||0)+1,y=(s.get(m)||0)+1;Kn.set(m,v),s.set(m,y),a.push(m),v===1&&p&&Yo.set(m,!0),y===1&&m.setAttribute(r,""),p||m.setAttribute(o,"true")}catch(g){console.error("[zag-js > ariaHidden] cannot operate on ",m,g)}})};return d(n),l.clear(),Ks++,()=>{a.forEach(h=>{const m=Kn.get(h)-1,g=s.get(h)-1;Kn.set(h,m),s.set(h,g),m||(Yo.has(h)||h.removeAttribute(o),Yo.delete(h)),g||h.removeAttribute(r)}),Ks--,Ks||(Kn=new WeakMap,Kn=new WeakMap,Yo=new WeakMap,Xo={})}},Dx=e=>(Array.isArray(e)?e[0]:e).ownerDocument.body,zx=(e,t=Dx(e),n="data-aria-hidden")=>{if(t)return Lx(e,{parentNode:t,markerName:n,controlAttribute:"aria-hidden"})},Mx=e=>{const t=requestAnimationFrame(()=>e());return()=>cancelAnimationFrame(t)};function Nu(e,t={}){const{defer:n=!0}=t,r=n?Mx:i=>i(),o=[];return o.push(r(()=>{const s=(typeof e=="function"?e():e).filter(Boolean);s.length!==0&&o.push(zx(s))})),()=>{o.forEach(i=>i?.())}}var Au=j("combobox").parts("root","clearTrigger","content","control","input","item","itemGroup","itemGroupLabel","itemIndicator","itemText","label","list","positioner","trigger");Au.build();var _u=e=>new Go(e);_u.empty=()=>new Go({items:[]});var jx=e=>e.ids?.control??`combobox:${e.id}:control`,$x=e=>e.ids?.input??`combobox:${e.id}:input`,Bx=e=>e.ids?.content??`combobox:${e.id}:content`,Wx=e=>e.ids?.positioner??`combobox:${e.id}:popper`,Hx=e=>e.ids?.trigger??`combobox:${e.id}:toggle-btn`,Ux=e=>e.ids?.clearTrigger??`combobox:${e.id}:clear-btn`,zt=e=>e.getById(Bx(e)),Yn=e=>e.getById($x(e)),Vu=e=>e.getById(Wx(e)),Fu=e=>e.getById(jx(e)),Nr=e=>e.getById(Hx(e)),Lu=e=>e.getById(Ux(e)),Ar=(e,t)=>{if(t==null)return null;const n=`[role=option][data-value="${CSS.escape(t)}"]`;return iv(zt(e),n)},Du=e=>{const t=Yn(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},Gx=e=>{const t=Nr(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},{guards:qx,createMachine:Kx,choose:Yx}=gc(),{and:ye,not:Ue}=qx;Kx({props({props:e}){return{loopFocus:!0,openOnClick:!1,defaultValue:[],closeOnSelect:!e.multiple,allowCustomValue:!1,alwaysSubmitOnEnter:!1,inputBehavior:"none",selectionBehavior:e.multiple?"clear":"replace",openOnKeyPress:!0,openOnChange:!0,composite:!0,navigate({node:t}){lc(t)},collection:_u.empty(),...e,positioning:{placement:"bottom",sameWidth:!0,...e.positioning},translations:{triggerLabel:"Toggle suggestions",clearTriggerLabel:"Clear value",...e.translations}}},initialState({prop:e}){return e("open")||e("defaultOpen")?"suggesting":"idle"},context({prop:e,bindable:t,getContext:n,getEvent:r}){return{currentPlacement:t(()=>({defaultValue:void 0})),value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),isEqual:Ze,hash(o){return o.join(",")},onChange(o){const i=n(),s=i.get("selectedItems"),a=e("collection"),l=o.map(c=>s.find(d=>a.getItemValue(d)===c)||a.find(c));i.set("selectedItems",l),e("onValueChange")?.({value:o,items:l})}})),highlightedValue:t(()=>({defaultValue:e("defaultHighlightedValue")||null,value:e("highlightedValue"),onChange(o){const i=e("collection").find(o);e("onHighlightChange")?.({highlightedValue:o,highlightedItem:i})}})),inputValue:t(()=>{let o=e("inputValue")||e("defaultInputValue")||"";const i=e("defaultValue")||e("value")||[];if(!o.trim()&&!e("multiple")){const s=e("collection").stringifyMany(i);o=bt(e("selectionBehavior"),{preserve:o||s,replace:s,clear:""})}return{defaultValue:o,value:e("inputValue"),onChange(s){const a=r(),l=(a.previousEvent||a).src;e("onInputValueChange")?.({inputValue:s,reason:l})}}}),highlightedItem:t(()=>{const o=e("highlightedValue");return{defaultValue:e("collection").find(o)}}),selectedItems:t(()=>{const o=e("value")||e("defaultValue")||[];return{defaultValue:e("collection").findMany(o)}})}},computed:{isInputValueEmpty:({context:e})=>e.get("inputValue").length===0,isInteractive:({prop:e})=>!(e("readOnly")||e("disabled")),autoComplete:({prop:e})=>e("inputBehavior")==="autocomplete",autoHighlight:({prop:e})=>e("inputBehavior")==="autohighlight",hasSelectedItems:({context:e})=>e.get("value").length>0,valueAsString:({context:e,prop:t})=>t("collection").stringifyItems(e.get("selectedItems")),isCustomValue:({context:e,computed:t})=>e.get("inputValue")!==t("valueAsString")},watch({context:e,prop:t,track:n,action:r,send:o}){n([()=>e.hash("value")],()=>{r(["syncSelectedItems"])}),n([()=>e.get("inputValue")],()=>{r(["syncInputValue"])}),n([()=>e.get("highlightedValue")],()=>{r(["syncHighlightedItem","autofillInputValue"])}),n([()=>t("open")],()=>{r(["toggleVisibility"])}),n([()=>t("collection").toString()],()=>{o({type:"CHILDREN_CHANGE"})})},on:{"SELECTED_ITEMS.SYNC":{actions:["syncSelectedItems"]},"HIGHLIGHTED_VALUE.SET":{actions:["setHighlightedValue"]},"HIGHLIGHTED_VALUE.CLEAR":{actions:["clearHighlightedValue"]},"ITEM.SELECT":{actions:["selectItem"]},"ITEM.CLEAR":{actions:["clearItem"]},"VALUE.SET":{actions:["setValue"]},"INPUT_VALUE.SET":{actions:["setInputValue"]},"POSITIONING.SET":{actions:["reposition"]}},entry:Yx([{guard:"autoFocus",actions:["setInitialFocus"]}]),states:{idle:{tags:["idle","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":{target:"interacting"},"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.FOCUS":{target:"focused"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{target:"focused",actions:["clearInputValue","clearSelectedItems","setInitialFocus"]}}},focused:{tags:["focused","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":[{guard:"isChangeEvent",target:"suggesting"},{target:"interacting"}],"INPUT.CHANGE":[{guard:ye("isOpenControlled","openOnChange"),actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{guard:"openOnChange",target:"suggesting",actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{actions:["setInputValue"]}],"LAYER.INTERACT_OUTSIDE":{target:"idle"},"INPUT.ESCAPE":{guard:ye("isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue"]},"INPUT.BLUR":{target:"idle"},"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_DOWN":[{guard:ye("isOpenControlled","autoComplete"),actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"isOpenControlled",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_UP":[{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]}],OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{actions:["clearInputValue","clearSelectedItems"]}}},interacting:{tags:["open","focused"],entry:["setInitialFocus"],effects:["scrollToHighlightedItem","trackDismissableLayer","trackPlacement","hideOtherElements"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]},{actions:["scrollToHighlightedItem"]}],"INPUT.HOME":{actions:["highlightFirstItem"]},"INPUT.END":{actions:["highlightLastItem"]},"INPUT.ARROW_DOWN":[{guard:ye("autoComplete","isLastItemHighlighted"),actions:["clearHighlightedValue","scrollContentToTop"]},{actions:["highlightNextItem"]}],"INPUT.ARROW_UP":[{guard:ye("autoComplete","isFirstItemHighlighted"),actions:["clearHighlightedValue"]},{actions:["highlightPrevItem"]}],"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":[{guard:"autoComplete",target:"suggesting",actions:["setInputValue"]},{target:"suggesting",actions:["clearHighlightedValue","setInputValue"]}],"ITEM.POINTER_MOVE":{actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],"LAYER.ESCAPE":[{guard:ye("isOpenControlled","autoComplete"),actions:["syncInputValue","invokeOnClose"]},{guard:"autoComplete",target:"focused",actions:["syncInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}},suggesting:{tags:["open","focused"],effects:["trackDismissableLayer","scrollToHighlightedItem","trackPlacement","hideOtherElements"],entry:["setInitialFocus"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"autoHighlight",actions:["highlightFirstItem"]},{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]}],"INPUT.ARROW_DOWN":{target:"interacting",actions:["highlightNextItem"]},"INPUT.ARROW_UP":{target:"interacting",actions:["highlightPrevItem"]},"INPUT.HOME":{target:"interacting",actions:["highlightFirstItem"]},"INPUT.END":{target:"interacting",actions:["highlightLastItem"]},"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":{actions:["setInputValue"]},"LAYER.ESCAPE":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.POINTER_MOVE":{target:"interacting",actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}}},implementations:{guards:{isInputValueEmpty:({computed:e})=>e("isInputValueEmpty"),autoComplete:({computed:e,prop:t})=>e("autoComplete")&&!t("multiple"),autoHighlight:({computed:e})=>e("autoHighlight"),isFirstItemHighlighted:({prop:e,context:t})=>e("collection").firstValue===t.get("highlightedValue"),isLastItemHighlighted:({prop:e,context:t})=>e("collection").lastValue===t.get("highlightedValue"),isCustomValue:({computed:e})=>e("isCustomValue"),allowCustomValue:({prop:e})=>!!e("allowCustomValue"),hasHighlightedItem:({context:e})=>e.get("highlightedValue")!=null,closeOnSelect:({prop:e})=>!!e("closeOnSelect"),isOpenControlled:({prop:e})=>e("open")!=null,openOnChange:({prop:e,context:t})=>{const n=e("openOnChange");return Mm(n)?n:!!n?.({inputValue:t.get("inputValue")})},restoreFocus:({event:e})=>e.restoreFocus==null?!0:!!e.restoreFocus,isChangeEvent:({event:e})=>e.previousEvent?.type==="INPUT.CHANGE",autoFocus:({prop:e})=>!!e("autoFocus"),isHighlightedItemRemoved:({prop:e,context:t})=>!e("collection").has(t.get("highlightedValue"))},effects:{trackDismissableLayer({send:e,prop:t,scope:n}){return t("disableLayer")?void 0:Gn(()=>zt(n),{type:"listbox",defer:!0,exclude:()=>[Yn(n),Nr(n),Lu(n)],onFocusOutside:t("onFocusOutside"),onPointerDownOutside:t("onPointerDownOutside"),onInteractOutside:t("onInteractOutside"),onEscapeKeyDown(o){o.preventDefault(),o.stopPropagation(),e({type:"LAYER.ESCAPE",src:"escape-key"})},onDismiss(){e({type:"LAYER.INTERACT_OUTSIDE",src:"interact-outside",restoreFocus:!1})}})},hideOtherElements({scope:e}){return Nu([Yn(e),zt(e),Nr(e),Lu(e)])},trackPlacement({context:e,prop:t,scope:n}){const r=()=>Fu(n)||Nr(n),o=()=>Vu(n);return e.set("currentPlacement",t("positioning").placement),nt(r,o,{...t("positioning"),defer:!0,onComplete(i){e.set("currentPlacement",i.placement)}})},scrollToHighlightedItem({context:e,prop:t,scope:n,event:r}){const o=Yn(n);let i=[];const s=c=>{const u=r.current().type.includes("POINTER"),d=e.get("highlightedValue");if(u||!d)return;const h=zt(n),m=t("scrollToIndexFn");if(m){const v=t("collection").indexOf(d);m({index:v,immediate:c,getElement:()=>Ar(n,d)});return}const g=Ar(n,d),p=G(()=>{wo(g,{rootEl:h,block:"nearest"})});i.push(p)},a=G(()=>s(!0));i.push(a);const l=Co(o,{attributes:["aria-activedescendant"],callback:()=>s(!1)});return i.push(l),()=>{i.forEach(c=>c())}}},actions:{reposition({context:e,prop:t,scope:n,event:r}){nt(()=>Fu(n),()=>Vu(n),{...t("positioning"),...r.options,defer:!0,listeners:!1,onComplete(s){e.set("currentPlacement",s.placement)}})},setHighlightedValue({context:e,event:t}){t.value!=null&&e.set("highlightedValue",t.value)},clearHighlightedValue({context:e}){e.set("highlightedValue",null)},selectHighlightedItem(e){const{context:t,prop:n}=e,r=n("collection"),o=t.get("highlightedValue");if(!o||!r.has(o))return;const i=n("multiple")?Ln(t.get("value"),o):[o];n("onSelect")?.({value:i,itemValue:o}),t.set("value",i);const s=bt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(i),clear:""});t.set("inputValue",s)},scrollToHighlightedItem({context:e,prop:t,scope:n}){ys(()=>{const r=e.get("highlightedValue");if(r==null)return;const o=Ar(n,r),i=zt(n),s=t("scrollToIndexFn");if(s){const a=t("collection").indexOf(r);s({index:a,immediate:!0,getElement:()=>Ar(n,r)});return}wo(o,{rootEl:i,block:"nearest"})})},selectItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=o("multiple")?Ln(t.get("value"),n.value):[n.value];o("onSelect")?.({value:i,itemValue:n.value}),t.set("value",i);const s=bt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},clearItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=qt(t.get("value"),n.value);t.set("value",i);const s=bt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},setInitialFocus({scope:e}){G(()=>{Du(e)})},setFinalFocus({scope:e}){G(()=>{Nr(e)?.dataset.focusable==null?Du(e):Gx(e)})},syncInputValue({context:e,scope:t,event:n}){const r=Yn(t);r&&(r.value=e.get("inputValue"),queueMicrotask(()=>{n.current().type!=="INPUT.CHANGE"&&d0(r)}))},setInputValue({context:e,event:t}){e.set("inputValue",t.value)},clearInputValue({context:e}){e.set("inputValue","")},revertInputValue({context:e,prop:t,computed:n}){const r=t("selectionBehavior"),o=bt(r,{replace:n("hasSelectedItems")?n("valueAsString"):"",preserve:e.get("inputValue"),clear:""});e.set("inputValue",o)},setValue(e){const{context:t,flush:n,event:r,prop:o}=e;n(()=>{t.set("value",r.value);const i=bt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(r.value),clear:""});t.set("inputValue",i)})},clearSelectedItems(e){const{context:t,flush:n,prop:r}=e;n(()=>{t.set("value",[]);const o=bt(r("selectionBehavior"),{preserve:t.get("inputValue"),replace:r("collection").stringifyMany([]),clear:""});t.set("inputValue",o)})},scrollContentToTop({prop:e,scope:t}){const n=e("scrollToIndexFn");if(n){const r=e("collection").firstValue;n({index:0,immediate:!0,getElement:()=>Ar(t,r)})}else{const r=zt(t);if(!r)return;r.scrollTop=0}},invokeOnOpen({prop:e,event:t}){const n=zu(t);e("onOpenChange")?.({open:!0,reason:n})},invokeOnClose({prop:e,event:t}){const n=zu(t);e("onOpenChange")?.({open:!1,reason:n})},highlightFirstItem({context:e,prop:t,scope:n}){(zt(n)?queueMicrotask:G)(()=>{const o=t("collection").firstValue;o&&e.set("highlightedValue",o)})},highlightFirstItemIfNeeded({computed:e,action:t}){e("autoHighlight")&&t(["highlightFirstItem"])},highlightLastItem({context:e,prop:t,scope:n}){(zt(n)?queueMicrotask:G)(()=>{const o=t("collection").lastValue;o&&e.set("highlightedValue",o)})},highlightNextItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getNextValue(r),!n&&t("loopFocus")&&(n=o.firstValue)):n=o.firstValue,n&&e.set("highlightedValue",n)},highlightPrevItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getPreviousValue(r),!n&&t("loopFocus")&&(n=o.lastValue)):n=o.lastValue,n&&e.set("highlightedValue",n)},highlightFirstSelectedItem({context:e,prop:t}){G(()=>{const[n]=t("collection").sort(e.get("value"));n&&e.set("highlightedValue",n)})},highlightFirstOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{let r=null;n("hasSelectedItems")?r=t("collection").sort(e.get("value"))[0]:r=t("collection").firstValue,r&&e.set("highlightedValue",r)})},highlightLastOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{const r=t("collection");let o=null;n("hasSelectedItems")?o=r.sort(e.get("value"))[0]:o=r.lastValue,o&&e.set("highlightedValue",o)})},autofillInputValue({context:e,computed:t,prop:n,event:r,scope:o}){const i=Yn(o),s=n("collection");if(!t("autoComplete")||!i||!r.keypress)return;const a=s.stringify(e.get("highlightedValue"));G(()=>{i.value=a||e.get("inputValue")})},syncSelectedItems(e){queueMicrotask(()=>{const{context:t,prop:n}=e,r=n("collection"),o=t.get("value"),i=o.map(a=>t.get("selectedItems").find(c=>r.getItemValue(c)===a)||r.find(a));t.set("selectedItems",i);const s=bt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(o),clear:""});t.set("inputValue",s)})},syncHighlightedItem({context:e,prop:t}){const n=t("collection").find(e.get("highlightedValue"));e.set("highlightedItem",n)},toggleVisibility({event:e,send:t,prop:n}){t({type:n("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:e})}}}});function zu(e){return(e.previousEvent||e).src}M()(["allowCustomValue","autoFocus","closeOnSelect","collection","composite","defaultHighlightedValue","defaultInputValue","defaultOpen","defaultValue","dir","disabled","disableLayer","form","getRootNode","highlightedValue","id","ids","inputBehavior","inputValue","invalid","loopFocus","multiple","name","navigate","onFocusOutside","onHighlightChange","onInputValueChange","onInteractOutside","onOpenChange","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","openOnChange","openOnClick","openOnKeyPress","placeholder","positioning","readOnly","required","scrollToIndexFn","selectionBehavior","translations","value","alwaysSubmitOnEnter"]),M()(["htmlFor"]),M()(["id"]),M()(["item","persistFocus"]);const Xx=Au.extendWith("empty"),[Mu,Mt]=Dn({name:"DialogContext",hookName:"useDialogContext",providerName:""}),ju=P.forwardRef((e,t)=>{const n=Mt(),r=qy(),o=Os({...r,present:n.open}),i=qe(n.getBackdropProps(),o.getPresenceProps(),e);return o.unmounted?null:f.jsx(yt.div,{...i,ref:Io(o.ref,t)})});ju.displayName="DialogBackdrop";const $u=P.forwardRef((e,t)=>{const n=Mt(),r=qe(n.getCloseTriggerProps(),e);return f.jsx(yt.button,{...r,ref:t})});$u.displayName="DialogCloseTrigger";const Bu=P.forwardRef((e,t)=>{const n=Mt(),r=Ps(),o=qe(n.getContentProps(),r.getPresenceProps(),e);return r.unmounted?null:f.jsx(yt.div,{...o,ref:Io(r.ref,t)})});Bu.displayName="DialogContent";const Wu=P.forwardRef((e,t)=>{const n=Mt(),r=qe(n.getDescriptionProps(),e);return f.jsx(yt.div,{...r,ref:t})});Wu.displayName="DialogDescription";const Hu=P.forwardRef((e,t)=>{const n=Mt(),r=qe(n.getPositionerProps(),e);return Ps().unmounted?null:f.jsx(yt.div,{...r,ref:t})});Hu.displayName="DialogPositioner";var Qx=Object.defineProperty,Jx=(e,t,n)=>t in e?Qx(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,ae=(e,t,n)=>Jx(e,typeof t!="symbol"?t+"":t,n),Uu={activateTrap(e,t){if(e.length>0){const r=e[e.length-1];r!==t&&r.pause()}const n=e.indexOf(t);n===-1||e.splice(n,1),e.push(t)},deactivateTrap(e,t){const n=e.indexOf(t);n!==-1&&e.splice(n,1),e.length>0&&e[e.length-1].unpause()}},Zx=[],e1=class{constructor(e,t){ae(this,"trapStack"),ae(this,"config"),ae(this,"doc"),ae(this,"state",{containers:[],containerGroups:[],tabbableGroups:[],nodeFocusedBeforeActivation:null,mostRecentlyFocusedNode:null,active:!1,paused:!1,delayInitialFocusTimer:void 0,recentNavEvent:void 0}),ae(this,"listenerCleanups",[]),ae(this,"handleFocus",r=>{const o=$e(r),i=this.findContainerIndex(o,r)>=0;if(i||hs(o))i&&(this.state.mostRecentlyFocusedNode=o);else{r.stopImmediatePropagation();let s,a=!0;if(this.state.mostRecentlyFocusedNode)if(Sr(this.state.mostRecentlyFocusedNode)>0){const l=this.findContainerIndex(this.state.mostRecentlyFocusedNode),{tabbableNodes:c}=this.state.containerGroups[l];if(c.length>0){const u=c.findIndex(d=>d===this.state.mostRecentlyFocusedNode);u>=0&&(this.config.isKeyForward(this.state.recentNavEvent)?u+1=0&&(s=c[u-1],a=!1))}}else this.state.containerGroups.some(l=>l.tabbableNodes.some(c=>Sr(c)>0))||(a=!1);else a=!1;a&&(s=this.findNextNavNode({target:this.state.mostRecentlyFocusedNode,isBackward:this.config.isKeyBackward(this.state.recentNavEvent)})),s?this.tryFocus(s):this.tryFocus(this.state.mostRecentlyFocusedNode||this.getInitialFocusNode())}this.state.recentNavEvent=void 0}),ae(this,"handlePointerDown",r=>{const o=$e(r);if(!(this.findContainerIndex(o,r)>=0)){if(Vr(this.config.clickOutsideDeactivates,r)){this.deactivate({returnFocus:this.config.returnFocusOnDeactivate});return}Vr(this.config.allowOutsideClick,r)||r.preventDefault()}}),ae(this,"handleClick",r=>{const o=$e(r);this.findContainerIndex(o,r)>=0||Vr(this.config.clickOutsideDeactivates,r)||Vr(this.config.allowOutsideClick,r)||(r.preventDefault(),r.stopImmediatePropagation())}),ae(this,"handleTabKey",r=>{if(this.config.isKeyForward(r)||this.config.isKeyBackward(r)){this.state.recentNavEvent=r;const o=this.config.isKeyBackward(r),i=this.findNextNavNode({event:r,isBackward:o});if(!i)return;_r(r)&&r.preventDefault(),this.tryFocus(i)}}),ae(this,"handleEscapeKey",r=>{t1(r)&&Vr(this.config.escapeDeactivates,r)!==!1&&(r.preventDefault(),this.deactivate())}),ae(this,"_mutationObserver"),ae(this,"setupMutationObserver",()=>{const r=this.doc.defaultView||window;this._mutationObserver=new r.MutationObserver(o=>{o.some(s=>Array.from(s.removedNodes).some(l=>l===this.state.mostRecentlyFocusedNode))&&this.tryFocus(this.getInitialFocusNode())})}),ae(this,"updateObservedNodes",()=>{this._mutationObserver?.disconnect(),this.state.active&&!this.state.paused&&this.state.containers.map(r=>{this._mutationObserver?.observe(r,{subtree:!0,childList:!0})})}),ae(this,"getInitialFocusNode",()=>{let r=this.getNodeForOption("initialFocus",{hasFallback:!0});if(r===!1)return!1;if(r===void 0||r&&!Vt(r)){const o=Cr(this.doc);if(o&&this.findContainerIndex(o)>=0)r=o;else{const i=this.state.tabbableGroups[0];r=i&&i.firstTabbableNode||this.getNodeForOption("fallbackFocus")}}else r===null&&(r=this.getNodeForOption("fallbackFocus"));if(!r)throw new Error("Your focus-trap needs to have at least one focusable element");return r.isConnected||(r=this.getNodeForOption("fallbackFocus")),r}),ae(this,"tryFocus",r=>{if(r!==!1&&r!==Cr(this.doc)){if(!r||!r.focus){this.tryFocus(this.getInitialFocusNode());return}r.focus({preventScroll:!!this.config.preventScroll}),this.state.mostRecentlyFocusedNode=r,n1(r)&&r.select()}}),ae(this,"deactivate",r=>{if(!this.state.active)return this;const o={onDeactivate:this.config.onDeactivate,onPostDeactivate:this.config.onPostDeactivate,checkCanReturnFocus:this.config.checkCanReturnFocus,...r};clearTimeout(this.state.delayInitialFocusTimer),this.state.delayInitialFocusTimer=void 0,this.removeListeners(),this.state.active=!1,this.state.paused=!1,this.updateObservedNodes(),Uu.deactivateTrap(this.trapStack,this);const i=this.getOption(o,"onDeactivate"),s=this.getOption(o,"onPostDeactivate"),a=this.getOption(o,"checkCanReturnFocus"),l=this.getOption(o,"returnFocus","returnFocusOnDeactivate");i?.();const c=()=>{Gu(()=>{if(l){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);this.tryFocus(u)}s?.()})};if(l&&a){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);return a(u).then(c,c),this}return c(),this}),ae(this,"pause",r=>{if(this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onPause"),i=this.getOption(r,"onPostPause");return this.state.paused=!0,o?.(),this.removeListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"unpause",r=>{if(!this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onUnpause"),i=this.getOption(r,"onPostUnpause");return this.state.paused=!1,o?.(),this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"updateContainerElements",r=>(this.state.containers=Array.isArray(r)?r.filter(Boolean):[r].filter(Boolean),this.state.active&&this.updateTabbableNodes(),this.updateObservedNodes(),this)),ae(this,"getReturnFocusNode",r=>{const o=this.getNodeForOption("setReturnFocus",{params:[r]});return o||(o===!1?!1:r)}),ae(this,"getOption",(r,o,i)=>r&&r[o]!==void 0?r[o]:this.config[i||o]),ae(this,"getNodeForOption",(r,{hasFallback:o=!1,params:i=[]}={})=>{let s=this.config[r];if(typeof s=="function"&&(s=s(...i)),s===!0&&(s=void 0),!s){if(s===void 0||s===!1)return s;throw new Error(`\`${r}\` was specified but was not a node, or did not return a node`)}let a=s;if(typeof s=="string"){try{a=this.doc.querySelector(s)}catch(l){throw new Error(`\`${r}\` appears to be an invalid selector; error="${l.message}"`)}if(!a&&!o)throw new Error(`\`${r}\` as selector refers to no known node`)}return a}),ae(this,"findNextNavNode",r=>{const{event:o,isBackward:i=!1}=r,s=r.target||$e(o);this.updateTabbableNodes();let a=null;if(this.state.tabbableGroups.length>0){const l=this.findContainerIndex(s,o),c=l>=0?this.state.containerGroups[l]:void 0;if(l<0)i?a=this.state.tabbableGroups[this.state.tabbableGroups.length-1].lastTabbableNode:a=this.state.tabbableGroups[0].firstTabbableNode;else if(i){let u=this.state.tabbableGroups.findIndex(({firstTabbableNode:d})=>s===d);if(u<0&&(c?.container===s||Vt(s)&&!Qt(s)&&!c?.nextTabbableNode(s,!1))&&(u=l),u>=0){const d=u===0?this.state.tabbableGroups.length-1:u-1,h=this.state.tabbableGroups[d];a=Sr(s)>=0?h.lastTabbableNode:h.lastDomTabbableNode}else _r(o)||(a=c?.nextTabbableNode(s,!1))}else{let u=this.state.tabbableGroups.findIndex(({lastTabbableNode:d})=>s===d);if(u<0&&(c?.container===s||Vt(s)&&!Qt(s)&&!c?.nextTabbableNode(s))&&(u=l),u>=0){const d=u===this.state.tabbableGroups.length-1?0:u+1,h=this.state.tabbableGroups[d];a=Sr(s)>=0?h.firstTabbableNode:h.firstDomTabbableNode}else _r(o)||(a=c?.nextTabbableNode(s))}}else a=this.getNodeForOption("fallbackFocus");return a}),this.trapStack=t.trapStack||Zx;const n={returnFocusOnDeactivate:!0,escapeDeactivates:!0,delayInitialFocus:!0,isKeyForward(r){return _r(r)&&!r.shiftKey},isKeyBackward(r){return _r(r)&&r.shiftKey},...t};this.doc=n.document||Fe(Array.isArray(e)?e[0]:e),this.config=n,this.updateContainerElements(e),this.setupMutationObserver()}get active(){return this.state.active}get paused(){return this.state.paused}findContainerIndex(e,t){const n=typeof t?.composedPath=="function"?t.composedPath():void 0;return this.state.containerGroups.findIndex(({container:r,tabbableNodes:o})=>r.contains(e)||n?.includes(r)||o.find(i=>i===e))}updateTabbableNodes(){if(this.state.containerGroups=this.state.containers.map(e=>{const t=vs(e),n=ac(e),r=t.length>0?t[0]:void 0,o=t.length>0?t[t.length-1]:void 0,i=n.find(c=>Qt(c)),s=n.slice().reverse().find(c=>Qt(c)),a=!!t.find(c=>Sr(c)>0);function l(c,u=!0){const d=t.indexOf(c);return d<0?u?n.slice(n.indexOf(c)+1).find(h=>Qt(h)):n.slice(0,n.indexOf(c)).reverse().find(h=>Qt(h)):t[d+(u?1:-1)]}return{container:e,tabbableNodes:t,focusableNodes:n,posTabIndexesFound:a,firstTabbableNode:r,lastTabbableNode:o,firstDomTabbableNode:i,lastDomTabbableNode:s,nextTabbableNode:l}}),this.state.tabbableGroups=this.state.containerGroups.filter(e=>e.tabbableNodes.length>0),this.state.tabbableGroups.length<=0&&!this.getNodeForOption("fallbackFocus"))throw new Error("Your focus-trap must have at least one container with at least one tabbable node in it at all times");if(this.state.containerGroups.find(e=>e.posTabIndexesFound)&&this.state.containerGroups.length>1)throw new Error("At least one node with a positive tabindex was found in one of your focus-trap's multiple containers. Positive tabindexes are only supported in single-container focus-traps.")}addListeners(){if(this.state.active)return Uu.activateTrap(this.trapStack,this),this.state.delayInitialFocusTimer=this.config.delayInitialFocus?Gu(()=>{this.tryFocus(this.getInitialFocusNode())}):this.tryFocus(this.getInitialFocusNode()),this.listenerCleanups.push(se(this.doc,"focusin",this.handleFocus,!0),se(this.doc,"mousedown",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"touchstart",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"click",this.handleClick,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleTabKey,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleEscapeKey)),this}removeListeners(){if(this.state.active)return this.listenerCleanups.forEach(e=>e()),this.listenerCleanups=[],this}activate(e){if(this.state.active)return this;const t=this.getOption(e,"onActivate"),n=this.getOption(e,"onPostActivate"),r=this.getOption(e,"checkCanFocusTrap");r||this.updateTabbableNodes(),this.state.active=!0,this.state.paused=!1,this.state.nodeFocusedBeforeActivation=Cr(this.doc),t?.();const o=()=>{r&&this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),n?.()};return r?(r(this.state.containers.concat()).then(o,o),this):(o(),this)}},_r=e=>e.key==="Tab",Vr=(e,...t)=>typeof e=="function"?e(...t):e,t1=e=>!e.isComposing&&e.key==="Escape",Gu=e=>setTimeout(e,0),n1=e=>e.localName==="input"&&"select"in e&&typeof e.select=="function";function r1(e,t={}){let n;const r=G(()=>{const o=typeof e=="function"?e():e;if(o){n=new e1(o,{escapeDeactivates:!1,allowOutsideClick:!0,preventScroll:!0,returnFocusOnDeactivate:!0,delayInitialFocus:!1,fallbackFocus:o,...t,document:Fe(o)});try{n.activate()}catch{}}});return function(){n?.deactivate(),r()}}var Ys="data-scroll-lock";function o1(e){const t=e.getBoundingClientRect().left;return Math.round(t)+e.scrollLeft?"paddingLeft":"paddingRight"}function i1(e){const t=e??document,n=t.defaultView??window,{documentElement:r,body:o}=t;if(o.hasAttribute(Ys))return;const s=n.innerWidth-r.clientWidth;o.setAttribute(Ys,"");const a=()=>fv(r,"--scrollbar-width",`${s}px`),l=o1(r),c=()=>Po(o,{overflow:"hidden",[l]:`${s}px`}),u=()=>{const{scrollX:h,scrollY:m,visualViewport:g}=n,p=g?.offsetLeft??0,v=g?.offsetTop??0,y=Po(o,{position:"fixed",overflow:"hidden",top:`${-(m-Math.floor(v))}px`,left:`${-(h-Math.floor(p))}px`,right:"0",[l]:`${s}px`});return()=>{y?.(),n.scrollTo({left:h,top:m,behavior:"instant"})}},d=[a(),yo()?u():c()];return()=>{d.forEach(h=>h?.()),o.removeAttribute(Ys)}}var Xs=j("dialog").parts("trigger","backdrop","positioner","content","title","description","closeTrigger"),nn=Xs.build(),qu=e=>e.ids?.positioner??`dialog:${e.id}:positioner`,Ku=e=>e.ids?.backdrop??`dialog:${e.id}:backdrop`,Qs=e=>e.ids?.content??`dialog:${e.id}:content`,Yu=e=>e.ids?.trigger??`dialog:${e.id}:trigger`,Js=e=>e.ids?.title??`dialog:${e.id}:title`,Zs=e=>e.ids?.description??`dialog:${e.id}:description`,Xu=e=>e.ids?.closeTrigger??`dialog:${e.id}:close`,Qo=e=>e.getById(Qs(e)),s1=e=>e.getById(qu(e)),a1=e=>e.getById(Ku(e)),l1=e=>e.getById(Yu(e)),c1=e=>e.getById(Js(e)),u1=e=>e.getById(Zs(e)),d1=e=>e.getById(Xu(e));function h1(e,t){const{state:n,send:r,context:o,prop:i,scope:s}=e,a=i("aria-label"),l=n.matches("open");return{open:l,setOpen(c){n.matches("open")!==c&&r({type:c?"OPEN":"CLOSE"})},getTriggerProps(){return t.button({...nn.trigger.attrs,dir:i("dir"),id:Yu(s),"aria-haspopup":"dialog",type:"button","aria-expanded":l,"data-state":l?"open":"closed","aria-controls":Qs(s),onClick(c){c.defaultPrevented||r({type:"TOGGLE"})}})},getBackdropProps(){return t.element({...nn.backdrop.attrs,dir:i("dir"),hidden:!l,id:Ku(s),"data-state":l?"open":"closed"})},getPositionerProps(){return t.element({...nn.positioner.attrs,dir:i("dir"),id:qu(s),style:{pointerEvents:l?void 0:"none"}})},getContentProps(){const c=o.get("rendered");return t.element({...nn.content.attrs,dir:i("dir"),role:i("role"),hidden:!l,id:Qs(s),tabIndex:-1,"data-state":l?"open":"closed","aria-modal":!0,"aria-label":a||void 0,"aria-labelledby":a||!c.title?void 0:Js(s),"aria-describedby":c.description?Zs(s):void 0})},getTitleProps(){return t.element({...nn.title.attrs,dir:i("dir"),id:Js(s)})},getDescriptionProps(){return t.element({...nn.description.attrs,dir:i("dir"),id:Zs(s)})},getCloseTriggerProps(){return t.button({...nn.closeTrigger.attrs,dir:i("dir"),id:Xu(s),type:"button",onClick(c){c.defaultPrevented||(c.stopPropagation(),r({type:"CLOSE"}))}})}}}var f1={props({props:e,scope:t}){const n=e.role==="alertdialog",r=n?()=>d1(t):void 0,o=typeof e.modal=="boolean"?e.modal:!0;return{role:"dialog",modal:o,trapFocus:o,preventScroll:o,closeOnInteractOutside:!n,closeOnEscape:!0,restoreFocus:!0,initialFocusEl:r,...e}},initialState({prop:e}){return e("open")||e("defaultOpen")?"open":"closed"},context({bindable:e}){return{rendered:e(()=>({defaultValue:{title:!0,description:!0}}))}},watch({track:e,action:t,prop:n}){e([()=>n("open")],()=>{t(["toggleVisibility"])})},states:{open:{entry:["checkRenderedElements","syncZIndex"],effects:["trackDismissableElement","trapFocus","preventScroll","hideContentBelow"],on:{"CONTROLLED.CLOSE":{target:"closed"},CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}]}},closed:{on:{"CONTROLLED.OPEN":{target:"open"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}]}}},implementations:{guards:{isOpenControlled:({prop:e})=>e("open")!=null},effects:{trackDismissableElement({scope:e,send:t,prop:n}){return Gn(()=>Qo(e),{type:"dialog",defer:!0,pointerBlocking:n("modal"),exclude:[l1(e)],onInteractOutside(o){n("onInteractOutside")?.(o),n("closeOnInteractOutside")||o.preventDefault()},persistentElements:n("persistentElements"),onFocusOutside:n("onFocusOutside"),onPointerDownOutside:n("onPointerDownOutside"),onRequestDismiss:n("onRequestDismiss"),onEscapeKeyDown(o){n("onEscapeKeyDown")?.(o),n("closeOnEscape")||o.preventDefault()},onDismiss(){t({type:"CLOSE",src:"interact-outside"})}})},preventScroll({scope:e,prop:t}){if(t("preventScroll"))return i1(e.getDoc())},trapFocus({scope:e,prop:t}){return t("trapFocus")?r1(()=>Qo(e),{preventScroll:!0,returnFocusOnDeactivate:!!t("restoreFocus"),initialFocus:t("initialFocusEl"),setReturnFocus:r=>t("finalFocusEl")?.()??r}):void 0},hideContentBelow({scope:e,prop:t}){return t("modal")?Nu(()=>[Qo(e)],{defer:!0}):void 0}},actions:{checkRenderedElements({context:e,scope:t}){G(()=>{e.set("rendered",{title:!!c1(t),description:!!u1(t)})})},syncZIndex({scope:e}){G(()=>{const t=Qo(e);if(!t)return;const n=vo(t);[s1(e),a1(e)].forEach(o=>{o?.style.setProperty("--z-index",n.zIndex),o?.style.setProperty("--layer-index",n.getPropertyValue("--layer-index"))})})},invokeOnClose({prop:e}){e("onOpenChange")?.({open:!1})},invokeOnOpen({prop:e}){e("onOpenChange")?.({open:!0})},toggleVisibility({prop:e,send:t,event:n}){t({type:e("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:n})}}}};M()(["aria-label","closeOnEscape","closeOnInteractOutside","dir","finalFocusEl","getRootNode","getRootNode","id","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","preventScroll","restoreFocus","role","trapFocus"]);const g1=e=>{const t=P.useId(),{getRootNode:n}=Ql(),{dir:r}=Sv(),o={id:t,getRootNode:n,dir:r,...e},i=bc(f1,o);return h1(i,Tv)},p1=e=>{const[t,{children:n,...r}]=mc(e),[o]=uu(t),i=g1(r),s=Os(qe({present:i.open},t));return f.jsx(Mu,{value:i,children:f.jsx(cu,{value:o,children:f.jsx(kc,{value:s,children:n})})})},m1=e=>{const[t,{value:n,children:r}]=mc(e),[o]=uu(t),i=Os(qe({present:n.open},t));return f.jsx(Mu,{value:n,children:f.jsx(cu,{value:o,children:f.jsx(kc,{value:i,children:r})})})},Qu=P.forwardRef((e,t)=>{const n=Mt(),r=qe(n.getTitleProps(),e);return f.jsx(yt.h2,{...r,ref:t})});Qu.displayName="DialogTitle";const Ju=P.forwardRef((e,t)=>{const n=Mt(),r=Ps(),o=qe({...n.getTriggerProps(),"aria-controls":r.unmounted?void 0:n.getTriggerProps()["aria-controls"]},e);return f.jsx(yt.button,{...o,ref:t})});Ju.displayName="DialogTrigger";var Zu=j("editable").parts("root","area","label","preview","input","editTrigger","submitTrigger","cancelTrigger","control");Zu.build(),M()(["activationMode","autoResize","dir","disabled","finalFocusEl","form","getRootNode","id","ids","invalid","maxLength","name","onEditChange","onFocusOutside","onInteractOutside","onPointerDownOutside","onValueChange","onValueCommit","onValueRevert","placeholder","readOnly","required","selectOnFocus","edit","defaultEdit","submitMode","translations","defaultValue","value"]);const ed=P.forwardRef((e,t)=>{const n=xu(),r=qe(n?.getInputProps(),e);return f.jsx(yt.input,{...r,ref:t})});ed.displayName="FieldInput";const td=j("field").parts("root","errorText","helperText","input","label","select","textarea","requiredIndicator");td.build();var v1=e=>{if(!e)return;const t=vo(e),n=be(e),r=Fe(e),o=()=>{requestAnimationFrame(()=>{e.style.height="auto";let c;t.boxSizing==="content-box"?c=e.scrollHeight-(parseFloat(t.paddingTop)+parseFloat(t.paddingBottom)):c=e.scrollHeight+parseFloat(t.borderTopWidth)+parseFloat(t.borderBottomWidth),t.maxHeight!=="none"&&c>parseFloat(t.maxHeight)?(t.overflowY==="hidden"&&(e.style.overflowY="scroll"),c=parseFloat(t.maxHeight)):t.overflowY!=="hidden"&&(e.style.overflowY="hidden"),e.style.height=`${c}px`})};e.addEventListener("input",o),e.form?.addEventListener("reset",o);const i=Object.getPrototypeOf(e),s=Object.getOwnPropertyDescriptor(i,"value");Object.defineProperty(e,"value",{...s,set(){s?.set?.apply(this,arguments),o()}});const a=new n.ResizeObserver(()=>{requestAnimationFrame(()=>o())});a.observe(e);const l=new n.MutationObserver(()=>o());return l.observe(e,{attributes:!0,attributeFilter:["rows","placeholder"]}),r.fonts?.addEventListener("loadingdone",o),()=>{e.removeEventListener("input",o),e.form?.removeEventListener("reset",o),r.fonts?.removeEventListener("loadingdone",o),a.disconnect(),l.disconnect()}};const nd=P.forwardRef((e,t)=>{const{autoresize:n,...r}=e,o=P.useRef(null),i=xu(),s=qe(i?.getTextareaProps(),{style:{resize:n?"none":void 0}},r);return P.useEffect(()=>{if(n)return v1(o.current)},[n]),f.jsx(yt.textarea,{...s,ref:Io(t,o)})});nd.displayName="FieldTextarea";const rd=j("fieldset").parts("root","errorText","helperText","legend");rd.build();var od=j("file-upload").parts("root","dropzone","item","itemDeleteTrigger","itemGroup","itemName","itemPreview","itemPreviewImage","itemSizeText","label","trigger","clearTrigger");od.build(),M()(["accept","acceptedFiles","allowDrop","capture","defaultAcceptedFiles","dir","directory","disabled","getRootNode","id","ids","invalid","locale","maxFiles","maxFileSize","minFileSize","name","onFileAccept","onFileChange","onFileReject","preventDocumentDrop","required","transformFiles","translations","validate"]),M()(["file","type"]);var id=j("hoverCard").parts("arrow","arrowTip","trigger","positioner","content");id.build();var b1=e=>e.ids?.trigger??`hover-card:${e.id}:trigger`,y1=e=>e.ids?.content??`hover-card:${e.id}:content`,x1=e=>e.ids?.positioner??`hover-card:${e.id}:popper`,ea=e=>e.getById(b1(e)),k1=e=>e.getById(y1(e)),sd=e=>e.getById(x1(e)),{not:Jo,and:ad}=Ft();ad("isOpenControlled",Jo("isPointer")),Jo("isPointer"),ad("isOpenControlled",Jo("isPointer")),Jo("isPointer"),M()(["closeDelay","dir","getRootNode","id","ids","disabled","onOpenChange","defaultOpen","open","openDelay","positioning","onInteractOutside","onPointerDownOutside","onFocusOutside"]);var ld=j("tree-view").parts("branch","branchContent","branchControl","branchIndentGuide","branchIndicator","branchText","branchTrigger","item","itemIndicator","itemText","label","nodeCheckbox","root","tree");ld.build();var cd=e=>new Ru(e);cd.empty=()=>new Ru({rootNode:{children:[]}});var C1=(e,t)=>e.ids?.node?.(t)??`tree:${e.id}:node:${t}`,rn=(e,t)=>{t!=null&&e.getById(C1(e,t))?.focus()};function S1(e,t,n){const r=e.getDescendantValues(t),o=r.every(i=>n.includes(i));return Fn(o?qt(n,...r):Gt(n,...r))}function Zo(e,t){const{context:n,prop:r,refs:o}=e;if(!r("loadChildren")){n.set("expandedValue",p=>Fn(Gt(p,...t)));return}const i=n.get("loadingStatus"),[s,a]=zl(t,p=>i[p]==="loaded");if(s.length>0&&n.set("expandedValue",p=>Fn(Gt(p,...s))),a.length===0)return;const l=r("collection"),[c,u]=zl(a,p=>{const v=l.findNode(p);return l.getNodeChildren(v).length>0});if(c.length>0&&n.set("expandedValue",p=>Fn(Gt(p,...c))),u.length===0)return;n.set("loadingStatus",p=>({...p,...u.reduce((v,y)=>({...v,[y]:"loading"}),{})}));const d=u.map(p=>{const v=l.getIndexPath(p),y=l.getValuePath(v),x=l.findNode(p);return{id:p,indexPath:v,valuePath:y,node:x}}),h=o.get("pendingAborts"),m=r("loadChildren");Xl(m,()=>"[zag-js/tree-view] `loadChildren` is required for async expansion");const g=d.map(({id:p,indexPath:v,valuePath:y,node:x})=>{const b=h.get(p);b&&(b.abort(),h.delete(p));const C=new AbortController;return h.set(p,C),m({valuePath:y,indexPath:v,node:x,signal:C.signal})});Promise.allSettled(g).then(p=>{const v=[],y=[],x=n.get("loadingStatus");let b=r("collection");p.forEach((C,k)=>{const{id:E,indexPath:I,node:R,valuePath:_}=d[k];C.status==="fulfilled"?(x[E]="loaded",v.push(E),b=b.replace(I,{...R,children:C.value})):(h.delete(E),Reflect.deleteProperty(x,E),y.push({node:R,error:C.reason,indexPath:I,valuePath:_}))}),n.set("loadingStatus",x),v.length&&(n.set("expandedValue",C=>Fn(Gt(C,...v))),r("onLoadChildrenComplete")?.({collection:b})),y.length&&r("onLoadChildrenError")?.({nodes:y})})}function jt(e){const{prop:t,context:n}=e;return function({indexPath:o}){return t("collection").getValuePath(o).slice(0,-1).some(s=>!n.get("expandedValue").includes(s))}}var{and:ft}=Ft();ft("isMultipleSelection","moveFocus"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isBranchFocused","isBranchExpanded"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),M()(["ids","collection","dir","expandedValue","expandOnClick","defaultFocusedValue","focusedValue","getRootNode","id","onExpandedChange","onFocusChange","onSelectionChange","checkedValue","selectedValue","selectionMode","typeahead","defaultExpandedValue","defaultSelectedValue","defaultCheckedValue","onCheckedChange","onLoadChildrenComplete","onLoadChildrenError","loadChildren"]),M()(["node","indexPath"]);var ud=j("listbox").parts("label","input","item","itemText","itemIndicator","itemGroup","itemGroupLabel","content","root","valueText");ud.build(),M()(["collection","defaultHighlightedValue","defaultValue","dir","disabled","deselectable","disallowSelectAll","getRootNode","highlightedValue","id","ids","loopFocus","onHighlightChange","onSelect","onValueChange","orientation","scrollToIndexFn","selectionMode","selectOnHighlight","typeahead","value"]),M()(["item","highlightOnHover"]),M()(["id"]),M()(["htmlFor"]);const w1=ud.extendWith("empty");var dd=j("menu").parts("arrow","arrowTip","content","contextTrigger","indicator","item","itemGroup","itemGroupLabel","itemIndicator","itemText","positioner","separator","trigger","triggerItem");dd.build();var hd=e=>e.ids?.trigger??`menu:${e.id}:trigger`,E1=e=>e.ids?.contextTrigger??`menu:${e.id}:ctx-trigger`,fd=e=>e.ids?.content??`menu:${e.id}:content`,O1=e=>e.ids?.positioner??`menu:${e.id}:popper`,ta=(e,t)=>`${e.id}/${t}`,on=e=>e?.dataset.value??null,$t=e=>e.getById(fd(e)),gd=e=>e.getById(O1(e)),ei=e=>e.getById(hd(e)),P1=(e,t)=>t?e.getById(ta(e,t)):null,na=e=>e.getById(E1(e)),Fr=e=>{const n=`[role^="menuitem"][data-ownedby=${CSS.escape(fd(e))}]:not([data-disabled])`;return Oo($t(e),n)},I1=e=>yr(Fr(e)),R1=e=>ls(Fr(e)),ra=(e,t)=>t?e.id===t||e.dataset.value===t:!1,T1=(e,t)=>{const n=Fr(e),r=n.findIndex(o=>ra(o,t.value));return Fm(n,r,{loop:t.loop??t.loopFocus})},N1=(e,t)=>{const n=Fr(e),r=n.findIndex(o=>ra(o,t.value));return Dm(n,r,{loop:t.loop??t.loopFocus})},A1=(e,t)=>{const n=Fr(e),r=n.find(o=>ra(o,t.value));return wr(n,{state:t.typeaheadState,key:t.key,activeId:r?.id??null})},_1=e=>!!e?.getAttribute("role")?.startsWith("menuitem")&&!!e?.hasAttribute("aria-controls"),V1="menu:select";function F1(e,t){if(!e)return;const n=be(e),r=new n.CustomEvent(V1,{detail:{value:t}});e.dispatchEvent(r)}var{not:Xe,and:Xn,or:L1}=Ft();Xe("isSubmenu"),L1("isOpenAutoFocusEvent","isArrowDownEvent"),Xn(Xe("isTriggerItem"),"isOpenControlled"),Xe("isTriggerItem"),Xn("isSubmenu","isOpenControlled"),Xe("isPointerSuspended"),Xn(Xe("isPointerSuspended"),Xe("isTriggerItem")),Xn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect","isOpenControlled"),Xn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect"),Xn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"));function pd(e){let t=e.parent;for(;t&&t.context.get("isSubmenu");)t=t.refs.get("parent");t?.send({type:"CLOSE"})}function D1(e,t){return e?ox(e,t):!1}function z1(e,t,n){const r=Object.keys(e).length>0;if(!t)return null;if(!r)return ta(n,t);for(const o in e){const i=e[o],s=hd(i.scope);if(s===t)return s}return ta(n,t)}M()(["anchorPoint","aria-label","closeOnSelect","composite","defaultHighlightedValue","defaultOpen","dir","getRootNode","highlightedValue","id","ids","loopFocus","navigate","onEscapeKeyDown","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","onSelect","open","positioning","typeahead"]),M()(["closeOnSelect","disabled","value","valueText"]),M()(["htmlFor"]),M()(["id"]),M()(["checked","closeOnSelect","disabled","onCheckedChange","type","value","valueText"]);let oa=new Map,ia=!1;try{ia=new Intl.NumberFormat("de-DE",{signDisplay:"exceptZero"}).resolvedOptions().signDisplay==="exceptZero"}catch{}let ti=!1;try{ti=new Intl.NumberFormat("de-DE",{style:"unit",unit:"degree"}).resolvedOptions().style==="unit"}catch{}const md={degree:{narrow:{default:"°","ja-JP":" 度","zh-TW":"度","sl-SI":" °"}}};class M1{format(t){let n="";if(!ia&&this.options.signDisplay!=null?n=$1(this.numberFormatter,this.options.signDisplay,t):n=this.numberFormatter.format(t),this.options.style==="unit"&&!ti){var r;let{unit:o,unitDisplay:i="short",locale:s}=this.resolvedOptions();if(!o)return n;let a=(r=md[o])===null||r===void 0?void 0:r[i];n+=a[s]||a.default}return n}formatToParts(t){return this.numberFormatter.formatToParts(t)}formatRange(t,n){if(typeof this.numberFormatter.formatRange=="function")return this.numberFormatter.formatRange(t,n);if(n= start date");return`${this.format(t)} – ${this.format(n)}`}formatRangeToParts(t,n){if(typeof this.numberFormatter.formatRangeToParts=="function")return this.numberFormatter.formatRangeToParts(t,n);if(n= start date");let r=this.numberFormatter.formatToParts(t),o=this.numberFormatter.formatToParts(n);return[...r.map(i=>({...i,source:"startRange"})),{type:"literal",value:" – ",source:"shared"},...o.map(i=>({...i,source:"endRange"}))]}resolvedOptions(){let t=this.numberFormatter.resolvedOptions();return!ia&&this.options.signDisplay!=null&&(t={...t,signDisplay:this.options.signDisplay}),!ti&&this.options.style==="unit"&&(t={...t,style:"unit",unit:this.options.unit,unitDisplay:this.options.unitDisplay}),t}constructor(t,n={}){this.numberFormatter=j1(t,n),this.options=n}}function j1(e,t={}){let{numberingSystem:n}=t;if(n&&e.includes("-nu-")&&(e.includes("-u-")||(e+="-u-"),e+=`-nu-${n}`),t.style==="unit"&&!ti){var r;let{unit:s,unitDisplay:a="short"}=t;if(!s)throw new Error('unit option must be provided with style: "unit"');if(!(!((r=md[s])===null||r===void 0)&&r[a]))throw new Error(`Unsupported unit ${s} with unitDisplay = ${a}`);t={...t,style:"decimal"}}let o=e+(t?Object.entries(t).sort((s,a)=>s[0]0||Object.is(n,0):t==="exceptZero"&&(Object.is(n,-0)||Object.is(n,0)?n=Math.abs(n):r=n>0),r){let o=e.format(-n),i=e.format(n),s=o.replace(i,"").replace(/\u200e|\u061C/,"");return[...s].length!==1&&console.warn("@react-aria/i18n polyfill for NumberFormat signDisplay: Unsupported case"),o.replace(i,"!!!").replace(s,"+").replace("!!!",i)}else return e.format(n)}}const B1=new RegExp("^.*\\(.*\\).*$"),W1=["latn","arab","hanidec","deva","beng","fullwide"];class vd{parse(t){return sa(this.locale,this.options,t).parse(t)}isValidPartialNumber(t,n,r){return sa(this.locale,this.options,t).isValidPartialNumber(t,n,r)}getNumberingSystem(t){return sa(this.locale,this.options,t).options.numberingSystem}constructor(t,n={}){this.locale=t,this.options=n}}const bd=new Map;function sa(e,t,n){let r=yd(e,t);if(!e.includes("-nu-")&&!r.isValidPartialNumber(n)){for(let o of W1)if(o!==r.options.numberingSystem){let i=yd(e+(e.includes("-u-")?"-nu-":"-u-nu-")+o,t);if(i.isValidPartialNumber(n))return i}}return r}function yd(e,t){let n=e+(t?Object.entries(t).sort((o,i)=>o[0]-1&&(n=`-${n}`)}let r=n?+n:NaN;if(isNaN(r))return NaN;if(this.options.style==="percent"){var o,i;let s={...this.options,style:"decimal",minimumFractionDigits:Math.min(((o=this.options.minimumFractionDigits)!==null&&o!==void 0?o:0)+2,20),maximumFractionDigits:Math.min(((i=this.options.maximumFractionDigits)!==null&&i!==void 0?i:0)+2,20)};return new vd(this.locale,s).parse(new M1(this.locale,s).format(r))}return this.options.currencySign==="accounting"&&B1.test(t)&&(r=-1*r),r}sanitize(t){return t=t.replace(this.symbols.literals,""),this.symbols.minusSign&&(t=t.replace("-",this.symbols.minusSign)),this.options.numberingSystem==="arab"&&(this.symbols.decimal&&(t=t.replace(",",this.symbols.decimal),t=t.replace("،",this.symbols.decimal)),this.symbols.group&&(t=Qn(t,".",this.symbols.group))),this.symbols.group==="’"&&t.includes("'")&&(t=Qn(t,"'",this.symbols.group)),this.options.locale==="fr-FR"&&this.symbols.group&&(t=Qn(t," ",this.symbols.group),t=Qn(t,/\u00A0/g,this.symbols.group)),t}isValidPartialNumber(t,n=-1/0,r=1/0){return t=this.sanitize(t),this.symbols.minusSign&&t.startsWith(this.symbols.minusSign)&&n<0?t=t.slice(this.symbols.minusSign.length):this.symbols.plusSign&&t.startsWith(this.symbols.plusSign)&&r>0&&(t=t.slice(this.symbols.plusSign.length)),this.symbols.group&&t.startsWith(this.symbols.group)||this.symbols.decimal&&t.indexOf(this.symbols.decimal)>-1&&this.options.maximumFractionDigits===0?!1:(this.symbols.group&&(t=Qn(t,this.symbols.group,"")),t=t.replace(this.symbols.numeral,""),this.symbols.decimal&&(t=t.replace(this.symbols.decimal,"")),t.length===0)}constructor(t,n={}){this.locale=t,n.roundingIncrement!==1&&n.roundingIncrement!=null&&(n.maximumFractionDigits==null&&n.minimumFractionDigits==null?(n.maximumFractionDigits=0,n.minimumFractionDigits=0):n.maximumFractionDigits==null?n.maximumFractionDigits=n.minimumFractionDigits:n.minimumFractionDigits==null&&(n.minimumFractionDigits=n.maximumFractionDigits)),this.formatter=new Intl.NumberFormat(t,n),this.options=this.formatter.resolvedOptions(),this.symbols=G1(t,this.formatter,this.options,n);var r,o;this.options.style==="percent"&&(((r=this.options.minimumFractionDigits)!==null&&r!==void 0?r:0)>18||((o=this.options.maximumFractionDigits)!==null&&o!==void 0?o:0)>18)&&console.warn("NumberParser cannot handle percentages with greater than 18 decimal places, please reduce the number in your options.")}}const xd=new Set(["decimal","fraction","integer","minusSign","plusSign","group"]),U1=[0,4,2,1,11,20,3,7,100,21,.1,1.1];function G1(e,t,n,r){var o,i,s,a;let l=new Intl.NumberFormat(e,{...n,minimumSignificantDigits:1,maximumSignificantDigits:21,roundingIncrement:1,roundingPriority:"auto",roundingMode:"halfExpand"}),c=l.formatToParts(-10000.111),u=l.formatToParts(10000.111),d=U1.map(A=>l.formatToParts(A));var h;let m=(h=(o=c.find(A=>A.type==="minusSign"))===null||o===void 0?void 0:o.value)!==null&&h!==void 0?h:"-",g=(i=u.find(A=>A.type==="plusSign"))===null||i===void 0?void 0:i.value;!g&&(r?.signDisplay==="exceptZero"||r?.signDisplay==="always")&&(g="+");let v=(s=new Intl.NumberFormat(e,{...n,minimumFractionDigits:2,maximumFractionDigits:2}).formatToParts(.001).find(A=>A.type==="decimal"))===null||s===void 0?void 0:s.value,y=(a=c.find(A=>A.type==="group"))===null||a===void 0?void 0:a.value,x=c.filter(A=>!xd.has(A.type)).map(A=>kd(A.value)),b=d.flatMap(A=>A.filter(T=>!xd.has(T.type)).map(T=>kd(T.value))),C=[...new Set([...x,...b])].sort((A,T)=>T.length-A.length),k=C.length===0?new RegExp("[\\p{White_Space}]","gu"):new RegExp(`${C.join("|")}|[\\p{White_Space}]`,"gu"),E=[...new Intl.NumberFormat(n.locale,{useGrouping:!1}).format(9876543210)].reverse(),I=new Map(E.map((A,T)=>[A,T])),R=new RegExp(`[${E.join("")}]`,"g");return{minusSign:m,plusSign:g,decimal:v,group:y,literals:k,numeral:R,index:A=>String(I.get(A))}}function Qn(e,t,n){return e.replaceAll?e.replaceAll(t,n):e.split(t).join(n)}function kd(e){return e.replace(/[.*+?^${}()|[\]\\]/g,"\\$&")}var Cd=j("numberInput").parts("root","label","input","control","valueText","incrementTrigger","decrementTrigger","scrubber");Cd.build();var q1=e=>e.ids?.input??`number-input:${e.id}:input`,K1=e=>e.ids?.incrementTrigger??`number-input:${e.id}:inc`,Y1=e=>e.ids?.decrementTrigger??`number-input:${e.id}:dec`,Sd=e=>`number-input:${e.id}:cursor`,ni=e=>e.getById(q1(e)),X1=e=>e.getById(K1(e)),Q1=e=>e.getById(Y1(e)),wd=e=>e.getDoc().getElementById(Sd(e)),J1=(e,t)=>{let n=null;return t==="increment"&&(n=X1(e)),t==="decrement"&&(n=Q1(e)),n},Z1=(e,t)=>{if(!oc())return nk(e,t),()=>{wd(e)?.remove()}},ek=e=>{const t=e.getDoc(),n=t.documentElement,r=t.body;return r.style.pointerEvents="none",n.style.userSelect="none",n.style.cursor="ew-resize",()=>{r.style.pointerEvents="",n.style.userSelect="",n.style.cursor="",n.style.length||n.removeAttribute("style"),r.style.length||r.removeAttribute("style")}},tk=(e,t)=>{const{point:n,isRtl:r,event:o}=t,i=e.getWin(),s=ds(o.movementX,i.devicePixelRatio),a=ds(o.movementY,i.devicePixelRatio);let l=s>0?"increment":s<0?"decrement":null;r&&l==="increment"&&(l="decrement"),r&&l==="decrement"&&(l="increment");const c={x:n.x+s,y:n.y+a},u=i.innerWidth,d=ds(7.5,i.devicePixelRatio);return c.x=e0(c.x+d,u)-d,{hint:l,point:c}},nk=(e,t)=>{const n=e.getDoc(),r=n.createElement("div");r.className="scrubber--cursor",r.id=Sd(e),Object.assign(r.style,{width:"15px",height:"15px",position:"fixed",pointerEvents:"none",left:"0px",top:"0px",zIndex:f0,transform:t?`translate3d(${t.x}px, ${t.y}px, 0px)`:void 0,willChange:"transform"}),r.innerHTML=` + */var Il;function im(){if(Il)return Q;Il=1;var e=typeof Symbol=="function"&&Symbol.for,t=e?Symbol.for("react.element"):60103,n=e?Symbol.for("react.portal"):60106,r=e?Symbol.for("react.fragment"):60107,o=e?Symbol.for("react.strict_mode"):60108,i=e?Symbol.for("react.profiler"):60114,s=e?Symbol.for("react.provider"):60109,a=e?Symbol.for("react.context"):60110,l=e?Symbol.for("react.async_mode"):60111,c=e?Symbol.for("react.concurrent_mode"):60111,u=e?Symbol.for("react.forward_ref"):60112,h=e?Symbol.for("react.suspense"):60113,d=e?Symbol.for("react.suspense_list"):60120,m=e?Symbol.for("react.memo"):60115,f=e?Symbol.for("react.lazy"):60116,p=e?Symbol.for("react.block"):60121,v=e?Symbol.for("react.fundamental"):60117,b=e?Symbol.for("react.responder"):60118,y=e?Symbol.for("react.scope"):60119;function x(k){if(typeof k=="object"&&k!==null){var O=k.$$typeof;switch(O){case t:switch(k=k.type,k){case l:case c:case r:case i:case o:case h:return k;default:switch(k=k&&k.$$typeof,k){case a:case u:case f:case m:case s:return k;default:return O}}case n:return O}}}function w(k){return x(k)===c}return Q.AsyncMode=l,Q.ConcurrentMode=c,Q.ContextConsumer=a,Q.ContextProvider=s,Q.Element=t,Q.ForwardRef=u,Q.Fragment=r,Q.Lazy=f,Q.Memo=m,Q.Portal=n,Q.Profiler=i,Q.StrictMode=o,Q.Suspense=h,Q.isAsyncMode=function(k){return w(k)||x(k)===l},Q.isConcurrentMode=w,Q.isContextConsumer=function(k){return x(k)===a},Q.isContextProvider=function(k){return x(k)===s},Q.isElement=function(k){return typeof k=="object"&&k!==null&&k.$$typeof===t},Q.isForwardRef=function(k){return x(k)===u},Q.isFragment=function(k){return x(k)===r},Q.isLazy=function(k){return x(k)===f},Q.isMemo=function(k){return x(k)===m},Q.isPortal=function(k){return x(k)===n},Q.isProfiler=function(k){return x(k)===i},Q.isStrictMode=function(k){return x(k)===o},Q.isSuspense=function(k){return x(k)===h},Q.isValidElementType=function(k){return typeof k=="string"||typeof k=="function"||k===r||k===c||k===i||k===o||k===h||k===d||typeof k=="object"&&k!==null&&(k.$$typeof===f||k.$$typeof===m||k.$$typeof===s||k.$$typeof===a||k.$$typeof===u||k.$$typeof===v||k.$$typeof===b||k.$$typeof===y||k.$$typeof===p)},Q.typeOf=x,Q}var Tl;function sm(){return Tl||(Tl=1,Zi.exports=im()),Zi.exports}var es,Nl;function am(){if(Nl)return es;Nl=1;var e=sm(),t={childContextTypes:!0,contextType:!0,contextTypes:!0,defaultProps:!0,displayName:!0,getDefaultProps:!0,getDerivedStateFromError:!0,getDerivedStateFromProps:!0,mixins:!0,propTypes:!0,type:!0},n={name:!0,length:!0,prototype:!0,caller:!0,callee:!0,arguments:!0,arity:!0},r={$$typeof:!0,render:!0,defaultProps:!0,displayName:!0,propTypes:!0},o={$$typeof:!0,compare:!0,defaultProps:!0,displayName:!0,propTypes:!0,type:!0},i={};i[e.ForwardRef]=r,i[e.Memo]=o;function s(f){return e.isMemo(f)?o:i[f.$$typeof]||t}var a=Object.defineProperty,l=Object.getOwnPropertyNames,c=Object.getOwnPropertySymbols,u=Object.getOwnPropertyDescriptor,h=Object.getPrototypeOf,d=Object.prototype;function m(f,p,v){if(typeof p!="string"){if(d){var b=h(p);b&&b!==d&&m(f,b,v)}var y=l(p);c&&(y=y.concat(c(p)));for(var x=s(f),w=s(p),k=0;k=4;++r,o-=4)n=e.charCodeAt(r)&255|(e.charCodeAt(++r)&255)<<8|(e.charCodeAt(++r)&255)<<16|(e.charCodeAt(++r)&255)<<24,n=(n&65535)*1540483477+((n>>>16)*59797<<16),n^=n>>>24,t=(n&65535)*1540483477+((n>>>16)*59797<<16)^(t&65535)*1540483477+((t>>>16)*59797<<16);switch(o){case 3:t^=(e.charCodeAt(r+2)&255)<<16;case 2:t^=(e.charCodeAt(r+1)&255)<<8;case 1:t^=e.charCodeAt(r)&255,t=(t&65535)*1540483477+((t>>>16)*59797<<16)}return t^=t>>>13,t=(t&65535)*1540483477+((t>>>16)*59797<<16),((t^t>>>15)>>>0).toString(36)}var um={animationIterationCount:1,aspectRatio:1,borderImageOutset:1,borderImageSlice:1,borderImageWidth:1,boxFlex:1,boxFlexGroup:1,boxOrdinalGroup:1,columnCount:1,columns:1,flex:1,flexGrow:1,flexPositive:1,flexShrink:1,flexNegative:1,flexOrder:1,gridRow:1,gridRowEnd:1,gridRowSpan:1,gridRowStart:1,gridColumn:1,gridColumnEnd:1,gridColumnSpan:1,gridColumnStart:1,msGridRow:1,msGridRowSpan:1,msGridColumn:1,msGridColumnSpan:1,fontWeight:1,lineHeight:1,opacity:1,order:1,orphans:1,scale:1,tabSize:1,widows:1,zIndex:1,zoom:1,WebkitLineClamp:1,fillOpacity:1,floodOpacity:1,stopOpacity:1,strokeDasharray:1,strokeDashoffset:1,strokeMiterlimit:1,strokeOpacity:1,strokeWidth:1},dm=/[A-Z]|^ms/g,hm=/_EMO_([^_]+?)_([^]*?)_EMO_/g,_l=function(t){return t.charCodeAt(1)===45},Vl=function(t){return t!=null&&typeof t!="boolean"},rs=bl(function(e){return _l(e)?e:e.replace(dm,"-$&").toLowerCase()}),Fl=function(t,n){switch(t){case"animation":case"animationName":if(typeof n=="string")return n.replace(hm,function(r,o,i){return at={name:o,styles:i,next:at},o})}return um[t]!==1&&!_l(t)&&typeof n=="number"&&n!==0?n+"px":n};function vr(e,t,n){if(n==null)return"";var r=n;if(r.__emotion_styles!==void 0)return r;switch(typeof n){case"boolean":return"";case"object":{var o=n;if(o.anim===1)return at={name:o.name,styles:o.styles,next:at},o.name;var i=n;if(i.styles!==void 0){var s=i.next;if(s!==void 0)for(;s!==void 0;)at={name:s.name,styles:s.styles,next:at},s=s.next;var a=i.styles+";";return a}return fm(e,t,n)}case"function":{if(e!==void 0){var l=at,c=n(e);return at=l,vr(e,t,c)}break}}var u=n;if(t==null)return u;var h=t[u];return h!==void 0?h:u}function fm(e,t,n){var r="";if(Array.isArray(n))for(var o=0;or?.(...n))}}const xm=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),km=/^on[A-Z]/;function br(...e){let t={};for(let n of e){for(let r in t){if(km.test(r)&&typeof t[r]=="function"&&typeof n[r]=="function"){t[r]=ym(t[r],n[r]);continue}if(r==="className"||r==="class"){t[r]=xm(t[r],n[r]);continue}if(r==="style"){t[r]=Object.assign({},t[r]??{},n[r]??{});continue}t[r]=n[r]!==void 0?n[r]:t[r]}for(let r in n)t[r]===void 0&&(t[r]=n[r])}return t}function Cm(e,t){if(e!=null){if(typeof e=="function"){e(t);return}try{e.current=t}catch{throw new Error(`Cannot assign value '${t}' to ref '${e}'`)}}}function wm(...e){return t=>{e.forEach(n=>{Cm(n,t)})}}function yr(e){const t=Object.assign({},e);for(let n in t)t[n]===void 0&&delete t[n];return t}const Ge=(...e)=>e.filter(Boolean).map(t=>t.trim()).join(" ");function Sm(e){return e.default||e}const _e=e=>e!=null&&typeof e=="object"&&!Array.isArray(e),lt=e=>typeof e=="string",cs=e=>typeof e=="function";function Em(e){const t=C.version;return!lt(t)||t.startsWith("18.")?e?.ref:e?.props?.ref}const Wl=(...e)=>{const t=e.reduce((n,r)=>(r?.forEach(o=>n.add(o)),n),new Set([]));return Array.from(t)};function Om(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Fn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??Om(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[Pm,ho]=Fn({name:"ChakraContext",strict:!0,providerName:""});function Rm(e){const{value:t,children:n}=e;return g.jsxs(Pm,{value:t,children:[!t._config.disableLayers&&g.jsx(Bl,{styles:t.layers.atRule}),g.jsx(Bl,{styles:t._global}),n]})}const Im=(e,t)=>{const n={},r={},o=Object.keys(e);for(const i of o)t(i)?r[i]=e[i]:n[i]=e[i];return[r,n]},Ln=(e,t)=>{const n=cs(t)?t:r=>t.includes(r);return Im(e,n)},Tm=new Set(["htmlWidth","htmlHeight","htmlSize","htmlTranslate"]);function Nm(e){return typeof e=="string"&&Tm.has(e)}function Am(e,t,n){const{css:r,isValidProperty:o}=ho(),{children:i,...s}=e,a=P.useMemo(()=>{const[d,m]=Ln(s,y=>n(y,t.variantKeys)),[f,p]=Ln(m,t.variantKeys),[v,b]=Ln(p,o);return{forwardedProps:d,variantProps:f,styleProps:v,elementProps:b}},[t.variantKeys,n,s,o]),{css:l,...c}=a.styleProps,u=P.useMemo(()=>{const d={...a.variantProps};return t.variantKeys.includes("colorPalette")||(d.colorPalette=s.colorPalette),t.variantKeys.includes("orientation")||(d.orientation=s.orientation),t(d)},[t,a.variantProps,s.colorPalette,s.orientation]);return{styles:P.useMemo(()=>r(u,..._m(l),c),[r,u,l,c]),props:{...a.forwardedProps,...a.elementProps,children:i}}}const _m=e=>(Array.isArray(e)?e:[e]).filter(Boolean).flat(),Vm=Sm(Ip),Fm=e=>e!=="theme",Lm=(e,t,n)=>{let r;if(t){const o=t.shouldForwardProp;r=e.__emotion_forwardProp&&o?i=>e.__emotion_forwardProp(i)&&o(i):o}return typeof r!="function"&&n&&(r=e.__emotion_forwardProp),r};let Dm=typeof document<"u";const Hl=({cache:e,serialized:t,isStringTag:n})=>{ts(e,t,n);const r=zl(()=>ns(e,t,n));if(!Dm&&r!==void 0){let o=t.name,i=t.next;for(;i!==void 0;)o=Ge(o,i.name),i=i.next;return g.jsx("style",{"data-emotion":Ge(e.key,o),dangerouslySetInnerHTML:{__html:r},nonce:e.sheet.nonce})}return null},Ul={path:["d"],text:["x","y"],circle:["cx","cy","r"],rect:["width","height","x","y","rx","ry"],ellipse:["cx","cy","rx","ry"],g:["transform"],stop:["offset","stopOpacity"]},zm=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),us=((e,t={},n={})=>{if(zm(Ul,e)){n.forwardProps||(n.forwardProps=[]);const c=Ul[e];n.forwardProps=Wl([...n.forwardProps,...c])}const r=e.__emotion_real===e,o=r&&e.__emotion_base||e;let i,s;n!==void 0&&(i=n.label,s=n.target);let a=[];const l=is((c,u,h)=>{const{cva:d,isValidProperty:m}=ho(),f=t.__cva__?t:d(t),p=Mm(e.__emotion_cva,f),v=z=>(W,J)=>z.includes(W)?!0:!J?.includes(W)&&!m(W);!n.shouldForwardProp&&n.forwardProps&&(n.shouldForwardProp=v(n.forwardProps));const b=(z,W)=>{const J=typeof e=="string"&&e.charCodeAt(0)>96?Vm:Fm,ee=!W?.includes(z)&&!m(z);return J(z)&&ee},y=Lm(e,n,r)||b,x=C.useMemo(()=>Object.assign({},n.defaultProps,yr(c)),[c]),{props:w,styles:k}=Am(x,p,y);let O="",R=[k],I=w;if(w.theme==null){I={};for(let z in w)I[z]=w[z];I.theme=C.useContext(ss)}typeof w.className=="string"?O=Al(u.registered,R,w.className):w.className!=null&&(O=Ge(O,w.className));const N=os(a.concat(R),u.registered,I);N.styles&&(O=Ge(O,`${u.key}-${N.name}`)),s!==void 0&&(O=Ge(O,s));const A=!y("as");let T=A&&w.as||o,S={};for(let z in w)if(!(A&&z==="as")){if(Nm(z)){const W=z.replace("html","").toLowerCase();S[W]=w[z];continue}y(z)&&(S[z]=w[z])}let _=O.trim();_?S.className=_:Reflect.deleteProperty(S,"className"),S.ref=h;const D=n.forwardAsChild||n.forwardProps?.includes("asChild");if(w.asChild&&!D){const z=C.isValidElement(w.children)?C.Children.only(w.children):C.Children.toArray(w.children).find(C.isValidElement);if(!z)throw new Error("[chakra-ui > factory] No valid child found");T=z.type,S.children=null,Reflect.deleteProperty(S,"asChild"),S=br(S,z.props),S.ref=wm(h,Em(z))}return S.as&&D?(S.as=void 0,g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{asChild:!0,...S,children:g.jsx(w.as,{children:S.children})})]})):g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{...S})]})});return l.displayName=i!==void 0?i:`chakra(${typeof o=="string"?o:o.displayName||o.name||"Component"})`,l.__emotion_real=l,l.__emotion_base=o,l.__emotion_forwardProp=n.shouldForwardProp,l.__emotion_cva=t,Object.defineProperty(l,"toString",{value(){return`.${s}`}}),l}).bind(),ds=new Map,ve=new Proxy(us,{apply(e,t,n){return us(...n)},get(e,t){return ds.has(t)||ds.set(t,us(t)),ds.get(t)}}),Mm=(e,t)=>e&&!t?e:!e&&t?t:e.merge(t),xr=ve("div");xr.displayName="Box";const jm=Object.freeze({}),$m=Object.freeze({});function Bm(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getRecipe(t):{});return r.cva(structuredClone(o))},[t,n,r])}const Wm=e=>e.charAt(0).toUpperCase()+e.slice(1);function Lt(e){const{key:t,recipe:n}=e,r=Wm(t||n.className||"Component"),[o,i]=Fn({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`});function s(c){const{unstyled:u,...h}=c,d=Bm({key:t,recipe:h.recipe||n}),[m,f]=P.useMemo(()=>d.splitVariantProps(h),[d,h]);return{styles:u?jm:d(m),className:d.className,props:f}}const a=(c,u)=>{const h=ve(c,{},u),d=P.forwardRef((m,f)=>{const p=i(),v=P.useMemo(()=>br(p,m),[m,p]),{styles:b,className:y,props:x}=s(v);return g.jsx(h,{...x,ref:f,css:[b,v.css],className:Ge(y,v.className)})});return d.displayName=c.displayName||c.name,d};function l(){return o}return{withContext:a,PropsProvider:o,withPropsProvider:l,usePropsContext:i,useRecipeResult:s}}function fo(e){return e==null?[]:Array.isArray(e)?e:[e]}var kr=e=>e[0],hs=e=>e[e.length-1],Hm=(e,t)=>e.indexOf(t)!==-1,Kt=(e,...t)=>e.concat(t),Yt=(e,...t)=>e.filter(n=>!t.includes(n)),Dn=e=>Array.from(new Set(e)),fs=(e,t)=>{const n=new Set(t);return e.filter(r=>!n.has(r))},zn=(e,t)=>Hm(e,t)?Yt(e,t):Kt(e,t);function Gl(e,t,n={}){const{step:r=1,loop:o=!0}=n,i=t+r,s=e.length,a=s-1;return t===-1?r>0?0:a:i<0?o?a:0:i>=s?o?0:t>s?s:t:i}function Um(e,t,n={}){return e[Gl(e,t,n)]}function Gm(e,t,n={}){const{step:r=1,loop:o=!0}=n;return Gl(e,t,{step:-r,loop:o})}function qm(e,t,n={}){return e[Gm(e,t,n)]}function ql(e,t){return e.reduce(([n,r],o)=>(t(o)?n.push(o):r.push(o),[n,r]),[[],[]])}var Kl=e=>e?.constructor.name==="Array",Km=(e,t)=>{if(e.length!==t.length)return!1;for(let n=0;n{if(Object.is(e,t))return!0;if(e==null&&t!=null||e!=null&&t==null)return!1;if(typeof e?.isEqual=="function"&&typeof t?.isEqual=="function")return e.isEqual(t);if(typeof e=="function"&&typeof t=="function")return e.toString()===t.toString();if(Kl(e)&&Kl(t))return Km(Array.from(e),Array.from(t));if(typeof e!="object"||typeof t!="object")return!1;const n=Object.keys(t??Object.create(null)),r=n.length;for(let o=0;oArray.isArray(e),Ym=e=>e===!0||e===!1,Yl=e=>e!=null&&typeof e=="object",Xt=e=>Yl(e)&&!Cr(e),go=e=>typeof e=="string",Qt=e=>typeof e=="function",Xm=e=>e==null,Dt=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),Qm=e=>Object.prototype.toString.call(e),Xl=Function.prototype.toString,Jm=Xl.call(Object),Zm=e=>{if(!Yl(e)||Qm(e)!="[object Object]"||n0(e))return!1;const t=Object.getPrototypeOf(e);if(t===null)return!0;const n=Dt(t,"constructor")&&t.constructor;return typeof n=="function"&&n instanceof n&&Xl.call(n)==Jm},e0=e=>typeof e=="object"&&e!==null&&"$$typeof"in e&&"props"in e,t0=e=>typeof e=="object"&&e!==null&&"__v_isVNode"in e,n0=e=>e0(e)||t0(e),po=(e,...t)=>(typeof e=="function"?e(...t):e)??void 0,r0=e=>e(),o0=()=>{},mo=(...e)=>(...t)=>{e.forEach(function(n){n?.(...t)})},i0=(()=>{let e=0;return()=>(e++,e.toString(36))})();function xt(e,t,...n){if(e in t){const o=t[e];return Qt(o)?o(...n):o}const r=new Error(`No matching key: ${JSON.stringify(e)} in ${JSON.stringify(Object.keys(t))}`);throw Error.captureStackTrace?.(r,xt),r}var Ql=(e,t)=>{try{return e()}catch(n){return n instanceof Error&&Error.captureStackTrace?.(n,Ql),t?.()}},{floor:Jl,abs:Zl,round:vo,min:s0,max:a0,pow:l0,sign:c0}=Math,gs=e=>Number.isNaN(e),zt=e=>gs(e)?0:e,ec=(e,t)=>(e%t+t)%t,u0=(e,t)=>(e%t+t)%t,d0=(e,t)=>zt(e)>=t,h0=(e,t)=>zt(e)<=t,f0=(e,t,n)=>{const r=zt(e),o=t==null||r>=t,i=n==null||r<=n;return o&&i},g0=(e,t,n)=>vo((zt(e)-t)/n)*n+t,Ve=(e,t,n)=>s0(a0(zt(e),t),n),p0=(e,t,n)=>(zt(e)-t)/(n-t),m0=(e,t,n,r)=>Ve(g0(e*(n-t)+t,t,r),t,n),tc=(e,t)=>{let n=e,r=t.toString(),o=r.indexOf("."),i=o>=0?r.length-o:0;if(i>0){let s=l0(10,i);n=vo(n*s)/s}return n},ps=(e,t)=>typeof t=="number"?Jl(e*t+.5)/t:vo(e),nc=(e,t,n,r)=>{const o=t!=null?Number(t):0,i=Number(n),s=(e-o)%r;let a=Zl(s)*2>=r?e+c0(s)*(r-Zl(s)):e-s;if(a=tc(a,r),!gs(o)&&ai){const l=Jl((i-o)/r),c=o+l*r;a=l<=0||c{const r=Math.pow(n,t);return vo(e*r)/r},rc=e=>{if(!Number.isFinite(e))return 0;let t=1,n=0;for(;Math.round(e*t)/t!==e;)t*=10,n+=1;return n},oc=(e,t,n)=>{let r=t==="+"?e+n:e-n;if(e%1!==0||n%1!==0){const o=10**Math.max(rc(e),rc(n));e=Math.round(e*o),n=Math.round(n*o),r=t==="+"?e+n:e-n,r/=o}return r},v0=(e,t)=>oc(zt(e),"+",t),b0=(e,t)=>oc(zt(e),"-",t);function bo(e){if(!Zm(e)||e===void 0)return e;const t=Reflect.ownKeys(e).filter(r=>typeof r=="string"),n={};for(const r of t){const o=e[r];o!==void 0&&(n[r]=bo(o))}return n}function y0(e,t=Object.is){let n={...e};const r=new Set,o=u=>(r.add(u),()=>r.delete(u)),i=()=>{r.forEach(u=>u())};return{subscribe:o,get:u=>n[u],set:(u,h)=>{t(n[u],h)||(n[u]=h,i())},update:u=>{let h=!1;for(const d in u){const m=u[d];m!==void 0&&!t(n[d],m)&&(n[d]=m,h=!0)}h&&i()},snapshot:()=>({...n})}}function wr(...e){e.length===1?e[0]:e[1],e.length===2&&e[0]}function ic(e,t){if(e==null)throw new Error(t())}function x0(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Mn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??x0(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[CT,sc]=Mn({name:"EnvironmentContext",hookName:"useEnvironmentContext",providerName:"",strict:!1,defaultValue:{getRootNode:()=>document,getDocument:()=>document,getWindow:()=>window}});function k0(e){if(!e)return;const t=e.selectionStart??0,n=e.selectionEnd??0;Math.abs(n-t)===0&&t===0&&e.setSelectionRange(e.value.length,e.value.length)}var ac=e=>Math.max(0,Math.min(1,e)),C0=(e,t)=>e.map((n,r)=>e[(Math.max(t,0)+r)%e.length]),lc=()=>{},yo=e=>typeof e=="object"&&e!==null,w0=2147483647,S0=1,E0=9,O0=11,Te=e=>yo(e)&&e.nodeType===S0&&typeof e.nodeName=="string",ms=e=>yo(e)&&e.nodeType===E0,P0=e=>yo(e)&&e===e.window,cc=e=>Te(e)?e.localName||"":"#document";function R0(e){return["html","body","#document"].includes(cc(e))}var I0=e=>yo(e)&&e.nodeType!==void 0,jn=e=>I0(e)&&e.nodeType===O0&&"host"in e,T0=e=>Te(e)&&e.localName==="input",N0=e=>!!e?.matches("a[href]"),A0=e=>Te(e)?e.offsetWidth>0||e.offsetHeight>0||e.getClientRects().length>0:!1;function _0(e){if(!e)return!1;const t=e.getRootNode();return Sr(t)===e}var V0=/(textarea|select)/;function uc(e){if(e==null||!Te(e))return!1;try{return T0(e)&&e.selectionStart!=null||V0.test(e.localName)||e.isContentEditable||e.getAttribute("contenteditable")==="true"||e.getAttribute("contenteditable")===""}catch{return!1}}function Jt(e,t){if(!e||!t||!Te(e)||!Te(t))return!1;const n=t.getRootNode?.();if(e===t||e.contains(t))return!0;if(n&&jn(n)){let r=t;for(;r;){if(e===r)return!0;r=r.parentNode||r.host}}return!1}function Fe(e){return ms(e)?e:P0(e)?e.document:e?.ownerDocument??document}function F0(e){return Fe(e).documentElement}function be(e){return jn(e)?be(e.host):ms(e)?e.defaultView??window:Te(e)?e.ownerDocument?.defaultView??window:window}function Sr(e){let t=e.activeElement;for(;t?.shadowRoot;){const n=t.shadowRoot.activeElement;if(!n||n===t)break;t=n}return t}function L0(e){if(cc(e)==="html")return e;const t=e.assignedSlot||e.parentNode||jn(e)&&e.host||F0(e);return jn(t)?t.host:t}var vs=new WeakMap;function xo(e){return vs.has(e)||vs.set(e,be(e).getComputedStyle(e)),vs.get(e)}var ko=()=>typeof document<"u";function D0(){return navigator.userAgentData?.platform??navigator.platform}function z0(){const e=navigator.userAgentData;return e&&Array.isArray(e.brands)?e.brands.map(({brand:t,version:n})=>`${t}/${n}`).join(" "):navigator.userAgent}var bs=e=>ko()&&e.test(D0()),dc=e=>ko()&&e.test(z0()),M0=e=>ko()&&e.test(navigator.vendor),hc=()=>ko()&&!!navigator.maxTouchPoints,j0=()=>bs(/^iPhone/i),$0=()=>bs(/^iPad/i)||wo()&&navigator.maxTouchPoints>1,Co=()=>j0()||$0(),B0=()=>wo()||Co(),wo=()=>bs(/^Mac/i),fc=()=>B0()&&M0(/apple/i),W0=()=>dc(/Firefox/i),H0=()=>dc(/Android/i);function U0(e){return e.composedPath?.()??e.nativeEvent?.composedPath?.()}function $e(e){return U0(e)?.[0]??e.target}function G0(e){return X0(e).isComposing||e.keyCode===229}function q0(e){return e.pointerType===""&&e.isTrusted?!0:H0()&&e.pointerType?e.type==="click"&&e.buttons===1:e.detail===0&&!e.pointerType}var K0=e=>e.button===2||wo()&&e.ctrlKey&&e.button===0,Y0=e=>"touches"in e&&e.touches.length>0;function X0(e){return e.nativeEvent??e}function gc(e,t="client"){const n=Y0(e)?e.touches[0]||e.changedTouches[0]:e;return{x:n[`${t}X`],y:n[`${t}Y`]}}var se=(e,t,n,r)=>{const o=typeof e=="function"?e():e;return o?.addEventListener(t,n,r),()=>{o?.removeEventListener(t,n,r)}};function Q0(e,t){const{type:n="HTMLInputElement",property:r="value"}=t,o=be(e)[n].prototype;return Object.getOwnPropertyDescriptor(o,r)??{}}function J0(e){if(e.localName==="input")return"HTMLInputElement";if(e.localName==="textarea")return"HTMLTextAreaElement";if(e.localName==="select")return"HTMLSelectElement"}function So(e,t,n="value"){if(!e)return;const r=J0(e);r&&Q0(e,{type:r,property:n}).set?.call(e,t),e.setAttribute(n,t)}function Z0(e,t){const{value:n,bubbles:r=!0}=t;if(!e)return;const o=be(e);e instanceof o.HTMLInputElement&&(So(e,`${n}`),e.dispatchEvent(new o.Event("input",{bubbles:r})))}function ev(e){return tv(e)?e.form:e.closest("form")}function tv(e){return e.matches("textarea, input, select, button")}function nv(e,t){if(!e)return;const n=ev(e),r=o=>{o.defaultPrevented||t()};return n?.addEventListener("reset",r,{passive:!0}),()=>n?.removeEventListener("reset",r)}function rv(e,t){const n=e?.closest("fieldset");if(!n)return;t(n.disabled);const r=be(n),o=new r.MutationObserver(()=>t(n.disabled));return o.observe(n,{attributes:!0,attributeFilter:["disabled"]}),()=>o.disconnect()}function ys(e,t){if(!e)return;const{onFieldsetDisabledChange:n,onFormReset:r}=t,o=[nv(e,r),rv(e,n)];return()=>o.forEach(i=>i?.())}var pc=e=>Te(e)&&e.tagName==="IFRAME",ov=e=>!Number.isNaN(parseInt(e.getAttribute("tabindex")||"0",10)),iv=e=>parseInt(e.getAttribute("tabindex")||"0",10)<0,xs="input:not([type='hidden']):not([disabled]), select:not([disabled]), textarea:not([disabled]), a[href], button:not([disabled]), [tabindex], iframe, object, embed, area[href], audio[controls], video[controls], [contenteditable]:not([contenteditable='false']), details > summary:first-of-type",mc=(e,t=!1)=>{if(!e)return[];const n=Array.from(e.querySelectorAll(xs));(t==!0||t=="if-empty"&&n.length===0)&&Te(e)&&Mt(e)&&n.unshift(e);const o=n.filter(Mt);return o.forEach((i,s)=>{if(pc(i)&&i.contentDocument){const a=i.contentDocument.body;o.splice(s,1,...mc(a))}}),o};function Mt(e){return!e||e.closest("[inert]")?!1:e.matches(xs)&&A0(e)}function ks(e,t){if(!e)return[];const r=Array.from(e.querySelectorAll(xs)).filter(Zt);return r.forEach((o,i)=>{if(pc(o)&&o.contentDocument){const s=o.contentDocument.body,a=ks(s);r.splice(i,1,...a)}}),r.length,r}function Zt(e){return e!=null&&e.tabIndex>0?!0:Mt(e)&&!iv(e)}function Er(e){return e.tabIndex<0&&(/^(audio|video|details)$/.test(e.localName)||uc(e))&&!ov(e)?0:e.tabIndex}function Cs(e){const{root:t,getInitialEl:n,filter:r,enabled:o=!0}=e;if(!o)return;let i=null;if(i||(i=typeof n=="function"?n():n),i||(i=t?.querySelector("[data-autofocus],[autofocus]")),!i){const s=ks(t);i=r?s.filter(r)[0]:s[0]}return i||t||void 0}function ws(e){const t=new Set;function n(r){const o=globalThis.requestAnimationFrame(r);t.add(()=>globalThis.cancelAnimationFrame(o))}return n(()=>n(e)),function(){t.forEach(o=>o())}}function G(e){let t;const n=globalThis.requestAnimationFrame(()=>{t=e()});return()=>{globalThis.cancelAnimationFrame(n),t?.()}}function sv(e,t,n){const r=G(()=>{e.removeEventListener(t,o,!0),n()}),o=()=>{r(),n()};return e.addEventListener(t,o,{once:!0,capture:!0}),r}function av(e,t){if(!e)return;const{attributes:n,callback:r}=t,o=e.ownerDocument.defaultView||window,i=new o.MutationObserver(s=>{for(const a of s)a.type==="attributes"&&a.attributeName&&n.includes(a.attributeName)&&r(a)});return i.observe(e,{attributes:!0,attributeFilter:n}),()=>i.disconnect()}function Eo(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(av(i,t))})),()=>{o.forEach(i=>i?.())}}function vc(e){const t=()=>{const n=be(e);e.dispatchEvent(new n.MouseEvent("click"))};W0()?sv(e,"keyup",t):queueMicrotask(t)}function Oo(e){const t=L0(e);return R0(t)?Fe(t).body:Te(t)&&Ss(t)?t:Oo(t)}function bc(e,t=[]){const n=Oo(e),r=n===e.ownerDocument.body,o=be(n);return r?t.concat(o,o.visualViewport||[],Ss(n)?n:[]):t.concat(n,bc(n,[]))}var lv=/auto|scroll|overlay|hidden|clip/,cv=new Set(["inline","contents"]);function Ss(e){const t=be(e),{overflow:n,overflowX:r,overflowY:o,display:i}=t.getComputedStyle(e);return lv.test(n+o+r)&&!cv.has(i)}function uv(e){return e.scrollHeight>e.clientHeight||e.scrollWidth>e.clientWidth}function Po(e,t){const{rootEl:n,...r}=t||{};!e||!n||!Ss(n)||!uv(n)||e.scrollIntoView(r)}function yc(e,t){const{left:n,top:r,width:o,height:i}=t.getBoundingClientRect(),s={x:e.x-n,y:e.y-r},a={x:ac(s.x/o),y:ac(s.y/i)};function l(c={}){const{dir:u="ltr",orientation:h="horizontal",inverted:d}=c,m=typeof d=="object"?d.x:d,f=typeof d=="object"?d.y:d;return h==="horizontal"?u==="rtl"||m?1-a.x:a.x:f?1-a.y:a.y}return{offset:s,percent:a,getPercentValue:l}}function dv(e,t){const n=e.body,r="pointerLockElement"in e||"mozPointerLockElement"in e,o=()=>!!e.pointerLockElement;function i(){}function s(l){o(),console.error("PointerLock error occurred:",l),e.exitPointerLock()}if(!r)return;try{n.requestPointerLock()}catch{}const a=[se(e,"pointerlockchange",i,!1),se(e,"pointerlockerror",s,!1)];return()=>{a.forEach(l=>l()),e.exitPointerLock()}}var $n="default",Es="",Ro=new WeakMap;function hv(e={}){const{target:t,doc:n}=e,r=n??document,o=r.documentElement;return Co()?($n==="default"&&(Es=o.style.webkitUserSelect,o.style.webkitUserSelect="none"),$n="disabled"):t&&(Ro.set(t,t.style.userSelect),t.style.userSelect="none"),()=>fv({target:t,doc:r})}function fv(e={}){const{target:t,doc:n}=e,o=(n??document).documentElement;if(Co()){if($n!=="disabled")return;$n="restoring",setTimeout(()=>{ws(()=>{$n==="restoring"&&(o.style.webkitUserSelect==="none"&&(o.style.webkitUserSelect=Es||""),Es="",$n="default")})},300)}else if(t&&Ro.has(t)){const i=Ro.get(t);t.style.userSelect==="none"&&(t.style.userSelect=i??""),t.getAttribute("style")===""&&t.removeAttribute("style"),Ro.delete(t)}}function xc(e={}){const{defer:t,target:n,...r}=e,o=t?G:s=>s(),i=[];return i.push(o(()=>{const s=typeof n=="function"?n():n;i.push(hv({...r,target:s}))})),()=>{i.forEach(s=>s?.())}}function gv(e,t){const{onPointerMove:n,onPointerUp:r}=t,o=a=>{const l=gc(a),c=Math.sqrt(l.x**2+l.y**2),u=a.pointerType==="touch"?10:5;if(!(c{const l=gc(a);r({point:l,event:a})},s=[se(e,"pointermove",o,!1),se(e,"pointerup",i,!1),se(e,"pointercancel",i,!1),se(e,"contextmenu",i,!1),xc({doc:e})];return()=>{s.forEach(a=>a())}}function Io(e,t){return Array.from(e?.querySelectorAll(t)??[])}function pv(e,t){return e?.querySelector(t)??null}var Os=e=>e.id;function mv(e,t,n=Os){return e.find(r=>n(r)===t)}function Ps(e,t,n=Os){const r=mv(e,t,n);return r?e.indexOf(r):-1}function vv(e,t,n=!0){let r=Ps(e,t);return r=n?(r+1)%e.length:Math.min(r+1,e.length-1),e[r]}function bv(e,t,n=!0){let r=Ps(e,t);return r===-1?n?e[e.length-1]:null:(r=n?(r-1+e.length)%e.length:Math.max(0,r-1),e[r])}var yv=e=>e.split("").map(t=>{const n=t.charCodeAt(0);return n>0&&n<128?t:n>=128&&n<=255?`/x${n.toString(16)}`.replace("/","\\"):""}).join("").trim(),xv=e=>yv(e.dataset?.valuetext??e.textContent??""),kv=(e,t)=>e.trim().toLowerCase().startsWith(t.toLowerCase());function Cv(e,t,n,r=Os){const o=n?Ps(e,n,r):-1;let i=n?C0(e,o):e;return t.length===1&&(i=i.filter(a=>r(a)!==n)),i.find(a=>kv(xv(a),t))}function To(e,t){if(!e)return lc;const n=Object.keys(t).reduce((r,o)=>(r[o]=e.style.getPropertyValue(o),r),{});return Object.assign(e.style,t),()=>{Object.assign(e.style,n),e.style.length===0&&e.removeAttribute("style")}}function wv(e,t,n){if(!e)return lc;const r=e.style.getPropertyValue(t);return e.style.setProperty(t,n),()=>{e.style.setProperty(t,r),e.style.length===0&&e.removeAttribute("style")}}function Sv(e,t){const{state:n,activeId:r,key:o,timeout:i=350,itemToId:s}=t,a=n.keysSoFar+o,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a;let u=e.slice();const h=Cv(u,c,r,s);function d(){clearTimeout(n.timer),n.timer=-1}function m(f){n.keysSoFar=f,d(),f!==""&&(n.timer=+setTimeout(()=>{m(""),d()},i))}return m(a),h}var Or=Object.assign(Sv,{defaultOptions:{keysSoFar:"",timer:-1},isValidEvent:Ev});function Ev(e){return e.key.length===1&&!e.ctrlKey&&!e.metaKey}function Ov(e,t,n){const{signal:r}=t;return[new Promise((s,a)=>{const l=setTimeout(()=>{a(new Error(`Timeout of ${n}ms exceeded`))},n);r.addEventListener("abort",()=>{clearTimeout(l),a(new Error("Promise aborted"))}),e.then(c=>{r.aborted||(clearTimeout(l),s(c))}).catch(c=>{r.aborted||(clearTimeout(l),a(c))})}),()=>t.abort()]}function Pv(e,t){const{timeout:n,rootNode:r}=t,o=be(r),i=Fe(r),s=new o.AbortController;return Ov(new Promise(a=>{const l=e();if(l){a(l);return}const c=new o.MutationObserver(()=>{const u=e();u&&u.isConnected&&(c.disconnect(),a(u))});c.observe(i.body,{childList:!0,subtree:!0})}),s,n)}var Rv=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),Iv=/((?:--)?(?:\w+-?)+)\s*:\s*([^;]*)/g,kc=e=>{const t={};let n;for(;n=Iv.exec(e);)t[n[1]]=n[2];return t},Tv=(e,t)=>{if(go(e)){if(go(t))return`${e};${t}`;e=kc(e)}else go(t)&&(t=kc(t));return Object.assign({},e??{},t??{})};function qe(...e){let t={};for(let n of e){if(!n)continue;for(let o in t){if(o.startsWith("on")&&typeof t[o]=="function"&&typeof n[o]=="function"){t[o]=mo(n[o],t[o]);continue}if(o==="className"||o==="class"){t[o]=Rv(t[o],n[o]);continue}if(o==="style"){t[o]=Tv(t[o],n[o]);continue}t[o]=n[o]!==void 0?n[o]:t[o]}for(let o in n)t[o]===void 0&&(t[o]=n[o]);const r=Object.getOwnPropertySymbols(n);for(let o of r)t[o]=n[o]}return t}function Cc(e,t,n){let r=[],o;return i=>{const s=e(i);return(s.length!==r.length||s.some((l,c)=>!Ze(r[c],l)))&&(r=s,o=t(s,i)),o}}function jt(){return{and:(...e)=>function(n){return e.every(r=>n.guard(r))},or:(...e)=>function(n){return e.some(r=>n.guard(r))},not:e=>function(n){return!n.guard(e)}}}function wT(e){return e}function wc(){return{guards:jt(),createMachine:e=>e,choose:e=>function({choose:n}){return n(e)?.actions}}}var Bn=(e=>(e.NotStarted="Not Started",e.Started="Started",e.Stopped="Stopped",e))(Bn||{}),Rs="__init__";function Nv(e){const t=()=>e.getRootNode?.()??document,n=()=>Fe(t());return{...e,getRootNode:t,getDoc:n,getWin:()=>n().defaultView??window,getActiveElement:()=>Sr(t()),isActiveElement:_0,getById:s=>t().getElementById(s)}}function No(...e){return t=>{const n=[];for(const r of e)if(typeof r=="function"){const o=r(t);typeof o=="function"&&n.push(o)}else r&&(r.current=t);if(n.length)return()=>{for(const r of n)r()}}}function Av(e){let t=Object.getOwnPropertyDescriptor(e.props,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning;return n?e.ref:(t=Object.getOwnPropertyDescriptor(e,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning,n?e.props.ref:e.props.ref||e.ref)}const Is=e=>{const t=P.memo(P.forwardRef((n,r)=>{const{asChild:o,children:i,...s}=n;if(!o)return P.createElement(e,{...s,ref:r},i);if(!P.isValidElement(i))return null;const a=P.Children.only(i),l=Av(a);return P.cloneElement(a,{...qe(s,a.props),ref:r?No(r,l):l})}));return t.displayName=e.displayName||e.name,t},kt=(()=>{const e=new Map;return new Proxy(Is,{apply(t,n,r){return Is(r[0])},get(t,n){const r=n;return e.has(r)||e.set(r,Is(r)),e.get(r)}})})(),[ET,_v]=Mn({name:"LocaleContext",hookName:"useLocaleContext",providerName:"",strict:!1,defaultValue:{dir:"ltr",locale:"en-US"}}),Sc=()=>(e,t)=>t.reduce((n,r)=>{const[o,i]=n,s=r;return i[s]!==void 0&&(o[s]=i[s]),delete i[s],[o,i]},[{},{...e}]),Ec=e=>Sc()(e,["immediate","lazyMount","onExitComplete","present","skipAnimationOnMount","unmountOnExit"]);function Vv(e){return new Proxy({},{get(t,n){return n==="style"?r=>e({style:r}).style:e}})}var M=()=>e=>Array.from(new Set(e));function Fv(e,t){const{state:n,send:r,context:o}=e,i=n.matches("mounted","unmountSuspended");return{skip:!o.get("initial"),present:i,setNode(s){s&&r({type:"NODE.SET",node:s})},unmount(){r({type:"UNMOUNT"})}}}var Lv={props({props:e}){return{...e,present:!!e.present}},initialState({prop:e}){return e("present")?"mounted":"unmounted"},refs(){return{node:null,styles:null}},context({bindable:e}){return{unmountAnimationName:e(()=>({defaultValue:null})),prevAnimationName:e(()=>({defaultValue:null})),present:e(()=>({defaultValue:!1})),initial:e(()=>({sync:!0,defaultValue:!1}))}},exit:["clearInitial","cleanupNode"],watch({track:e,prop:t,send:n}){e([()=>t("present")],()=>{n({type:"PRESENCE.CHANGED"})})},on:{"NODE.SET":{actions:["setupNode"]},"PRESENCE.CHANGED":{actions:["setInitial","syncPresence"]}},states:{mounted:{on:{UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]},"UNMOUNT.SUSPEND":{target:"unmountSuspended"}}},unmountSuspended:{effects:["trackAnimationEvents"],on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]},UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]}}},unmounted:{on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]}}}},implementations:{actions:{setInitial:({context:e})=>{e.get("initial")||queueMicrotask(()=>{e.set("initial",!0)})},clearInitial:({context:e})=>{e.set("initial",!1)},invokeOnExitComplete:({prop:e})=>{e("onExitComplete")?.()},setupNode:({refs:e,event:t})=>{e.get("node")!==t.node&&(e.set("node",t.node),e.set("styles",xo(t.node)))},cleanupNode:({refs:e})=>{e.set("node",null),e.set("styles",null)},syncPresence:({context:e,refs:t,send:n,prop:r})=>{const o=r("present");if(o)return n({type:"MOUNT",src:"presence.changed"});const i=t.get("node");if(!o&&i?.ownerDocument.visibilityState==="hidden")return n({type:"UNMOUNT",src:"visibilitychange"});G(()=>{const s=Ao(t.get("styles"));e.set("unmountAnimationName",s),s==="none"||s===e.get("prevAnimationName")||t.get("styles")?.display==="none"||t.get("styles")?.animationDuration==="0s"?n({type:"UNMOUNT",src:"presence.changed"}):n({type:"UNMOUNT.SUSPEND"})})},setPrevAnimationName:({context:e,refs:t})=>{G(()=>{e.set("prevAnimationName",Ao(t.get("styles")))})},clearPrevAnimationName:({context:e})=>{e.set("prevAnimationName",null)}},effects:{trackAnimationEvents:({context:e,refs:t,send:n})=>{const r=t.get("node");if(!r)return;const o=a=>{(a.composedPath?.()?.[0]??a.target)===r&&e.set("prevAnimationName",Ao(t.get("styles")))},i=a=>{const l=Ao(t.get("styles"));$e(a)===r&&l===e.get("unmountAnimationName")&&n({type:"UNMOUNT",src:"animationend"})};r.addEventListener("animationstart",o),r.addEventListener("animationcancel",i),r.addEventListener("animationend",i);const s=To(r,{animationFillMode:"forwards"});return()=>{r.removeEventListener("animationstart",o),r.removeEventListener("animationcancel",i),r.removeEventListener("animationend",i),ws(()=>s())}}}}};function Ao(e){return e?.animationName||"none"}M()(["onExitComplete","present","immediate"]);var Oc=typeof globalThis.document<"u"?P.useLayoutEffect:P.useEffect;function _o(e){const t=e().value??e().defaultValue,n=e().isEqual??Object.is,[r]=P.useState(t),[o,i]=P.useState(r),s=e().value!==void 0,a=P.useRef(o);a.current=s?e().value:o;const l=P.useRef(a.current);Oc(()=>{l.current=a.current},[o,e().value]);const c=h=>{const d=l.current,m=Qt(h)?h(d):h;e().debug&&console.log(`[bindable > ${e().debug}] setValue`,{next:m,prev:d}),s||i(m),n(m,d)||e().onChange?.(m,d)};function u(){return s?e().value:o}return{initial:r,ref:a,get:u,set(h){(e().sync?Ft.flushSync:r0)(()=>c(h))},invoke(h,d){e().onChange?.(h,d)},hash(h){return e().hash?.(h)??String(h)}}}_o.cleanup=e=>{P.useEffect(()=>e,[])},_o.ref=e=>{const t=P.useRef(e);return{get:()=>t.current,set:n=>{t.current=n}}};function Dv(e){const t=P.useRef(e);return{get(n){return t.current[n]},set(n,r){t.current[n]=r}}}var zv=(e,t)=>{const n=P.useRef(!1),r=P.useRef(!1);P.useEffect(()=>{if(n.current&&r.current)return t();r.current=!0},[...(e??[]).map(o=>typeof o=="function"?o():o)]),P.useEffect(()=>(n.current=!0,()=>{n.current=!1}),[])};function Pc(e,t={}){const n=P.useMemo(()=>{const{id:T,ids:S,getRootNode:_}=t;return Nv({id:T,ids:S,getRootNode:_})},[t]),r=(...T)=>{e.debug&&console.log(...T)},o=e.props?.({props:bo(t),scope:n})??t,i=Mv(o),s=e.context?.({prop:i,bindable:_o,scope:n,flush:Ic,getContext(){return l},getComputed(){return k},getRefs(){return p},getEvent(){return m()}}),a=Rc(s),l={get(T){return a.current?.[T].ref.current},set(T,S){a.current?.[T].set(S)},initial(T){return a.current?.[T].initial},hash(T){const S=a.current?.[T].get();return a.current?.[T].hash(S)}},c=P.useRef(new Map),u=P.useRef(null),h=P.useRef(null),d=P.useRef({type:""}),m=()=>({...d.current,current(){return d.current},previous(){return h.current}}),f=()=>({...O,matches(...T){return T.includes(O.ref.current)},hasTag(T){return!!e.states[O.ref.current]?.tags?.includes(T)}}),p=Dv(e.refs?.({prop:i,context:l})??{}),v=()=>({state:f(),context:l,event:m(),prop:i,send:A,action:b,guard:y,track:zv,refs:p,computed:k,flush:Ic,scope:n,choose:w}),b=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(D=>{const z=e.implementations?.actions?.[D];return z||wr(`[zag-js] No implementation found for action "${JSON.stringify(D)}"`),z});for(const D of _)D?.(v())},y=T=>Qt(T)?T(v()):e.implementations?.guards?.[T](v()),x=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(z=>{const W=e.implementations?.effects?.[z];return W||wr(`[zag-js] No implementation found for effect "${JSON.stringify(z)}"`),W}),D=[];for(const z of _){const W=z?.(v());W&&D.push(W)}return()=>D.forEach(z=>z?.())},w=T=>fo(T).find(S=>{let _=!S.guard;return go(S.guard)?_=!!y(S.guard):Qt(S.guard)&&(_=S.guard(v())),_}),k=T=>{ic(e.computed,()=>"[zag-js] No computed object found on machine");const S=e.computed[T];return S({context:l,event:m(),prop:i,refs:p,scope:n,computed:k})},O=_o(()=>({defaultValue:e.initialState({prop:i}),onChange(T,S){S&&(c.current.get(S)?.(),c.current.delete(S)),S&&b(e.states[S]?.exit),b(u.current?.actions);const _=x(e.states[T]?.effects);if(_&&c.current.set(T,_),S===Rs){b(e.entry);const D=x(e.effects);D&&c.current.set(Rs,D)}b(e.states[T]?.entry)}})),R=P.useRef(void 0),I=P.useRef(Bn.NotStarted);Oc(()=>{queueMicrotask(()=>{const _=I.current===Bn.Started;I.current=Bn.Started,r(_?"rehydrating...":"initializing...");const D=R.current??O.initial;O.invoke(D,_?O.get():Rs)});const T=c.current,S=O.ref.current;return()=>{r("unmounting..."),R.current=S,I.current=Bn.Stopped,T.forEach(_=>_?.()),c.current=new Map,u.current=null,queueMicrotask(()=>{b(e.exit)})}},[]);const N=()=>"ref"in O?O.ref.current:O.get(),A=T=>{queueMicrotask(()=>{if(I.current!==Bn.Started)return;h.current=d.current,d.current=T;let S=N();const _=e.states[S].on?.[T.type]??e.on?.[T.type],D=w(_);if(!D)return;u.current=D;const z=D.target??S;r("transition",T.type,D.target||S,`(${D.actions})`);const W=z!==S;W?Ft.flushSync(()=>O.set(z)):D.reenter&&!W?O.invoke(S,S):b(D.actions??[])})};return e.watch?.(v()),{state:f(),send:A,context:l,prop:i,scope:n,refs:p,computed:k,event:m(),getStatus:()=>I.current}}function Rc(e){const t=P.useRef(e);return t.current=e,t}function Mv(e){const t=Rc(e);return function(r){return t.current[r]}}function Ic(e){queueMicrotask(()=>{Ft.flushSync(()=>e())})}var jv=Vv(e=>e);function $v(e,t={}){const{sync:n=!1}=t,r=Bv(e);return P.useCallback((...o)=>n?queueMicrotask(()=>r.current?.(...o)):r.current?.(...o),[n,r])}function Bv(e){const t=P.useRef(e);return t.current=e,t}const Ts=(e={})=>{const{lazyMount:t,unmountOnExit:n,present:r,skipAnimationOnMount:o=!1,...i}=e,s=P.useRef(!1),a={...i,present:r,onExitComplete:$v(e.onExitComplete)},l=Pc(Lv,a),c=Fv(l);c.present&&(s.current=!0);const u=!c.present&&!s.current&&t||n&&!c.present&&s.current,h=()=>({"data-state":c.skip&&o?void 0:r?"open":"closed",hidden:!c.present});return{ref:c.setNode,getPresenceProps:h,present:c.present,unmounted:u}},[Tc,Ns]=Mn({name:"PresenceContext",hookName:"usePresenceContext",providerName:""}),Vo=ve("span"),{withContext:Wv}=Lt({key:"text"}),Le=Wv("p");function Nc(e,t=[]){const n=P.useRef(()=>{throw new Error("Cannot call an event handler while rendering.")});return P.useInsertionEffect(()=>{n.current=e}),P.useCallback((...r)=>n.current?.(...r),t)}function en(e={}){const t=Nc(e.onOpen),n=Nc(e.onClose),[r,o]=P.useState(e.defaultOpen||!1),i=e.open!==void 0?e.open:r,s=e.open!==void 0,a=P.useCallback(()=>{s||o(!1),n?.()},[s,n]),l=P.useCallback(()=>{s||o(!0),t?.()},[s,t]),c=P.useCallback(()=>{i?a():l()},[i,l,a]);return{open:i,onOpen:l,onClose:a,onToggle:c,setOpen:o}}var j=(e,t=[])=>({parts:(...n)=>{if(Hv(t))return j(e,n);throw new Error("createAnatomy().parts(...) should only be called once. Did you mean to use .extendWith(...) ?")},extendWith:(...n)=>j(e,[...t,...n]),omit:(...n)=>j(e,t.filter(r=>!n.includes(r))),rename:n=>j(n,t),keys:()=>t,build:()=>[...new Set(t)].reduce((n,r)=>Object.assign(n,{[r]:{selector:[`&[data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`,`& [data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`].join(", "),attrs:{"data-scope":Wn(e),"data-part":Wn(r)}}}),{})}),Wn=e=>e.replace(/([A-Z])([A-Z])/g,"$1-$2").replace(/([a-z])([A-Z])/g,"$1-$2").replace(/[\s_]+/g,"-").toLowerCase(),Hv=e=>e.length===0,Ac=j("collapsible").parts("root","trigger","content","indicator");Ac.build(),M()(["dir","disabled","getRootNode","id","ids","onExitComplete","onOpenChange","defaultOpen","open"]);var Uv=Object.defineProperty,Gv=(e,t,n)=>t in e?Uv(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,As=(e,t,n)=>Gv(e,t+"",n),qv=(e,t)=>{if(Object.keys(e).length!==Object.keys(t).length)return!1;for(let n in e)if(e[n]!==t[n])return!1;return!0},_s=class{toHexInt(){return this.toFormat("rgba").toHexInt()}getChannelValue(e){if(e in this)return this[e];throw new Error("Unsupported color channel: "+e)}getChannelValuePercent(e,t){const n=t??this.getChannelValue(e),{minValue:r,maxValue:o}=this.getChannelRange(e);return p0(n,r,o)}getChannelPercentValue(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=m0(t,n,r,o);return nc(i,n,r,o)}withChannelValue(e,t){const{minValue:n,maxValue:r}=this.getChannelRange(e);if(e in this){let o=this.clone();return o[e]=Ve(t,n,r),o}throw new Error("Unsupported color channel: "+e)}getColorAxes(e){let{xChannel:t,yChannel:n}=e,r=t||this.getChannels().find(s=>s!==n),o=n||this.getChannels().find(s=>s!==r),i=this.getChannels().find(s=>s!==r&&s!==o);return{xChannel:r,yChannel:o,zChannel:i}}incrementChannel(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=nc(Ve(this.getChannelValue(e)+t,n,r),n,r,o);return this.withChannelValue(e,i)}decrementChannel(e,t){return this.incrementChannel(e,-t)}isEqual(e){return qv(this.toJSON(),e.toJSON())&&this.getChannelValue("alpha")===e.getChannelValue("alpha")}},Kv=/^#[\da-f]+$/i,Yv=/^rgba?\((.*)\)$/,Xv=/[^#]/gi,_c=class Hi extends _s{constructor(t,n,r,o){super(),this.red=t,this.green=n,this.blue=r,this.alpha=o}static parse(t){let n=[];if(Kv.test(t)&&[4,5,7,9].includes(t.length)){const o=(t.length<6?t.replace(Xv,"$&$&"):t).slice(1).split("");for(;o.length>0;)n.push(parseInt(o.splice(0,2).join(""),16));n[3]=n[3]!==void 0?n[3]/255:void 0}const r=t.match(Yv);return r?.[1]&&(n=r[1].split(",").map(o=>Number(o.trim())).map((o,i)=>Ve(o,0,i<3?255:1))),n.length<3?void 0:new Hi(n[0],n[1],n[2],n[3]??1)}toString(t){switch(t){case"hex":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")).toUpperCase();case"hexa":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")+Math.round(this.alpha*255).toString(16).padStart(2,"0")).toUpperCase();case"rgb":return`rgb(${this.red}, ${this.green}, ${this.blue})`;case"css":case"rgba":return`rgba(${this.red}, ${this.green}, ${this.blue}, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"hsb":return this.toHSB().toString("hsb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"rgba":return this;case"hsba":return this.toHSB();case"hsla":return this.toHSL();default:throw new Error("Unsupported color conversion: rgb -> "+t)}}toHexInt(){return this.red<<16|this.green<<8|this.blue}toHSB(){const t=this.red/255,n=this.green/255,r=this.blue/255,o=Math.min(t,n,r),i=Math.max(t,n,r),s=i-o,a=i===0?0:s/i;let l=0;if(s!==0){switch(i){case t:l=(n-r)/s+(nNumber(a.trim().replace("%","")));return new Ui(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsl":return`hsl(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%)`;case"css":case"hsla":return`hsla(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%, ${this.alpha})`;case"hsb":return this.toHSB().toString("hsb");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsla":return this;case"hsba":return this.toHSB();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsl -> "+t)}}toHSB(){let t=this.saturation/100,n=this.lightness/100,r=n+t*Math.min(n,1-n);return t=r===0?0:2*(1-n/r),new Ls(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.lightness/100,o=n*Math.min(r,1-r),i=(s,a=(s+t/30)%12)=>r-o*Math.max(Math.min(a-3,9-a,1),-1);return new Vs(Math.round(i(0)*255),Math.round(i(8)*255),Math.round(i(4)*255),oe(this.alpha,2))}clone(){return new Ui(this.hue,this.saturation,this.lightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"lightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="lightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"lightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,l:this.lightness,a:this.alpha}}getFormat(){return"hsla"}getChannels(){return Ui.colorChannels}};As(Vc,"colorChannels",["hue","saturation","lightness"]);var Fs=Vc,Jv=/hsb\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%)\)|hsba\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d(.\d+)?)\)/,Fc=class Gi extends _s{constructor(t,n,r,o){super(),this.hue=t,this.saturation=n,this.brightness=r,this.alpha=o}static parse(t){let n;if(n=t.match(Jv)){const[r,o,i,s]=(n[1]??n[2]).split(",").map(a=>Number(a.trim().replace("%","")));return new Gi(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"css":return this.toHSL().toString("css");case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsb":return`hsb(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%)`;case"hsba":return`hsba(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsba":return this;case"hsla":return this.toHSL();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsb -> "+t)}}toHSL(){let t=this.saturation/100,n=this.brightness/100,r=n*(1-t/2);return t=r===0||r===1?0:(n-r)/Math.min(r,1-r),new Fs(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.brightness/100,o=(i,s=(i+t/60)%6)=>r-n*r*Math.max(Math.min(s,4-s,1),0);return new Vs(Math.round(o(5)*255),Math.round(o(3)*255),Math.round(o(1)*255),oe(this.alpha,2))}clone(){return new Gi(this.hue,this.saturation,this.brightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"brightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="brightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"brightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,b:this.brightness,a:this.alpha}}getFormat(){return"hsba"}getChannels(){return Gi.colorChannels}};As(Fc,"colorChannels",["hue","saturation","brightness"]);var Ls=Fc,Zv="aliceblue:f0f8ff,antiquewhite:faebd7,aqua:00ffff,aquamarine:7fffd4,azure:f0ffff,beige:f5f5dc,bisque:ffe4c4,black:000000,blanchedalmond:ffebcd,blue:0000ff,blueviolet:8a2be2,brown:a52a2a,burlywood:deb887,cadetblue:5f9ea0,chartreuse:7fff00,chocolate:d2691e,coral:ff7f50,cornflowerblue:6495ed,cornsilk:fff8dc,crimson:dc143c,cyan:00ffff,darkblue:00008b,darkcyan:008b8b,darkgoldenrod:b8860b,darkgray:a9a9a9,darkgreen:006400,darkkhaki:bdb76b,darkmagenta:8b008b,darkolivegreen:556b2f,darkorange:ff8c00,darkorchid:9932cc,darkred:8b0000,darksalmon:e9967a,darkseagreen:8fbc8f,darkslateblue:483d8b,darkslategray:2f4f4f,darkturquoise:00ced1,darkviolet:9400d3,deeppink:ff1493,deepskyblue:00bfff,dimgray:696969,dodgerblue:1e90ff,firebrick:b22222,floralwhite:fffaf0,forestgreen:228b22,fuchsia:ff00ff,gainsboro:dcdcdc,ghostwhite:f8f8ff,gold:ffd700,goldenrod:daa520,gray:808080,green:008000,greenyellow:adff2f,honeydew:f0fff0,hotpink:ff69b4,indianred:cd5c5c,indigo:4b0082,ivory:fffff0,khaki:f0e68c,lavender:e6e6fa,lavenderblush:fff0f5,lawngreen:7cfc00,lemonchiffon:fffacd,lightblue:add8e6,lightcoral:f08080,lightcyan:e0ffff,lightgoldenrodyellow:fafad2,lightgrey:d3d3d3,lightgreen:90ee90,lightpink:ffb6c1,lightsalmon:ffa07a,lightseagreen:20b2aa,lightskyblue:87cefa,lightslategray:778899,lightsteelblue:b0c4de,lightyellow:ffffe0,lime:00ff00,limegreen:32cd32,linen:faf0e6,magenta:ff00ff,maroon:800000,mediumaquamarine:66cdaa,mediumblue:0000cd,mediumorchid:ba55d3,mediumpurple:9370d8,mediumseagreen:3cb371,mediumslateblue:7b68ee,mediumspringgreen:00fa9a,mediumturquoise:48d1cc,mediumvioletred:c71585,midnightblue:191970,mintcream:f5fffa,mistyrose:ffe4e1,moccasin:ffe4b5,navajowhite:ffdead,navy:000080,oldlace:fdf5e6,olive:808000,olivedrab:6b8e23,orange:ffa500,orangered:ff4500,orchid:da70d6,palegoldenrod:eee8aa,palegreen:98fb98,paleturquoise:afeeee,palevioletred:d87093,papayawhip:ffefd5,peachpuff:ffdab9,peru:cd853f,pink:ffc0cb,plum:dda0dd,powderblue:b0e0e6,purple:800080,rebeccapurple:663399,red:ff0000,rosybrown:bc8f8f,royalblue:4169e1,saddlebrown:8b4513,salmon:fa8072,sandybrown:f4a460,seagreen:2e8b57,seashell:fff5ee,sienna:a0522d,silver:c0c0c0,skyblue:87ceeb,slateblue:6a5acd,slategray:708090,snow:fffafa,springgreen:00ff7f,steelblue:4682b4,tan:d2b48c,teal:008080,thistle:d8bfd8,tomato:ff6347,turquoise:40e0d0,violet:ee82ee,wheat:f5deb3,white:ffffff,whitesmoke:f5f5f5,yellow:ffff00,yellowgreen:9acd32",eb=e=>{const t=new Map,n=e.split(",");for(let r=0;r{if(Lc.has(e))return Fo(Lc.get(e));const t=Vs.parse(e)||Ls.parse(e)||Fs.parse(e);if(!t){const n=new Error("Invalid color value: "+e);throw Error.captureStackTrace?.(n,Fo),n}return t};const tb=["top","right","bottom","left"],$t=Math.min,Be=Math.max,Lo=Math.round,Do=Math.floor,ct=e=>({x:e,y:e}),nb={left:"right",right:"left",bottom:"top",top:"bottom"},rb={start:"end",end:"start"};function Ds(e,t,n){return Be(e,$t(t,n))}function Ct(e,t){return typeof e=="function"?e(t):e}function wt(e){return e.split("-")[0]}function Hn(e){return e.split("-")[1]}function zs(e){return e==="x"?"y":"x"}function Ms(e){return e==="y"?"height":"width"}const ob=new Set(["top","bottom"]);function ut(e){return ob.has(wt(e))?"y":"x"}function js(e){return zs(ut(e))}function ib(e,t,n){n===void 0&&(n=!1);const r=Hn(e),o=js(e),i=Ms(o);let s=o==="x"?r===(n?"end":"start")?"right":"left":r==="start"?"bottom":"top";return t.reference[i]>t.floating[i]&&(s=zo(s)),[s,zo(s)]}function sb(e){const t=zo(e);return[$s(e),t,$s(t)]}function $s(e){return e.replace(/start|end/g,t=>rb[t])}const Dc=["left","right"],zc=["right","left"],ab=["top","bottom"],lb=["bottom","top"];function cb(e,t,n){switch(e){case"top":case"bottom":return n?t?zc:Dc:t?Dc:zc;case"left":case"right":return t?ab:lb;default:return[]}}function ub(e,t,n,r){const o=Hn(e);let i=cb(wt(e),n==="start",r);return o&&(i=i.map(s=>s+"-"+o),t&&(i=i.concat(i.map($s)))),i}function zo(e){return e.replace(/left|right|bottom|top/g,t=>nb[t])}function db(e){return{top:0,right:0,bottom:0,left:0,...e}}function Mc(e){return typeof e!="number"?db(e):{top:e,right:e,bottom:e,left:e}}function Mo(e){const{x:t,y:n,width:r,height:o}=e;return{width:r,height:o,top:n,left:t,right:t+r,bottom:n+o,x:t,y:n}}function jc(e,t,n){let{reference:r,floating:o}=e;const i=ut(t),s=js(t),a=Ms(s),l=wt(t),c=i==="y",u=r.x+r.width/2-o.width/2,h=r.y+r.height/2-o.height/2,d=r[a]/2-o[a]/2;let m;switch(l){case"top":m={x:u,y:r.y-o.height};break;case"bottom":m={x:u,y:r.y+r.height};break;case"right":m={x:r.x+r.width,y:h};break;case"left":m={x:r.x-o.width,y:h};break;default:m={x:r.x,y:r.y}}switch(Hn(t)){case"start":m[s]-=d*(n&&c?-1:1);break;case"end":m[s]+=d*(n&&c?-1:1);break}return m}const hb=async(e,t,n)=>{const{placement:r="bottom",strategy:o="absolute",middleware:i=[],platform:s}=n,a=i.filter(Boolean),l=await(s.isRTL==null?void 0:s.isRTL(t));let c=await s.getElementRects({reference:e,floating:t,strategy:o}),{x:u,y:h}=jc(c,r,l),d=r,m={},f=0;for(let p=0;p({name:"arrow",options:e,async fn(t){const{x:n,y:r,placement:o,rects:i,platform:s,elements:a,middlewareData:l}=t,{element:c,padding:u=0}=Ct(e,t)||{};if(c==null)return{};const h=Mc(u),d={x:n,y:r},m=js(o),f=Ms(m),p=await s.getDimensions(c),v=m==="y",b=v?"top":"left",y=v?"bottom":"right",x=v?"clientHeight":"clientWidth",w=i.reference[f]+i.reference[m]-d[m]-i.floating[f],k=d[m]-i.reference[m],O=await(s.getOffsetParent==null?void 0:s.getOffsetParent(c));let R=O?O[x]:0;(!R||!await(s.isElement==null?void 0:s.isElement(O)))&&(R=a.floating[x]||i.floating[f]);const I=w/2-k/2,N=R/2-p[f]/2-1,A=$t(h[b],N),T=$t(h[y],N),S=A,_=R-p[f]-T,D=R/2-p[f]/2+I,z=Ds(S,D,_),W=!l.arrow&&Hn(o)!=null&&D!==z&&i.reference[f]/2-(DD<=0)){var T,S;const D=(((T=i.flip)==null?void 0:T.index)||0)+1,z=R[D];if(z&&(!(h==="alignment"?y!==ut(z):!1)||A.every(ee=>ut(ee.placement)===y?ee.overflows[0]>0:!0)))return{data:{index:D,overflows:A},reset:{placement:z}};let W=(S=A.filter(J=>J.overflows[0]<=0).sort((J,ee)=>J.overflows[1]-ee.overflows[1])[0])==null?void 0:S.placement;if(!W)switch(m){case"bestFit":{var _;const J=(_=A.filter(ee=>{if(O){const $=ut(ee.placement);return $===y||$==="y"}return!0}).map(ee=>[ee.placement,ee.overflows.filter($=>$>0).reduce(($,q)=>$+q,0)]).sort((ee,$)=>ee[1]-$[1])[0])==null?void 0:_[0];J&&(W=J);break}case"initialPlacement":W=a;break}if(o!==W)return{reset:{placement:W}}}return{}}}};function $c(e,t){return{top:e.top-t.height,right:e.right-t.width,bottom:e.bottom-t.height,left:e.left-t.width}}function Bc(e){return tb.some(t=>e[t]>=0)}const pb=function(e){return e===void 0&&(e={}),{name:"hide",options:e,async fn(t){const{rects:n}=t,{strategy:r="referenceHidden",...o}=Ct(e,t);switch(r){case"referenceHidden":{const i=await Pr(t,{...o,elementContext:"reference"}),s=$c(i,n.reference);return{data:{referenceHiddenOffsets:s,referenceHidden:Bc(s)}}}case"escaped":{const i=await Pr(t,{...o,altBoundary:!0}),s=$c(i,n.floating);return{data:{escapedOffsets:s,escaped:Bc(s)}}}default:return{}}}}},Wc=new Set(["left","top"]);async function mb(e,t){const{placement:n,platform:r,elements:o}=e,i=await(r.isRTL==null?void 0:r.isRTL(o.floating)),s=wt(n),a=Hn(n),l=ut(n)==="y",c=Wc.has(s)?-1:1,u=i&&l?-1:1,h=Ct(t,e);let{mainAxis:d,crossAxis:m,alignmentAxis:f}=typeof h=="number"?{mainAxis:h,crossAxis:0,alignmentAxis:null}:{mainAxis:h.mainAxis||0,crossAxis:h.crossAxis||0,alignmentAxis:h.alignmentAxis};return a&&typeof f=="number"&&(m=a==="end"?f*-1:f),l?{x:m*u,y:d*c}:{x:d*c,y:m*u}}const vb=function(e){return e===void 0&&(e=0),{name:"offset",options:e,async fn(t){var n,r;const{x:o,y:i,placement:s,middlewareData:a}=t,l=await mb(t,e);return s===((n=a.offset)==null?void 0:n.placement)&&(r=a.arrow)!=null&&r.alignmentOffset?{}:{x:o+l.x,y:i+l.y,data:{...l,placement:s}}}}},bb=function(e){return e===void 0&&(e={}),{name:"shift",options:e,async fn(t){const{x:n,y:r,placement:o}=t,{mainAxis:i=!0,crossAxis:s=!1,limiter:a={fn:v=>{let{x:b,y}=v;return{x:b,y}}},...l}=Ct(e,t),c={x:n,y:r},u=await Pr(t,l),h=ut(wt(o)),d=zs(h);let m=c[d],f=c[h];if(i){const v=d==="y"?"top":"left",b=d==="y"?"bottom":"right",y=m+u[v],x=m-u[b];m=Ds(y,m,x)}if(s){const v=h==="y"?"top":"left",b=h==="y"?"bottom":"right",y=f+u[v],x=f-u[b];f=Ds(y,f,x)}const p=a.fn({...t,[d]:m,[h]:f});return{...p,data:{x:p.x-n,y:p.y-r,enabled:{[d]:i,[h]:s}}}}}},yb=function(e){return e===void 0&&(e={}),{options:e,fn(t){const{x:n,y:r,placement:o,rects:i,middlewareData:s}=t,{offset:a=0,mainAxis:l=!0,crossAxis:c=!0}=Ct(e,t),u={x:n,y:r},h=ut(o),d=zs(h);let m=u[d],f=u[h];const p=Ct(a,t),v=typeof p=="number"?{mainAxis:p,crossAxis:0}:{mainAxis:0,crossAxis:0,...p};if(l){const x=d==="y"?"height":"width",w=i.reference[d]-i.floating[x]+v.mainAxis,k=i.reference[d]+i.reference[x]-v.mainAxis;mk&&(m=k)}if(c){var b,y;const x=d==="y"?"width":"height",w=Wc.has(wt(o)),k=i.reference[h]-i.floating[x]+(w&&((b=s.offset)==null?void 0:b[h])||0)+(w?0:v.crossAxis),O=i.reference[h]+i.reference[x]+(w?0:((y=s.offset)==null?void 0:y[h])||0)-(w?v.crossAxis:0);fO&&(f=O)}return{[d]:m,[h]:f}}}},xb=function(e){return e===void 0&&(e={}),{name:"size",options:e,async fn(t){var n,r;const{placement:o,rects:i,platform:s,elements:a}=t,{apply:l=()=>{},...c}=Ct(e,t),u=await Pr(t,c),h=wt(o),d=Hn(o),m=ut(o)==="y",{width:f,height:p}=i.floating;let v,b;h==="top"||h==="bottom"?(v=h,b=d===(await(s.isRTL==null?void 0:s.isRTL(a.floating))?"start":"end")?"left":"right"):(b=h,v=d==="end"?"top":"bottom");const y=p-u.top-u.bottom,x=f-u.left-u.right,w=$t(p-u[v],y),k=$t(f-u[b],x),O=!t.middlewareData.shift;let R=w,I=k;if((n=t.middlewareData.shift)!=null&&n.enabled.x&&(I=x),(r=t.middlewareData.shift)!=null&&r.enabled.y&&(R=y),O&&!d){const A=Be(u.left,0),T=Be(u.right,0),S=Be(u.top,0),_=Be(u.bottom,0);m?I=f-2*(A!==0||T!==0?A+T:Be(u.left,u.right)):R=p-2*(S!==0||_!==0?S+_:Be(u.top,u.bottom))}await l({...t,availableWidth:I,availableHeight:R});const N=await s.getDimensions(a.floating);return f!==N.width||p!==N.height?{reset:{rects:!0}}:{}}}};function jo(){return typeof window<"u"}function Un(e){return Hc(e)?(e.nodeName||"").toLowerCase():"#document"}function We(e){var t;return(e==null||(t=e.ownerDocument)==null?void 0:t.defaultView)||window}function dt(e){var t;return(t=(Hc(e)?e.ownerDocument:e.document)||window.document)==null?void 0:t.documentElement}function Hc(e){return jo()?e instanceof Node||e instanceof We(e).Node:!1}function et(e){return jo()?e instanceof Element||e instanceof We(e).Element:!1}function ht(e){return jo()?e instanceof HTMLElement||e instanceof We(e).HTMLElement:!1}function Uc(e){return!jo()||typeof ShadowRoot>"u"?!1:e instanceof ShadowRoot||e instanceof We(e).ShadowRoot}const kb=new Set(["inline","contents"]);function Rr(e){const{overflow:t,overflowX:n,overflowY:r,display:o}=tt(e);return/auto|scroll|overlay|hidden|clip/.test(t+r+n)&&!kb.has(o)}const Cb=new Set(["table","td","th"]);function wb(e){return Cb.has(Un(e))}const Sb=[":popover-open",":modal"];function $o(e){return Sb.some(t=>{try{return e.matches(t)}catch{return!1}})}const Eb=["transform","translate","scale","rotate","perspective"],Ob=["transform","translate","scale","rotate","perspective","filter"],Pb=["paint","layout","strict","content"];function Bs(e){const t=Ws(),n=et(e)?tt(e):e;return Eb.some(r=>n[r]?n[r]!=="none":!1)||(n.containerType?n.containerType!=="normal":!1)||!t&&(n.backdropFilter?n.backdropFilter!=="none":!1)||!t&&(n.filter?n.filter!=="none":!1)||Ob.some(r=>(n.willChange||"").includes(r))||Pb.some(r=>(n.contain||"").includes(r))}function Rb(e){let t=Bt(e);for(;ht(t)&&!Gn(t);){if(Bs(t))return t;if($o(t))return null;t=Bt(t)}return null}function Ws(){return typeof CSS>"u"||!CSS.supports?!1:CSS.supports("-webkit-backdrop-filter","none")}const Ib=new Set(["html","body","#document"]);function Gn(e){return Ib.has(Un(e))}function tt(e){return We(e).getComputedStyle(e)}function Bo(e){return et(e)?{scrollLeft:e.scrollLeft,scrollTop:e.scrollTop}:{scrollLeft:e.scrollX,scrollTop:e.scrollY}}function Bt(e){if(Un(e)==="html")return e;const t=e.assignedSlot||e.parentNode||Uc(e)&&e.host||dt(e);return Uc(t)?t.host:t}function Gc(e){const t=Bt(e);return Gn(t)?e.ownerDocument?e.ownerDocument.body:e.body:ht(t)&&Rr(t)?t:Gc(t)}function Ir(e,t,n){var r;t===void 0&&(t=[]),n===void 0&&(n=!0);const o=Gc(e),i=o===((r=e.ownerDocument)==null?void 0:r.body),s=We(o);if(i){const a=Hs(s);return t.concat(s,s.visualViewport||[],Rr(o)?o:[],a&&n?Ir(a):[])}return t.concat(o,Ir(o,[],n))}function Hs(e){return e.parent&&Object.getPrototypeOf(e.parent)?e.frameElement:null}function qc(e){const t=tt(e);let n=parseFloat(t.width)||0,r=parseFloat(t.height)||0;const o=ht(e),i=o?e.offsetWidth:n,s=o?e.offsetHeight:r,a=Lo(n)!==i||Lo(r)!==s;return a&&(n=i,r=s),{width:n,height:r,$:a}}function Us(e){return et(e)?e:e.contextElement}function qn(e){const t=Us(e);if(!ht(t))return ct(1);const n=t.getBoundingClientRect(),{width:r,height:o,$:i}=qc(t);let s=(i?Lo(n.width):n.width)/r,a=(i?Lo(n.height):n.height)/o;return(!s||!Number.isFinite(s))&&(s=1),(!a||!Number.isFinite(a))&&(a=1),{x:s,y:a}}const Tb=ct(0);function Kc(e){const t=We(e);return!Ws()||!t.visualViewport?Tb:{x:t.visualViewport.offsetLeft,y:t.visualViewport.offsetTop}}function Nb(e,t,n){return t===void 0&&(t=!1),!n||t&&n!==We(e)?!1:t}function tn(e,t,n,r){t===void 0&&(t=!1),n===void 0&&(n=!1);const o=e.getBoundingClientRect(),i=Us(e);let s=ct(1);t&&(r?et(r)&&(s=qn(r)):s=qn(e));const a=Nb(i,n,r)?Kc(i):ct(0);let l=(o.left+a.x)/s.x,c=(o.top+a.y)/s.y,u=o.width/s.x,h=o.height/s.y;if(i){const d=We(i),m=r&&et(r)?We(r):r;let f=d,p=Hs(f);for(;p&&r&&m!==f;){const v=qn(p),b=p.getBoundingClientRect(),y=tt(p),x=b.left+(p.clientLeft+parseFloat(y.paddingLeft))*v.x,w=b.top+(p.clientTop+parseFloat(y.paddingTop))*v.y;l*=v.x,c*=v.y,u*=v.x,h*=v.y,l+=x,c+=w,f=We(p),p=Hs(f)}}return Mo({width:u,height:h,x:l,y:c})}function Wo(e,t){const n=Bo(e).scrollLeft;return t?t.left+n:tn(dt(e)).left+n}function Yc(e,t){const n=e.getBoundingClientRect(),r=n.left+t.scrollLeft-Wo(e,n),o=n.top+t.scrollTop;return{x:r,y:o}}function Ab(e){let{elements:t,rect:n,offsetParent:r,strategy:o}=e;const i=o==="fixed",s=dt(r),a=t?$o(t.floating):!1;if(r===s||a&&i)return n;let l={scrollLeft:0,scrollTop:0},c=ct(1);const u=ct(0),h=ht(r);if((h||!h&&!i)&&((Un(r)!=="body"||Rr(s))&&(l=Bo(r)),ht(r))){const m=tn(r);c=qn(r),u.x=m.x+r.clientLeft,u.y=m.y+r.clientTop}const d=s&&!h&&!i?Yc(s,l):ct(0);return{width:n.width*c.x,height:n.height*c.y,x:n.x*c.x-l.scrollLeft*c.x+u.x+d.x,y:n.y*c.y-l.scrollTop*c.y+u.y+d.y}}function _b(e){return Array.from(e.getClientRects())}function Vb(e){const t=dt(e),n=Bo(e),r=e.ownerDocument.body,o=Be(t.scrollWidth,t.clientWidth,r.scrollWidth,r.clientWidth),i=Be(t.scrollHeight,t.clientHeight,r.scrollHeight,r.clientHeight);let s=-n.scrollLeft+Wo(e);const a=-n.scrollTop;return tt(r).direction==="rtl"&&(s+=Be(t.clientWidth,r.clientWidth)-o),{width:o,height:i,x:s,y:a}}const Xc=25;function Fb(e,t){const n=We(e),r=dt(e),o=n.visualViewport;let i=r.clientWidth,s=r.clientHeight,a=0,l=0;if(o){i=o.width,s=o.height;const u=Ws();(!u||u&&t==="fixed")&&(a=o.offsetLeft,l=o.offsetTop)}const c=Wo(r);if(c<=0){const u=r.ownerDocument,h=u.body,d=getComputedStyle(h),m=u.compatMode==="CSS1Compat"&&parseFloat(d.marginLeft)+parseFloat(d.marginRight)||0,f=Math.abs(r.clientWidth-h.clientWidth-m);f<=Xc&&(i-=f)}else c<=Xc&&(i+=c);return{width:i,height:s,x:a,y:l}}const Lb=new Set(["absolute","fixed"]);function Db(e,t){const n=tn(e,!0,t==="fixed"),r=n.top+e.clientTop,o=n.left+e.clientLeft,i=ht(e)?qn(e):ct(1),s=e.clientWidth*i.x,a=e.clientHeight*i.y,l=o*i.x,c=r*i.y;return{width:s,height:a,x:l,y:c}}function Qc(e,t,n){let r;if(t==="viewport")r=Fb(e,n);else if(t==="document")r=Vb(dt(e));else if(et(t))r=Db(t,n);else{const o=Kc(e);r={x:t.x-o.x,y:t.y-o.y,width:t.width,height:t.height}}return Mo(r)}function Jc(e,t){const n=Bt(e);return n===t||!et(n)||Gn(n)?!1:tt(n).position==="fixed"||Jc(n,t)}function zb(e,t){const n=t.get(e);if(n)return n;let r=Ir(e,[],!1).filter(a=>et(a)&&Un(a)!=="body"),o=null;const i=tt(e).position==="fixed";let s=i?Bt(e):e;for(;et(s)&&!Gn(s);){const a=tt(s),l=Bs(s);!l&&a.position==="fixed"&&(o=null),(i?!l&&!o:!l&&a.position==="static"&&!!o&&Lb.has(o.position)||Rr(s)&&!l&&Jc(e,s))?r=r.filter(u=>u!==s):o=a,s=Bt(s)}return t.set(e,r),r}function Mb(e){let{element:t,boundary:n,rootBoundary:r,strategy:o}=e;const s=[...n==="clippingAncestors"?$o(t)?[]:zb(t,this._c):[].concat(n),r],a=s[0],l=s.reduce((c,u)=>{const h=Qc(t,u,o);return c.top=Be(h.top,c.top),c.right=$t(h.right,c.right),c.bottom=$t(h.bottom,c.bottom),c.left=Be(h.left,c.left),c},Qc(t,a,o));return{width:l.right-l.left,height:l.bottom-l.top,x:l.left,y:l.top}}function jb(e){const{width:t,height:n}=qc(e);return{width:t,height:n}}function $b(e,t,n){const r=ht(t),o=dt(t),i=n==="fixed",s=tn(e,!0,i,t);let a={scrollLeft:0,scrollTop:0};const l=ct(0);function c(){l.x=Wo(o)}if(r||!r&&!i)if((Un(t)!=="body"||Rr(o))&&(a=Bo(t)),r){const m=tn(t,!0,i,t);l.x=m.x+t.clientLeft,l.y=m.y+t.clientTop}else o&&c();i&&!r&&o&&c();const u=o&&!r&&!i?Yc(o,a):ct(0),h=s.left+a.scrollLeft-l.x-u.x,d=s.top+a.scrollTop-l.y-u.y;return{x:h,y:d,width:s.width,height:s.height}}function Gs(e){return tt(e).position==="static"}function Zc(e,t){if(!ht(e)||tt(e).position==="fixed")return null;if(t)return t(e);let n=e.offsetParent;return dt(e)===n&&(n=n.ownerDocument.body),n}function eu(e,t){const n=We(e);if($o(e))return n;if(!ht(e)){let o=Bt(e);for(;o&&!Gn(o);){if(et(o)&&!Gs(o))return o;o=Bt(o)}return n}let r=Zc(e,t);for(;r&&wb(r)&&Gs(r);)r=Zc(r,t);return r&&Gn(r)&&Gs(r)&&!Bs(r)?n:r||Rb(e)||n}const Bb=async function(e){const t=this.getOffsetParent||eu,n=this.getDimensions,r=await n(e.floating);return{reference:$b(e.reference,await t(e.floating),e.strategy),floating:{x:0,y:0,width:r.width,height:r.height}}};function Wb(e){return tt(e).direction==="rtl"}const Hb={convertOffsetParentRelativeRectToViewportRelativeRect:Ab,getDocumentElement:dt,getClippingRect:Mb,getOffsetParent:eu,getElementRects:Bb,getClientRects:_b,getDimensions:jb,getScale:qn,isElement:et,isRTL:Wb};function tu(e,t){return e.x===t.x&&e.y===t.y&&e.width===t.width&&e.height===t.height}function Ub(e,t){let n=null,r;const o=dt(e);function i(){var a;clearTimeout(r),(a=n)==null||a.disconnect(),n=null}function s(a,l){a===void 0&&(a=!1),l===void 0&&(l=1),i();const c=e.getBoundingClientRect(),{left:u,top:h,width:d,height:m}=c;if(a||t(),!d||!m)return;const f=Do(h),p=Do(o.clientWidth-(u+d)),v=Do(o.clientHeight-(h+m)),b=Do(u),x={rootMargin:-f+"px "+-p+"px "+-v+"px "+-b+"px",threshold:Be(0,$t(1,l))||1};let w=!0;function k(O){const R=O[0].intersectionRatio;if(R!==l){if(!w)return s();R?s(!1,R):r=setTimeout(()=>{s(!1,1e-7)},1e3)}R===1&&!tu(c,e.getBoundingClientRect())&&s(),w=!1}try{n=new IntersectionObserver(k,{...x,root:o.ownerDocument})}catch{n=new IntersectionObserver(k,x)}n.observe(e)}return s(!0),i}function Gb(e,t,n,r){r===void 0&&(r={});const{ancestorScroll:o=!0,ancestorResize:i=!0,elementResize:s=typeof ResizeObserver=="function",layoutShift:a=typeof IntersectionObserver=="function",animationFrame:l=!1}=r,c=Us(e),u=o||i?[...c?Ir(c):[],...Ir(t)]:[];u.forEach(b=>{o&&b.addEventListener("scroll",n,{passive:!0}),i&&b.addEventListener("resize",n)});const h=c&&a?Ub(c,n):null;let d=-1,m=null;s&&(m=new ResizeObserver(b=>{let[y]=b;y&&y.target===c&&m&&(m.unobserve(t),cancelAnimationFrame(d),d=requestAnimationFrame(()=>{var x;(x=m)==null||x.observe(t)})),n()}),c&&!l&&m.observe(c),m.observe(t));let f,p=l?tn(e):null;l&&v();function v(){const b=tn(e);p&&!tu(p,b)&&n(),p=b,f=requestAnimationFrame(v)}return n(),()=>{var b;u.forEach(y=>{o&&y.removeEventListener("scroll",n),i&&y.removeEventListener("resize",n)}),h?.(),(b=m)==null||b.disconnect(),m=null,l&&cancelAnimationFrame(f)}}const qb=vb,Kb=bb,Yb=gb,Xb=xb,Qb=pb,Jb=fb,Zb=yb,ey=(e,t,n)=>{const r=new Map,o={platform:Hb,...n},i={...o.platform,_c:r};return hb(e,t,{...o,platform:i})};function nu(e=0,t=0,n=0,r=0){if(typeof DOMRect=="function")return new DOMRect(e,t,n,r);const o={x:e,y:t,width:n,height:r,top:t,right:e+n,bottom:t+r,left:e};return{...o,toJSON:()=>o}}function ty(e){if(!e)return nu();const{x:t,y:n,width:r,height:o}=e;return nu(t,n,r,o)}function ny(e,t){return{contextElement:Te(e)?e:void 0,getBoundingClientRect:()=>{const n=e,r=t?.(n);return r||!n?ty(r):n.getBoundingClientRect()}}}var ru=e=>({variable:e,reference:`var(${e})`}),ou={transformOrigin:ru("--transform-origin"),arrowOffset:ru("--arrow-offset")},ry=e=>e==="top"||e==="bottom"?"y":"x";function oy(e,t){return{name:"transformOrigin",fn(n){const{elements:r,middlewareData:o,placement:i,rects:s,y:a}=n,l=i.split("-")[0],c=ry(l),u=o.arrow?.x||0,h=o.arrow?.y||0,d=t?.clientWidth||0,m=t?.clientHeight||0,f=u+d/2,p=h+m/2,v=Math.abs(o.shift?.y||0),b=s.reference.height/2,y=m/2,x=e.offset?.mainAxis??e.gutter,w=typeof x=="number"?x+y:x??y,k=v>w,O={top:`${f}px calc(100% + ${w}px)`,bottom:`${f}px ${-w}px`,left:`calc(100% + ${w}px) ${p}px`,right:`${-w}px ${p}px`}[l],R=`${f}px ${s.reference.y+b-a}px`,I=!!e.overlap&&c==="y"&&k;return r.floating.style.setProperty(ou.transformOrigin.variable,I?R:O),{data:{transformOrigin:I?R:O}}}}}var iy={name:"rects",fn({rects:e}){return{data:e}}},sy=e=>{if(e)return{name:"shiftArrow",fn({placement:t,middlewareData:n}){if(!n.arrow)return{};const{x:r,y:o}=n.arrow,i=t.split("-")[0];return Object.assign(e.style,{left:r!=null?`${r}px`:"",top:o!=null?`${o}px`:"",[i]:`calc(100% + ${ou.arrowOffset.reference})`}),{}}}};function ay(e){const[t,n]=e.split("-");return{side:t,align:n,hasAlign:n!=null}}function ly(e){return e.split("-")[0]}var cy={strategy:"absolute",placement:"bottom",listeners:!0,gutter:8,flip:!0,slide:!0,overlap:!1,sameWidth:!1,fitViewport:!1,overflowPadding:8,arrowPadding:4};function iu(e,t){const n=e.devicePixelRatio||1;return Math.round(t*n)/n}function qs(e){return typeof e=="function"?e():e==="clipping-ancestors"?"clippingAncestors":e}function uy(e,t,n){const r=e||t.createElement("div");return Jb({element:r,padding:n.arrowPadding})}function dy(e,t){if(!Xm(t.offset??t.gutter))return qb(({placement:n})=>{const r=(e?.clientHeight||0)/2,o=t.offset?.mainAxis??t.gutter,i=typeof o=="number"?o+r:o??r,{hasAlign:s}=ay(n),a=s?void 0:t.shift,l=t.offset?.crossAxis??a;return bo({crossAxis:l,mainAxis:i,alignmentAxis:t.shift})})}function hy(e){if(!e.flip)return;const t=qs(e.boundary);return Yb({...t?{boundary:t}:void 0,padding:e.overflowPadding,fallbackPlacements:e.flip===!0?void 0:e.flip})}function fy(e){if(!e.slide&&!e.overlap)return;const t=qs(e.boundary);return Kb({...t?{boundary:t}:void 0,mainAxis:e.slide,crossAxis:e.overlap,padding:e.overflowPadding,limiter:Zb()})}function gy(e){return Xb({padding:e.overflowPadding,apply({elements:t,rects:n,availableHeight:r,availableWidth:o}){const i=t.floating,s=Math.round(n.reference.width),a=Math.round(n.reference.height);o=Math.floor(o),r=Math.floor(r),i.style.setProperty("--reference-width",`${s}px`),i.style.setProperty("--reference-height",`${a}px`),i.style.setProperty("--available-width",`${o}px`),i.style.setProperty("--available-height",`${r}px`)}})}function py(e){if(e.hideWhenDetached)return Qb({strategy:"referenceHidden",boundary:qs(e.boundary)??"clippingAncestors"})}function my(e){return e?e===!0?{ancestorResize:!0,ancestorScroll:!0,elementResize:!0,layoutShift:!0}:e:{}}function vy(e,t,n={}){const r=ny(e,n.getAnchorRect);if(!t||!r)return;const o=Object.assign({},cy,n),i=t.querySelector("[data-part=arrow]"),s=[dy(i,o),hy(o),fy(o),uy(i,t.ownerDocument,o),sy(i),oy({gutter:o.gutter,offset:o.offset,overlap:o.overlap},i),gy(o),py(o),iy],{placement:a,strategy:l,onComplete:c,onPositioned:u}=o,h=async()=>{if(!r||!t)return;const p=await ey(r,t,{placement:a,middleware:s,strategy:l});c?.(p),u?.({placed:!0});const v=be(t),b=iu(v,p.x),y=iu(v,p.y);t.style.setProperty("--x",`${b}px`),t.style.setProperty("--y",`${y}px`),o.hideWhenDetached&&(p.middlewareData.hide?.referenceHidden?(t.style.setProperty("visibility","hidden"),t.style.setProperty("pointer-events","none")):(t.style.removeProperty("visibility"),t.style.removeProperty("pointer-events")));const x=t.firstElementChild;if(x){const w=xo(x);t.style.setProperty("--z-index",w.zIndex)}},d=async()=>{n.updatePosition?(await n.updatePosition({updatePosition:h,floatingElement:t}),u?.({placed:!0})):await h()},m=my(o.listeners),f=o.listeners?Gb(r,t,d,m):o0;return d(),()=>{f?.(),u?.({placed:!1})}}function nt(e,t,n={}){const{defer:r,...o}=n,i=r?G:a=>a(),s=[];return s.push(i(()=>{const a=typeof e=="function"?e():e,l=typeof t=="function"?t():t;s.push(vy(a,l,o))})),()=>{s.forEach(a=>a?.())}}function by(e){const t={each(n){for(let r=0;r{try{i.document.addEventListener(n,r,o)}catch{}}),()=>{try{t.removeEventListener(n,r,o)}catch{}}},removeEventListener(n,r,o){t.each(i=>{try{i.document.removeEventListener(n,r,o)}catch{}})}};return t}function yy(e){const t=e.frameElement!=null?e.parent:null;return{addEventListener:(n,r,o)=>{try{t?.addEventListener(n,r,o)}catch{}return()=>{try{t?.removeEventListener(n,r,o)}catch{}}},removeEventListener:(n,r,o)=>{try{t?.removeEventListener(n,r,o)}catch{}}}}var su="pointerdown.outside",au="focus.outside";function xy(e){for(const t of e)if(Te(t)&&Mt(t))return!0;return!1}var lu=e=>"clientY"in e;function ky(e,t){if(!lu(t)||!e)return!1;const n=e.getBoundingClientRect();return n.width===0||n.height===0?!1:n.top<=t.clientY&&t.clientY<=n.top+n.height&&n.left<=t.clientX&&t.clientX<=n.left+n.width}function Cy(e,t){return e.y<=t.y&&t.y<=e.y+e.height&&e.x<=t.x&&t.x<=e.x+e.width}function cu(e,t){if(!t||!lu(e))return!1;const n=t.scrollHeight>t.clientHeight,r=n&&e.clientX>t.offsetLeft+t.clientWidth,o=t.scrollWidth>t.clientWidth,i=o&&e.clientY>t.offsetTop+t.clientHeight,s={x:t.offsetLeft,y:t.offsetTop,width:t.clientWidth+(n?16:0),height:t.clientHeight+(o?16:0)},a={x:e.clientX,y:e.clientY};return Cy(s,a)?r||i:!1}function wy(e,t){const{exclude:n,onFocusOutside:r,onPointerDownOutside:o,onInteractOutside:i,defer:s}=t;if(!e)return;const a=Fe(e),l=be(e),c=by(l),u=yy(l);function h(y,x){if(!Te(x)||!x.isConnected||Jt(e,x)||ky(e,y))return!1;const w=a.querySelector(`[aria-controls="${e.id}"]`);if(w){const O=Oo(w);if(cu(y,O))return!1}const k=Oo(e);return cu(y,k)?!1:!n?.(x)}const d=new Set,m=jn(e?.getRootNode());function f(y){function x(w){const k=s&&!hc()?G:I=>I(),O=w??y,R=O?.composedPath?.()??[O?.target];k(()=>{const I=m?R[0]:$e(y);if(!(!e||!h(y,I))){if(o||i){const N=mo(o,i);e.addEventListener(su,N,{once:!0})}uu(e,su,{bubbles:!1,cancelable:!0,detail:{originalEvent:O,contextmenu:K0(O),focusable:xy(R),target:I}})}})}y.pointerType==="touch"?(d.forEach(w=>w()),d.add(se(a,"click",x,{once:!0})),d.add(u.addEventListener("click",x,{once:!0})),d.add(c.addEventListener("click",x,{once:!0}))):x()}const p=new Set,v=setTimeout(()=>{p.add(se(a,"pointerdown",f,!0)),p.add(u.addEventListener("pointerdown",f,!0)),p.add(c.addEventListener("pointerdown",f,!0))},0);function b(y){(s?G:w=>w())(()=>{const w=$e(y);if(!(!e||!h(y,w))){if(r||i){const k=mo(r,i);e.addEventListener(au,k,{once:!0})}uu(e,au,{bubbles:!1,cancelable:!0,detail:{originalEvent:y,contextmenu:!1,focusable:Mt(w),target:w}})}})}return hc()||(p.add(se(a,"focusin",b,!0)),p.add(u.addEventListener("focusin",b,!0)),p.add(c.addEventListener("focusin",b,!0))),()=>{clearTimeout(v),d.forEach(y=>y()),p.forEach(y=>y())}}function Sy(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(wy(i,t))})),()=>{o.forEach(i=>i?.())}}function uu(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,n);return e.dispatchEvent(o)}function Ey(e,t){const n=r=>{r.key==="Escape"&&(r.isComposing||t?.(r))};return se(Fe(e),"keydown",n,{capture:!0})}var du="layer:request-dismiss",Ke={layers:[],branches:[],count(){return this.layers.length},pointerBlockingLayers(){return this.layers.filter(e=>e.pointerBlocking)},topMostPointerBlockingLayer(){return[...this.pointerBlockingLayers()].slice(-1)[0]},hasPointerBlockingLayer(){return this.pointerBlockingLayers().length>0},isBelowPointerBlockingLayer(e){const t=this.indexOf(e),n=this.topMostPointerBlockingLayer()?this.indexOf(this.topMostPointerBlockingLayer()?.node):-1;return tt.type===e)},getNestedLayersByType(e,t){const n=this.indexOf(e);return n===-1?[]:this.layers.slice(n+1).filter(r=>r.type===t)},getParentLayerOfType(e,t){const n=this.indexOf(e);if(!(n<=0))return this.layers.slice(0,n).reverse().find(r=>r.type===t)},countNestedLayersOfType(e,t){return this.getNestedLayersByType(e,t).length},isInNestedLayer(e,t){return this.getNestedLayers(e).some(n=>Jt(n.node,t))},isInBranch(e){return Array.from(this.branches).some(t=>Jt(t,e))},add(e){this.layers.push(e),this.syncLayers()},addBranch(e){this.branches.push(e)},remove(e){const t=this.indexOf(e);t<0||(tKe.dismiss(r.node,e)),this.layers.splice(t,1),this.syncLayers())},removeBranch(e){const t=this.branches.indexOf(e);t>=0&&this.branches.splice(t,1)},syncLayers(){this.layers.forEach((e,t)=>{e.node.style.setProperty("--layer-index",`${t}`),e.node.removeAttribute("data-nested"),e.node.removeAttribute("data-has-nested"),this.getParentLayerOfType(e.node,e.type)&&e.node.setAttribute("data-nested",e.type);const r=this.countNestedLayersOfType(e.node,e.type);r>0&&e.node.setAttribute("data-has-nested",e.type),e.node.style.setProperty("--nested-layer-count",`${r}`)})},indexOf(e){return this.layers.findIndex(t=>t.node===e)},dismiss(e,t){const n=this.indexOf(e);if(n===-1)return;const r=this.layers[n];Py(e,du,o=>{r.requestDismiss?.(o),o.defaultPrevented||r?.dismiss()}),Oy(e,du,{originalLayer:e,targetLayer:t,originalIndex:n,targetIndex:t?this.indexOf(t):-1}),this.syncLayers()},clear(){this.remove(this.layers[0].node)}};function Oy(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,{cancelable:!0,bubbles:!0,detail:n});return e.dispatchEvent(o)}function Py(e,t,n){e.addEventListener(t,n,{once:!0})}var hu;function fu(){Ke.layers.forEach(({node:e})=>{e.style.pointerEvents=Ke.isBelowPointerBlockingLayer(e)?"none":"auto"})}function Ry(e){e.style.pointerEvents=""}function Iy(e,t){const n=Fe(e),r=[];return Ke.hasPointerBlockingLayer()&&!n.body.hasAttribute("data-inert")&&(hu=document.body.style.pointerEvents,queueMicrotask(()=>{n.body.style.pointerEvents="none",n.body.setAttribute("data-inert","")})),t?.forEach(o=>{const[i,s]=Pv(()=>{const a=o();return Te(a)?a:null},{timeout:1e3});i.then(a=>r.push(To(a,{pointerEvents:"auto"}))),r.push(s)}),()=>{Ke.hasPointerBlockingLayer()||(queueMicrotask(()=>{n.body.style.pointerEvents=hu,n.body.removeAttribute("data-inert"),n.body.style.length===0&&n.body.removeAttribute("style")}),r.forEach(o=>o()))}}function Ty(e,t){const{warnOnMissingNode:n=!0}=t;if(n&&!e){wr("[@zag-js/dismissable] node is `null` or `undefined`");return}if(!e)return;const{onDismiss:r,onRequestDismiss:o,pointerBlocking:i,exclude:s,debug:a,type:l="dialog"}=t,c={dismiss:r,node:e,type:l,pointerBlocking:i,requestDismiss:o};Ke.add(c),fu();function u(p){const v=$e(p.detail.originalEvent);Ke.isBelowPointerBlockingLayer(e)||Ke.isInBranch(v)||(t.onPointerDownOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onPointerDownOutside:",p.detail.originalEvent),r?.()))}function h(p){const v=$e(p.detail.originalEvent);Ke.isInBranch(v)||(t.onFocusOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onFocusOutside:",p.detail.originalEvent),r?.()))}function d(p){Ke.isTopMost(e)&&(t.onEscapeKeyDown?.(p),!p.defaultPrevented&&r&&(p.preventDefault(),r()))}function m(p){if(!e)return!1;const v=typeof s=="function"?s():s,b=Array.isArray(v)?v:[v],y=t.persistentElements?.map(x=>x()).filter(Te);return y&&b.push(...y),b.some(x=>Jt(x,p))||Ke.isInNestedLayer(e,p)}const f=[i?Iy(e,t.persistentElements):void 0,Ey(e,d),Sy(e,{exclude:m,onFocusOutside:h,onPointerDownOutside:u,defer:t.defer})];return()=>{Ke.remove(e),fu(),Ry(e),f.forEach(p=>p?.())}}function Kn(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=Qt(e)?e():e;o.push(Ty(i,t))})),()=>{o.forEach(i=>i?.())}}var gu=j("color-picker",["root","label","control","trigger","positioner","content","area","areaThumb","valueText","areaBackground","channelSlider","channelSliderLabel","channelSliderTrack","channelSliderThumb","channelSliderValueText","channelInput","transparencyGrid","swatchGroup","swatchTrigger","swatchIndicator","swatch","eyeDropperTrigger","formatTrigger","formatSelect"]);gu.build();var Ny=e=>e.ids?.hiddenInput??`color-picker:${e.id}:hidden-input`,Ay=e=>e.ids?.control??`color-picker:${e.id}:control`,_y=e=>e.ids?.trigger??`color-picker:${e.id}:trigger`,Vy=e=>e.ids?.content??`color-picker:${e.id}:content`,Fy=e=>e.ids?.positioner??`color-picker:${e.id}:positioner`,Ly=e=>e.ids?.formatSelect??`color-picker:${e.id}:format-select`,Dy=e=>e.ids?.area??`color-picker:${e.id}:area`,zy=e=>e.ids?.areaThumb??`color-picker:${e.id}:area-thumb`,My=(e,t)=>e.ids?.channelSliderTrack?.(t)??`color-picker:${e.id}:slider-track:${t}`,jy=(e,t)=>e.ids?.channelSliderThumb?.(t)??`color-picker:${e.id}:slider-thumb:${t}`,Ho=e=>e.getById(Vy(e)),$y=e=>e.getById(zy(e)),By=(e,t)=>e.getById(jy(e,t)),Wy=e=>e.getById(Ly(e)),pu=e=>e.getById(Ny(e)),Hy=e=>e.getById(Dy(e)),Uy=(e,t,n)=>{const r=Hy(e);if(!r)return;const{getPercentValue:o}=yc(t,r);return{x:o({dir:n,orientation:"horizontal"}),y:o({orientation:"vertical"})}},Gy=e=>e.getById(Ay(e)),Ks=e=>e.getById(_y(e)),qy=e=>e.getById(Fy(e)),Ky=(e,t)=>e.getById(My(e,t)),Yy=(e,t,n,r)=>{const o=Ky(e,n);if(!o)return;const{getPercentValue:i}=yc(t,o);return{x:i({dir:r,orientation:"horizontal"}),y:i({orientation:"vertical"})}},Xy=e=>[...Io(Ho(e),"input[data-channel]"),...Io(Gy(e),"input[data-channel]")];function Qy(e,t){if(t==null)return"";if(t==="hex")return e.toString("hex");if(t==="css")return e.toString("css");if(t in e)return e.getChannelValue(t).toString();const n=e.getFormat()==="hsla";switch(t){case"hue":return n?e.toFormat("hsla").getChannelValue("hue").toString():e.toFormat("hsba").getChannelValue("hue").toString();case"saturation":return n?e.toFormat("hsla").getChannelValue("saturation").toString():e.toFormat("hsba").getChannelValue("saturation").toString();case"lightness":return e.toFormat("hsla").getChannelValue("lightness").toString();case"brightness":return e.toFormat("hsba").getChannelValue("brightness").toString();case"red":case"green":case"blue":return e.toFormat("rgba").getChannelValue(t).toString();default:return e.getChannelValue(t).toString()}}var mu=e=>Fo(e),Jy=/^[0-9a-fA-F]{3,8}$/;function Zy(e){return Jy.test(e)}function ex(e){return e.startsWith("#")?e:Zy(e)?`#${e}`:e}var{and:tx}=jt();tx("isOpenControlled","closeOnSelect");function vu(e,t,n){const r=Xy(e);G(()=>{r.forEach(o=>{const i=o.dataset.channel;So(o,Qy(n||t,i))})})}function nx(e,t){const n=Wy(e);n&&G(()=>So(n,t))}M()(["closeOnSelect","dir","disabled","format","defaultFormat","getRootNode","id","ids","initialFocusEl","inline","name","positioning","onFocusOutside","onFormatChange","onInteractOutside","onOpenChange","onPointerDownOutside","onValueChange","onValueChangeEnd","defaultOpen","open","positioning","required","readOnly","value","defaultValue","invalid","openAutoFocus"]),M()(["xChannel","yChannel"]),M()(["channel","orientation"]),M()(["value","disabled"]),M()(["value","respectAlpha"]),M()(["size"]);const[bu,rx]=Mn({name:"RenderStrategyContext",hookName:"useRenderStrategyContext",providerName:""}),yu=e=>Sc()(e,["lazyMount","unmountOnExit"]);var xu=j("accordion").parts("root","item","itemTrigger","itemContent","itemIndicator");xu.build();var ku=e=>e.ids?.root??`accordion:${e.id}`,Cu=(e,t)=>e.ids?.itemTrigger?.(t)??`accordion:${e.id}:trigger:${t}`,ox=e=>e.getById(ku(e)),Uo=e=>{const n=`[aria-controls][data-ownedby='${CSS.escape(ku(e))}']:not([disabled])`;return Io(ox(e),n)},ix=e=>kr(Uo(e)),sx=e=>hs(Uo(e)),ax=(e,t)=>vv(Uo(e),Cu(e,t)),lx=(e,t)=>bv(Uo(e),Cu(e,t)),{and:cx,not:ux}=jt();cx("isExpanded","canToggle"),ux("isExpanded"),M()(["collapsible","dir","disabled","getRootNode","id","ids","multiple","onFocusChange","onValueChange","orientation","value","defaultValue"]),M()(["value","disabled"]);var Tr=(e,t)=>({x:e,y:t});function dx(e){const{x:t,y:n,width:r,height:o}=e,i=t+r/2,s=n+o/2;return{x:t,y:n,width:r,height:o,minX:t,minY:n,maxX:t+r,maxY:n+o,midX:i,midY:s,center:Tr(i,s)}}function hx(e){const t=Tr(e.minX,e.minY),n=Tr(e.maxX,e.minY),r=Tr(e.maxX,e.maxY),o=Tr(e.minX,e.maxY);return{top:t,right:n,bottom:r,left:o}}function fx(e,t){const n=dx(e),{top:r,right:o,left:i,bottom:s}=hx(n),[a]=t.split("-");return{top:[i,r,o,s],right:[r,o,s,i],bottom:[r,i,s,o],left:[o,r,i,s]}[a]}function gx(e,t){const{x:n,y:r}=t;let o=!1;for(let i=0,s=e.length-1;ir!=u>r&&n<(c-a)*(r-l)/(u-l)+a&&(o=!o)}return o}var wu=j("avatar").parts("root","image","fallback");wu.build(),M()(["dir","id","ids","onStatusChange","getRootNode"]);function px(e){return!(e.metaKey||!wo()&&e.altKey||e.ctrlKey||e.key==="Control"||e.key==="Shift"||e.key==="Meta")}var mx=new Set(["checkbox","radio","range","color","file","image","button","submit","reset"]);function vx(e,t,n){const r=n?$e(n):null,o=be(r);return e=e||r instanceof o.HTMLInputElement&&!mx.has(r?.type)||r instanceof o.HTMLTextAreaElement||r instanceof o.HTMLElement&&r.isContentEditable,!(e&&t==="keyboard"&&n instanceof o.KeyboardEvent&&!Reflect.has(bx,n.key))}var nn=null,Ys=new Set,Nr=new Map,rn=!1,Xs=!1,bx={Tab:!0,Escape:!0};function Go(e,t){for(let n of Ys)n(e,t)}function qo(e){rn=!0,px(e)&&(nn="keyboard",Go("keyboard",e))}function Ye(e){nn="pointer",(e.type==="mousedown"||e.type==="pointerdown")&&(rn=!0,Go("pointer",e))}function Su(e){q0(e)&&(rn=!0,nn="virtual")}function Eu(e){const t=$e(e);t===be(t)||t===Fe(t)||(!rn&&!Xs&&(nn="virtual",Go("virtual",e)),rn=!1,Xs=!1)}function Ou(){rn=!1,Xs=!0}function yx(e){if(typeof window>"u"||Nr.get(be(e)))return;const t=be(e),n=Fe(e);let r=t.HTMLElement.prototype.focus;function o(){nn="virtual",Go("virtual",null),rn=!0,r.apply(this,arguments)}Object.defineProperty(t.HTMLElement.prototype,"focus",{configurable:!0,value:o}),n.addEventListener("keydown",qo,!0),n.addEventListener("keyup",qo,!0),n.addEventListener("click",Su,!0),t.addEventListener("focus",Eu,!0),t.addEventListener("blur",Ou,!1),typeof t.PointerEvent<"u"?(n.addEventListener("pointerdown",Ye,!0),n.addEventListener("pointermove",Ye,!0),n.addEventListener("pointerup",Ye,!0)):(n.addEventListener("mousedown",Ye,!0),n.addEventListener("mousemove",Ye,!0),n.addEventListener("mouseup",Ye,!0)),t.addEventListener("beforeunload",()=>{xx(e)},{once:!0}),Nr.set(t,{focus:r})}var xx=(e,t)=>{const n=be(e),r=Fe(e);Nr.has(n)&&(n.HTMLElement.prototype.focus=Nr.get(n).focus,r.removeEventListener("keydown",qo,!0),r.removeEventListener("keyup",qo,!0),r.removeEventListener("click",Su,!0),n.removeEventListener("focus",Eu,!0),n.removeEventListener("blur",Ou,!1),typeof n.PointerEvent<"u"?(r.removeEventListener("pointerdown",Ye,!0),r.removeEventListener("pointermove",Ye,!0),r.removeEventListener("pointerup",Ye,!0)):(r.removeEventListener("mousedown",Ye,!0),r.removeEventListener("mousemove",Ye,!0),r.removeEventListener("mouseup",Ye,!0)),Nr.delete(n))};function Pu(){return nn==="keyboard"}function kx(e={}){const{isTextInput:t,autoFocus:n,onChange:r,root:o}=e;yx(o),r?.({isFocusVisible:n||Pu(),modality:nn});const i=(s,a)=>{vx(!!t,s,a)&&r?.({isFocusVisible:Pu(),modality:s})};return Ys.add(i),()=>{Ys.delete(i)}}var Ru=j("checkbox").parts("root","label","control","indicator");Ru.build(),M()(["defaultChecked","checked","dir","disabled","form","getRootNode","id","ids","invalid","name","onCheckedChange","readOnly","required","value"]);const Cx=Ru.extendWith("group"),[OT,Iu]=Mn({name:"FieldContext",hookName:"useFieldContext",providerName:"",strict:!1});var Tu=j("clipboard").parts("root","control","trigger","indicator","input","label");Tu.build(),M()(["getRootNode","id","ids","value","defaultValue","timeout","onStatusChange","onValueChange"]),M()(["copied"]);const wx=gu.extendWith("view");var Sx=Object.defineProperty,Ex=(e,t,n)=>t in e?Sx(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,V=(e,t,n)=>Ex(e,typeof t!="symbol"?t+"":t,n),Ko={itemToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},itemToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Ko.itemToValue(e)},isItemDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1}},Yo=class kp{constructor(t){this.options=t,V(this,"items"),V(this,"indexMap",null),V(this,"copy",n=>new kp({...this.options,items:n??[...this.items]})),V(this,"isEqual",n=>Ze(this.items,n.items)),V(this,"setItems",n=>this.copy(n)),V(this,"getValues",(n=this.items)=>{const r=[];for(const o of n){const i=this.getItemValue(o);i!=null&&r.push(i)}return r}),V(this,"find",n=>{if(n==null)return null;const r=this.indexOf(n);return r!==-1?this.at(r):null}),V(this,"findMany",n=>{const r=[];for(const o of n){const i=this.find(o);i!=null&&r.push(i)}return r}),V(this,"at",n=>{if(!this.options.groupBy&&!this.options.groupSort)return this.items[n]??null;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){if(r===n)return s;r++}return null}),V(this,"sortFn",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return(o??0)-(i??0)}),V(this,"sort",n=>[...n].sort(this.sortFn.bind(this))),V(this,"getItemValue",n=>n==null?null:this.options.itemToValue?.(n)??Ko.itemToValue(n)),V(this,"getItemDisabled",n=>n==null?!1:this.options.isItemDisabled?.(n)??Ko.isItemDisabled(n)),V(this,"stringifyItem",n=>n==null?null:this.options.itemToString?.(n)??Ko.itemToString(n)),V(this,"stringify",n=>n==null?null:this.stringifyItem(this.find(n))),V(this,"stringifyItems",(n,r=", ")=>{const o=[];for(const i of n){const s=this.stringifyItem(i);s!=null&&o.push(s)}return o.join(r)}),V(this,"stringifyMany",(n,r)=>this.stringifyItems(this.findMany(n),r)),V(this,"has",n=>this.indexOf(n)!==-1),V(this,"hasItem",n=>n==null?!1:this.has(this.getItemValue(n))),V(this,"group",()=>{const{groupBy:n,groupSort:r}=this.options;if(!n)return[["",[...this.items]]];const o=new Map;this.items.forEach((s,a)=>{const l=n(s,a);o.has(l)||o.set(l,[]),o.get(l).push(s)});let i=Array.from(o.entries());return r&&i.sort(([s],[a])=>{if(typeof r=="function")return r(s,a);if(Array.isArray(r)){const l=r.indexOf(s),c=r.indexOf(a);return l===-1?1:c===-1?-1:l-c}return r==="asc"?s.localeCompare(a):r==="desc"?a.localeCompare(s):0}),i}),V(this,"getNextValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.min(i+r,this.size-1):i+r;i<=this.size&&this.getItemDisabled(this.at(i));)i++;return this.getItemValue(this.at(i))}),V(this,"getPreviousValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.max(i-r,0):i-r;i>=0&&this.getItemDisabled(this.at(i));)i--;return this.getItemValue(this.at(i))}),V(this,"indexOf",n=>{if(n==null)return-1;if(!this.options.groupBy&&!this.options.groupSort)return this.items.findIndex(r=>this.getItemValue(r)===n);if(!this.indexMap){this.indexMap=new Map;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){const a=this.getItemValue(s);a!=null&&this.indexMap.set(a,r),r++}}return this.indexMap.get(n)??-1}),V(this,"getByText",(n,r)=>{const o=r!=null?this.indexOf(r):-1,i=n.length===1;for(let s=0;s{const{state:o,currentValue:i,timeout:s=350}=r,a=o.keysSoFar+n,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a,u=this.getByText(c,i),h=this.getItemValue(u);function d(){clearTimeout(o.timer),o.timer=-1}function m(f){o.keysSoFar=f,d(),f!==""&&(o.timer=+setTimeout(()=>{m(""),d()},s))}return m(a),h}),V(this,"update",(n,r)=>{let o=this.indexOf(n);return o===-1?this:this.copy([...this.items.slice(0,o),r,...this.items.slice(o+1)])}),V(this,"upsert",(n,r,o="append")=>{let i=this.indexOf(n);return i===-1?(o==="append"?this.append:this.prepend)(r):this.copy([...this.items.slice(0,i),r,...this.items.slice(i+1)])}),V(this,"insert",(n,...r)=>this.copy(Ar(this.items,n,...r))),V(this,"insertBefore",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Ar(this.items,o,...r))}),V(this,"insertAfter",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Ar(this.items,o+1,...r))}),V(this,"prepend",(...n)=>this.copy(Ar(this.items,0,...n))),V(this,"append",(...n)=>this.copy(Ar(this.items,this.items.length,...n))),V(this,"filter",n=>{const r=this.items.filter((o,i)=>n(this.stringifyItem(o),i,o));return this.copy(r)}),V(this,"remove",(...n)=>{const r=n.map(o=>typeof o=="string"?o:this.getItemValue(o));return this.copy(this.items.filter(o=>{const i=this.getItemValue(o);return i==null?!1:!r.includes(i)}))}),V(this,"move",(n,r)=>{const o=this.indexOf(n);return o===-1?this:this.copy(Xo(this.items,[o],r))}),V(this,"moveBefore",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Xo(this.items,i,o))}),V(this,"moveAfter",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Xo(this.items,i,o+1))}),V(this,"reorder",(n,r)=>this.copy(Xo(this.items,[n],r))),V(this,"compareValue",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return oi?1:0}),V(this,"range",(n,r)=>{let o=[],i=n;for(;i!=null;){if(this.find(i)&&o.push(i),i===r)return o;i=this.getNextValue(i)}return[]}),V(this,"getValueRange",(n,r)=>n&&r?this.compareValue(n,r)<=0?this.range(n,r):this.range(r,n):[]),V(this,"toString",()=>{let n="";for(const r of this.items){const o=this.getItemValue(r),i=this.stringifyItem(r),s=this.getItemDisabled(r),a=[o,i,s].filter(Boolean).join(":");n+=a+","}return n}),V(this,"toJSON",()=>({size:this.size,first:this.firstValue,last:this.lastValue})),this.items=[...t.items]}get size(){return this.items.length}get firstValue(){let t=0;for(;this.getItemDisabled(this.at(t));)t++;return this.getItemValue(this.at(t))}get lastValue(){let t=this.size-1;for(;this.getItemDisabled(this.at(t));)t--;return this.getItemValue(this.at(t))}*[Symbol.iterator](){yield*this.items}},Ox=(e,t)=>!!e?.toLowerCase().startsWith(t.toLowerCase());function Ar(e,t,...n){return[...e.slice(0,t),...n,...e.slice(t)]}function Xo(e,t,n){t=[...t].sort((o,i)=>o-i);const r=t.map(o=>e[o]);for(let o=t.length-1;o>=0;o--)e=[...e.slice(0,t[o]),...e.slice(t[o]+1)];return n=Math.max(0,n-t.filter(o=>ot[n])return 1}return e.length-t.length}function Rx(e){return e.sort(Au)}function Ix(e,t){let n;return He(e,{...t,onEnter:(r,o)=>{if(t.predicate(r,o))return n=r,"stop"}}),n}function Tx(e,t){const n=[];return He(e,{onEnter:(r,o)=>{t.predicate(r,o)&&n.push(r)},getChildren:t.getChildren}),n}function _u(e,t){let n;return He(e,{onEnter:(r,o)=>{if(t.predicate(r,o))return n=[...o],"stop"},getChildren:t.getChildren}),n}function Nx(e,t){let n=t.initialResult;return He(e,{...t,onEnter:(r,o)=>{n=t.nextResult(n,r,o)}}),n}function Ax(e,t){return Nx(e,{...t,initialResult:[],nextResult:(n,r,o)=>(n.push(...t.transform(r,o)),n)})}function _x(e,t){const{predicate:n,create:r,getChildren:o}=t,i=(s,a)=>{const l=o(s,a),c=[];l.forEach((m,f)=>{const p=[...a,f],v=i(m,p);v&&c.push(v)});const u=a.length===0,h=n(s,a),d=c.length>0;return u||h||d?r(s,c,a):null};return i(e,[])||r(e,[],[])}function Vx(e,t){const n=[];let r=0;const o=new Map,i=new Map;return He(e,{getChildren:t.getChildren,onEnter:(s,a)=>{o.has(s)||o.set(s,r++);const l=t.getChildren(s,a);l.forEach(m=>{i.has(m)||i.set(m,s),o.has(m)||o.set(m,r++)});const c=l.length>0?l.map(m=>o.get(m)):void 0,u=i.get(s),h=u?o.get(u):void 0,d=o.get(s);n.push({...s,_children:c,_parent:h,_index:d})}}),n}function Fx(e,t){return{type:"insert",index:e,nodes:t}}function Lx(e){return{type:"remove",indexes:e}}function Qs(){return{type:"replace"}}function Vu(e){return[e.slice(0,-1),e[e.length-1]]}function Fu(e,t,n=new Map){const[r,o]=Vu(e);for(let s=r.length-1;s>=0;s--){const a=r.slice(0,s).join();switch(n.get(a)?.type){case"remove":continue}n.set(a,Qs())}const i=n.get(r.join());switch(i?.type){case"remove":n.set(r.join(),{type:"removeThenInsert",removeIndexes:i.indexes,insertIndex:o,insertNodes:t});break;default:n.set(r.join(),Fx(o,t))}return n}function Lu(e){const t=new Map,n=new Map;for(const r of e){const o=r.slice(0,-1).join(),i=n.get(o)??[];i.push(r[r.length-1]),n.set(o,i.sort((s,a)=>s-a))}for(const r of e)for(let o=r.length-2;o>=0;o--){const i=r.slice(0,o).join();t.has(i)||t.set(i,Qs())}for(const[r,o]of n)t.set(r,Lx(o));return t}function Dx(e,t){const n=new Map,[r,o]=Vu(e);for(let i=r.length-1;i>=0;i--){const s=r.slice(0,i).join();n.set(s,Qs())}return n.set(r.join(),{type:"removeThenInsert",removeIndexes:[o],insertIndex:o,insertNodes:[t]}),n}function Qo(e,t,n){return zx(e,{...n,getChildren:(r,o)=>{const i=o.join();switch(t.get(i)?.type){case"replace":case"remove":case"removeThenInsert":case"insert":return n.getChildren(r,o);default:return[]}},transform:(r,o,i)=>{const s=i.join(),a=t.get(s);switch(a?.type){case"remove":return n.create(r,o.filter((u,h)=>!a.indexes.includes(h)),i);case"removeThenInsert":const l=o.filter((u,h)=>!a.removeIndexes.includes(h)),c=a.removeIndexes.reduce((u,h)=>h{const i=[0,...o],s=i.join(),a=t.transform(r,n[s]??[],o),l=i.slice(0,-1).join(),c=n[l]??[];c.push(a),n[l]=c}}),n[""][0]}function Mx(e,t){const{nodes:n,at:r}=t;if(r.length===0)throw new Error("Can't insert nodes at the root");const o=Fu(r,n);return Qo(e,o,t)}function jx(e,t){if(t.at.length===0)return t.node;const n=Dx(t.at,t.node);return Qo(e,n,t)}function $x(e,t){if(t.indexPaths.length===0)return e;for(const r of t.indexPaths)if(r.length===0)throw new Error("Can't remove the root node");const n=Lu(t.indexPaths);return Qo(e,n,t)}function Bx(e,t){if(t.indexPaths.length===0)return e;for(const i of t.indexPaths)if(i.length===0)throw new Error("Can't move the root node");if(t.to.length===0)throw new Error("Can't move nodes to the root");const n=Px(t.indexPaths),r=n.map(i=>Nu(e,i,t)),o=Fu(t.to,r,Lu(n));return Qo(e,o,t)}function He(e,t){const{onEnter:n,onLeave:r,getChildren:o}=t;let i=[],s=[{node:e}];const a=t.reuseIndexPath?()=>i:()=>i.slice();for(;s.length>0;){let l=s[s.length-1];if(l.state===void 0){const u=n?.(l.node,a());if(u==="stop")return;l.state=u==="skip"?-1:0}const c=l.children||o(l.node,a());if(l.children||(l.children=c),l.state!==-1){if(l.stateZe(this.rootNode,n.rootNode)),V(this,"getNodeChildren",n=>this.options.nodeToChildren?.(n)??Yn.nodeToChildren(n)??[]),V(this,"resolveIndexPath",n=>typeof n=="string"?this.getIndexPath(n):n),V(this,"resolveNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(r):void 0}),V(this,"getNodeChildrenCount",n=>this.options.nodeToChildrenCount?.(n)??Yn.nodeToChildrenCount(n)),V(this,"getNodeValue",n=>this.options.nodeToValue?.(n)??Yn.nodeToValue(n)),V(this,"getNodeDisabled",n=>this.options.isNodeDisabled?.(n)??Yn.isNodeDisabled(n)),V(this,"stringify",n=>{const r=this.findNode(n);return r?this.stringifyNode(r):null}),V(this,"stringifyNode",n=>this.options.nodeToString?.(n)??Yn.nodeToString(n)),V(this,"getFirstNode",(n=this.rootNode)=>{let r;return He(n,{getChildren:this.getNodeChildren,onEnter:(o,i)=>{if(!r&&i.length>0&&!this.getNodeDisabled(o))return r=o,"stop"}}),r}),V(this,"getLastNode",(n=this.rootNode,r={})=>{let o;return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isSameNode(i,n)){if(r.skip?.({value:this.getNodeValue(i),node:i,indexPath:s}))return"skip";s.length>0&&!this.getNodeDisabled(i)&&(o=i)}}}),o}),V(this,"at",n=>Nu(this.rootNode,n,{getChildren:this.getNodeChildren})),V(this,"findNode",(n,r=this.rootNode)=>Ix(r,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})),V(this,"findNodes",(n,r=this.rootNode)=>{const o=new Set(n.filter(i=>i!=null));return Tx(r,{getChildren:this.getNodeChildren,predicate:i=>o.has(this.getNodeValue(i))})}),V(this,"sort",n=>n.reduce((r,o)=>{const i=this.getIndexPath(o);return i&&r.push({value:o,indexPath:i}),r},[]).sort((r,o)=>Au(r.indexPath,o.indexPath)).map(({value:r})=>r)),V(this,"getIndexPath",n=>_u(this.rootNode,{getChildren:this.getNodeChildren,predicate:r=>this.getNodeValue(r)===n})),V(this,"getValue",n=>{const r=this.at(n);return r?this.getNodeValue(r):void 0}),V(this,"getValuePath",n=>{if(!n)return[];const r=[];let o=[...n];for(;o.length>0;){const i=this.at(o);i&&r.unshift(this.getNodeValue(i)),o.pop()}return r}),V(this,"getDepth",n=>_u(this.rootNode,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})?.length??0),V(this,"isSameNode",(n,r)=>this.getNodeValue(n)===this.getNodeValue(r)),V(this,"isRootNode",n=>this.isSameNode(n,this.rootNode)),V(this,"contains",(n,r)=>!n||!r?!1:r.slice(0,n.length).every((o,i)=>n[i]===r[i])),V(this,"getNextNode",(n,r={})=>{let o=!1,i;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return l===n&&(o=!0),"skip";if(o&&!this.getNodeDisabled(s))return i=s,"stop";l===n&&(o=!0)}}),i}),V(this,"getPreviousNode",(n,r={})=>{let o,i=!1;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return"skip";if(l===n)return i=!0,"stop";this.getNodeDisabled(s)||(o=s)}}),i?o:void 0}),V(this,"getParentNodes",n=>{const r=this.resolveIndexPath(n)?.slice();if(!r)return[];const o=[];for(;r.length>0;){r.pop();const i=this.at(r);i&&!this.isRootNode(i)&&o.unshift(i)}return o}),V(this,"getDescendantNodes",(n,r)=>{const o=this.resolveNode(n);if(!o)return[];const i=[];return He(o,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{a.length!==0&&(!r?.withBranch&&this.isBranchNode(s)||i.push(s))}}),i}),V(this,"getDescendantValues",(n,r)=>this.getDescendantNodes(n,r).map(i=>this.getNodeValue(i))),V(this,"getParentIndexPath",n=>n.slice(0,-1)),V(this,"getParentNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(this.getParentIndexPath(r)):void 0}),V(this,"visit",n=>{const{skip:r,...o}=n;He(this.rootNode,{...o,getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isRootNode(i))return r?.({value:this.getNodeValue(i),node:i,indexPath:s})?"skip":o.onEnter?.(i,s)}})}),V(this,"getPreviousSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;--i>=0;){const s=o[i];if(!this.getNodeDisabled(s))return s}}),V(this,"getNextSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;++i{const r=this.getParentNode(n);return r?this.getNodeChildren(r):[]}),V(this,"getValues",(n=this.rootNode)=>Ax(n,{getChildren:this.getNodeChildren,transform:o=>[this.getNodeValue(o)]}).slice(1)),V(this,"isValidDepth",(n,r)=>r==null?!0:typeof r=="function"?r(n.length):n.length===r),V(this,"isBranchNode",n=>this.getNodeChildren(n).length>0||this.getNodeChildrenCount(n)!=null),V(this,"getBranchValues",(n=this.rootNode,r={})=>{let o=[];return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(s.length===0)return;const a=this.getNodeValue(i);if(r.skip?.({value:a,node:i,indexPath:s}))return"skip";this.isBranchNode(i)&&this.isValidDepth(s,r.depth)&&o.push(this.getNodeValue(i))}}),o}),V(this,"flatten",(n=this.rootNode)=>Vx(n,{getChildren:this.getNodeChildren})),V(this,"_create",(n,r)=>this.getNodeChildren(n).length>0||r.length>0?{...n,children:r}:{...n}),V(this,"_insert",(n,r,o)=>this.copy(Mx(n,{at:r,nodes:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"copy",n=>new Cp({...this.options,rootNode:n})),V(this,"_replace",(n,r,o)=>this.copy(jx(n,{at:r,node:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_move",(n,r,o)=>this.copy(Bx(n,{indexPaths:r,to:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_remove",(n,r)=>this.copy($x(n,{indexPaths:r,getChildren:this.getNodeChildren,create:this._create}))),V(this,"replace",(n,r)=>this._replace(this.rootNode,n,r)),V(this,"remove",n=>this._remove(this.rootNode,n)),V(this,"insertBefore",(n,r)=>this.getParentNode(n)?this._insert(this.rootNode,n,r):void 0),V(this,"insertAfter",(n,r)=>{if(!this.getParentNode(n))return;const i=[...n.slice(0,-1),n[n.length-1]+1];return this._insert(this.rootNode,i,r)}),V(this,"move",(n,r)=>this._move(this.rootNode,n,r)),V(this,"filter",n=>{const r=_x(this.rootNode,{predicate:n,getChildren:this.getNodeChildren,create:this._create});return this.copy(r)}),V(this,"toJSON",()=>this.getValues(this.rootNode)),this.rootNode=t.rootNode}},Yn={nodeToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},nodeToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Yn.nodeToValue(e)},isNodeDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1},nodeToChildren(e){return e.children},nodeToChildrenCount(e){if(Xt(e)&&Dt(e,"childrenCount"))return e.childrenCount}},Xn=new WeakMap,Jo=new WeakMap,Zo={},Js=0,Mu=e=>e&&(e.host||Mu(e.parentNode)),Wx=(e,t)=>t.map(n=>{if(e.contains(n))return n;const r=Mu(n);return r&&e.contains(r)?r:(console.error("[zag-js > ariaHidden] target",n,"in not contained inside",e,". Doing nothing"),null)}).filter(n=>!!n),Hx=new Set(["script","output","status","next-route-announcer"]),Ux=e=>Hx.has(e.localName)||e.role==="status"||e.hasAttribute("aria-live")?!0:e.matches("[data-live-announcer]"),Gx=(e,t)=>{const{parentNode:n,markerName:r,controlAttribute:o}=t,i=Wx(n,Array.isArray(e)?e:[e]);Zo[r]||(Zo[r]=new WeakMap);const s=Zo[r],a=[],l=new Set,c=new Set(i),u=d=>{!d||l.has(d)||(l.add(d),u(d.parentNode))};i.forEach(u);const h=d=>{!d||c.has(d)||Array.prototype.forEach.call(d.children,m=>{if(l.has(m))h(m);else try{if(Ux(m))return;const p=m.getAttribute(o)==="true",v=(Xn.get(m)||0)+1,b=(s.get(m)||0)+1;Xn.set(m,v),s.set(m,b),a.push(m),v===1&&p&&Jo.set(m,!0),b===1&&m.setAttribute(r,""),p||m.setAttribute(o,"true")}catch(f){console.error("[zag-js > ariaHidden] cannot operate on ",m,f)}})};return h(n),l.clear(),Js++,()=>{a.forEach(d=>{const m=Xn.get(d)-1,f=s.get(d)-1;Xn.set(d,m),s.set(d,f),m||(Jo.has(d)||d.removeAttribute(o),Jo.delete(d)),f||d.removeAttribute(r)}),Js--,Js||(Xn=new WeakMap,Xn=new WeakMap,Jo=new WeakMap,Zo={})}},qx=e=>(Array.isArray(e)?e[0]:e).ownerDocument.body,Kx=(e,t=qx(e),n="data-aria-hidden")=>{if(t)return Gx(e,{parentNode:t,markerName:n,controlAttribute:"aria-hidden"})},Yx=e=>{const t=requestAnimationFrame(()=>e());return()=>cancelAnimationFrame(t)};function ju(e,t={}){const{defer:n=!0}=t,r=n?Yx:i=>i(),o=[];return o.push(r(()=>{const s=(typeof e=="function"?e():e).filter(Boolean);s.length!==0&&o.push(Kx(s))})),()=>{o.forEach(i=>i?.())}}var $u=j("combobox").parts("root","clearTrigger","content","control","input","item","itemGroup","itemGroupLabel","itemIndicator","itemText","label","list","positioner","trigger");$u.build();var Bu=e=>new Yo(e);Bu.empty=()=>new Yo({items:[]});var Xx=e=>e.ids?.control??`combobox:${e.id}:control`,Qx=e=>e.ids?.input??`combobox:${e.id}:input`,Jx=e=>e.ids?.content??`combobox:${e.id}:content`,Zx=e=>e.ids?.positioner??`combobox:${e.id}:popper`,e1=e=>e.ids?.trigger??`combobox:${e.id}:toggle-btn`,t1=e=>e.ids?.clearTrigger??`combobox:${e.id}:clear-btn`,Wt=e=>e.getById(Jx(e)),Qn=e=>e.getById(Qx(e)),Wu=e=>e.getById(Zx(e)),Hu=e=>e.getById(Xx(e)),_r=e=>e.getById(e1(e)),Uu=e=>e.getById(t1(e)),Vr=(e,t)=>{if(t==null)return null;const n=`[role=option][data-value="${CSS.escape(t)}"]`;return pv(Wt(e),n)},Gu=e=>{const t=Qn(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},n1=e=>{const t=_r(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},{guards:r1,createMachine:o1,choose:i1}=wc(),{and:ye,not:Ue}=r1;o1({props({props:e}){return{loopFocus:!0,openOnClick:!1,defaultValue:[],closeOnSelect:!e.multiple,allowCustomValue:!1,alwaysSubmitOnEnter:!1,inputBehavior:"none",selectionBehavior:e.multiple?"clear":"replace",openOnKeyPress:!0,openOnChange:!0,composite:!0,navigate({node:t}){vc(t)},collection:Bu.empty(),...e,positioning:{placement:"bottom",sameWidth:!0,...e.positioning},translations:{triggerLabel:"Toggle suggestions",clearTriggerLabel:"Clear value",...e.translations}}},initialState({prop:e}){return e("open")||e("defaultOpen")?"suggesting":"idle"},context({prop:e,bindable:t,getContext:n,getEvent:r}){return{currentPlacement:t(()=>({defaultValue:void 0})),value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),isEqual:Ze,hash(o){return o.join(",")},onChange(o){const i=n(),s=i.get("selectedItems"),a=e("collection"),l=o.map(c=>s.find(h=>a.getItemValue(h)===c)||a.find(c));i.set("selectedItems",l),e("onValueChange")?.({value:o,items:l})}})),highlightedValue:t(()=>({defaultValue:e("defaultHighlightedValue")||null,value:e("highlightedValue"),onChange(o){const i=e("collection").find(o);e("onHighlightChange")?.({highlightedValue:o,highlightedItem:i})}})),inputValue:t(()=>{let o=e("inputValue")||e("defaultInputValue")||"";const i=e("defaultValue")||e("value")||[];if(!o.trim()&&!e("multiple")){const s=e("collection").stringifyMany(i);o=xt(e("selectionBehavior"),{preserve:o||s,replace:s,clear:""})}return{defaultValue:o,value:e("inputValue"),onChange(s){const a=r(),l=(a.previousEvent||a).src;e("onInputValueChange")?.({inputValue:s,reason:l})}}}),highlightedItem:t(()=>{const o=e("highlightedValue");return{defaultValue:e("collection").find(o)}}),selectedItems:t(()=>{const o=e("value")||e("defaultValue")||[];return{defaultValue:e("collection").findMany(o)}})}},computed:{isInputValueEmpty:({context:e})=>e.get("inputValue").length===0,isInteractive:({prop:e})=>!(e("readOnly")||e("disabled")),autoComplete:({prop:e})=>e("inputBehavior")==="autocomplete",autoHighlight:({prop:e})=>e("inputBehavior")==="autohighlight",hasSelectedItems:({context:e})=>e.get("value").length>0,valueAsString:({context:e,prop:t})=>t("collection").stringifyItems(e.get("selectedItems")),isCustomValue:({context:e,computed:t})=>e.get("inputValue")!==t("valueAsString")},watch({context:e,prop:t,track:n,action:r,send:o}){n([()=>e.hash("value")],()=>{r(["syncSelectedItems"])}),n([()=>e.get("inputValue")],()=>{r(["syncInputValue"])}),n([()=>e.get("highlightedValue")],()=>{r(["syncHighlightedItem","autofillInputValue"])}),n([()=>t("open")],()=>{r(["toggleVisibility"])}),n([()=>t("collection").toString()],()=>{o({type:"CHILDREN_CHANGE"})})},on:{"SELECTED_ITEMS.SYNC":{actions:["syncSelectedItems"]},"HIGHLIGHTED_VALUE.SET":{actions:["setHighlightedValue"]},"HIGHLIGHTED_VALUE.CLEAR":{actions:["clearHighlightedValue"]},"ITEM.SELECT":{actions:["selectItem"]},"ITEM.CLEAR":{actions:["clearItem"]},"VALUE.SET":{actions:["setValue"]},"INPUT_VALUE.SET":{actions:["setInputValue"]},"POSITIONING.SET":{actions:["reposition"]}},entry:i1([{guard:"autoFocus",actions:["setInitialFocus"]}]),states:{idle:{tags:["idle","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":{target:"interacting"},"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.FOCUS":{target:"focused"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{target:"focused",actions:["clearInputValue","clearSelectedItems","setInitialFocus"]}}},focused:{tags:["focused","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":[{guard:"isChangeEvent",target:"suggesting"},{target:"interacting"}],"INPUT.CHANGE":[{guard:ye("isOpenControlled","openOnChange"),actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{guard:"openOnChange",target:"suggesting",actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{actions:["setInputValue"]}],"LAYER.INTERACT_OUTSIDE":{target:"idle"},"INPUT.ESCAPE":{guard:ye("isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue"]},"INPUT.BLUR":{target:"idle"},"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_DOWN":[{guard:ye("isOpenControlled","autoComplete"),actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"isOpenControlled",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_UP":[{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]}],OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{actions:["clearInputValue","clearSelectedItems"]}}},interacting:{tags:["open","focused"],entry:["setInitialFocus"],effects:["scrollToHighlightedItem","trackDismissableLayer","trackPlacement","hideOtherElements"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]},{actions:["scrollToHighlightedItem"]}],"INPUT.HOME":{actions:["highlightFirstItem"]},"INPUT.END":{actions:["highlightLastItem"]},"INPUT.ARROW_DOWN":[{guard:ye("autoComplete","isLastItemHighlighted"),actions:["clearHighlightedValue","scrollContentToTop"]},{actions:["highlightNextItem"]}],"INPUT.ARROW_UP":[{guard:ye("autoComplete","isFirstItemHighlighted"),actions:["clearHighlightedValue"]},{actions:["highlightPrevItem"]}],"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":[{guard:"autoComplete",target:"suggesting",actions:["setInputValue"]},{target:"suggesting",actions:["clearHighlightedValue","setInputValue"]}],"ITEM.POINTER_MOVE":{actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],"LAYER.ESCAPE":[{guard:ye("isOpenControlled","autoComplete"),actions:["syncInputValue","invokeOnClose"]},{guard:"autoComplete",target:"focused",actions:["syncInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}},suggesting:{tags:["open","focused"],effects:["trackDismissableLayer","scrollToHighlightedItem","trackPlacement","hideOtherElements"],entry:["setInitialFocus"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"autoHighlight",actions:["highlightFirstItem"]},{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]}],"INPUT.ARROW_DOWN":{target:"interacting",actions:["highlightNextItem"]},"INPUT.ARROW_UP":{target:"interacting",actions:["highlightPrevItem"]},"INPUT.HOME":{target:"interacting",actions:["highlightFirstItem"]},"INPUT.END":{target:"interacting",actions:["highlightLastItem"]},"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":{actions:["setInputValue"]},"LAYER.ESCAPE":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.POINTER_MOVE":{target:"interacting",actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}}},implementations:{guards:{isInputValueEmpty:({computed:e})=>e("isInputValueEmpty"),autoComplete:({computed:e,prop:t})=>e("autoComplete")&&!t("multiple"),autoHighlight:({computed:e})=>e("autoHighlight"),isFirstItemHighlighted:({prop:e,context:t})=>e("collection").firstValue===t.get("highlightedValue"),isLastItemHighlighted:({prop:e,context:t})=>e("collection").lastValue===t.get("highlightedValue"),isCustomValue:({computed:e})=>e("isCustomValue"),allowCustomValue:({prop:e})=>!!e("allowCustomValue"),hasHighlightedItem:({context:e})=>e.get("highlightedValue")!=null,closeOnSelect:({prop:e})=>!!e("closeOnSelect"),isOpenControlled:({prop:e})=>e("open")!=null,openOnChange:({prop:e,context:t})=>{const n=e("openOnChange");return Ym(n)?n:!!n?.({inputValue:t.get("inputValue")})},restoreFocus:({event:e})=>e.restoreFocus==null?!0:!!e.restoreFocus,isChangeEvent:({event:e})=>e.previousEvent?.type==="INPUT.CHANGE",autoFocus:({prop:e})=>!!e("autoFocus"),isHighlightedItemRemoved:({prop:e,context:t})=>!e("collection").has(t.get("highlightedValue"))},effects:{trackDismissableLayer({send:e,prop:t,scope:n}){return t("disableLayer")?void 0:Kn(()=>Wt(n),{type:"listbox",defer:!0,exclude:()=>[Qn(n),_r(n),Uu(n)],onFocusOutside:t("onFocusOutside"),onPointerDownOutside:t("onPointerDownOutside"),onInteractOutside:t("onInteractOutside"),onEscapeKeyDown(o){o.preventDefault(),o.stopPropagation(),e({type:"LAYER.ESCAPE",src:"escape-key"})},onDismiss(){e({type:"LAYER.INTERACT_OUTSIDE",src:"interact-outside",restoreFocus:!1})}})},hideOtherElements({scope:e}){return ju([Qn(e),Wt(e),_r(e),Uu(e)])},trackPlacement({context:e,prop:t,scope:n}){const r=()=>Hu(n)||_r(n),o=()=>Wu(n);return e.set("currentPlacement",t("positioning").placement),nt(r,o,{...t("positioning"),defer:!0,onComplete(i){e.set("currentPlacement",i.placement)}})},scrollToHighlightedItem({context:e,prop:t,scope:n,event:r}){const o=Qn(n);let i=[];const s=c=>{const u=r.current().type.includes("POINTER"),h=e.get("highlightedValue");if(u||!h)return;const d=Wt(n),m=t("scrollToIndexFn");if(m){const v=t("collection").indexOf(h);m({index:v,immediate:c,getElement:()=>Vr(n,h)});return}const f=Vr(n,h),p=G(()=>{Po(f,{rootEl:d,block:"nearest"})});i.push(p)},a=G(()=>s(!0));i.push(a);const l=Eo(o,{attributes:["aria-activedescendant"],callback:()=>s(!1)});return i.push(l),()=>{i.forEach(c=>c())}}},actions:{reposition({context:e,prop:t,scope:n,event:r}){nt(()=>Hu(n),()=>Wu(n),{...t("positioning"),...r.options,defer:!0,listeners:!1,onComplete(s){e.set("currentPlacement",s.placement)}})},setHighlightedValue({context:e,event:t}){t.value!=null&&e.set("highlightedValue",t.value)},clearHighlightedValue({context:e}){e.set("highlightedValue",null)},selectHighlightedItem(e){const{context:t,prop:n}=e,r=n("collection"),o=t.get("highlightedValue");if(!o||!r.has(o))return;const i=n("multiple")?zn(t.get("value"),o):[o];n("onSelect")?.({value:i,itemValue:o}),t.set("value",i);const s=xt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(i),clear:""});t.set("inputValue",s)},scrollToHighlightedItem({context:e,prop:t,scope:n}){ws(()=>{const r=e.get("highlightedValue");if(r==null)return;const o=Vr(n,r),i=Wt(n),s=t("scrollToIndexFn");if(s){const a=t("collection").indexOf(r);s({index:a,immediate:!0,getElement:()=>Vr(n,r)});return}Po(o,{rootEl:i,block:"nearest"})})},selectItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=o("multiple")?zn(t.get("value"),n.value):[n.value];o("onSelect")?.({value:i,itemValue:n.value}),t.set("value",i);const s=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},clearItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=Yt(t.get("value"),n.value);t.set("value",i);const s=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},setInitialFocus({scope:e}){G(()=>{Gu(e)})},setFinalFocus({scope:e}){G(()=>{_r(e)?.dataset.focusable==null?Gu(e):n1(e)})},syncInputValue({context:e,scope:t,event:n}){const r=Qn(t);r&&(r.value=e.get("inputValue"),queueMicrotask(()=>{n.current().type!=="INPUT.CHANGE"&&k0(r)}))},setInputValue({context:e,event:t}){e.set("inputValue",t.value)},clearInputValue({context:e}){e.set("inputValue","")},revertInputValue({context:e,prop:t,computed:n}){const r=t("selectionBehavior"),o=xt(r,{replace:n("hasSelectedItems")?n("valueAsString"):"",preserve:e.get("inputValue"),clear:""});e.set("inputValue",o)},setValue(e){const{context:t,flush:n,event:r,prop:o}=e;n(()=>{t.set("value",r.value);const i=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(r.value),clear:""});t.set("inputValue",i)})},clearSelectedItems(e){const{context:t,flush:n,prop:r}=e;n(()=>{t.set("value",[]);const o=xt(r("selectionBehavior"),{preserve:t.get("inputValue"),replace:r("collection").stringifyMany([]),clear:""});t.set("inputValue",o)})},scrollContentToTop({prop:e,scope:t}){const n=e("scrollToIndexFn");if(n){const r=e("collection").firstValue;n({index:0,immediate:!0,getElement:()=>Vr(t,r)})}else{const r=Wt(t);if(!r)return;r.scrollTop=0}},invokeOnOpen({prop:e,event:t}){const n=qu(t);e("onOpenChange")?.({open:!0,reason:n})},invokeOnClose({prop:e,event:t}){const n=qu(t);e("onOpenChange")?.({open:!1,reason:n})},highlightFirstItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").firstValue;o&&e.set("highlightedValue",o)})},highlightFirstItemIfNeeded({computed:e,action:t}){e("autoHighlight")&&t(["highlightFirstItem"])},highlightLastItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").lastValue;o&&e.set("highlightedValue",o)})},highlightNextItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getNextValue(r),!n&&t("loopFocus")&&(n=o.firstValue)):n=o.firstValue,n&&e.set("highlightedValue",n)},highlightPrevItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getPreviousValue(r),!n&&t("loopFocus")&&(n=o.lastValue)):n=o.lastValue,n&&e.set("highlightedValue",n)},highlightFirstSelectedItem({context:e,prop:t}){G(()=>{const[n]=t("collection").sort(e.get("value"));n&&e.set("highlightedValue",n)})},highlightFirstOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{let r=null;n("hasSelectedItems")?r=t("collection").sort(e.get("value"))[0]:r=t("collection").firstValue,r&&e.set("highlightedValue",r)})},highlightLastOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{const r=t("collection");let o=null;n("hasSelectedItems")?o=r.sort(e.get("value"))[0]:o=r.lastValue,o&&e.set("highlightedValue",o)})},autofillInputValue({context:e,computed:t,prop:n,event:r,scope:o}){const i=Qn(o),s=n("collection");if(!t("autoComplete")||!i||!r.keypress)return;const a=s.stringify(e.get("highlightedValue"));G(()=>{i.value=a||e.get("inputValue")})},syncSelectedItems(e){queueMicrotask(()=>{const{context:t,prop:n}=e,r=n("collection"),o=t.get("value"),i=o.map(a=>t.get("selectedItems").find(c=>r.getItemValue(c)===a)||r.find(a));t.set("selectedItems",i);const s=xt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(o),clear:""});t.set("inputValue",s)})},syncHighlightedItem({context:e,prop:t}){const n=t("collection").find(e.get("highlightedValue"));e.set("highlightedItem",n)},toggleVisibility({event:e,send:t,prop:n}){t({type:n("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:e})}}}});function qu(e){return(e.previousEvent||e).src}M()(["allowCustomValue","autoFocus","closeOnSelect","collection","composite","defaultHighlightedValue","defaultInputValue","defaultOpen","defaultValue","dir","disabled","disableLayer","form","getRootNode","highlightedValue","id","ids","inputBehavior","inputValue","invalid","loopFocus","multiple","name","navigate","onFocusOutside","onHighlightChange","onInputValueChange","onInteractOutside","onOpenChange","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","openOnChange","openOnClick","openOnKeyPress","placeholder","positioning","readOnly","required","scrollToIndexFn","selectionBehavior","translations","value","alwaysSubmitOnEnter"]),M()(["htmlFor"]),M()(["id"]),M()(["item","persistFocus"]);const s1=$u.extendWith("empty"),[Ku,Ht]=Mn({name:"DialogContext",hookName:"useDialogContext",providerName:""}),Yu=P.forwardRef((e,t)=>{const n=Ht(),r=rx(),o=Ts({...r,present:n.open}),i=qe(n.getBackdropProps(),o.getPresenceProps(),e);return o.unmounted?null:g.jsx(kt.div,{...i,ref:No(o.ref,t)})});Yu.displayName="DialogBackdrop";const Xu=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getCloseTriggerProps(),e);return g.jsx(kt.button,{...r,ref:t})});Xu.displayName="DialogCloseTrigger";const Qu=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe(n.getContentProps(),r.getPresenceProps(),e);return r.unmounted?null:g.jsx(kt.div,{...o,ref:No(r.ref,t)})});Qu.displayName="DialogContent";const Ju=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getDescriptionProps(),e);return g.jsx(kt.div,{...r,ref:t})});Ju.displayName="DialogDescription";const Zu=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getPositionerProps(),e);return Ns().unmounted?null:g.jsx(kt.div,{...r,ref:t})});Zu.displayName="DialogPositioner";var a1=Object.defineProperty,l1=(e,t,n)=>t in e?a1(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,ae=(e,t,n)=>l1(e,typeof t!="symbol"?t+"":t,n),ed={activateTrap(e,t){if(e.length>0){const r=e[e.length-1];r!==t&&r.pause()}const n=e.indexOf(t);n===-1||e.splice(n,1),e.push(t)},deactivateTrap(e,t){const n=e.indexOf(t);n!==-1&&e.splice(n,1),e.length>0&&e[e.length-1].unpause()}},c1=[],u1=class{constructor(e,t){ae(this,"trapStack"),ae(this,"config"),ae(this,"doc"),ae(this,"state",{containers:[],containerGroups:[],tabbableGroups:[],nodeFocusedBeforeActivation:null,mostRecentlyFocusedNode:null,active:!1,paused:!1,delayInitialFocusTimer:void 0,recentNavEvent:void 0}),ae(this,"listenerCleanups",[]),ae(this,"handleFocus",r=>{const o=$e(r),i=this.findContainerIndex(o,r)>=0;if(i||ms(o))i&&(this.state.mostRecentlyFocusedNode=o);else{r.stopImmediatePropagation();let s,a=!0;if(this.state.mostRecentlyFocusedNode)if(Er(this.state.mostRecentlyFocusedNode)>0){const l=this.findContainerIndex(this.state.mostRecentlyFocusedNode),{tabbableNodes:c}=this.state.containerGroups[l];if(c.length>0){const u=c.findIndex(h=>h===this.state.mostRecentlyFocusedNode);u>=0&&(this.config.isKeyForward(this.state.recentNavEvent)?u+1=0&&(s=c[u-1],a=!1))}}else this.state.containerGroups.some(l=>l.tabbableNodes.some(c=>Er(c)>0))||(a=!1);else a=!1;a&&(s=this.findNextNavNode({target:this.state.mostRecentlyFocusedNode,isBackward:this.config.isKeyBackward(this.state.recentNavEvent)})),s?this.tryFocus(s):this.tryFocus(this.state.mostRecentlyFocusedNode||this.getInitialFocusNode())}this.state.recentNavEvent=void 0}),ae(this,"handlePointerDown",r=>{const o=$e(r);if(!(this.findContainerIndex(o,r)>=0)){if(Lr(this.config.clickOutsideDeactivates,r)){this.deactivate({returnFocus:this.config.returnFocusOnDeactivate});return}Lr(this.config.allowOutsideClick,r)||r.preventDefault()}}),ae(this,"handleClick",r=>{const o=$e(r);this.findContainerIndex(o,r)>=0||Lr(this.config.clickOutsideDeactivates,r)||Lr(this.config.allowOutsideClick,r)||(r.preventDefault(),r.stopImmediatePropagation())}),ae(this,"handleTabKey",r=>{if(this.config.isKeyForward(r)||this.config.isKeyBackward(r)){this.state.recentNavEvent=r;const o=this.config.isKeyBackward(r),i=this.findNextNavNode({event:r,isBackward:o});if(!i)return;Fr(r)&&r.preventDefault(),this.tryFocus(i)}}),ae(this,"handleEscapeKey",r=>{d1(r)&&Lr(this.config.escapeDeactivates,r)!==!1&&(r.preventDefault(),this.deactivate())}),ae(this,"_mutationObserver"),ae(this,"setupMutationObserver",()=>{const r=this.doc.defaultView||window;this._mutationObserver=new r.MutationObserver(o=>{o.some(s=>Array.from(s.removedNodes).some(l=>l===this.state.mostRecentlyFocusedNode))&&this.tryFocus(this.getInitialFocusNode())})}),ae(this,"updateObservedNodes",()=>{this._mutationObserver?.disconnect(),this.state.active&&!this.state.paused&&this.state.containers.map(r=>{this._mutationObserver?.observe(r,{subtree:!0,childList:!0})})}),ae(this,"getInitialFocusNode",()=>{let r=this.getNodeForOption("initialFocus",{hasFallback:!0});if(r===!1)return!1;if(r===void 0||r&&!Mt(r)){const o=Sr(this.doc);if(o&&this.findContainerIndex(o)>=0)r=o;else{const i=this.state.tabbableGroups[0];r=i&&i.firstTabbableNode||this.getNodeForOption("fallbackFocus")}}else r===null&&(r=this.getNodeForOption("fallbackFocus"));if(!r)throw new Error("Your focus-trap needs to have at least one focusable element");return r.isConnected||(r=this.getNodeForOption("fallbackFocus")),r}),ae(this,"tryFocus",r=>{if(r!==!1&&r!==Sr(this.doc)){if(!r||!r.focus){this.tryFocus(this.getInitialFocusNode());return}r.focus({preventScroll:!!this.config.preventScroll}),this.state.mostRecentlyFocusedNode=r,h1(r)&&r.select()}}),ae(this,"deactivate",r=>{if(!this.state.active)return this;const o={onDeactivate:this.config.onDeactivate,onPostDeactivate:this.config.onPostDeactivate,checkCanReturnFocus:this.config.checkCanReturnFocus,...r};clearTimeout(this.state.delayInitialFocusTimer),this.state.delayInitialFocusTimer=void 0,this.removeListeners(),this.state.active=!1,this.state.paused=!1,this.updateObservedNodes(),ed.deactivateTrap(this.trapStack,this);const i=this.getOption(o,"onDeactivate"),s=this.getOption(o,"onPostDeactivate"),a=this.getOption(o,"checkCanReturnFocus"),l=this.getOption(o,"returnFocus","returnFocusOnDeactivate");i?.();const c=()=>{td(()=>{if(l){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);this.tryFocus(u)}s?.()})};if(l&&a){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);return a(u).then(c,c),this}return c(),this}),ae(this,"pause",r=>{if(this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onPause"),i=this.getOption(r,"onPostPause");return this.state.paused=!0,o?.(),this.removeListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"unpause",r=>{if(!this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onUnpause"),i=this.getOption(r,"onPostUnpause");return this.state.paused=!1,o?.(),this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"updateContainerElements",r=>(this.state.containers=Array.isArray(r)?r.filter(Boolean):[r].filter(Boolean),this.state.active&&this.updateTabbableNodes(),this.updateObservedNodes(),this)),ae(this,"getReturnFocusNode",r=>{const o=this.getNodeForOption("setReturnFocus",{params:[r]});return o||(o===!1?!1:r)}),ae(this,"getOption",(r,o,i)=>r&&r[o]!==void 0?r[o]:this.config[i||o]),ae(this,"getNodeForOption",(r,{hasFallback:o=!1,params:i=[]}={})=>{let s=this.config[r];if(typeof s=="function"&&(s=s(...i)),s===!0&&(s=void 0),!s){if(s===void 0||s===!1)return s;throw new Error(`\`${r}\` was specified but was not a node, or did not return a node`)}let a=s;if(typeof s=="string"){try{a=this.doc.querySelector(s)}catch(l){throw new Error(`\`${r}\` appears to be an invalid selector; error="${l.message}"`)}if(!a&&!o)throw new Error(`\`${r}\` as selector refers to no known node`)}return a}),ae(this,"findNextNavNode",r=>{const{event:o,isBackward:i=!1}=r,s=r.target||$e(o);this.updateTabbableNodes();let a=null;if(this.state.tabbableGroups.length>0){const l=this.findContainerIndex(s,o),c=l>=0?this.state.containerGroups[l]:void 0;if(l<0)i?a=this.state.tabbableGroups[this.state.tabbableGroups.length-1].lastTabbableNode:a=this.state.tabbableGroups[0].firstTabbableNode;else if(i){let u=this.state.tabbableGroups.findIndex(({firstTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s,!1))&&(u=l),u>=0){const h=u===0?this.state.tabbableGroups.length-1:u-1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.lastTabbableNode:d.lastDomTabbableNode}else Fr(o)||(a=c?.nextTabbableNode(s,!1))}else{let u=this.state.tabbableGroups.findIndex(({lastTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s))&&(u=l),u>=0){const h=u===this.state.tabbableGroups.length-1?0:u+1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.firstTabbableNode:d.firstDomTabbableNode}else Fr(o)||(a=c?.nextTabbableNode(s))}}else a=this.getNodeForOption("fallbackFocus");return a}),this.trapStack=t.trapStack||c1;const n={returnFocusOnDeactivate:!0,escapeDeactivates:!0,delayInitialFocus:!0,isKeyForward(r){return Fr(r)&&!r.shiftKey},isKeyBackward(r){return Fr(r)&&r.shiftKey},...t};this.doc=n.document||Fe(Array.isArray(e)?e[0]:e),this.config=n,this.updateContainerElements(e),this.setupMutationObserver()}get active(){return this.state.active}get paused(){return this.state.paused}findContainerIndex(e,t){const n=typeof t?.composedPath=="function"?t.composedPath():void 0;return this.state.containerGroups.findIndex(({container:r,tabbableNodes:o})=>r.contains(e)||n?.includes(r)||o.find(i=>i===e))}updateTabbableNodes(){if(this.state.containerGroups=this.state.containers.map(e=>{const t=ks(e),n=mc(e),r=t.length>0?t[0]:void 0,o=t.length>0?t[t.length-1]:void 0,i=n.find(c=>Zt(c)),s=n.slice().reverse().find(c=>Zt(c)),a=!!t.find(c=>Er(c)>0);function l(c,u=!0){const h=t.indexOf(c);return h<0?u?n.slice(n.indexOf(c)+1).find(d=>Zt(d)):n.slice(0,n.indexOf(c)).reverse().find(d=>Zt(d)):t[h+(u?1:-1)]}return{container:e,tabbableNodes:t,focusableNodes:n,posTabIndexesFound:a,firstTabbableNode:r,lastTabbableNode:o,firstDomTabbableNode:i,lastDomTabbableNode:s,nextTabbableNode:l}}),this.state.tabbableGroups=this.state.containerGroups.filter(e=>e.tabbableNodes.length>0),this.state.tabbableGroups.length<=0&&!this.getNodeForOption("fallbackFocus"))throw new Error("Your focus-trap must have at least one container with at least one tabbable node in it at all times");if(this.state.containerGroups.find(e=>e.posTabIndexesFound)&&this.state.containerGroups.length>1)throw new Error("At least one node with a positive tabindex was found in one of your focus-trap's multiple containers. Positive tabindexes are only supported in single-container focus-traps.")}addListeners(){if(this.state.active)return ed.activateTrap(this.trapStack,this),this.state.delayInitialFocusTimer=this.config.delayInitialFocus?td(()=>{this.tryFocus(this.getInitialFocusNode())}):this.tryFocus(this.getInitialFocusNode()),this.listenerCleanups.push(se(this.doc,"focusin",this.handleFocus,!0),se(this.doc,"mousedown",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"touchstart",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"click",this.handleClick,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleTabKey,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleEscapeKey)),this}removeListeners(){if(this.state.active)return this.listenerCleanups.forEach(e=>e()),this.listenerCleanups=[],this}activate(e){if(this.state.active)return this;const t=this.getOption(e,"onActivate"),n=this.getOption(e,"onPostActivate"),r=this.getOption(e,"checkCanFocusTrap");r||this.updateTabbableNodes(),this.state.active=!0,this.state.paused=!1,this.state.nodeFocusedBeforeActivation=Sr(this.doc),t?.();const o=()=>{r&&this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),n?.()};return r?(r(this.state.containers.concat()).then(o,o),this):(o(),this)}},Fr=e=>e.key==="Tab",Lr=(e,...t)=>typeof e=="function"?e(...t):e,d1=e=>!e.isComposing&&e.key==="Escape",td=e=>setTimeout(e,0),h1=e=>e.localName==="input"&&"select"in e&&typeof e.select=="function";function f1(e,t={}){let n;const r=G(()=>{const o=typeof e=="function"?e():e;if(o){n=new u1(o,{escapeDeactivates:!1,allowOutsideClick:!0,preventScroll:!0,returnFocusOnDeactivate:!0,delayInitialFocus:!1,fallbackFocus:o,...t,document:Fe(o)});try{n.activate()}catch{}}});return function(){n?.deactivate(),r()}}var Zs="data-scroll-lock";function g1(e){const t=e.getBoundingClientRect().left;return Math.round(t)+e.scrollLeft?"paddingLeft":"paddingRight"}function p1(e){const t=e??document,n=t.defaultView??window,{documentElement:r,body:o}=t;if(o.hasAttribute(Zs))return;const s=n.innerWidth-r.clientWidth;o.setAttribute(Zs,"");const a=()=>wv(r,"--scrollbar-width",`${s}px`),l=g1(r),c=()=>To(o,{overflow:"hidden",[l]:`${s}px`}),u=()=>{const{scrollX:d,scrollY:m,visualViewport:f}=n,p=f?.offsetLeft??0,v=f?.offsetTop??0,b=To(o,{position:"fixed",overflow:"hidden",top:`${-(m-Math.floor(v))}px`,left:`${-(d-Math.floor(p))}px`,right:"0",[l]:`${s}px`});return()=>{b?.(),n.scrollTo({left:d,top:m,behavior:"instant"})}},h=[a(),Co()?u():c()];return()=>{h.forEach(d=>d?.()),o.removeAttribute(Zs)}}var ea=j("dialog").parts("trigger","backdrop","positioner","content","title","description","closeTrigger"),on=ea.build(),nd=e=>e.ids?.positioner??`dialog:${e.id}:positioner`,rd=e=>e.ids?.backdrop??`dialog:${e.id}:backdrop`,ta=e=>e.ids?.content??`dialog:${e.id}:content`,od=e=>e.ids?.trigger??`dialog:${e.id}:trigger`,na=e=>e.ids?.title??`dialog:${e.id}:title`,ra=e=>e.ids?.description??`dialog:${e.id}:description`,id=e=>e.ids?.closeTrigger??`dialog:${e.id}:close`,ei=e=>e.getById(ta(e)),m1=e=>e.getById(nd(e)),v1=e=>e.getById(rd(e)),b1=e=>e.getById(od(e)),y1=e=>e.getById(na(e)),x1=e=>e.getById(ra(e)),k1=e=>e.getById(id(e));function C1(e,t){const{state:n,send:r,context:o,prop:i,scope:s}=e,a=i("aria-label"),l=n.matches("open");return{open:l,setOpen(c){n.matches("open")!==c&&r({type:c?"OPEN":"CLOSE"})},getTriggerProps(){return t.button({...on.trigger.attrs,dir:i("dir"),id:od(s),"aria-haspopup":"dialog",type:"button","aria-expanded":l,"data-state":l?"open":"closed","aria-controls":ta(s),onClick(c){c.defaultPrevented||r({type:"TOGGLE"})}})},getBackdropProps(){return t.element({...on.backdrop.attrs,dir:i("dir"),hidden:!l,id:rd(s),"data-state":l?"open":"closed"})},getPositionerProps(){return t.element({...on.positioner.attrs,dir:i("dir"),id:nd(s),style:{pointerEvents:l?void 0:"none"}})},getContentProps(){const c=o.get("rendered");return t.element({...on.content.attrs,dir:i("dir"),role:i("role"),hidden:!l,id:ta(s),tabIndex:-1,"data-state":l?"open":"closed","aria-modal":!0,"aria-label":a||void 0,"aria-labelledby":a||!c.title?void 0:na(s),"aria-describedby":c.description?ra(s):void 0})},getTitleProps(){return t.element({...on.title.attrs,dir:i("dir"),id:na(s)})},getDescriptionProps(){return t.element({...on.description.attrs,dir:i("dir"),id:ra(s)})},getCloseTriggerProps(){return t.button({...on.closeTrigger.attrs,dir:i("dir"),id:id(s),type:"button",onClick(c){c.defaultPrevented||(c.stopPropagation(),r({type:"CLOSE"}))}})}}}var w1={props({props:e,scope:t}){const n=e.role==="alertdialog",r=n?()=>k1(t):void 0,o=typeof e.modal=="boolean"?e.modal:!0;return{role:"dialog",modal:o,trapFocus:o,preventScroll:o,closeOnInteractOutside:!n,closeOnEscape:!0,restoreFocus:!0,initialFocusEl:r,...e}},initialState({prop:e}){return e("open")||e("defaultOpen")?"open":"closed"},context({bindable:e}){return{rendered:e(()=>({defaultValue:{title:!0,description:!0}}))}},watch({track:e,action:t,prop:n}){e([()=>n("open")],()=>{t(["toggleVisibility"])})},states:{open:{entry:["checkRenderedElements","syncZIndex"],effects:["trackDismissableElement","trapFocus","preventScroll","hideContentBelow"],on:{"CONTROLLED.CLOSE":{target:"closed"},CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}]}},closed:{on:{"CONTROLLED.OPEN":{target:"open"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}]}}},implementations:{guards:{isOpenControlled:({prop:e})=>e("open")!=null},effects:{trackDismissableElement({scope:e,send:t,prop:n}){return Kn(()=>ei(e),{type:"dialog",defer:!0,pointerBlocking:n("modal"),exclude:[b1(e)],onInteractOutside(o){n("onInteractOutside")?.(o),n("closeOnInteractOutside")||o.preventDefault()},persistentElements:n("persistentElements"),onFocusOutside:n("onFocusOutside"),onPointerDownOutside:n("onPointerDownOutside"),onRequestDismiss:n("onRequestDismiss"),onEscapeKeyDown(o){n("onEscapeKeyDown")?.(o),n("closeOnEscape")||o.preventDefault()},onDismiss(){t({type:"CLOSE",src:"interact-outside"})}})},preventScroll({scope:e,prop:t}){if(t("preventScroll"))return p1(e.getDoc())},trapFocus({scope:e,prop:t}){return t("trapFocus")?f1(()=>ei(e),{preventScroll:!0,returnFocusOnDeactivate:!!t("restoreFocus"),initialFocus:t("initialFocusEl"),setReturnFocus:r=>t("finalFocusEl")?.()??r}):void 0},hideContentBelow({scope:e,prop:t}){return t("modal")?ju(()=>[ei(e)],{defer:!0}):void 0}},actions:{checkRenderedElements({context:e,scope:t}){G(()=>{e.set("rendered",{title:!!y1(t),description:!!x1(t)})})},syncZIndex({scope:e}){G(()=>{const t=ei(e);if(!t)return;const n=xo(t);[m1(e),v1(e)].forEach(o=>{o?.style.setProperty("--z-index",n.zIndex),o?.style.setProperty("--layer-index",n.getPropertyValue("--layer-index"))})})},invokeOnClose({prop:e}){e("onOpenChange")?.({open:!1})},invokeOnOpen({prop:e}){e("onOpenChange")?.({open:!0})},toggleVisibility({prop:e,send:t,event:n}){t({type:e("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:n})}}}};M()(["aria-label","closeOnEscape","closeOnInteractOutside","dir","finalFocusEl","getRootNode","getRootNode","id","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","preventScroll","restoreFocus","role","trapFocus"]);const S1=e=>{const t=P.useId(),{getRootNode:n}=sc(),{dir:r}=_v(),o={id:t,getRootNode:n,dir:r,...e},i=Pc(w1,o);return C1(i,jv)},E1=e=>{const[t,{children:n,...r}]=Ec(e),[o]=yu(t),i=S1(r),s=Ts(qe({present:i.open},t));return g.jsx(Ku,{value:i,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:s,children:n})})})},O1=e=>{const[t,{value:n,children:r}]=Ec(e),[o]=yu(t),i=Ts(qe({present:n.open},t));return g.jsx(Ku,{value:n,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:i,children:r})})})},sd=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getTitleProps(),e);return g.jsx(kt.h2,{...r,ref:t})});sd.displayName="DialogTitle";const ad=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe({...n.getTriggerProps(),"aria-controls":r.unmounted?void 0:n.getTriggerProps()["aria-controls"]},e);return g.jsx(kt.button,{...o,ref:t})});ad.displayName="DialogTrigger";var ld=j("editable").parts("root","area","label","preview","input","editTrigger","submitTrigger","cancelTrigger","control");ld.build(),M()(["activationMode","autoResize","dir","disabled","finalFocusEl","form","getRootNode","id","ids","invalid","maxLength","name","onEditChange","onFocusOutside","onInteractOutside","onPointerDownOutside","onValueChange","onValueCommit","onValueRevert","placeholder","readOnly","required","selectOnFocus","edit","defaultEdit","submitMode","translations","defaultValue","value"]);const cd=P.forwardRef((e,t)=>{const n=Iu(),r=qe(n?.getInputProps(),e);return g.jsx(kt.input,{...r,ref:t})});cd.displayName="FieldInput";const ud=j("field").parts("root","errorText","helperText","input","label","select","textarea","requiredIndicator");ud.build();var P1=e=>{if(!e)return;const t=xo(e),n=be(e),r=Fe(e),o=()=>{requestAnimationFrame(()=>{e.style.height="auto";let c;t.boxSizing==="content-box"?c=e.scrollHeight-(parseFloat(t.paddingTop)+parseFloat(t.paddingBottom)):c=e.scrollHeight+parseFloat(t.borderTopWidth)+parseFloat(t.borderBottomWidth),t.maxHeight!=="none"&&c>parseFloat(t.maxHeight)?(t.overflowY==="hidden"&&(e.style.overflowY="scroll"),c=parseFloat(t.maxHeight)):t.overflowY!=="hidden"&&(e.style.overflowY="hidden"),e.style.height=`${c}px`})};e.addEventListener("input",o),e.form?.addEventListener("reset",o);const i=Object.getPrototypeOf(e),s=Object.getOwnPropertyDescriptor(i,"value");Object.defineProperty(e,"value",{...s,set(){s?.set?.apply(this,arguments),o()}});const a=new n.ResizeObserver(()=>{requestAnimationFrame(()=>o())});a.observe(e);const l=new n.MutationObserver(()=>o());return l.observe(e,{attributes:!0,attributeFilter:["rows","placeholder"]}),r.fonts?.addEventListener("loadingdone",o),()=>{e.removeEventListener("input",o),e.form?.removeEventListener("reset",o),r.fonts?.removeEventListener("loadingdone",o),a.disconnect(),l.disconnect()}};const dd=P.forwardRef((e,t)=>{const{autoresize:n,...r}=e,o=P.useRef(null),i=Iu(),s=qe(i?.getTextareaProps(),{style:{resize:n?"none":void 0}},r);return P.useEffect(()=>{if(n)return P1(o.current)},[n]),g.jsx(kt.textarea,{...s,ref:No(t,o)})});dd.displayName="FieldTextarea";const hd=j("fieldset").parts("root","errorText","helperText","legend");hd.build();var fd=j("file-upload").parts("root","dropzone","item","itemDeleteTrigger","itemGroup","itemName","itemPreview","itemPreviewImage","itemSizeText","label","trigger","clearTrigger");fd.build(),M()(["accept","acceptedFiles","allowDrop","capture","defaultAcceptedFiles","dir","directory","disabled","getRootNode","id","ids","invalid","locale","maxFiles","maxFileSize","minFileSize","name","onFileAccept","onFileChange","onFileReject","preventDocumentDrop","required","transformFiles","translations","validate"]),M()(["file","type"]);var gd=j("hoverCard").parts("arrow","arrowTip","trigger","positioner","content");gd.build();var R1=e=>e.ids?.trigger??`hover-card:${e.id}:trigger`,I1=e=>e.ids?.content??`hover-card:${e.id}:content`,T1=e=>e.ids?.positioner??`hover-card:${e.id}:popper`,oa=e=>e.getById(R1(e)),N1=e=>e.getById(I1(e)),pd=e=>e.getById(T1(e)),{not:ti,and:md}=jt();md("isOpenControlled",ti("isPointer")),ti("isPointer"),md("isOpenControlled",ti("isPointer")),ti("isPointer"),M()(["closeDelay","dir","getRootNode","id","ids","disabled","onOpenChange","defaultOpen","open","openDelay","positioning","onInteractOutside","onPointerDownOutside","onFocusOutside"]);var vd=j("tree-view").parts("branch","branchContent","branchControl","branchIndentGuide","branchIndicator","branchText","branchTrigger","item","itemIndicator","itemText","label","nodeCheckbox","root","tree");vd.build();var bd=e=>new zu(e);bd.empty=()=>new zu({rootNode:{children:[]}});var A1=(e,t)=>e.ids?.node?.(t)??`tree:${e.id}:node:${t}`,sn=(e,t)=>{t!=null&&e.getById(A1(e,t))?.focus()};function _1(e,t,n){const r=e.getDescendantValues(t),o=r.every(i=>n.includes(i));return Dn(o?Yt(n,...r):Kt(n,...r))}function ni(e,t){const{context:n,prop:r,refs:o}=e;if(!r("loadChildren")){n.set("expandedValue",p=>Dn(Kt(p,...t)));return}const i=n.get("loadingStatus"),[s,a]=ql(t,p=>i[p]==="loaded");if(s.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...s))),a.length===0)return;const l=r("collection"),[c,u]=ql(a,p=>{const v=l.findNode(p);return l.getNodeChildren(v).length>0});if(c.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...c))),u.length===0)return;n.set("loadingStatus",p=>({...p,...u.reduce((v,b)=>({...v,[b]:"loading"}),{})}));const h=u.map(p=>{const v=l.getIndexPath(p),b=l.getValuePath(v),y=l.findNode(p);return{id:p,indexPath:v,valuePath:b,node:y}}),d=o.get("pendingAborts"),m=r("loadChildren");ic(m,()=>"[zag-js/tree-view] `loadChildren` is required for async expansion");const f=h.map(({id:p,indexPath:v,valuePath:b,node:y})=>{const x=d.get(p);x&&(x.abort(),d.delete(p));const w=new AbortController;return d.set(p,w),m({valuePath:b,indexPath:v,node:y,signal:w.signal})});Promise.allSettled(f).then(p=>{const v=[],b=[],y=n.get("loadingStatus");let x=r("collection");p.forEach((w,k)=>{const{id:O,indexPath:R,node:I,valuePath:N}=h[k];w.status==="fulfilled"?(y[O]="loaded",v.push(O),x=x.replace(R,{...I,children:w.value})):(d.delete(O),Reflect.deleteProperty(y,O),b.push({node:I,error:w.reason,indexPath:R,valuePath:N}))}),n.set("loadingStatus",y),v.length&&(n.set("expandedValue",w=>Dn(Kt(w,...v))),r("onLoadChildrenComplete")?.({collection:x})),b.length&&r("onLoadChildrenError")?.({nodes:b})})}function Ut(e){const{prop:t,context:n}=e;return function({indexPath:o}){return t("collection").getValuePath(o).slice(0,-1).some(s=>!n.get("expandedValue").includes(s))}}var{and:ft}=jt();ft("isMultipleSelection","moveFocus"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isBranchFocused","isBranchExpanded"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),M()(["ids","collection","dir","expandedValue","expandOnClick","defaultFocusedValue","focusedValue","getRootNode","id","onExpandedChange","onFocusChange","onSelectionChange","checkedValue","selectedValue","selectionMode","typeahead","defaultExpandedValue","defaultSelectedValue","defaultCheckedValue","onCheckedChange","onLoadChildrenComplete","onLoadChildrenError","loadChildren"]),M()(["node","indexPath"]);var yd=j("listbox").parts("label","input","item","itemText","itemIndicator","itemGroup","itemGroupLabel","content","root","valueText");yd.build(),M()(["collection","defaultHighlightedValue","defaultValue","dir","disabled","deselectable","disallowSelectAll","getRootNode","highlightedValue","id","ids","loopFocus","onHighlightChange","onSelect","onValueChange","orientation","scrollToIndexFn","selectionMode","selectOnHighlight","typeahead","value"]),M()(["item","highlightOnHover"]),M()(["id"]),M()(["htmlFor"]);const V1=yd.extendWith("empty");var xd=j("menu").parts("arrow","arrowTip","content","contextTrigger","indicator","item","itemGroup","itemGroupLabel","itemIndicator","itemText","positioner","separator","trigger","triggerItem");xd.build();var kd=e=>e.ids?.trigger??`menu:${e.id}:trigger`,F1=e=>e.ids?.contextTrigger??`menu:${e.id}:ctx-trigger`,Cd=e=>e.ids?.content??`menu:${e.id}:content`,L1=e=>e.ids?.positioner??`menu:${e.id}:popper`,ia=(e,t)=>`${e.id}/${t}`,an=e=>e?.dataset.value??null,Gt=e=>e.getById(Cd(e)),wd=e=>e.getById(L1(e)),ri=e=>e.getById(kd(e)),D1=(e,t)=>t?e.getById(ia(e,t)):null,sa=e=>e.getById(F1(e)),Dr=e=>{const n=`[role^="menuitem"][data-ownedby=${CSS.escape(Cd(e))}]:not([data-disabled])`;return Io(Gt(e),n)},z1=e=>kr(Dr(e)),M1=e=>hs(Dr(e)),aa=(e,t)=>t?e.id===t||e.dataset.value===t:!1,j1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return Um(n,r,{loop:t.loop??t.loopFocus})},$1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return qm(n,r,{loop:t.loop??t.loopFocus})},B1=(e,t)=>{const n=Dr(e),r=n.find(o=>aa(o,t.value));return Or(n,{state:t.typeaheadState,key:t.key,activeId:r?.id??null})},W1=e=>!!e?.getAttribute("role")?.startsWith("menuitem")&&!!e?.hasAttribute("aria-controls"),H1="menu:select";function U1(e,t){if(!e)return;const n=be(e),r=new n.CustomEvent(H1,{detail:{value:t}});e.dispatchEvent(r)}var{not:Xe,and:Jn,or:G1}=jt();Xe("isSubmenu"),G1("isOpenAutoFocusEvent","isArrowDownEvent"),Jn(Xe("isTriggerItem"),"isOpenControlled"),Xe("isTriggerItem"),Jn("isSubmenu","isOpenControlled"),Xe("isPointerSuspended"),Jn(Xe("isPointerSuspended"),Xe("isTriggerItem")),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect","isOpenControlled"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"));function Sd(e){let t=e.parent;for(;t&&t.context.get("isSubmenu");)t=t.refs.get("parent");t?.send({type:"CLOSE"})}function q1(e,t){return e?gx(e,t):!1}function K1(e,t,n){const r=Object.keys(e).length>0;if(!t)return null;if(!r)return ia(n,t);for(const o in e){const i=e[o],s=kd(i.scope);if(s===t)return s}return ia(n,t)}M()(["anchorPoint","aria-label","closeOnSelect","composite","defaultHighlightedValue","defaultOpen","dir","getRootNode","highlightedValue","id","ids","loopFocus","navigate","onEscapeKeyDown","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","onSelect","open","positioning","typeahead"]),M()(["closeOnSelect","disabled","value","valueText"]),M()(["htmlFor"]),M()(["id"]),M()(["checked","closeOnSelect","disabled","onCheckedChange","type","value","valueText"]);let la=new Map,ca=!1;try{ca=new Intl.NumberFormat("de-DE",{signDisplay:"exceptZero"}).resolvedOptions().signDisplay==="exceptZero"}catch{}let oi=!1;try{oi=new Intl.NumberFormat("de-DE",{style:"unit",unit:"degree"}).resolvedOptions().style==="unit"}catch{}const Ed={degree:{narrow:{default:"°","ja-JP":" 度","zh-TW":"度","sl-SI":" °"}}};class Y1{format(t){let n="";if(!ca&&this.options.signDisplay!=null?n=Q1(this.numberFormatter,this.options.signDisplay,t):n=this.numberFormatter.format(t),this.options.style==="unit"&&!oi){var r;let{unit:o,unitDisplay:i="short",locale:s}=this.resolvedOptions();if(!o)return n;let a=(r=Ed[o])===null||r===void 0?void 0:r[i];n+=a[s]||a.default}return n}formatToParts(t){return this.numberFormatter.formatToParts(t)}formatRange(t,n){if(typeof this.numberFormatter.formatRange=="function")return this.numberFormatter.formatRange(t,n);if(n= start date");return`${this.format(t)} – ${this.format(n)}`}formatRangeToParts(t,n){if(typeof this.numberFormatter.formatRangeToParts=="function")return this.numberFormatter.formatRangeToParts(t,n);if(n= start date");let r=this.numberFormatter.formatToParts(t),o=this.numberFormatter.formatToParts(n);return[...r.map(i=>({...i,source:"startRange"})),{type:"literal",value:" – ",source:"shared"},...o.map(i=>({...i,source:"endRange"}))]}resolvedOptions(){let t=this.numberFormatter.resolvedOptions();return!ca&&this.options.signDisplay!=null&&(t={...t,signDisplay:this.options.signDisplay}),!oi&&this.options.style==="unit"&&(t={...t,style:"unit",unit:this.options.unit,unitDisplay:this.options.unitDisplay}),t}constructor(t,n={}){this.numberFormatter=X1(t,n),this.options=n}}function X1(e,t={}){let{numberingSystem:n}=t;if(n&&e.includes("-nu-")&&(e.includes("-u-")||(e+="-u-"),e+=`-nu-${n}`),t.style==="unit"&&!oi){var r;let{unit:s,unitDisplay:a="short"}=t;if(!s)throw new Error('unit option must be provided with style: "unit"');if(!(!((r=Ed[s])===null||r===void 0)&&r[a]))throw new Error(`Unsupported unit ${s} with unitDisplay = ${a}`);t={...t,style:"decimal"}}let o=e+(t?Object.entries(t).sort((s,a)=>s[0]0||Object.is(n,0):t==="exceptZero"&&(Object.is(n,-0)||Object.is(n,0)?n=Math.abs(n):r=n>0),r){let o=e.format(-n),i=e.format(n),s=o.replace(i,"").replace(/\u200e|\u061C/,"");return[...s].length!==1&&console.warn("@react-aria/i18n polyfill for NumberFormat signDisplay: Unsupported case"),o.replace(i,"!!!").replace(s,"+").replace("!!!",i)}else return e.format(n)}}const J1=new RegExp("^.*\\(.*\\).*$"),Z1=["latn","arab","hanidec","deva","beng","fullwide"];class Od{parse(t){return ua(this.locale,this.options,t).parse(t)}isValidPartialNumber(t,n,r){return ua(this.locale,this.options,t).isValidPartialNumber(t,n,r)}getNumberingSystem(t){return ua(this.locale,this.options,t).options.numberingSystem}constructor(t,n={}){this.locale=t,this.options=n}}const Pd=new Map;function ua(e,t,n){let r=Rd(e,t);if(!e.includes("-nu-")&&!r.isValidPartialNumber(n)){for(let o of Z1)if(o!==r.options.numberingSystem){let i=Rd(e+(e.includes("-u-")?"-nu-":"-u-nu-")+o,t);if(i.isValidPartialNumber(n))return i}}return r}function Rd(e,t){let n=e+(t?Object.entries(t).sort((o,i)=>o[0]-1&&(n=`-${n}`)}let r=n?+n:NaN;if(isNaN(r))return NaN;if(this.options.style==="percent"){var o,i;let s={...this.options,style:"decimal",minimumFractionDigits:Math.min(((o=this.options.minimumFractionDigits)!==null&&o!==void 0?o:0)+2,20),maximumFractionDigits:Math.min(((i=this.options.maximumFractionDigits)!==null&&i!==void 0?i:0)+2,20)};return new Od(this.locale,s).parse(new Y1(this.locale,s).format(r))}return this.options.currencySign==="accounting"&&J1.test(t)&&(r=-1*r),r}sanitize(t){return t=t.replace(this.symbols.literals,""),this.symbols.minusSign&&(t=t.replace("-",this.symbols.minusSign)),this.options.numberingSystem==="arab"&&(this.symbols.decimal&&(t=t.replace(",",this.symbols.decimal),t=t.replace("،",this.symbols.decimal)),this.symbols.group&&(t=Zn(t,".",this.symbols.group))),this.symbols.group==="’"&&t.includes("'")&&(t=Zn(t,"'",this.symbols.group)),this.options.locale==="fr-FR"&&this.symbols.group&&(t=Zn(t," ",this.symbols.group),t=Zn(t,/\u00A0/g,this.symbols.group)),t}isValidPartialNumber(t,n=-1/0,r=1/0){return t=this.sanitize(t),this.symbols.minusSign&&t.startsWith(this.symbols.minusSign)&&n<0?t=t.slice(this.symbols.minusSign.length):this.symbols.plusSign&&t.startsWith(this.symbols.plusSign)&&r>0&&(t=t.slice(this.symbols.plusSign.length)),this.symbols.group&&t.startsWith(this.symbols.group)||this.symbols.decimal&&t.indexOf(this.symbols.decimal)>-1&&this.options.maximumFractionDigits===0?!1:(this.symbols.group&&(t=Zn(t,this.symbols.group,"")),t=t.replace(this.symbols.numeral,""),this.symbols.decimal&&(t=t.replace(this.symbols.decimal,"")),t.length===0)}constructor(t,n={}){this.locale=t,n.roundingIncrement!==1&&n.roundingIncrement!=null&&(n.maximumFractionDigits==null&&n.minimumFractionDigits==null?(n.maximumFractionDigits=0,n.minimumFractionDigits=0):n.maximumFractionDigits==null?n.maximumFractionDigits=n.minimumFractionDigits:n.minimumFractionDigits==null&&(n.minimumFractionDigits=n.maximumFractionDigits)),this.formatter=new Intl.NumberFormat(t,n),this.options=this.formatter.resolvedOptions(),this.symbols=nk(t,this.formatter,this.options,n);var r,o;this.options.style==="percent"&&(((r=this.options.minimumFractionDigits)!==null&&r!==void 0?r:0)>18||((o=this.options.maximumFractionDigits)!==null&&o!==void 0?o:0)>18)&&console.warn("NumberParser cannot handle percentages with greater than 18 decimal places, please reduce the number in your options.")}}const Id=new Set(["decimal","fraction","integer","minusSign","plusSign","group"]),tk=[0,4,2,1,11,20,3,7,100,21,.1,1.1];function nk(e,t,n,r){var o,i,s,a;let l=new Intl.NumberFormat(e,{...n,minimumSignificantDigits:1,maximumSignificantDigits:21,roundingIncrement:1,roundingPriority:"auto",roundingMode:"halfExpand"}),c=l.formatToParts(-10000.111),u=l.formatToParts(10000.111),h=tk.map(A=>l.formatToParts(A));var d;let m=(d=(o=c.find(A=>A.type==="minusSign"))===null||o===void 0?void 0:o.value)!==null&&d!==void 0?d:"-",f=(i=u.find(A=>A.type==="plusSign"))===null||i===void 0?void 0:i.value;!f&&(r?.signDisplay==="exceptZero"||r?.signDisplay==="always")&&(f="+");let v=(s=new Intl.NumberFormat(e,{...n,minimumFractionDigits:2,maximumFractionDigits:2}).formatToParts(.001).find(A=>A.type==="decimal"))===null||s===void 0?void 0:s.value,b=(a=c.find(A=>A.type==="group"))===null||a===void 0?void 0:a.value,y=c.filter(A=>!Id.has(A.type)).map(A=>Td(A.value)),x=h.flatMap(A=>A.filter(T=>!Id.has(T.type)).map(T=>Td(T.value))),w=[...new Set([...y,...x])].sort((A,T)=>T.length-A.length),k=w.length===0?new RegExp("[\\p{White_Space}]","gu"):new RegExp(`${w.join("|")}|[\\p{White_Space}]`,"gu"),O=[...new Intl.NumberFormat(n.locale,{useGrouping:!1}).format(9876543210)].reverse(),R=new Map(O.map((A,T)=>[A,T])),I=new RegExp(`[${O.join("")}]`,"g");return{minusSign:m,plusSign:f,decimal:v,group:b,literals:k,numeral:I,index:A=>String(R.get(A))}}function Zn(e,t,n){return e.replaceAll?e.replaceAll(t,n):e.split(t).join(n)}function Td(e){return e.replace(/[.*+?^${}()|[\]\\]/g,"\\$&")}var Nd=j("numberInput").parts("root","label","input","control","valueText","incrementTrigger","decrementTrigger","scrubber");Nd.build();var rk=e=>e.ids?.input??`number-input:${e.id}:input`,ok=e=>e.ids?.incrementTrigger??`number-input:${e.id}:inc`,ik=e=>e.ids?.decrementTrigger??`number-input:${e.id}:dec`,Ad=e=>`number-input:${e.id}:cursor`,ii=e=>e.getById(rk(e)),sk=e=>e.getById(ok(e)),ak=e=>e.getById(ik(e)),_d=e=>e.getDoc().getElementById(Ad(e)),lk=(e,t)=>{let n=null;return t==="increment"&&(n=sk(e)),t==="decrement"&&(n=ak(e)),n},ck=(e,t)=>{if(!fc())return hk(e,t),()=>{_d(e)?.remove()}},uk=e=>{const t=e.getDoc(),n=t.documentElement,r=t.body;return r.style.pointerEvents="none",n.style.userSelect="none",n.style.cursor="ew-resize",()=>{r.style.pointerEvents="",n.style.userSelect="",n.style.cursor="",n.style.length||n.removeAttribute("style"),r.style.length||r.removeAttribute("style")}},dk=(e,t)=>{const{point:n,isRtl:r,event:o}=t,i=e.getWin(),s=ps(o.movementX,i.devicePixelRatio),a=ps(o.movementY,i.devicePixelRatio);let l=s>0?"increment":s<0?"decrement":null;r&&l==="increment"&&(l="decrement"),r&&l==="decrement"&&(l="increment");const c={x:n.x+s,y:n.y+a},u=i.innerWidth,h=ps(7.5,i.devicePixelRatio);return c.x=u0(c.x+h,u)-h,{hint:l,point:c}},hk=(e,t)=>{const n=e.getDoc(),r=n.createElement("div");r.className="scrubber--cursor",r.id=Ad(e),Object.assign(r.style,{width:"15px",height:"15px",position:"fixed",pointerEvents:"none",left:"0px",top:"0px",zIndex:w0,transform:t?`translate3d(${t.x}px, ${t.y}px, 0px)`:void 0,willChange:"transform"}),r.innerHTML=` - `,n.body.appendChild(r)};function rk(e,t){if(!(!e||!t.isActiveElement(e)))try{const{selectionStart:n,selectionEnd:r,value:o}=e,i=o.substring(0,n),s=o.substring(r);return{start:n,end:r,value:o,beforeTxt:i,afterTxt:s}}catch{}}function ok(e,t,n){if(!(!e||!n.isActiveElement(e))){if(!t){e.setSelectionRange(e.value.length,e.value.length);return}try{const{value:r}=e,{beforeTxt:o="",afterTxt:i="",start:s}=t;let a=r.length;if(r.endsWith(i))a=r.length-i.length;else if(r.startsWith(o))a=o.length;else if(s!=null){const l=o[s-1],c=r.indexOf(l,s-1);c!==-1&&(a=c+1)}e.setSelectionRange(a,a)}catch{}}}var ik=(e,t={})=>new Intl.NumberFormat(e,t),sk=(e,t={})=>new vd(e,t),aa=(e,t)=>{const{prop:n,computed:r}=t;return n("formatOptions")?e===""?Number.NaN:r("parser").parse(e):parseFloat(e)},sn=(e,t)=>{const{prop:n,computed:r}=t;return Number.isNaN(e)?"":n("formatOptions")?r("formatter").format(e):e.toString()},ak=(e,t)=>{let n=e!==void 0&&!Number.isNaN(e)?e:1;return t?.style==="percent"&&(e===void 0||Number.isNaN(e))&&(n=.01),n},{choose:lk,guards:ck,createMachine:uk}=gc(),{not:Ed,and:Od}=ck;uk({props({props:e}){const t=ak(e.step,e.formatOptions);return{dir:"ltr",locale:"en-US",focusInputOnChange:!0,clampValueOnBlur:!e.allowOverflow,allowOverflow:!1,inputMode:"decimal",pattern:"-?[0-9]*(.[0-9]+)?",defaultValue:"",step:t,min:Number.MIN_SAFE_INTEGER,max:Number.MAX_SAFE_INTEGER,spinOnPress:!0,...e,translations:{incrementLabel:"increment value",decrementLabel:"decrease value",...e.translations}}},initialState(){return"idle"},context({prop:e,bindable:t,getComputed:n}){return{value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),onChange(r){const o=n(),i=aa(r,{computed:o,prop:e});e("onValueChange")?.({value:r,valueAsNumber:i})}})),hint:t(()=>({defaultValue:null})),scrubberCursorPoint:t(()=>({defaultValue:null,hash(r){return r?`x:${r.x}, y:${r.y}`:""}})),fieldsetDisabled:t(()=>({defaultValue:!1}))}},computed:{isRtl:({prop:e})=>e("dir")==="rtl",valueAsNumber:({context:e,computed:t,prop:n})=>aa(e.get("value"),{computed:t,prop:n}),formattedValue:({computed:e,prop:t})=>sn(e("valueAsNumber"),{computed:e,prop:t}),isAtMin:({computed:e,prop:t})=>n0(e("valueAsNumber"),t("min")),isAtMax:({computed:e,prop:t})=>t0(e("valueAsNumber"),t("max")),isOutOfRange:({computed:e,prop:t})=>!r0(e("valueAsNumber"),t("min"),t("max")),isValueEmpty:({context:e})=>e.get("value")==="",isDisabled:({prop:e,context:t})=>!!e("disabled")||t.get("fieldsetDisabled"),canIncrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMax"),canDecrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMin"),valueText:({prop:e,context:t})=>e("translations").valueText?.(t.get("value")),formatter:fc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>ik(e,t)),parser:fc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>sk(e,t))},watch({track:e,action:t,context:n,computed:r,prop:o}){e([()=>n.get("value"),()=>o("locale")],()=>{t(["syncInputElement"])}),e([()=>r("isOutOfRange")],()=>{t(["invokeOnInvalid"])}),e([()=>n.hash("scrubberCursorPoint")],()=>{t(["setVirtualCursorPosition"])})},effects:["trackFormControl"],on:{"VALUE.SET":{actions:["setRawValue"]},"VALUE.CLEAR":{actions:["clearValue"]},"VALUE.INCREMENT":{actions:["increment"]},"VALUE.DECREMENT":{actions:["decrement"]}},states:{idle:{on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","invokeOnFocus","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","invokeOnFocus","setHint","setCursorPoint"]},"INPUT.FOCUS":{target:"focused",actions:["focusInput","invokeOnFocus"]}}},focused:{tags:["focus"],effects:["attachWheelListener"],on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","setHint","setCursorPoint"]},"INPUT.ARROW_UP":{actions:["increment"]},"INPUT.ARROW_DOWN":{actions:["decrement"]},"INPUT.HOME":{actions:["decrementToMin"]},"INPUT.END":{actions:["incrementToMax"]},"INPUT.CHANGE":{actions:["setValue","setHint"]},"INPUT.BLUR":[{guard:Od("clampValueOnBlur",Ed("isInRange")),target:"idle",actions:["setClampedValue","clearHint","invokeOnBlur"]},{guard:Ed("isInRange"),target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur","invokeOnInvalid"]},{target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur"]}],"INPUT.ENTER":{actions:["setFormattedValue","clearHint","invokeOnBlur"]}}},"before:spin":{tags:["focus"],effects:["trackButtonDisabled","waitForChangeDelay"],entry:lk([{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}]),on:{CHANGE_DELAY:{target:"spinning",guard:Od("isInRange","spinOnPress")},"TRIGGER.PRESS_UP":[{guard:"isTouchPointer",target:"focused",actions:["clearHint"]},{target:"focused",actions:["focusInput","clearHint"]}]}},spinning:{tags:["focus"],effects:["trackButtonDisabled","spinValue"],on:{SPIN:[{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}],"TRIGGER.PRESS_UP":{target:"focused",actions:["focusInput","clearHint"]}}},scrubbing:{tags:["focus"],effects:["activatePointerLock","trackMousemove","setupVirtualCursor","preventTextSelection"],on:{"SCRUBBER.POINTER_UP":{target:"focused",actions:["focusInput","clearCursorPoint"]},"SCRUBBER.POINTER_MOVE":[{guard:"isIncrementHint",actions:["increment","setCursorPoint"]},{guard:"isDecrementHint",actions:["decrement","setCursorPoint"]}]}}},implementations:{guards:{clampValueOnBlur:({prop:e})=>e("clampValueOnBlur"),spinOnPress:({prop:e})=>!!e("spinOnPress"),isInRange:({computed:e})=>!e("isOutOfRange"),isDecrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="decrement",isIncrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="increment",isTouchPointer:({event:e})=>e.pointerType==="touch"},effects:{waitForChangeDelay({send:e}){const t=setTimeout(()=>{e({type:"CHANGE_DELAY"})},300);return()=>clearTimeout(t)},spinValue({send:e}){const t=setInterval(()=>{e({type:"SPIN"})},50);return()=>clearInterval(t)},trackFormControl({context:e,scope:t}){const n=ni(t);return ps(n,{onFieldsetDisabledChange(r){e.set("fieldsetDisabled",r)},onFormReset(){e.set("value",e.initial("value"))}})},setupVirtualCursor({context:e,scope:t}){const n=e.get("scrubberCursorPoint");return Z1(t,n)},preventTextSelection({scope:e}){return ek(e)},trackButtonDisabled({context:e,scope:t,send:n}){const r=e.get("hint"),o=J1(t,r);return Co(o,{attributes:["disabled"],callback(){n({type:"TRIGGER.PRESS_UP",src:"attr"})}})},attachWheelListener({scope:e,send:t,prop:n}){const r=ni(e);if(!r||!e.isActiveElement(r)||!n("allowMouseWheel"))return;function o(i){i.preventDefault();const s=Math.sign(i.deltaY)*-1;s===1?t({type:"VALUE.INCREMENT"}):s===-1&&t({type:"VALUE.DECREMENT"})}return se(r,"wheel",o,{passive:!1})},activatePointerLock({scope:e}){if(!oc())return tv(e.getDoc())},trackMousemove({scope:e,send:t,context:n,computed:r}){const o=e.getDoc();function i(a){const l=n.get("scrubberCursorPoint"),c=r("isRtl"),u=tk(e,{point:l,isRtl:c,event:a});u.hint&&t({type:"SCRUBBER.POINTER_MOVE",hint:u.hint,point:u.point})}function s(){t({type:"SCRUBBER.POINTER_UP"})}return fo(se(o,"mousemove",i,!1),se(o,"mouseup",s,!1))}},actions:{focusInput({scope:e,prop:t}){if(!t("focusInputOnChange"))return;const n=ni(e);e.isActiveElement(n)||G(()=>n?.focus({preventScroll:!0}))},increment({context:e,event:t,prop:n,computed:r}){let o=a0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",sn(o,{computed:r,prop:n}))},decrement({context:e,event:t,prop:n,computed:r}){let o=l0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",sn(o,{computed:r,prop:n}))},setClampedValue({context:e,prop:t,computed:n}){const r=Ve(n("valueAsNumber"),t("min"),t("max"));e.set("value",sn(r,{computed:n,prop:t}))},setRawValue({context:e,event:t,prop:n,computed:r}){let o=aa(t.value,{computed:r,prop:n});n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",sn(o,{computed:r,prop:n}))},setValue({context:e,event:t}){const n=t.target?.value??t.value;e.set("value",n)},clearValue({context:e}){e.set("value","")},incrementToMax({context:e,prop:t,computed:n}){const r=sn(t("max"),{computed:n,prop:t});e.set("value",r)},decrementToMin({context:e,prop:t,computed:n}){const r=sn(t("min"),{computed:n,prop:t});e.set("value",r)},setHint({context:e,event:t}){e.set("hint",t.hint)},clearHint({context:e}){e.set("hint",null)},invokeOnFocus({computed:e,prop:t}){t("onFocusChange")?.({focused:!0,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnBlur({computed:e,prop:t}){t("onFocusChange")?.({focused:!1,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnInvalid({computed:e,prop:t,event:n}){if(n.type==="INPUT.CHANGE")return;const r=e("valueAsNumber")>t("max")?"rangeOverflow":"rangeUnderflow";t("onValueInvalid")?.({reason:r,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},syncInputElement({context:e,event:t,computed:n,scope:r}){const o=t.type.endsWith("CHANGE")?e.get("value"):n("formattedValue"),i=ni(r),s=rk(i,r);G(()=>{ko(i,o),ok(i,s,r)})},setFormattedValue({context:e,computed:t}){e.set("value",t("formattedValue"))},setCursorPoint({context:e,event:t}){e.set("scrubberCursorPoint",t.point)},clearCursorPoint({context:e}){e.set("scrubberCursorPoint",null)},setVirtualCursorPosition({context:e,scope:t}){const n=wd(t),r=e.get("scrubberCursorPoint");!n||!r||(n.style.transform=`translate3d(${r.x}px, ${r.y}px, 0px)`)}}}}),M()(["allowMouseWheel","allowOverflow","clampValueOnBlur","dir","disabled","focusInputOnChange","form","formatOptions","getRootNode","id","ids","inputMode","invalid","locale","max","min","name","onFocusChange","onValueChange","onValueInvalid","pattern","required","readOnly","spinOnPress","step","translations","value","defaultValue"]);var Pd=j("pinInput").parts("root","label","input","control");Pd.build(),M()(["autoFocus","blurOnComplete","count","defaultValue","dir","disabled","form","getRootNode","id","ids","invalid","mask","name","onValueChange","onValueComplete","onValueInvalid","otp","pattern","placeholder","readOnly","required","selectOnFocus","translations","type","value"]);var Id=j("popover").parts("arrow","arrowTip","anchor","trigger","indicator","positioner","content","title","description","closeTrigger");Id.build(),M()(["autoFocus","closeOnEscape","closeOnInteractOutside","dir","getRootNode","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","portalled","positioning"]);const an=e=>{const{children:t,disabled:n}=e,[r,o]=P.useState(e.container?.current),i=P.useSyncExternalStore(hk,()=>!1,()=>!0),{getRootNode:s}=Ql();if(P.useEffect(()=>{o(()=>e.container?.current)},[e.container]),i||n)return f.jsx(f.Fragment,{children:t});const a=r??dk(s);return f.jsx(f.Fragment,{children:P.Children.map(t,l=>Tt.createPortal(l,a))})},dk=e=>{const t=e?.(),n=t.getRootNode();return zn(n)?n:Fe(t).body},hk=()=>()=>{};var la=j("progress").parts("root","label","track","range","valueText","view","circle","circleTrack","circleRange");la.build(),M()(["dir","getRootNode","id","ids","max","min","orientation","translations","value","onValueChange","defaultValue","formatOptions","locale"]);var Rd=j("qr-code").parts("root","frame","pattern","overlay","downloadTrigger");Rd.build(),M()(["ids","defaultValue","value","id","encoding","dir","getRootNode","onValueChange","pixelSize"]);var ca=j("radio-group").parts("root","label","item","itemText","itemControl","indicator");ca.build(),M()(["dir","disabled","form","getRootNode","id","ids","name","onValueChange","orientation","readOnly","value","defaultValue"]),M()(["value","disabled","invalid"]);var Td=j("rating-group").parts("root","label","item","control");Td.build(),M()(["allowHalf","autoFocus","count","dir","disabled","form","getRootNode","id","ids","name","onHoverChange","onValueChange","required","readOnly","translations","value","defaultValue"]),M()(["index"]);var Nd=j("scroll-area").parts("root","viewport","content","scrollbar","thumb","corner");Nd.build(),M()(["dir","getRootNode","ids","id"]);const Ad=ca.rename("segment-group");Ad.build();var _d=j("select").parts("label","positioner","trigger","indicator","clearTrigger","item","itemText","itemIndicator","itemGroup","itemGroupLabel","list","content","root","control","valueText");_d.build();var Vd=e=>new Go(e);Vd.empty=()=>new Go({items:[]});var fk=e=>e.ids?.content??`select:${e.id}:content`,gk=e=>e.ids?.trigger??`select:${e.id}:trigger`,pk=e=>e.ids?.clearTrigger??`select:${e.id}:clear-trigger`,mk=(e,t)=>e.ids?.item?.(t)??`select:${e.id}:option:${t}`,vk=e=>e.ids?.hiddenSelect??`select:${e.id}:select`,bk=e=>e.ids?.positioner??`select:${e.id}:positioner`,ua=e=>e.getById(vk(e)),Lr=e=>e.getById(fk(e)),ri=e=>e.getById(gk(e)),yk=e=>e.getById(pk(e)),Fd=e=>e.getById(bk(e)),da=(e,t)=>t==null?null:e.getById(mk(e,t)),{and:Dr,not:ln,or:xk}=Ft();xk("isTriggerArrowDownEvent","isTriggerEnterEvent"),Dr(ln("multiple"),"hasSelectedItems"),ln("multiple"),Dr(ln("multiple"),"hasSelectedItems"),ln("multiple"),ln("multiple"),ln("multiple"),ln("multiple"),Dr("closeOnSelect","isOpenControlled"),Dr("hasHighlightedItem","loop","isLastItemHighlighted"),Dr("hasHighlightedItem","loop","isFirstItemHighlighted");function Ld(e){const t=e.restoreFocus??e.previousEvent?.restoreFocus;return t==null||!!t}M()(["closeOnSelect","collection","composite","defaultHighlightedValue","defaultOpen","defaultValue","deselectable","dir","disabled","form","getRootNode","highlightedValue","id","ids","invalid","loopFocus","multiple","name","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","positioning","readOnly","required","scrollToIndexFn","value"]),M()(["item","persistFocus"]),M()(["id"]),M()(["htmlFor"]);var Dd=j("slider").parts("root","label","thumb","valueText","track","range","control","markerGroup","marker","draggingIndicator");Dd.build(),M()(["aria-label","aria-labelledby","dir","disabled","form","getAriaValueText","getRootNode","id","ids","invalid","max","min","minStepsBetweenThumbs","name","onFocusChange","onValueChange","onValueChangeEnd","orientation","origin","readOnly","step","thumbAlignment","thumbAlignment","thumbSize","value","defaultValue"]),M()(["index","name"]);var zd=j("switch").parts("root","label","control","thumb");zd.build(),M()(["checked","defaultChecked","dir","disabled","form","getRootNode","id","ids","invalid","label","name","onCheckedChange","readOnly","required","value"]);var kk=j("toast").parts("group","root","title","description","actionTrigger","closeTrigger");kk.build();var Ck=(e,t)=>({...t,...po(e)});function Sk(e={}){const t=Ck(e,{placement:"bottom",overlap:!1,max:24,gap:16,offsets:"1rem",hotkey:["altKey","KeyT"],removeDelay:200,pauseOnPageIdle:!0});let n=[],r=[],o=new Set,i=[];const s=S=>(n.push(S),()=>{const N=n.indexOf(S);n.splice(N,1)}),a=S=>(n.forEach(N=>N(S)),S),l=S=>{if(r.length>=t.max){i.push(S);return}a(S),r.unshift(S)},c=()=>{for(;i.length>0&&r.length{const N=S.id??`toast:${Ym()}`,D=r.find(z=>z.id===N);return o.has(N)&&o.delete(N),D?r=r.map(z=>z.id===N?a({...z,...S,id:N}):z):l({id:N,duration:t.duration,removeDelay:t.removeDelay,type:"info",...S,stacked:!t.overlap,gap:t.gap}),N},d=S=>(o.add(S),S?(n.forEach(N=>N({id:S,dismiss:!0})),r=r.filter(N=>N.id!==S),c()):(r.forEach(N=>{n.forEach(D=>D({id:N.id,dismiss:!0}))}),r=[],i=[]),S);return{attrs:t,subscribe:s,create:u,update:(S,N)=>u({id:S,...N}),remove:d,dismiss:S=>{S!=null?r=r.map(N=>N.id===S?a({...N,message:"DISMISS"}):N):r=r.map(N=>a({...N,message:"DISMISS"}))},error:S=>u({...S,type:"error"}),success:S=>u({...S,type:"success"}),info:S=>u({...S,type:"info"}),warning:S=>u({...S,type:"warning"}),loading:S=>u({...S,type:"loading"}),getVisibleToasts:()=>r.filter(S=>!o.has(S.id)),getCount:()=>r.length,promise:(S,N,D={})=>{if(!N||!N.loading){kr("[zag-js > toast] toaster.promise() requires at least a 'loading' option to be specified");return}const z=u({...D,...N.loading,promise:S,type:"loading"});let W=!0,J;const ee=ho(S).then(async q=>{if(J=["resolve",q],wk(q)&&!q.ok){W=!1;const Z=ho(N.error,`HTTP Error! status: ${q.status}`);u({...D,...Z,id:z,type:"error"})}else if(N.success!==void 0){W=!1;const Z=ho(N.success,q);u({...D,...Z,id:z,type:"success"})}}).catch(async q=>{if(J=["reject",q],N.error!==void 0){W=!1;const Z=ho(N.error,q);u({...D,...Z,id:z,type:"error"})}}).finally(()=>{W&&d(z),N.finally?.()});return{id:z,unwrap:()=>new Promise((q,Z)=>ee.then(()=>J[0]==="reject"?Z(J[1]):q(J[1])).catch(Z))}},pause:S=>{S!=null?r=r.map(N=>N.id===S?a({...N,message:"PAUSE"}):N):r=r.map(N=>a({...N,message:"PAUSE"}))},resume:S=>{S!=null?r=r.map(N=>N.id===S?a({...N,message:"RESUME"}):N):r=r.map(N=>a({...N,message:"RESUME"}))},isVisible:S=>!o.has(S)&&!!r.find(N=>N.id===S),isDismissed:S=>o.has(S),expand:()=>{r=r.map(S=>a({...S,stacked:!0}))},collapse:()=>{r=r.map(S=>a({...S,stacked:!1}))}}}var wk=e=>e&&typeof e=="object"&&"ok"in e&&typeof e.ok=="boolean"&&"status"in e&&typeof e.status=="number";const Ek=e=>Sk(e);var Md=j("tooltip").parts("trigger","arrow","arrowTip","positioner","content");Md.build();var Ok=e=>e.ids?.trigger??`tooltip:${e.id}:trigger`,Pk=e=>e.ids?.positioner??`tooltip:${e.id}:popper`,ha=e=>e.getById(Ok(e)),jd=e=>e.getById(Pk(e)),cn=c0({id:null}),{and:Ik,not:$d}=Ft();Ik("noVisibleTooltip",$d("hasPointerMoveOpened")),$d("hasPointerMoveOpened"),M()(["aria-label","closeDelay","closeOnEscape","closeOnPointerDown","closeOnScroll","closeOnClick","dir","disabled","getRootNode","id","ids","interactive","onOpenChange","defaultOpen","open","openDelay","positioning"]);function Bd(e,t=[]){const n=Object.assign({},e);for(const r of t)r in n&&delete n[r];return n}const Rk=(e,t)=>{if(!e||typeof e!="string")return{invalid:!0,value:e};const[n,r]=e.split("/");if(!n||!r||n==="currentBg")return{invalid:!0,value:n};const o=t(`colors.${n}`),i=t.raw(`opacity.${r}`)?.value;if(!i&&isNaN(Number(r)))return{invalid:!0,value:n};const s=i?Number(i)*100+"%":`${r}%`,a=o??n;return{invalid:!1,color:a,value:`color-mix(in srgb, ${a} ${s}, transparent)`}},te=e=>(t,n)=>{const r=n.utils.colorMix(t);if(r.invalid)return{[e]:t};const o="--mix-"+e;return{[o]:r.value,[e]:`var(${o}, ${r.color})`}};function fa(e){if(e===null||typeof e!="object")return e;if(Array.isArray(e))return e.map(n=>fa(n));const t=Object.create(Object.getPrototypeOf(e));for(const n of Object.keys(e))t[n]=fa(e[n]);return t}function ga(e,t){if(t==null)return e;for(const n of Object.keys(t))if(!(t[n]===void 0||n==="__proto__"))if(!_e(e[n])&&_e(t[n]))Object.assign(e,{[n]:t[n]});else if(e[n]&&_e(t[n]))ga(e[n],t[n]);else if(Array.isArray(t[n])&&Array.isArray(e[n])){let r=0;for(;re!=null;function gt(e,t,n={}){const{stop:r,getKey:o}=n;function i(s,a=[]){if(_e(s)||Array.isArray(s)){const l={};for(const[c,u]of Object.entries(s)){const d=o?.(c,u)??c,h=[...a,d];if(r?.(s,h))return t(s,a);const m=i(u,h);pa(m)&&(l[d]=m)}return l}return t(s,a)}return i(e)}function Wd(e,t){return Array.isArray(e)?e.map(n=>pa(n)?t(n):n):_e(e)?gt(e,n=>t(n)):pa(e)?t(e):e}const oi=["value","type","description"],Tk=e=>e&&typeof e=="object"&&!Array.isArray(e),Hd=(...e)=>{const t=Jn({},...e.map(fa));return t.theme?.tokens&>(t.theme.tokens,n=>{const i=Object.keys(n).filter(a=>!oi.includes(a)).length>0,s=oi.some(a=>n[a]!=null);return i&&s&&(n.DEFAULT||(n.DEFAULT={}),oi.forEach(a=>{var l;n[a]!=null&&((l=n.DEFAULT)[a]||(l[a]=n[a]),delete n[a])})),n},{stop(n){return Tk(n)&&Object.keys(n).some(r=>oi.includes(r)||r!==r.toLowerCase()&&r!==r.toUpperCase())}}),t},Nk=e=>e,fe=e=>e,B=e=>e,Ak=e=>e,_k=e=>e,Zn=e=>e,Vk=e=>e,Fk=e=>e,Lk=e=>e;function Ud(){const e=t=>t;return new Proxy(e,{get(){return e}})}const de=Ud(),ma=Ud(),va=e=>e,Dk=/[^a-zA-Z0-9_\u0081-\uffff-]/g;function zk(e){return`${e}`.replace(Dk,t=>`\\${t}`)}const Mk=/[A-Z]/g;function jk(e){return e.replace(Mk,t=>`-${t.toLowerCase()}`)}function Gd(e,t={}){const{fallback:n="",prefix:r=""}=t,o=jk(["-",r,zk(e)].filter(Boolean).join("-"));return{var:o,ref:`var(${o}${n?`, ${n}`:""})`}}const $k=e=>/^var\(--.+\)$/.test(e),xe=(e,t)=>t!=null?`${e}(${t})`:t,un=e=>{if($k(e)||e==null)return e;const t=typeof e=="string"&&!e.endsWith("deg");return typeof e=="number"||t?`${e}deg`:e},qd=e=>({values:["outside","inside","mixed","none"],transform(t,{token:n}){const r=n("colors.colorPalette.focusRing");return{inside:{"--focus-ring-color":r,[e]:{outlineOffset:"0px",outlineWidth:"var(--focus-ring-width, 1px)",outlineColor:"var(--focus-ring-color)",outlineStyle:"var(--focus-ring-style, solid)",borderColor:"var(--focus-ring-color)"}},outside:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 2px)",outlineOffset:"var(--focus-ring-offset, 2px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"var(--focus-ring-color)"}},mixed:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 3px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"color-mix(in srgb, var(--focus-ring-color), transparent 60%)",borderColor:"var(--focus-ring-color)"}},none:{"--focus-ring-color":r,[e]:{outline:"none"}}}[t]??{}}}),Bk=te("borderColor"),Ct=e=>({transition:e,transitionTimingFunction:"cubic-bezier(0.4, 0, 0.2, 1)",transitionDuration:"150ms"}),Wk=Nk({hover:["@media (hover: hover)","&:is(:hover, [data-hover]):not(:disabled, [data-disabled])"],active:"&:is(:active, [data-active]):not(:disabled, [data-disabled], [data-state=open])",focus:"&:is(:focus, [data-focus])",focusWithin:"&:is(:focus-within, [data-focus-within])",focusVisible:"&:is(:focus-visible, [data-focus-visible])",disabled:"&:is(:disabled, [disabled], [data-disabled], [aria-disabled=true])",visited:"&:visited",target:"&:target",readOnly:"&:is([data-readonly], [aria-readonly=true], [readonly])",readWrite:"&:read-write",empty:"&:is(:empty, [data-empty])",checked:"&:is(:checked, [data-checked], [aria-checked=true], [data-state=checked])",enabled:"&:enabled",expanded:"&:is([aria-expanded=true], [data-expanded], [data-state=expanded])",highlighted:"&[data-highlighted]",complete:"&[data-complete]",incomplete:"&[data-incomplete]",dragging:"&[data-dragging]",before:"&::before",after:"&::after",firstLetter:"&::first-letter",firstLine:"&::first-line",marker:"&::marker",selection:"&::selection",file:"&::file-selector-button",backdrop:"&::backdrop",first:"&:first-of-type",last:"&:last-of-type",notFirst:"&:not(:first-of-type)",notLast:"&:not(:last-of-type)",only:"&:only-child",even:"&:nth-of-type(even)",odd:"&:nth-of-type(odd)",peerFocus:".peer:is(:focus, [data-focus]) ~ &",peerHover:".peer:is(:hover, [data-hover]):not(:disabled, [data-disabled]) ~ &",peerActive:".peer:is(:active, [data-active]):not(:disabled, [data-disabled]) ~ &",peerFocusWithin:".peer:focus-within ~ &",peerFocusVisible:".peer:is(:focus-visible, [data-focus-visible]) ~ &",peerDisabled:".peer:is(:disabled, [disabled], [data-disabled]) ~ &",peerChecked:".peer:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) ~ &",peerInvalid:".peer:is(:invalid, [data-invalid], [aria-invalid=true]) ~ &",peerExpanded:".peer:is([aria-expanded=true], [data-expanded], [data-state=expanded]) ~ &",peerPlaceholderShown:".peer:placeholder-shown ~ &",groupFocus:".group:is(:focus, [data-focus]) &",groupHover:".group:is(:hover, [data-hover]):not(:disabled, [data-disabled]) &",groupActive:".group:is(:active, [data-active]):not(:disabled, [data-disabled]) &",groupFocusWithin:".group:focus-within &",groupFocusVisible:".group:is(:focus-visible, [data-focus-visible]) &",groupDisabled:".group:is(:disabled, [disabled], [data-disabled]) &",groupChecked:".group:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) &",groupExpanded:".group:is([aria-expanded=true], [data-expanded], [data-state=expanded]) &",groupInvalid:".group:invalid &",indeterminate:"&:is(:indeterminate, [data-indeterminate], [aria-checked=mixed], [data-state=indeterminate])",required:"&:is([data-required], [aria-required=true])",valid:"&:is([data-valid], [data-state=valid])",invalid:"&:is([data-invalid], [aria-invalid=true], [data-state=invalid])",autofill:"&:autofill",inRange:"&:is(:in-range, [data-in-range])",outOfRange:"&:is(:out-of-range, [data-outside-range])",placeholder:"&::placeholder, &[data-placeholder]",placeholderShown:"&:is(:placeholder-shown, [data-placeholder-shown])",pressed:"&:is([aria-pressed=true], [data-pressed])",selected:"&:is([aria-selected=true], [data-selected])",grabbed:"&:is([aria-grabbed=true], [data-grabbed])",underValue:"&[data-state=under-value]",overValue:"&[data-state=over-value]",atValue:"&[data-state=at-value]",default:"&:default",optional:"&:optional",open:"&:is([open], [data-open], [data-state=open])",closed:"&:is([closed], [data-closed], [data-state=closed])",fullscreen:"&:is(:fullscreen, [data-fullscreen])",loading:"&:is([data-loading], [aria-busy=true])",hidden:"&:is([hidden], [data-hidden])",current:"&[data-current]",currentPage:"&[aria-current=page]",currentStep:"&[aria-current=step]",today:"&[data-today]",unavailable:"&[data-unavailable]",rangeStart:"&[data-range-start]",rangeEnd:"&[data-range-end]",now:"&[data-now]",topmost:"&[data-topmost]",motionReduce:"@media (prefers-reduced-motion: reduce)",motionSafe:"@media (prefers-reduced-motion: no-preference)",print:"@media print",landscape:"@media (orientation: landscape)",portrait:"@media (orientation: portrait)",dark:".dark &, .dark .chakra-theme:not(.light) &",light:":root &, .light &",osDark:"@media (prefers-color-scheme: dark)",osLight:"@media (prefers-color-scheme: light)",highContrast:"@media (forced-colors: active)",lessContrast:"@media (prefers-contrast: less)",moreContrast:"@media (prefers-contrast: more)",ltr:"[dir=ltr] &",rtl:"[dir=rtl] &",scrollbar:"&::-webkit-scrollbar",scrollbarThumb:"&::-webkit-scrollbar-thumb",scrollbarTrack:"&::-webkit-scrollbar-track",horizontal:"&[data-orientation=horizontal]",vertical:"&[data-orientation=vertical]",icon:"& :where(svg)",starting:"@starting-style"}),er=Gd("bg-currentcolor"),Kd=e=>e===er.ref||e==="currentBg",ne=e=>({...e("colors"),currentBg:er}),Hk=va({conditions:Wk,utilities:{background:{values:ne,shorthand:["bg"],transform(e,t){if(Kd(t.raw))return{background:er.ref};const n=te("background")(e,t);return{...n,[er.var]:n?.background}}},backgroundColor:{values:ne,shorthand:["bgColor"],transform(e,t){if(Kd(t.raw))return{backgroundColor:er.ref};const n=te("backgroundColor")(e,t);return{...n,[er.var]:n?.backgroundColor}}},backgroundSize:{shorthand:["bgSize"]},backgroundPosition:{shorthand:["bgPos"]},backgroundRepeat:{shorthand:["bgRepeat"]},backgroundAttachment:{shorthand:["bgAttachment"]},backgroundClip:{shorthand:["bgClip"],values:["text"],transform(e){return e==="text"?{color:"transparent",backgroundClip:"text"}:{backgroundClip:e}}},backgroundGradient:{shorthand:["bgGradient"],values(e){return{...e("gradients"),"to-t":"linear-gradient(to top, var(--gradient))","to-tr":"linear-gradient(to top right, var(--gradient))","to-r":"linear-gradient(to right, var(--gradient))","to-br":"linear-gradient(to bottom right, var(--gradient))","to-b":"linear-gradient(to bottom, var(--gradient))","to-bl":"linear-gradient(to bottom left, var(--gradient))","to-l":"linear-gradient(to left, var(--gradient))","to-tl":"linear-gradient(to top left, var(--gradient))"}},transform(e){return{"--gradient-stops":"var(--gradient-from), var(--gradient-to)","--gradient":"var(--gradient-via-stops, var(--gradient-stops))",backgroundImage:e}}},gradientFrom:{values:ne,transform:te("--gradient-from")},gradientTo:{values:ne,transform:te("--gradient-to")},gradientVia:{values:ne,transform(e,t){return{...te("--gradient-via")(e,t),"--gradient-via-stops":"var(--gradient-from), var(--gradient-via), var(--gradient-to)"}}},backgroundImage:{values(e){return{...e("gradients"),...e("assets")}},shorthand:["bgImg","bgImage"]},border:{values:"borders"},borderTop:{values:"borders"},borderLeft:{values:"borders"},borderBlockStart:{values:"borders"},borderRight:{values:"borders"},borderBottom:{values:"borders"},borderBlockEnd:{values:"borders"},borderInlineStart:{values:"borders",shorthand:["borderStart"]},borderInlineEnd:{values:"borders",shorthand:["borderEnd"]},borderInline:{values:"borders",shorthand:["borderX"]},borderBlock:{values:"borders",shorthand:["borderY"]},borderColor:{values:ne,transform:te("borderColor")},borderTopColor:{values:ne,transform:te("borderTopColor")},borderBlockStartColor:{values:ne,transform:te("borderBlockStartColor")},borderBottomColor:{values:ne,transform:te("borderBottomColor")},borderBlockEndColor:{values:ne,transform:te("borderBlockEndColor")},borderLeftColor:{values:ne,transform:te("borderLeftColor")},borderInlineStartColor:{values:ne,shorthand:["borderStartColor"],transform:te("borderInlineStartColor")},borderRightColor:{values:ne,transform:te("borderRightColor")},borderInlineEndColor:{values:ne,shorthand:["borderEndColor"],transform:te("borderInlineEndColor")},borderStyle:{values:"borderStyles"},borderTopStyle:{values:"borderStyles"},borderBlockStartStyle:{values:"borderStyles"},borderBottomStyle:{values:"borderStyles"},borderBlockEndStyle:{values:"borderStyles"},borderInlineStartStyle:{values:"borderStyles",shorthand:["borderStartStyle"]},borderInlineEndStyle:{values:"borderStyles",shorthand:["borderEndStyle"]},borderLeftStyle:{values:"borderStyles"},borderRightStyle:{values:"borderStyles"},borderRadius:{values:"radii",shorthand:["rounded"]},borderTopLeftRadius:{values:"radii",shorthand:["roundedTopLeft"]},borderStartStartRadius:{values:"radii",shorthand:["roundedStartStart","borderTopStartRadius"]},borderEndStartRadius:{values:"radii",shorthand:["roundedEndStart","borderBottomStartRadius"]},borderTopRightRadius:{values:"radii",shorthand:["roundedTopRight"]},borderStartEndRadius:{values:"radii",shorthand:["roundedStartEnd","borderTopEndRadius"]},borderEndEndRadius:{values:"radii",shorthand:["roundedEndEnd","borderBottomEndRadius"]},borderBottomLeftRadius:{values:"radii",shorthand:["roundedBottomLeft"]},borderBottomRightRadius:{values:"radii",shorthand:["roundedBottomRight"]},borderInlineStartRadius:{values:"radii",property:"borderRadius",shorthand:["roundedStart","borderStartRadius"],transform:e=>({borderStartStartRadius:e,borderEndStartRadius:e})},borderInlineEndRadius:{values:"radii",property:"borderRadius",shorthand:["roundedEnd","borderEndRadius"],transform:e=>({borderStartEndRadius:e,borderEndEndRadius:e})},borderTopRadius:{values:"radii",property:"borderRadius",shorthand:["roundedTop"],transform:e=>({borderTopLeftRadius:e,borderTopRightRadius:e})},borderBottomRadius:{values:"radii",property:"borderRadius",shorthand:["roundedBottom"],transform:e=>({borderBottomLeftRadius:e,borderBottomRightRadius:e})},borderLeftRadius:{values:"radii",property:"borderRadius",shorthand:["roundedLeft"],transform:e=>({borderTopLeftRadius:e,borderBottomLeftRadius:e})},borderRightRadius:{values:"radii",property:"borderRadius",shorthand:["roundedRight"],transform:e=>({borderTopRightRadius:e,borderBottomRightRadius:e})},borderWidth:{values:"borderWidths"},borderBlockStartWidth:{values:"borderWidths"},borderTopWidth:{values:"borderWidths"},borderBottomWidth:{values:"borderWidths"},borderBlockEndWidth:{values:"borderWidths"},borderRightWidth:{values:"borderWidths"},borderInlineWidth:{values:"borderWidths",shorthand:["borderXWidth"]},borderInlineStartWidth:{values:"borderWidths",shorthand:["borderStartWidth"]},borderInlineEndWidth:{values:"borderWidths",shorthand:["borderEndWidth"]},borderLeftWidth:{values:"borderWidths"},borderBlockWidth:{values:"borderWidths",shorthand:["borderYWidth"]},color:{values:ne,transform:te("color")},fill:{values:ne,transform:te("fill")},stroke:{values:ne,transform:te("stroke")},accentColor:{values:ne,transform:te("accentColor")},divideX:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderInlineStartWidth:e,borderInlineEndWidth:"0px"}}}},divideY:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderTopWidth:e,borderBottomWidth:"0px"}}}},divideColor:{values:ne,transform(e,t){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":Bk(e,t)}}},divideStyle:{property:"borderStyle",transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderStyle:e}}}},boxShadow:{values:"shadows",shorthand:["shadow"]},boxShadowColor:{values:ne,transform:te("--shadow-color"),shorthand:["shadowColor"]},mixBlendMode:{shorthand:["blendMode"]},backgroundBlendMode:{shorthand:["bgBlendMode"]},opacity:{values:"opacity"},filter:{transform(e){return e!=="auto"?{filter:e}:{filter:"var(--blur) var(--brightness) var(--contrast) var(--grayscale) var(--hue-rotate) var(--invert) var(--saturate) var(--sepia) var(--drop-shadow)"}}},blur:{values:"blurs",transform:e=>({"--blur":xe("blur",e)})},brightness:{transform:e=>({"--brightness":xe("brightness",e)})},contrast:{transform:e=>({"--contrast":xe("contrast",e)})},grayscale:{transform:e=>({"--grayscale":xe("grayscale",e)})},hueRotate:{transform:e=>({"--hue-rotate":xe("hue-rotate",un(e))})},invert:{transform:e=>({"--invert":xe("invert",e)})},saturate:{transform:e=>({"--saturate":xe("saturate",e)})},sepia:{transform:e=>({"--sepia":xe("sepia",e)})},dropShadow:{transform:e=>({"--drop-shadow":xe("drop-shadow",e)})},backdropFilter:{transform(e){return e!=="auto"?{backdropFilter:e}:{backdropFilter:"var(--backdrop-blur) var(--backdrop-brightness) var(--backdrop-contrast) var(--backdrop-grayscale) var(--backdrop-hue-rotate) var(--backdrop-invert) var(--backdrop-opacity) var(--backdrop-saturate) var(--backdrop-sepia)"}}},backdropBlur:{values:"blurs",transform:e=>({"--backdrop-blur":xe("blur",e)})},backdropBrightness:{transform:e=>({"--backdrop-brightness":xe("brightness",e)})},backdropContrast:{transform:e=>({"--backdrop-contrast":xe("contrast",e)})},backdropGrayscale:{transform:e=>({"--backdrop-grayscale":xe("grayscale",e)})},backdropHueRotate:{transform:e=>({"--backdrop-hue-rotate":xe("hue-rotate",un(e))})},backdropInvert:{transform:e=>({"--backdrop-invert":xe("invert",e)})},backdropOpacity:{transform:e=>({"--backdrop-opacity":xe("opacity",e)})},backdropSaturate:{transform:e=>({"--backdrop-saturate":xe("saturate",e)})},backdropSepia:{transform:e=>({"--backdrop-sepia":xe("sepia",e)})},flexBasis:{values:"sizes"},gap:{values:"spacing"},rowGap:{values:"spacing",shorthand:["gapY"]},columnGap:{values:"spacing",shorthand:["gapX"]},flexDirection:{shorthand:["flexDir"]},gridGap:{values:"spacing"},gridColumnGap:{values:"spacing"},gridRowGap:{values:"spacing"},outlineColor:{values:ne,transform:te("outlineColor")},focusRing:qd("&:is(:focus, [data-focus])"),focusVisibleRing:qd("&:is(:focus-visible, [data-focus-visible])"),focusRingColor:{values:ne,transform:te("--focus-ring-color")},focusRingOffset:{values:"spacing",transform:e=>({"--focus-ring-offset":e})},focusRingWidth:{values:"borderWidths",property:"outlineWidth",transform:e=>({"--focus-ring-width":e})},focusRingStyle:{values:"borderStyles",property:"outlineStyle",transform:e=>({"--focus-ring-style":e})},aspectRatio:{values:"aspectRatios"},width:{values:"sizes",shorthand:["w"]},inlineSize:{values:"sizes"},height:{values:"sizes",shorthand:["h"]},blockSize:{values:"sizes"},boxSize:{values:"sizes",property:"width",transform:e=>({width:e,height:e})},minWidth:{values:"sizes",shorthand:["minW"]},minInlineSize:{values:"sizes"},minHeight:{values:"sizes",shorthand:["minH"]},minBlockSize:{values:"sizes"},maxWidth:{values:"sizes",shorthand:["maxW"]},maxInlineSize:{values:"sizes"},maxHeight:{values:"sizes",shorthand:["maxH"]},maxBlockSize:{values:"sizes"},hideFrom:{values:"breakpoints",transform:(e,{raw:t,token:n})=>({[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}`:`@media screen and (min-width: ${e})`]:{display:"none"}})},hideBelow:{values:"breakpoints",transform(e,{raw:t,token:n}){return{[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}Down`:`@media screen and (max-width: ${e})`]:{display:"none"}}}},overscrollBehavior:{shorthand:["overscroll"]},overscrollBehaviorX:{shorthand:["overscrollX"]},overscrollBehaviorY:{shorthand:["overscrollY"]},scrollbar:{values:["visible","hidden"],transform(e){switch(e){case"visible":return{msOverflowStyle:"auto",scrollbarWidth:"auto","&::-webkit-scrollbar":{display:"block"}};case"hidden":return{msOverflowStyle:"none",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}};default:return{}}}},scrollbarColor:{values:ne,transform:te("scrollbarColor")},scrollbarGutter:{values:"spacing"},scrollbarWidth:{values:"sizes"},scrollMargin:{values:"spacing"},scrollMarginTop:{values:"spacing"},scrollMarginBottom:{values:"spacing"},scrollMarginLeft:{values:"spacing"},scrollMarginRight:{values:"spacing"},scrollMarginX:{values:"spacing",transform:e=>({scrollMarginLeft:e,scrollMarginRight:e})},scrollMarginY:{values:"spacing",transform:e=>({scrollMarginTop:e,scrollMarginBottom:e})},scrollPadding:{values:"spacing"},scrollPaddingTop:{values:"spacing"},scrollPaddingBottom:{values:"spacing"},scrollPaddingLeft:{values:"spacing"},scrollPaddingRight:{values:"spacing"},scrollPaddingInline:{values:"spacing",shorthand:["scrollPaddingX"]},scrollPaddingBlock:{values:"spacing",shorthand:["scrollPaddingY"]},scrollSnapType:{values:{none:"none",x:"x var(--scroll-snap-strictness)",y:"y var(--scroll-snap-strictness)",both:"both var(--scroll-snap-strictness)"}},scrollSnapStrictness:{values:["mandatory","proximity"],transform:e=>({"--scroll-snap-strictness":e})},scrollSnapMargin:{values:"spacing"},scrollSnapMarginTop:{values:"spacing"},scrollSnapMarginBottom:{values:"spacing"},scrollSnapMarginLeft:{values:"spacing"},scrollSnapMarginRight:{values:"spacing"},listStylePosition:{shorthand:["listStylePos"]},listStyleImage:{values:"assets",shorthand:["listStyleImg"]},position:{shorthand:["pos"]},zIndex:{values:"zIndex"},inset:{values:"spacing"},insetInline:{values:"spacing",shorthand:["insetX"]},insetBlock:{values:"spacing",shorthand:["insetY"]},top:{values:"spacing"},insetBlockStart:{values:"spacing"},bottom:{values:"spacing"},insetBlockEnd:{values:"spacing"},left:{values:"spacing"},right:{values:"spacing"},insetInlineStart:{values:"spacing",shorthand:["insetStart"]},insetInlineEnd:{values:"spacing",shorthand:["insetEnd"]},ring:{transform(e){return{"--ring-offset-shadow":"var(--ring-inset) 0 0 0 var(--ring-offset-width) var(--ring-offset-color)","--ring-shadow":"var(--ring-inset) 0 0 0 calc(var(--ring-width) + var(--ring-offset-width)) var(--ring-color)","--ring-width":e,boxShadow:"var(--ring-offset-shadow), var(--ring-shadow), var(--shadow, 0 0 #0000)"}}},ringColor:{values:ne,transform:te("--ring-color")},ringOffset:{transform:e=>({"--ring-offset-width":e})},ringOffsetColor:{values:ne,transform:te("--ring-offset-color")},ringInset:{transform:e=>({"--ring-inset":e})},margin:{values:"spacing",shorthand:["m"]},marginTop:{values:"spacing",shorthand:["mt"]},marginBlockStart:{values:"spacing"},marginRight:{values:"spacing",shorthand:["mr"]},marginBottom:{values:"spacing",shorthand:["mb"]},marginBlockEnd:{values:"spacing"},marginLeft:{values:"spacing",shorthand:["ml"]},marginInlineStart:{values:"spacing",shorthand:["ms","marginStart"]},marginInlineEnd:{values:"spacing",shorthand:["me","marginEnd"]},marginInline:{values:"spacing",shorthand:["mx","marginX"]},marginBlock:{values:"spacing",shorthand:["my","marginY"]},padding:{values:"spacing",shorthand:["p"]},paddingTop:{values:"spacing",shorthand:["pt"]},paddingRight:{values:"spacing",shorthand:["pr"]},paddingBottom:{values:"spacing",shorthand:["pb"]},paddingBlockStart:{values:"spacing"},paddingBlockEnd:{values:"spacing"},paddingLeft:{values:"spacing",shorthand:["pl"]},paddingInlineStart:{values:"spacing",shorthand:["ps","paddingStart"]},paddingInlineEnd:{values:"spacing",shorthand:["pe","paddingEnd"]},paddingInline:{values:"spacing",shorthand:["px","paddingX"]},paddingBlock:{values:"spacing",shorthand:["py","paddingY"]},textDecoration:{shorthand:["textDecor"]},textDecorationColor:{values:ne,transform:te("textDecorationColor")},textShadow:{values:"shadows"},transform:{transform:e=>{let t=e;return e==="auto"&&(t="translateX(var(--translate-x, 0)) translateY(var(--translate-y, 0)) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),e==="auto-gpu"&&(t="translate3d(var(--translate-x, 0), var(--translate-y, 0), 0) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),{transform:t}}},skewX:{transform:e=>({"--skew-x":un(e)})},skewY:{transform:e=>({"--skew-y":un(e)})},scaleX:{transform:e=>({"--scale-x":e})},scaleY:{transform:e=>({"--scale-y":e})},scale:{transform(e){return e!=="auto"?{scale:e}:{scale:"var(--scale-x, 1) var(--scale-y, 1)"}}},spaceXReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":e?"1":void 0}}}},spaceX:{property:"marginInlineStart",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":"0",marginInlineStart:`calc(${e} * calc(1 - var(--space-x-reverse)))`,marginInlineEnd:`calc(${e} * var(--space-x-reverse))`}})},spaceYReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":e?"1":void 0}}}},spaceY:{property:"marginTop",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":"0",marginTop:`calc(${e} * calc(1 - var(--space-y-reverse)))`,marginBottom:`calc(${e} * var(--space-y-reverse))`}})},rotate:{transform(e){return e!=="auto"?{rotate:un(e)}:{rotate:"var(--rotate-x, 0) var(--rotate-y, 0) var(--rotate-z, 0)"}}},rotateX:{transform(e){return{"--rotate-x":un(e)}}},rotateY:{transform(e){return{"--rotate-y":un(e)}}},translate:{transform(e){return e!=="auto"?{translate:e}:{translate:"var(--translate-x) var(--translate-y)"}}},translateX:{values:"spacing",transform:e=>({"--translate-x":e})},translateY:{values:"spacing",transform:e=>({"--translate-y":e})},transition:{values:["all","common","colors","opacity","position","backgrounds","size","shadow","transform"],transform(e){switch(e){case"all":return Ct("all");case"position":return Ct("left, right, top, bottom, inset-inline, inset-block");case"colors":return Ct("color, background-color, border-color, text-decoration-color, fill, stroke");case"opacity":return Ct("opacity");case"shadow":return Ct("box-shadow");case"transform":return Ct("transform");case"size":return Ct("width, height");case"backgrounds":return Ct("background, background-color, background-image, background-position");case"common":return Ct("color, background-color, border-color, text-decoration-color, fill, stroke, opacity, box-shadow, transform, filter, backdrop-filter");default:return{transition:e}}}},transitionDuration:{values:"durations"},transitionProperty:{values:{common:"background-color, border-color, color, fill, stroke, opacity, box-shadow, translate, transform",colors:"background-color, border-color, color, fill, stroke",size:"width, height",position:"left, right, top, bottom, inset-inline, inset-block",background:"background, background-color, background-image, background-position"}},transitionTimingFunction:{values:"easings"},animation:{values:"animations"},animationDuration:{values:"durations"},animationDelay:{values:"durations"},animationTimingFunction:{values:"easings"},fontFamily:{values:"fonts"},fontSize:{values:"fontSizes"},fontWeight:{values:"fontWeights"},lineHeight:{values:"lineHeights"},letterSpacing:{values:"letterSpacings"},textIndent:{values:"spacing"},truncate:{values:{type:"boolean"},transform(e){return e===!0?{overflow:"hidden",textOverflow:"ellipsis",whiteSpace:"nowrap"}:{}}},lineClamp:{transform(e){return e==="none"?{WebkitLineClamp:"unset"}:{overflow:"hidden",display:"-webkit-box",WebkitLineClamp:e,WebkitBoxOrient:"vertical",textWrap:"wrap"}}},borderSpacing:{values:e=>({...e("spacing"),auto:"var(--border-spacing-x, 0) var(--border-spacing-y, 0)"})},borderSpacingX:{values:"spacing",transform(e){return{"--border-spacing-x":e}}},borderSpacingY:{values:"spacing",transform(e){return{"--border-spacing-y":e}}},srOnly:{values:{type:"boolean"},transform(e){return Uk[e]||{}}},debug:{values:{type:"boolean"},transform(e){return e?{outline:"1px solid blue !important","& > *":{outline:"1px solid red !important"}}:{}}},caretColor:{values:ne,transform:te("caretColor")},cursor:{values:"cursor"}}}),Uk={true:{position:"absolute",width:"1px",height:"1px",padding:"0",margin:"-1px",overflow:"hidden",clip:"rect(0, 0, 0, 0)",whiteSpace:"nowrap",borderWidth:"0"},false:{position:"static",width:"auto",height:"auto",padding:"0",margin:"0",overflow:"visible",clip:"auto",whiteSpace:"normal"}};var Gk="",qk=Gk.split(","),Kk="WebkitAppearance,WebkitBorderBefore,WebkitBorderBeforeColor,WebkitBorderBeforeStyle,WebkitBorderBeforeWidth,WebkitBoxReflect,WebkitLineClamp,WebkitMask,WebkitMaskAttachment,WebkitMaskClip,WebkitMaskComposite,WebkitMaskImage,WebkitMaskOrigin,WebkitMaskPosition,WebkitMaskPositionX,WebkitMaskPositionY,WebkitMaskRepeat,WebkitMaskRepeatX,WebkitMaskRepeatY,WebkitMaskSize,WebkitOverflowScrolling,WebkitTapHighlightColor,WebkitTextFillColor,WebkitTextStroke,WebkitTextStrokeColor,WebkitTextStrokeWidth,WebkitTouchCallout,WebkitUserModify,WebkitUserSelect,accentColor,alignContent,alignItems,alignSelf,alignTracks,all,anchorName,anchorScope,animation,animationComposition,animationDelay,animationDirection,animationDuration,animationFillMode,animationIterationCount,animationName,animationPlayState,animationRange,animationRangeEnd,animationRangeStart,animationTimeline,animationTimingFunction,appearance,aspectRatio,backdropFilter,backfaceVisibility,background,backgroundAttachment,backgroundBlendMode,backgroundClip,backgroundColor,backgroundImage,backgroundOrigin,backgroundPosition,backgroundPositionX,backgroundPositionY,backgroundRepeat,backgroundSize,blockSize,border,borderBlock,borderBlockColor,borderBlockEnd,borderBlockEndColor,borderBlockEndStyle,borderBlockEndWidth,borderBlockStart,borderBlockStartColor,borderBlockStartStyle,borderBlockStartWidth,borderBlockStyle,borderBlockWidth,borderBottom,borderBottomColor,borderBottomLeftRadius,borderBottomRightRadius,borderBottomStyle,borderBottomWidth,borderCollapse,borderColor,borderEndEndRadius,borderEndStartRadius,borderImage,borderImageOutset,borderImageRepeat,borderImageSlice,borderImageSource,borderImageWidth,borderInline,borderInlineColor,borderInlineEnd,borderInlineEndColor,borderInlineEndStyle,borderInlineEndWidth,borderInlineStart,borderInlineStartColor,borderInlineStartStyle,borderInlineStartWidth,borderInlineStyle,borderInlineWidth,borderLeft,borderLeftColor,borderLeftStyle,borderLeftWidth,borderRadius,borderRight,borderRightColor,borderRightStyle,borderRightWidth,borderSpacing,borderStartEndRadius,borderStartStartRadius,borderStyle,borderTop,borderTopColor,borderTopLeftRadius,borderTopRightRadius,borderTopStyle,borderTopWidth,borderWidth,bottom,boxAlign,boxDecorationBreak,boxDirection,boxFlex,boxFlexGroup,boxLines,boxOrdinalGroup,boxOrient,boxPack,boxShadow,boxSizing,breakAfter,breakBefore,breakInside,captionSide,caret,caretColor,caretShape,clear,clip,clipPath,clipRule,color,colorInterpolationFilters,colorScheme,columnCount,columnFill,columnGap,columnRule,columnRuleColor,columnRuleStyle,columnRuleWidth,columnSpan,columnWidth,columns,contain,containIntrinsicBlockSize,containIntrinsicHeight,containIntrinsicInlineSize,containIntrinsicSize,containIntrinsicWidth,container,containerName,containerType,content,contentVisibility,counterIncrement,counterReset,counterSet,cursor,cx,cy,d,direction,display,dominantBaseline,emptyCells,fieldSizing,fill,fillOpacity,fillRule,filter,flex,flexBasis,flexDirection,flexFlow,flexGrow,flexShrink,flexWrap,float,floodColor,floodOpacity,font,fontFamily,fontFeatureSettings,fontKerning,fontLanguageOverride,fontOpticalSizing,fontPalette,fontSize,fontSizeAdjust,fontSmooth,fontStretch,fontStyle,fontSynthesis,fontSynthesisPosition,fontSynthesisSmallCaps,fontSynthesisStyle,fontSynthesisWeight,fontVariant,fontVariantAlternates,fontVariantCaps,fontVariantEastAsian,fontVariantEmoji,fontVariantLigatures,fontVariantNumeric,fontVariantPosition,fontVariationSettings,fontWeight,forcedColorAdjust,gap,grid,gridArea,gridAutoColumns,gridAutoFlow,gridAutoRows,gridColumn,gridColumnEnd,gridColumnGap,gridColumnStart,gridGap,gridRow,gridRowEnd,gridRowGap,gridRowStart,gridTemplate,gridTemplateAreas,gridTemplateColumns,gridTemplateRows,hangingPunctuation,height,hyphenateCharacter,hyphenateLimitChars,hyphens,imageOrientation,imageRendering,imageResolution,imeMode,initialLetter,initialLetterAlign,inlineSize,inset,insetBlock,insetBlockEnd,insetBlockStart,insetInline,insetInlineEnd,insetInlineStart,interpolateSize,isolation,justifyContent,justifyItems,justifySelf,justifyTracks,left,letterSpacing,lightingColor,lineBreak,lineClamp,lineHeight,lineHeightStep,listStyle,listStyleImage,listStylePosition,listStyleType,margin,marginBlock,marginBlockEnd,marginBlockStart,marginBottom,marginInline,marginInlineEnd,marginInlineStart,marginLeft,marginRight,marginTop,marginTrim,marker,markerEnd,markerMid,markerStart,mask,maskBorder,maskBorderMode,maskBorderOutset,maskBorderRepeat,maskBorderSlice,maskBorderSource,maskBorderWidth,maskClip,maskComposite,maskImage,maskMode,maskOrigin,maskPosition,maskRepeat,maskSize,maskType,masonryAutoFlow,mathDepth,mathShift,mathStyle,maxBlockSize,maxHeight,maxInlineSize,maxLines,maxWidth,minBlockSize,minHeight,minInlineSize,minWidth,mixBlendMode,objectFit,objectPosition,offset,offsetAnchor,offsetDistance,offsetPath,offsetPosition,offsetRotate,opacity,order,orphans,outline,outlineColor,outlineOffset,outlineStyle,outlineWidth,overflow,overflowAnchor,overflowBlock,overflowClipBox,overflowClipMargin,overflowInline,overflowWrap,overflowX,overflowY,overlay,overscrollBehavior,overscrollBehaviorBlock,overscrollBehaviorInline,overscrollBehaviorX,overscrollBehaviorY,padding,paddingBlock,paddingBlockEnd,paddingBlockStart,paddingBottom,paddingInline,paddingInlineEnd,paddingInlineStart,paddingLeft,paddingRight,paddingTop,page,pageBreakAfter,pageBreakBefore,pageBreakInside,paintOrder,perspective,perspectiveOrigin,placeContent,placeItems,placeSelf,pointerEvents,position,positionAnchor,positionArea,positionTry,positionTryFallbacks,positionTryOrder,positionVisibility,printColorAdjust,quotes,r,resize,right,rotate,rowGap,rubyAlign,rubyMerge,rubyPosition,rx,ry,scale,scrollBehavior,scrollMargin,scrollMarginBlock,scrollMarginBlockEnd,scrollMarginBlockStart,scrollMarginBottom,scrollMarginInline,scrollMarginInlineEnd,scrollMarginInlineStart,scrollMarginLeft,scrollMarginRight,scrollMarginTop,scrollPadding,scrollPaddingBlock,scrollPaddingBlockEnd,scrollPaddingBlockStart,scrollPaddingBottom,scrollPaddingInline,scrollPaddingInlineEnd,scrollPaddingInlineStart,scrollPaddingLeft,scrollPaddingRight,scrollPaddingTop,scrollSnapAlign,scrollSnapCoordinate,scrollSnapDestination,scrollSnapPointsX,scrollSnapPointsY,scrollSnapStop,scrollSnapType,scrollSnapTypeX,scrollSnapTypeY,scrollTimeline,scrollTimelineAxis,scrollTimelineName,scrollbarColor,scrollbarGutter,scrollbarWidth,shapeImageThreshold,shapeMargin,shapeOutside,shapeRendering,stopColor,stopOpacity,stroke,strokeDasharray,strokeDashoffset,strokeLinecap,strokeLinejoin,strokeMiterlimit,strokeOpacity,strokeWidth,tabSize,tableLayout,textAlign,textAlignLast,textAnchor,textBox,textBoxEdge,textBoxTrim,textCombineUpright,textDecoration,textDecorationColor,textDecorationLine,textDecorationSkip,textDecorationSkipInk,textDecorationStyle,textDecorationThickness,textEmphasis,textEmphasisColor,textEmphasisPosition,textEmphasisStyle,textIndent,textJustify,textOrientation,textOverflow,textRendering,textShadow,textSizeAdjust,textSpacingTrim,textTransform,textUnderlineOffset,textUnderlinePosition,textWrap,textWrapMode,textWrapStyle,timelineScope,top,touchAction,transform,transformBox,transformOrigin,transformStyle,transition,transitionBehavior,transitionDelay,transitionDuration,transitionProperty,transitionTimingFunction,translate,unicodeBidi,userSelect,vectorEffect,verticalAlign,viewTimeline,viewTimelineAxis,viewTimelineInset,viewTimelineName,viewTransitionName,visibility,whiteSpace,whiteSpaceCollapse,widows,width,willChange,wordBreak,wordSpacing,wordWrap,writingMode,x,y,zIndex,zoom,alignmentBaseline,baselineShift,colorInterpolation,colorRendering,glyphOrientationVertical",Yk=Kk.split(",").concat(qk),Xk=new Map(Yk.map(e=>[e,!0]));function Qk(e){const t=Object.create(null);return n=>(t[n]===void 0&&(t[n]=e(n)),t[n])}var Jk=/&|@/,Zk=Qk(e=>Xk.has(e)||e.startsWith("--")||Jk.test(e));function Yd(e,t){const n={};return gt(e,(r,o)=>{r&&(n[o.join(".")]=r.value)},{stop:t}),n}var ba,Xd;function eC(){if(Xd)return ba;Xd=1,ba=i,i.default=i,i.stable=c,i.stableStringify=c;var e="[...]",t="[Circular]",n=[],r=[];function o(){return{depthLimit:Number.MAX_SAFE_INTEGER,edgesLimit:Number.MAX_SAFE_INTEGER}}function i(h,m,g,p){typeof p>"u"&&(p=o()),a(h,"",0,[],void 0,0,p);var v;try{r.length===0?v=JSON.stringify(h,m,g):v=JSON.stringify(h,d(m),g)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var y=n.pop();y.length===4?Object.defineProperty(y[0],y[1],y[3]):y[0][y[1]]=y[2]}}return v}function s(h,m,g,p){var v=Object.getOwnPropertyDescriptor(p,g);v.get!==void 0?v.configurable?(Object.defineProperty(p,g,{value:h}),n.push([p,g,m,v])):r.push([m,g,h]):(p[g]=h,n.push([p,g,m]))}function a(h,m,g,p,v,y,x){y+=1;var b;if(typeof h=="object"&&h!==null){for(b=0;bx.depthLimit){s(e,h,m,v);return}if(typeof x.edgesLimit<"u"&&g+1>x.edgesLimit){s(e,h,m,v);return}if(p.push(h),Array.isArray(h))for(b=0;bm?1:0}function c(h,m,g,p){typeof p>"u"&&(p=o());var v=u(h,"",0,[],void 0,0,p)||h,y;try{r.length===0?y=JSON.stringify(v,m,g):y=JSON.stringify(v,d(m),g)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var x=n.pop();x.length===4?Object.defineProperty(x[0],x[1],x[3]):x[0][x[1]]=x[2]}}return y}function u(h,m,g,p,v,y,x){y+=1;var b;if(typeof h=="object"&&h!==null){for(b=0;bx.depthLimit){s(e,h,m,v);return}if(typeof x.edgesLimit<"u"&&g+1>x.edgesLimit){s(e,h,m,v);return}if(p.push(h),Array.isArray(h))for(b=0;b0)for(var p=0;p{const t=Object.create(null);function n(...r){const o=r.map(i=>nC(i)).join("|");return t[o]===void 0&&(t[o]=e(...r)),t[o]}return n},Qd=16,ii="px",ya="em",zr="rem";function Jd(e=""){const t=new RegExp(String.raw`-?\d+(?:\.\d+|\d*)`),n=new RegExp(`${ii}|${ya}|${zr}`);return e.match(new RegExp(`${t.source}(${n.source})`))?.[1]}function Zd(e=""){if(typeof e=="number")return`${e}px`;const t=Jd(e);if(!t||t===ii)return e;if(t===ya||t===zr)return`${parseFloat(e)*Qd}${ii}`}function eh(e=""){const t=Jd(e);if(!t||t===zr)return e;if(t===ya)return`${parseFloat(e)}${zr}`;if(t===ii)return`${parseFloat(e)/Qd}${zr}`}const rC=e=>e.charAt(0).toUpperCase()+e.slice(1);function oC(e){const t=iC(e),n=Object.fromEntries(t);function r(h){return n[h]}function o(h){return tr(r(h))}function i(){const h=Object.keys(n),m=sC(h),g=h.flatMap(p=>{const v=r(p),y=[`${p}Down`,tr({max:si(v.min)})],x=[p,tr({min:v.min})],b=[`${p}Only`,o(p)];return[x,b,y]}).filter(([,p])=>p!=="").concat(m.map(([p,v])=>{const y=r(p),x=r(v);return[`${p}To${rC(v)}`,tr({min:y.min,max:si(x.min)})]}));return Object.fromEntries(g)}function s(){const h=i();return Object.fromEntries(Object.entries(h))}const a=s(),l=h=>a[h];function c(){return["base",...Object.keys(n)]}function u(h){return tr({min:r(h).min})}function d(h){return tr({max:si(r(h).min)})}return{values:Object.values(n),only:o,keys:c,conditions:a,getCondition:l,up:u,down:d}}function si(e){const t=parseFloat(Zd(e)??"")-.04;return eh(`${t}px`)}function iC(e){return Object.entries(e).sort(([,n],[,r])=>parseInt(n,10){let s=null;return o<=i.length-1&&(s=i[o+1]?.[1]),s!=null&&(s=si(s)),[n,{name:n,min:eh(r),max:s}]})}function sC(e){const t=[];return e.forEach((n,r)=>{let o=r;o++;let i=e[o];for(;i;)t.push([n,i]),o++,i=e[o]}),t}function tr({min:e,max:t}){return e==null&&t==null?"":["@media screen",e&&`(min-width: ${e})`,t&&`(max-width: ${t})`].filter(Boolean).join(" and ")}const aC=(e,t)=>Object.fromEntries(Object.entries(e).map(([n,r])=>t(n,r))),lC=e=>{const{breakpoints:t,conditions:n={}}=e,r=aC(n,(u,d)=>[`_${u}`,d]),o=Object.assign({},r,t.conditions);function i(){return Object.keys(o)}function s(u){return i().includes(u)||/^@|&|&$/.test(u)||u.startsWith("_")}function a(u){return u.filter(d=>d!=="base").sort((d,h)=>{const m=s(d),g=s(h);return m&&!g?1:!m&&g?-1:0})}function l(u){return u.startsWith("@breakpoint")?t.getCondition(u.replace("@breakpoint ","")):u}function c(u){return Reflect.get(o,u)||u}return{keys:i,sort:a,has:s,resolve:c,breakpoints:t.keys(),expandAtRule:l}},th=e=>({minMax:new RegExp(`(!?\\(\\s*min(-device-)?-${e})(.| + `,n.body.appendChild(r)};function fk(e,t){if(!(!e||!t.isActiveElement(e)))try{const{selectionStart:n,selectionEnd:r,value:o}=e,i=o.substring(0,n),s=o.substring(r);return{start:n,end:r,value:o,beforeTxt:i,afterTxt:s}}catch{}}function gk(e,t,n){if(!(!e||!n.isActiveElement(e))){if(!t){e.setSelectionRange(e.value.length,e.value.length);return}try{const{value:r}=e,{beforeTxt:o="",afterTxt:i="",start:s}=t;let a=r.length;if(r.endsWith(i))a=r.length-i.length;else if(r.startsWith(o))a=o.length;else if(s!=null){const l=o[s-1],c=r.indexOf(l,s-1);c!==-1&&(a=c+1)}e.setSelectionRange(a,a)}catch{}}}var pk=(e,t={})=>new Intl.NumberFormat(e,t),mk=(e,t={})=>new Od(e,t),da=(e,t)=>{const{prop:n,computed:r}=t;return n("formatOptions")?e===""?Number.NaN:r("parser").parse(e):parseFloat(e)},ln=(e,t)=>{const{prop:n,computed:r}=t;return Number.isNaN(e)?"":n("formatOptions")?r("formatter").format(e):e.toString()},vk=(e,t)=>{let n=e!==void 0&&!Number.isNaN(e)?e:1;return t?.style==="percent"&&(e===void 0||Number.isNaN(e))&&(n=.01),n},{choose:bk,guards:yk,createMachine:xk}=wc(),{not:Vd,and:Fd}=yk;xk({props({props:e}){const t=vk(e.step,e.formatOptions);return{dir:"ltr",locale:"en-US",focusInputOnChange:!0,clampValueOnBlur:!e.allowOverflow,allowOverflow:!1,inputMode:"decimal",pattern:"-?[0-9]*(.[0-9]+)?",defaultValue:"",step:t,min:Number.MIN_SAFE_INTEGER,max:Number.MAX_SAFE_INTEGER,spinOnPress:!0,...e,translations:{incrementLabel:"increment value",decrementLabel:"decrease value",...e.translations}}},initialState(){return"idle"},context({prop:e,bindable:t,getComputed:n}){return{value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),onChange(r){const o=n(),i=da(r,{computed:o,prop:e});e("onValueChange")?.({value:r,valueAsNumber:i})}})),hint:t(()=>({defaultValue:null})),scrubberCursorPoint:t(()=>({defaultValue:null,hash(r){return r?`x:${r.x}, y:${r.y}`:""}})),fieldsetDisabled:t(()=>({defaultValue:!1}))}},computed:{isRtl:({prop:e})=>e("dir")==="rtl",valueAsNumber:({context:e,computed:t,prop:n})=>da(e.get("value"),{computed:t,prop:n}),formattedValue:({computed:e,prop:t})=>ln(e("valueAsNumber"),{computed:e,prop:t}),isAtMin:({computed:e,prop:t})=>h0(e("valueAsNumber"),t("min")),isAtMax:({computed:e,prop:t})=>d0(e("valueAsNumber"),t("max")),isOutOfRange:({computed:e,prop:t})=>!f0(e("valueAsNumber"),t("min"),t("max")),isValueEmpty:({context:e})=>e.get("value")==="",isDisabled:({prop:e,context:t})=>!!e("disabled")||t.get("fieldsetDisabled"),canIncrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMax"),canDecrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMin"),valueText:({prop:e,context:t})=>e("translations").valueText?.(t.get("value")),formatter:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>pk(e,t)),parser:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>mk(e,t))},watch({track:e,action:t,context:n,computed:r,prop:o}){e([()=>n.get("value"),()=>o("locale")],()=>{t(["syncInputElement"])}),e([()=>r("isOutOfRange")],()=>{t(["invokeOnInvalid"])}),e([()=>n.hash("scrubberCursorPoint")],()=>{t(["setVirtualCursorPosition"])})},effects:["trackFormControl"],on:{"VALUE.SET":{actions:["setRawValue"]},"VALUE.CLEAR":{actions:["clearValue"]},"VALUE.INCREMENT":{actions:["increment"]},"VALUE.DECREMENT":{actions:["decrement"]}},states:{idle:{on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","invokeOnFocus","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","invokeOnFocus","setHint","setCursorPoint"]},"INPUT.FOCUS":{target:"focused",actions:["focusInput","invokeOnFocus"]}}},focused:{tags:["focus"],effects:["attachWheelListener"],on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","setHint","setCursorPoint"]},"INPUT.ARROW_UP":{actions:["increment"]},"INPUT.ARROW_DOWN":{actions:["decrement"]},"INPUT.HOME":{actions:["decrementToMin"]},"INPUT.END":{actions:["incrementToMax"]},"INPUT.CHANGE":{actions:["setValue","setHint"]},"INPUT.BLUR":[{guard:Fd("clampValueOnBlur",Vd("isInRange")),target:"idle",actions:["setClampedValue","clearHint","invokeOnBlur"]},{guard:Vd("isInRange"),target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur","invokeOnInvalid"]},{target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur"]}],"INPUT.ENTER":{actions:["setFormattedValue","clearHint","invokeOnBlur"]}}},"before:spin":{tags:["focus"],effects:["trackButtonDisabled","waitForChangeDelay"],entry:bk([{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}]),on:{CHANGE_DELAY:{target:"spinning",guard:Fd("isInRange","spinOnPress")},"TRIGGER.PRESS_UP":[{guard:"isTouchPointer",target:"focused",actions:["clearHint"]},{target:"focused",actions:["focusInput","clearHint"]}]}},spinning:{tags:["focus"],effects:["trackButtonDisabled","spinValue"],on:{SPIN:[{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}],"TRIGGER.PRESS_UP":{target:"focused",actions:["focusInput","clearHint"]}}},scrubbing:{tags:["focus"],effects:["activatePointerLock","trackMousemove","setupVirtualCursor","preventTextSelection"],on:{"SCRUBBER.POINTER_UP":{target:"focused",actions:["focusInput","clearCursorPoint"]},"SCRUBBER.POINTER_MOVE":[{guard:"isIncrementHint",actions:["increment","setCursorPoint"]},{guard:"isDecrementHint",actions:["decrement","setCursorPoint"]}]}}},implementations:{guards:{clampValueOnBlur:({prop:e})=>e("clampValueOnBlur"),spinOnPress:({prop:e})=>!!e("spinOnPress"),isInRange:({computed:e})=>!e("isOutOfRange"),isDecrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="decrement",isIncrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="increment",isTouchPointer:({event:e})=>e.pointerType==="touch"},effects:{waitForChangeDelay({send:e}){const t=setTimeout(()=>{e({type:"CHANGE_DELAY"})},300);return()=>clearTimeout(t)},spinValue({send:e}){const t=setInterval(()=>{e({type:"SPIN"})},50);return()=>clearInterval(t)},trackFormControl({context:e,scope:t}){const n=ii(t);return ys(n,{onFieldsetDisabledChange(r){e.set("fieldsetDisabled",r)},onFormReset(){e.set("value",e.initial("value"))}})},setupVirtualCursor({context:e,scope:t}){const n=e.get("scrubberCursorPoint");return ck(t,n)},preventTextSelection({scope:e}){return uk(e)},trackButtonDisabled({context:e,scope:t,send:n}){const r=e.get("hint"),o=lk(t,r);return Eo(o,{attributes:["disabled"],callback(){n({type:"TRIGGER.PRESS_UP",src:"attr"})}})},attachWheelListener({scope:e,send:t,prop:n}){const r=ii(e);if(!r||!e.isActiveElement(r)||!n("allowMouseWheel"))return;function o(i){i.preventDefault();const s=Math.sign(i.deltaY)*-1;s===1?t({type:"VALUE.INCREMENT"}):s===-1&&t({type:"VALUE.DECREMENT"})}return se(r,"wheel",o,{passive:!1})},activatePointerLock({scope:e}){if(!fc())return dv(e.getDoc())},trackMousemove({scope:e,send:t,context:n,computed:r}){const o=e.getDoc();function i(a){const l=n.get("scrubberCursorPoint"),c=r("isRtl"),u=dk(e,{point:l,isRtl:c,event:a});u.hint&&t({type:"SCRUBBER.POINTER_MOVE",hint:u.hint,point:u.point})}function s(){t({type:"SCRUBBER.POINTER_UP"})}return mo(se(o,"mousemove",i,!1),se(o,"mouseup",s,!1))}},actions:{focusInput({scope:e,prop:t}){if(!t("focusInputOnChange"))return;const n=ii(e);e.isActiveElement(n)||G(()=>n?.focus({preventScroll:!0}))},increment({context:e,event:t,prop:n,computed:r}){let o=v0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},decrement({context:e,event:t,prop:n,computed:r}){let o=b0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setClampedValue({context:e,prop:t,computed:n}){const r=Ve(n("valueAsNumber"),t("min"),t("max"));e.set("value",ln(r,{computed:n,prop:t}))},setRawValue({context:e,event:t,prop:n,computed:r}){let o=da(t.value,{computed:r,prop:n});n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setValue({context:e,event:t}){const n=t.target?.value??t.value;e.set("value",n)},clearValue({context:e}){e.set("value","")},incrementToMax({context:e,prop:t,computed:n}){const r=ln(t("max"),{computed:n,prop:t});e.set("value",r)},decrementToMin({context:e,prop:t,computed:n}){const r=ln(t("min"),{computed:n,prop:t});e.set("value",r)},setHint({context:e,event:t}){e.set("hint",t.hint)},clearHint({context:e}){e.set("hint",null)},invokeOnFocus({computed:e,prop:t}){t("onFocusChange")?.({focused:!0,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnBlur({computed:e,prop:t}){t("onFocusChange")?.({focused:!1,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnInvalid({computed:e,prop:t,event:n}){if(n.type==="INPUT.CHANGE")return;const r=e("valueAsNumber")>t("max")?"rangeOverflow":"rangeUnderflow";t("onValueInvalid")?.({reason:r,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},syncInputElement({context:e,event:t,computed:n,scope:r}){const o=t.type.endsWith("CHANGE")?e.get("value"):n("formattedValue"),i=ii(r),s=fk(i,r);G(()=>{So(i,o),gk(i,s,r)})},setFormattedValue({context:e,computed:t}){e.set("value",t("formattedValue"))},setCursorPoint({context:e,event:t}){e.set("scrubberCursorPoint",t.point)},clearCursorPoint({context:e}){e.set("scrubberCursorPoint",null)},setVirtualCursorPosition({context:e,scope:t}){const n=_d(t),r=e.get("scrubberCursorPoint");!n||!r||(n.style.transform=`translate3d(${r.x}px, ${r.y}px, 0px)`)}}}}),M()(["allowMouseWheel","allowOverflow","clampValueOnBlur","dir","disabled","focusInputOnChange","form","formatOptions","getRootNode","id","ids","inputMode","invalid","locale","max","min","name","onFocusChange","onValueChange","onValueInvalid","pattern","required","readOnly","spinOnPress","step","translations","value","defaultValue"]);var Ld=j("pinInput").parts("root","label","input","control");Ld.build(),M()(["autoFocus","blurOnComplete","count","defaultValue","dir","disabled","form","getRootNode","id","ids","invalid","mask","name","onValueChange","onValueComplete","onValueInvalid","otp","pattern","placeholder","readOnly","required","selectOnFocus","translations","type","value"]);var Dd=j("popover").parts("arrow","arrowTip","anchor","trigger","indicator","positioner","content","title","description","closeTrigger");Dd.build(),M()(["autoFocus","closeOnEscape","closeOnInteractOutside","dir","getRootNode","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","portalled","positioning"]);const cn=e=>{const{children:t,disabled:n}=e,[r,o]=P.useState(e.container?.current),i=P.useSyncExternalStore(Ck,()=>!1,()=>!0),{getRootNode:s}=sc();if(P.useEffect(()=>{o(()=>e.container?.current)},[e.container]),i||n)return g.jsx(g.Fragment,{children:t});const a=r??kk(s);return g.jsx(g.Fragment,{children:P.Children.map(t,l=>Ft.createPortal(l,a))})},kk=e=>{const t=e?.(),n=t.getRootNode();return jn(n)?n:Fe(t).body},Ck=()=>()=>{};var ha=j("progress").parts("root","label","track","range","valueText","view","circle","circleTrack","circleRange");ha.build(),M()(["dir","getRootNode","id","ids","max","min","orientation","translations","value","onValueChange","defaultValue","formatOptions","locale"]);var zd=j("qr-code").parts("root","frame","pattern","overlay","downloadTrigger");zd.build(),M()(["ids","defaultValue","value","id","encoding","dir","getRootNode","onValueChange","pixelSize"]);var fa=j("radio-group").parts("root","label","item","itemText","itemControl","indicator");fa.build(),M()(["dir","disabled","form","getRootNode","id","ids","name","onValueChange","orientation","readOnly","value","defaultValue"]),M()(["value","disabled","invalid"]);var Md=j("rating-group").parts("root","label","item","control");Md.build(),M()(["allowHalf","autoFocus","count","dir","disabled","form","getRootNode","id","ids","name","onHoverChange","onValueChange","required","readOnly","translations","value","defaultValue"]),M()(["index"]);var jd=j("scroll-area").parts("root","viewport","content","scrollbar","thumb","corner");jd.build(),M()(["dir","getRootNode","ids","id"]);const $d=fa.rename("segment-group");$d.build();var Bd=j("select").parts("label","positioner","trigger","indicator","clearTrigger","item","itemText","itemIndicator","itemGroup","itemGroupLabel","list","content","root","control","valueText");Bd.build();var Wd=e=>new Yo(e);Wd.empty=()=>new Yo({items:[]});var wk=e=>e.ids?.content??`select:${e.id}:content`,Sk=e=>e.ids?.trigger??`select:${e.id}:trigger`,Ek=e=>e.ids?.clearTrigger??`select:${e.id}:clear-trigger`,Ok=(e,t)=>e.ids?.item?.(t)??`select:${e.id}:option:${t}`,Pk=e=>e.ids?.hiddenSelect??`select:${e.id}:select`,Rk=e=>e.ids?.positioner??`select:${e.id}:positioner`,ga=e=>e.getById(Pk(e)),zr=e=>e.getById(wk(e)),si=e=>e.getById(Sk(e)),Ik=e=>e.getById(Ek(e)),Hd=e=>e.getById(Rk(e)),pa=(e,t)=>t==null?null:e.getById(Ok(e,t)),{and:Mr,not:un,or:Tk}=jt();Tk("isTriggerArrowDownEvent","isTriggerEnterEvent"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),un("multiple"),un("multiple"),un("multiple"),Mr("closeOnSelect","isOpenControlled"),Mr("hasHighlightedItem","loop","isLastItemHighlighted"),Mr("hasHighlightedItem","loop","isFirstItemHighlighted");function Ud(e){const t=e.restoreFocus??e.previousEvent?.restoreFocus;return t==null||!!t}M()(["closeOnSelect","collection","composite","defaultHighlightedValue","defaultOpen","defaultValue","deselectable","dir","disabled","form","getRootNode","highlightedValue","id","ids","invalid","loopFocus","multiple","name","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","positioning","readOnly","required","scrollToIndexFn","value"]),M()(["item","persistFocus"]),M()(["id"]),M()(["htmlFor"]);var Gd=j("slider").parts("root","label","thumb","valueText","track","range","control","markerGroup","marker","draggingIndicator");Gd.build(),M()(["aria-label","aria-labelledby","dir","disabled","form","getAriaValueText","getRootNode","id","ids","invalid","max","min","minStepsBetweenThumbs","name","onFocusChange","onValueChange","onValueChangeEnd","orientation","origin","readOnly","step","thumbAlignment","thumbAlignment","thumbSize","value","defaultValue"]),M()(["index","name"]);var qd=j("switch").parts("root","label","control","thumb");qd.build(),M()(["checked","defaultChecked","dir","disabled","form","getRootNode","id","ids","invalid","label","name","onCheckedChange","readOnly","required","value"]);var Nk=j("toast").parts("group","root","title","description","actionTrigger","closeTrigger");Nk.build();var Ak=(e,t)=>({...t,...bo(e)});function _k(e={}){const t=Ak(e,{placement:"bottom",overlap:!1,max:24,gap:16,offsets:"1rem",hotkey:["altKey","KeyT"],removeDelay:200,pauseOnPageIdle:!0});let n=[],r=[],o=new Set,i=[];const s=S=>(n.push(S),()=>{const _=n.indexOf(S);n.splice(_,1)}),a=S=>(n.forEach(_=>_(S)),S),l=S=>{if(r.length>=t.max){i.push(S);return}a(S),r.unshift(S)},c=()=>{for(;i.length>0&&r.length{const _=S.id??`toast:${i0()}`,D=r.find(z=>z.id===_);return o.has(_)&&o.delete(_),D?r=r.map(z=>z.id===_?a({...z,...S,id:_}):z):l({id:_,duration:t.duration,removeDelay:t.removeDelay,type:"info",...S,stacked:!t.overlap,gap:t.gap}),_},h=S=>(o.add(S),S?(n.forEach(_=>_({id:S,dismiss:!0})),r=r.filter(_=>_.id!==S),c()):(r.forEach(_=>{n.forEach(D=>D({id:_.id,dismiss:!0}))}),r=[],i=[]),S);return{attrs:t,subscribe:s,create:u,update:(S,_)=>u({id:S,..._}),remove:h,dismiss:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"DISMISS"}):_):r=r.map(_=>a({..._,message:"DISMISS"}))},error:S=>u({...S,type:"error"}),success:S=>u({...S,type:"success"}),info:S=>u({...S,type:"info"}),warning:S=>u({...S,type:"warning"}),loading:S=>u({...S,type:"loading"}),getVisibleToasts:()=>r.filter(S=>!o.has(S.id)),getCount:()=>r.length,promise:(S,_,D={})=>{if(!_||!_.loading){wr("[zag-js > toast] toaster.promise() requires at least a 'loading' option to be specified");return}const z=u({...D,..._.loading,promise:S,type:"loading"});let W=!0,J;const ee=po(S).then(async q=>{if(J=["resolve",q],Vk(q)&&!q.ok){W=!1;const Z=po(_.error,`HTTP Error! status: ${q.status}`);u({...D,...Z,id:z,type:"error"})}else if(_.success!==void 0){W=!1;const Z=po(_.success,q);u({...D,...Z,id:z,type:"success"})}}).catch(async q=>{if(J=["reject",q],_.error!==void 0){W=!1;const Z=po(_.error,q);u({...D,...Z,id:z,type:"error"})}}).finally(()=>{W&&h(z),_.finally?.()});return{id:z,unwrap:()=>new Promise((q,Z)=>ee.then(()=>J[0]==="reject"?Z(J[1]):q(J[1])).catch(Z))}},pause:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"PAUSE"}):_):r=r.map(_=>a({..._,message:"PAUSE"}))},resume:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"RESUME"}):_):r=r.map(_=>a({..._,message:"RESUME"}))},isVisible:S=>!o.has(S)&&!!r.find(_=>_.id===S),isDismissed:S=>o.has(S),expand:()=>{r=r.map(S=>a({...S,stacked:!0}))},collapse:()=>{r=r.map(S=>a({...S,stacked:!1}))}}}var Vk=e=>e&&typeof e=="object"&&"ok"in e&&typeof e.ok=="boolean"&&"status"in e&&typeof e.status=="number";const Fk=e=>_k(e);var Kd=j("tooltip").parts("trigger","arrow","arrowTip","positioner","content");Kd.build();var Lk=e=>e.ids?.trigger??`tooltip:${e.id}:trigger`,Dk=e=>e.ids?.positioner??`tooltip:${e.id}:popper`,ma=e=>e.getById(Lk(e)),Yd=e=>e.getById(Dk(e)),dn=y0({id:null}),{and:zk,not:Xd}=jt();zk("noVisibleTooltip",Xd("hasPointerMoveOpened")),Xd("hasPointerMoveOpened"),M()(["aria-label","closeDelay","closeOnEscape","closeOnPointerDown","closeOnScroll","closeOnClick","dir","disabled","getRootNode","id","ids","interactive","onOpenChange","defaultOpen","open","openDelay","positioning"]);function Qd(e,t=[]){const n=Object.assign({},e);for(const r of t)r in n&&delete n[r];return n}const Mk=(e,t)=>{if(!e||typeof e!="string")return{invalid:!0,value:e};const[n,r]=e.split("/");if(!n||!r||n==="currentBg")return{invalid:!0,value:n};const o=t(`colors.${n}`),i=t.raw(`opacity.${r}`)?.value;if(!i&&isNaN(Number(r)))return{invalid:!0,value:n};const s=i?Number(i)*100+"%":`${r}%`,a=o??n;return{invalid:!1,color:a,value:`color-mix(in srgb, ${a} ${s}, transparent)`}},te=e=>(t,n)=>{const r=n.utils.colorMix(t);if(r.invalid)return{[e]:t};const o="--mix-"+e;return{[o]:r.value,[e]:`var(${o}, ${r.color})`}};function va(e){if(e===null||typeof e!="object")return e;if(Array.isArray(e))return e.map(n=>va(n));const t=Object.create(Object.getPrototypeOf(e));for(const n of Object.keys(e))t[n]=va(e[n]);return t}function ba(e,t){if(t==null)return e;for(const n of Object.keys(t))if(!(t[n]===void 0||n==="__proto__"))if(!_e(e[n])&&_e(t[n]))Object.assign(e,{[n]:t[n]});else if(e[n]&&_e(t[n]))ba(e[n],t[n]);else if(Array.isArray(t[n])&&Array.isArray(e[n])){let r=0;for(;re!=null;function gt(e,t,n={}){const{stop:r,getKey:o}=n;function i(s,a=[]){if(_e(s)||Array.isArray(s)){const l={};for(const[c,u]of Object.entries(s)){const h=o?.(c,u)??c,d=[...a,h];if(r?.(s,d))return t(s,a);const m=i(u,d);ya(m)&&(l[h]=m)}return l}return t(s,a)}return i(e)}function Jd(e,t){return Array.isArray(e)?e.map(n=>ya(n)?t(n):n):_e(e)?gt(e,n=>t(n)):ya(e)?t(e):e}const ai=["value","type","description"],jk=e=>e&&typeof e=="object"&&!Array.isArray(e),Zd=(...e)=>{const t=er({},...e.map(va));return t.theme?.tokens&>(t.theme.tokens,n=>{const i=Object.keys(n).filter(a=>!ai.includes(a)).length>0,s=ai.some(a=>n[a]!=null);return i&&s&&(n.DEFAULT||(n.DEFAULT={}),ai.forEach(a=>{var l;n[a]!=null&&((l=n.DEFAULT)[a]||(l[a]=n[a]),delete n[a])})),n},{stop(n){return jk(n)&&Object.keys(n).some(r=>ai.includes(r)||r!==r.toLowerCase()&&r!==r.toUpperCase())}}),t},$k=e=>e,fe=e=>e,B=e=>e,Bk=e=>e,Wk=e=>e,tr=e=>e,Hk=e=>e,Uk=e=>e,Gk=e=>e;function eh(){const e=t=>t;return new Proxy(e,{get(){return e}})}const de=eh(),xa=eh(),ka=e=>e,qk=/[^a-zA-Z0-9_\u0081-\uffff-]/g;function Kk(e){return`${e}`.replace(qk,t=>`\\${t}`)}const Yk=/[A-Z]/g;function Xk(e){return e.replace(Yk,t=>`-${t.toLowerCase()}`)}function th(e,t={}){const{fallback:n="",prefix:r=""}=t,o=Xk(["-",r,Kk(e)].filter(Boolean).join("-"));return{var:o,ref:`var(${o}${n?`, ${n}`:""})`}}const Qk=e=>/^var\(--.+\)$/.test(e),xe=(e,t)=>t!=null?`${e}(${t})`:t,hn=e=>{if(Qk(e)||e==null)return e;const t=typeof e=="string"&&!e.endsWith("deg");return typeof e=="number"||t?`${e}deg`:e},nh=e=>({values:["outside","inside","mixed","none"],transform(t,{token:n}){const r=n("colors.colorPalette.focusRing");return{inside:{"--focus-ring-color":r,[e]:{outlineOffset:"0px",outlineWidth:"var(--focus-ring-width, 1px)",outlineColor:"var(--focus-ring-color)",outlineStyle:"var(--focus-ring-style, solid)",borderColor:"var(--focus-ring-color)"}},outside:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 2px)",outlineOffset:"var(--focus-ring-offset, 2px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"var(--focus-ring-color)"}},mixed:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 3px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"color-mix(in srgb, var(--focus-ring-color), transparent 60%)",borderColor:"var(--focus-ring-color)"}},none:{"--focus-ring-color":r,[e]:{outline:"none"}}}[t]??{}}}),Jk=te("borderColor"),St=e=>({transition:e,transitionTimingFunction:"cubic-bezier(0.4, 0, 0.2, 1)",transitionDuration:"150ms"}),Zk=$k({hover:["@media (hover: hover)","&:is(:hover, [data-hover]):not(:disabled, [data-disabled])"],active:"&:is(:active, [data-active]):not(:disabled, [data-disabled], [data-state=open])",focus:"&:is(:focus, [data-focus])",focusWithin:"&:is(:focus-within, [data-focus-within])",focusVisible:"&:is(:focus-visible, [data-focus-visible])",disabled:"&:is(:disabled, [disabled], [data-disabled], [aria-disabled=true])",visited:"&:visited",target:"&:target",readOnly:"&:is([data-readonly], [aria-readonly=true], [readonly])",readWrite:"&:read-write",empty:"&:is(:empty, [data-empty])",checked:"&:is(:checked, [data-checked], [aria-checked=true], [data-state=checked])",enabled:"&:enabled",expanded:"&:is([aria-expanded=true], [data-expanded], [data-state=expanded])",highlighted:"&[data-highlighted]",complete:"&[data-complete]",incomplete:"&[data-incomplete]",dragging:"&[data-dragging]",before:"&::before",after:"&::after",firstLetter:"&::first-letter",firstLine:"&::first-line",marker:"&::marker",selection:"&::selection",file:"&::file-selector-button",backdrop:"&::backdrop",first:"&:first-of-type",last:"&:last-of-type",notFirst:"&:not(:first-of-type)",notLast:"&:not(:last-of-type)",only:"&:only-child",even:"&:nth-of-type(even)",odd:"&:nth-of-type(odd)",peerFocus:".peer:is(:focus, [data-focus]) ~ &",peerHover:".peer:is(:hover, [data-hover]):not(:disabled, [data-disabled]) ~ &",peerActive:".peer:is(:active, [data-active]):not(:disabled, [data-disabled]) ~ &",peerFocusWithin:".peer:focus-within ~ &",peerFocusVisible:".peer:is(:focus-visible, [data-focus-visible]) ~ &",peerDisabled:".peer:is(:disabled, [disabled], [data-disabled]) ~ &",peerChecked:".peer:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) ~ &",peerInvalid:".peer:is(:invalid, [data-invalid], [aria-invalid=true]) ~ &",peerExpanded:".peer:is([aria-expanded=true], [data-expanded], [data-state=expanded]) ~ &",peerPlaceholderShown:".peer:placeholder-shown ~ &",groupFocus:".group:is(:focus, [data-focus]) &",groupHover:".group:is(:hover, [data-hover]):not(:disabled, [data-disabled]) &",groupActive:".group:is(:active, [data-active]):not(:disabled, [data-disabled]) &",groupFocusWithin:".group:focus-within &",groupFocusVisible:".group:is(:focus-visible, [data-focus-visible]) &",groupDisabled:".group:is(:disabled, [disabled], [data-disabled]) &",groupChecked:".group:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) &",groupExpanded:".group:is([aria-expanded=true], [data-expanded], [data-state=expanded]) &",groupInvalid:".group:invalid &",indeterminate:"&:is(:indeterminate, [data-indeterminate], [aria-checked=mixed], [data-state=indeterminate])",required:"&:is([data-required], [aria-required=true])",valid:"&:is([data-valid], [data-state=valid])",invalid:"&:is([data-invalid], [aria-invalid=true], [data-state=invalid])",autofill:"&:autofill",inRange:"&:is(:in-range, [data-in-range])",outOfRange:"&:is(:out-of-range, [data-outside-range])",placeholder:"&::placeholder, &[data-placeholder]",placeholderShown:"&:is(:placeholder-shown, [data-placeholder-shown])",pressed:"&:is([aria-pressed=true], [data-pressed])",selected:"&:is([aria-selected=true], [data-selected])",grabbed:"&:is([aria-grabbed=true], [data-grabbed])",underValue:"&[data-state=under-value]",overValue:"&[data-state=over-value]",atValue:"&[data-state=at-value]",default:"&:default",optional:"&:optional",open:"&:is([open], [data-open], [data-state=open])",closed:"&:is([closed], [data-closed], [data-state=closed])",fullscreen:"&:is(:fullscreen, [data-fullscreen])",loading:"&:is([data-loading], [aria-busy=true])",hidden:"&:is([hidden], [data-hidden])",current:"&[data-current]",currentPage:"&[aria-current=page]",currentStep:"&[aria-current=step]",today:"&[data-today]",unavailable:"&[data-unavailable]",rangeStart:"&[data-range-start]",rangeEnd:"&[data-range-end]",now:"&[data-now]",topmost:"&[data-topmost]",motionReduce:"@media (prefers-reduced-motion: reduce)",motionSafe:"@media (prefers-reduced-motion: no-preference)",print:"@media print",landscape:"@media (orientation: landscape)",portrait:"@media (orientation: portrait)",dark:".dark &, .dark .chakra-theme:not(.light) &",light:":root &, .light &",osDark:"@media (prefers-color-scheme: dark)",osLight:"@media (prefers-color-scheme: light)",highContrast:"@media (forced-colors: active)",lessContrast:"@media (prefers-contrast: less)",moreContrast:"@media (prefers-contrast: more)",ltr:"[dir=ltr] &",rtl:"[dir=rtl] &",scrollbar:"&::-webkit-scrollbar",scrollbarThumb:"&::-webkit-scrollbar-thumb",scrollbarTrack:"&::-webkit-scrollbar-track",horizontal:"&[data-orientation=horizontal]",vertical:"&[data-orientation=vertical]",icon:"& :where(svg)",starting:"@starting-style"}),nr=th("bg-currentcolor"),rh=e=>e===nr.ref||e==="currentBg",ne=e=>({...e("colors"),currentBg:nr}),eC=ka({conditions:Zk,utilities:{background:{values:ne,shorthand:["bg"],transform(e,t){if(rh(t.raw))return{background:nr.ref};const n=te("background")(e,t);return{...n,[nr.var]:n?.background}}},backgroundColor:{values:ne,shorthand:["bgColor"],transform(e,t){if(rh(t.raw))return{backgroundColor:nr.ref};const n=te("backgroundColor")(e,t);return{...n,[nr.var]:n?.backgroundColor}}},backgroundSize:{shorthand:["bgSize"]},backgroundPosition:{shorthand:["bgPos"]},backgroundRepeat:{shorthand:["bgRepeat"]},backgroundAttachment:{shorthand:["bgAttachment"]},backgroundClip:{shorthand:["bgClip"],values:["text"],transform(e){return e==="text"?{color:"transparent",backgroundClip:"text"}:{backgroundClip:e}}},backgroundGradient:{shorthand:["bgGradient"],values(e){return{...e("gradients"),"to-t":"linear-gradient(to top, var(--gradient))","to-tr":"linear-gradient(to top right, var(--gradient))","to-r":"linear-gradient(to right, var(--gradient))","to-br":"linear-gradient(to bottom right, var(--gradient))","to-b":"linear-gradient(to bottom, var(--gradient))","to-bl":"linear-gradient(to bottom left, var(--gradient))","to-l":"linear-gradient(to left, var(--gradient))","to-tl":"linear-gradient(to top left, var(--gradient))"}},transform(e){return{"--gradient-stops":"var(--gradient-from), var(--gradient-to)","--gradient":"var(--gradient-via-stops, var(--gradient-stops))",backgroundImage:e}}},gradientFrom:{values:ne,transform:te("--gradient-from")},gradientTo:{values:ne,transform:te("--gradient-to")},gradientVia:{values:ne,transform(e,t){return{...te("--gradient-via")(e,t),"--gradient-via-stops":"var(--gradient-from), var(--gradient-via), var(--gradient-to)"}}},backgroundImage:{values(e){return{...e("gradients"),...e("assets")}},shorthand:["bgImg","bgImage"]},border:{values:"borders"},borderTop:{values:"borders"},borderLeft:{values:"borders"},borderBlockStart:{values:"borders"},borderRight:{values:"borders"},borderBottom:{values:"borders"},borderBlockEnd:{values:"borders"},borderInlineStart:{values:"borders",shorthand:["borderStart"]},borderInlineEnd:{values:"borders",shorthand:["borderEnd"]},borderInline:{values:"borders",shorthand:["borderX"]},borderBlock:{values:"borders",shorthand:["borderY"]},borderColor:{values:ne,transform:te("borderColor")},borderTopColor:{values:ne,transform:te("borderTopColor")},borderBlockStartColor:{values:ne,transform:te("borderBlockStartColor")},borderBottomColor:{values:ne,transform:te("borderBottomColor")},borderBlockEndColor:{values:ne,transform:te("borderBlockEndColor")},borderLeftColor:{values:ne,transform:te("borderLeftColor")},borderInlineStartColor:{values:ne,shorthand:["borderStartColor"],transform:te("borderInlineStartColor")},borderRightColor:{values:ne,transform:te("borderRightColor")},borderInlineEndColor:{values:ne,shorthand:["borderEndColor"],transform:te("borderInlineEndColor")},borderStyle:{values:"borderStyles"},borderTopStyle:{values:"borderStyles"},borderBlockStartStyle:{values:"borderStyles"},borderBottomStyle:{values:"borderStyles"},borderBlockEndStyle:{values:"borderStyles"},borderInlineStartStyle:{values:"borderStyles",shorthand:["borderStartStyle"]},borderInlineEndStyle:{values:"borderStyles",shorthand:["borderEndStyle"]},borderLeftStyle:{values:"borderStyles"},borderRightStyle:{values:"borderStyles"},borderRadius:{values:"radii",shorthand:["rounded"]},borderTopLeftRadius:{values:"radii",shorthand:["roundedTopLeft"]},borderStartStartRadius:{values:"radii",shorthand:["roundedStartStart","borderTopStartRadius"]},borderEndStartRadius:{values:"radii",shorthand:["roundedEndStart","borderBottomStartRadius"]},borderTopRightRadius:{values:"radii",shorthand:["roundedTopRight"]},borderStartEndRadius:{values:"radii",shorthand:["roundedStartEnd","borderTopEndRadius"]},borderEndEndRadius:{values:"radii",shorthand:["roundedEndEnd","borderBottomEndRadius"]},borderBottomLeftRadius:{values:"radii",shorthand:["roundedBottomLeft"]},borderBottomRightRadius:{values:"radii",shorthand:["roundedBottomRight"]},borderInlineStartRadius:{values:"radii",property:"borderRadius",shorthand:["roundedStart","borderStartRadius"],transform:e=>({borderStartStartRadius:e,borderEndStartRadius:e})},borderInlineEndRadius:{values:"radii",property:"borderRadius",shorthand:["roundedEnd","borderEndRadius"],transform:e=>({borderStartEndRadius:e,borderEndEndRadius:e})},borderTopRadius:{values:"radii",property:"borderRadius",shorthand:["roundedTop"],transform:e=>({borderTopLeftRadius:e,borderTopRightRadius:e})},borderBottomRadius:{values:"radii",property:"borderRadius",shorthand:["roundedBottom"],transform:e=>({borderBottomLeftRadius:e,borderBottomRightRadius:e})},borderLeftRadius:{values:"radii",property:"borderRadius",shorthand:["roundedLeft"],transform:e=>({borderTopLeftRadius:e,borderBottomLeftRadius:e})},borderRightRadius:{values:"radii",property:"borderRadius",shorthand:["roundedRight"],transform:e=>({borderTopRightRadius:e,borderBottomRightRadius:e})},borderWidth:{values:"borderWidths"},borderBlockStartWidth:{values:"borderWidths"},borderTopWidth:{values:"borderWidths"},borderBottomWidth:{values:"borderWidths"},borderBlockEndWidth:{values:"borderWidths"},borderRightWidth:{values:"borderWidths"},borderInlineWidth:{values:"borderWidths",shorthand:["borderXWidth"]},borderInlineStartWidth:{values:"borderWidths",shorthand:["borderStartWidth"]},borderInlineEndWidth:{values:"borderWidths",shorthand:["borderEndWidth"]},borderLeftWidth:{values:"borderWidths"},borderBlockWidth:{values:"borderWidths",shorthand:["borderYWidth"]},color:{values:ne,transform:te("color")},fill:{values:ne,transform:te("fill")},stroke:{values:ne,transform:te("stroke")},accentColor:{values:ne,transform:te("accentColor")},divideX:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderInlineStartWidth:e,borderInlineEndWidth:"0px"}}}},divideY:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderTopWidth:e,borderBottomWidth:"0px"}}}},divideColor:{values:ne,transform(e,t){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":Jk(e,t)}}},divideStyle:{property:"borderStyle",transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderStyle:e}}}},boxShadow:{values:"shadows",shorthand:["shadow"]},boxShadowColor:{values:ne,transform:te("--shadow-color"),shorthand:["shadowColor"]},mixBlendMode:{shorthand:["blendMode"]},backgroundBlendMode:{shorthand:["bgBlendMode"]},opacity:{values:"opacity"},filter:{transform(e){return e!=="auto"?{filter:e}:{filter:"var(--blur) var(--brightness) var(--contrast) var(--grayscale) var(--hue-rotate) var(--invert) var(--saturate) var(--sepia) var(--drop-shadow)"}}},blur:{values:"blurs",transform:e=>({"--blur":xe("blur",e)})},brightness:{transform:e=>({"--brightness":xe("brightness",e)})},contrast:{transform:e=>({"--contrast":xe("contrast",e)})},grayscale:{transform:e=>({"--grayscale":xe("grayscale",e)})},hueRotate:{transform:e=>({"--hue-rotate":xe("hue-rotate",hn(e))})},invert:{transform:e=>({"--invert":xe("invert",e)})},saturate:{transform:e=>({"--saturate":xe("saturate",e)})},sepia:{transform:e=>({"--sepia":xe("sepia",e)})},dropShadow:{transform:e=>({"--drop-shadow":xe("drop-shadow",e)})},backdropFilter:{transform(e){return e!=="auto"?{backdropFilter:e}:{backdropFilter:"var(--backdrop-blur) var(--backdrop-brightness) var(--backdrop-contrast) var(--backdrop-grayscale) var(--backdrop-hue-rotate) var(--backdrop-invert) var(--backdrop-opacity) var(--backdrop-saturate) var(--backdrop-sepia)"}}},backdropBlur:{values:"blurs",transform:e=>({"--backdrop-blur":xe("blur",e)})},backdropBrightness:{transform:e=>({"--backdrop-brightness":xe("brightness",e)})},backdropContrast:{transform:e=>({"--backdrop-contrast":xe("contrast",e)})},backdropGrayscale:{transform:e=>({"--backdrop-grayscale":xe("grayscale",e)})},backdropHueRotate:{transform:e=>({"--backdrop-hue-rotate":xe("hue-rotate",hn(e))})},backdropInvert:{transform:e=>({"--backdrop-invert":xe("invert",e)})},backdropOpacity:{transform:e=>({"--backdrop-opacity":xe("opacity",e)})},backdropSaturate:{transform:e=>({"--backdrop-saturate":xe("saturate",e)})},backdropSepia:{transform:e=>({"--backdrop-sepia":xe("sepia",e)})},flexBasis:{values:"sizes"},gap:{values:"spacing"},rowGap:{values:"spacing",shorthand:["gapY"]},columnGap:{values:"spacing",shorthand:["gapX"]},flexDirection:{shorthand:["flexDir"]},gridGap:{values:"spacing"},gridColumnGap:{values:"spacing"},gridRowGap:{values:"spacing"},outlineColor:{values:ne,transform:te("outlineColor")},focusRing:nh("&:is(:focus, [data-focus])"),focusVisibleRing:nh("&:is(:focus-visible, [data-focus-visible])"),focusRingColor:{values:ne,transform:te("--focus-ring-color")},focusRingOffset:{values:"spacing",transform:e=>({"--focus-ring-offset":e})},focusRingWidth:{values:"borderWidths",property:"outlineWidth",transform:e=>({"--focus-ring-width":e})},focusRingStyle:{values:"borderStyles",property:"outlineStyle",transform:e=>({"--focus-ring-style":e})},aspectRatio:{values:"aspectRatios"},width:{values:"sizes",shorthand:["w"]},inlineSize:{values:"sizes"},height:{values:"sizes",shorthand:["h"]},blockSize:{values:"sizes"},boxSize:{values:"sizes",property:"width",transform:e=>({width:e,height:e})},minWidth:{values:"sizes",shorthand:["minW"]},minInlineSize:{values:"sizes"},minHeight:{values:"sizes",shorthand:["minH"]},minBlockSize:{values:"sizes"},maxWidth:{values:"sizes",shorthand:["maxW"]},maxInlineSize:{values:"sizes"},maxHeight:{values:"sizes",shorthand:["maxH"]},maxBlockSize:{values:"sizes"},hideFrom:{values:"breakpoints",transform:(e,{raw:t,token:n})=>({[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}`:`@media screen and (min-width: ${e})`]:{display:"none"}})},hideBelow:{values:"breakpoints",transform(e,{raw:t,token:n}){return{[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}Down`:`@media screen and (max-width: ${e})`]:{display:"none"}}}},overscrollBehavior:{shorthand:["overscroll"]},overscrollBehaviorX:{shorthand:["overscrollX"]},overscrollBehaviorY:{shorthand:["overscrollY"]},scrollbar:{values:["visible","hidden"],transform(e){switch(e){case"visible":return{msOverflowStyle:"auto",scrollbarWidth:"auto","&::-webkit-scrollbar":{display:"block"}};case"hidden":return{msOverflowStyle:"none",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}};default:return{}}}},scrollbarColor:{values:ne,transform:te("scrollbarColor")},scrollbarGutter:{values:"spacing"},scrollbarWidth:{values:"sizes"},scrollMargin:{values:"spacing"},scrollMarginTop:{values:"spacing"},scrollMarginBottom:{values:"spacing"},scrollMarginLeft:{values:"spacing"},scrollMarginRight:{values:"spacing"},scrollMarginX:{values:"spacing",transform:e=>({scrollMarginLeft:e,scrollMarginRight:e})},scrollMarginY:{values:"spacing",transform:e=>({scrollMarginTop:e,scrollMarginBottom:e})},scrollPadding:{values:"spacing"},scrollPaddingTop:{values:"spacing"},scrollPaddingBottom:{values:"spacing"},scrollPaddingLeft:{values:"spacing"},scrollPaddingRight:{values:"spacing"},scrollPaddingInline:{values:"spacing",shorthand:["scrollPaddingX"]},scrollPaddingBlock:{values:"spacing",shorthand:["scrollPaddingY"]},scrollSnapType:{values:{none:"none",x:"x var(--scroll-snap-strictness)",y:"y var(--scroll-snap-strictness)",both:"both var(--scroll-snap-strictness)"}},scrollSnapStrictness:{values:["mandatory","proximity"],transform:e=>({"--scroll-snap-strictness":e})},scrollSnapMargin:{values:"spacing"},scrollSnapMarginTop:{values:"spacing"},scrollSnapMarginBottom:{values:"spacing"},scrollSnapMarginLeft:{values:"spacing"},scrollSnapMarginRight:{values:"spacing"},listStylePosition:{shorthand:["listStylePos"]},listStyleImage:{values:"assets",shorthand:["listStyleImg"]},position:{shorthand:["pos"]},zIndex:{values:"zIndex"},inset:{values:"spacing"},insetInline:{values:"spacing",shorthand:["insetX"]},insetBlock:{values:"spacing",shorthand:["insetY"]},top:{values:"spacing"},insetBlockStart:{values:"spacing"},bottom:{values:"spacing"},insetBlockEnd:{values:"spacing"},left:{values:"spacing"},right:{values:"spacing"},insetInlineStart:{values:"spacing",shorthand:["insetStart"]},insetInlineEnd:{values:"spacing",shorthand:["insetEnd"]},ring:{transform(e){return{"--ring-offset-shadow":"var(--ring-inset) 0 0 0 var(--ring-offset-width) var(--ring-offset-color)","--ring-shadow":"var(--ring-inset) 0 0 0 calc(var(--ring-width) + var(--ring-offset-width)) var(--ring-color)","--ring-width":e,boxShadow:"var(--ring-offset-shadow), var(--ring-shadow), var(--shadow, 0 0 #0000)"}}},ringColor:{values:ne,transform:te("--ring-color")},ringOffset:{transform:e=>({"--ring-offset-width":e})},ringOffsetColor:{values:ne,transform:te("--ring-offset-color")},ringInset:{transform:e=>({"--ring-inset":e})},margin:{values:"spacing",shorthand:["m"]},marginTop:{values:"spacing",shorthand:["mt"]},marginBlockStart:{values:"spacing"},marginRight:{values:"spacing",shorthand:["mr"]},marginBottom:{values:"spacing",shorthand:["mb"]},marginBlockEnd:{values:"spacing"},marginLeft:{values:"spacing",shorthand:["ml"]},marginInlineStart:{values:"spacing",shorthand:["ms","marginStart"]},marginInlineEnd:{values:"spacing",shorthand:["me","marginEnd"]},marginInline:{values:"spacing",shorthand:["mx","marginX"]},marginBlock:{values:"spacing",shorthand:["my","marginY"]},padding:{values:"spacing",shorthand:["p"]},paddingTop:{values:"spacing",shorthand:["pt"]},paddingRight:{values:"spacing",shorthand:["pr"]},paddingBottom:{values:"spacing",shorthand:["pb"]},paddingBlockStart:{values:"spacing"},paddingBlockEnd:{values:"spacing"},paddingLeft:{values:"spacing",shorthand:["pl"]},paddingInlineStart:{values:"spacing",shorthand:["ps","paddingStart"]},paddingInlineEnd:{values:"spacing",shorthand:["pe","paddingEnd"]},paddingInline:{values:"spacing",shorthand:["px","paddingX"]},paddingBlock:{values:"spacing",shorthand:["py","paddingY"]},textDecoration:{shorthand:["textDecor"]},textDecorationColor:{values:ne,transform:te("textDecorationColor")},textShadow:{values:"shadows"},transform:{transform:e=>{let t=e;return e==="auto"&&(t="translateX(var(--translate-x, 0)) translateY(var(--translate-y, 0)) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),e==="auto-gpu"&&(t="translate3d(var(--translate-x, 0), var(--translate-y, 0), 0) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),{transform:t}}},skewX:{transform:e=>({"--skew-x":hn(e)})},skewY:{transform:e=>({"--skew-y":hn(e)})},scaleX:{transform:e=>({"--scale-x":e})},scaleY:{transform:e=>({"--scale-y":e})},scale:{transform(e){return e!=="auto"?{scale:e}:{scale:"var(--scale-x, 1) var(--scale-y, 1)"}}},spaceXReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":e?"1":void 0}}}},spaceX:{property:"marginInlineStart",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":"0",marginInlineStart:`calc(${e} * calc(1 - var(--space-x-reverse)))`,marginInlineEnd:`calc(${e} * var(--space-x-reverse))`}})},spaceYReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":e?"1":void 0}}}},spaceY:{property:"marginTop",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":"0",marginTop:`calc(${e} * calc(1 - var(--space-y-reverse)))`,marginBottom:`calc(${e} * var(--space-y-reverse))`}})},rotate:{transform(e){return e!=="auto"?{rotate:hn(e)}:{rotate:"var(--rotate-x, 0) var(--rotate-y, 0) var(--rotate-z, 0)"}}},rotateX:{transform(e){return{"--rotate-x":hn(e)}}},rotateY:{transform(e){return{"--rotate-y":hn(e)}}},translate:{transform(e){return e!=="auto"?{translate:e}:{translate:"var(--translate-x) var(--translate-y)"}}},translateX:{values:"spacing",transform:e=>({"--translate-x":e})},translateY:{values:"spacing",transform:e=>({"--translate-y":e})},transition:{values:["all","common","colors","opacity","position","backgrounds","size","shadow","transform"],transform(e){switch(e){case"all":return St("all");case"position":return St("left, right, top, bottom, inset-inline, inset-block");case"colors":return St("color, background-color, border-color, text-decoration-color, fill, stroke");case"opacity":return St("opacity");case"shadow":return St("box-shadow");case"transform":return St("transform");case"size":return St("width, height");case"backgrounds":return St("background, background-color, background-image, background-position");case"common":return St("color, background-color, border-color, text-decoration-color, fill, stroke, opacity, box-shadow, transform, filter, backdrop-filter");default:return{transition:e}}}},transitionDuration:{values:"durations"},transitionProperty:{values:{common:"background-color, border-color, color, fill, stroke, opacity, box-shadow, translate, transform",colors:"background-color, border-color, color, fill, stroke",size:"width, height",position:"left, right, top, bottom, inset-inline, inset-block",background:"background, background-color, background-image, background-position"}},transitionTimingFunction:{values:"easings"},animation:{values:"animations"},animationDuration:{values:"durations"},animationDelay:{values:"durations"},animationTimingFunction:{values:"easings"},fontFamily:{values:"fonts"},fontSize:{values:"fontSizes"},fontWeight:{values:"fontWeights"},lineHeight:{values:"lineHeights"},letterSpacing:{values:"letterSpacings"},textIndent:{values:"spacing"},truncate:{values:{type:"boolean"},transform(e){return e===!0?{overflow:"hidden",textOverflow:"ellipsis",whiteSpace:"nowrap"}:{}}},lineClamp:{transform(e){return e==="none"?{WebkitLineClamp:"unset"}:{overflow:"hidden",display:"-webkit-box",WebkitLineClamp:e,WebkitBoxOrient:"vertical",textWrap:"wrap"}}},borderSpacing:{values:e=>({...e("spacing"),auto:"var(--border-spacing-x, 0) var(--border-spacing-y, 0)"})},borderSpacingX:{values:"spacing",transform(e){return{"--border-spacing-x":e}}},borderSpacingY:{values:"spacing",transform(e){return{"--border-spacing-y":e}}},srOnly:{values:{type:"boolean"},transform(e){return tC[e]||{}}},debug:{values:{type:"boolean"},transform(e){return e?{outline:"1px solid blue !important","& > *":{outline:"1px solid red !important"}}:{}}},caretColor:{values:ne,transform:te("caretColor")},cursor:{values:"cursor"}}}),tC={true:{position:"absolute",width:"1px",height:"1px",padding:"0",margin:"-1px",overflow:"hidden",clip:"rect(0, 0, 0, 0)",whiteSpace:"nowrap",borderWidth:"0"},false:{position:"static",width:"auto",height:"auto",padding:"0",margin:"0",overflow:"visible",clip:"auto",whiteSpace:"normal"}};var nC="",rC=nC.split(","),oC="WebkitAppearance,WebkitBorderBefore,WebkitBorderBeforeColor,WebkitBorderBeforeStyle,WebkitBorderBeforeWidth,WebkitBoxReflect,WebkitLineClamp,WebkitMask,WebkitMaskAttachment,WebkitMaskClip,WebkitMaskComposite,WebkitMaskImage,WebkitMaskOrigin,WebkitMaskPosition,WebkitMaskPositionX,WebkitMaskPositionY,WebkitMaskRepeat,WebkitMaskRepeatX,WebkitMaskRepeatY,WebkitMaskSize,WebkitOverflowScrolling,WebkitTapHighlightColor,WebkitTextFillColor,WebkitTextStroke,WebkitTextStrokeColor,WebkitTextStrokeWidth,WebkitTouchCallout,WebkitUserModify,WebkitUserSelect,accentColor,alignContent,alignItems,alignSelf,alignTracks,all,anchorName,anchorScope,animation,animationComposition,animationDelay,animationDirection,animationDuration,animationFillMode,animationIterationCount,animationName,animationPlayState,animationRange,animationRangeEnd,animationRangeStart,animationTimeline,animationTimingFunction,appearance,aspectRatio,backdropFilter,backfaceVisibility,background,backgroundAttachment,backgroundBlendMode,backgroundClip,backgroundColor,backgroundImage,backgroundOrigin,backgroundPosition,backgroundPositionX,backgroundPositionY,backgroundRepeat,backgroundSize,blockSize,border,borderBlock,borderBlockColor,borderBlockEnd,borderBlockEndColor,borderBlockEndStyle,borderBlockEndWidth,borderBlockStart,borderBlockStartColor,borderBlockStartStyle,borderBlockStartWidth,borderBlockStyle,borderBlockWidth,borderBottom,borderBottomColor,borderBottomLeftRadius,borderBottomRightRadius,borderBottomStyle,borderBottomWidth,borderCollapse,borderColor,borderEndEndRadius,borderEndStartRadius,borderImage,borderImageOutset,borderImageRepeat,borderImageSlice,borderImageSource,borderImageWidth,borderInline,borderInlineColor,borderInlineEnd,borderInlineEndColor,borderInlineEndStyle,borderInlineEndWidth,borderInlineStart,borderInlineStartColor,borderInlineStartStyle,borderInlineStartWidth,borderInlineStyle,borderInlineWidth,borderLeft,borderLeftColor,borderLeftStyle,borderLeftWidth,borderRadius,borderRight,borderRightColor,borderRightStyle,borderRightWidth,borderSpacing,borderStartEndRadius,borderStartStartRadius,borderStyle,borderTop,borderTopColor,borderTopLeftRadius,borderTopRightRadius,borderTopStyle,borderTopWidth,borderWidth,bottom,boxAlign,boxDecorationBreak,boxDirection,boxFlex,boxFlexGroup,boxLines,boxOrdinalGroup,boxOrient,boxPack,boxShadow,boxSizing,breakAfter,breakBefore,breakInside,captionSide,caret,caretColor,caretShape,clear,clip,clipPath,clipRule,color,colorInterpolationFilters,colorScheme,columnCount,columnFill,columnGap,columnRule,columnRuleColor,columnRuleStyle,columnRuleWidth,columnSpan,columnWidth,columns,contain,containIntrinsicBlockSize,containIntrinsicHeight,containIntrinsicInlineSize,containIntrinsicSize,containIntrinsicWidth,container,containerName,containerType,content,contentVisibility,counterIncrement,counterReset,counterSet,cursor,cx,cy,d,direction,display,dominantBaseline,emptyCells,fieldSizing,fill,fillOpacity,fillRule,filter,flex,flexBasis,flexDirection,flexFlow,flexGrow,flexShrink,flexWrap,float,floodColor,floodOpacity,font,fontFamily,fontFeatureSettings,fontKerning,fontLanguageOverride,fontOpticalSizing,fontPalette,fontSize,fontSizeAdjust,fontSmooth,fontStretch,fontStyle,fontSynthesis,fontSynthesisPosition,fontSynthesisSmallCaps,fontSynthesisStyle,fontSynthesisWeight,fontVariant,fontVariantAlternates,fontVariantCaps,fontVariantEastAsian,fontVariantEmoji,fontVariantLigatures,fontVariantNumeric,fontVariantPosition,fontVariationSettings,fontWeight,forcedColorAdjust,gap,grid,gridArea,gridAutoColumns,gridAutoFlow,gridAutoRows,gridColumn,gridColumnEnd,gridColumnGap,gridColumnStart,gridGap,gridRow,gridRowEnd,gridRowGap,gridRowStart,gridTemplate,gridTemplateAreas,gridTemplateColumns,gridTemplateRows,hangingPunctuation,height,hyphenateCharacter,hyphenateLimitChars,hyphens,imageOrientation,imageRendering,imageResolution,imeMode,initialLetter,initialLetterAlign,inlineSize,inset,insetBlock,insetBlockEnd,insetBlockStart,insetInline,insetInlineEnd,insetInlineStart,interpolateSize,isolation,justifyContent,justifyItems,justifySelf,justifyTracks,left,letterSpacing,lightingColor,lineBreak,lineClamp,lineHeight,lineHeightStep,listStyle,listStyleImage,listStylePosition,listStyleType,margin,marginBlock,marginBlockEnd,marginBlockStart,marginBottom,marginInline,marginInlineEnd,marginInlineStart,marginLeft,marginRight,marginTop,marginTrim,marker,markerEnd,markerMid,markerStart,mask,maskBorder,maskBorderMode,maskBorderOutset,maskBorderRepeat,maskBorderSlice,maskBorderSource,maskBorderWidth,maskClip,maskComposite,maskImage,maskMode,maskOrigin,maskPosition,maskRepeat,maskSize,maskType,masonryAutoFlow,mathDepth,mathShift,mathStyle,maxBlockSize,maxHeight,maxInlineSize,maxLines,maxWidth,minBlockSize,minHeight,minInlineSize,minWidth,mixBlendMode,objectFit,objectPosition,offset,offsetAnchor,offsetDistance,offsetPath,offsetPosition,offsetRotate,opacity,order,orphans,outline,outlineColor,outlineOffset,outlineStyle,outlineWidth,overflow,overflowAnchor,overflowBlock,overflowClipBox,overflowClipMargin,overflowInline,overflowWrap,overflowX,overflowY,overlay,overscrollBehavior,overscrollBehaviorBlock,overscrollBehaviorInline,overscrollBehaviorX,overscrollBehaviorY,padding,paddingBlock,paddingBlockEnd,paddingBlockStart,paddingBottom,paddingInline,paddingInlineEnd,paddingInlineStart,paddingLeft,paddingRight,paddingTop,page,pageBreakAfter,pageBreakBefore,pageBreakInside,paintOrder,perspective,perspectiveOrigin,placeContent,placeItems,placeSelf,pointerEvents,position,positionAnchor,positionArea,positionTry,positionTryFallbacks,positionTryOrder,positionVisibility,printColorAdjust,quotes,r,resize,right,rotate,rowGap,rubyAlign,rubyMerge,rubyPosition,rx,ry,scale,scrollBehavior,scrollMargin,scrollMarginBlock,scrollMarginBlockEnd,scrollMarginBlockStart,scrollMarginBottom,scrollMarginInline,scrollMarginInlineEnd,scrollMarginInlineStart,scrollMarginLeft,scrollMarginRight,scrollMarginTop,scrollPadding,scrollPaddingBlock,scrollPaddingBlockEnd,scrollPaddingBlockStart,scrollPaddingBottom,scrollPaddingInline,scrollPaddingInlineEnd,scrollPaddingInlineStart,scrollPaddingLeft,scrollPaddingRight,scrollPaddingTop,scrollSnapAlign,scrollSnapCoordinate,scrollSnapDestination,scrollSnapPointsX,scrollSnapPointsY,scrollSnapStop,scrollSnapType,scrollSnapTypeX,scrollSnapTypeY,scrollTimeline,scrollTimelineAxis,scrollTimelineName,scrollbarColor,scrollbarGutter,scrollbarWidth,shapeImageThreshold,shapeMargin,shapeOutside,shapeRendering,stopColor,stopOpacity,stroke,strokeDasharray,strokeDashoffset,strokeLinecap,strokeLinejoin,strokeMiterlimit,strokeOpacity,strokeWidth,tabSize,tableLayout,textAlign,textAlignLast,textAnchor,textBox,textBoxEdge,textBoxTrim,textCombineUpright,textDecoration,textDecorationColor,textDecorationLine,textDecorationSkip,textDecorationSkipInk,textDecorationStyle,textDecorationThickness,textEmphasis,textEmphasisColor,textEmphasisPosition,textEmphasisStyle,textIndent,textJustify,textOrientation,textOverflow,textRendering,textShadow,textSizeAdjust,textSpacingTrim,textTransform,textUnderlineOffset,textUnderlinePosition,textWrap,textWrapMode,textWrapStyle,timelineScope,top,touchAction,transform,transformBox,transformOrigin,transformStyle,transition,transitionBehavior,transitionDelay,transitionDuration,transitionProperty,transitionTimingFunction,translate,unicodeBidi,userSelect,vectorEffect,verticalAlign,viewTimeline,viewTimelineAxis,viewTimelineInset,viewTimelineName,viewTransitionName,visibility,whiteSpace,whiteSpaceCollapse,widows,width,willChange,wordBreak,wordSpacing,wordWrap,writingMode,x,y,zIndex,zoom,alignmentBaseline,baselineShift,colorInterpolation,colorRendering,glyphOrientationVertical",iC=oC.split(",").concat(rC),sC=new Map(iC.map(e=>[e,!0]));function aC(e){const t=Object.create(null);return n=>(t[n]===void 0&&(t[n]=e(n)),t[n])}var lC=/&|@/,cC=aC(e=>sC.has(e)||e.startsWith("--")||lC.test(e));function oh(e,t){const n={};return gt(e,(r,o)=>{r&&(n[o.join(".")]=r.value)},{stop:t}),n}var Ca,ih;function uC(){if(ih)return Ca;ih=1,Ca=i,i.default=i,i.stable=c,i.stableStringify=c;var e="[...]",t="[Circular]",n=[],r=[];function o(){return{depthLimit:Number.MAX_SAFE_INTEGER,edgesLimit:Number.MAX_SAFE_INTEGER}}function i(d,m,f,p){typeof p>"u"&&(p=o()),a(d,"",0,[],void 0,0,p);var v;try{r.length===0?v=JSON.stringify(d,m,f):v=JSON.stringify(d,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var b=n.pop();b.length===4?Object.defineProperty(b[0],b[1],b[3]):b[0][b[1]]=b[2]}}return v}function s(d,m,f,p){var v=Object.getOwnPropertyDescriptor(p,f);v.get!==void 0?v.configurable?(Object.defineProperty(p,f,{value:d}),n.push([p,f,m,v])):r.push([m,f,d]):(p[f]=d,n.push([p,f,m]))}function a(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;xm?1:0}function c(d,m,f,p){typeof p>"u"&&(p=o());var v=u(d,"",0,[],void 0,0,p)||d,b;try{r.length===0?b=JSON.stringify(v,m,f):b=JSON.stringify(v,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var y=n.pop();y.length===4?Object.defineProperty(y[0],y[1],y[3]):y[0][y[1]]=y[2]}}return b}function u(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;x0)for(var p=0;p{const t=Object.create(null);function n(...r){const o=r.map(i=>hC(i)).join("|");return t[o]===void 0&&(t[o]=e(...r)),t[o]}return n},sh=16,li="px",wa="em",jr="rem";function ah(e=""){const t=new RegExp(String.raw`-?\d+(?:\.\d+|\d*)`),n=new RegExp(`${li}|${wa}|${jr}`);return e.match(new RegExp(`${t.source}(${n.source})`))?.[1]}function lh(e=""){if(typeof e=="number")return`${e}px`;const t=ah(e);if(!t||t===li)return e;if(t===wa||t===jr)return`${parseFloat(e)*sh}${li}`}function ch(e=""){const t=ah(e);if(!t||t===jr)return e;if(t===wa)return`${parseFloat(e)}${jr}`;if(t===li)return`${parseFloat(e)/sh}${jr}`}const fC=e=>e.charAt(0).toUpperCase()+e.slice(1);function gC(e){const t=pC(e),n=Object.fromEntries(t);function r(d){return n[d]}function o(d){return rr(r(d))}function i(){const d=Object.keys(n),m=mC(d),f=d.flatMap(p=>{const v=r(p),b=[`${p}Down`,rr({max:ci(v.min)})],y=[p,rr({min:v.min})],x=[`${p}Only`,o(p)];return[y,x,b]}).filter(([,p])=>p!=="").concat(m.map(([p,v])=>{const b=r(p),y=r(v);return[`${p}To${fC(v)}`,rr({min:b.min,max:ci(y.min)})]}));return Object.fromEntries(f)}function s(){const d=i();return Object.fromEntries(Object.entries(d))}const a=s(),l=d=>a[d];function c(){return["base",...Object.keys(n)]}function u(d){return rr({min:r(d).min})}function h(d){return rr({max:ci(r(d).min)})}return{values:Object.values(n),only:o,keys:c,conditions:a,getCondition:l,up:u,down:h}}function ci(e){const t=parseFloat(lh(e)??"")-.04;return ch(`${t}px`)}function pC(e){return Object.entries(e).sort(([,n],[,r])=>parseInt(n,10){let s=null;return o<=i.length-1&&(s=i[o+1]?.[1]),s!=null&&(s=ci(s)),[n,{name:n,min:ch(r),max:s}]})}function mC(e){const t=[];return e.forEach((n,r)=>{let o=r;o++;let i=e[o];for(;i;)t.push([n,i]),o++,i=e[o]}),t}function rr({min:e,max:t}){return e==null&&t==null?"":["@media screen",e&&`(min-width: ${e})`,t&&`(max-width: ${t})`].filter(Boolean).join(" and ")}const vC=(e,t)=>Object.fromEntries(Object.entries(e).map(([n,r])=>t(n,r))),bC=e=>{const{breakpoints:t,conditions:n={}}=e,r=vC(n,(u,h)=>[`_${u}`,h]),o=Object.assign({},r,t.conditions);function i(){return Object.keys(o)}function s(u){return i().includes(u)||/^@|&|&$/.test(u)||u.startsWith("_")}function a(u){return u.filter(h=>h!=="base").sort((h,d)=>{const m=s(h),f=s(d);return m&&!f?1:!m&&f?-1:0})}function l(u){return u.startsWith("@breakpoint")?t.getCondition(u.replace("@breakpoint ","")):u}function c(u){return Reflect.get(o,u)||u}return{keys:i,sort:a,has:s,resolve:c,breakpoints:t.keys(),expandAtRule:l}},uh=e=>({minMax:new RegExp(`(!?\\(\\s*min(-device-)?-${e})(.| )+\\(\\s*max(-device)?-${e}`,"i"),min:new RegExp(`\\(\\s*min(-device)?-${e}`,"i"),maxMin:new RegExp(`(!?\\(\\s*max(-device)?-${e})(.| -)+\\(\\s*min(-device)?-${e}`,"i"),max:new RegExp(`\\(\\s*max(-device)?-${e}`,"i")}),cC=th("width"),uC=th("height"),nh=e=>({isMin:lh(e.minMax,e.maxMin,e.min),isMax:lh(e.maxMin,e.minMax,e.max)}),{isMin:xa,isMax:rh}=nh(cC),{isMin:ka,isMax:oh}=nh(uC),ih=/print/i,sh=/^print$/i,dC=/(-?\d*\.?\d+)(ch|em|ex|px|rem)/,hC=/(\d)/,Mr=Number.MAX_VALUE,fC={ch:8.8984375,em:16,rem:16,ex:8.296875,px:1};function ah(e){const t=dC.exec(e)||(xa(e)||ka(e)?hC.exec(e):null);if(!t)return Mr;if(t[0]==="0")return 0;const n=parseFloat(t[1]),r=t[2];return n*(fC[r]||1)}function lh(e,t,n){return r=>e.test(r)||!t.test(r)&&n.test(r)}function gC(e,t){const n=ih.test(e),r=sh.test(e),o=ih.test(t),i=sh.test(t);return n&&o?!r&&i?1:r&&!i?-1:e.localeCompare(t):n?1:o?-1:null}const pC=rt((e,t)=>{const n=gC(e,t);if(n!==null)return n;const r=xa(e)||ka(e),o=rh(e)||oh(e),i=xa(t)||ka(t),s=rh(t)||oh(t);if(r&&s)return-1;if(o&&i)return 1;const a=ah(e),l=ah(t);return a===Mr&&l===Mr?e.localeCompare(t):a===Mr?1:l===Mr?-1:a!==l?a>l?o?-1:1:o?1:-1:e.localeCompare(t)});function ch(e){return e.sort(([t],[n])=>pC(t,n))}function uh(e){const t=[],n=[],r={};for(const[s,a]of Object.entries(e))s.startsWith("@media")?t.push([s,a]):s.startsWith("@container")?n.push([s,a]):_e(a)?r[s]=uh(a):r[s]=a;const o=ch(t),i=ch(n);return{...r,...Object.fromEntries(o),...Object.fromEntries(i)}}const dh=/\s*!(important)?/i,mC=e=>lt(e)?dh.test(e):!1,vC=e=>lt(e)?e.replace(dh,"").trim():e;function hh(e){const{transform:t,conditions:n,normalize:r}=e,o=xC(e);return rt(function(...s){const a=o(...s),l=r(a),c=Object.create(null);return gt(l,(u,d)=>{const h=mC(u);if(u==null)return;const[m,...g]=n.sort(d).map(n.resolve);h&&(u=vC(u));let p=t(m,u)??Object.create(null);p=gt(p,v=>lt(v)&&h?`${v} !important`:v,{getKey:v=>n.expandAtRule(v)}),bC(c,g.flat(),p)}),uh(c)})}function bC(e,t,n){let r=e;for(const o of t)o&&(r[o]||(r[o]=Object.create(null)),r=r[o]);Jn(r,n)}function yC(...e){return e.filter(t=>_e(t)&&Object.keys(vr(t)).length>0)}function xC(e){function t(n){const r=yC(...n);return r.length===1?r:r.map(o=>e.normalize(o))}return rt(function(...r){return Jn({},...t(r))})}const fh=e=>({base:{},variants:{},defaultVariants:{},compoundVariants:[],...e});function kC(e){const{css:t,conditions:n,normalize:r,layers:o}=e;function i(a={}){const{base:l,variants:c,defaultVariants:u,compoundVariants:d}=fh(a),h=hh({conditions:n,normalize:r,transform(x,b){return c[x]?.[b]}}),m=(x={})=>{const b=r({...u,...vr(x)});let C={...l};Jn(C,h(b));const k=s(d,b);return o.wrap("recipes",t(C,k))},g=Object.keys(c),p=x=>{const b=Bd(x,["recipe"]),[C,k]=Vn(b,g);return g.includes("colorPalette")||(C.colorPalette=x.colorPalette||u.colorPalette),g.includes("orientation")&&(k.orientation=x.orientation),[C,k]},v=Object.fromEntries(Object.entries(c).map(([x,b])=>[x,Object.keys(b)]));return Object.assign(x=>t(m(x)),{className:a.className,__cva__:!0,variantMap:v,variantKeys:g,raw:m,config:a,splitVariantProps:p,merge(x){return i(CC(e)(this,x))}})}function s(a,l){let c={};return a.forEach(u=>{Object.entries(u).every(([h,m])=>h==="css"?!0:(Array.isArray(m)?m:[m]).some(p=>l[h]===p))&&(c=t(c,u.css))}),c}return i}function CC(e){const{css:t}=e;return function(r,o){const i=fh(o.config),s=Vl(r.variantKeys,Object.keys(o.variants)),a=t(r.base,i.base),l=Object.fromEntries(s.map(h=>[h,t(r.config.variants[h],i.variants[h])])),c=Jn(r.config.defaultVariants,i.defaultVariants),u=[...r.compoundVariants,...i.compoundVariants];return{className:Ge(r.className,o.className),base:a,variants:l,defaultVariants:c,compoundVariants:u}}}const SC={reset:"reset",base:"base",tokens:"tokens",recipes:"recipes"},gh={reset:0,base:1,tokens:2,recipes:3};function wC(e){const t=e.layers??SC,r=Object.values(t).sort((o,i)=>gh[o]-gh[i]);return{names:r,atRule:`@layer ${r.join(", ")};`,wrap(o,i){return e.disableLayers?i:{[`@layer ${t[o]}`]:i}}}}function EC(e){const{utility:t,normalize:n}=e,{hasShorthand:r,resolveShorthand:o}=t;return function(i){return gt(i,n,{stop:s=>Array.isArray(s),getKey:r?o:void 0})}}function OC(e){const{preflight:t}=e;if(!t)return{};const{scope:n="",level:r="parent"}=_e(t)?t:{};let o="";n&&r==="parent"?o=`${n} `:n&&r==="element"&&(o=`&${n}`);const i={"*":{margin:"0px",padding:"0px",font:"inherit",wordWrap:"break-word",WebkitTapHighlightColor:"transparent"},"*, *::before, *::after, *::backdrop":{boxSizing:"border-box",borderWidth:"0px",borderStyle:"solid",borderColor:"var(--global-color-border, currentColor)"},hr:{height:"0px",color:"inherit",borderTopWidth:"1px"},body:{minHeight:"100dvh",position:"relative"},img:{borderStyle:"none"},"img, svg, video, canvas, audio, iframe, embed, object":{display:"block",verticalAlign:"middle"},iframe:{border:"none"},"img, video":{maxWidth:"100%",height:"auto"},"p, h1, h2, h3, h4, h5, h6":{overflowWrap:"break-word"},"ol, ul":{listStyle:"none"},"code, kbd, pre, samp":{fontSize:"1em"},"button, [type='button'], [type='reset'], [type='submit']":{WebkitAppearance:"button",backgroundColor:"transparent",backgroundImage:"none"},"button, input, optgroup, select, textarea":{color:"inherit"},"button, select":{textTransform:"none"},table:{textIndent:"0px",borderColor:"inherit",borderCollapse:"collapse"},"*::placeholder":{opacity:"unset",color:"#9ca3af",userSelect:"none"},textarea:{resize:"vertical"},summary:{display:"list-item"},small:{fontSize:"80%"},"sub, sup":{fontSize:"75%",lineHeight:0,position:"relative",verticalAlign:"baseline"},sub:{bottom:"-0.25em"},sup:{top:"-0.5em"},dialog:{padding:"0px"},a:{color:"inherit",textDecoration:"inherit"},"abbr:where([title])":{textDecoration:"underline dotted"},"b, strong":{fontWeight:"bolder"},"code, kbd, samp, pre":{fontSize:"1em","--font-mono-fallback":"ui-monospace, SFMono-Regular, Menlo, Monaco, Consolas, 'Liberation Mono', 'Courier New'",fontFamily:"var(--global-font-mono, var(--font-mono-fallback))"},'input[type="text"], input[type="email"], input[type="search"], input[type="password"]':{WebkitAppearance:"none",MozAppearance:"none"},"input[type='search']":{WebkitAppearance:"textfield",outlineOffset:"-2px"},"::-webkit-search-decoration, ::-webkit-search-cancel-button":{WebkitAppearance:"none"},"::-webkit-file-upload-button":{WebkitAppearance:"button",font:"inherit"},'input[type="number"]::-webkit-inner-spin-button, input[type="number"]::-webkit-outer-spin-button':{height:"auto"},"input[type='number']":{MozAppearance:"textfield"},":-moz-ui-invalid":{boxShadow:"none"},":-moz-focusring":{outline:"auto"},"[hidden]:where(:not([hidden='until-found']))":{display:"none !important"}},s={[n||"html"]:{lineHeight:1.5,"--font-fallback":"ui-sans-serif, system-ui, -apple-system, BlinkMacSystemFont, 'Segoe UI', Roboto, 'Helvetica Neue', Arial, 'Noto Sans', sans-serif, 'Apple Color Emoji', 'Segoe UI Emoji', 'Segoe UI Symbol', 'Noto Color Emoji'",WebkitTextSizeAdjust:"100%",WebkitFontSmoothing:"antialiased",MozOsxFontSmoothing:"grayscale",textRendering:"optimizeLegibility",touchAction:"manipulation",MozTabSize:"4",tabSize:"4",fontFamily:"var(--global-font-body, var(--font-fallback))"}};if(r==="element"){const a=Object.entries(i).reduce((l,[c,u])=>(l[c]={[o]:u},l),{});Object.assign(s,a)}else o?s[o]=i:Object.assign(s,i);return s}function PC(e){const{conditions:t,isValidProperty:n}=e;return function(o){return gt(o,i=>i,{getKey:(i,s)=>_e(s)&&!t.has(i)&&!n(i)?IC(i).map(a=>"&"+a).join(", "):i})}}function IC(e){const t=[];let n=0,r="",o=!1;for(let i=0;i{const t=o=>({base:e.base?.[o]??{},variants:{},defaultVariants:e.defaultVariants??{},compoundVariants:e.compoundVariants?TC(e.compoundVariants,o):[]}),r=(e.slots??[]).map(o=>[o,t(o)]);for(const[o,i]of Object.entries(e.variants??{}))for(const[s,a]of Object.entries(i))r.forEach(([l,c])=>{var u;(u=c.variants)[o]??(u[o]={}),c.variants[o][s]=a[l]??{}});return Object.fromEntries(r)},TC=(e,t)=>e.filter(n=>n.css[t]).map(n=>({...n,css:n.css[t]}));function NC(e){const{cva:t}=e;return function(r={}){const o=Object.entries(RC(r)).map(([d,h])=>[d,t(h)]);function i(d){const h=o.map(([m,g])=>[m,g(d)]);return Object.fromEntries(h)}const s=r.variants??{},a=Object.keys(s);function l(d){const h=Bd(d,["recipe"]),[m,g]=Vn(h,a);return a.includes("colorPalette")||(m.colorPalette=d.colorPalette||r.defaultVariants?.colorPalette),a.includes("orientation")&&(g.orientation=d.orientation),[m,g]}const c=Object.fromEntries(Object.entries(s).map(([d,h])=>[d,Object.keys(h)]));let u={};return r.className&&(u=Object.fromEntries(r.slots.map(d=>[d,`${r.className}__${d}`]))),Object.assign(i,{variantMap:c,variantKeys:a,splitVariantProps:l,classNameMap:u})}}const AC=()=>e=>Array.from(new Set(e)),_C=/([\0-\x1f\x7f]|^-?\d)|^-$|^-|[^\x80-\uFFFF\w-]/g,VC=function(e,t){return t?e==="\0"?"�":e==="-"&&e.length===1?"\\-":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16):"\\"+e},ph=e=>(e+"").replace(_C,VC),mh=(e,t)=>{let n="",r=0,o="char",i="",s="";const a=[];for(;r{let t=0;const n=["("];for(;t{n instanceof Map?t[r]=Object.fromEntries(n):t[r]=n}),t}const bh=/({([^}]*)})/g,LC=/[{}]/g,DC=/\w+\.\w+/,yh=e=>{if(!lt(e))return[];const t=e.match(bh);return t?t.map(n=>n.replace(LC,"")).map(n=>n.trim()):[]},zC=e=>bh.test(e);function xh(e){if(!e.extensions?.references)return e.extensions?.cssVar?.ref??e.value;const t=e.extensions.references??{};return e.value=Object.keys(t).reduce((n,r)=>{const o=t[r];if(o.extensions.conditions)return n;const i=xh(o);return n.replace(`{${r}}`,i)},e.value),delete e.extensions.references,e.value}function kh(e){return _e(e)&&e.reference?e.reference:String(e)}const ai=(e,...t)=>t.map(kh).join(` ${e} `).replace(/calc/g,""),Ch=(...e)=>`calc(${ai("+",...e)})`,Sh=(...e)=>`calc(${ai("-",...e)})`,Ca=(...e)=>`calc(${ai("*",...e)})`,wh=(...e)=>`calc(${ai("/",...e)})`,Eh=e=>{const t=kh(e);return t!=null&&!Number.isNaN(parseFloat(t))?String(t).startsWith("-")?String(t).slice(1):`-${t}`:Ca(t,-1)},nr=Object.assign(e=>({add:(...t)=>nr(Ch(e,...t)),subtract:(...t)=>nr(Sh(e,...t)),multiply:(...t)=>nr(Ca(e,...t)),divide:(...t)=>nr(wh(e,...t)),negate:()=>nr(Eh(e)),toString:()=>e.toString()}),{add:Ch,subtract:Sh,multiply:Ca,divide:wh,negate:Eh}),MC={enforce:"pre",transform(e){const{prefix:t,allTokens:n,formatCssVar:r,formatTokenName:o,registerToken:i}=e;n.filter(({extensions:a})=>a.category==="spacing").forEach(a=>{const l=a.path.slice(),c=r(l,t);if(lt(a.value)&&a.value==="0rem")return;const u=structuredClone(a);Object.assign(u.extensions,{negative:!0,prop:`-${a.extensions.prop}`,originalPath:l}),u.value=nr.negate(c.ref);const d=u.path[u.path.length-1];d!=null&&(u.path[u.path.length-1]=`-${d}`),u.path&&(u.name=o(u.path)),i(u)})}},jC=new Set(["spacing","sizes","borderWidths","fontSizes","radii"]),$C=[MC,{enforce:"post",transform(e){const{allTokens:t,registerToken:n,formatTokenName:r}=e,o=t.filter(({extensions:a})=>a.category==="colors"),i=new Map,s=new Map;o.forEach(a=>{const{colorPalette:l}=a.extensions;l&&(l.keys.forEach(c=>{i.set(r(c),c)}),l.roots.forEach(c=>{const u=r(c),d=s.get(u)||[];if(d.push(a),s.set(u,d),a.extensions.default&&c.length===1){const h=l.keys[0]?.filter(Boolean);if(!h.length)return;const m=c.concat(h);i.set(r(m),[])}}))}),i.forEach(a=>{const l=["colors","colorPalette",...a].filter(Boolean),c=r(l),u=r(l.slice(1));n({name:c,value:c,originalValue:c,path:l,extensions:{condition:"base",originalPath:l,category:"colors",prop:u,virtual:!0}},"pre")})}},{enforce:"post",transform(e){e.allTokens.filter(n=>jC.has(n.extensions.category)&&!n.extensions.negative).forEach(n=>{Object.assign(n.extensions,{pixelValue:Zd(n.value)})})}},{enforce:"post",transform(e){e.allTokens=e.allTokens.filter(t=>t.value!=="")}}],BC=[{type:"extensions",enforce:"pre",name:"tokens/css-var",transform(e,t){const{prefix:n,formatCssVar:r}=t,{negative:o,originalPath:i}=e.extensions,s=o?i:e.path;return{cssVar:r(s.filter(Boolean),n)}}},{enforce:"post",type:"value",name:"tokens/conditionals",transform(e,t){const{prefix:n,formatCssVar:r}=t,o=yh(e.value);return o.length&&o.forEach(i=>{const s=r(i.split("."),n);e.value=e.value.replace(`{${s.ref}}`,s)}),e.value}},{type:"extensions",enforce:"pre",name:"tokens/colors/colorPalette",match(e){return e.extensions.category==="colors"&&!e.extensions.virtual},transform(e,t){let n=e.path.slice();if(n.pop(),n.shift(),n.length===0){const a=[...e.path];a.shift(),n=a}if(n.length===0)return{};const r=n.reduce((a,l,c,u)=>{const d=u.slice(0,c+1);return a.push(d),a},[]),o=n[0],i=t.formatTokenName(n),s=e.path.slice(e.path.indexOf(o)+1).reduce((a,l,c,u)=>(a.push(u.slice(c)),a),[]);return s.length===0&&s.push([""]),{colorPalette:{value:i,roots:r,keys:s}}}}],Oh=e=>_e(e)&&Object.prototype.hasOwnProperty.call(e,"value");function WC(e){return e?{breakpoints:Wd(e,t=>({value:t})),sizes:Object.fromEntries(Object.entries(e).map(([t,n])=>[`breakpoint-${t}`,{value:n}]))}:{breakpoints:{},sizes:{}}}function HC(e){const{prefix:t="",tokens:n={},semanticTokens:r={},breakpoints:o={}}=e,i=L=>L.join("."),s=(L,F)=>Gd(L.join("-"),{prefix:F}),a=[],l=new Map,c=new Map,u=new Map,d=new Map,h=new Map,m=new Map,g=new Map,p=new Map,v=[];function y(L,F){a.push(L),l.set(L.name,L),F&&p.forEach(K=>{K.enforce===F&&q(K,L)})}const x=WC(o),b=vr({...n,breakpoints:x.breakpoints,sizes:{...n.sizes,...x.sizes}});function C(){gt(b,(L,F)=>{const K=F.includes("DEFAULT");F=Ph(F);const ie=F[0],me=i(F),Oe=lt(L)?{value:L}:L,Rt={value:Oe.value,originalValue:Oe.value,name:me,path:F,extensions:{condition:"base",originalPath:F,category:ie,prop:i(F.slice(1))}};K&&(Rt.extensions.default=!0),y(Rt)},{stop:Oh}),gt(r,(L,F)=>{const K=F.includes("DEFAULT");F=Ih(Ph(F));const ie=F[0],me=i(F),Oe=lt(L.value)?{value:{base:L.value}}:L,Rt={value:Oe.value.base||"",originalValue:Oe.value.base||"",name:me,path:F,extensions:{originalPath:F,category:ie,conditions:Oe.value,condition:"base",prop:i(F.slice(1))}};K&&(Rt.extensions.default=!0),y(Rt)},{stop:Oh})}function k(L){return l.get(L)}function E(L){const{condition:F}=L.extensions;F&&(c.has(F)||c.set(F,new Set),c.get(F).add(L))}function I(L){const{category:F,prop:K}=L.extensions;F&&(g.has(F)||g.set(F,new Map),g.get(F).set(K,L))}function R(L){const{condition:F,negative:K,virtual:ie,cssVar:me}=L.extensions;K||ie||!F||!me||(u.has(F)||u.set(F,new Map),u.get(F).set(me.var,L.value))}function _(L){const{category:F,prop:K,cssVar:ie,negative:me}=L.extensions;if(!F)return;m.has(F)||m.set(F,new Map);const Oe=me?L.extensions.conditions?L.originalValue:L.value:ie.ref;m.get(F).set(K,Oe),h.set([F,K].join("."),Oe)}function A(L){const{colorPalette:F,virtual:K,default:ie}=L.extensions;!F||K||F.roots.forEach(me=>{const Oe=i(me);d.has(Oe)||d.set(Oe,new Map);const Rt=GC([...L.path],[...me]),Mi=i(Rt),il=k(Mi);if(!il||!il.extensions.cssVar)return;const{var:$R}=il.extensions.cssVar;if(d.get(Oe).set($R,L.extensions.cssVar.ref),ie&&me.length===1){const BR=i(["colors","colorPalette"]),lp=k(BR);if(!lp)return;const WR=i(L.path),cp=k(WR);if(!cp)return;const up=F.keys[0]?.filter(Boolean);if(!up.length)return;const sl=i(me.concat(up));d.has(sl)||d.set(sl,new Map),d.get(sl).set(lp.extensions.cssVar.var,cp.extensions.cssVar.ref)}})}let T={};function S(){a.forEach(L=>{E(L),I(L),R(L),_(L),A(L)}),T=vh(m)}const N=(L,F)=>{if(!L||typeof L!="string")return{invalid:!0,value:L};const[K,ie]=L.split("/");if(!K||!ie)return{invalid:!0,value:K};const me=F(K),Oe=k(`opacity.${ie}`)?.value;if(!Oe&&isNaN(Number(ie)))return{invalid:!0,value:K};const Rt=Oe?Number(Oe)*100+"%":`${ie}%`,Mi=me??K;return{invalid:!1,color:Mi,value:`color-mix(in srgb, ${Mi} ${Rt}, transparent)`}},D=rt((L,F)=>h.get(L)??F),z=rt(L=>T[L]||null),W=rt(L=>mh(L,F=>{if(!F)return;if(F.includes("/")){const ie=N(F,me=>D(me));if(ie.invalid)throw new Error("Invalid color mix at "+F+": "+ie.value);return ie.value}const K=D(F);return K||(DC.test(F)?ph(F):F)})),J={prefix:t,allTokens:a,tokenMap:l,registerToken:y,getByName:k,formatTokenName:i,formatCssVar:s,flatMap:h,cssVarMap:u,categoryMap:g,colorPaletteMap:d,getVar:D,getCategoryValues:z,expandReferenceInValue:W};function ee(...L){L.forEach(F=>{p.set(F.name,F)})}function $(...L){v.push(...L)}function q(L,F){if(F.extensions.references||is(L.match)&&!L.match(F))return;const ie=(me=>L.transform(me,J))(F);switch(!0){case L.type==="extensions":Object.assign(F.extensions,ie);break;case L.type==="value":F.value=ie;break;default:F[L.type]=ie;break}}function Z(L){v.forEach(F=>{F.enforce===L&&F.transform(J)})}function pe(L){p.forEach(F=>{F.enforce===L&&a.forEach(K=>{q(F,K)})})}function Je(){a.forEach(L=>{const F=UC(L);!F||F.length===0||F.forEach(K=>{y(K)})})}function vt(L){return yh(L).map(K=>k(K)).filter(Boolean)}function It(){a.forEach(L=>{if(!zC(L.value))return;const F=vt(L.value);L.extensions.references=F.reduce((K,ie)=>(K[ie.name]=ie,K),{})})}function ol(){a.forEach(L=>{xh(L)})}function jR(){Z("pre"),pe("pre"),Je(),It(),ol(),Z("post"),pe("post"),S()}return C(),ee(...BC),$(...$C),jR(),J}function Ph(e){return e[0]==="DEFAULT"?e:e.filter(t=>t!=="DEFAULT")}function Ih(e){return e.filter(t=>t!=="base")}function UC(e){if(!e.extensions.conditions)return;const{conditions:t}=e.extensions,n=[];return gt(t,(r,o)=>{const i=Ih(o);if(!i.length)return;const s=structuredClone(e);s.value=r,s.extensions.condition=i.join(":"),n.push(s)}),n}function GC(e,t){const n=e.findIndex((r,o)=>t.every((i,s)=>e[o+s]===i));return n===-1||(e.splice(n,t.length),e.splice(n,0,"colorPalette")),e}AC()(["aspectRatios","zIndex","opacity","colors","fonts","fontSizes","fontWeights","lineHeights","letterSpacings","sizes","shadows","spacing","radii","cursor","borders","borderWidths","borderStyles","durations","easings","animations","blurs","gradients","breakpoints","assets"]);function oT(e){return e}function qC(e){return Object.fromEntries(Object.entries(e).map(([t,n])=>[t,n]))}function KC(e){const t=qC(e.config),n=e.tokens,r=new Map,o=new Map;function i(R,_){t[R]=_,s(R,_)}const s=(R,_)=>{const A=p(_);A&&(o.set(R,A),d(R,_))},a=()=>{for(const[R,_]of Object.entries(t))_&&s(R,_)},l=()=>{for(const[R,_]of Object.entries(t)){const{shorthand:A}=_??{};if(!A)continue;(Array.isArray(A)?A:[A]).forEach(S=>r.set(S,R))}},c=()=>{const R=vh(n.colorPaletteMap);i("colorPalette",{values:Object.keys(R),transform:rt(_=>R[_])})},u=new Map,d=(R,_)=>{if(!_)return;const A=p(_,S=>`type:Tokens["${S}"]`);if(typeof A=="object"&&A.type){u.set(R,new Set([`type:${A.type}`]));return}if(A){const S=new Set(Object.keys(A));u.set(R,S)}const T=u.get(R)??new Set;_.property&&u.set(R,T.add(`CssProperties["${_.property}"]`))},h=()=>{for(const[R,_]of Object.entries(t))_&&d(R,_)},m=(R,_)=>{const A=u.get(R)??new Set;u.set(R,new Set([...A,..._]))},g=()=>{const R=new Map;for(const[_,A]of u.entries()){if(A.size===0){R.set(_,["string"]);continue}const T=Array.from(A).map(S=>S.startsWith("CssProperties")?S:S.startsWith("type:")?S.replace("type:",""):JSON.stringify(S));R.set(_,T)}return R},p=(R,_)=>{const{values:A}=R,T=S=>{const N=_?.(S);return N?{[N]:N}:void 0};return lt(A)?T?.(A)??n.getCategoryValues(A)??{}:Array.isArray(A)?A.reduce((S,N)=>(S[N]=N,S),{}):is(A)?A(_?T:n.getCategoryValues):A},v=rt((R,_)=>({[R]:R.startsWith("--")?n.getVar(_,_):_})),y=Object.assign(n.getVar,{raw:R=>n.getByName(R)}),x=rt((R,_)=>{const A=k(R);lt(_)&&!_.includes("_EMO_")&&(_=n.expandReferenceInValue(_));const T=t[A];if(!T)return v(A,_);const S=o.get(A)?.[_];if(!T.transform)return v(R,S??_);const N=D=>Rk(D,y);return T.transform(S??_,{raw:_,token:y,utils:{colorMix:N}})});function b(){l(),c(),a(),h()}b();const C=r.size>0,k=rt(R=>r.get(R)??R);return{keys:()=>[...Array.from(r.keys()),...Object.keys(t)],hasShorthand:C,transform:x,shorthands:r,resolveShorthand:k,register:i,getTypes:g,addPropertyType:m}}const De={};function Rh(...e){const t=Hd(...e),{theme:n={},utilities:r={},globalCss:o={},cssVarsRoot:i=":where(:root, :host)",cssVarsPrefix:s="chakra",preflight:a}=t,l=wC(t),c=HC({breakpoints:n.breakpoints,tokens:n.tokens,semanticTokens:n.semanticTokens,prefix:s}),u=oC(n.breakpoints??De),d=lC({conditions:t.conditions??De,breakpoints:u}),h=KC({config:r,tokens:c});function m(){const{textStyles:$,layerStyles:q,animationStyles:Z}=n,pe=vr({textStyle:$,layerStyle:q,animationStyle:Z});for(const[Je,vt]of Object.entries(pe)){const It=Yd(vt??De,Th);h.register(Je,{values:Object.keys(It),transform(ol){return b(It[ol])}})}}m(),h.addPropertyType("animationName",Object.keys(n.keyframes??De));const g=new Set(["css",...h.keys(),...d.keys()]),p=rt($=>g.has($)||Zk($)),v=$=>Array.isArray($)?$.reduce((q,Z,pe)=>{const Je=d.breakpoints[pe];return Z!=null&&(q[Je]=Z),q},{}):$,y=EC({utility:h,normalize:v}),x=PC({conditions:d,isValidProperty:p}),b=hh({transform:h.transform,conditions:d,normalize:y}),C=kC({css:b,conditions:d,normalize:y,layers:l}),k=NC({cva:C});function E(){const $={};for(const[q,Z]of c.cssVarMap.entries()){const pe=Object.fromEntries(Z);if(Object.keys(pe).length===0)continue;const Je=q==="base"?i:d.resolve(q),vt=Je.startsWith("@"),It=b(x({[Je]:vt?{[i]:pe}:pe}));Jn($,It)}return l.wrap("tokens",$)}function I(){const $=Object.fromEntries(Object.entries(n.keyframes??De).map(([Z,pe])=>[`@keyframes ${Z}`,pe])),q=Object.assign({},$,b(x(o)));return l.wrap("base",q)}function R($){return Vn($,p)}function _(){const $=OC({preflight:a});return l.wrap("reset",$)}const A=YC(c),T=($,q)=>A.get($)?.value||q;T.var=($,q)=>A.get($)?.variable||q;function S($,q){return n.recipes?.[$]??q}function N($,q){return n.slotRecipes?.[$]??q}function D($){return Object.hasOwnProperty.call(n.recipes??De,$)}function z($){return Object.hasOwnProperty.call(n.slotRecipes??De,$)}function W($){return D($)||z($)}const J=[_(),I(),E()],ee={layerStyles:Sa(n.layerStyles??De),textStyles:Sa(n.textStyles??De),animationStyles:Sa(n.animationStyles??De),tokens:Nh(c,Object.keys(n.tokens??De),($,q)=>!$.extensions.conditions&&!q.includes("colorPalette")),semanticTokens:Nh(c,Object.keys(n.semanticTokens??De),$=>!!$.extensions.conditions),keyframes:Ah(n.keyframes??De),breakpoints:Ah(n.breakpoints??De)};return{$$chakra:!0,_config:t,_global:J,breakpoints:u,tokens:c,conditions:d,utility:h,token:T,properties:g,layers:l,isValidProperty:p,splitCssProps:R,normalizeValue:v,getTokenCss:E,getGlobalCss:I,getPreflightCss:_,css:b,cva:C,sva:k,getRecipe:S,getSlotRecipe:N,hasRecipe:W,isRecipe:D,isSlotRecipe:z,query:ee}}function YC(e){const t=new Map;return e.allTokens.forEach(n=>{const{cssVar:r,virtual:o,conditions:i}=n.extensions,s=i||o?r.ref:n.value;t.set(n.name,{value:s,variable:r.ref})}),t}const Th=e=>_e(e)&&"value"in e,Sa=e=>({list(){return Object.keys(Yd(e,Th))},search(t){return this.list().filter(n=>n.includes(t))}}),Nh=(e,t,n)=>({categoryKeys:t,list(r){return Array.from(e.categoryMap.get(r)?.entries()??[]).reduce((o,[i,s])=>(n(s,i)&&o.push(i),o),[])},search(r,o){return this.list(r).filter(i=>i.includes(o))}}),Ah=e=>({list(){return Object.keys(e)},search(t){return this.list().filter(n=>n.includes(t))}}),XC={sm:"480px",md:"768px",lg:"1024px",xl:"1280px","2xl":"1536px"},wa="var(--chakra-empty,/*!*/ /*!*/)",QC=_k({"*":{fontFeatureSettings:'"cv11"',"--ring-inset":wa,"--ring-offset-width":"0px","--ring-offset-color":"#fff","--ring-color":"rgba(66, 153, 225, 0.6)","--ring-offset-shadow":"0 0 #0000","--ring-shadow":"0 0 #0000",...Object.fromEntries(["brightness","contrast","grayscale","hue-rotate","invert","saturate","sepia","drop-shadow"].map(e=>[`--${e}`,wa])),...Object.fromEntries(["blur","brightness","contrast","grayscale","hue-rotate","invert","opacity","saturate","sepia"].map(e=>[`--backdrop-${e}`,wa])),"--global-font-mono":"fonts.mono","--global-font-body":"fonts.body","--global-color-border":"colors.border"},html:{color:"fg",bg:"bg",lineHeight:"1.5",colorPalette:"gray"},"*::placeholder, *[data-placeholder]":{color:"fg.muted/80"},"*::selection":{bg:"colorPalette.emphasized/80"}}),JC=Lk({"fill.muted":{value:{background:"colorPalette.muted",color:"colorPalette.fg"}},"fill.subtle":{value:{background:"colorPalette.subtle",color:"colorPalette.fg"}},"fill.surface":{value:{background:"colorPalette.subtle",color:"colorPalette.fg",boxShadow:"0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.muted"}},"fill.solid":{value:{background:"colorPalette.solid",color:"colorPalette.contrast"}},"outline.subtle":{value:{color:"colorPalette.fg",boxShadow:"inset 0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.subtle"}},"outline.solid":{value:{borderWidth:"1px",borderColor:"colorPalette.solid",color:"colorPalette.fg"}},"indicator.bottom":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",bottom:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.top":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",top:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.start":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineStart:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.end":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineEnd:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},disabled:{value:{opacity:"0.5",cursor:"not-allowed"}},none:{value:{}}}),ZC=Fk({"slide-fade-in":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-from-bottom, fade-in"},"&[data-placement^=bottom]":{animationName:"slide-from-top, fade-in"},"&[data-placement^=left]":{animationName:"slide-from-right, fade-in"},"&[data-placement^=right]":{animationName:"slide-from-left, fade-in"}}},"slide-fade-out":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-to-bottom, fade-out"},"&[data-placement^=bottom]":{animationName:"slide-to-top, fade-out"},"&[data-placement^=left]":{animationName:"slide-to-right, fade-out"},"&[data-placement^=right]":{animationName:"slide-to-left, fade-out"}}},"scale-fade-in":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-in, fade-in"}},"scale-fade-out":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-out, fade-out"}}}),Ea=fe({className:"chakra-badge",base:{display:"inline-flex",alignItems:"center",borderRadius:"l2",gap:"1",fontWeight:"medium",fontVariantNumeric:"tabular-nums",whiteSpace:"nowrap",userSelect:"none"},variants:{variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg"},outline:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},plain:{color:"colorPalette.fg"}},size:{xs:{textStyle:"2xs",px:"1",minH:"4"},sm:{textStyle:"xs",px:"1.5",minH:"5"},md:{textStyle:"sm",px:"2",minH:"6"},lg:{textStyle:"sm",px:"2.5",minH:"7"}}},defaultVariants:{variant:"subtle",size:"sm"}}),eS=fe({className:"chakra-button",base:{display:"inline-flex",appearance:"none",alignItems:"center",justifyContent:"center",userSelect:"none",position:"relative",borderRadius:"l2",whiteSpace:"nowrap",verticalAlign:"middle",borderWidth:"1px",borderColor:"transparent",cursor:"button",flexShrink:"0",outline:"0",lineHeight:"1.2",isolation:"isolate",fontWeight:"medium",transitionProperty:"common",transitionDuration:"moderate",focusVisibleRing:"outside",_disabled:{layerStyle:"disabled"},_icon:{flexShrink:"0"}},variants:{size:{"2xs":{h:"6",minW:"6",textStyle:"xs",px:"2",gap:"1",_icon:{width:"3.5",height:"3.5"}},xs:{h:"8",minW:"8",textStyle:"xs",px:"2.5",gap:"1",_icon:{width:"4",height:"4"}},sm:{h:"9",minW:"9",px:"3.5",textStyle:"sm",gap:"2",_icon:{width:"4",height:"4"}},md:{h:"10",minW:"10",textStyle:"sm",px:"4",gap:"2",_icon:{width:"5",height:"5"}},lg:{h:"11",minW:"11",textStyle:"md",px:"5",gap:"3",_icon:{width:"5",height:"5"}},xl:{h:"12",minW:"12",textStyle:"md",px:"5",gap:"2.5",_icon:{width:"5",height:"5"}},"2xl":{h:"16",minW:"16",textStyle:"lg",px:"7",gap:"3",_icon:{width:"6",height:"6"}}},variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"transparent",_hover:{bg:"colorPalette.solid/90"},_expanded:{bg:"colorPalette.solid/90"}},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"transparent",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},outline:{borderWidth:"1px",borderColor:"colorPalette.muted",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},ghost:{bg:"transparent",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},plain:{color:"colorPalette.fg"}}},defaultVariants:{size:"md",variant:"solid"}}),we=fe({className:"chakra-checkmark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"l1",cursor:"checkbox",focusVisibleRing:"outside",_icon:{boxSize:"full"},_invalid:{colorPalette:"red",borderColor:"border.error"},_disabled:{opacity:"0.5",cursor:"disabled"}},variants:{size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5",p:"0.5"},lg:{boxSize:"6",p:"0.5"}},variant:{solid:{borderColor:"border.emphasized","&:is([data-state=checked], [data-state=indeterminate])":{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},outline:{borderColor:"border","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg",borderColor:"colorPalette.solid"}},subtle:{bg:"colorPalette.muted",borderColor:"colorPalette.muted","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},plain:{"&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},inverted:{borderColor:"border",color:"colorPalette.fg","&:is([data-state=checked], [data-state=indeterminate])":{borderColor:"colorPalette.solid"}}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),{variants:tS,defaultVariants:nS}=Ea,rS=fe({className:"chakra-code",base:{fontFamily:"mono",alignItems:"center",display:"inline-flex",borderRadius:"l2"},variants:tS,defaultVariants:nS}),_h=fe({className:"color-swatch",base:{boxSize:"var(--swatch-size)",shadow:"inset 0 0 0 1px rgba(0, 0, 0, 0.1)","--checker-size":"8px","--checker-bg":"colors.bg","--checker-fg":"colors.bg.emphasized",background:"linear-gradient(var(--color), var(--color)), repeating-conic-gradient(var(--checker-fg) 0%, var(--checker-fg) 25%, var(--checker-bg) 0%, var(--checker-bg) 50%) 0% 50% / var(--checker-size) var(--checker-size) !important",display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},variants:{size:{"2xs":{"--swatch-size":"sizes.3.5"},xs:{"--swatch-size":"sizes.4"},sm:{"--swatch-size":"sizes.4.5"},md:{"--swatch-size":"sizes.5"},lg:{"--swatch-size":"sizes.6"},xl:{"--swatch-size":"sizes.7"},"2xl":{"--swatch-size":"sizes.8"},inherit:{"--swatch-size":"inherit"},full:{"--swatch-size":"100%"}},shape:{square:{borderRadius:"none"},circle:{borderRadius:"full"},rounded:{borderRadius:"l1"}}},defaultVariants:{size:"md",shape:"rounded"}}),oS=fe({className:"chakra-container",base:{position:"relative",maxWidth:"8xl",w:"100%",mx:"auto",px:{base:"4",md:"6",lg:"8"}},variants:{centerContent:{true:{display:"flex",flexDirection:"column",alignItems:"center"}},fluid:{true:{maxWidth:"full"}}}}),iS=fe({className:"chakra-heading",base:{fontFamily:"heading",fontWeight:"semibold"},variants:{size:{xs:{textStyle:"xs"},sm:{textStyle:"sm"},md:{textStyle:"md"},lg:{textStyle:"lg"},xl:{textStyle:"xl"},"2xl":{textStyle:"2xl"},"3xl":{textStyle:"3xl"},"4xl":{textStyle:"4xl"},"5xl":{textStyle:"5xl"},"6xl":{textStyle:"6xl"},"7xl":{textStyle:"7xl"}}},defaultVariants:{size:"xl"}}),sS=fe({className:"chakra-icon",base:{display:"inline-block",lineHeight:"1em",flexShrink:"0",color:"currentcolor",verticalAlign:"middle"},variants:{size:{inherit:{},xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"},xl:{boxSize:"7"},"2xl":{boxSize:"8"}}},defaultVariants:{size:"inherit"}}),he=fe({className:"chakra-input",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},height:"var(--input-height)",minW:"var(--input-height)","--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{"2xs":{textStyle:"xs",px:"2","--input-height":"sizes.7"},xs:{textStyle:"xs",px:"2","--input-height":"sizes.8"},sm:{textStyle:"sm",px:"2.5","--input-height":"sizes.9"},md:{textStyle:"sm",px:"3","--input-height":"sizes.10"},lg:{textStyle:"md",px:"4","--input-height":"sizes.11"},xl:{textStyle:"md",px:"4.5","--input-height":"sizes.12"},"2xl":{textStyle:"lg",px:"5","--input-height":"sizes.16"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)",_invalid:{borderColor:"var(--error-color)",boxShadow:"0px 1px 0px 0px var(--error-color)"}}}}},defaultVariants:{size:"md",variant:"outline"}}),aS=fe({className:"chakra-input-addon",base:{flex:"0 0 auto",width:"auto",display:"flex",alignItems:"center",whiteSpace:"nowrap",alignSelf:"stretch",borderRadius:"l2"},variants:{size:he.variants.size,variant:{outline:{borderWidth:"1px",borderColor:"border",bg:"bg.muted"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.emphasized"},flushed:{borderBottom:"1px solid",borderColor:"inherit",borderRadius:"0",px:"0",bg:"transparent"}}},defaultVariants:{size:"md",variant:"outline"}}),lS=fe({className:"chakra-kbd",base:{display:"inline-flex",alignItems:"center",fontWeight:"medium",fontFamily:"mono",flexShrink:"0",whiteSpace:"nowrap",wordSpacing:"-0.5em",userSelect:"none",px:"1",borderRadius:"l2"},variants:{variant:{raised:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderWidth:"1px",borderBottomWidth:"2px",borderColor:"colorPalette.muted"},outline:{borderWidth:"1px",color:"colorPalette.fg"},subtle:{bg:"colorPalette.muted",color:"colorPalette.fg"},plain:{color:"colorPalette.fg"}},size:{sm:{textStyle:"xs",height:"4.5"},md:{textStyle:"sm",height:"5"},lg:{textStyle:"md",height:"6"}}},defaultVariants:{size:"md",variant:"raised"}}),cS=fe({className:"chakra-link",base:{display:"inline-flex",alignItems:"center",outline:"none",gap:"1.5",cursor:"pointer",borderRadius:"l1",focusRing:"outside"},variants:{variant:{underline:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"},plain:{color:"colorPalette.fg",_hover:{textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"}}}},defaultVariants:{variant:"plain"}}),uS=fe({className:"chakra-mark",base:{bg:"transparent",color:"inherit",whiteSpace:"nowrap"},variants:{variant:{subtle:{bg:"colorPalette.subtle",color:"inherit"},solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},text:{fontWeight:"medium"},plain:{}}}}),Ee=fe({className:"chakra-radiomark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:0,verticalAlign:"top",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"full",cursor:"radio",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing",outlineOffset:"2px"},_invalid:{colorPalette:"red",borderColor:"red.500"},_disabled:{opacity:"0.5",cursor:"disabled"},"& .dot":{height:"100%",width:"100%",borderRadius:"full",bg:"currentColor",scale:"0.4"}},variants:{variant:{solid:{borderWidth:"1px",borderColor:"border.emphasized",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},subtle:{borderWidth:"1px",bg:"colorPalette.muted",borderColor:"colorPalette.muted",color:"transparent",_checked:{color:"colorPalette.fg"}},outline:{borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.fg",borderColor:"colorPalette.solid"},"& .dot":{scale:"0.6"}},inverted:{bg:"bg",borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.solid",borderColor:"currentcolor"}}},size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),dS=fe({className:"chakra-separator",base:{display:"block",borderColor:"border"},variants:{variant:{solid:{borderStyle:"solid"},dashed:{borderStyle:"dashed"},dotted:{borderStyle:"dotted"}},orientation:{vertical:{borderInlineStartWidth:"var(--separator-thickness)"},horizontal:{borderTopWidth:"var(--separator-thickness)"}},size:{xs:{"--separator-thickness":"0.5px"},sm:{"--separator-thickness":"1px"},md:{"--separator-thickness":"2px"},lg:{"--separator-thickness":"3px"}}},defaultVariants:{size:"sm",variant:"solid",orientation:"horizontal"}}),hS=fe({className:"chakra-skeleton",base:{},variants:{loading:{true:{borderRadius:"l2",boxShadow:"none",backgroundClip:"padding-box",cursor:"default",color:"transparent",pointerEvents:"none",userSelect:"none",flexShrink:"0","&::before, &::after, *":{visibility:"hidden"}},false:{background:"unset",animation:"fade-in var(--fade-duration, 0.1s) ease-out !important"}},variant:{pulse:{background:"bg.emphasized",animation:"pulse",animationDuration:"var(--duration, 1.2s)"},shine:{"--animate-from":"200%","--animate-to":"-200%","--start-color":"colors.bg.muted","--end-color":"colors.bg.emphasized",backgroundImage:"linear-gradient(270deg,var(--start-color),var(--end-color),var(--end-color),var(--start-color))",backgroundSize:"400% 100%",animation:"bg-position var(--duration, 5s) ease-in-out infinite"},none:{animation:"none"}}},defaultVariants:{variant:"pulse",loading:!0}}),fS=fe({className:"chakra-skip-nav",base:{display:"inline-flex",bg:"bg.panel",padding:"2.5",borderRadius:"l2",fontWeight:"semibold",focusVisibleRing:"outside",textStyle:"sm",userSelect:"none",border:"0",height:"1px",width:"1px",margin:"-1px",outline:"0",overflow:"hidden",position:"absolute",clip:"rect(0 0 0 0)",_focusVisible:{clip:"auto",width:"auto",height:"auto",position:"fixed",top:"6",insetStart:"6"}}}),gS=fe({className:"chakra-spinner",base:{display:"inline-block",borderColor:"currentColor",borderStyle:"solid",borderWidth:"2px",borderRadius:"full",width:"var(--spinner-size)",height:"var(--spinner-size)",animation:"spin",animationDuration:"slowest","--spinner-track-color":"transparent",borderBottomColor:"var(--spinner-track-color)",borderInlineStartColor:"var(--spinner-track-color)"},variants:{size:{inherit:{"--spinner-size":"1em"},xs:{"--spinner-size":"sizes.3"},sm:{"--spinner-size":"sizes.4"},md:{"--spinner-size":"sizes.5"},lg:{"--spinner-size":"sizes.8"},xl:{"--spinner-size":"sizes.10"}}},defaultVariants:{size:"md"}}),pS=fe({className:"chakra-textarea",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{xs:{textStyle:"xs",px:"2",py:"1.5",scrollPaddingBottom:"1.5"},sm:{textStyle:"sm",px:"2.5",py:"2",scrollPaddingBottom:"2"},md:{textStyle:"sm",px:"3",py:"2",scrollPaddingBottom:"2"},lg:{textStyle:"md",px:"4",py:"3",scrollPaddingBottom:"3"},xl:{textStyle:"md",px:"4.5",py:"3.5",scrollPaddingBottom:"3.5"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}}}},defaultVariants:{size:"md",variant:"outline"}}),mS={badge:Ea,button:eS,code:rS,container:oS,heading:iS,input:he,inputAddon:aS,kbd:lS,link:cS,mark:uS,separator:dS,skeleton:hS,skipNavLink:fS,spinner:gS,textarea:pS,icon:sS,checkmark:we,radiomark:Ee,colorSwatch:_h},vS=ma.colors({bg:{DEFAULT:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},emphasized:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},inverted:{value:{_light:"{colors.black}",_dark:"{colors.white}"}},panel:{value:{_light:"{colors.white}",_dark:"{colors.gray.950}"}},error:{value:{_light:"{colors.red.50}",_dark:"{colors.red.950}"}},warning:{value:{_light:"{colors.orange.50}",_dark:"{colors.orange.950}"}},success:{value:{_light:"{colors.green.50}",_dark:"{colors.green.950}"}},info:{value:{_light:"{colors.blue.50}",_dark:"{colors.blue.950}"}}},fg:{DEFAULT:{value:{_light:"{colors.black}",_dark:"{colors.gray.50}"}},muted:{value:{_light:"{colors.gray.600}",_dark:"{colors.gray.400}"}},subtle:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.500}"}},inverted:{value:{_light:"{colors.gray.50}",_dark:"{colors.black}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.300}"}},success:{value:{_light:"{colors.green.600}",_dark:"{colors.green.300}"}},info:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.300}"}}},border:{DEFAULT:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},inverted:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.400}"}},success:{value:{_light:"{colors.green.500}",_dark:"{colors.green.400}"}},info:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.400}"}}},gray:{contrast:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},fg:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},subtle:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},muted:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},solid:{value:{_light:"{colors.gray.900}",_dark:"{colors.white}"}},focusRing:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.400}"}}},red:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.red.700}",_dark:"{colors.red.300}"}},subtle:{value:{_light:"{colors.red.100}",_dark:"{colors.red.900}"}},muted:{value:{_light:"{colors.red.200}",_dark:"{colors.red.800}"}},emphasized:{value:{_light:"{colors.red.300}",_dark:"{colors.red.700}"}},solid:{value:{_light:"{colors.red.600}",_dark:"{colors.red.600}"}},focusRing:{value:{_light:"{colors.red.500}",_dark:"{colors.red.500}"}}},orange:{contrast:{value:{_light:"white",_dark:"black"}},fg:{value:{_light:"{colors.orange.700}",_dark:"{colors.orange.300}"}},subtle:{value:{_light:"{colors.orange.100}",_dark:"{colors.orange.900}"}},muted:{value:{_light:"{colors.orange.200}",_dark:"{colors.orange.800}"}},emphasized:{value:{_light:"{colors.orange.300}",_dark:"{colors.orange.700}"}},solid:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.500}"}},focusRing:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.500}"}}},green:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.green.700}",_dark:"{colors.green.300}"}},subtle:{value:{_light:"{colors.green.100}",_dark:"{colors.green.900}"}},muted:{value:{_light:"{colors.green.200}",_dark:"{colors.green.800}"}},emphasized:{value:{_light:"{colors.green.300}",_dark:"{colors.green.700}"}},solid:{value:{_light:"{colors.green.600}",_dark:"{colors.green.600}"}},focusRing:{value:{_light:"{colors.green.500}",_dark:"{colors.green.500}"}}},blue:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.blue.700}",_dark:"{colors.blue.300}"}},subtle:{value:{_light:"{colors.blue.100}",_dark:"{colors.blue.900}"}},muted:{value:{_light:"{colors.blue.200}",_dark:"{colors.blue.800}"}},emphasized:{value:{_light:"{colors.blue.300}",_dark:"{colors.blue.700}"}},solid:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.600}"}},focusRing:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.500}"}}},yellow:{contrast:{value:{_light:"black",_dark:"black"}},fg:{value:{_light:"{colors.yellow.800}",_dark:"{colors.yellow.300}"}},subtle:{value:{_light:"{colors.yellow.100}",_dark:"{colors.yellow.900}"}},muted:{value:{_light:"{colors.yellow.200}",_dark:"{colors.yellow.800}"}},emphasized:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.700}"}},solid:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.300}"}},focusRing:{value:{_light:"{colors.yellow.500}",_dark:"{colors.yellow.500}"}}},teal:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.teal.700}",_dark:"{colors.teal.300}"}},subtle:{value:{_light:"{colors.teal.100}",_dark:"{colors.teal.900}"}},muted:{value:{_light:"{colors.teal.200}",_dark:"{colors.teal.800}"}},emphasized:{value:{_light:"{colors.teal.300}",_dark:"{colors.teal.700}"}},solid:{value:{_light:"{colors.teal.600}",_dark:"{colors.teal.600}"}},focusRing:{value:{_light:"{colors.teal.500}",_dark:"{colors.teal.500}"}}},purple:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.purple.700}",_dark:"{colors.purple.300}"}},subtle:{value:{_light:"{colors.purple.100}",_dark:"{colors.purple.900}"}},muted:{value:{_light:"{colors.purple.200}",_dark:"{colors.purple.800}"}},emphasized:{value:{_light:"{colors.purple.300}",_dark:"{colors.purple.700}"}},solid:{value:{_light:"{colors.purple.600}",_dark:"{colors.purple.600}"}},focusRing:{value:{_light:"{colors.purple.500}",_dark:"{colors.purple.500}"}}},pink:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.pink.700}",_dark:"{colors.pink.300}"}},subtle:{value:{_light:"{colors.pink.100}",_dark:"{colors.pink.900}"}},muted:{value:{_light:"{colors.pink.200}",_dark:"{colors.pink.800}"}},emphasized:{value:{_light:"{colors.pink.300}",_dark:"{colors.pink.700}"}},solid:{value:{_light:"{colors.pink.600}",_dark:"{colors.pink.600}"}},focusRing:{value:{_light:"{colors.pink.500}",_dark:"{colors.pink.500}"}}},cyan:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.cyan.700}",_dark:"{colors.cyan.300}"}},subtle:{value:{_light:"{colors.cyan.100}",_dark:"{colors.cyan.900}"}},muted:{value:{_light:"{colors.cyan.200}",_dark:"{colors.cyan.800}"}},emphasized:{value:{_light:"{colors.cyan.300}",_dark:"{colors.cyan.700}"}},solid:{value:{_light:"{colors.cyan.600}",_dark:"{colors.cyan.600}"}},focusRing:{value:{_light:"{colors.cyan.500}",_dark:"{colors.cyan.500}"}}}}),bS=ma.radii({l1:{value:"{radii.xs}"},l2:{value:"{radii.sm}"},l3:{value:"{radii.md}"}}),yS=ma.shadows({xs:{value:{_light:"0px 1px 2px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/20}",_dark:"0px 1px 1px {black/64}, 0px 0px 1px inset {colors.gray.300/20}"}},sm:{value:{_light:"0px 2px 4px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 2px 4px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},md:{value:{_light:"0px 4px 8px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 4px 8px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},lg:{value:{_light:"0px 8px 16px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 8px 16px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},xl:{value:{_light:"0px 16px 24px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 16px 24px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},"2xl":{value:{_light:"0px 24px 40px {colors.gray.900/16}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 24px 40px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},inner:{value:{_light:"inset 0 2px 4px 0 {black/5}",_dark:"inset 0 2px 4px 0 black"}},inset:{value:{_light:"inset 0 0 0 1px {black/5}",_dark:"inset 0 0 0 1px {colors.gray.300/5}"}}}),xS=du.extendWith("itemBody"),kS=j("action-bar").parts("positioner","content","separator","selectionTrigger","closeTrigger"),CS=j("alert").parts("title","description","root","indicator","content"),SS=j("breadcrumb").parts("link","currentLink","item","list","root","ellipsis","separator"),wS=j("blockquote").parts("root","icon","content","caption"),ES=j("card").parts("root","header","body","footer","title","description"),OS=j("checkbox-card",["root","control","label","description","addon","indicator","content"]),PS=j("data-list").parts("root","item","itemLabel","itemValue"),IS=Xs.extendWith("header","body","footer","backdrop"),RS=Xs.extendWith("header","body","footer","backdrop"),TS=Zu.extendWith("textarea"),NS=j("empty-state",["root","content","indicator","title","description"]),AS=td.extendWith("requiredIndicator"),_S=rd.extendWith("content"),VS=od.extendWith("itemContent","dropzoneContent","fileText"),FS=j("list").parts("root","item","indicator"),LS=dd.extendWith("itemCommand"),DS=j("select").parts("root","field","indicator"),zS=Id.extendWith("header","body","footer"),Vh=ca.extendWith("itemAddon","itemIndicator"),MS=Vh.extendWith("itemContent","itemDescription"),jS=Td.extendWith("itemIndicator"),$S=_d.extendWith("indicatorGroup"),BS=Xx.extendWith("indicatorGroup","empty"),WS=Dd.extendWith("markerIndicator"),HS=j("stat").parts("root","label","helpText","valueText","valueUnit","indicator"),US=j("status").parts("root","indicator"),GS=j("steps",["root","list","item","trigger","indicator","separator","content","title","description","nextTrigger","prevTrigger","progress"]),qS=zd.extendWith("indicator"),KS=j("table").parts("root","header","body","row","columnHeader","cell","footer","caption"),YS=j("toast").parts("root","title","description","indicator","closeTrigger","actionTrigger"),XS=j("tabs").parts("root","trigger","list","content","contentGroup","indicator"),QS=j("tag").parts("root","label","closeTrigger","startElement","endElement"),JS=j("timeline").parts("root","item","content","separator","indicator","connector","title","description"),ZS=fx.extendWith("channelText"),ew=j("code-block",["root","content","title","header","footer","control","overlay","code","codeText","copyTrigger","copyIndicator","collapseTrigger","collapseIndicator","collapseText"]);ku.extendWith("valueText");const tw=w1,nw=B({className:"chakra-accordion",slots:xS.keys(),base:{root:{width:"full","--accordion-radius":"radii.l2"},item:{overflowAnchor:"none"},itemTrigger:{display:"flex",alignItems:"center",textAlign:"start",width:"full",outline:"0",gap:"3",fontWeight:"medium",borderRadius:"var(--accordion-radius)",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{layerStyle:"disabled"}},itemBody:{pt:"var(--accordion-padding-y)",pb:"calc(var(--accordion-padding-y) * 2)"},itemContent:{overflow:"hidden",borderRadius:"var(--accordion-radius)",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}},itemIndicator:{transition:"rotate 0.2s",transformOrigin:"center",color:"fg.subtle",_open:{rotate:"180deg"},_icon:{width:"1.2em",height:"1.2em"}}},variants:{variant:{outline:{item:{borderBottomWidth:"1px"}},subtle:{itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{borderRadius:"var(--accordion-radius)",_open:{bg:"colorPalette.subtle"}}},enclosed:{root:{borderWidth:"1px",borderRadius:"var(--accordion-radius)",divideY:"1px",overflow:"hidden"},itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{_open:{bg:"bg.subtle"}}},plain:{}},size:{sm:{root:{"--accordion-padding-x":"spacing.3","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"sm",py:"var(--accordion-padding-y)"}},md:{root:{"--accordion-padding-x":"spacing.4","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"md",py:"var(--accordion-padding-y)"}},lg:{root:{"--accordion-padding-x":"spacing.4.5","--accordion-padding-y":"spacing.2.5"},itemTrigger:{textStyle:"lg",py:"var(--accordion-padding-y)"}}}},defaultVariants:{size:"md",variant:"outline"}}),rw=B({className:"chakra-action-bar",slots:kS.keys(),base:{positioner:{position:"fixed",display:"flex",justifyContent:"center",pointerEvents:"none",insetInline:"0",top:"unset",bottom:"calc(env(safe-area-inset-bottom) + 20px)"},content:{bg:"bg.panel",shadow:"md",display:"flex",alignItems:"center",gap:"3",borderRadius:"l3",py:"2.5",px:"3",pointerEvents:"auto",translate:"calc(-1 * var(--scrollbar-width) / 2) 0px",_open:{animationName:"slide-from-bottom, fade-in",animationDuration:"moderate"},_closed:{animationName:"slide-to-bottom, fade-out",animationDuration:"faster"}},separator:{width:"1px",height:"5",bg:"border"},selectionTrigger:{display:"inline-flex",alignItems:"center",gap:"2",alignSelf:"stretch",textStyle:"sm",px:"4",py:"1",borderRadius:"l2",borderWidth:"1px",borderStyle:"dashed"}}}),ow=B({slots:CS.keys(),className:"chakra-alert",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",borderRadius:"l3"},title:{fontWeight:"medium"},description:{display:"inline"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",width:"1em",height:"1em",_icon:{boxSize:"full"}},content:{display:"flex",flex:"1",gap:"1"}},variants:{status:{info:{root:{colorPalette:"blue"}},warning:{root:{colorPalette:"orange"}},success:{root:{colorPalette:"green"}},error:{root:{colorPalette:"red"}},neutral:{root:{colorPalette:"gray"}}},inline:{true:{content:{display:"inline-flex",flexDirection:"row",alignItems:"center"}},false:{content:{display:"flex",flexDirection:"column"}}},variant:{subtle:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg"}},surface:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"},indicator:{color:"colorPalette.contrast"}}},size:{sm:{root:{gap:"2",px:"3",py:"3",textStyle:"xs"},indicator:{textStyle:"lg"}},md:{root:{gap:"3",px:"4",py:"4",textStyle:"sm"},indicator:{textStyle:"xl"}},lg:{root:{gap:"3",px:"4",py:"4",textStyle:"md"},indicator:{textStyle:"2xl"}}}},defaultVariants:{status:"info",variant:"subtle",size:"md",inline:!1}}),iw=B({slots:gu.keys(),className:"chakra-avatar",base:{root:{display:"inline-flex",alignItems:"center",justifyContent:"center",fontWeight:"medium",position:"relative",verticalAlign:"top",flexShrink:"0",userSelect:"none",width:"var(--avatar-size)",height:"var(--avatar-size)",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)","&[data-group-item]":{borderWidth:"2px",borderColor:"bg"}},image:{width:"100%",height:"100%",objectFit:"cover",borderRadius:"var(--avatar-radius)"},fallback:{lineHeight:"1",textTransform:"uppercase",fontWeight:"medium",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)"}},variants:{size:{full:{root:{"--avatar-size":"100%","--avatar-font-size":"100%"}},"2xs":{root:{"--avatar-font-size":"fontSizes.2xs","--avatar-size":"sizes.6"}},xs:{root:{"--avatar-font-size":"fontSizes.xs","--avatar-size":"sizes.8"}},sm:{root:{"--avatar-font-size":"fontSizes.sm","--avatar-size":"sizes.9"}},md:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.10"}},lg:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.11"}},xl:{root:{"--avatar-font-size":"fontSizes.lg","--avatar-size":"sizes.12"}},"2xl":{root:{"--avatar-font-size":"fontSizes.xl","--avatar-size":"sizes.16"}}},variant:{solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},subtle:{root:{bg:"colorPalette.muted",color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",borderWidth:"1px",borderColor:"colorPalette.muted"}}},shape:{square:{},rounded:{root:{"--avatar-radius":"radii.l3"}},full:{root:{"--avatar-radius":"radii.full"}}},borderless:{true:{root:{"&[data-group-item]":{borderWidth:"0px"}}}}},defaultVariants:{size:"md",shape:"full",variant:"subtle"}}),sw=B({className:"chakra-blockquote",slots:wS.keys(),base:{root:{position:"relative",display:"flex",flexDirection:"column",gap:"2"},caption:{textStyle:"sm",color:"fg.muted"},icon:{boxSize:"5"}},variants:{justify:{start:{root:{alignItems:"flex-start",textAlign:"start"}},center:{root:{alignItems:"center",textAlign:"center"}},end:{root:{alignItems:"flex-end",textAlign:"end"}}},variant:{subtle:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.muted"},icon:{color:"colorPalette.fg"}},solid:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.solid"},icon:{color:"colorPalette.solid"}},plain:{root:{paddingX:"5"},icon:{color:"colorPalette.solid"}}}},defaultVariants:{variant:"subtle",justify:"start"}}),aw=B({className:"chakra-breadcrumb",slots:SS.keys(),base:{list:{display:"flex",alignItems:"center",wordBreak:"break-word",color:"fg.muted",listStyle:"none"},link:{outline:"0",textDecoration:"none",borderRadius:"l1",focusRing:"outside",display:"inline-flex",alignItems:"center",gap:"2"},item:{display:"inline-flex",alignItems:"center"},separator:{color:"fg.muted",opacity:"0.8",_icon:{boxSize:"1em"},_rtl:{rotate:"180deg"}},ellipsis:{display:"inline-flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"1em"}}},variants:{variant:{underline:{link:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"0.2em",textDecorationColor:"colorPalette.muted"},currentLink:{color:"colorPalette.fg"}},plain:{link:{color:"fg.muted",_hover:{color:"fg"}},currentLink:{color:"fg"}}},size:{sm:{list:{gap:"1",textStyle:"xs"}},md:{list:{gap:"1.5",textStyle:"sm"}},lg:{list:{gap:"2",textStyle:"md"}}}},defaultVariants:{variant:"plain",size:"md"}}),lw=B({className:"chakra-card",slots:ES.keys(),base:{root:{display:"flex",flexDirection:"column",position:"relative",minWidth:"0",wordWrap:"break-word",borderRadius:"l3",color:"fg",textAlign:"start"},title:{fontWeight:"semibold"},description:{color:"fg.muted",fontSize:"sm"},header:{paddingInline:"var(--card-padding)",paddingTop:"var(--card-padding)",display:"flex",flexDirection:"column",gap:"1.5"},body:{padding:"var(--card-padding)",flex:"1",display:"flex",flexDirection:"column"},footer:{display:"flex",alignItems:"center",gap:"2",paddingInline:"var(--card-padding)",paddingBottom:"var(--card-padding)"}},variants:{size:{sm:{root:{"--card-padding":"spacing.4"},title:{textStyle:"md"}},md:{root:{"--card-padding":"spacing.6"},title:{textStyle:"lg"}},lg:{root:{"--card-padding":"spacing.7"},title:{textStyle:"xl"}}},variant:{elevated:{root:{bg:"bg.panel",boxShadow:"md"}},outline:{root:{bg:"bg.panel",borderWidth:"1px",borderColor:"border"}},subtle:{root:{bg:"bg.muted"}}}},defaultVariants:{variant:"outline",size:"md"}}),cw=B({slots:hx.keys(),className:"chakra-checkbox",base:{root:{display:"inline-flex",gap:"2",alignItems:"center",verticalAlign:"top",position:"relative"},control:we.base,label:{fontWeight:"medium",userSelect:"none",_disabled:{opacity:"0.5"}}},variants:{size:{xs:{root:{gap:"1.5"},label:{textStyle:"xs"},control:we.variants?.size?.xs},sm:{root:{gap:"2"},label:{textStyle:"sm"},control:we.variants?.size?.sm},md:{root:{gap:"2.5"},label:{textStyle:"sm"},control:we.variants?.size?.md},lg:{root:{gap:"3"},label:{textStyle:"md"},control:we.variants?.size?.lg}},variant:{outline:{control:we.variants?.variant?.outline},solid:{control:we.variants?.variant?.solid},subtle:{control:we.variants?.variant?.subtle}}},defaultVariants:{variant:"solid",size:"md"}}),uw=B({slots:OS.keys(),className:"chakra-checkbox-card",base:{root:{display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",flex:"1",focusVisibleRing:"outside",_disabled:{opacity:"0.8"},_invalid:{outline:"2px solid",outlineColor:"border.error"}},control:{display:"inline-flex",flex:"1",position:"relative",borderRadius:"inherit",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"},label:{fontWeight:"medium",display:"flex",alignItems:"center",gap:"2",flex:"1",_disabled:{opacity:"0.5"}},description:{opacity:"0.64",textStyle:"sm",_disabled:{opacity:"0.5"}},addon:{_disabled:{opacity:"0.5"}},indicator:we.base,content:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"}},variants:{size:{sm:{root:{textStyle:"sm"},control:{padding:"3",gap:"1.5"},addon:{px:"3",py:"1.5",borderTopWidth:"1px"},indicator:we.variants?.size.sm},md:{root:{textStyle:"sm"},control:{padding:"4",gap:"2.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:we.variants?.size.md},lg:{root:{textStyle:"md"},control:{padding:"4",gap:"3.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:we.variants?.size.lg}},variant:{surface:{root:{borderWidth:"1px",borderColor:"border",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"},_disabled:{bg:"bg.muted"}},indicator:we.variants?.variant.solid},subtle:{root:{bg:"bg.muted"},control:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},indicator:we.variants?.variant.plain},outline:{root:{borderWidth:"1px",borderColor:"border",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},indicator:we.variants?.variant.solid},solid:{root:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},indicator:we.variants?.variant.inverted}},justify:{start:{root:{"--checkbox-card-justify":"flex-start"}},end:{root:{"--checkbox-card-justify":"flex-end"}},center:{root:{"--checkbox-card-justify":"center"}}},align:{start:{root:{"--checkbox-card-align":"flex-start"},content:{textAlign:"start"}},end:{root:{"--checkbox-card-align":"flex-end"},content:{textAlign:"end"}},center:{root:{"--checkbox-card-align":"center"},content:{textAlign:"center"}}},orientation:{vertical:{control:{flexDirection:"column"}},horizontal:{control:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),dw=B({slots:ew.keys(),className:"code-block",base:{root:{colorPalette:"gray",rounded:"var(--code-block-radius)",overflow:"hidden",bg:"bg",color:"fg",borderWidth:"1px","--code-block-max-height":"320px","--code-block-bg":"colors.bg","--code-block-fg":"colors.fg","--code-block-obscured-opacity":"0.5","--code-block-obscured-blur":"1px","--code-block-line-number-width":"sizes.3","--code-block-line-number-margin":"spacing.4","--code-block-highlight-bg":"{colors.teal.focusRing/20}","--code-block-highlight-border":"colors.teal.focusRing","--code-block-highlight-added-bg":"{colors.green.focusRing/20}","--code-block-highlight-added-border":"colors.green.focusRing","--code-block-highlight-removed-bg":"{colors.red.focusRing/20}","--code-block-highlight-removed-border":"colors.red.focusRing"},header:{display:"flex",alignItems:"center",gap:"2",position:"relative",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)",mb:"calc(var(--code-block-padding) / 2 * -1)"},title:{display:"inline-flex",alignItems:"center",gap:"1.5",flex:"1",color:"fg.muted"},control:{gap:"1.5",display:"inline-flex",alignItems:"center"},footer:{display:"flex",alignItems:"center",justifyContent:"center",gap:"2",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)"},content:{position:"relative",colorScheme:"dark",overflowX:"auto",overflowY:"hidden",borderBottomRadius:"var(--code-block-radius)",maxHeight:"var(--code-block-max-height)","& ::selection":{bg:"blue.500/40"},_expanded:{maxHeight:"unset"}},overlay:{"--bg":"{colors.black/50}",display:"flex",alignItems:"flex-end",justifyContent:"center",padding:"4",bgImage:"linear-gradient(0deg,var(--bg) 25%,transparent 100%)",color:"white",minH:"5rem",pos:"absolute",bottom:"0",insetInline:"0",zIndex:"1",fontWeight:"medium",_expanded:{display:"none"}},code:{fontFamily:"mono",lineHeight:"tall",whiteSpace:"pre",counterReset:"line 0"},codeText:{px:"var(--code-block-padding)",py:"var(--code-block-padding)",position:"relative",display:"block",width:"100%","&[data-has-focused]":{"& [data-line]:not([data-focused])":{transitionProperty:"opacity, filter",transitionDuration:"moderate",transitionTimingFunction:"ease-in-out",opacity:"var(--code-block-obscured-opacity)",filter:"blur(var(--code-block-obscured-blur))"},"&:hover":{"--code-block-obscured-opacity":"1","--code-block-obscured-blur":"0px"}},"&[data-has-line-numbers][data-plaintext]":{paddingInlineStart:"calc(var(--code-block-line-number-width) + var(--code-block-line-number-margin) + var(--code-block-padding))"},"& [data-line]":{position:"relative","--highlight-bg":"var(--code-block-highlight-bg)","--highlight-border":"var(--code-block-highlight-border)","&[data-highlight], &[data-diff]":{display:"inline-block",width:"full","&:after":{content:"''",display:"block",position:"absolute",insetStart:"calc(var(--code-block-padding) * -1)",insetEnd:"0px",width:"calc(100% + var(--code-block-padding) * 2)",height:"100%",bg:"var(--highlight-bg)",borderStartWidth:"2px",borderStartColor:"var(--highlight-border)"}},"&[data-diff='added']":{"--highlight-bg":"var(--code-block-highlight-added-bg)","--highlight-border":"var(--code-block-highlight-added-border)"},"&[data-diff='removed']":{"--highlight-bg":"var(--code-block-highlight-removed-bg)","--highlight-border":"var(--code-block-highlight-removed-border)"}},"&[data-word-wrap]":{"&[data-plaintext], & [data-line]":{whiteSpace:"pre-wrap",wordBreak:"break-all"}},"&[data-has-line-numbers]":{"--content":"counter(line)","& [data-line]:before":{content:"var(--content)",counterIncrement:"line",width:"var(--code-block-line-number-width)",marginRight:"var(--code-block-line-number-margin)",display:"inline-block",textAlign:"end",userSelect:"none",whiteSpace:"nowrap",opacity:.4},"& [data-diff='added']:before":{content:"'+'"},"& [data-diff='removed']:before":{content:"'-'"}}}},variants:{size:{sm:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.md","--code-block-header-height":"sizes.8"},title:{textStyle:"xs"},code:{fontSize:"xs"}},md:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.lg","--code-block-header-height":"sizes.10"},title:{textStyle:"xs"},code:{fontSize:"sm"}},lg:{root:{"--code-block-padding":"spacing.5","--code-block-radius":"radii.xl","--code-block-header-height":"sizes.12"},title:{textStyle:"sm"},code:{fontSize:"sm"}}}},defaultVariants:{size:"md"}}),hw=B({slots:Sc.keys(),className:"chakra-collapsible",base:{content:{overflow:"hidden",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}),fw=B({className:"colorPicker",slots:ZS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5"},label:{color:"fg",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},valueText:{textAlign:"start"},control:{display:"flex",alignItems:"center",flexDirection:"row",gap:"2",position:"relative"},swatchTrigger:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{display:"flex",alignItems:"center",justifyContent:"center",flexDirection:"row",flexShrink:"0",gap:"2",textStyle:"sm",minH:"var(--input-height)",minW:"var(--input-height)",px:"1",rounded:"l2",_disabled:{opacity:"0.5"},"--focus-color":"colors.colorPalette.focusRing","&:focus-visible":{borderColor:"var(--focus-color)",outline:"1px solid var(--focus-color)"},"&[data-fit-content]":{"--input-height":"unset",px:"0",border:"0"}},content:{display:"flex",flexDirection:"column",bg:"bg.panel",borderRadius:"l3",boxShadow:"lg",width:"64",p:"4",gap:"3",zIndex:"dropdown",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},area:{height:"180px",borderRadius:"l2",overflow:"hidden"},areaThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",focusVisibleRing:"mixed",focusRingColor:"white"},areaBackground:{height:"full"},channelSlider:{borderRadius:"l2",flex:"1"},channelSliderTrack:{height:"var(--slider-height)",borderRadius:"inherit",boxShadow:"inset 0 0 0 1px rgba(0,0,0,0.1)"},channelText:{textStyle:"xs",color:"fg.muted",fontWeight:"medium",textTransform:"capitalize"},swatchGroup:{display:"flex",flexDirection:"row",flexWrap:"wrap",gap:"2"},swatch:{..._h.base,borderRadius:"l1"},swatchIndicator:{color:"white",rounded:"full"},channelSliderThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",transform:"translate(-50%, -50%)",focusVisibleRing:"outside",focusRingOffset:"1px"},channelInput:{...he.base,"&::-webkit-inner-spin-button, &::-webkit-outer-spin-button":{WebkitAppearance:"none",margin:0}},formatSelect:{textStyle:"xs",textTransform:"uppercase",borderWidth:"1px",minH:"6",focusRing:"inside",rounded:"l2"},transparencyGrid:{borderRadius:"l2"},view:{display:"flex",flexDirection:"column",gap:"2"}},variants:{size:{"2xs":{channelInput:he.variants?.size?.["2xs"],swatch:{"--swatch-size":"sizes.4.5"},trigger:{"--input-height":"sizes.7"},area:{"--thumb-size":"sizes.3"},channelSlider:{"--slider-height":"sizes.3","--thumb-size":"sizes.3"}},xs:{channelInput:he.variants?.size?.xs,swatch:{"--swatch-size":"sizes.5"},trigger:{"--input-height":"sizes.8"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},sm:{channelInput:he.variants?.size?.sm,swatch:{"--swatch-size":"sizes.6"},trigger:{"--input-height":"sizes.9"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},md:{channelInput:he.variants?.size?.md,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.10"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},lg:{channelInput:he.variants?.size?.lg,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.11"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},xl:{channelInput:he.variants?.size?.xl,swatch:{"--swatch-size":"sizes.8"},trigger:{"--input-height":"sizes.12"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},"2xl":{channelInput:he.variants?.size?.["2xl"],swatch:{"--swatch-size":"sizes.10"},trigger:{"--input-height":"sizes.16"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}}},variant:{outline:{channelInput:he.variants?.variant?.outline,trigger:{borderWidth:"1px"}},subtle:{channelInput:he.variants?.variant?.subtle,trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}}},defaultVariants:{size:"md",variant:"outline"}}),gw=B({className:"chakra-combobox",slots:BS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},input:{display:"flex",alignItems:"center",justifyContent:"space-between",background:"bg.panel",width:"full",minH:"var(--combobox-input-height)",px:"var(--combobox-input-padding-x)","--input-height":"var(--combobox-input-height)",borderRadius:"l2",outline:0,userSelect:"none",textAlign:"start",_placeholderShown:{color:"fg.muted"},_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},trigger:{display:"inline-flex",alignItems:"center",justifyContent:"center","--input-height":"var(--combobox-input-height)"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"},control:{pos:"relative"},indicatorGroup:{display:"flex",alignItems:"center",justifyContent:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--combobox-input-padding-x)",_icon:{boxSize:"var(--combobox-indicator-size)"},"[data-disabled] &":{opacity:.5}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"0s"},"&[data-empty]:not(:has([data-scope=combobox][data-part=empty]))":{opacity:0}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{boxSize:"var(--combobox-indicator-size)"}},empty:{py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{pb:"var(--combobox-item-padding-y)",_last:{pb:"0"}},itemGroupLabel:{fontWeight:"medium",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"}},variants:{variant:{outline:{input:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"}},subtle:{input:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"}},flushed:{input:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}},indicatorGroup:{px:"0"}}},size:{xs:{root:{"--combobox-input-height":"sizes.8","--combobox-input-padding-x":"spacing.2","--combobox-indicator-size":"sizes.3.5"},input:{textStyle:"xs"},content:{"--combobox-item-padding-x":"spacing.1.5","--combobox-item-padding-y":"spacing.1","--combobox-indicator-size":"sizes.3.5",p:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"}},sm:{root:{"--combobox-input-height":"sizes.9","--combobox-input-padding-x":"spacing.2.5","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"}},md:{root:{"--combobox-input-height":"sizes.10","--combobox-input-padding-x":"spacing.3","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{textStyle:"sm",gap:"2"}},lg:{root:{"--combobox-input-height":"sizes.12","--combobox-input-padding-x":"spacing.4","--combobox-indicator-size":"sizes.5"},input:{textStyle:"md"},content:{"--combobox-item-padding-y":"spacing.2","--combobox-item-padding-x":"spacing.3","--combobox-indicator-size":"sizes.5",p:"1.5",textStyle:"md"},trigger:{textStyle:"md",py:"3",gap:"2"}}}},defaultVariants:{size:"md",variant:"outline"}}),pw=B({slots:PS.keys(),className:"chakra-data-list",base:{itemLabel:{display:"flex",alignItems:"center",gap:"1"},itemValue:{display:"flex",minWidth:"0",flex:"1"}},variants:{orientation:{horizontal:{root:{display:"flex",flexDirection:"column"},item:{display:"inline-flex",alignItems:"center",gap:"4"},itemLabel:{minWidth:"120px"}},vertical:{root:{display:"flex",flexDirection:"column"},item:{display:"flex",flexDirection:"column",gap:"1"}}},size:{sm:{root:{gap:"3"},item:{textStyle:"xs"}},md:{root:{gap:"4"},item:{textStyle:"sm"}},lg:{root:{gap:"5"},item:{textStyle:"md"}}},variant:{subtle:{itemLabel:{color:"fg.muted"}},bold:{itemLabel:{fontWeight:"medium"},itemValue:{color:"fg.muted"}}}},defaultVariants:{size:"md",orientation:"vertical",variant:"subtle"}}),mw=B({slots:IS.keys(),className:"chakra-dialog",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",left:0,top:0,w:"100dvw",h:"100dvh",zIndex:"var(--z-index)",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100dvw",height:"100dvh",position:"fixed",left:0,top:0,"--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",justifyContent:"center",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,borderRadius:"l3",textStyle:"sm",my:"var(--dialog-margin, var(--dialog-base-margin))","--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"moderate"},_closed:{animationDuration:"faster"}},header:{display:"flex",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{flex:"1",px:"6",pt:"2",pb:"6"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"2",insetEnd:"2"}},variants:{placement:{center:{positioner:{alignItems:"center"},content:{"--dialog-base-margin":"auto",mx:"auto"}},top:{positioner:{alignItems:"flex-start"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}},bottom:{positioner:{alignItems:"flex-end"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}}},scrollBehavior:{inside:{positioner:{overflow:"hidden"},content:{maxH:"calc(100% - 7.5rem)"},body:{overflow:"auto"}},outside:{positioner:{overflow:"auto",pointerEvents:"auto"}}},size:{xs:{content:{maxW:"sm"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},cover:{positioner:{padding:"10"},content:{width:"100%",height:"100%","--dialog-margin":"0"}},full:{content:{maxW:"100dvw",minH:"100dvh","--dialog-margin":"0",borderRadius:"0"}}},motionPreset:{scale:{content:{_open:{animationName:"scale-in, fade-in"},_closed:{animationName:"scale-out, fade-out"}}},"slide-in-bottom":{content:{_open:{animationName:"slide-from-bottom, fade-in"},_closed:{animationName:"slide-to-bottom, fade-out"}}},"slide-in-top":{content:{_open:{animationName:"slide-from-top, fade-in"},_closed:{animationName:"slide-to-top, fade-out"}}},"slide-in-left":{content:{_open:{animationName:"slide-from-left, fade-in"},_closed:{animationName:"slide-to-left, fade-out"}}},"slide-in-right":{content:{_open:{animationName:"slide-from-right, fade-in"},_closed:{animationName:"slide-to-right, fade-out"}}},none:{}}},defaultVariants:{size:"md",scrollBehavior:"outside",placement:"top",motionPreset:"scale"}}),vw=B({slots:RS.keys(),className:"chakra-drawer",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",insetInlineStart:0,top:0,w:"100vw",h:"100dvh",zIndex:"overlay",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100vw",height:"100dvh",position:"fixed",insetInlineStart:0,top:0,zIndex:"modal",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,zIndex:"modal",textStyle:"sm",maxH:"100dvh",color:"inherit",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"slowest",animationTimingFunction:"ease-in-smooth"},_closed:{animationDuration:"slower",animationTimingFunction:"ease-in-smooth"}},header:{display:"flex",alignItems:"center",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{px:"6",py:"2",flex:"1",overflow:"auto"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{flex:"1",textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"3",insetEnd:"2"}},variants:{size:{xs:{content:{maxW:"xs"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},full:{content:{maxW:"100vw",h:"100dvh"}}},placement:{start:{positioner:{justifyContent:"flex-start",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-left-full, fade-in",_rtl:"slide-from-right-full, fade-in"}},_closed:{animationName:{base:"slide-to-left-full, fade-out",_rtl:"slide-to-right-full, fade-out"}}}},end:{positioner:{justifyContent:"flex-end",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-right-full, fade-in",_rtl:"slide-from-left-full, fade-in"}},_closed:{animationName:{base:"slide-to-right-full, fade-out",_rtl:"slide-to-left-full, fade-out"}}}},top:{positioner:{justifyContent:"stretch",alignItems:"flex-start"},content:{maxW:"100%",_open:{animationName:"slide-from-top-full, fade-in"},_closed:{animationName:"slide-to-top-full, fade-out"}}},bottom:{positioner:{justifyContent:"stretch",alignItems:"flex-end"},content:{maxW:"100%",_open:{animationName:"slide-from-bottom-full, fade-in"},_closed:{animationName:"slide-to-bottom-full, fade-out"}}}},contained:{true:{positioner:{padding:"4"},content:{borderRadius:"l3"}}}},defaultVariants:{size:"xs",placement:"end"}}),Fh=Zn({fontSize:"inherit",fontWeight:"inherit",textAlign:"inherit",bg:"transparent",borderRadius:"l2"}),bw=B({slots:TS.keys(),className:"chakra-editable",base:{root:{display:"inline-flex",alignItems:"center",position:"relative",gap:"1.5",width:"full"},preview:{...Fh,py:"1",px:"1",display:"inline-flex",alignItems:"center",transitionProperty:"common",transitionDuration:"moderate",cursor:"text",_hover:{bg:"bg.muted"},_disabled:{userSelect:"none"}},input:{...Fh,outline:"0",py:"1",px:"1",transitionProperty:"common",transitionDuration:"normal",width:"full",focusVisibleRing:"inside",focusRingWidth:"2px",_placeholder:{opacity:.6}},control:{display:"inline-flex",alignItems:"center",gap:"1.5"}},variants:{size:{sm:{root:{textStyle:"sm"},preview:{minH:"8"},input:{minH:"8"}},md:{root:{textStyle:"sm"},preview:{minH:"9"},input:{minH:"9"}},lg:{root:{textStyle:"md"},preview:{minH:"10"},input:{minH:"10"}}}},defaultVariants:{size:"md"}}),yw=B({slots:NS.keys(),className:"chakra-empty-state",base:{root:{width:"full"},content:{display:"flex",flexDirection:"column",alignItems:"center",justifyContent:"center"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:"fg.subtle",_icon:{boxSize:"1em"}},title:{fontWeight:"semibold"},description:{textStyle:"sm",color:"fg.muted"}},variants:{size:{sm:{root:{px:"4",py:"6"},title:{textStyle:"md"},content:{gap:"4"},indicator:{textStyle:"2xl"}},md:{root:{px:"8",py:"12"},title:{textStyle:"lg"},content:{gap:"6"},indicator:{textStyle:"4xl"}},lg:{root:{px:"12",py:"16"},title:{textStyle:"xl"},content:{gap:"8"},indicator:{textStyle:"6xl"}}}},defaultVariants:{size:"md"}}),xw=B({className:"chakra-field",slots:AS.keys(),base:{requiredIndicator:{color:"fg.error",lineHeight:"1"},root:{display:"flex",width:"100%",position:"relative",gap:"1.5"},label:{display:"flex",alignItems:"center",textAlign:"start",textStyle:"sm",fontWeight:"medium",gap:"1",userSelect:"none",_disabled:{opacity:"0.5"}},errorText:{display:"inline-flex",alignItems:"center",fontWeight:"medium",gap:"1",color:"fg.error",textStyle:"xs"},helperText:{color:"fg.muted",textStyle:"xs"}},variants:{orientation:{vertical:{root:{flexDirection:"column",alignItems:"flex-start"}},horizontal:{root:{flexDirection:"row",alignItems:"center",justifyContent:"space-between"},label:{flex:"0 0 var(--field-label-width, 80px)"}}}},defaultVariants:{orientation:"vertical"}}),kw=B({className:"fieldset",slots:_S.keys(),base:{root:{display:"flex",flexDirection:"column",width:"full"},content:{display:"flex",flexDirection:"column",width:"full"},legend:{color:"fg",fontWeight:"medium",_disabled:{opacity:"0.5"}},helperText:{color:"fg.muted",textStyle:"sm"},errorText:{display:"inline-flex",alignItems:"center",color:"fg.error",gap:"2",fontWeight:"medium",textStyle:"sm"}},variants:{size:{sm:{root:{spaceY:"2"},content:{gap:"1.5"},legend:{textStyle:"sm"}},md:{root:{spaceY:"4"},content:{gap:"4"},legend:{textStyle:"sm"}},lg:{root:{spaceY:"6"},content:{gap:"4"},legend:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),Cw=B({className:"chakra-file-upload",slots:VS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"4",width:"100%",alignItems:"flex-start"},label:{fontWeight:"medium",textStyle:"sm"},dropzone:{background:"bg",borderRadius:"l3",borderWidth:"2px",borderStyle:"dashed",display:"flex",alignItems:"center",flexDirection:"column",gap:"4",justifyContent:"center",minHeight:"2xs",px:"3",py:"2",transition:"backgrounds",focusVisibleRing:"outside",_hover:{bg:"bg.subtle"},_dragging:{bg:"colorPalette.subtle",borderStyle:"solid",borderColor:"colorPalette.solid"}},dropzoneContent:{display:"flex",flexDirection:"column",alignItems:"center",textAlign:"center",gap:"1",textStyle:"sm"},item:{pos:"relative",textStyle:"sm",animationName:"fade-in",animationDuration:"moderate",background:"bg",borderRadius:"l2",borderWidth:"1px",width:"100%",display:"flex",alignItems:"center",gap:"3",p:"4"},itemGroup:{width:"100%",display:"flex",flexDirection:"column",gap:"3",_empty:{display:"none"}},itemName:{color:"fg",fontWeight:"medium",lineClamp:"1"},itemContent:{display:"flex",flexDirection:"column",gap:"0.5",flex:"1"},itemSizeText:{color:"fg.muted",textStyle:"xs"},itemDeleteTrigger:{display:"flex",alignItems:"center",justifyContent:"center",alignSelf:"flex-start",boxSize:"5",p:"2px",color:"fg.muted",cursor:"button"},itemPreview:{color:"fg.muted",_icon:{boxSize:"4.5"}}},defaultVariants:{}}),Sw=B({className:"chakra-hover-card",slots:id.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--hovercard-bg":"colors.bg.panel",bg:"var(--hovercard-bg)",boxShadow:"lg",maxWidth:"80",borderRadius:"l3",zIndex:"popover",transformOrigin:"var(--transform-origin)",outline:"0",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--hovercard-bg)"},arrowTip:{borderTopWidth:"0.5px",borderInlineStartWidth:"0.5px"}},variants:{size:{xs:{content:{padding:"3"}},sm:{content:{padding:"4"}},md:{content:{padding:"5"}},lg:{content:{padding:"6"}}}},defaultVariants:{size:"md"}}),ww=B({className:"chakra-list",slots:FS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"var(--list-gap)","& :where(ul, ol)":{marginTop:"var(--list-gap)"}},item:{whiteSpace:"normal",display:"list-item"},indicator:{marginEnd:"2",minHeight:"1lh",flexShrink:0,display:"inline-block",verticalAlign:"middle"}},variants:{variant:{marker:{root:{listStyle:"revert"},item:{_marker:{color:"fg.subtle"}}},plain:{item:{alignItems:"flex-start",display:"inline-flex"}}},align:{center:{item:{alignItems:"center"}},start:{item:{alignItems:"flex-start"}},end:{item:{alignItems:"flex-end"}}}},defaultVariants:{variant:"marker"}}),Ew=B({className:"chakra-listbox",slots:tw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},content:{display:"flex",maxH:"96",p:"1",gap:"1",textStyle:"sm",outline:"none",scrollPadding:"1",_horizontal:{flexDirection:"row",overflowX:"auto"},_vertical:{flexDirection:"column",overflowY:"auto"},"--listbox-item-padding-x":"spacing.2","--listbox-item-padding-y":"spacing.1.5"},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"pointer",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)",_highlighted:{outline:"2px solid",outlineColor:"border.emphasized"},_disabled:{pointerEvents:"none",opacity:"0.5"}},empty:{py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{mt:"1.5",_first:{mt:"0"}},itemGroupLabel:{py:"1.5",px:"2",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"4"}}},variants:{variant:{subtle:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_hover:{bg:"bg.emphasized/60"},_selected:{bg:"bg.muted"}}},solid:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_selected:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}},plain:{}}},defaultVariants:{variant:"subtle"}}),Ow=B({className:"chakra-menu",slots:LS.keys(),base:{content:{outline:0,bg:"bg.panel",boxShadow:"lg",color:"fg",maxHeight:"var(--available-height)","--menu-z-index":"zIndex.dropdown",zIndex:"calc(var(--menu-z-index) + var(--layer-index, 0))",borderRadius:"l2",overflow:"hidden",overflowY:"auto",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},item:{textDecoration:"none",color:"fg",userSelect:"none",borderRadius:"l1",width:"100%",display:"flex",cursor:"menuitem",alignItems:"center",textAlign:"start",position:"relative",flex:"0 0 auto",outline:0,_disabled:{layerStyle:"disabled"},"&[data-type]":{ps:"8"}},itemText:{flex:"1"},itemIndicator:{position:"absolute",insetStart:"2",transform:"translateY(-50%)",top:"50%"},itemGroupLabel:{px:"2",py:"1.5",fontWeight:"semibold",textStyle:"sm"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},itemCommand:{opacity:"0.6",textStyle:"xs",ms:"auto",ps:"4",letterSpacing:"widest",fontFamily:"inherit"},separator:{height:"1px",bg:"bg.muted",my:"1",mx:"-1"}},variants:{variant:{subtle:{item:{_highlighted:{bg:"bg.emphasized/60"}}},solid:{item:{_highlighted:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}}},size:{sm:{content:{minW:"8rem",padding:"1",scrollPadding:"1"},item:{gap:"1",textStyle:"xs",py:"1",px:"1.5"}},md:{content:{minW:"8rem",padding:"1.5",scrollPadding:"1.5"},item:{gap:"2",textStyle:"sm",py:"1.5",px:"2"}}}},defaultVariants:{size:"md",variant:"subtle"}}),li=B({className:"chakra-select",slots:$S.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},trigger:{display:"flex",alignItems:"center",justifyContent:"space-between",width:"full",minH:"var(--select-trigger-height)","--input-height":"var(--select-trigger-height)",px:"var(--select-trigger-padding-x)",borderRadius:"l2",userSelect:"none",textAlign:"start",focusVisibleRing:"inside",_placeholderShown:{color:"fg.muted/80"},_disabled:{layerStyle:"disabled"},_invalid:{borderColor:"border.error"}},indicatorGroup:{display:"flex",alignItems:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--select-trigger-padding-x)",pointerEvents:"none"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:{base:"fg.muted",_disabled:"fg.subtle",_invalid:"fg.error"}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"fastest"}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{width:"4",height:"4"}},control:{pos:"relative"},itemText:{flex:"1"},itemGroup:{_first:{mt:"0"}},itemGroupLabel:{py:"1",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"}},variants:{variant:{outline:{trigger:{bg:"transparent",borderWidth:"1px",borderColor:"border",_expanded:{borderColor:"border.emphasized"}}},subtle:{trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}},size:{xs:{root:{"--select-trigger-height":"sizes.8","--select-trigger-padding-x":"spacing.2"},content:{p:"1",gap:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"},item:{py:"1",px:"2"},itemGroupLabel:{py:"1",px:"2"},indicator:{_icon:{width:"3.5",height:"3.5"}}},sm:{root:{"--select-trigger-height":"sizes.9","--select-trigger-padding-x":"spacing.2.5"},content:{p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"},indicator:{_icon:{width:"4",height:"4"}},item:{py:"1",px:"1.5"},itemGroup:{mt:"1"},itemGroupLabel:{py:"1",px:"1.5"}},md:{root:{"--select-trigger-height":"sizes.10","--select-trigger-padding-x":"spacing.3"},content:{p:"1",textStyle:"sm"},itemGroup:{mt:"1.5"},item:{py:"1.5",px:"2"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},itemGroupLabel:{py:"1.5",px:"2"},trigger:{textStyle:"sm",gap:"2"},indicator:{_icon:{width:"4",height:"4"}}},lg:{root:{"--select-trigger-height":"sizes.12","--select-trigger-padding-x":"spacing.4"},content:{p:"1.5",textStyle:"md"},itemGroup:{mt:"2"},item:{py:"2",px:"3"},itemGroupLabel:{py:"2",px:"3"},trigger:{textStyle:"md",py:"3",gap:"2"},indicator:{_icon:{width:"5",height:"5"}}}}},defaultVariants:{size:"md",variant:"outline"}}),Pw=B({className:"chakra-native-select",slots:DS.keys(),base:{root:{height:"fit-content",display:"flex",width:"100%",position:"relative"},field:{width:"100%",minWidth:"0",outline:"0",appearance:"none",borderRadius:"l2","--error-color":"colors.border.error","--input-height":"var(--select-field-height)",height:"var(--select-field-height)",_disabled:{layerStyle:"disabled"},_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"},focusVisibleRing:"inside",lineHeight:"normal","& > option, & > optgroup":{bg:"bg"}},indicator:{position:"absolute",display:"inline-flex",alignItems:"center",justifyContent:"center",pointerEvents:"none",top:"50%",transform:"translateY(-50%)",height:"100%",color:"fg.muted",_disabled:{opacity:"0.5"},_invalid:{color:"fg.error"},_icon:{width:"1em",height:"1em"}}},variants:{variant:{outline:{field:li.variants?.variant.outline.trigger},subtle:{field:li.variants?.variant.subtle.trigger},plain:{field:{bg:"transparent",color:"fg",focusRingWidth:"2px"}}},size:{xs:{root:{"--select-field-height":"sizes.8"},field:{textStyle:"xs",ps:"2",pe:"6"},indicator:{textStyle:"sm",insetEnd:"1.5"}},sm:{root:{"--select-field-height":"sizes.9"},field:{textStyle:"sm",ps:"2.5",pe:"8"},indicator:{textStyle:"md",insetEnd:"2"}},md:{root:{"--select-field-height":"sizes.10"},field:{textStyle:"sm",ps:"3",pe:"8"},indicator:{textStyle:"lg",insetEnd:"2"}},lg:{root:{"--select-field-height":"sizes.11"},field:{textStyle:"md",ps:"4",pe:"8"},indicator:{textStyle:"xl",insetEnd:"3"}},xl:{root:{"--select-field-height":"sizes.12"},field:{textStyle:"md",ps:"4.5",pe:"10"},indicator:{textStyle:"xl",insetEnd:"3"}}}},defaultVariants:li.defaultVariants});function Oa(e,t){const n={};for(const r in e){const o=t(r,e[r]);n[o[0]]=o[1]}return n}const Lh=Zn({display:"flex",justifyContent:"center",alignItems:"center",flex:"1",userSelect:"none",cursor:"button",lineHeight:"1",color:"fg.muted","--stepper-base-radius":"radii.l1","--stepper-radius":"calc(var(--stepper-base-radius) + 1px)",_icon:{boxSize:"1em"},_disabled:{opacity:"0.5"},_hover:{bg:"bg.muted"},_active:{bg:"bg.emphasized"}}),Iw=B({className:"chakra-number-input",slots:Cd.keys(),base:{root:{position:"relative",zIndex:"0",isolation:"isolate"},input:{...he.base,verticalAlign:"top",pe:"calc(var(--stepper-width) + 0.5rem)"},control:{display:"flex",flexDirection:"column",position:"absolute",top:"0",insetEnd:"0px",margin:"1px",width:"var(--stepper-width)",height:"calc(100% - 2px)",zIndex:"1",borderStartWidth:"1px",divideY:"1px"},incrementTrigger:{...Lh,borderTopEndRadius:"var(--stepper-radius)"},decrementTrigger:{...Lh,borderBottomEndRadius:"var(--stepper-radius)"},valueText:{fontWeight:"medium",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums"}},variants:{size:{xs:{input:he.variants.size.xs,control:{fontSize:"2xs","--stepper-width":"sizes.4"}},sm:{input:he.variants.size.sm,control:{fontSize:"xs","--stepper-width":"sizes.5"}},md:{input:he.variants.size.md,control:{fontSize:"sm","--stepper-width":"sizes.6"}},lg:{input:he.variants.size.lg,control:{fontSize:"sm","--stepper-width":"sizes.6"}}},variant:Oa(he.variants.variant,(e,t)=>[e,{input:t}])},defaultVariants:{size:"md",variant:"outline"}}),{variants:Dh,defaultVariants:Rw}=he,Tw=B({className:"chakra-pin-input",slots:Pd.keys(),base:{input:{...he.base,textAlign:"center",width:"var(--input-height)"},control:{display:"inline-flex",gap:"2",isolation:"isolate"}},variants:{size:Oa(Dh.size,(e,t)=>[e,{input:{...t,px:"1"}}]),variant:Oa(Dh.variant,(e,t)=>[e,{input:t}]),attached:{true:{control:{gap:"0",spaceX:"-1px"},input:{_notFirst:{borderStartRadius:"0"},_notLast:{borderEndRadius:"0"},_focusVisible:{zIndex:"1"}}}}},defaultVariants:Rw}),Nw=B({className:"chakra-popover",slots:zS.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--popover-bg":"colors.bg.panel",bg:"var(--popover-bg)",boxShadow:"lg","--popover-size":"sizes.xs","--popover-mobile-size":"calc(100dvw - 1rem)",width:{base:"min(var(--popover-mobile-size), var(--popover-size))",sm:"var(--popover-size)"},borderRadius:"l3","--popover-z-index":"zIndex.popover",zIndex:"calc(var(--popover-z-index) + var(--layer-index, 0))",outline:"0",transformOrigin:"var(--transform-origin)",maxHeight:"var(--available-height)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"faster"}},header:{paddingInline:"var(--popover-padding)",paddingTop:"var(--popover-padding)"},body:{padding:"var(--popover-padding)",flex:"1"},footer:{display:"flex",alignItems:"center",paddingInline:"var(--popover-padding)",paddingBottom:"var(--popover-padding)"},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--popover-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px"}},variants:{size:{xs:{content:{"--popover-padding":"spacing.3"}},sm:{content:{"--popover-padding":"spacing.4"}},md:{content:{"--popover-padding":"spacing.5"}},lg:{content:{"--popover-padding":"spacing.6"}}}},defaultVariants:{size:"md"}}),Aw=B({slots:la.keys(),className:"chakra-progress",base:{root:{textStyle:"sm",position:"relative"},track:{overflow:"hidden",position:"relative"},range:{display:"flex",alignItems:"center",justifyContent:"center",transitionProperty:"width, height",transitionDuration:"slow",height:"100%",bgColor:"var(--track-color)",_indeterminate:{"--animate-from-x":"-40%","--animate-to-x":"100%",position:"absolute",willChange:"left",minWidth:"50%",animation:"position 1s ease infinite normal none running",backgroundImage:"linear-gradient(to right, transparent 0%, var(--track-color) 50%, transparent 100%)"}},label:{display:"inline-flex",fontWeight:"medium",alignItems:"center",gap:"1"},valueText:{textStyle:"xs",lineHeight:"1",fontWeight:"medium"}},variants:{variant:{outline:{track:{shadow:"inset",bgColor:"bg.muted"},range:{bgColor:"colorPalette.solid"}},subtle:{track:{bgColor:"colorPalette.muted"},range:{bgColor:"colorPalette.solid/72"}}},shape:{square:{},rounded:{track:{borderRadius:"l1"}},full:{track:{borderRadius:"full"}}},striped:{true:{range:{backgroundImage:"linear-gradient(45deg, var(--stripe-color) 25%, transparent 25%, transparent 50%, var(--stripe-color) 50%, var(--stripe-color) 75%, transparent 75%, transparent)",backgroundSize:"var(--stripe-size) var(--stripe-size)","--stripe-size":"1rem","--stripe-color":{_light:"rgba(255, 255, 255, 0.3)",_dark:"rgba(0, 0, 0, 0.3)"}}}},animated:{true:{range:{"--animate-from":"var(--stripe-size)",animation:"bg-position 1s linear infinite"}}},size:{xs:{track:{h:"1.5"}},sm:{track:{h:"2"}},md:{track:{h:"2.5"}},lg:{track:{h:"3"}},xl:{track:{h:"4"}}}},defaultVariants:{variant:"outline",size:"md",shape:"rounded"}}),_w=B({className:"chakra-progress-circle",slots:la.keys(),base:{root:{display:"inline-flex",textStyle:"sm",position:"relative"},circle:{_indeterminate:{animation:"spin 2s linear infinite"}},circleTrack:{"--track-color":"colors.colorPalette.muted",stroke:"var(--track-color)"},circleRange:{stroke:"colorPalette.solid",transitionProperty:"stroke-dashoffset, stroke-dasharray",transitionDuration:"0.6s",_indeterminate:{animation:"circular-progress 1.5s linear infinite"}},label:{display:"inline-flex"},valueText:{lineHeight:"1",fontWeight:"medium",letterSpacing:"tight",fontVariantNumeric:"tabular-nums"}},variants:{size:{xs:{circle:{"--size":"24px","--thickness":"4px"},valueText:{textStyle:"2xs"}},sm:{circle:{"--size":"32px","--thickness":"5px"},valueText:{textStyle:"2xs"}},md:{circle:{"--size":"40px","--thickness":"6px"},valueText:{textStyle:"xs"}},lg:{circle:{"--size":"48px","--thickness":"7px"},valueText:{textStyle:"sm"}},xl:{circle:{"--size":"64px","--thickness":"8px"},valueText:{textStyle:"sm"}}}},defaultVariants:{size:"md"}}),Vw=B({slots:Rd.keys(),className:"chakra-qr-code",base:{root:{position:"relative",width:"fit-content","--qr-code-overlay-size":"calc(var(--qr-code-size) / 3)"},frame:{width:"var(--qr-code-size)",height:"var(--qr-code-size)",fill:"currentColor"},overlay:{display:"flex",alignItems:"center",justifyContent:"center",width:"var(--qr-code-overlay-size)",height:"var(--qr-code-overlay-size)",padding:"1",bg:"bg",rounded:"l1"}},variants:{size:{"2xs":{root:{"--qr-code-size":"40px"}},xs:{root:{"--qr-code-size":"64px"}},sm:{root:{"--qr-code-size":"80px"}},md:{root:{"--qr-code-size":"120px"}},lg:{root:{"--qr-code-size":"160px"}},xl:{root:{"--qr-code-size":"200px"}},"2xl":{root:{"--qr-code-size":"240px"}},full:{root:{"--qr-code-size":"100%"}}}},defaultVariants:{size:"md"}}),Fw=B({className:"chakra-radio-card",slots:MS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",isolation:"isolate"},item:{flex:"1",display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",_focus:{bg:"colorPalette.muted/20"},_disabled:{opacity:"0.8",borderColor:"border.disabled"},_checked:{zIndex:"1"}},label:{display:"inline-flex",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},itemText:{fontWeight:"medium",flex:"1"},itemDescription:{opacity:"0.64",textStyle:"sm"},itemControl:{display:"inline-flex",flex:"1",pos:"relative",rounded:"inherit",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)",_disabled:{bg:"bg.muted"}},itemIndicator:Ee.base,itemAddon:{roundedBottom:"inherit",_disabled:{color:"fg.muted"}},itemContent:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)"}},variants:{size:{sm:{item:{textStyle:"sm"},itemControl:{padding:"3",gap:"1.5"},itemAddon:{px:"3",py:"1.5",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.sm},md:{item:{textStyle:"sm"},itemControl:{padding:"4",gap:"2.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.md},lg:{item:{textStyle:"md"},itemControl:{padding:"4",gap:"3.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.lg}},variant:{surface:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"}},itemIndicator:Ee.variants?.variant.solid},subtle:{item:{bg:"bg.muted"},itemControl:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},itemIndicator:Ee.variants?.variant.outline},outline:{item:{borderWidth:"1px",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.solid},solid:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.inverted}},justify:{start:{item:{"--radio-card-justify":"flex-start"}},end:{item:{"--radio-card-justify":"flex-end"}},center:{item:{"--radio-card-justify":"center"}}},align:{start:{item:{"--radio-card-align":"flex-start"},itemControl:{textAlign:"start"}},end:{item:{"--radio-card-align":"flex-end"},itemControl:{textAlign:"end"}},center:{item:{"--radio-card-align":"center"},itemControl:{textAlign:"center"}}},orientation:{vertical:{itemControl:{flexDirection:"column"}},horizontal:{itemControl:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),Lw=B({className:"chakra-radio-group",slots:Vh.keys(),base:{item:{display:"inline-flex",alignItems:"center",position:"relative",fontWeight:"medium",_disabled:{cursor:"disabled"}},itemControl:Ee.base,label:{userSelect:"none",textStyle:"sm",_disabled:{opacity:"0.5"}}},variants:{variant:{outline:{itemControl:Ee.variants?.variant?.outline},subtle:{itemControl:Ee.variants?.variant?.subtle},solid:{itemControl:Ee.variants?.variant?.solid}},size:{xs:{item:{textStyle:"xs",gap:"1.5"},itemControl:Ee.variants?.size?.xs},sm:{item:{textStyle:"sm",gap:"2"},itemControl:Ee.variants?.size?.sm},md:{item:{textStyle:"sm",gap:"2.5"},itemControl:Ee.variants?.size?.md},lg:{item:{textStyle:"md",gap:"3"},itemControl:Ee.variants?.size?.lg}}},defaultVariants:{size:"md",variant:"solid"}}),Dw=B({className:"chakra-rating-group",slots:jS.keys(),base:{root:{display:"inline-flex"},control:{display:"inline-flex",alignItems:"center"},item:{display:"inline-flex",alignItems:"center",justifyContent:"center",userSelect:"none"},itemIndicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",width:"1em",height:"1em",position:"relative","--clip-path":{base:"inset(0 50% 0 0)",_rtl:"inset(0 0 0 50%)"},_icon:{stroke:"currentColor",width:"100%",height:"100%",display:"inline-block",flexShrink:0,position:"absolute",left:0,top:0},"& [data-bg]":{color:"bg.emphasized"},"& [data-fg]":{color:"transparent"},"&[data-highlighted]:not([data-half])":{"& [data-fg]":{color:"colorPalette.solid"}},"&[data-half]":{"& [data-fg]":{color:"colorPalette.solid",clipPath:"var(--clip-path)"}}}},variants:{size:{xs:{item:{textStyle:"sm"}},sm:{item:{textStyle:"md"}},md:{item:{textStyle:"xl"}},lg:{item:{textStyle:"2xl"}}}},defaultVariants:{size:"md"}}),zw=B({className:"chakra-scroll-area",slots:Nd.keys(),base:{root:{display:"flex",flexDirection:"column",width:"100%",height:"100%",position:"relative",overflow:"hidden","--scrollbar-margin":"2px","--scrollbar-click-area":"calc(var(--scrollbar-size) + calc(var(--scrollbar-margin) * 2))"},viewport:{display:"flex",flexDirection:"column",height:"100%",width:"100%",borderRadius:"inherit",WebkitOverflowScrolling:"touch",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}},content:{minWidth:"100%"},scrollbar:{display:"flex",userSelect:"none",touchAction:"none",borderRadius:"full",colorPalette:"gray",transition:"opacity 150ms 300ms",position:"relative",margin:"var(--scrollbar-margin)","&:not([data-overflow-x], [data-overflow-y])":{display:"none"},bg:"{colors.colorPalette.solid/10}","--thumb-bg":"{colors.colorPalette.solid/25}","&:is(:hover, :active)":{"--thumb-bg":"{colors.colorPalette.solid/50}"},_before:{content:'""',position:"absolute"},_vertical:{width:"var(--scrollbar-size)",flexDirection:"column","&::before":{width:"var(--scrollbar-click-area)",height:"100%",insetInlineStart:"calc(var(--scrollbar-margin) * -1)"}},_horizontal:{height:"var(--scrollbar-size)",flexDirection:"row","&::before":{height:"var(--scrollbar-click-area)",width:"100%",top:"calc(var(--scrollbar-margin) * -1)"}}},thumb:{borderRadius:"inherit",bg:"var(--thumb-bg)",transition:"backgrounds",_vertical:{width:"full"},_horizontal:{height:"full"}},corner:{bg:"bg.muted",margin:"var(--scrollbar-margin)",opacity:0,transition:"opacity 150ms 300ms","&[data-hover]":{transitionDelay:"0ms",opacity:1}}},variants:{variant:{hover:{scrollbar:{opacity:"0","&[data-hover], &[data-scrolling]":{opacity:"1",transitionDuration:"faster",transitionDelay:"0ms"}}},always:{scrollbar:{opacity:"1"}}},size:{xs:{root:{"--scrollbar-size":"sizes.1"}},sm:{root:{"--scrollbar-size":"sizes.1.5"}},md:{root:{"--scrollbar-size":"sizes.2"}},lg:{root:{"--scrollbar-size":"sizes.3"}}}},defaultVariants:{size:"md",variant:"hover"}}),Mw=B({className:"chakra-segment-group",slots:Ad.keys(),base:{root:{"--segment-radius":"radii.l2",borderRadius:"l2",display:"inline-flex",boxShadow:"inset",minW:"max-content",textAlign:"center",position:"relative",isolation:"isolate",bg:"bg.muted",_vertical:{flexDirection:"column"}},item:{display:"flex",alignItems:"center",justifyContent:"center",userSelect:"none",fontSize:"sm",position:"relative",color:"fg",borderRadius:"var(--segment-radius)",_disabled:{opacity:"0.5"},"&:has(input:focus-visible)":{focusRing:"outside"},_before:{content:'""',position:"absolute",bg:"border",transition:"opacity 0.2s"},_horizontal:{_before:{insetInlineStart:0,insetBlock:"1.5",width:"1px"}},_vertical:{_before:{insetBlockStart:0,insetInline:"1.5",height:"1px"}},"& + &[data-state=checked], &[data-state=checked] + &, &:first-of-type":{_before:{opacity:"0"}},"&[data-state=checked][data-ssr]":{shadow:"sm",bg:"bg",borderRadius:"var(--segment-radius)"}},indicator:{shadow:"sm",pos:"absolute",bg:{_light:"bg",_dark:"bg.emphasized"},width:"var(--width)",height:"var(--height)",top:"var(--top)",left:"var(--left)",zIndex:-1,borderRadius:"var(--segment-radius)"}},variants:{size:{xs:{item:{textStyle:"xs",px:"3",gap:"1",height:"6"}},sm:{item:{textStyle:"sm",px:"4",gap:"2",height:"8"}},md:{item:{textStyle:"sm",px:"4",gap:"2",height:"10"}},lg:{item:{textStyle:"md",px:"4.5",gap:"3",height:"11"}}}},defaultVariants:{size:"md"}}),jw=B({className:"chakra-slider",slots:WS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",textStyle:"sm",position:"relative",isolation:"isolate",touchAction:"none"},label:{fontWeight:"medium",textStyle:"sm"},control:{display:"inline-flex",alignItems:"center",position:"relative"},track:{overflow:"hidden",borderRadius:"full",flex:"1"},range:{width:"inherit",height:"inherit",_disabled:{bg:"border.emphasized!"}},markerGroup:{position:"absolute!",zIndex:"1"},marker:{"--marker-bg":{base:"white",_underValue:"colors.bg"},display:"flex",alignItems:"center",gap:"calc(var(--slider-thumb-size) / 2)",color:"fg.muted",textStyle:"xs"},markerIndicator:{width:"var(--slider-marker-size)",height:"var(--slider-marker-size)",borderRadius:"full",bg:"var(--marker-bg)"},thumb:{width:"var(--slider-thumb-size)",height:"var(--slider-thumb-size)",display:"flex",alignItems:"center",justifyContent:"center",outline:0,zIndex:"2",borderRadius:"full",_focusVisible:{ring:"2px",ringColor:"colorPalette.focusRing",ringOffset:"2px",ringOffsetColor:"bg"}}},variants:{size:{sm:{root:{"--slider-thumb-size":"sizes.4","--slider-track-size":"sizes.1.5","--slider-marker-center":"6px","--slider-marker-size":"sizes.1","--slider-marker-inset":"3px"}},md:{root:{"--slider-thumb-size":"sizes.5","--slider-track-size":"sizes.2","--slider-marker-center":"8px","--slider-marker-size":"sizes.1","--slider-marker-inset":"4px"}},lg:{root:{"--slider-thumb-size":"sizes.6","--slider-track-size":"sizes.2.5","--slider-marker-center":"9px","--slider-marker-size":"sizes.1.5","--slider-marker-inset":"5px"}}},variant:{outline:{track:{shadow:"inset",bg:"bg.emphasized/72"},range:{bg:"colorPalette.solid"},thumb:{borderWidth:"2px",borderColor:"colorPalette.solid",bg:"bg",_disabled:{bg:"border.emphasized",borderColor:"border.emphasized"}}},solid:{track:{bg:"colorPalette.subtle",_disabled:{bg:"bg.muted"}},range:{bg:"colorPalette.solid"},thumb:{bg:"colorPalette.solid",_disabled:{bg:"border.emphasized"}}}},orientation:{vertical:{root:{display:"inline-flex"},control:{flexDirection:"column",height:"100%",minWidth:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginEnd:"4"}},track:{width:"var(--slider-track-size)"},thumb:{left:"50%",translate:"-50% 0"},markerGroup:{insetStart:"var(--slider-marker-center)",insetBlock:"var(--slider-marker-inset)"},marker:{flexDirection:"row"}},horizontal:{control:{flexDirection:"row",width:"100%",minHeight:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginBottom:"4"}},track:{height:"var(--slider-track-size)"},thumb:{top:"50%",translate:"0 -50%"},markerGroup:{top:"var(--slider-marker-center)",insetInline:"var(--slider-marker-inset)"},marker:{flexDirection:"column"}}}},defaultVariants:{size:"md",variant:"outline",orientation:"horizontal"}}),$w=B({className:"chakra-stat",slots:HS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",position:"relative",flex:"1"},label:{display:"inline-flex",gap:"1.5",alignItems:"center",color:"fg.muted",textStyle:"sm"},helpText:{color:"fg.muted",textStyle:"xs"},valueUnit:{color:"fg.muted",textStyle:"xs",fontWeight:"initial",letterSpacing:"initial"},valueText:{verticalAlign:"baseline",fontWeight:"semibold",letterSpacing:"tight",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums",display:"inline-flex",gap:"1"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",marginEnd:1,"& :where(svg)":{w:"1em",h:"1em"},"&[data-type=up]":{color:"fg.success"},"&[data-type=down]":{color:"fg.error"}}},variants:{size:{sm:{valueText:{textStyle:"xl"}},md:{valueText:{textStyle:"2xl"}},lg:{valueText:{textStyle:"3xl"}}}},defaultVariants:{size:"md"}}),Bw=B({className:"chakra-status",slots:US.keys(),base:{root:{display:"inline-flex",alignItems:"center",gap:"2"},indicator:{width:"0.64em",height:"0.64em",flexShrink:0,borderRadius:"full",forcedColorAdjust:"none",bg:"colorPalette.solid"}},variants:{size:{sm:{root:{textStyle:"xs"}},md:{root:{textStyle:"sm"}},lg:{root:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),Ww=B({className:"chakra-steps",slots:GS.keys(),base:{root:{display:"flex",width:"full"},list:{display:"flex",justifyContent:"space-between","--steps-gutter":"spacing.3","--steps-thickness":"2px"},title:{fontWeight:"medium",color:"fg"},description:{color:"fg.muted"},separator:{bg:"border",flex:"1"},indicator:{display:"flex",justifyContent:"center",alignItems:"center",flexShrink:"0",borderRadius:"full",fontWeight:"medium",width:"var(--steps-size)",height:"var(--steps-size)",_icon:{flexShrink:"0",width:"var(--steps-icon-size)",height:"var(--steps-icon-size)"}},item:{position:"relative",display:"flex",gap:"3",flex:"1 0 0","&:last-of-type":{flex:"initial","& [data-part=separator]":{display:"none"}}},trigger:{display:"flex",alignItems:"center",gap:"3",textAlign:"start",focusVisibleRing:"outside",borderRadius:"l2"},content:{focusVisibleRing:"outside"}},variants:{orientation:{vertical:{root:{flexDirection:"row",height:"100%"},list:{flexDirection:"column",alignItems:"flex-start"},separator:{position:"absolute",width:"var(--steps-thickness)",height:"100%",maxHeight:"calc(100% - var(--steps-size) - var(--steps-gutter) * 2)",top:"calc(var(--steps-size) + var(--steps-gutter))",insetStart:"calc(var(--steps-size) / 2 - 1px)"},item:{alignItems:"flex-start"}},horizontal:{root:{flexDirection:"column",width:"100%"},list:{flexDirection:"row",alignItems:"center"},separator:{width:"100%",height:"var(--steps-thickness)",marginX:"var(--steps-gutter)"},item:{alignItems:"center"}}},variant:{solid:{indicator:{_incomplete:{borderWidth:"var(--steps-thickness)"},_current:{bg:"colorPalette.muted",borderWidth:"var(--steps-thickness)",borderColor:"colorPalette.solid",color:"colorPalette.fg"},_complete:{bg:"colorPalette.solid",borderColor:"colorPalette.solid",color:"colorPalette.contrast"}},separator:{_complete:{bg:"colorPalette.solid"}}},subtle:{indicator:{_incomplete:{bg:"bg.muted"},_current:{bg:"colorPalette.muted",color:"colorPalette.fg"},_complete:{bg:"colorPalette.emphasized",color:"colorPalette.fg"}},separator:{_complete:{bg:"colorPalette.emphasized"}}}},size:{xs:{root:{gap:"2.5"},list:{"--steps-size":"sizes.6","--steps-icon-size":"sizes.3.5",textStyle:"xs"},title:{textStyle:"sm"}},sm:{root:{gap:"3"},list:{"--steps-size":"sizes.8","--steps-icon-size":"sizes.4",textStyle:"xs"},title:{textStyle:"sm"}},md:{root:{gap:"4"},list:{"--steps-size":"sizes.10","--steps-icon-size":"sizes.4",textStyle:"sm"},title:{textStyle:"sm"}},lg:{root:{gap:"6"},list:{"--steps-size":"sizes.11","--steps-icon-size":"sizes.5",textStyle:"md"},title:{textStyle:"md"}}}},defaultVariants:{size:"md",variant:"solid",orientation:"horizontal"}}),Hw=B({slots:qS.keys(),className:"chakra-switch",base:{root:{display:"inline-flex",gap:"2.5",alignItems:"center",position:"relative",verticalAlign:"middle","--switch-diff":"calc(var(--switch-width) - var(--switch-height))","--switch-x":{base:"var(--switch-diff)",_rtl:"calc(var(--switch-diff) * -1)"}},label:{lineHeight:"1",userSelect:"none",fontSize:"sm",fontWeight:"medium",_disabled:{opacity:"0.5"}},indicator:{position:"absolute",height:"var(--switch-height)",width:"var(--switch-height)",fontSize:"var(--switch-indicator-font-size)",fontWeight:"medium",flexShrink:0,userSelect:"none",display:"grid",placeContent:"center",transition:"inset-inline-start 0.12s ease",insetInlineStart:"calc(var(--switch-x) - 2px)",_checked:{insetInlineStart:"2px"}},control:{display:"inline-flex",gap:"0.5rem",flexShrink:0,justifyContent:"flex-start",cursor:"switch",borderRadius:"full",position:"relative",width:"var(--switch-width)",height:"var(--switch-height)",transition:"backgrounds",_disabled:{opacity:"0.5",cursor:"not-allowed"},_invalid:{outline:"2px solid",outlineColor:"border.error",outlineOffset:"2px"}},thumb:{display:"flex",alignItems:"center",justifyContent:"center",flexShrink:0,transitionProperty:"translate",transitionDuration:"fast",borderRadius:"inherit",_checked:{translate:"var(--switch-x) 0"}}},variants:{variant:{solid:{control:{borderRadius:"full",bg:"bg.emphasized",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}},thumb:{bg:"white",width:"var(--switch-height)",height:"var(--switch-height)",scale:"0.8",boxShadow:"sm",_checked:{bg:"colorPalette.contrast"}}},raised:{control:{borderRadius:"full",height:"calc(var(--switch-height) / 2)",bg:"bg.muted",boxShadow:"inset",_checked:{bg:"colorPalette.solid/60"}},thumb:{width:"var(--switch-height)",height:"var(--switch-height)",position:"relative",top:"calc(var(--switch-height) * -0.25)",bg:"white",boxShadow:"xs",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}}}},size:{xs:{root:{"--switch-width":"sizes.6","--switch-height":"sizes.3","--switch-indicator-font-size":"fontSizes.xs"}},sm:{root:{"--switch-width":"sizes.8","--switch-height":"sizes.4","--switch-indicator-font-size":"fontSizes.xs"}},md:{root:{"--switch-width":"sizes.10","--switch-height":"sizes.5","--switch-indicator-font-size":"fontSizes.sm"}},lg:{root:{"--switch-width":"sizes.12","--switch-height":"sizes.6","--switch-indicator-font-size":"fontSizes.md"}}}},defaultVariants:{variant:"solid",size:"md"}}),Uw=B({className:"chakra-table",slots:KS.keys(),base:{root:{fontVariantNumeric:"lining-nums tabular-nums",borderCollapse:"collapse",width:"full",textAlign:"start",verticalAlign:"top"},row:{_selected:{bg:"colorPalette.subtle"}},cell:{textAlign:"start",alignItems:"center"},columnHeader:{fontWeight:"medium",textAlign:"start",color:"fg"},caption:{fontWeight:"medium",textStyle:"xs"},footer:{fontWeight:"medium"}},variants:{interactive:{true:{body:{"& tr":{_hover:{bg:"colorPalette.subtle"}}}}},stickyHeader:{true:{header:{"& :where(tr)":{top:"var(--table-sticky-offset, 0)",position:"sticky",zIndex:1}}}},striped:{true:{row:{"&:nth-of-type(odd) td":{bg:"bg.muted"}}}},showColumnBorder:{true:{columnHeader:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}},cell:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}}}},variant:{line:{columnHeader:{borderBottomWidth:"1px"},cell:{borderBottomWidth:"1px"},row:{bg:"bg"}},outline:{root:{boxShadow:"0 0 0 1px {colors.border}",overflow:"hidden"},columnHeader:{borderBottomWidth:"1px"},header:{bg:"bg.muted"},row:{"&:not(:last-of-type)":{borderBottomWidth:"1px"}},footer:{borderTopWidth:"1px"}}},size:{sm:{root:{textStyle:"sm"},columnHeader:{px:"2",py:"2"},cell:{px:"2",py:"2"}},md:{root:{textStyle:"sm"},columnHeader:{px:"3",py:"3"},cell:{px:"3",py:"3"}},lg:{root:{textStyle:"md"},columnHeader:{px:"4",py:"3"},cell:{px:"4",py:"3"}}}},defaultVariants:{variant:"line",size:"md"}}),Gw=B({slots:XS.keys(),className:"chakra-tabs",base:{root:{"--tabs-trigger-radius":"radii.l2",position:"relative",_horizontal:{display:"block"},_vertical:{display:"flex"}},list:{display:"inline-flex",position:"relative",isolation:"isolate","--tabs-indicator-shadow":"shadows.xs","--tabs-indicator-bg":"colors.bg",minH:"var(--tabs-height)",_horizontal:{flexDirection:"row"},_vertical:{flexDirection:"column"}},trigger:{outline:"0",minW:"var(--tabs-height)",height:"var(--tabs-height)",display:"flex",alignItems:"center",fontWeight:"medium",position:"relative",cursor:"button",gap:"2",_focusVisible:{zIndex:1,outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{cursor:"not-allowed",opacity:.5}},content:{focusVisibleRing:"inside",_horizontal:{width:"100%",pt:"var(--tabs-content-padding)"},_vertical:{height:"100%",ps:"var(--tabs-content-padding)"}},indicator:{width:"var(--width)",height:"var(--height)",borderRadius:"var(--tabs-indicator-radius)",bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",zIndex:-1}},variants:{fitted:{true:{list:{display:"flex"},trigger:{flex:1,textAlign:"center",justifyContent:"center"}}},justify:{start:{list:{justifyContent:"flex-start"}},center:{list:{justifyContent:"center"}},end:{list:{justifyContent:"flex-end"}}},size:{sm:{root:{"--tabs-height":"sizes.9","--tabs-content-padding":"spacing.3"},trigger:{py:"1",px:"3",textStyle:"sm"}},md:{root:{"--tabs-height":"sizes.10","--tabs-content-padding":"spacing.4"},trigger:{py:"2",px:"4",textStyle:"sm"}},lg:{root:{"--tabs-height":"sizes.11","--tabs-content-padding":"spacing.4.5"},trigger:{py:"2",px:"4.5",textStyle:"md"}}},variant:{line:{list:{display:"flex",borderColor:"border",_horizontal:{borderBottomWidth:"1px"},_vertical:{borderEndWidth:"1px"}},trigger:{color:"fg.muted",_disabled:{_active:{bg:"initial"}},_selected:{color:"fg",_horizontal:{layerStyle:"indicator.bottom","--indicator-offset-y":"-1px","--indicator-color":"colors.colorPalette.solid"},_vertical:{layerStyle:"indicator.end","--indicator-offset-x":"-1px"}}}},subtle:{trigger:{borderRadius:"var(--tabs-trigger-radius)",color:"fg.muted",_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}},enclosed:{list:{bg:"bg.muted",padding:"1",borderRadius:"l3",minH:"calc(var(--tabs-height) - 4px)"},trigger:{justifyContent:"center",color:"fg.muted",borderRadius:"var(--tabs-trigger-radius)",_selected:{bg:"bg",color:"colorPalette.fg",shadow:"xs"}}},outline:{list:{"--line-thickness":"1px","--line-offset":"calc(var(--line-thickness) * -1)",borderColor:"border",display:"flex",_horizontal:{_before:{content:'""',position:"absolute",bottom:"0px",width:"100%",borderBottomWidth:"var(--line-thickness)",borderBottomColor:"border"}},_vertical:{_before:{content:'""',position:"absolute",insetInline:"var(--line-offset)",height:"calc(100% - calc(var(--line-thickness) * 2))",borderEndWidth:"var(--line-thickness)",borderEndColor:"border"}}},trigger:{color:"fg.muted",borderWidth:"1px",borderColor:"transparent",_selected:{bg:"currentBg",color:"colorPalette.fg"},_horizontal:{borderTopRadius:"var(--tabs-trigger-radius)",marginBottom:"var(--line-offset)",marginEnd:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderBottomColor:"transparent"}},_vertical:{borderStartRadius:"var(--tabs-trigger-radius)",marginEnd:"var(--line-offset)",marginBottom:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderEndColor:"transparent"}}}},plain:{trigger:{color:"fg.muted",_selected:{color:"colorPalette.fg"},borderRadius:"var(--tabs-trigger-radius)","&[data-selected][data-ssr]":{bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",borderRadius:"var(--tabs-indicator-radius)"}}}}},defaultVariants:{size:"md",variant:"line"}}),ci=Ea.variants?.variant,qw=B({slots:QS.keys(),className:"chakra-tag",base:{root:{display:"inline-flex",alignItems:"center",verticalAlign:"top",maxWidth:"100%",userSelect:"none",borderRadius:"l2",focusVisibleRing:"outside"},label:{lineClamp:"1"},closeTrigger:{display:"flex",alignItems:"center",justifyContent:"center",outline:"0",borderRadius:"l1",color:"currentColor",focusVisibleRing:"inside",focusRingWidth:"2px"},startElement:{flexShrink:0,boxSize:"var(--tag-element-size)",ms:"var(--tag-element-offset)","&:has([data-scope=avatar])":{boxSize:"var(--tag-avatar-size)",ms:"calc(var(--tag-element-offset) * 1.5)"},_icon:{boxSize:"100%"}},endElement:{flexShrink:0,boxSize:"var(--tag-element-size)",me:"var(--tag-element-offset)",_icon:{boxSize:"100%"},"&:has(button)":{ms:"calc(var(--tag-element-offset) * -1)"}}},variants:{size:{sm:{root:{px:"1.5",minH:"4.5",gap:"1","--tag-avatar-size":"spacing.3","--tag-element-size":"spacing.3","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},md:{root:{px:"1.5",minH:"5",gap:"1","--tag-avatar-size":"spacing.3.5","--tag-element-size":"spacing.3.5","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},lg:{root:{px:"2",minH:"6",gap:"1.5","--tag-avatar-size":"spacing.4.5","--tag-element-size":"spacing.4","--tag-element-offset":"-3px"},label:{textStyle:"sm"}},xl:{root:{px:"2.5",minH:"8",gap:"1.5","--tag-avatar-size":"spacing.6","--tag-element-size":"spacing.4.5","--tag-element-offset":"-4px"},label:{textStyle:"sm"}}},variant:{subtle:{root:ci?.subtle},solid:{root:ci?.solid},outline:{root:ci?.outline},surface:{root:ci?.surface}}},defaultVariants:{size:"md",variant:"surface"}}),Kw=B({slots:JS.keys(),className:"chakra-timeline",base:{root:{display:"flex",flexDirection:"column",width:"full","--timeline-thickness":"1px","--timeline-gutter":"4px"},item:{display:"flex",position:"relative",alignItems:"flex-start",flexShrink:0,gap:"4",_last:{"& :where(.chakra-timeline__separator)":{display:"none"}}},separator:{position:"absolute",borderStartWidth:"var(--timeline-thickness)",ms:"calc(-1 * var(--timeline-thickness) / 2)",insetInlineStart:"calc(var(--timeline-indicator-size) / 2)",insetBlock:"0",borderColor:"border"},indicator:{outline:"2px solid {colors.bg}",position:"relative",flexShrink:"0",boxSize:"var(--timeline-indicator-size)",fontSize:"var(--timeline-font-size)",display:"flex",alignItems:"center",justifyContent:"center",borderRadius:"full",fontWeight:"medium"},connector:{alignSelf:"stretch",position:"relative"},content:{pb:"6",display:"flex",flexDirection:"column",width:"full",gap:"2"},title:{display:"flex",fontWeight:"medium",flexWrap:"wrap",gap:"1.5",alignItems:"center",mt:"var(--timeline-margin)"},description:{color:"fg.muted",textStyle:"xs"}},variants:{variant:{subtle:{indicator:{bg:"colorPalette.muted"}},solid:{indicator:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},outline:{indicator:{bg:"currentBg",borderWidth:"1px",borderColor:"colorPalette.muted"}},plain:{}},size:{sm:{root:{"--timeline-indicator-size":"sizes.4","--timeline-font-size":"fontSizes.2xs"},title:{textStyle:"xs"}},md:{root:{"--timeline-indicator-size":"sizes.5","--timeline-font-size":"fontSizes.xs"},title:{textStyle:"sm"}},lg:{root:{"--timeline-indicator-size":"sizes.6","--timeline-font-size":"fontSizes.xs"},title:{mt:"0.5",textStyle:"sm"}},xl:{root:{"--timeline-indicator-size":"sizes.8","--timeline-font-size":"fontSizes.sm"},title:{mt:"1.5",textStyle:"sm"}}}},defaultVariants:{size:"md",variant:"solid"}}),Yw=B({slots:YS.keys(),className:"chakra-toast",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",gap:"3",py:"4",ps:"4",pe:"6",borderRadius:"l2",translate:"var(--x) var(--y)",scale:"var(--scale)",zIndex:"var(--z-index)",height:"var(--height)",opacity:"var(--opacity)",willChange:"translate, opacity, scale",transition:"translate 400ms, scale 400ms, opacity 400ms, height 400ms, box-shadow 200ms",transitionTimingFunction:"cubic-bezier(0.21, 1.02, 0.73, 1)",_closed:{transition:"translate 400ms, scale 400ms, opacity 200ms",transitionTimingFunction:"cubic-bezier(0.06, 0.71, 0.55, 1)"},bg:"bg.panel",color:"fg",boxShadow:"xl","--toast-trigger-bg":"colors.bg.muted","&[data-type=warning]":{bg:"orange.solid",color:"orange.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=success]":{bg:"green.solid",color:"green.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=error]":{bg:"red.solid",color:"red.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"}},title:{fontWeight:"medium",textStyle:"sm",marginEnd:"2"},description:{display:"inline",textStyle:"sm",opacity:"0.8"},indicator:{flexShrink:"0",boxSize:"5"},actionTrigger:{textStyle:"sm",fontWeight:"medium",height:"8",px:"3",borderRadius:"l2",alignSelf:"center",borderWidth:"1px",borderColor:"var(--toast-border-color, inherit)",transition:"background 200ms",_hover:{bg:"var(--toast-trigger-bg)"}},closeTrigger:{position:"absolute",top:"1",insetEnd:"1",padding:"1",display:"inline-flex",alignItems:"center",justifyContent:"center",color:"{currentColor/60}",borderRadius:"l2",textStyle:"md",transition:"background 200ms",_icon:{boxSize:"1em"}}}}),Xw=B({slots:Md.keys(),className:"chakra-tooltip",base:{content:{"--tooltip-bg":"colors.bg.inverted",bg:"var(--tooltip-bg)",color:"fg.inverted",px:"2.5",py:"1",borderRadius:"l2",fontWeight:"medium",textStyle:"xs",boxShadow:"md",maxW:"xs",zIndex:"tooltip",transformOrigin:"var(--transform-origin)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"fast"}},arrow:{"--arrow-size":"sizes.2","--arrow-background":"var(--tooltip-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px",borderColor:"var(--tooltip-bg)"}}}),zh=Zn({display:"flex",alignItems:"center",gap:"var(--tree-item-gap)",rounded:"l2",userSelect:"none",position:"relative","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-icon-offset":"calc(var(--tree-icon-size) * var(--tree-depth) * 0.5)","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset) + var(--tree-icon-offset))",ps:"var(--tree-offset)",pe:"var(--tree-padding-inline)",py:"var(--tree-padding-block)",focusVisibleRing:"inside",focusRingColor:"border.emphasized",focusRingWidth:"2px","&:hover, &:focus-visible":{bg:"bg.muted"},_disabled:{layerStyle:"disabled"}}),Mh=Zn({flex:"1"}),jh=Zn({_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}),$h=Zn({_selected:{layerStyle:"fill.solid"}}),Qw=B({slots:ld.keys(),className:"chakra-tree-view",base:{root:{width:"full",display:"flex",flexDirection:"column",gap:"2"},tree:{display:"flex",flexDirection:"column","--tree-item-gap":"spacing.2",_icon:{boxSize:"var(--tree-icon-size)"}},label:{fontWeight:"medium",textStyle:"sm"},branch:{position:"relative"},branchContent:{position:"relative"},branchIndentGuide:{height:"100%",width:"1px",bg:"border",position:"absolute","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset))","--tree-icon-offset":"calc(var(--tree-icon-size) * 0.5 * var(--depth))",insetInlineStart:"calc(var(--tree-offset) + var(--tree-icon-offset))",zIndex:"1"},branchIndicator:{color:"fg.muted",transformOrigin:"center",transitionDuration:"normal",transitionProperty:"transform",transitionTimingFunction:"default",_open:{transform:"rotate(90deg)"}},branchTrigger:{display:"inline-flex",alignItems:"center",justifyContent:"center"},branchControl:zh,item:zh,itemText:Mh,branchText:Mh,nodeCheckbox:{display:"inline-flex"}},variants:{size:{md:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1.5","--tree-icon-size":"spacing.4"}},sm:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}},xs:{tree:{textStyle:"xs","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.2","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}}},variant:{subtle:{branchControl:jh,item:jh},solid:{branchControl:$h,item:$h}},animateContent:{true:{branchContent:{_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}},defaultVariants:{size:"md",variant:"subtle"}}),Jw={accordion:nw,actionBar:rw,alert:ow,avatar:iw,blockquote:sw,breadcrumb:aw,card:lw,checkbox:cw,checkboxCard:uw,codeBlock:dw,collapsible:hw,dataList:pw,dialog:mw,drawer:vw,editable:bw,emptyState:yw,field:xw,fieldset:kw,fileUpload:Cw,hoverCard:Sw,list:ww,listbox:Ew,menu:Ow,nativeSelect:Pw,numberInput:Iw,pinInput:Tw,popover:Nw,progress:Aw,progressCircle:_w,radioCard:Fw,radioGroup:Lw,ratingGroup:Dw,scrollArea:zw,segmentGroup:Mw,select:li,combobox:gw,slider:jw,stat:$w,steps:Ww,switch:Hw,table:Uw,tabs:Gw,tag:qw,toast:Yw,tooltip:Xw,status:Bw,timeline:Kw,colorPicker:fw,qrCode:Vw,treeView:Qw},Zw=Vk({"2xs":{value:{fontSize:"2xs",lineHeight:"0.75rem"}},xs:{value:{fontSize:"xs",lineHeight:"1rem"}},sm:{value:{fontSize:"sm",lineHeight:"1.25rem"}},md:{value:{fontSize:"md",lineHeight:"1.5rem"}},lg:{value:{fontSize:"lg",lineHeight:"1.75rem"}},xl:{value:{fontSize:"xl",lineHeight:"1.875rem"}},"2xl":{value:{fontSize:"2xl",lineHeight:"2rem"}},"3xl":{value:{fontSize:"3xl",lineHeight:"2.375rem"}},"4xl":{value:{fontSize:"4xl",lineHeight:"2.75rem",letterSpacing:"-0.025em"}},"5xl":{value:{fontSize:"5xl",lineHeight:"3.75rem",letterSpacing:"-0.025em"}},"6xl":{value:{fontSize:"6xl",lineHeight:"4.5rem",letterSpacing:"-0.025em"}},"7xl":{value:{fontSize:"7xl",lineHeight:"5.75rem",letterSpacing:"-0.025em"}},none:{value:{}},label:{value:{fontSize:"sm",lineHeight:"1.25rem",fontWeight:"medium"}}}),eE=de.animations({spin:{value:"spin 1s linear infinite"},ping:{value:"ping 1s cubic-bezier(0, 0, 0.2, 1) infinite"},pulse:{value:"pulse 2s cubic-bezier(0.4, 0, 0.6, 1) infinite"},bounce:{value:"bounce 1s infinite"}}),tE=de.aspectRatios({square:{value:"1 / 1"},landscape:{value:"4 / 3"},portrait:{value:"3 / 4"},wide:{value:"16 / 9"},ultrawide:{value:"18 / 5"},golden:{value:"1.618 / 1"}}),nE=de.blurs({none:{value:" "},sm:{value:"4px"},md:{value:"8px"},lg:{value:"12px"},xl:{value:"16px"},"2xl":{value:"24px"},"3xl":{value:"40px"},"4xl":{value:"64px"}}),rE=de.borders({xs:{value:"0.5px solid"},sm:{value:"1px solid"},md:{value:"2px solid"},lg:{value:"4px solid"},xl:{value:"8px solid"}}),oE=de.colors({transparent:{value:"transparent"},current:{value:"currentColor"},black:{value:"#09090B"},white:{value:"#FFFFFF"},whiteAlpha:{50:{value:"rgba(255, 255, 255, 0.04)"},100:{value:"rgba(255, 255, 255, 0.06)"},200:{value:"rgba(255, 255, 255, 0.08)"},300:{value:"rgba(255, 255, 255, 0.16)"},400:{value:"rgba(255, 255, 255, 0.24)"},500:{value:"rgba(255, 255, 255, 0.36)"},600:{value:"rgba(255, 255, 255, 0.48)"},700:{value:"rgba(255, 255, 255, 0.64)"},800:{value:"rgba(255, 255, 255, 0.80)"},900:{value:"rgba(255, 255, 255, 0.92)"},950:{value:"rgba(255, 255, 255, 0.95)"}},blackAlpha:{50:{value:"rgba(0, 0, 0, 0.04)"},100:{value:"rgba(0, 0, 0, 0.06)"},200:{value:"rgba(0, 0, 0, 0.08)"},300:{value:"rgba(0, 0, 0, 0.16)"},400:{value:"rgba(0, 0, 0, 0.24)"},500:{value:"rgba(0, 0, 0, 0.36)"},600:{value:"rgba(0, 0, 0, 0.48)"},700:{value:"rgba(0, 0, 0, 0.64)"},800:{value:"rgba(0, 0, 0, 0.80)"},900:{value:"rgba(0, 0, 0, 0.92)"},950:{value:"rgba(0, 0, 0, 0.95)"}},gray:{50:{value:"#fafafa"},100:{value:"#f4f4f5"},200:{value:"#e4e4e7"},300:{value:"#d4d4d8"},400:{value:"#a1a1aa"},500:{value:"#71717a"},600:{value:"#52525b"},700:{value:"#3f3f46"},800:{value:"#27272a"},900:{value:"#18181b"},950:{value:"#111111"}},red:{50:{value:"#fef2f2"},100:{value:"#fee2e2"},200:{value:"#fecaca"},300:{value:"#fca5a5"},400:{value:"#f87171"},500:{value:"#ef4444"},600:{value:"#dc2626"},700:{value:"#991919"},800:{value:"#511111"},900:{value:"#300c0c"},950:{value:"#1f0808"}},orange:{50:{value:"#fff7ed"},100:{value:"#ffedd5"},200:{value:"#fed7aa"},300:{value:"#fdba74"},400:{value:"#fb923c"},500:{value:"#f97316"},600:{value:"#ea580c"},700:{value:"#92310a"},800:{value:"#6c2710"},900:{value:"#3b1106"},950:{value:"#220a04"}},yellow:{50:{value:"#fefce8"},100:{value:"#fef9c3"},200:{value:"#fef08a"},300:{value:"#fde047"},400:{value:"#facc15"},500:{value:"#eab308"},600:{value:"#ca8a04"},700:{value:"#845209"},800:{value:"#713f12"},900:{value:"#422006"},950:{value:"#281304"}},green:{50:{value:"#f0fdf4"},100:{value:"#dcfce7"},200:{value:"#bbf7d0"},300:{value:"#86efac"},400:{value:"#4ade80"},500:{value:"#22c55e"},600:{value:"#16a34a"},700:{value:"#116932"},800:{value:"#124a28"},900:{value:"#042713"},950:{value:"#03190c"}},teal:{50:{value:"#f0fdfa"},100:{value:"#ccfbf1"},200:{value:"#99f6e4"},300:{value:"#5eead4"},400:{value:"#2dd4bf"},500:{value:"#14b8a6"},600:{value:"#0d9488"},700:{value:"#0c5d56"},800:{value:"#114240"},900:{value:"#032726"},950:{value:"#021716"}},blue:{50:{value:"#eff6ff"},100:{value:"#dbeafe"},200:{value:"#bfdbfe"},300:{value:"#a3cfff"},400:{value:"#60a5fa"},500:{value:"#3b82f6"},600:{value:"#2563eb"},700:{value:"#173da6"},800:{value:"#1a3478"},900:{value:"#14204a"},950:{value:"#0c142e"}},cyan:{50:{value:"#ecfeff"},100:{value:"#cffafe"},200:{value:"#a5f3fc"},300:{value:"#67e8f9"},400:{value:"#22d3ee"},500:{value:"#06b6d4"},600:{value:"#0891b2"},700:{value:"#0c5c72"},800:{value:"#134152"},900:{value:"#072a38"},950:{value:"#051b24"}},purple:{50:{value:"#faf5ff"},100:{value:"#f3e8ff"},200:{value:"#e9d5ff"},300:{value:"#d8b4fe"},400:{value:"#c084fc"},500:{value:"#a855f7"},600:{value:"#9333ea"},700:{value:"#641ba3"},800:{value:"#4a1772"},900:{value:"#2f0553"},950:{value:"#1a032e"}},pink:{50:{value:"#fdf2f8"},100:{value:"#fce7f3"},200:{value:"#fbcfe8"},300:{value:"#f9a8d4"},400:{value:"#f472b6"},500:{value:"#ec4899"},600:{value:"#db2777"},700:{value:"#a41752"},800:{value:"#6d0e34"},900:{value:"#45061f"},950:{value:"#2c0514"}}}),iE=de.cursor({button:{value:"pointer"},checkbox:{value:"default"},disabled:{value:"not-allowed"},menuitem:{value:"default"},option:{value:"default"},radio:{value:"default"},slider:{value:"default"},switch:{value:"pointer"}}),sE=de.durations({fastest:{value:"50ms"},faster:{value:"100ms"},fast:{value:"150ms"},moderate:{value:"200ms"},slow:{value:"300ms"},slower:{value:"400ms"},slowest:{value:"500ms"}}),aE=de.easings({"ease-in":{value:"cubic-bezier(0.42, 0, 1, 1)"},"ease-out":{value:"cubic-bezier(0, 0, 0.58, 1)"},"ease-in-out":{value:"cubic-bezier(0.42, 0, 0.58, 1)"},"ease-in-smooth":{value:"cubic-bezier(0.32, 0.72, 0, 1)"}}),lE=de.fontSizes({"2xs":{value:"0.625rem"},xs:{value:"0.75rem"},sm:{value:"0.875rem"},md:{value:"1rem"},lg:{value:"1.125rem"},xl:{value:"1.25rem"},"2xl":{value:"1.5rem"},"3xl":{value:"1.875rem"},"4xl":{value:"2.25rem"},"5xl":{value:"3rem"},"6xl":{value:"3.75rem"},"7xl":{value:"4.5rem"},"8xl":{value:"6rem"},"9xl":{value:"8rem"}}),cE=de.fontWeights({thin:{value:"100"},extralight:{value:"200"},light:{value:"300"},normal:{value:"400"},medium:{value:"500"},semibold:{value:"600"},bold:{value:"700"},extrabold:{value:"800"},black:{value:"900"}}),Bh='-apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol"',uE=de.fonts({heading:{value:`Inter, ${Bh}`},body:{value:`Inter, ${Bh}`},mono:{value:'SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace'}}),dE=Ak({spin:{"0%":{transform:"rotate(0deg)"},"100%":{transform:"rotate(360deg)"}},pulse:{"50%":{opacity:"0.5"}},ping:{"75%, 100%":{transform:"scale(2)",opacity:"0"}},bounce:{"0%, 100%":{transform:"translateY(-25%)",animationTimingFunction:"cubic-bezier(0.8,0,1,1)"},"50%":{transform:"none",animationTimingFunction:"cubic-bezier(0,0,0.2,1)"}},"bg-position":{from:{backgroundPosition:"var(--animate-from, 1rem) 0"},to:{backgroundPosition:"var(--animate-to, 0) 0"}},position:{from:{insetInlineStart:"var(--animate-from-x)",insetBlockStart:"var(--animate-from-y)"},to:{insetInlineStart:"var(--animate-to-x)",insetBlockStart:"var(--animate-to-y)"}},"circular-progress":{"0%":{strokeDasharray:"1, 400",strokeDashoffset:"0"},"50%":{strokeDasharray:"400, 400",strokeDashoffset:"-100%"},"100%":{strokeDasharray:"400, 400",strokeDashoffset:"-260%"}},"expand-height":{from:{height:"0"},to:{height:"var(--height)"}},"collapse-height":{from:{height:"var(--height)"},to:{height:"0"}},"expand-width":{from:{width:"0"},to:{width:"var(--width)"}},"collapse-width":{from:{height:"var(--width)"},to:{height:"0"}},"fade-in":{from:{opacity:0},to:{opacity:1}},"fade-out":{from:{opacity:1},to:{opacity:0}},"slide-from-left-full":{from:{translate:"-100% 0"},to:{translate:"0 0"}},"slide-from-right-full":{from:{translate:"100% 0"},to:{translate:"0 0"}},"slide-from-top-full":{from:{translate:"0 -100%"},to:{translate:"0 0"}},"slide-from-bottom-full":{from:{translate:"0 100%"},to:{translate:"0 0"}},"slide-to-left-full":{from:{translate:"0 0"},to:{translate:"-100% 0"}},"slide-to-right-full":{from:{translate:"0 0"},to:{translate:"100% 0"}},"slide-to-top-full":{from:{translate:"0 0"},to:{translate:"0 -100%"}},"slide-to-bottom-full":{from:{translate:"0 0"},to:{translate:"0 100%"}},"slide-from-top":{"0%":{translate:"0 -0.5rem"},to:{translate:"0"}},"slide-from-bottom":{"0%":{translate:"0 0.5rem"},to:{translate:"0"}},"slide-from-left":{"0%":{translate:"-0.5rem 0"},to:{translate:"0"}},"slide-from-right":{"0%":{translate:"0.5rem 0"},to:{translate:"0"}},"slide-to-top":{"0%":{translate:"0"},to:{translate:"0 -0.5rem"}},"slide-to-bottom":{"0%":{translate:"0"},to:{translate:"0 0.5rem"}},"slide-to-left":{"0%":{translate:"0"},to:{translate:"-0.5rem 0"}},"slide-to-right":{"0%":{translate:"0"},to:{translate:"0.5rem 0"}},"scale-in":{from:{scale:"0.95"},to:{scale:"1"}},"scale-out":{from:{scale:"1"},to:{scale:"0.95"}}}),hE=de.letterSpacings({tighter:{value:"-0.05em"},tight:{value:"-0.025em"},wide:{value:"0.025em"},wider:{value:"0.05em"},widest:{value:"0.1em"}}),fE=de.lineHeights({shorter:{value:1.25},short:{value:1.375},moderate:{value:1.5},tall:{value:1.625},taller:{value:2}}),gE=de.radii({none:{value:"0"},"2xs":{value:"0.0625rem"},xs:{value:"0.125rem"},sm:{value:"0.25rem"},md:{value:"0.375rem"},lg:{value:"0.5rem"},xl:{value:"0.75rem"},"2xl":{value:"1rem"},"3xl":{value:"1.5rem"},"4xl":{value:"2rem"},full:{value:"9999px"}}),Wh=de.spacing({.5:{value:"0.125rem"},1:{value:"0.25rem"},1.5:{value:"0.375rem"},2:{value:"0.5rem"},2.5:{value:"0.625rem"},3:{value:"0.75rem"},3.5:{value:"0.875rem"},4:{value:"1rem"},4.5:{value:"1.125rem"},5:{value:"1.25rem"},6:{value:"1.5rem"},7:{value:"1.75rem"},8:{value:"2rem"},9:{value:"2.25rem"},10:{value:"2.5rem"},11:{value:"2.75rem"},12:{value:"3rem"},14:{value:"3.5rem"},16:{value:"4rem"},20:{value:"5rem"},24:{value:"6rem"},28:{value:"7rem"},32:{value:"8rem"},36:{value:"9rem"},40:{value:"10rem"},44:{value:"11rem"},48:{value:"12rem"},52:{value:"13rem"},56:{value:"14rem"},60:{value:"15rem"},64:{value:"16rem"},72:{value:"18rem"},80:{value:"20rem"},96:{value:"24rem"}}),pE=de.sizes({"3xs":{value:"14rem"},"2xs":{value:"16rem"},xs:{value:"20rem"},sm:{value:"24rem"},md:{value:"28rem"},lg:{value:"32rem"},xl:{value:"36rem"},"2xl":{value:"42rem"},"3xl":{value:"48rem"},"4xl":{value:"56rem"},"5xl":{value:"64rem"},"6xl":{value:"72rem"},"7xl":{value:"80rem"},"8xl":{value:"90rem"}}),mE=de.sizes({max:{value:"max-content"},min:{value:"min-content"},fit:{value:"fit-content"},prose:{value:"60ch"},full:{value:"100%"},dvh:{value:"100dvh"},svh:{value:"100svh"},lvh:{value:"100lvh"},dvw:{value:"100dvw"},svw:{value:"100svw"},lvw:{value:"100lvw"},vw:{value:"100vw"},vh:{value:"100vh"}}),vE=de.sizes({"1/2":{value:"50%"},"1/3":{value:"33.333333%"},"2/3":{value:"66.666667%"},"1/4":{value:"25%"},"3/4":{value:"75%"},"1/5":{value:"20%"},"2/5":{value:"40%"},"3/5":{value:"60%"},"4/5":{value:"80%"},"1/6":{value:"16.666667%"},"2/6":{value:"33.333333%"},"3/6":{value:"50%"},"4/6":{value:"66.666667%"},"5/6":{value:"83.333333%"},"1/12":{value:"8.333333%"},"2/12":{value:"16.666667%"},"3/12":{value:"25%"},"4/12":{value:"33.333333%"},"5/12":{value:"41.666667%"},"6/12":{value:"50%"},"7/12":{value:"58.333333%"},"8/12":{value:"66.666667%"},"9/12":{value:"75%"},"10/12":{value:"83.333333%"},"11/12":{value:"91.666667%"}}),bE=de.sizes({...pE,...Wh,...vE,...mE}),yE=de.zIndex({hide:{value:-1},base:{value:0},docked:{value:10},dropdown:{value:1e3},sticky:{value:1100},banner:{value:1200},overlay:{value:1300},modal:{value:1400},popover:{value:1500},skipNav:{value:1600},toast:{value:1700},tooltip:{value:1800},max:{value:2147483647}}),xE=va({preflight:!0,cssVarsPrefix:"chakra",cssVarsRoot:":where(html, .chakra-theme)",globalCss:QC,theme:{breakpoints:XC,keyframes:dE,tokens:{aspectRatios:tE,animations:eE,blurs:nE,borders:rE,colors:oE,durations:sE,easings:aE,fonts:uE,fontSizes:lE,fontWeights:cE,letterSpacings:hE,lineHeights:fE,radii:gE,spacing:Wh,sizes:bE,zIndex:yE,cursor:iE},semanticTokens:{colors:vS,shadows:yS,radii:bS},recipes:mS,slotRecipes:Jw,textStyles:Zw,layerStyles:JC,animationStyles:ZC}}),Hh=Hd(Hk,xE);Rh(Hh);function kE(e){const{key:t,recipe:n}=e,r=lo();return P.useMemo(()=>{const o=n||(t!=null?r.getSlotRecipe(t):{});return r.sva(structuredClone(o))},[t,n,r])}const CE=e=>e.charAt(0).toUpperCase()+e.slice(1),ui=e=>{const{key:t,recipe:n}=e,r=CE(t||n.className||"Component"),[o,i]=_n({name:`${r}StylesContext`,errorMessage:`use${r}Styles returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `}),[s,a]=_n({name:`${r}ClassNameContext`,errorMessage:`use${r}ClassNames returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `,strict:!1}),[l,c]=_n({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`,defaultValue:{}});function u(g){const{unstyled:p,...v}=g,y=kE({key:t,recipe:v.recipe||n}),[x,b]=P.useMemo(()=>y.splitVariantProps(v),[v,y]);return{styles:P.useMemo(()=>p?Nm:y(x),[p,x,y]),classNames:y.classNameMap,props:b}}function d(g,p={}){const{defaultProps:v}=p,y=x=>{const b=c(),C=P.useMemo(()=>mr(v,b,x),[b,x]),{styles:k,classNames:E,props:I}=u(C);return f.jsx(o,{value:k,children:f.jsx(s,{value:E,children:f.jsx(g,{...I})})})};return y.displayName=g.displayName||g.name,y}return{StylesProvider:o,ClassNamesProvider:s,PropsProvider:l,usePropsContext:c,useRecipeResult:u,withProvider:(g,p,v)=>{const{defaultProps:y,...x}=v??{},b=ve(g,{},x),C=P.forwardRef((k,E)=>{const I=c(),R=P.useMemo(()=>mr(y??{},I,k),[I,k]),{styles:_,props:A,classNames:T}=u(R),S=T[p],N=f.jsx(o,{value:_,children:f.jsx(s,{value:T,children:f.jsx(b,{ref:E,...A,css:[_[p],R.css],className:Ge(R.className,S)})})});return v?.wrapElement?.(N,R)??N});return C.displayName=g.displayName||g.name,C},withContext:(g,p,v)=>{const y=ve(g,{},v),x=P.forwardRef((b,C)=>{const{unstyled:k,...E}=b,I=i(),_=a()?.[p];return f.jsx(y,{...E,css:[!k&&p?I[p]:void 0,b.css],ref:C,className:Ge(b.className,_)})});return x.displayName=g.displayName||g.name,x},withRootProvider:d,useStyles:i,useClassNames:a}},Uh=ve("div",{base:{position:"absolute",display:"flex",alignItems:"center",justifyContent:"center"},variants:{axis:{horizontal:{insetStart:"50%",translate:"-50%",_rtl:{translate:"50%"}},vertical:{top:"50%",translate:"0 -50%"},both:{insetStart:"50%",top:"50%",translate:"-50% -50%",_rtl:{translate:"50% -50%"}}}},defaultVariants:{axis:"both"}});Uh.displayName="AbsoluteCenter";const SE=e=>f.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:f.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11.0026 16L6.75999 11.7574L8.17421 10.3431L11.0026 13.1716L16.6595 7.51472L18.0737 8.92893L11.0026 16Z"})}),Gh=e=>f.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:f.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 15H13V17H11V15ZM11 7H13V13H11V7Z"})}),qh=e=>f.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",stroke:"currentColor",strokeWidth:"0",...e,children:f.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 7H13V9H11V7ZM11 11H13V17H11V11Z"})}),wE=e=>f.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",...e,children:f.jsx("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M18.7071 6.70711C19.0976 6.31658 19.0976 5.68342 18.7071 5.29289C18.3166 4.90237 17.6834 4.90237 17.2929 5.29289L12 10.5858L6.70711 5.29289C6.31658 4.90237 5.68342 4.90237 5.29289 5.29289C4.90237 5.68342 4.90237 6.31658 5.29289 6.70711L10.5858 12L5.29289 17.2929C4.90237 17.6834 4.90237 18.3166 5.29289 18.7071C5.68342 19.0976 6.31658 19.0976 6.70711 18.7071L12 13.4142L17.2929 18.7071C17.6834 19.0976 18.3166 19.0976 18.7071 18.7071C19.0976 18.3166 19.0976 17.6834 18.7071 17.2929L13.4142 12L18.7071 6.70711Z"})}),[EE,OE]=_n({name:"AlertStatusContext",hookName:"useAlertStatusContext",providerName:""}),{withProvider:PE,withContext:Pa,useStyles:IE}=ui({key:"alert"}),RE=PE("div","root",{forwardAsChild:!0,wrapElement(e,t){return f.jsx(EE,{value:{status:t.status||"info"},children:e})}}),Kh=Pa("div","title"),TE=Pa("div","description"),NE=Pa("div","content"),AE={info:qh,warning:Gh,success:SE,error:Gh,neutral:qh},_E=P.forwardRef(function(t,n){const r=OE(),o=IE(),i=typeof r.status=="string"?AE[r.status]:P.Fragment,{children:s=f.jsx(i,{}),...a}=t;return f.jsx(ve.span,{ref:n,...a,css:[o.indicator,t.css],children:s})}),VE=e=>e?"":void 0,{withContext:FE}=Nt({key:"badge"}),Yh=FE("span"),{withContext:LE}=Nt({key:"spinner"}),DE=LE("span"),zE=O.forwardRef(function(t,n){const{spinner:r=f.jsx(DE,{size:"inherit",borderWidth:"0.125em",color:"inherit"}),spinnerPlacement:o="start",children:i,text:s,visible:a=!0,...l}=t;return a?s?f.jsxs(No,{ref:n,display:"contents",...l,children:[o==="start"&&r,s,o==="end"&&r]}):r?f.jsxs(No,{ref:n,display:"contents",...l,children:[f.jsx(Uh,{display:"inline-flex",children:r}),f.jsx(No,{visibility:"hidden",display:"contents",children:i})]}):f.jsx(No,{ref:n,display:"contents",...l,children:i}):i}),{useRecipeResult:ME,usePropsContext:jE}=Nt({key:"button"}),Ie=P.forwardRef(function(t,n){const r=jE(),o=P.useMemo(()=>mr(r,t),[r,t]),i=ME(o),{loading:s,loadingText:a,children:l,spinner:c,spinnerPlacement:u,...d}=i.props;return f.jsx(ve.button,{type:"button",ref:n,...d,"data-loading":VE(s),disabled:s||d.disabled,className:Ge(i.className,o.className),css:[i.styles,o.css],children:!o.asChild&&s?f.jsx(zE,{spinner:c,text:a,spinnerPlacement:u,children:l}):l})}),St=P.forwardRef(function(t,n){return f.jsx(Ie,{px:"0",py:"0",_icon:{fontSize:"1.2em"},ref:n,...t})}),dn=O.forwardRef(function(t,n){return f.jsx(St,{variant:"ghost","aria-label":"Close",ref:n,...t,children:t.children??f.jsx(wE,{})})}),Xh=ve("div",{base:{display:"flex",alignItems:"center",justifyContent:"center"},variants:{inline:{true:{display:"inline-flex"}}}});Xh.displayName="Center";const{withContext:$E}=Nt({key:"code"}),BE=$E("code");function WE(e){const{gap:t,direction:n}=e,r={column:{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},"column-reverse":{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},row:{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0},"row-reverse":{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0}};return{"&":Wd(n,o=>r[o])}}function HE(e){return P.Children.toArray(e).filter(t=>P.isValidElement(t))}const Qh=P.forwardRef(function(t,n){const{direction:r="column",align:o,justify:i,gap:s="0.5rem",wrap:a,children:l,separator:c,className:u,...d}=t,h=P.useMemo(()=>WE({gap:s,direction:r}),[s,r]),m=P.useMemo(()=>P.isValidElement(c)?HE(l).map((g,p,v)=>{const y=typeof g.key<"u"?g.key:p,x=c,b=P.cloneElement(x,{css:[h,x.props.css]});return f.jsxs(P.Fragment,{children:[g,p===v.length-1?null:b]},y)}):l,[l,c,h]);return f.jsx(ve.div,{ref:n,display:"flex",alignItems:o,justifyContent:i,flexDirection:r,flexWrap:a,gap:c?void 0:s,className:Ge("chakra-stack",u),...d,children:m})}),{withRootProvider:Jh,withContext:pt}=ui({key:"dialog"});Jh(m1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});const hn=Jh(p1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});pt(Ju,"trigger",{forwardAsChild:!0});const fn=pt(Hu,"positioner",{forwardAsChild:!0}),gn=pt(Bu,"content",{forwardAsChild:!0});pt(Wu,"description",{forwardAsChild:!0});const pn=pt(Qu,"title",{forwardAsChild:!0}),mn=pt($u,"closeTrigger",{forwardAsChild:!0}),vn=P.forwardRef(function(t,n){const r=Mt();return f.jsx(ve.button,{...t,ref:n,onClick:()=>r.setOpen(!1)})}),bn=pt(ju,"backdrop",{forwardAsChild:!0}),yn=pt("div","body"),xn=pt("div","footer"),kn=pt("div","header");function UE(e){const{each:t,fallback:n,children:r}=e;return t?.length===0?n||null:t?.map(r)}const di=P.forwardRef(function(t,n){const{direction:r,align:o,justify:i,wrap:s,basis:a,grow:l,shrink:c,inline:u,...d}=t;return f.jsx(ve.div,{ref:n,...d,css:{display:u?"inline-flex":"flex",flexDirection:r,alignItems:o,justifyContent:i,flexWrap:s,flexBasis:a,flexGrow:l,flexShrink:c,...t.css}})}),{withContext:GE}=Nt({key:"input"}),qE=GE(ed),{withContext:KE}=Nt({key:"link"}),jr=KE("a"),{withProvider:YE,withContext:Zh}=ui({key:"list"}),ef=YE("ul","root",{defaultProps:{role:"list"}}),tf=Zh("li","item");Zh("span","indicator");const XE=P.forwardRef(function(t,n){return f.jsx(Qh,{align:"center",...t,direction:"row",ref:n})}),hi=P.forwardRef(function(t,n){return f.jsx(Qh,{align:"center",...t,direction:"column",ref:n})}),{StylesProvider:QE,ClassNamesProvider:JE,useRecipeResult:ZE,withContext:wt}=ui({key:"table"}),nf=P.forwardRef(function({native:t,...n},r){const{styles:o,props:i,classNames:s}=ZE(n),a=P.useMemo(()=>t?{...o.root,"& thead":o.header,"& tbody":o.body,"& tfoot":o.footer,"& thead th":o.columnHeader,"& tr":o.row,"& td":o.cell,"& caption":o.caption}:o.root,[o,t]);return f.jsx(JE,{value:s,children:f.jsx(QE,{value:o,children:f.jsx(ve.table,{ref:r,...i,css:[a,n.css],className:Ge(s?.root,n.className)})})})}),fi=wt("tr","row");ve("div",{base:{display:"block",whiteSpace:"nowrap",WebkitOverflowScrolling:"touch",overflow:"auto",maxWidth:"100%"}});const rf=wt("thead","header");wt("tfoot","footer");const ke=wt("th","columnHeader"),Ce=wt("td","cell");wt("caption","caption",{defaultProps:{captionSide:"bottom"}});const of=wt("tbody","body");wt("colgroup"),wt("col");const{withContext:e2}=Nt({key:"textarea"}),sf=e2(nd);var rr=class{constructor(){this.listeners=new Set,this.subscribe=this.subscribe.bind(this)}subscribe(e){return this.listeners.add(e),this.onSubscribe(),()=>{this.listeners.delete(e),this.onUnsubscribe()}}hasListeners(){return this.listeners.size>0}onSubscribe(){}onUnsubscribe(){}},t2={setTimeout:(e,t)=>setTimeout(e,t),clearTimeout:e=>clearTimeout(e),setInterval:(e,t)=>setInterval(e,t),clearInterval:e=>clearInterval(e)},n2=class{#e=t2;#t=!1;setTimeoutProvider(e){this.#e=e}setTimeout(e,t){return this.#e.setTimeout(e,t)}clearTimeout(e){this.#e.clearTimeout(e)}setInterval(e,t){return this.#e.setInterval(e,t)}clearInterval(e){this.#e.clearInterval(e)}},Cn=new n2;function r2(e){setTimeout(e,0)}var Sn=typeof window>"u"||"Deno"in globalThis;function Ne(){}function o2(e,t){return typeof e=="function"?e(t):e}function Ia(e){return typeof e=="number"&&e>=0&&e!==1/0}function af(e,t){return Math.max(e+(t||0)-Date.now(),0)}function Bt(e,t){return typeof e=="function"?e(t):e}function Qe(e,t){return typeof e=="function"?e(t):e}function lf(e,t){const{type:n="all",exact:r,fetchStatus:o,predicate:i,queryKey:s,stale:a}=e;if(s){if(r){if(t.queryHash!==Ra(s,t.options))return!1}else if(!$r(t.queryKey,s))return!1}if(n!=="all"){const l=t.isActive();if(n==="active"&&!l||n==="inactive"&&l)return!1}return!(typeof a=="boolean"&&t.isStale()!==a||o&&o!==t.state.fetchStatus||i&&!i(t))}function cf(e,t){const{exact:n,status:r,predicate:o,mutationKey:i}=e;if(i){if(!t.options.mutationKey)return!1;if(n){if(wn(t.options.mutationKey)!==wn(i))return!1}else if(!$r(t.options.mutationKey,i))return!1}return!(r&&t.state.status!==r||o&&!o(t))}function Ra(e,t){return(t?.queryKeyHashFn||wn)(e)}function wn(e){return JSON.stringify(e,(t,n)=>Ta(n)?Object.keys(n).sort().reduce((r,o)=>(r[o]=n[o],r),{}):n)}function $r(e,t){return e===t?!0:typeof e!=typeof t?!1:e&&t&&typeof e=="object"&&typeof t=="object"?Object.keys(t).every(n=>$r(e[n],t[n])):!1}var i2=Object.prototype.hasOwnProperty;function uf(e,t){if(e===t)return e;const n=df(e)&&df(t);if(!n&&!(Ta(e)&&Ta(t)))return t;const o=(n?e:Object.keys(e)).length,i=n?t:Object.keys(t),s=i.length,a=n?new Array(s):{};let l=0;for(let c=0;c{Cn.setTimeout(t,e)})}function Na(e,t,n){return typeof n.structuralSharing=="function"?n.structuralSharing(e,t):n.structuralSharing!==!1?uf(e,t):t}function a2(e,t,n=0){const r=[...e,t];return n&&r.length>n?r.slice(1):r}function l2(e,t,n=0){const r=[t,...e];return n&&r.length>n?r.slice(0,-1):r}var Aa=Symbol();function ff(e,t){return!e.queryFn&&t?.initialPromise?()=>t.initialPromise:!e.queryFn||e.queryFn===Aa?()=>Promise.reject(new Error(`Missing queryFn: '${e.queryHash}'`)):e.queryFn}function gf(e,t){return typeof e=="function"?e(...t):!!e}var c2=class extends rr{#e;#t;#n;constructor(){super(),this.#n=e=>{if(!Sn&&window.addEventListener){const t=()=>e();return window.addEventListener("visibilitychange",t,!1),()=>{window.removeEventListener("visibilitychange",t)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(t=>{typeof t=="boolean"?this.setFocused(t):this.onFocus()})}setFocused(e){this.#e!==e&&(this.#e=e,this.onFocus())}onFocus(){const e=this.isFocused();this.listeners.forEach(t=>{t(e)})}isFocused(){return typeof this.#e=="boolean"?this.#e:globalThis.document?.visibilityState!=="hidden"}},_a=new c2;function Va(){let e,t;const n=new Promise((o,i)=>{e=o,t=i});n.status="pending",n.catch(()=>{});function r(o){Object.assign(n,o),delete n.resolve,delete n.reject}return n.resolve=o=>{r({status:"fulfilled",value:o}),e(o)},n.reject=o=>{r({status:"rejected",reason:o}),t(o)},n}var u2=r2;function d2(){let e=[],t=0,n=a=>{a()},r=a=>{a()},o=u2;const i=a=>{t?e.push(a):o(()=>{n(a)})},s=()=>{const a=e;e=[],a.length&&o(()=>{r(()=>{a.forEach(l=>{n(l)})})})};return{batch:a=>{let l;t++;try{l=a()}finally{t--,t||s()}return l},batchCalls:a=>(...l)=>{i(()=>{a(...l)})},schedule:i,setNotifyFunction:a=>{n=a},setBatchNotifyFunction:a=>{r=a},setScheduler:a=>{o=a}}}var ge=d2(),h2=class extends rr{#e=!0;#t;#n;constructor(){super(),this.#n=e=>{if(!Sn&&window.addEventListener){const t=()=>e(!0),n=()=>e(!1);return window.addEventListener("online",t,!1),window.addEventListener("offline",n,!1),()=>{window.removeEventListener("online",t),window.removeEventListener("offline",n)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(this.setOnline.bind(this))}setOnline(e){this.#e!==e&&(this.#e=e,this.listeners.forEach(n=>{n(e)}))}isOnline(){return this.#e}},pi=new h2;function f2(e){return Math.min(1e3*2**e,3e4)}function pf(e){return(e??"online")==="online"?pi.isOnline():!0}var Fa=class extends Error{constructor(e){super("CancelledError"),this.revert=e?.revert,this.silent=e?.silent}};function mf(e){let t=!1,n=0,r;const o=Va(),i=()=>o.status!=="pending",s=p=>{if(!i()){const v=new Fa(p);h(v),e.onCancel?.(v)}},a=()=>{t=!0},l=()=>{t=!1},c=()=>_a.isFocused()&&(e.networkMode==="always"||pi.isOnline())&&e.canRun(),u=()=>pf(e.networkMode)&&e.canRun(),d=p=>{i()||(r?.(),o.resolve(p))},h=p=>{i()||(r?.(),o.reject(p))},m=()=>new Promise(p=>{r=v=>{(i()||c())&&p(v)},e.onPause?.()}).then(()=>{r=void 0,i()||e.onContinue?.()}),g=()=>{if(i())return;let p;const v=n===0?e.initialPromise:void 0;try{p=v??e.fn()}catch(y){p=Promise.reject(y)}Promise.resolve(p).then(d).catch(y=>{if(i())return;const x=e.retry??(Sn?0:3),b=e.retryDelay??f2,C=typeof b=="function"?b(n,y):b,k=x===!0||typeof x=="number"&&nc()?void 0:m()).then(()=>{t?h(y):g()})})};return{promise:o,status:()=>o.status,cancel:s,continue:()=>(r?.(),o),cancelRetry:a,continueRetry:l,canStart:u,start:()=>(u()?g():m().then(g),o)}}var vf=class{#e;destroy(){this.clearGcTimeout()}scheduleGc(){this.clearGcTimeout(),Ia(this.gcTime)&&(this.#e=Cn.setTimeout(()=>{this.optionalRemove()},this.gcTime))}updateGcTime(e){this.gcTime=Math.max(this.gcTime||0,e??(Sn?1/0:300*1e3))}clearGcTimeout(){this.#e&&(Cn.clearTimeout(this.#e),this.#e=void 0)}},g2=class extends vf{#e;#t;#n;#o;#r;#s;#a;constructor(e){super(),this.#a=!1,this.#s=e.defaultOptions,this.setOptions(e.options),this.observers=[],this.#o=e.client,this.#n=this.#o.getQueryCache(),this.queryKey=e.queryKey,this.queryHash=e.queryHash,this.#e=yf(this.options),this.state=e.state??this.#e,this.scheduleGc()}get meta(){return this.options.meta}get promise(){return this.#r?.promise}setOptions(e){if(this.options={...this.#s,...e},this.updateGcTime(this.options.gcTime),this.state&&this.state.data===void 0){const t=yf(this.options);t.data!==void 0&&(this.setData(t.data,{updatedAt:t.dataUpdatedAt,manual:!0}),this.#e=t)}}optionalRemove(){!this.observers.length&&this.state.fetchStatus==="idle"&&this.#n.remove(this)}setData(e,t){const n=Na(this.state.data,e,this.options);return this.#i({data:n,type:"success",dataUpdatedAt:t?.updatedAt,manual:t?.manual}),n}setState(e,t){this.#i({type:"setState",state:e,setStateOptions:t})}cancel(e){const t=this.#r?.promise;return this.#r?.cancel(e),t?t.then(Ne).catch(Ne):Promise.resolve()}destroy(){super.destroy(),this.cancel({silent:!0})}reset(){this.destroy(),this.setState(this.#e)}isActive(){return this.observers.some(e=>Qe(e.options.enabled,this)!==!1)}isDisabled(){return this.getObserversCount()>0?!this.isActive():this.options.queryFn===Aa||this.state.dataUpdateCount+this.state.errorUpdateCount===0}isStatic(){return this.getObserversCount()>0?this.observers.some(e=>Bt(e.options.staleTime,this)==="static"):!1}isStale(){return this.getObserversCount()>0?this.observers.some(e=>e.getCurrentResult().isStale):this.state.data===void 0||this.state.isInvalidated}isStaleByTime(e=0){return this.state.data===void 0?!0:e==="static"?!1:this.state.isInvalidated?!0:!af(this.state.dataUpdatedAt,e)}onFocus(){this.observers.find(t=>t.shouldFetchOnWindowFocus())?.refetch({cancelRefetch:!1}),this.#r?.continue()}onOnline(){this.observers.find(t=>t.shouldFetchOnReconnect())?.refetch({cancelRefetch:!1}),this.#r?.continue()}addObserver(e){this.observers.includes(e)||(this.observers.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",query:this,observer:e}))}removeObserver(e){this.observers.includes(e)&&(this.observers=this.observers.filter(t=>t!==e),this.observers.length||(this.#r&&(this.#a?this.#r.cancel({revert:!0}):this.#r.cancelRetry()),this.scheduleGc()),this.#n.notify({type:"observerRemoved",query:this,observer:e}))}getObserversCount(){return this.observers.length}invalidate(){this.state.isInvalidated||this.#i({type:"invalidate"})}async fetch(e,t){if(this.state.fetchStatus!=="idle"&&this.#r?.status()!=="rejected"){if(this.state.data!==void 0&&t?.cancelRefetch)this.cancel({silent:!0});else if(this.#r)return this.#r.continueRetry(),this.#r.promise}if(e&&this.setOptions(e),!this.options.queryFn){const a=this.observers.find(l=>l.options.queryFn);a&&this.setOptions(a.options)}const n=new AbortController,r=a=>{Object.defineProperty(a,"signal",{enumerable:!0,get:()=>(this.#a=!0,n.signal)})},o=()=>{const a=ff(this.options,t),c=(()=>{const u={client:this.#o,queryKey:this.queryKey,meta:this.meta};return r(u),u})();return this.#a=!1,this.options.persister?this.options.persister(a,c,this):a(c)},s=(()=>{const a={fetchOptions:t,options:this.options,queryKey:this.queryKey,client:this.#o,state:this.state,fetchFn:o};return r(a),a})();this.options.behavior?.onFetch(s,this),this.#t=this.state,(this.state.fetchStatus==="idle"||this.state.fetchMeta!==s.fetchOptions?.meta)&&this.#i({type:"fetch",meta:s.fetchOptions?.meta}),this.#r=mf({initialPromise:t?.initialPromise,fn:s.fetchFn,onCancel:a=>{a instanceof Fa&&a.revert&&this.setState({...this.#t,fetchStatus:"idle"}),n.abort()},onFail:(a,l)=>{this.#i({type:"failed",failureCount:a,error:l})},onPause:()=>{this.#i({type:"pause"})},onContinue:()=>{this.#i({type:"continue"})},retry:s.options.retry,retryDelay:s.options.retryDelay,networkMode:s.options.networkMode,canRun:()=>!0});try{const a=await this.#r.start();if(a===void 0)throw new Error(`${this.queryHash} data is undefined`);return this.setData(a),this.#n.config.onSuccess?.(a,this),this.#n.config.onSettled?.(a,this.state.error,this),a}catch(a){if(a instanceof Fa){if(a.silent)return this.#r.promise;if(a.revert){if(this.state.data===void 0)throw a;return this.state.data}}throw this.#i({type:"error",error:a}),this.#n.config.onError?.(a,this),this.#n.config.onSettled?.(this.state.data,a,this),a}finally{this.scheduleGc()}}#i(e){const t=n=>{switch(e.type){case"failed":return{...n,fetchFailureCount:e.failureCount,fetchFailureReason:e.error};case"pause":return{...n,fetchStatus:"paused"};case"continue":return{...n,fetchStatus:"fetching"};case"fetch":return{...n,...bf(n.data,this.options),fetchMeta:e.meta??null};case"success":const r={...n,data:e.data,dataUpdateCount:n.dataUpdateCount+1,dataUpdatedAt:e.dataUpdatedAt??Date.now(),error:null,isInvalidated:!1,status:"success",...!e.manual&&{fetchStatus:"idle",fetchFailureCount:0,fetchFailureReason:null}};return this.#t=e.manual?r:void 0,r;case"error":const o=e.error;return{...n,error:o,errorUpdateCount:n.errorUpdateCount+1,errorUpdatedAt:Date.now(),fetchFailureCount:n.fetchFailureCount+1,fetchFailureReason:o,fetchStatus:"idle",status:"error"};case"invalidate":return{...n,isInvalidated:!0};case"setState":return{...n,...e.state}}};this.state=t(this.state),ge.batch(()=>{this.observers.forEach(n=>{n.onQueryUpdate()}),this.#n.notify({query:this,type:"updated",action:e})})}};function bf(e,t){return{fetchFailureCount:0,fetchFailureReason:null,fetchStatus:pf(t.networkMode)?"fetching":"paused",...e===void 0&&{error:null,status:"pending"}}}function yf(e){const t=typeof e.initialData=="function"?e.initialData():e.initialData,n=t!==void 0,r=n?typeof e.initialDataUpdatedAt=="function"?e.initialDataUpdatedAt():e.initialDataUpdatedAt:0;return{data:t,dataUpdateCount:0,dataUpdatedAt:n?r??Date.now():0,error:null,errorUpdateCount:0,errorUpdatedAt:0,fetchFailureCount:0,fetchFailureReason:null,fetchMeta:null,isInvalidated:!1,status:n?"success":"pending",fetchStatus:"idle"}}var p2=class extends rr{constructor(e,t){super(),this.options=t,this.#e=e,this.#i=null,this.#a=Va(),this.bindMethods(),this.setOptions(t)}#e;#t=void 0;#n=void 0;#o=void 0;#r;#s;#a;#i;#p;#h;#f;#c;#u;#l;#g=new Set;bindMethods(){this.refetch=this.refetch.bind(this)}onSubscribe(){this.listeners.size===1&&(this.#t.addObserver(this),xf(this.#t,this.options)?this.#d():this.updateResult(),this.#y())}onUnsubscribe(){this.hasListeners()||this.destroy()}shouldFetchOnReconnect(){return La(this.#t,this.options,this.options.refetchOnReconnect)}shouldFetchOnWindowFocus(){return La(this.#t,this.options,this.options.refetchOnWindowFocus)}destroy(){this.listeners=new Set,this.#x(),this.#k(),this.#t.removeObserver(this)}setOptions(e){const t=this.options,n=this.#t;if(this.options=this.#e.defaultQueryOptions(e),this.options.enabled!==void 0&&typeof this.options.enabled!="boolean"&&typeof this.options.enabled!="function"&&typeof Qe(this.options.enabled,this.#t)!="boolean")throw new Error("Expected enabled to be a boolean or a callback that returns a boolean");this.#C(),this.#t.setOptions(this.options),t._defaulted&&!gi(this.options,t)&&this.#e.getQueryCache().notify({type:"observerOptionsUpdated",query:this.#t,observer:this});const r=this.hasListeners();r&&kf(this.#t,n,this.options,t)&&this.#d(),this.updateResult(),r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||Bt(this.options.staleTime,this.#t)!==Bt(t.staleTime,this.#t))&&this.#m();const o=this.#v();r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||o!==this.#l)&&this.#b(o)}getOptimisticResult(e){const t=this.#e.getQueryCache().build(this.#e,e),n=this.createResult(t,e);return v2(this,n)&&(this.#o=n,this.#s=this.options,this.#r=this.#t.state),n}getCurrentResult(){return this.#o}trackResult(e,t){return new Proxy(e,{get:(n,r)=>(this.trackProp(r),t?.(r),r==="promise"&&!this.options.experimental_prefetchInRender&&this.#a.status==="pending"&&this.#a.reject(new Error("experimental_prefetchInRender feature flag is not enabled")),Reflect.get(n,r))})}trackProp(e){this.#g.add(e)}getCurrentQuery(){return this.#t}refetch({...e}={}){return this.fetch({...e})}fetchOptimistic(e){const t=this.#e.defaultQueryOptions(e),n=this.#e.getQueryCache().build(this.#e,t);return n.fetch().then(()=>this.createResult(n,t))}fetch(e){return this.#d({...e,cancelRefetch:e.cancelRefetch??!0}).then(()=>(this.updateResult(),this.#o))}#d(e){this.#C();let t=this.#t.fetch(this.options,e);return e?.throwOnError||(t=t.catch(Ne)),t}#m(){this.#x();const e=Bt(this.options.staleTime,this.#t);if(Sn||this.#o.isStale||!Ia(e))return;const n=af(this.#o.dataUpdatedAt,e)+1;this.#c=Cn.setTimeout(()=>{this.#o.isStale||this.updateResult()},n)}#v(){return(typeof this.options.refetchInterval=="function"?this.options.refetchInterval(this.#t):this.options.refetchInterval)??!1}#b(e){this.#k(),this.#l=e,!(Sn||Qe(this.options.enabled,this.#t)===!1||!Ia(this.#l)||this.#l===0)&&(this.#u=Cn.setInterval(()=>{(this.options.refetchIntervalInBackground||_a.isFocused())&&this.#d()},this.#l))}#y(){this.#m(),this.#b(this.#v())}#x(){this.#c&&(Cn.clearTimeout(this.#c),this.#c=void 0)}#k(){this.#u&&(Cn.clearInterval(this.#u),this.#u=void 0)}createResult(e,t){const n=this.#t,r=this.options,o=this.#o,i=this.#r,s=this.#s,l=e!==n?e.state:this.#n,{state:c}=e;let u={...c},d=!1,h;if(t._optimisticResults){const R=this.hasListeners(),_=!R&&xf(e,t),A=R&&kf(e,n,t,r);(_||A)&&(u={...u,...bf(c.data,e.options)}),t._optimisticResults==="isRestoring"&&(u.fetchStatus="idle")}let{error:m,errorUpdatedAt:g,status:p}=u;h=u.data;let v=!1;if(t.placeholderData!==void 0&&h===void 0&&p==="pending"){let R;o?.isPlaceholderData&&t.placeholderData===s?.placeholderData?(R=o.data,v=!0):R=typeof t.placeholderData=="function"?t.placeholderData(this.#f?.state.data,this.#f):t.placeholderData,R!==void 0&&(p="success",h=Na(o?.data,R,t),d=!0)}if(t.select&&h!==void 0&&!v)if(o&&h===i?.data&&t.select===this.#p)h=this.#h;else try{this.#p=t.select,h=t.select(h),h=Na(o?.data,h,t),this.#h=h,this.#i=null}catch(R){this.#i=R}this.#i&&(m=this.#i,h=this.#h,g=Date.now(),p="error");const y=u.fetchStatus==="fetching",x=p==="pending",b=p==="error",C=x&&y,k=h!==void 0,I={status:p,fetchStatus:u.fetchStatus,isPending:x,isSuccess:p==="success",isError:b,isInitialLoading:C,isLoading:C,data:h,dataUpdatedAt:u.dataUpdatedAt,error:m,errorUpdatedAt:g,failureCount:u.fetchFailureCount,failureReason:u.fetchFailureReason,errorUpdateCount:u.errorUpdateCount,isFetched:u.dataUpdateCount>0||u.errorUpdateCount>0,isFetchedAfterMount:u.dataUpdateCount>l.dataUpdateCount||u.errorUpdateCount>l.errorUpdateCount,isFetching:y,isRefetching:y&&!x,isLoadingError:b&&!k,isPaused:u.fetchStatus==="paused",isPlaceholderData:d,isRefetchError:b&&k,isStale:Da(e,t),refetch:this.refetch,promise:this.#a,isEnabled:Qe(t.enabled,e)!==!1};if(this.options.experimental_prefetchInRender){const R=T=>{I.status==="error"?T.reject(I.error):I.data!==void 0&&T.resolve(I.data)},_=()=>{const T=this.#a=I.promise=Va();R(T)},A=this.#a;switch(A.status){case"pending":e.queryHash===n.queryHash&&R(A);break;case"fulfilled":(I.status==="error"||I.data!==A.value)&&_();break;case"rejected":(I.status!=="error"||I.error!==A.reason)&&_();break}}return I}updateResult(){const e=this.#o,t=this.createResult(this.#t,this.options);if(this.#r=this.#t.state,this.#s=this.options,this.#r.data!==void 0&&(this.#f=this.#t),gi(t,e))return;this.#o=t;const n=()=>{if(!e)return!0;const{notifyOnChangeProps:r}=this.options,o=typeof r=="function"?r():r;if(o==="all"||!o&&!this.#g.size)return!0;const i=new Set(o??this.#g);return this.options.throwOnError&&i.add("error"),Object.keys(this.#o).some(s=>{const a=s;return this.#o[a]!==e[a]&&i.has(a)})};this.#S({listeners:n()})}#C(){const e=this.#e.getQueryCache().build(this.#e,this.options);if(e===this.#t)return;const t=this.#t;this.#t=e,this.#n=e.state,this.hasListeners()&&(t?.removeObserver(this),e.addObserver(this))}onQueryUpdate(){this.updateResult(),this.hasListeners()&&this.#y()}#S(e){ge.batch(()=>{e.listeners&&this.listeners.forEach(t=>{t(this.#o)}),this.#e.getQueryCache().notify({query:this.#t,type:"observerResultsUpdated"})})}};function m2(e,t){return Qe(t.enabled,e)!==!1&&e.state.data===void 0&&!(e.state.status==="error"&&t.retryOnMount===!1)}function xf(e,t){return m2(e,t)||e.state.data!==void 0&&La(e,t,t.refetchOnMount)}function La(e,t,n){if(Qe(t.enabled,e)!==!1&&Bt(t.staleTime,e)!=="static"){const r=typeof n=="function"?n(e):n;return r==="always"||r!==!1&&Da(e,t)}return!1}function kf(e,t,n,r){return(e!==t||Qe(r.enabled,e)===!1)&&(!n.suspense||e.state.status!=="error")&&Da(e,n)}function Da(e,t){return Qe(t.enabled,e)!==!1&&e.isStaleByTime(Bt(t.staleTime,e))}function v2(e,t){return!gi(e.getCurrentResult(),t)}function Cf(e){return{onFetch:(t,n)=>{const r=t.options,o=t.fetchOptions?.meta?.fetchMore?.direction,i=t.state.data?.pages||[],s=t.state.data?.pageParams||[];let a={pages:[],pageParams:[]},l=0;const c=async()=>{let u=!1;const d=g=>{Object.defineProperty(g,"signal",{enumerable:!0,get:()=>(t.signal.aborted?u=!0:t.signal.addEventListener("abort",()=>{u=!0}),t.signal)})},h=ff(t.options,t.fetchOptions),m=async(g,p,v)=>{if(u)return Promise.reject();if(p==null&&g.pages.length)return Promise.resolve(g);const x=(()=>{const E={client:t.client,queryKey:t.queryKey,pageParam:p,direction:v?"backward":"forward",meta:t.options.meta};return d(E),E})(),b=await h(x),{maxPages:C}=t.options,k=v?l2:a2;return{pages:k(g.pages,b,C),pageParams:k(g.pageParams,p,C)}};if(o&&i.length){const g=o==="backward",p=g?b2:Sf,v={pages:i,pageParams:s},y=p(r,v);a=await m(v,y,g)}else{const g=e??i.length;do{const p=l===0?s[0]??r.initialPageParam:Sf(r,a);if(l>0&&p==null)break;a=await m(a,p),l++}while(lt.options.persister?.(c,{client:t.client,queryKey:t.queryKey,meta:t.options.meta,signal:t.signal},n):t.fetchFn=c}}}function Sf(e,{pages:t,pageParams:n}){const r=t.length-1;return t.length>0?e.getNextPageParam(t[r],t,n[r],n):void 0}function b2(e,{pages:t,pageParams:n}){return t.length>0?e.getPreviousPageParam?.(t[0],t,n[0],n):void 0}var y2=class extends vf{#e;#t;#n;#o;constructor(e){super(),this.#e=e.client,this.mutationId=e.mutationId,this.#n=e.mutationCache,this.#t=[],this.state=e.state||wf(),this.setOptions(e.options),this.scheduleGc()}setOptions(e){this.options=e,this.updateGcTime(this.options.gcTime)}get meta(){return this.options.meta}addObserver(e){this.#t.includes(e)||(this.#t.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",mutation:this,observer:e}))}removeObserver(e){this.#t=this.#t.filter(t=>t!==e),this.scheduleGc(),this.#n.notify({type:"observerRemoved",mutation:this,observer:e})}optionalRemove(){this.#t.length||(this.state.status==="pending"?this.scheduleGc():this.#n.remove(this))}continue(){return this.#o?.continue()??this.execute(this.state.variables)}async execute(e){const t=()=>{this.#r({type:"continue"})},n={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};this.#o=mf({fn:()=>this.options.mutationFn?this.options.mutationFn(e,n):Promise.reject(new Error("No mutationFn found")),onFail:(i,s)=>{this.#r({type:"failed",failureCount:i,error:s})},onPause:()=>{this.#r({type:"pause"})},onContinue:t,retry:this.options.retry??0,retryDelay:this.options.retryDelay,networkMode:this.options.networkMode,canRun:()=>this.#n.canRun(this)});const r=this.state.status==="pending",o=!this.#o.canStart();try{if(r)t();else{this.#r({type:"pending",variables:e,isPaused:o}),await this.#n.config.onMutate?.(e,this,n);const s=await this.options.onMutate?.(e,n);s!==this.state.context&&this.#r({type:"pending",context:s,variables:e,isPaused:o})}const i=await this.#o.start();return await this.#n.config.onSuccess?.(i,e,this.state.context,this,n),await this.options.onSuccess?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(i,null,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(i,null,e,this.state.context,n),this.#r({type:"success",data:i}),i}catch(i){try{throw await this.#n.config.onError?.(i,e,this.state.context,this,n),await this.options.onError?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(void 0,i,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(void 0,i,e,this.state.context,n),i}finally{this.#r({type:"error",error:i})}}finally{this.#n.runNext(this)}}#r(e){const t=n=>{switch(e.type){case"failed":return{...n,failureCount:e.failureCount,failureReason:e.error};case"pause":return{...n,isPaused:!0};case"continue":return{...n,isPaused:!1};case"pending":return{...n,context:e.context,data:void 0,failureCount:0,failureReason:null,error:null,isPaused:e.isPaused,status:"pending",variables:e.variables,submittedAt:Date.now()};case"success":return{...n,data:e.data,failureCount:0,failureReason:null,error:null,status:"success",isPaused:!1};case"error":return{...n,data:void 0,error:e.error,failureCount:n.failureCount+1,failureReason:e.error,isPaused:!1,status:"error"}}};this.state=t(this.state),ge.batch(()=>{this.#t.forEach(n=>{n.onMutationUpdate(e)}),this.#n.notify({mutation:this,type:"updated",action:e})})}};function wf(){return{context:void 0,data:void 0,error:null,failureCount:0,failureReason:null,isPaused:!1,status:"idle",variables:void 0,submittedAt:0}}var x2=class extends rr{constructor(e={}){super(),this.config=e,this.#e=new Set,this.#t=new Map,this.#n=0}#e;#t;#n;build(e,t,n){const r=new y2({client:e,mutationCache:this,mutationId:++this.#n,options:e.defaultMutationOptions(t),state:n});return this.add(r),r}add(e){this.#e.add(e);const t=mi(e);if(typeof t=="string"){const n=this.#t.get(t);n?n.push(e):this.#t.set(t,[e])}this.notify({type:"added",mutation:e})}remove(e){if(this.#e.delete(e)){const t=mi(e);if(typeof t=="string"){const n=this.#t.get(t);if(n)if(n.length>1){const r=n.indexOf(e);r!==-1&&n.splice(r,1)}else n[0]===e&&this.#t.delete(t)}}this.notify({type:"removed",mutation:e})}canRun(e){const t=mi(e);if(typeof t=="string"){const r=this.#t.get(t)?.find(o=>o.state.status==="pending");return!r||r===e}else return!0}runNext(e){const t=mi(e);return typeof t=="string"?this.#t.get(t)?.find(r=>r!==e&&r.state.isPaused)?.continue()??Promise.resolve():Promise.resolve()}clear(){ge.batch(()=>{this.#e.forEach(e=>{this.notify({type:"removed",mutation:e})}),this.#e.clear(),this.#t.clear()})}getAll(){return Array.from(this.#e)}find(e){const t={exact:!0,...e};return this.getAll().find(n=>cf(t,n))}findAll(e={}){return this.getAll().filter(t=>cf(e,t))}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}resumePausedMutations(){const e=this.getAll().filter(t=>t.state.isPaused);return ge.batch(()=>Promise.all(e.map(t=>t.continue().catch(Ne))))}};function mi(e){return e.options.scope?.id}var k2=class extends rr{#e;#t=void 0;#n;#o;constructor(e,t){super(),this.#e=e,this.setOptions(t),this.bindMethods(),this.#r()}bindMethods(){this.mutate=this.mutate.bind(this),this.reset=this.reset.bind(this)}setOptions(e){const t=this.options;this.options=this.#e.defaultMutationOptions(e),gi(this.options,t)||this.#e.getMutationCache().notify({type:"observerOptionsUpdated",mutation:this.#n,observer:this}),t?.mutationKey&&this.options.mutationKey&&wn(t.mutationKey)!==wn(this.options.mutationKey)?this.reset():this.#n?.state.status==="pending"&&this.#n.setOptions(this.options)}onUnsubscribe(){this.hasListeners()||this.#n?.removeObserver(this)}onMutationUpdate(e){this.#r(),this.#s(e)}getCurrentResult(){return this.#t}reset(){this.#n?.removeObserver(this),this.#n=void 0,this.#r(),this.#s()}mutate(e,t){return this.#o=t,this.#n?.removeObserver(this),this.#n=this.#e.getMutationCache().build(this.#e,this.options),this.#n.addObserver(this),this.#n.execute(e)}#r(){const e=this.#n?.state??wf();this.#t={...e,isPending:e.status==="pending",isSuccess:e.status==="success",isError:e.status==="error",isIdle:e.status==="idle",mutate:this.mutate,reset:this.reset}}#s(e){ge.batch(()=>{if(this.#o&&this.hasListeners()){const t=this.#t.variables,n=this.#t.context,r={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};e?.type==="success"?(this.#o.onSuccess?.(e.data,t,n,r),this.#o.onSettled?.(e.data,null,t,n,r)):e?.type==="error"&&(this.#o.onError?.(e.error,t,n,r),this.#o.onSettled?.(void 0,e.error,t,n,r))}this.listeners.forEach(t=>{t(this.#t)})})}},C2=class extends rr{constructor(e={}){super(),this.config=e,this.#e=new Map}#e;build(e,t,n){const r=t.queryKey,o=t.queryHash??Ra(r,t);let i=this.get(o);return i||(i=new g2({client:e,queryKey:r,queryHash:o,options:e.defaultQueryOptions(t),state:n,defaultOptions:e.getQueryDefaults(r)}),this.add(i)),i}add(e){this.#e.has(e.queryHash)||(this.#e.set(e.queryHash,e),this.notify({type:"added",query:e}))}remove(e){const t=this.#e.get(e.queryHash);t&&(e.destroy(),t===e&&this.#e.delete(e.queryHash),this.notify({type:"removed",query:e}))}clear(){ge.batch(()=>{this.getAll().forEach(e=>{this.remove(e)})})}get(e){return this.#e.get(e)}getAll(){return[...this.#e.values()]}find(e){const t={exact:!0,...e};return this.getAll().find(n=>lf(t,n))}findAll(e={}){const t=this.getAll();return Object.keys(e).length>0?t.filter(n=>lf(e,n)):t}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}onFocus(){ge.batch(()=>{this.getAll().forEach(e=>{e.onFocus()})})}onOnline(){ge.batch(()=>{this.getAll().forEach(e=>{e.onOnline()})})}},S2=class{#e;#t;#n;#o;#r;#s;#a;#i;constructor(e={}){this.#e=e.queryCache||new C2,this.#t=e.mutationCache||new x2,this.#n=e.defaultOptions||{},this.#o=new Map,this.#r=new Map,this.#s=0}mount(){this.#s++,this.#s===1&&(this.#a=_a.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onFocus())}),this.#i=pi.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onOnline())}))}unmount(){this.#s--,this.#s===0&&(this.#a?.(),this.#a=void 0,this.#i?.(),this.#i=void 0)}isFetching(e){return this.#e.findAll({...e,fetchStatus:"fetching"}).length}isMutating(e){return this.#t.findAll({...e,status:"pending"}).length}getQueryData(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state.data}ensureQueryData(e){const t=this.defaultQueryOptions(e),n=this.#e.build(this,t),r=n.state.data;return r===void 0?this.fetchQuery(e):(e.revalidateIfStale&&n.isStaleByTime(Bt(t.staleTime,n))&&this.prefetchQuery(t),Promise.resolve(r))}getQueriesData(e){return this.#e.findAll(e).map(({queryKey:t,state:n})=>{const r=n.data;return[t,r]})}setQueryData(e,t,n){const r=this.defaultQueryOptions({queryKey:e}),i=this.#e.get(r.queryHash)?.state.data,s=o2(t,i);if(s!==void 0)return this.#e.build(this,r).setData(s,{...n,manual:!0})}setQueriesData(e,t,n){return ge.batch(()=>this.#e.findAll(e).map(({queryKey:r})=>[r,this.setQueryData(r,t,n)]))}getQueryState(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state}removeQueries(e){const t=this.#e;ge.batch(()=>{t.findAll(e).forEach(n=>{t.remove(n)})})}resetQueries(e,t){const n=this.#e;return ge.batch(()=>(n.findAll(e).forEach(r=>{r.reset()}),this.refetchQueries({type:"active",...e},t)))}cancelQueries(e,t={}){const n={revert:!0,...t},r=ge.batch(()=>this.#e.findAll(e).map(o=>o.cancel(n)));return Promise.all(r).then(Ne).catch(Ne)}invalidateQueries(e,t={}){return ge.batch(()=>(this.#e.findAll(e).forEach(n=>{n.invalidate()}),e?.refetchType==="none"?Promise.resolve():this.refetchQueries({...e,type:e?.refetchType??e?.type??"active"},t)))}refetchQueries(e,t={}){const n={...t,cancelRefetch:t.cancelRefetch??!0},r=ge.batch(()=>this.#e.findAll(e).filter(o=>!o.isDisabled()&&!o.isStatic()).map(o=>{let i=o.fetch(void 0,n);return n.throwOnError||(i=i.catch(Ne)),o.state.fetchStatus==="paused"?Promise.resolve():i}));return Promise.all(r).then(Ne)}fetchQuery(e){const t=this.defaultQueryOptions(e);t.retry===void 0&&(t.retry=!1);const n=this.#e.build(this,t);return n.isStaleByTime(Bt(t.staleTime,n))?n.fetch(t):Promise.resolve(n.state.data)}prefetchQuery(e){return this.fetchQuery(e).then(Ne).catch(Ne)}fetchInfiniteQuery(e){return e.behavior=Cf(e.pages),this.fetchQuery(e)}prefetchInfiniteQuery(e){return this.fetchInfiniteQuery(e).then(Ne).catch(Ne)}ensureInfiniteQueryData(e){return e.behavior=Cf(e.pages),this.ensureQueryData(e)}resumePausedMutations(){return pi.isOnline()?this.#t.resumePausedMutations():Promise.resolve()}getQueryCache(){return this.#e}getMutationCache(){return this.#t}getDefaultOptions(){return this.#n}setDefaultOptions(e){this.#n=e}setQueryDefaults(e,t){this.#o.set(wn(e),{queryKey:e,defaultOptions:t})}getQueryDefaults(e){const t=[...this.#o.values()],n={};return t.forEach(r=>{$r(e,r.queryKey)&&Object.assign(n,r.defaultOptions)}),n}setMutationDefaults(e,t){this.#r.set(wn(e),{mutationKey:e,defaultOptions:t})}getMutationDefaults(e){const t=[...this.#r.values()],n={};return t.forEach(r=>{$r(e,r.mutationKey)&&Object.assign(n,r.defaultOptions)}),n}defaultQueryOptions(e){if(e._defaulted)return e;const t={...this.#n.queries,...this.getQueryDefaults(e.queryKey),...e,_defaulted:!0};return t.queryHash||(t.queryHash=Ra(t.queryKey,t)),t.refetchOnReconnect===void 0&&(t.refetchOnReconnect=t.networkMode!=="always"),t.throwOnError===void 0&&(t.throwOnError=!!t.suspense),!t.networkMode&&t.persister&&(t.networkMode="offlineFirst"),t.queryFn===Aa&&(t.enabled=!1),t}defaultMutationOptions(e){return e?._defaulted?e:{...this.#n.mutations,...e?.mutationKey&&this.getMutationDefaults(e.mutationKey),...e,_defaulted:!0}}clear(){this.#e.clear(),this.#t.clear()}},Ef=O.createContext(void 0),Of=e=>{const t=O.useContext(Ef);if(!t)throw new Error("No QueryClient set, use QueryClientProvider to set one");return t},w2=({client:e,children:t})=>(O.useEffect(()=>(e.mount(),()=>{e.unmount()}),[e]),f.jsx(Ef.Provider,{value:e,children:t})),Pf=O.createContext(!1),E2=()=>O.useContext(Pf);Pf.Provider;function O2(){let e=!1;return{clearReset:()=>{e=!1},reset:()=>{e=!0},isReset:()=>e}}var P2=O.createContext(O2()),I2=()=>O.useContext(P2),R2=(e,t)=>{(e.suspense||e.throwOnError||e.experimental_prefetchInRender)&&(t.isReset()||(e.retryOnMount=!1))},T2=e=>{O.useEffect(()=>{e.clearReset()},[e])},N2=({result:e,errorResetBoundary:t,throwOnError:n,query:r,suspense:o})=>e.isError&&!t.isReset()&&!e.isFetching&&r&&(o&&e.data===void 0||gf(n,[e.error,r])),A2=e=>{if(e.suspense){const n=o=>o==="static"?o:Math.max(o??1e3,1e3),r=e.staleTime;e.staleTime=typeof r=="function"?(...o)=>n(r(...o)):n(r),typeof e.gcTime=="number"&&(e.gcTime=Math.max(e.gcTime,1e3))}},_2=(e,t)=>e.isLoading&&e.isFetching&&!t,V2=(e,t)=>e?.suspense&&t.isPending,If=(e,t,n)=>t.fetchOptimistic(e).catch(()=>{n.clearReset()});function F2(e,t,n){const r=E2(),o=I2(),i=Of(),s=i.defaultQueryOptions(e);i.getDefaultOptions().queries?._experimental_beforeQuery?.(s),s._optimisticResults=r?"isRestoring":"optimistic",A2(s),R2(s,o),T2(o);const a=!i.getQueryCache().get(s.queryHash),[l]=O.useState(()=>new t(i,s)),c=l.getOptimisticResult(s),u=!r&&e.subscribed!==!1;if(O.useSyncExternalStore(O.useCallback(d=>{const h=u?l.subscribe(ge.batchCalls(d)):Ne;return l.updateResult(),h},[l,u]),()=>l.getCurrentResult(),()=>l.getCurrentResult()),O.useEffect(()=>{l.setOptions(s)},[s,l]),V2(s,c))throw If(s,l,o);if(N2({result:c,errorResetBoundary:o,throwOnError:s.throwOnError,query:i.getQueryCache().get(s.queryHash),suspense:s.suspense}))throw c.error;return i.getDefaultOptions().queries?._experimental_afterQuery?.(s,c),s.experimental_prefetchInRender&&!Sn&&_2(c,r)&&(a?If(s,l,o):i.getQueryCache().get(s.queryHash)?.promise)?.catch(Ne).finally(()=>{l.updateResult()}),s.notifyOnChangeProps?c:l.trackResult(c)}function Rf(e,t){return F2(e,p2)}function En(e,t){const n=Of(),[r]=O.useState(()=>new k2(n,e));O.useEffect(()=>{r.setOptions(e)},[r,e]);const o=O.useSyncExternalStore(O.useCallback(s=>r.subscribe(ge.batchCalls(s)),[r]),()=>r.getCurrentResult(),()=>r.getCurrentResult()),i=O.useCallback((s,a)=>{r.mutate(s,a).catch(Ne)},[r]);if(o.error&&gf(r.options.throwOnError,[o.error]))throw o.error;return{...o,mutate:i,mutateAsync:o.mutate}}function Tf(e,t){return function(){return e.apply(t,arguments)}}const{toString:L2}=Object.prototype,{getPrototypeOf:za}=Object,{iterator:vi,toStringTag:Nf}=Symbol,bi=(e=>t=>{const n=L2.call(t);return e[n]||(e[n]=n.slice(8,-1).toLowerCase())})(Object.create(null)),ot=e=>(e=e.toLowerCase(),t=>bi(t)===e),yi=e=>t=>typeof t===e,{isArray:or}=Array,ir=yi("undefined");function Br(e){return e!==null&&!ir(e)&&e.constructor!==null&&!ir(e.constructor)&&ze(e.constructor.isBuffer)&&e.constructor.isBuffer(e)}const Af=ot("ArrayBuffer");function D2(e){let t;return typeof ArrayBuffer<"u"&&ArrayBuffer.isView?t=ArrayBuffer.isView(e):t=e&&e.buffer&&Af(e.buffer),t}const z2=yi("string"),ze=yi("function"),_f=yi("number"),Wr=e=>e!==null&&typeof e=="object",M2=e=>e===!0||e===!1,xi=e=>{if(bi(e)!=="object")return!1;const t=za(e);return(t===null||t===Object.prototype||Object.getPrototypeOf(t)===null)&&!(Nf in e)&&!(vi in e)},j2=e=>{if(!Wr(e)||Br(e))return!1;try{return Object.keys(e).length===0&&Object.getPrototypeOf(e)===Object.prototype}catch{return!1}},$2=ot("Date"),B2=ot("File"),W2=ot("Blob"),H2=ot("FileList"),U2=e=>Wr(e)&&ze(e.pipe),G2=e=>{let t;return e&&(typeof FormData=="function"&&e instanceof FormData||ze(e.append)&&((t=bi(e))==="formdata"||t==="object"&&ze(e.toString)&&e.toString()==="[object FormData]"))},q2=ot("URLSearchParams"),[K2,Y2,X2,Q2]=["ReadableStream","Request","Response","Headers"].map(ot),J2=e=>e.trim?e.trim():e.replace(/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,"");function Hr(e,t,{allOwnKeys:n=!1}={}){if(e===null||typeof e>"u")return;let r,o;if(typeof e!="object"&&(e=[e]),or(e))for(r=0,o=e.length;r0;)if(o=n[r],t===o.toLowerCase())return o;return null}const On=typeof globalThis<"u"?globalThis:typeof self<"u"?self:typeof window<"u"?window:globalThis,Ff=e=>!ir(e)&&e!==On;function Ma(){const{caseless:e,skipUndefined:t}=Ff(this)&&this||{},n={},r=(o,i)=>{const s=e&&Vf(n,i)||i;xi(n[s])&&xi(o)?n[s]=Ma(n[s],o):xi(o)?n[s]=Ma({},o):or(o)?n[s]=o.slice():(!t||!ir(o))&&(n[s]=o)};for(let o=0,i=arguments.length;o(Hr(t,(o,i)=>{n&&ze(o)?e[i]=Tf(o,n):e[i]=o},{allOwnKeys:r}),e),eO=e=>(e.charCodeAt(0)===65279&&(e=e.slice(1)),e),tO=(e,t,n,r)=>{e.prototype=Object.create(t.prototype,r),e.prototype.constructor=e,Object.defineProperty(e,"super",{value:t.prototype}),n&&Object.assign(e.prototype,n)},nO=(e,t,n,r)=>{let o,i,s;const a={};if(t=t||{},e==null)return t;do{for(o=Object.getOwnPropertyNames(e),i=o.length;i-- >0;)s=o[i],(!r||r(s,e,t))&&!a[s]&&(t[s]=e[s],a[s]=!0);e=n!==!1&&za(e)}while(e&&(!n||n(e,t))&&e!==Object.prototype);return t},rO=(e,t,n)=>{e=String(e),(n===void 0||n>e.length)&&(n=e.length),n-=t.length;const r=e.indexOf(t,n);return r!==-1&&r===n},oO=e=>{if(!e)return null;if(or(e))return e;let t=e.length;if(!_f(t))return null;const n=new Array(t);for(;t-- >0;)n[t]=e[t];return n},iO=(e=>t=>e&&t instanceof e)(typeof Uint8Array<"u"&&za(Uint8Array)),sO=(e,t)=>{const r=(e&&e[vi]).call(e);let o;for(;(o=r.next())&&!o.done;){const i=o.value;t.call(e,i[0],i[1])}},aO=(e,t)=>{let n;const r=[];for(;(n=e.exec(t))!==null;)r.push(n);return r},lO=ot("HTMLFormElement"),cO=e=>e.toLowerCase().replace(/[-_\s]([a-z\d])(\w*)/g,function(n,r,o){return r.toUpperCase()+o}),Lf=(({hasOwnProperty:e})=>(t,n)=>e.call(t,n))(Object.prototype),uO=ot("RegExp"),Df=(e,t)=>{const n=Object.getOwnPropertyDescriptors(e),r={};Hr(n,(o,i)=>{let s;(s=t(o,i,e))!==!1&&(r[i]=s||o)}),Object.defineProperties(e,r)},dO=e=>{Df(e,(t,n)=>{if(ze(e)&&["arguments","caller","callee"].indexOf(n)!==-1)return!1;const r=e[n];if(ze(r)){if(t.enumerable=!1,"writable"in t){t.writable=!1;return}t.set||(t.set=()=>{throw Error("Can not rewrite read-only method '"+n+"'")})}})},hO=(e,t)=>{const n={},r=o=>{o.forEach(i=>{n[i]=!0})};return or(e)?r(e):r(String(e).split(t)),n},fO=()=>{},gO=(e,t)=>e!=null&&Number.isFinite(e=+e)?e:t;function pO(e){return!!(e&&ze(e.append)&&e[Nf]==="FormData"&&e[vi])}const mO=e=>{const t=new Array(10),n=(r,o)=>{if(Wr(r)){if(t.indexOf(r)>=0)return;if(Br(r))return r;if(!("toJSON"in r)){t[o]=r;const i=or(r)?[]:{};return Hr(r,(s,a)=>{const l=n(s,o+1);!ir(l)&&(i[a]=l)}),t[o]=void 0,i}}return r};return n(e,0)},vO=ot("AsyncFunction"),bO=e=>e&&(Wr(e)||ze(e))&&ze(e.then)&&ze(e.catch),zf=((e,t)=>e?setImmediate:t?((n,r)=>(On.addEventListener("message",({source:o,data:i})=>{o===On&&i===n&&r.length&&r.shift()()},!1),o=>{r.push(o),On.postMessage(n,"*")}))(`axios@${Math.random()}`,[]):n=>setTimeout(n))(typeof setImmediate=="function",ze(On.postMessage)),yO=typeof queueMicrotask<"u"?queueMicrotask.bind(On):typeof process<"u"&&process.nextTick||zf,w={isArray:or,isArrayBuffer:Af,isBuffer:Br,isFormData:G2,isArrayBufferView:D2,isString:z2,isNumber:_f,isBoolean:M2,isObject:Wr,isPlainObject:xi,isEmptyObject:j2,isReadableStream:K2,isRequest:Y2,isResponse:X2,isHeaders:Q2,isUndefined:ir,isDate:$2,isFile:B2,isBlob:W2,isRegExp:uO,isFunction:ze,isStream:U2,isURLSearchParams:q2,isTypedArray:iO,isFileList:H2,forEach:Hr,merge:Ma,extend:Z2,trim:J2,stripBOM:eO,inherits:tO,toFlatObject:nO,kindOf:bi,kindOfTest:ot,endsWith:rO,toArray:oO,forEachEntry:sO,matchAll:aO,isHTMLForm:lO,hasOwnProperty:Lf,hasOwnProp:Lf,reduceDescriptors:Df,freezeMethods:dO,toObjectSet:hO,toCamelCase:cO,noop:fO,toFiniteNumber:gO,findKey:Vf,global:On,isContextDefined:Ff,isSpecCompliantForm:pO,toJSONObject:mO,isAsyncFn:vO,isThenable:bO,setImmediate:zf,asap:yO,isIterable:e=>e!=null&&ze(e[vi])};function H(e,t,n,r,o){Error.call(this),Error.captureStackTrace?Error.captureStackTrace(this,this.constructor):this.stack=new Error().stack,this.message=e,this.name="AxiosError",t&&(this.code=t),n&&(this.config=n),r&&(this.request=r),o&&(this.response=o,this.status=o.status?o.status:null)}w.inherits(H,Error,{toJSON:function(){return{message:this.message,name:this.name,description:this.description,number:this.number,fileName:this.fileName,lineNumber:this.lineNumber,columnNumber:this.columnNumber,stack:this.stack,config:w.toJSONObject(this.config),code:this.code,status:this.status}}});const Mf=H.prototype,jf={};["ERR_BAD_OPTION_VALUE","ERR_BAD_OPTION","ECONNABORTED","ETIMEDOUT","ERR_NETWORK","ERR_FR_TOO_MANY_REDIRECTS","ERR_DEPRECATED","ERR_BAD_RESPONSE","ERR_BAD_REQUEST","ERR_CANCELED","ERR_NOT_SUPPORT","ERR_INVALID_URL"].forEach(e=>{jf[e]={value:e}}),Object.defineProperties(H,jf),Object.defineProperty(Mf,"isAxiosError",{value:!0}),H.from=(e,t,n,r,o,i)=>{const s=Object.create(Mf);w.toFlatObject(e,s,function(u){return u!==Error.prototype},c=>c!=="isAxiosError");const a=e&&e.message?e.message:"Error",l=t==null&&e?e.code:t;return H.call(s,a,l,n,r,o),e&&s.cause==null&&Object.defineProperty(s,"cause",{value:e,configurable:!0}),s.name=e&&e.name||"Error",i&&Object.assign(s,i),s};const xO=null;function ja(e){return w.isPlainObject(e)||w.isArray(e)}function $f(e){return w.endsWith(e,"[]")?e.slice(0,-2):e}function Bf(e,t,n){return e?e.concat(t).map(function(o,i){return o=$f(o),!n&&i?"["+o+"]":o}).join(n?".":""):t}function kO(e){return w.isArray(e)&&!e.some(ja)}const CO=w.toFlatObject(w,{},null,function(t){return/^is[A-Z]/.test(t)});function ki(e,t,n){if(!w.isObject(e))throw new TypeError("target must be an object");t=t||new FormData,n=w.toFlatObject(n,{metaTokens:!0,dots:!1,indexes:!1},!1,function(p,v){return!w.isUndefined(v[p])});const r=n.metaTokens,o=n.visitor||u,i=n.dots,s=n.indexes,l=(n.Blob||typeof Blob<"u"&&Blob)&&w.isSpecCompliantForm(t);if(!w.isFunction(o))throw new TypeError("visitor must be a function");function c(g){if(g===null)return"";if(w.isDate(g))return g.toISOString();if(w.isBoolean(g))return g.toString();if(!l&&w.isBlob(g))throw new H("Blob is not supported. Use a Buffer instead.");return w.isArrayBuffer(g)||w.isTypedArray(g)?l&&typeof Blob=="function"?new Blob([g]):Buffer.from(g):g}function u(g,p,v){let y=g;if(g&&!v&&typeof g=="object"){if(w.endsWith(p,"{}"))p=r?p:p.slice(0,-2),g=JSON.stringify(g);else if(w.isArray(g)&&kO(g)||(w.isFileList(g)||w.endsWith(p,"[]"))&&(y=w.toArray(g)))return p=$f(p),y.forEach(function(b,C){!(w.isUndefined(b)||b===null)&&t.append(s===!0?Bf([p],C,i):s===null?p:p+"[]",c(b))}),!1}return ja(g)?!0:(t.append(Bf(v,p,i),c(g)),!1)}const d=[],h=Object.assign(CO,{defaultVisitor:u,convertValue:c,isVisitable:ja});function m(g,p){if(!w.isUndefined(g)){if(d.indexOf(g)!==-1)throw Error("Circular reference detected in "+p.join("."));d.push(g),w.forEach(g,function(y,x){(!(w.isUndefined(y)||y===null)&&o.call(t,y,w.isString(x)?x.trim():x,p,h))===!0&&m(y,p?p.concat(x):[x])}),d.pop()}}if(!w.isObject(e))throw new TypeError("data must be an object");return m(e),t}function Wf(e){const t={"!":"%21","'":"%27","(":"%28",")":"%29","~":"%7E","%20":"+","%00":"\0"};return encodeURIComponent(e).replace(/[!'()~]|%20|%00/g,function(r){return t[r]})}function $a(e,t){this._pairs=[],e&&ki(e,this,t)}const Hf=$a.prototype;Hf.append=function(t,n){this._pairs.push([t,n])},Hf.toString=function(t){const n=t?function(r){return t.call(this,r,Wf)}:Wf;return this._pairs.map(function(o){return n(o[0])+"="+n(o[1])},"").join("&")};function SO(e){return encodeURIComponent(e).replace(/%3A/gi,":").replace(/%24/g,"$").replace(/%2C/gi,",").replace(/%20/g,"+")}function Uf(e,t,n){if(!t)return e;const r=n&&n.encode||SO;w.isFunction(n)&&(n={serialize:n});const o=n&&n.serialize;let i;if(o?i=o(t,n):i=w.isURLSearchParams(t)?t.toString():new $a(t,n).toString(r),i){const s=e.indexOf("#");s!==-1&&(e=e.slice(0,s)),e+=(e.indexOf("?")===-1?"?":"&")+i}return e}class Gf{constructor(){this.handlers=[]}use(t,n,r){return this.handlers.push({fulfilled:t,rejected:n,synchronous:r?r.synchronous:!1,runWhen:r?r.runWhen:null}),this.handlers.length-1}eject(t){this.handlers[t]&&(this.handlers[t]=null)}clear(){this.handlers&&(this.handlers=[])}forEach(t){w.forEach(this.handlers,function(r){r!==null&&t(r)})}}const qf={silentJSONParsing:!0,forcedJSONParsing:!0,clarifyTimeoutError:!1},wO={isBrowser:!0,classes:{URLSearchParams:typeof URLSearchParams<"u"?URLSearchParams:$a,FormData:typeof FormData<"u"?FormData:null,Blob:typeof Blob<"u"?Blob:null},protocols:["http","https","file","blob","url","data"]},Ba=typeof window<"u"&&typeof document<"u",Wa=typeof navigator=="object"&&navigator||void 0,EO=Ba&&(!Wa||["ReactNative","NativeScript","NS"].indexOf(Wa.product)<0),OO=typeof WorkerGlobalScope<"u"&&self instanceof WorkerGlobalScope&&typeof self.importScripts=="function",PO=Ba&&window.location.href||"http://localhost",Re={...Object.freeze(Object.defineProperty({__proto__:null,hasBrowserEnv:Ba,hasStandardBrowserEnv:EO,hasStandardBrowserWebWorkerEnv:OO,navigator:Wa,origin:PO},Symbol.toStringTag,{value:"Module"})),...wO};function IO(e,t){return ki(e,new Re.classes.URLSearchParams,{visitor:function(n,r,o,i){return Re.isNode&&w.isBuffer(n)?(this.append(r,n.toString("base64")),!1):i.defaultVisitor.apply(this,arguments)},...t})}function RO(e){return w.matchAll(/\w+|\[(\w*)]/g,e).map(t=>t[0]==="[]"?"":t[1]||t[0])}function TO(e){const t={},n=Object.keys(e);let r;const o=n.length;let i;for(r=0;r=n.length;return s=!s&&w.isArray(o)?o.length:s,l?(w.hasOwnProp(o,s)?o[s]=[o[s],r]:o[s]=r,!a):((!o[s]||!w.isObject(o[s]))&&(o[s]=[]),t(n,r,o[s],i)&&w.isArray(o[s])&&(o[s]=TO(o[s])),!a)}if(w.isFormData(e)&&w.isFunction(e.entries)){const n={};return w.forEachEntry(e,(r,o)=>{t(RO(r),o,n,0)}),n}return null}function NO(e,t,n){if(w.isString(e))try{return(t||JSON.parse)(e),w.trim(e)}catch(r){if(r.name!=="SyntaxError")throw r}return(n||JSON.stringify)(e)}const Ur={transitional:qf,adapter:["xhr","http","fetch"],transformRequest:[function(t,n){const r=n.getContentType()||"",o=r.indexOf("application/json")>-1,i=w.isObject(t);if(i&&w.isHTMLForm(t)&&(t=new FormData(t)),w.isFormData(t))return o?JSON.stringify(Kf(t)):t;if(w.isArrayBuffer(t)||w.isBuffer(t)||w.isStream(t)||w.isFile(t)||w.isBlob(t)||w.isReadableStream(t))return t;if(w.isArrayBufferView(t))return t.buffer;if(w.isURLSearchParams(t))return n.setContentType("application/x-www-form-urlencoded;charset=utf-8",!1),t.toString();let a;if(i){if(r.indexOf("application/x-www-form-urlencoded")>-1)return IO(t,this.formSerializer).toString();if((a=w.isFileList(t))||r.indexOf("multipart/form-data")>-1){const l=this.env&&this.env.FormData;return ki(a?{"files[]":t}:t,l&&new l,this.formSerializer)}}return i||o?(n.setContentType("application/json",!1),NO(t)):t}],transformResponse:[function(t){const n=this.transitional||Ur.transitional,r=n&&n.forcedJSONParsing,o=this.responseType==="json";if(w.isResponse(t)||w.isReadableStream(t))return t;if(t&&w.isString(t)&&(r&&!this.responseType||o)){const s=!(n&&n.silentJSONParsing)&&o;try{return JSON.parse(t,this.parseReviver)}catch(a){if(s)throw a.name==="SyntaxError"?H.from(a,H.ERR_BAD_RESPONSE,this,null,this.response):a}}return t}],timeout:0,xsrfCookieName:"XSRF-TOKEN",xsrfHeaderName:"X-XSRF-TOKEN",maxContentLength:-1,maxBodyLength:-1,env:{FormData:Re.classes.FormData,Blob:Re.classes.Blob},validateStatus:function(t){return t>=200&&t<300},headers:{common:{Accept:"application/json, text/plain, */*","Content-Type":void 0}}};w.forEach(["delete","get","head","post","put","patch"],e=>{Ur.headers[e]={}});const AO=w.toObjectSet(["age","authorization","content-length","content-type","etag","expires","from","host","if-modified-since","if-unmodified-since","last-modified","location","max-forwards","proxy-authorization","referer","retry-after","user-agent"]),_O=e=>{const t={};let n,r,o;return e&&e.split(` -`).forEach(function(s){o=s.indexOf(":"),n=s.substring(0,o).trim().toLowerCase(),r=s.substring(o+1).trim(),!(!n||t[n]&&AO[n])&&(n==="set-cookie"?t[n]?t[n].push(r):t[n]=[r]:t[n]=t[n]?t[n]+", "+r:r)}),t},Yf=Symbol("internals");function Gr(e){return e&&String(e).trim().toLowerCase()}function Ci(e){return e===!1||e==null?e:w.isArray(e)?e.map(Ci):String(e)}function VO(e){const t=Object.create(null),n=/([^\s,;=]+)\s*(?:=\s*([^,;]+))?/g;let r;for(;r=n.exec(e);)t[r[1]]=r[2];return t}const FO=e=>/^[-_a-zA-Z0-9^`|~,!#$%&'*+.]+$/.test(e.trim());function Ha(e,t,n,r,o){if(w.isFunction(r))return r.call(this,t,n);if(o&&(t=n),!!w.isString(t)){if(w.isString(r))return t.indexOf(r)!==-1;if(w.isRegExp(r))return r.test(t)}}function LO(e){return e.trim().toLowerCase().replace(/([a-z\d])(\w*)/g,(t,n,r)=>n.toUpperCase()+r)}function DO(e,t){const n=w.toCamelCase(" "+t);["get","set","has"].forEach(r=>{Object.defineProperty(e,r+n,{value:function(o,i,s){return this[r].call(this,t,o,i,s)},configurable:!0})})}let Me=class{constructor(t){t&&this.set(t)}set(t,n,r){const o=this;function i(a,l,c){const u=Gr(l);if(!u)throw new Error("header name must be a non-empty string");const d=w.findKey(o,u);(!d||o[d]===void 0||c===!0||c===void 0&&o[d]!==!1)&&(o[d||l]=Ci(a))}const s=(a,l)=>w.forEach(a,(c,u)=>i(c,u,l));if(w.isPlainObject(t)||t instanceof this.constructor)s(t,n);else if(w.isString(t)&&(t=t.trim())&&!FO(t))s(_O(t),n);else if(w.isObject(t)&&w.isIterable(t)){let a={},l,c;for(const u of t){if(!w.isArray(u))throw TypeError("Object iterator must return a key-value pair");a[c=u[0]]=(l=a[c])?w.isArray(l)?[...l,u[1]]:[l,u[1]]:u[1]}s(a,n)}else t!=null&&i(n,t,r);return this}get(t,n){if(t=Gr(t),t){const r=w.findKey(this,t);if(r){const o=this[r];if(!n)return o;if(n===!0)return VO(o);if(w.isFunction(n))return n.call(this,o,r);if(w.isRegExp(n))return n.exec(o);throw new TypeError("parser must be boolean|regexp|function")}}}has(t,n){if(t=Gr(t),t){const r=w.findKey(this,t);return!!(r&&this[r]!==void 0&&(!n||Ha(this,this[r],r,n)))}return!1}delete(t,n){const r=this;let o=!1;function i(s){if(s=Gr(s),s){const a=w.findKey(r,s);a&&(!n||Ha(r,r[a],a,n))&&(delete r[a],o=!0)}}return w.isArray(t)?t.forEach(i):i(t),o}clear(t){const n=Object.keys(this);let r=n.length,o=!1;for(;r--;){const i=n[r];(!t||Ha(this,this[i],i,t,!0))&&(delete this[i],o=!0)}return o}normalize(t){const n=this,r={};return w.forEach(this,(o,i)=>{const s=w.findKey(r,i);if(s){n[s]=Ci(o),delete n[i];return}const a=t?LO(i):String(i).trim();a!==i&&delete n[i],n[a]=Ci(o),r[a]=!0}),this}concat(...t){return this.constructor.concat(this,...t)}toJSON(t){const n=Object.create(null);return w.forEach(this,(r,o)=>{r!=null&&r!==!1&&(n[o]=t&&w.isArray(r)?r.join(", "):r)}),n}[Symbol.iterator](){return Object.entries(this.toJSON())[Symbol.iterator]()}toString(){return Object.entries(this.toJSON()).map(([t,n])=>t+": "+n).join(` -`)}getSetCookie(){return this.get("set-cookie")||[]}get[Symbol.toStringTag](){return"AxiosHeaders"}static from(t){return t instanceof this?t:new this(t)}static concat(t,...n){const r=new this(t);return n.forEach(o=>r.set(o)),r}static accessor(t){const r=(this[Yf]=this[Yf]={accessors:{}}).accessors,o=this.prototype;function i(s){const a=Gr(s);r[a]||(DO(o,s),r[a]=!0)}return w.isArray(t)?t.forEach(i):i(t),this}};Me.accessor(["Content-Type","Content-Length","Accept","Accept-Encoding","User-Agent","Authorization"]),w.reduceDescriptors(Me.prototype,({value:e},t)=>{let n=t[0].toUpperCase()+t.slice(1);return{get:()=>e,set(r){this[n]=r}}}),w.freezeMethods(Me);function Ua(e,t){const n=this||Ur,r=t||n,o=Me.from(r.headers);let i=r.data;return w.forEach(e,function(a){i=a.call(n,i,o.normalize(),t?t.status:void 0)}),o.normalize(),i}function Xf(e){return!!(e&&e.__CANCEL__)}function sr(e,t,n){H.call(this,e??"canceled",H.ERR_CANCELED,t,n),this.name="CanceledError"}w.inherits(sr,H,{__CANCEL__:!0});function Qf(e,t,n){const r=n.config.validateStatus;!n.status||!r||r(n.status)?e(n):t(new H("Request failed with status code "+n.status,[H.ERR_BAD_REQUEST,H.ERR_BAD_RESPONSE][Math.floor(n.status/100)-4],n.config,n.request,n))}function zO(e){const t=/^([-+\w]{1,25})(:?\/\/|:)/.exec(e);return t&&t[1]||""}function MO(e,t){e=e||10;const n=new Array(e),r=new Array(e);let o=0,i=0,s;return t=t!==void 0?t:1e3,function(l){const c=Date.now(),u=r[i];s||(s=c),n[o]=l,r[o]=c;let d=i,h=0;for(;d!==o;)h+=n[d++],d=d%e;if(o=(o+1)%e,o===i&&(i=(i+1)%e),c-s{n=u,o=null,i&&(clearTimeout(i),i=null),e(...c)};return[(...c)=>{const u=Date.now(),d=u-n;d>=r?s(c,u):(o=c,i||(i=setTimeout(()=>{i=null,s(o)},r-d)))},()=>o&&s(o)]}const Si=(e,t,n=3)=>{let r=0;const o=MO(50,250);return jO(i=>{const s=i.loaded,a=i.lengthComputable?i.total:void 0,l=s-r,c=o(l),u=s<=a;r=s;const d={loaded:s,total:a,progress:a?s/a:void 0,bytes:l,rate:c||void 0,estimated:c&&a&&u?(a-s)/c:void 0,event:i,lengthComputable:a!=null,[t?"download":"upload"]:!0};e(d)},n)},Jf=(e,t)=>{const n=e!=null;return[r=>t[0]({lengthComputable:n,total:e,loaded:r}),t[1]]},Zf=e=>(...t)=>w.asap(()=>e(...t)),$O=Re.hasStandardBrowserEnv?((e,t)=>n=>(n=new URL(n,Re.origin),e.protocol===n.protocol&&e.host===n.host&&(t||e.port===n.port)))(new URL(Re.origin),Re.navigator&&/(msie|trident)/i.test(Re.navigator.userAgent)):()=>!0,BO=Re.hasStandardBrowserEnv?{write(e,t,n,r,o,i){const s=[e+"="+encodeURIComponent(t)];w.isNumber(n)&&s.push("expires="+new Date(n).toGMTString()),w.isString(r)&&s.push("path="+r),w.isString(o)&&s.push("domain="+o),i===!0&&s.push("secure"),document.cookie=s.join("; ")},read(e){const t=document.cookie.match(new RegExp("(^|;\\s*)("+e+")=([^;]*)"));return t?decodeURIComponent(t[3]):null},remove(e){this.write(e,"",Date.now()-864e5)}}:{write(){},read(){return null},remove(){}};function WO(e){return/^([a-z][a-z\d+\-.]*:)?\/\//i.test(e)}function HO(e,t){return t?e.replace(/\/?\/$/,"")+"/"+t.replace(/^\/+/,""):e}function eg(e,t,n){let r=!WO(t);return e&&(r||n==!1)?HO(e,t):t}const tg=e=>e instanceof Me?{...e}:e;function Pn(e,t){t=t||{};const n={};function r(c,u,d,h){return w.isPlainObject(c)&&w.isPlainObject(u)?w.merge.call({caseless:h},c,u):w.isPlainObject(u)?w.merge({},u):w.isArray(u)?u.slice():u}function o(c,u,d,h){if(w.isUndefined(u)){if(!w.isUndefined(c))return r(void 0,c,d,h)}else return r(c,u,d,h)}function i(c,u){if(!w.isUndefined(u))return r(void 0,u)}function s(c,u){if(w.isUndefined(u)){if(!w.isUndefined(c))return r(void 0,c)}else return r(void 0,u)}function a(c,u,d){if(d in t)return r(c,u);if(d in e)return r(void 0,c)}const l={url:i,method:i,data:i,baseURL:s,transformRequest:s,transformResponse:s,paramsSerializer:s,timeout:s,timeoutMessage:s,withCredentials:s,withXSRFToken:s,adapter:s,responseType:s,xsrfCookieName:s,xsrfHeaderName:s,onUploadProgress:s,onDownloadProgress:s,decompress:s,maxContentLength:s,maxBodyLength:s,beforeRedirect:s,transport:s,httpAgent:s,httpsAgent:s,cancelToken:s,socketPath:s,responseEncoding:s,validateStatus:a,headers:(c,u,d)=>o(tg(c),tg(u),d,!0)};return w.forEach(Object.keys({...e,...t}),function(u){const d=l[u]||o,h=d(e[u],t[u],u);w.isUndefined(h)&&d!==a||(n[u]=h)}),n}const ng=e=>{const t=Pn({},e);let{data:n,withXSRFToken:r,xsrfHeaderName:o,xsrfCookieName:i,headers:s,auth:a}=t;if(t.headers=s=Me.from(s),t.url=Uf(eg(t.baseURL,t.url,t.allowAbsoluteUrls),e.params,e.paramsSerializer),a&&s.set("Authorization","Basic "+btoa((a.username||"")+":"+(a.password?unescape(encodeURIComponent(a.password)):""))),w.isFormData(n)){if(Re.hasStandardBrowserEnv||Re.hasStandardBrowserWebWorkerEnv)s.setContentType(void 0);else if(w.isFunction(n.getHeaders)){const l=n.getHeaders(),c=["content-type","content-length"];Object.entries(l).forEach(([u,d])=>{c.includes(u.toLowerCase())&&s.set(u,d)})}}if(Re.hasStandardBrowserEnv&&(r&&w.isFunction(r)&&(r=r(t)),r||r!==!1&&$O(t.url))){const l=o&&i&&BO.read(i);l&&s.set(o,l)}return t},UO=typeof XMLHttpRequest<"u"&&function(e){return new Promise(function(n,r){const o=ng(e);let i=o.data;const s=Me.from(o.headers).normalize();let{responseType:a,onUploadProgress:l,onDownloadProgress:c}=o,u,d,h,m,g;function p(){m&&m(),g&&g(),o.cancelToken&&o.cancelToken.unsubscribe(u),o.signal&&o.signal.removeEventListener("abort",u)}let v=new XMLHttpRequest;v.open(o.method.toUpperCase(),o.url,!0),v.timeout=o.timeout;function y(){if(!v)return;const b=Me.from("getAllResponseHeaders"in v&&v.getAllResponseHeaders()),k={data:!a||a==="text"||a==="json"?v.responseText:v.response,status:v.status,statusText:v.statusText,headers:b,config:e,request:v};Qf(function(I){n(I),p()},function(I){r(I),p()},k),v=null}"onloadend"in v?v.onloadend=y:v.onreadystatechange=function(){!v||v.readyState!==4||v.status===0&&!(v.responseURL&&v.responseURL.indexOf("file:")===0)||setTimeout(y)},v.onabort=function(){v&&(r(new H("Request aborted",H.ECONNABORTED,e,v)),v=null)},v.onerror=function(C){const k=C&&C.message?C.message:"Network Error",E=new H(k,H.ERR_NETWORK,e,v);E.event=C||null,r(E),v=null},v.ontimeout=function(){let C=o.timeout?"timeout of "+o.timeout+"ms exceeded":"timeout exceeded";const k=o.transitional||qf;o.timeoutErrorMessage&&(C=o.timeoutErrorMessage),r(new H(C,k.clarifyTimeoutError?H.ETIMEDOUT:H.ECONNABORTED,e,v)),v=null},i===void 0&&s.setContentType(null),"setRequestHeader"in v&&w.forEach(s.toJSON(),function(C,k){v.setRequestHeader(k,C)}),w.isUndefined(o.withCredentials)||(v.withCredentials=!!o.withCredentials),a&&a!=="json"&&(v.responseType=o.responseType),c&&([h,g]=Si(c,!0),v.addEventListener("progress",h)),l&&v.upload&&([d,m]=Si(l),v.upload.addEventListener("progress",d),v.upload.addEventListener("loadend",m)),(o.cancelToken||o.signal)&&(u=b=>{v&&(r(!b||b.type?new sr(null,e,v):b),v.abort(),v=null)},o.cancelToken&&o.cancelToken.subscribe(u),o.signal&&(o.signal.aborted?u():o.signal.addEventListener("abort",u)));const x=zO(o.url);if(x&&Re.protocols.indexOf(x)===-1){r(new H("Unsupported protocol "+x+":",H.ERR_BAD_REQUEST,e));return}v.send(i||null)})},GO=(e,t)=>{const{length:n}=e=e?e.filter(Boolean):[];if(t||n){let r=new AbortController,o;const i=function(c){if(!o){o=!0,a();const u=c instanceof Error?c:this.reason;r.abort(u instanceof H?u:new sr(u instanceof Error?u.message:u))}};let s=t&&setTimeout(()=>{s=null,i(new H(`timeout ${t} of ms exceeded`,H.ETIMEDOUT))},t);const a=()=>{e&&(s&&clearTimeout(s),s=null,e.forEach(c=>{c.unsubscribe?c.unsubscribe(i):c.removeEventListener("abort",i)}),e=null)};e.forEach(c=>c.addEventListener("abort",i));const{signal:l}=r;return l.unsubscribe=()=>w.asap(a),l}},qO=function*(e,t){let n=e.byteLength;if(n{const o=KO(e,t);let i=0,s,a=l=>{s||(s=!0,r&&r(l))};return new ReadableStream({async pull(l){try{const{done:c,value:u}=await o.next();if(c){a(),l.close();return}let d=u.byteLength;if(n){let h=i+=d;n(h)}l.enqueue(new Uint8Array(u))}catch(c){throw a(c),c}},cancel(l){return a(l),o.return()}},{highWaterMark:2})},og=64*1024,{isFunction:wi}=w,XO=(({Request:e,Response:t})=>({Request:e,Response:t}))(w.global),{ReadableStream:ig,TextEncoder:sg}=w.global,ag=(e,...t)=>{try{return!!e(...t)}catch{return!1}},QO=e=>{e=w.merge.call({skipUndefined:!0},XO,e);const{fetch:t,Request:n,Response:r}=e,o=t?wi(t):typeof fetch=="function",i=wi(n),s=wi(r);if(!o)return!1;const a=o&&wi(ig),l=o&&(typeof sg=="function"?(g=>p=>g.encode(p))(new sg):async g=>new Uint8Array(await new n(g).arrayBuffer())),c=i&&a&&ag(()=>{let g=!1;const p=new n(Re.origin,{body:new ig,method:"POST",get duplex(){return g=!0,"half"}}).headers.has("Content-Type");return g&&!p}),u=s&&a&&ag(()=>w.isReadableStream(new r("").body)),d={stream:u&&(g=>g.body)};o&&["text","arrayBuffer","blob","formData","stream"].forEach(g=>{!d[g]&&(d[g]=(p,v)=>{let y=p&&p[g];if(y)return y.call(p);throw new H(`Response type '${g}' is not supported`,H.ERR_NOT_SUPPORT,v)})});const h=async g=>{if(g==null)return 0;if(w.isBlob(g))return g.size;if(w.isSpecCompliantForm(g))return(await new n(Re.origin,{method:"POST",body:g}).arrayBuffer()).byteLength;if(w.isArrayBufferView(g)||w.isArrayBuffer(g))return g.byteLength;if(w.isURLSearchParams(g)&&(g=g+""),w.isString(g))return(await l(g)).byteLength},m=async(g,p)=>{const v=w.toFiniteNumber(g.getContentLength());return v??h(p)};return async g=>{let{url:p,method:v,data:y,signal:x,cancelToken:b,timeout:C,onDownloadProgress:k,onUploadProgress:E,responseType:I,headers:R,withCredentials:_="same-origin",fetchOptions:A}=ng(g),T=t||fetch;I=I?(I+"").toLowerCase():"text";let S=GO([x,b&&b.toAbortSignal()],C),N=null;const D=S&&S.unsubscribe&&(()=>{S.unsubscribe()});let z;try{if(E&&c&&v!=="get"&&v!=="head"&&(z=await m(R,y))!==0){let Z=new n(p,{method:"POST",body:y,duplex:"half"}),pe;if(w.isFormData(y)&&(pe=Z.headers.get("content-type"))&&R.setContentType(pe),Z.body){const[Je,vt]=Jf(z,Si(Zf(E)));y=rg(Z.body,og,Je,vt)}}w.isString(_)||(_=_?"include":"omit");const W=i&&"credentials"in n.prototype,J={...A,signal:S,method:v.toUpperCase(),headers:R.normalize().toJSON(),body:y,duplex:"half",credentials:W?_:void 0};N=i&&new n(p,J);let ee=await(i?T(N,A):T(p,J));const $=u&&(I==="stream"||I==="response");if(u&&(k||$&&D)){const Z={};["status","statusText","headers"].forEach(It=>{Z[It]=ee[It]});const pe=w.toFiniteNumber(ee.headers.get("content-length")),[Je,vt]=k&&Jf(pe,Si(Zf(k),!0))||[];ee=new r(rg(ee.body,og,Je,()=>{vt&&vt(),D&&D()}),Z)}I=I||"text";let q=await d[w.findKey(d,I)||"text"](ee,g);return!$&&D&&D(),await new Promise((Z,pe)=>{Qf(Z,pe,{data:q,headers:Me.from(ee.headers),status:ee.status,statusText:ee.statusText,config:g,request:N})})}catch(W){throw D&&D(),W&&W.name==="TypeError"&&/Load failed|fetch/i.test(W.message)?Object.assign(new H("Network Error",H.ERR_NETWORK,g,N),{cause:W.cause||W}):H.from(W,W&&W.code,g,N)}}},JO=new Map,lg=e=>{let t=e?e.env:{};const{fetch:n,Request:r,Response:o}=t,i=[r,o,n];let s=i.length,a=s,l,c,u=JO;for(;a--;)l=i[a],c=u.get(l),c===void 0&&u.set(l,c=a?new Map:QO(t)),u=c;return c};lg();const Ga={http:xO,xhr:UO,fetch:{get:lg}};w.forEach(Ga,(e,t)=>{if(e){try{Object.defineProperty(e,"name",{value:t})}catch{}Object.defineProperty(e,"adapterName",{value:t})}});const cg=e=>`- ${e}`,ZO=e=>w.isFunction(e)||e===null||e===!1,ug={getAdapter:(e,t)=>{e=w.isArray(e)?e:[e];const{length:n}=e;let r,o;const i={};for(let s=0;s`adapter ${l} `+(c===!1?"is not supported by the environment":"is not available in the build"));let a=n?s.length>1?`since : -`+s.map(cg).join(` -`):" "+cg(s[0]):"as no adapter specified";throw new H("There is no suitable adapter to dispatch the request "+a,"ERR_NOT_SUPPORT")}return o},adapters:Ga};function qa(e){if(e.cancelToken&&e.cancelToken.throwIfRequested(),e.signal&&e.signal.aborted)throw new sr(null,e)}function dg(e){return qa(e),e.headers=Me.from(e.headers),e.data=Ua.call(e,e.transformRequest),["post","put","patch"].indexOf(e.method)!==-1&&e.headers.setContentType("application/x-www-form-urlencoded",!1),ug.getAdapter(e.adapter||Ur.adapter,e)(e).then(function(r){return qa(e),r.data=Ua.call(e,e.transformResponse,r),r.headers=Me.from(r.headers),r},function(r){return Xf(r)||(qa(e),r&&r.response&&(r.response.data=Ua.call(e,e.transformResponse,r.response),r.response.headers=Me.from(r.response.headers))),Promise.reject(r)})}const hg="1.12.2",Ei={};["object","boolean","number","function","string","symbol"].forEach((e,t)=>{Ei[e]=function(r){return typeof r===e||"a"+(t<1?"n ":" ")+e}});const fg={};Ei.transitional=function(t,n,r){function o(i,s){return"[Axios v"+hg+"] Transitional option '"+i+"'"+s+(r?". "+r:"")}return(i,s,a)=>{if(t===!1)throw new H(o(s," has been removed"+(n?" in "+n:"")),H.ERR_DEPRECATED);return n&&!fg[s]&&(fg[s]=!0,console.warn(o(s," has been deprecated since v"+n+" and will be removed in the near future"))),t?t(i,s,a):!0}},Ei.spelling=function(t){return(n,r)=>(console.warn(`${r} is likely a misspelling of ${t}`),!0)};function eP(e,t,n){if(typeof e!="object")throw new H("options must be an object",H.ERR_BAD_OPTION_VALUE);const r=Object.keys(e);let o=r.length;for(;o-- >0;){const i=r[o],s=t[i];if(s){const a=e[i],l=a===void 0||s(a,i,e);if(l!==!0)throw new H("option "+i+" must be "+l,H.ERR_BAD_OPTION_VALUE);continue}if(n!==!0)throw new H("Unknown option "+i,H.ERR_BAD_OPTION)}}const Oi={assertOptions:eP,validators:Ei},mt=Oi.validators;let In=class{constructor(t){this.defaults=t||{},this.interceptors={request:new Gf,response:new Gf}}async request(t,n){try{return await this._request(t,n)}catch(r){if(r instanceof Error){let o={};Error.captureStackTrace?Error.captureStackTrace(o):o=new Error;const i=o.stack?o.stack.replace(/^.+\n/,""):"";try{r.stack?i&&!String(r.stack).endsWith(i.replace(/^.+\n.+\n/,""))&&(r.stack+=` -`+i):r.stack=i}catch{}}throw r}}_request(t,n){typeof t=="string"?(n=n||{},n.url=t):n=t||{},n=Pn(this.defaults,n);const{transitional:r,paramsSerializer:o,headers:i}=n;r!==void 0&&Oi.assertOptions(r,{silentJSONParsing:mt.transitional(mt.boolean),forcedJSONParsing:mt.transitional(mt.boolean),clarifyTimeoutError:mt.transitional(mt.boolean)},!1),o!=null&&(w.isFunction(o)?n.paramsSerializer={serialize:o}:Oi.assertOptions(o,{encode:mt.function,serialize:mt.function},!0)),n.allowAbsoluteUrls!==void 0||(this.defaults.allowAbsoluteUrls!==void 0?n.allowAbsoluteUrls=this.defaults.allowAbsoluteUrls:n.allowAbsoluteUrls=!0),Oi.assertOptions(n,{baseUrl:mt.spelling("baseURL"),withXsrfToken:mt.spelling("withXSRFToken")},!0),n.method=(n.method||this.defaults.method||"get").toLowerCase();let s=i&&w.merge(i.common,i[n.method]);i&&w.forEach(["delete","get","head","post","put","patch","common"],g=>{delete i[g]}),n.headers=Me.concat(s,i);const a=[];let l=!0;this.interceptors.request.forEach(function(p){typeof p.runWhen=="function"&&p.runWhen(n)===!1||(l=l&&p.synchronous,a.unshift(p.fulfilled,p.rejected))});const c=[];this.interceptors.response.forEach(function(p){c.push(p.fulfilled,p.rejected)});let u,d=0,h;if(!l){const g=[dg.bind(this),void 0];for(g.unshift(...a),g.push(...c),h=g.length,u=Promise.resolve(n);d{if(!r._listeners)return;let i=r._listeners.length;for(;i-- >0;)r._listeners[i](o);r._listeners=null}),this.promise.then=o=>{let i;const s=new Promise(a=>{r.subscribe(a),i=a}).then(o);return s.cancel=function(){r.unsubscribe(i)},s},t(function(i,s,a){r.reason||(r.reason=new sr(i,s,a),n(r.reason))})}throwIfRequested(){if(this.reason)throw this.reason}subscribe(t){if(this.reason){t(this.reason);return}this._listeners?this._listeners.push(t):this._listeners=[t]}unsubscribe(t){if(!this._listeners)return;const n=this._listeners.indexOf(t);n!==-1&&this._listeners.splice(n,1)}toAbortSignal(){const t=new AbortController,n=r=>{t.abort(r)};return this.subscribe(n),t.signal.unsubscribe=()=>this.unsubscribe(n),t.signal}static source(){let t;return{token:new fp(function(o){t=o}),cancel:t}}};function nP(e){return function(n){return e.apply(null,n)}}function rP(e){return w.isObject(e)&&e.isAxiosError===!0}const Ka={Continue:100,SwitchingProtocols:101,Processing:102,EarlyHints:103,Ok:200,Created:201,Accepted:202,NonAuthoritativeInformation:203,NoContent:204,ResetContent:205,PartialContent:206,MultiStatus:207,AlreadyReported:208,ImUsed:226,MultipleChoices:300,MovedPermanently:301,Found:302,SeeOther:303,NotModified:304,UseProxy:305,Unused:306,TemporaryRedirect:307,PermanentRedirect:308,BadRequest:400,Unauthorized:401,PaymentRequired:402,Forbidden:403,NotFound:404,MethodNotAllowed:405,NotAcceptable:406,ProxyAuthenticationRequired:407,RequestTimeout:408,Conflict:409,Gone:410,LengthRequired:411,PreconditionFailed:412,PayloadTooLarge:413,UriTooLong:414,UnsupportedMediaType:415,RangeNotSatisfiable:416,ExpectationFailed:417,ImATeapot:418,MisdirectedRequest:421,UnprocessableEntity:422,Locked:423,FailedDependency:424,TooEarly:425,UpgradeRequired:426,PreconditionRequired:428,TooManyRequests:429,RequestHeaderFieldsTooLarge:431,UnavailableForLegalReasons:451,InternalServerError:500,NotImplemented:501,BadGateway:502,ServiceUnavailable:503,GatewayTimeout:504,HttpVersionNotSupported:505,VariantAlsoNegotiates:506,InsufficientStorage:507,LoopDetected:508,NotExtended:510,NetworkAuthenticationRequired:511};Object.entries(Ka).forEach(([e,t])=>{Ka[t]=e});function gg(e){const t=new In(e),n=Tf(In.prototype.request,t);return w.extend(n,In.prototype,t,{allOwnKeys:!0}),w.extend(n,t,null,{allOwnKeys:!0}),n.create=function(o){return gg(Pn(e,o))},n}const le=gg(Ur);le.Axios=In,le.CanceledError=sr,le.CancelToken=tP,le.isCancel=Xf,le.VERSION=hg,le.toFormData=ki,le.AxiosError=H,le.Cancel=le.CanceledError,le.all=function(t){return Promise.all(t)},le.spread=nP,le.isAxiosError=rP,le.mergeConfig=Pn,le.AxiosHeaders=Me,le.formToJSON=e=>Kf(w.isHTMLForm(e)?new FormData(e):e),le.getAdapter=ug.getAdapter,le.HttpStatusCode=Ka,le.default=le;const{Axios:pT,AxiosError:mT,CanceledError:vT,isCancel:bT,CancelToken:yT,VERSION:xT,all:kT,Cancel:CT,isAxiosError:ST,spread:wT,toFormData:ET,AxiosHeaders:OT,HttpStatusCode:PT,formToJSON:IT,getAdapter:RT,mergeConfig:TT}=le;var oP=(e,t,n,r,o,i,s,a)=>{let l=document.documentElement,c=["light","dark"];function u(m){(Array.isArray(e)?e:[e]).forEach(g=>{let p=g==="class",v=p&&i?o.map(y=>i[y]||y):o;p?(l.classList.remove(...v),l.classList.add(i&&i[m]?i[m]:m)):l.setAttribute(g,m)}),d(m)}function d(m){a&&c.includes(m)&&(l.style.colorScheme=m)}function h(){return window.matchMedia("(prefers-color-scheme: dark)").matches?"dark":"light"}if(r)u(r);else try{let m=localStorage.getItem(t)||n,g=s&&m==="system"?h():m;u(g)}catch{}},pg=["light","dark"],mg="(prefers-color-scheme: dark)",iP=typeof window>"u",vg=O.createContext(void 0),sP=e=>O.useContext(vg)?O.createElement(O.Fragment,null,e.children):O.createElement(lP,{...e}),aP=["light","dark"],lP=({forcedTheme:e,disableTransitionOnChange:t=!1,enableSystem:n=!0,enableColorScheme:r=!0,storageKey:o="theme",themes:i=aP,defaultTheme:s=n?"system":"light",attribute:a="data-theme",value:l,children:c,nonce:u,scriptProps:d})=>{let[h,m]=O.useState(()=>uP(o,s)),[g,p]=O.useState(()=>h==="system"?Ya():h),v=l?Object.values(l):i,y=O.useCallback(k=>{let E=k;if(!E)return;k==="system"&&n&&(E=Ya());let I=l?l[E]:E,R=t?dP(u):null,_=document.documentElement,A=T=>{T==="class"?(_.classList.remove(...v),I&&_.classList.add(I)):T.startsWith("data-")&&(I?_.setAttribute(T,I):_.removeAttribute(T))};if(Array.isArray(a)?a.forEach(A):A(a),r){let T=pg.includes(s)?s:null,S=pg.includes(E)?E:T;_.style.colorScheme=S}R?.()},[u]),x=O.useCallback(k=>{let E=typeof k=="function"?k(h):k;m(E);try{localStorage.setItem(o,E)}catch{}},[h]),b=O.useCallback(k=>{let E=Ya(k);p(E),h==="system"&&n&&!e&&y("system")},[h,e]);O.useEffect(()=>{let k=window.matchMedia(mg);return k.addListener(b),b(k),()=>k.removeListener(b)},[b]),O.useEffect(()=>{let k=E=>{E.key===o&&(E.newValue?m(E.newValue):x(s))};return window.addEventListener("storage",k),()=>window.removeEventListener("storage",k)},[x]),O.useEffect(()=>{y(e??h)},[e,h]);let C=O.useMemo(()=>({theme:h,setTheme:x,forcedTheme:e,resolvedTheme:h==="system"?g:h,themes:n?[...i,"system"]:i,systemTheme:n?g:void 0}),[h,x,e,g,n,i]);return O.createElement(vg.Provider,{value:C},O.createElement(cP,{forcedTheme:e,storageKey:o,attribute:a,enableSystem:n,enableColorScheme:r,defaultTheme:s,value:l,themes:i,nonce:u,scriptProps:d}),c)},cP=O.memo(({forcedTheme:e,storageKey:t,attribute:n,enableSystem:r,enableColorScheme:o,defaultTheme:i,value:s,themes:a,nonce:l,scriptProps:c})=>{let u=JSON.stringify([n,t,i,e,a,s,r,o]).slice(1,-1);return O.createElement("script",{...c,suppressHydrationWarning:!0,nonce:typeof window>"u"?l:"",dangerouslySetInnerHTML:{__html:`(${oP.toString()})(${u})`}})}),uP=(e,t)=>{if(iP)return;let n;try{n=localStorage.getItem(e)||void 0}catch{}return n||t},dP=e=>{let t=document.createElement("style");return e&&t.setAttribute("nonce",e),t.appendChild(document.createTextNode("*,*::before,*::after{-webkit-transition:none!important;-moz-transition:none!important;-o-transition:none!important;-ms-transition:none!important;transition:none!important}")),document.head.appendChild(t),()=>{window.getComputedStyle(document.body),setTimeout(()=>{document.head.removeChild(t)},1)}},Ya=e=>(e||(e=window.matchMedia(mg)),e.matches?"dark":"light");const hP=e=>f.jsx(sP,{attribute:"class",disableTransitionOnChange:!0,...e});/** - * @remix-run/router v1.23.0 +)+\\(\\s*min(-device)?-${e}`,"i"),max:new RegExp(`\\(\\s*max(-device)?-${e}`,"i")}),yC=uh("width"),xC=uh("height"),dh=e=>({isMin:vh(e.minMax,e.maxMin,e.min),isMax:vh(e.maxMin,e.minMax,e.max)}),{isMin:Sa,isMax:hh}=dh(yC),{isMin:Ea,isMax:fh}=dh(xC),gh=/print/i,ph=/^print$/i,kC=/(-?\d*\.?\d+)(ch|em|ex|px|rem)/,CC=/(\d)/,$r=Number.MAX_VALUE,wC={ch:8.8984375,em:16,rem:16,ex:8.296875,px:1};function mh(e){const t=kC.exec(e)||(Sa(e)||Ea(e)?CC.exec(e):null);if(!t)return $r;if(t[0]==="0")return 0;const n=parseFloat(t[1]),r=t[2];return n*(wC[r]||1)}function vh(e,t,n){return r=>e.test(r)||!t.test(r)&&n.test(r)}function SC(e,t){const n=gh.test(e),r=ph.test(e),o=gh.test(t),i=ph.test(t);return n&&o?!r&&i?1:r&&!i?-1:e.localeCompare(t):n?1:o?-1:null}const EC=rt((e,t)=>{const n=SC(e,t);if(n!==null)return n;const r=Sa(e)||Ea(e),o=hh(e)||fh(e),i=Sa(t)||Ea(t),s=hh(t)||fh(t);if(r&&s)return-1;if(o&&i)return 1;const a=mh(e),l=mh(t);return a===$r&&l===$r?e.localeCompare(t):a===$r?1:l===$r?-1:a!==l?a>l?o?-1:1:o?1:-1:e.localeCompare(t)});function bh(e){return e.sort(([t],[n])=>EC(t,n))}function yh(e){const t=[],n=[],r={};for(const[s,a]of Object.entries(e))s.startsWith("@media")?t.push([s,a]):s.startsWith("@container")?n.push([s,a]):_e(a)?r[s]=yh(a):r[s]=a;const o=bh(t),i=bh(n);return{...r,...Object.fromEntries(o),...Object.fromEntries(i)}}const xh=/\s*!(important)?/i,OC=e=>lt(e)?xh.test(e):!1,PC=e=>lt(e)?e.replace(xh,"").trim():e;function kh(e){const{transform:t,conditions:n,normalize:r}=e,o=TC(e);return rt(function(...s){const a=o(...s),l=r(a),c=Object.create(null);return gt(l,(u,h)=>{const d=OC(u);if(u==null)return;const[m,...f]=n.sort(h).map(n.resolve);d&&(u=PC(u));let p=t(m,u)??Object.create(null);p=gt(p,v=>lt(v)&&d?`${v} !important`:v,{getKey:v=>n.expandAtRule(v)}),RC(c,f.flat(),p)}),yh(c)})}function RC(e,t,n){let r=e;for(const o of t)o&&(r[o]||(r[o]=Object.create(null)),r=r[o]);er(r,n)}function IC(...e){return e.filter(t=>_e(t)&&Object.keys(yr(t)).length>0)}function TC(e){function t(n){const r=IC(...n);return r.length===1?r:r.map(o=>e.normalize(o))}return rt(function(...r){return er({},...t(r))})}const Ch=e=>({base:{},variants:{},defaultVariants:{},compoundVariants:[],...e});function NC(e){const{css:t,conditions:n,normalize:r,layers:o}=e;function i(a={}){const{base:l,variants:c,defaultVariants:u,compoundVariants:h}=Ch(a),d=kh({conditions:n,normalize:r,transform(y,x){return c[y]?.[x]}}),m=(y={})=>{const x=r({...u,...yr(y)});let w={...l};er(w,d(x));const k=s(h,x);return o.wrap("recipes",t(w,k))},f=Object.keys(c),p=y=>{const x=Qd(y,["recipe"]),[w,k]=Ln(x,f);return f.includes("colorPalette")||(w.colorPalette=y.colorPalette||u.colorPalette),f.includes("orientation")&&(k.orientation=y.orientation),[w,k]},v=Object.fromEntries(Object.entries(c).map(([y,x])=>[y,Object.keys(x)]));return Object.assign(y=>t(m(y)),{className:a.className,__cva__:!0,variantMap:v,variantKeys:f,raw:m,config:a,splitVariantProps:p,merge(y){return i(AC(e)(this,y))}})}function s(a,l){let c={};return a.forEach(u=>{Object.entries(u).every(([d,m])=>d==="css"?!0:(Array.isArray(m)?m:[m]).some(p=>l[d]===p))&&(c=t(c,u.css))}),c}return i}function AC(e){const{css:t}=e;return function(r,o){const i=Ch(o.config),s=Wl(r.variantKeys,Object.keys(o.variants)),a=t(r.base,i.base),l=Object.fromEntries(s.map(d=>[d,t(r.config.variants[d],i.variants[d])])),c=er(r.config.defaultVariants,i.defaultVariants),u=[...r.compoundVariants,...i.compoundVariants];return{className:Ge(r.className,o.className),base:a,variants:l,defaultVariants:c,compoundVariants:u}}}const _C={reset:"reset",base:"base",tokens:"tokens",recipes:"recipes"},wh={reset:0,base:1,tokens:2,recipes:3};function VC(e){const t=e.layers??_C,r=Object.values(t).sort((o,i)=>wh[o]-wh[i]);return{names:r,atRule:`@layer ${r.join(", ")};`,wrap(o,i){return e.disableLayers?i:{[`@layer ${t[o]}`]:i}}}}function FC(e){const{utility:t,normalize:n}=e,{hasShorthand:r,resolveShorthand:o}=t;return function(i){return gt(i,n,{stop:s=>Array.isArray(s),getKey:r?o:void 0})}}function LC(e){const{preflight:t}=e;if(!t)return{};const{scope:n="",level:r="parent"}=_e(t)?t:{};let o="";n&&r==="parent"?o=`${n} `:n&&r==="element"&&(o=`&${n}`);const i={"*":{margin:"0px",padding:"0px",font:"inherit",wordWrap:"break-word",WebkitTapHighlightColor:"transparent"},"*, *::before, *::after, *::backdrop":{boxSizing:"border-box",borderWidth:"0px",borderStyle:"solid",borderColor:"var(--global-color-border, currentColor)"},hr:{height:"0px",color:"inherit",borderTopWidth:"1px"},body:{minHeight:"100dvh",position:"relative"},img:{borderStyle:"none"},"img, svg, video, canvas, audio, iframe, embed, object":{display:"block",verticalAlign:"middle"},iframe:{border:"none"},"img, video":{maxWidth:"100%",height:"auto"},"p, h1, h2, h3, h4, h5, h6":{overflowWrap:"break-word"},"ol, ul":{listStyle:"none"},"code, kbd, pre, samp":{fontSize:"1em"},"button, [type='button'], [type='reset'], [type='submit']":{WebkitAppearance:"button",backgroundColor:"transparent",backgroundImage:"none"},"button, input, optgroup, select, textarea":{color:"inherit"},"button, select":{textTransform:"none"},table:{textIndent:"0px",borderColor:"inherit",borderCollapse:"collapse"},"*::placeholder":{opacity:"unset",color:"#9ca3af",userSelect:"none"},textarea:{resize:"vertical"},summary:{display:"list-item"},small:{fontSize:"80%"},"sub, sup":{fontSize:"75%",lineHeight:0,position:"relative",verticalAlign:"baseline"},sub:{bottom:"-0.25em"},sup:{top:"-0.5em"},dialog:{padding:"0px"},a:{color:"inherit",textDecoration:"inherit"},"abbr:where([title])":{textDecoration:"underline dotted"},"b, strong":{fontWeight:"bolder"},"code, kbd, samp, pre":{fontSize:"1em","--font-mono-fallback":"ui-monospace, SFMono-Regular, Menlo, Monaco, Consolas, 'Liberation Mono', 'Courier New'",fontFamily:"var(--global-font-mono, var(--font-mono-fallback))"},'input[type="text"], input[type="email"], input[type="search"], input[type="password"]':{WebkitAppearance:"none",MozAppearance:"none"},"input[type='search']":{WebkitAppearance:"textfield",outlineOffset:"-2px"},"::-webkit-search-decoration, ::-webkit-search-cancel-button":{WebkitAppearance:"none"},"::-webkit-file-upload-button":{WebkitAppearance:"button",font:"inherit"},'input[type="number"]::-webkit-inner-spin-button, input[type="number"]::-webkit-outer-spin-button':{height:"auto"},"input[type='number']":{MozAppearance:"textfield"},":-moz-ui-invalid":{boxShadow:"none"},":-moz-focusring":{outline:"auto"},"[hidden]:where(:not([hidden='until-found']))":{display:"none !important"}},s={[n||"html"]:{lineHeight:1.5,"--font-fallback":"ui-sans-serif, system-ui, -apple-system, BlinkMacSystemFont, 'Segoe UI', Roboto, 'Helvetica Neue', Arial, 'Noto Sans', sans-serif, 'Apple Color Emoji', 'Segoe UI Emoji', 'Segoe UI Symbol', 'Noto Color Emoji'",WebkitTextSizeAdjust:"100%",WebkitFontSmoothing:"antialiased",MozOsxFontSmoothing:"grayscale",textRendering:"optimizeLegibility",touchAction:"manipulation",MozTabSize:"4",tabSize:"4",fontFamily:"var(--global-font-body, var(--font-fallback))"}};if(r==="element"){const a=Object.entries(i).reduce((l,[c,u])=>(l[c]={[o]:u},l),{});Object.assign(s,a)}else o?s[o]=i:Object.assign(s,i);return s}function DC(e){const{conditions:t,isValidProperty:n}=e;return function(o){return gt(o,i=>i,{getKey:(i,s)=>_e(s)&&!t.has(i)&&!n(i)?zC(i).map(a=>"&"+a).join(", "):i})}}function zC(e){const t=[];let n=0,r="",o=!1;for(let i=0;i{const t=o=>({base:e.base?.[o]??{},variants:{},defaultVariants:e.defaultVariants??{},compoundVariants:e.compoundVariants?jC(e.compoundVariants,o):[]}),r=(e.slots??[]).map(o=>[o,t(o)]);for(const[o,i]of Object.entries(e.variants??{}))for(const[s,a]of Object.entries(i))r.forEach(([l,c])=>{var u;(u=c.variants)[o]??(u[o]={}),c.variants[o][s]=a[l]??{}});return Object.fromEntries(r)},jC=(e,t)=>e.filter(n=>n.css[t]).map(n=>({...n,css:n.css[t]}));function $C(e){const{cva:t}=e;return function(r={}){const o=Object.entries(MC(r)).map(([h,d])=>[h,t(d)]);function i(h){const d=o.map(([m,f])=>[m,f(h)]);return Object.fromEntries(d)}const s=r.variants??{},a=Object.keys(s);function l(h){const d=Qd(h,["recipe"]),[m,f]=Ln(d,a);return a.includes("colorPalette")||(m.colorPalette=h.colorPalette||r.defaultVariants?.colorPalette),a.includes("orientation")&&(f.orientation=h.orientation),[m,f]}const c=Object.fromEntries(Object.entries(s).map(([h,d])=>[h,Object.keys(d)]));let u={};return r.className&&(u=Object.fromEntries(r.slots.map(h=>[h,`${r.className}__${h}`]))),Object.assign(i,{variantMap:c,variantKeys:a,splitVariantProps:l,classNameMap:u})}}const BC=()=>e=>Array.from(new Set(e)),WC=/([\0-\x1f\x7f]|^-?\d)|^-$|^-|[^\x80-\uFFFF\w-]/g,HC=function(e,t){return t?e==="\0"?"�":e==="-"&&e.length===1?"\\-":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16):"\\"+e},Sh=e=>(e+"").replace(WC,HC),Eh=(e,t)=>{let n="",r=0,o="char",i="",s="";const a=[];for(;r{let t=0;const n=["("];for(;t{n instanceof Map?t[r]=Object.fromEntries(n):t[r]=n}),t}const Ph=/({([^}]*)})/g,GC=/[{}]/g,qC=/\w+\.\w+/,Rh=e=>{if(!lt(e))return[];const t=e.match(Ph);return t?t.map(n=>n.replace(GC,"")).map(n=>n.trim()):[]},KC=e=>Ph.test(e);function Ih(e){if(!e.extensions?.references)return e.extensions?.cssVar?.ref??e.value;const t=e.extensions.references??{};return e.value=Object.keys(t).reduce((n,r)=>{const o=t[r];if(o.extensions.conditions)return n;const i=Ih(o);return n.replace(`{${r}}`,i)},e.value),delete e.extensions.references,e.value}function Th(e){return _e(e)&&e.reference?e.reference:String(e)}const ui=(e,...t)=>t.map(Th).join(` ${e} `).replace(/calc/g,""),Nh=(...e)=>`calc(${ui("+",...e)})`,Ah=(...e)=>`calc(${ui("-",...e)})`,Oa=(...e)=>`calc(${ui("*",...e)})`,_h=(...e)=>`calc(${ui("/",...e)})`,Vh=e=>{const t=Th(e);return t!=null&&!Number.isNaN(parseFloat(t))?String(t).startsWith("-")?String(t).slice(1):`-${t}`:Oa(t,-1)},or=Object.assign(e=>({add:(...t)=>or(Nh(e,...t)),subtract:(...t)=>or(Ah(e,...t)),multiply:(...t)=>or(Oa(e,...t)),divide:(...t)=>or(_h(e,...t)),negate:()=>or(Vh(e)),toString:()=>e.toString()}),{add:Nh,subtract:Ah,multiply:Oa,divide:_h,negate:Vh}),YC={enforce:"pre",transform(e){const{prefix:t,allTokens:n,formatCssVar:r,formatTokenName:o,registerToken:i}=e;n.filter(({extensions:a})=>a.category==="spacing").forEach(a=>{const l=a.path.slice(),c=r(l,t);if(lt(a.value)&&a.value==="0rem")return;const u=structuredClone(a);Object.assign(u.extensions,{negative:!0,prop:`-${a.extensions.prop}`,originalPath:l}),u.value=or.negate(c.ref);const h=u.path[u.path.length-1];h!=null&&(u.path[u.path.length-1]=`-${h}`),u.path&&(u.name=o(u.path)),i(u)})}},XC=new Set(["spacing","sizes","borderWidths","fontSizes","radii"]),QC=[YC,{enforce:"post",transform(e){const{allTokens:t,registerToken:n,formatTokenName:r}=e,o=t.filter(({extensions:a})=>a.category==="colors"),i=new Map,s=new Map;o.forEach(a=>{const{colorPalette:l}=a.extensions;l&&(l.keys.forEach(c=>{i.set(r(c),c)}),l.roots.forEach(c=>{const u=r(c),h=s.get(u)||[];if(h.push(a),s.set(u,h),a.extensions.default&&c.length===1){const d=l.keys[0]?.filter(Boolean);if(!d.length)return;const m=c.concat(d);i.set(r(m),[])}}))}),i.forEach(a=>{const l=["colors","colorPalette",...a].filter(Boolean),c=r(l),u=r(l.slice(1));n({name:c,value:c,originalValue:c,path:l,extensions:{condition:"base",originalPath:l,category:"colors",prop:u,virtual:!0}},"pre")})}},{enforce:"post",transform(e){e.allTokens.filter(n=>XC.has(n.extensions.category)&&!n.extensions.negative).forEach(n=>{Object.assign(n.extensions,{pixelValue:lh(n.value)})})}},{enforce:"post",transform(e){e.allTokens=e.allTokens.filter(t=>t.value!=="")}}],JC=[{type:"extensions",enforce:"pre",name:"tokens/css-var",transform(e,t){const{prefix:n,formatCssVar:r}=t,{negative:o,originalPath:i}=e.extensions,s=o?i:e.path;return{cssVar:r(s.filter(Boolean),n)}}},{enforce:"post",type:"value",name:"tokens/conditionals",transform(e,t){const{prefix:n,formatCssVar:r}=t,o=Rh(e.value);return o.length&&o.forEach(i=>{const s=r(i.split("."),n);e.value=e.value.replace(`{${s.ref}}`,s)}),e.value}},{type:"extensions",enforce:"pre",name:"tokens/colors/colorPalette",match(e){return e.extensions.category==="colors"&&!e.extensions.virtual},transform(e,t){let n=e.path.slice();if(n.pop(),n.shift(),n.length===0){const a=[...e.path];a.shift(),n=a}if(n.length===0)return{};const r=n.reduce((a,l,c,u)=>{const h=u.slice(0,c+1);return a.push(h),a},[]),o=n[0],i=t.formatTokenName(n),s=e.path.slice(e.path.indexOf(o)+1).reduce((a,l,c,u)=>(a.push(u.slice(c)),a),[]);return s.length===0&&s.push([""]),{colorPalette:{value:i,roots:r,keys:s}}}}],Fh=e=>_e(e)&&Object.prototype.hasOwnProperty.call(e,"value");function ZC(e){return e?{breakpoints:Jd(e,t=>({value:t})),sizes:Object.fromEntries(Object.entries(e).map(([t,n])=>[`breakpoint-${t}`,{value:n}]))}:{breakpoints:{},sizes:{}}}function ew(e){const{prefix:t="",tokens:n={},semanticTokens:r={},breakpoints:o={}}=e,i=L=>L.join("."),s=(L,F)=>th(L.join("-"),{prefix:F}),a=[],l=new Map,c=new Map,u=new Map,h=new Map,d=new Map,m=new Map,f=new Map,p=new Map,v=[];function b(L,F){a.push(L),l.set(L.name,L),F&&p.forEach(K=>{K.enforce===F&&q(K,L)})}const y=ZC(o),x=yr({...n,breakpoints:y.breakpoints,sizes:{...n.sizes,...y.sizes}});function w(){gt(x,(L,F)=>{const K=F.includes("DEFAULT");F=Lh(F);const ie=F[0],me=i(F),Oe=lt(L)?{value:L}:L,Vt={value:Oe.value,originalValue:Oe.value,name:me,path:F,extensions:{condition:"base",originalPath:F,category:ie,prop:i(F.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh}),gt(r,(L,F)=>{const K=F.includes("DEFAULT");F=Dh(Lh(F));const ie=F[0],me=i(F),Oe=lt(L.value)?{value:{base:L.value}}:L,Vt={value:Oe.value.base||"",originalValue:Oe.value.base||"",name:me,path:F,extensions:{originalPath:F,category:ie,conditions:Oe.value,condition:"base",prop:i(F.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh})}function k(L){return l.get(L)}function O(L){const{condition:F}=L.extensions;F&&(c.has(F)||c.set(F,new Set),c.get(F).add(L))}function R(L){const{category:F,prop:K}=L.extensions;F&&(f.has(F)||f.set(F,new Map),f.get(F).set(K,L))}function I(L){const{condition:F,negative:K,virtual:ie,cssVar:me}=L.extensions;K||ie||!F||!me||(u.has(F)||u.set(F,new Map),u.get(F).set(me.var,L.value))}function N(L){const{category:F,prop:K,cssVar:ie,negative:me}=L.extensions;if(!F)return;m.has(F)||m.set(F,new Map);const Oe=me?L.extensions.conditions?L.originalValue:L.value:ie.ref;m.get(F).set(K,Oe),d.set([F,K].join("."),Oe)}function A(L){const{colorPalette:F,virtual:K,default:ie}=L.extensions;!F||K||F.roots.forEach(me=>{const Oe=i(me);h.has(Oe)||h.set(Oe,new Map);const Vt=nw([...L.path],[...me]),Wi=i(Vt),gl=k(Wi);if(!gl||!gl.extensions.cssVar)return;const{var:mT}=gl.extensions.cssVar;if(h.get(Oe).set(mT,L.extensions.cssVar.ref),ie&&me.length===1){const vT=i(["colors","colorPalette"]),bp=k(vT);if(!bp)return;const bT=i(L.path),yp=k(bT);if(!yp)return;const xp=F.keys[0]?.filter(Boolean);if(!xp.length)return;const pl=i(me.concat(xp));h.has(pl)||h.set(pl,new Map),h.get(pl).set(bp.extensions.cssVar.var,yp.extensions.cssVar.ref)}})}let T={};function S(){a.forEach(L=>{O(L),R(L),I(L),N(L),A(L)}),T=Oh(m)}const _=(L,F)=>{if(!L||typeof L!="string")return{invalid:!0,value:L};const[K,ie]=L.split("/");if(!K||!ie)return{invalid:!0,value:K};const me=F(K),Oe=k(`opacity.${ie}`)?.value;if(!Oe&&isNaN(Number(ie)))return{invalid:!0,value:K};const Vt=Oe?Number(Oe)*100+"%":`${ie}%`,Wi=me??K;return{invalid:!1,color:Wi,value:`color-mix(in srgb, ${Wi} ${Vt}, transparent)`}},D=rt((L,F)=>d.get(L)??F),z=rt(L=>T[L]||null),W=rt(L=>Eh(L,F=>{if(!F)return;if(F.includes("/")){const ie=_(F,me=>D(me));if(ie.invalid)throw new Error("Invalid color mix at "+F+": "+ie.value);return ie.value}const K=D(F);return K||(qC.test(F)?Sh(F):F)})),J={prefix:t,allTokens:a,tokenMap:l,registerToken:b,getByName:k,formatTokenName:i,formatCssVar:s,flatMap:d,cssVarMap:u,categoryMap:f,colorPaletteMap:h,getVar:D,getCategoryValues:z,expandReferenceInValue:W};function ee(...L){L.forEach(F=>{p.set(F.name,F)})}function $(...L){v.push(...L)}function q(L,F){if(F.extensions.references||cs(L.match)&&!L.match(F))return;const ie=(me=>L.transform(me,J))(F);switch(!0){case L.type==="extensions":Object.assign(F.extensions,ie);break;case L.type==="value":F.value=ie;break;default:F[L.type]=ie;break}}function Z(L){v.forEach(F=>{F.enforce===L&&F.transform(J)})}function pe(L){p.forEach(F=>{F.enforce===L&&a.forEach(K=>{q(F,K)})})}function Je(){a.forEach(L=>{const F=tw(L);!F||F.length===0||F.forEach(K=>{b(K)})})}function yt(L){return Rh(L).map(K=>k(K)).filter(Boolean)}function _t(){a.forEach(L=>{if(!KC(L.value))return;const F=yt(L.value);L.extensions.references=F.reduce((K,ie)=>(K[ie.name]=ie,K),{})})}function fl(){a.forEach(L=>{Ih(L)})}function pT(){Z("pre"),pe("pre"),Je(),_t(),fl(),Z("post"),pe("post"),S()}return w(),ee(...JC),$(...QC),pT(),J}function Lh(e){return e[0]==="DEFAULT"?e:e.filter(t=>t!=="DEFAULT")}function Dh(e){return e.filter(t=>t!=="base")}function tw(e){if(!e.extensions.conditions)return;const{conditions:t}=e.extensions,n=[];return gt(t,(r,o)=>{const i=Dh(o);if(!i.length)return;const s=structuredClone(e);s.value=r,s.extensions.condition=i.join(":"),n.push(s)}),n}function nw(e,t){const n=e.findIndex((r,o)=>t.every((i,s)=>e[o+s]===i));return n===-1||(e.splice(n,t.length),e.splice(n,0,"colorPalette")),e}BC()(["aspectRatios","zIndex","opacity","colors","fonts","fontSizes","fontWeights","lineHeights","letterSpacings","sizes","shadows","spacing","radii","cursor","borders","borderWidths","borderStyles","durations","easings","animations","blurs","gradients","breakpoints","assets"]);function _T(e){return e}function rw(e){return Object.fromEntries(Object.entries(e).map(([t,n])=>[t,n]))}function ow(e){const t=rw(e.config),n=e.tokens,r=new Map,o=new Map;function i(I,N){t[I]=N,s(I,N)}const s=(I,N)=>{const A=p(N);A&&(o.set(I,A),h(I,N))},a=()=>{for(const[I,N]of Object.entries(t))N&&s(I,N)},l=()=>{for(const[I,N]of Object.entries(t)){const{shorthand:A}=N??{};if(!A)continue;(Array.isArray(A)?A:[A]).forEach(S=>r.set(S,I))}},c=()=>{const I=Oh(n.colorPaletteMap);i("colorPalette",{values:Object.keys(I),transform:rt(N=>I[N])})},u=new Map,h=(I,N)=>{if(!N)return;const A=p(N,S=>`type:Tokens["${S}"]`);if(typeof A=="object"&&A.type){u.set(I,new Set([`type:${A.type}`]));return}if(A){const S=new Set(Object.keys(A));u.set(I,S)}const T=u.get(I)??new Set;N.property&&u.set(I,T.add(`CssProperties["${N.property}"]`))},d=()=>{for(const[I,N]of Object.entries(t))N&&h(I,N)},m=(I,N)=>{const A=u.get(I)??new Set;u.set(I,new Set([...A,...N]))},f=()=>{const I=new Map;for(const[N,A]of u.entries()){if(A.size===0){I.set(N,["string"]);continue}const T=Array.from(A).map(S=>S.startsWith("CssProperties")?S:S.startsWith("type:")?S.replace("type:",""):JSON.stringify(S));I.set(N,T)}return I},p=(I,N)=>{const{values:A}=I,T=S=>{const _=N?.(S);return _?{[_]:_}:void 0};return lt(A)?T?.(A)??n.getCategoryValues(A)??{}:Array.isArray(A)?A.reduce((S,_)=>(S[_]=_,S),{}):cs(A)?A(N?T:n.getCategoryValues):A},v=rt((I,N)=>({[I]:I.startsWith("--")?n.getVar(N,N):N})),b=Object.assign(n.getVar,{raw:I=>n.getByName(I)}),y=rt((I,N)=>{const A=k(I);lt(N)&&!N.includes("_EMO_")&&(N=n.expandReferenceInValue(N));const T=t[A];if(!T)return v(A,N);const S=o.get(A)?.[N];if(!T.transform)return v(I,S??N);const _=D=>Mk(D,b);return T.transform(S??N,{raw:N,token:b,utils:{colorMix:_}})});function x(){l(),c(),a(),d()}x();const w=r.size>0,k=rt(I=>r.get(I)??I);return{keys:()=>[...Array.from(r.keys()),...Object.keys(t)],hasShorthand:w,transform:y,shorthands:r,resolveShorthand:k,register:i,getTypes:f,addPropertyType:m}}const De={};function zh(...e){const t=Zd(...e),{theme:n={},utilities:r={},globalCss:o={},cssVarsRoot:i=":where(:root, :host)",cssVarsPrefix:s="chakra",preflight:a}=t,l=VC(t),c=ew({breakpoints:n.breakpoints,tokens:n.tokens,semanticTokens:n.semanticTokens,prefix:s}),u=gC(n.breakpoints??De),h=bC({conditions:t.conditions??De,breakpoints:u}),d=ow({config:r,tokens:c});function m(){const{textStyles:$,layerStyles:q,animationStyles:Z}=n,pe=yr({textStyle:$,layerStyle:q,animationStyle:Z});for(const[Je,yt]of Object.entries(pe)){const _t=oh(yt??De,Mh);d.register(Je,{values:Object.keys(_t),transform(fl){return x(_t[fl])}})}}m(),d.addPropertyType("animationName",Object.keys(n.keyframes??De));const f=new Set(["css",...d.keys(),...h.keys()]),p=rt($=>f.has($)||cC($)),v=$=>Array.isArray($)?$.reduce((q,Z,pe)=>{const Je=h.breakpoints[pe];return Z!=null&&(q[Je]=Z),q},{}):$,b=FC({utility:d,normalize:v}),y=DC({conditions:h,isValidProperty:p}),x=kh({transform:d.transform,conditions:h,normalize:b}),w=NC({css:x,conditions:h,normalize:b,layers:l}),k=$C({cva:w});function O(){const $={};for(const[q,Z]of c.cssVarMap.entries()){const pe=Object.fromEntries(Z);if(Object.keys(pe).length===0)continue;const Je=q==="base"?i:h.resolve(q),yt=Je.startsWith("@"),_t=x(y({[Je]:yt?{[i]:pe}:pe}));er($,_t)}return l.wrap("tokens",$)}function R(){const $=Object.fromEntries(Object.entries(n.keyframes??De).map(([Z,pe])=>[`@keyframes ${Z}`,pe])),q=Object.assign({},$,x(y(o)));return l.wrap("base",q)}function I($){return Ln($,p)}function N(){const $=LC({preflight:a});return l.wrap("reset",$)}const A=iw(c),T=($,q)=>A.get($)?.value||q;T.var=($,q)=>A.get($)?.variable||q;function S($,q){return n.recipes?.[$]??q}function _($,q){return n.slotRecipes?.[$]??q}function D($){return Object.hasOwnProperty.call(n.recipes??De,$)}function z($){return Object.hasOwnProperty.call(n.slotRecipes??De,$)}function W($){return D($)||z($)}const J=[N(),R(),O()],ee={layerStyles:Pa(n.layerStyles??De),textStyles:Pa(n.textStyles??De),animationStyles:Pa(n.animationStyles??De),tokens:jh(c,Object.keys(n.tokens??De),($,q)=>!$.extensions.conditions&&!q.includes("colorPalette")),semanticTokens:jh(c,Object.keys(n.semanticTokens??De),$=>!!$.extensions.conditions),keyframes:$h(n.keyframes??De),breakpoints:$h(n.breakpoints??De)};return{$$chakra:!0,_config:t,_global:J,breakpoints:u,tokens:c,conditions:h,utility:d,token:T,properties:f,layers:l,isValidProperty:p,splitCssProps:I,normalizeValue:v,getTokenCss:O,getGlobalCss:R,getPreflightCss:N,css:x,cva:w,sva:k,getRecipe:S,getSlotRecipe:_,hasRecipe:W,isRecipe:D,isSlotRecipe:z,query:ee}}function iw(e){const t=new Map;return e.allTokens.forEach(n=>{const{cssVar:r,virtual:o,conditions:i}=n.extensions,s=i||o?r.ref:n.value;t.set(n.name,{value:s,variable:r.ref})}),t}const Mh=e=>_e(e)&&"value"in e,Pa=e=>({list(){return Object.keys(oh(e,Mh))},search(t){return this.list().filter(n=>n.includes(t))}}),jh=(e,t,n)=>({categoryKeys:t,list(r){return Array.from(e.categoryMap.get(r)?.entries()??[]).reduce((o,[i,s])=>(n(s,i)&&o.push(i),o),[])},search(r,o){return this.list(r).filter(i=>i.includes(o))}}),$h=e=>({list(){return Object.keys(e)},search(t){return this.list().filter(n=>n.includes(t))}}),sw={sm:"480px",md:"768px",lg:"1024px",xl:"1280px","2xl":"1536px"},Ra="var(--chakra-empty,/*!*/ /*!*/)",aw=Wk({"*":{fontFeatureSettings:'"cv11"',"--ring-inset":Ra,"--ring-offset-width":"0px","--ring-offset-color":"#fff","--ring-color":"rgba(66, 153, 225, 0.6)","--ring-offset-shadow":"0 0 #0000","--ring-shadow":"0 0 #0000",...Object.fromEntries(["brightness","contrast","grayscale","hue-rotate","invert","saturate","sepia","drop-shadow"].map(e=>[`--${e}`,Ra])),...Object.fromEntries(["blur","brightness","contrast","grayscale","hue-rotate","invert","opacity","saturate","sepia"].map(e=>[`--backdrop-${e}`,Ra])),"--global-font-mono":"fonts.mono","--global-font-body":"fonts.body","--global-color-border":"colors.border"},html:{color:"fg",bg:"bg",lineHeight:"1.5",colorPalette:"gray"},"*::placeholder, *[data-placeholder]":{color:"fg.muted/80"},"*::selection":{bg:"colorPalette.emphasized/80"}}),lw=Gk({"fill.muted":{value:{background:"colorPalette.muted",color:"colorPalette.fg"}},"fill.subtle":{value:{background:"colorPalette.subtle",color:"colorPalette.fg"}},"fill.surface":{value:{background:"colorPalette.subtle",color:"colorPalette.fg",boxShadow:"0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.muted"}},"fill.solid":{value:{background:"colorPalette.solid",color:"colorPalette.contrast"}},"outline.subtle":{value:{color:"colorPalette.fg",boxShadow:"inset 0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.subtle"}},"outline.solid":{value:{borderWidth:"1px",borderColor:"colorPalette.solid",color:"colorPalette.fg"}},"indicator.bottom":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",bottom:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.top":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",top:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.start":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineStart:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.end":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineEnd:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},disabled:{value:{opacity:"0.5",cursor:"not-allowed"}},none:{value:{}}}),cw=Uk({"slide-fade-in":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-from-bottom, fade-in"},"&[data-placement^=bottom]":{animationName:"slide-from-top, fade-in"},"&[data-placement^=left]":{animationName:"slide-from-right, fade-in"},"&[data-placement^=right]":{animationName:"slide-from-left, fade-in"}}},"slide-fade-out":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-to-bottom, fade-out"},"&[data-placement^=bottom]":{animationName:"slide-to-top, fade-out"},"&[data-placement^=left]":{animationName:"slide-to-right, fade-out"},"&[data-placement^=right]":{animationName:"slide-to-left, fade-out"}}},"scale-fade-in":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-in, fade-in"}},"scale-fade-out":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-out, fade-out"}}}),Ia=fe({className:"chakra-badge",base:{display:"inline-flex",alignItems:"center",borderRadius:"l2",gap:"1",fontWeight:"medium",fontVariantNumeric:"tabular-nums",whiteSpace:"nowrap",userSelect:"none"},variants:{variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg"},outline:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},plain:{color:"colorPalette.fg"}},size:{xs:{textStyle:"2xs",px:"1",minH:"4"},sm:{textStyle:"xs",px:"1.5",minH:"5"},md:{textStyle:"sm",px:"2",minH:"6"},lg:{textStyle:"sm",px:"2.5",minH:"7"}}},defaultVariants:{variant:"subtle",size:"sm"}}),uw=fe({className:"chakra-button",base:{display:"inline-flex",appearance:"none",alignItems:"center",justifyContent:"center",userSelect:"none",position:"relative",borderRadius:"l2",whiteSpace:"nowrap",verticalAlign:"middle",borderWidth:"1px",borderColor:"transparent",cursor:"button",flexShrink:"0",outline:"0",lineHeight:"1.2",isolation:"isolate",fontWeight:"medium",transitionProperty:"common",transitionDuration:"moderate",focusVisibleRing:"outside",_disabled:{layerStyle:"disabled"},_icon:{flexShrink:"0"}},variants:{size:{"2xs":{h:"6",minW:"6",textStyle:"xs",px:"2",gap:"1",_icon:{width:"3.5",height:"3.5"}},xs:{h:"8",minW:"8",textStyle:"xs",px:"2.5",gap:"1",_icon:{width:"4",height:"4"}},sm:{h:"9",minW:"9",px:"3.5",textStyle:"sm",gap:"2",_icon:{width:"4",height:"4"}},md:{h:"10",minW:"10",textStyle:"sm",px:"4",gap:"2",_icon:{width:"5",height:"5"}},lg:{h:"11",minW:"11",textStyle:"md",px:"5",gap:"3",_icon:{width:"5",height:"5"}},xl:{h:"12",minW:"12",textStyle:"md",px:"5",gap:"2.5",_icon:{width:"5",height:"5"}},"2xl":{h:"16",minW:"16",textStyle:"lg",px:"7",gap:"3",_icon:{width:"6",height:"6"}}},variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"transparent",_hover:{bg:"colorPalette.solid/90"},_expanded:{bg:"colorPalette.solid/90"}},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"transparent",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},outline:{borderWidth:"1px",borderColor:"colorPalette.muted",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},ghost:{bg:"transparent",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},plain:{color:"colorPalette.fg"}}},defaultVariants:{size:"md",variant:"solid"}}),Se=fe({className:"chakra-checkmark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"l1",cursor:"checkbox",focusVisibleRing:"outside",_icon:{boxSize:"full"},_invalid:{colorPalette:"red",borderColor:"border.error"},_disabled:{opacity:"0.5",cursor:"disabled"}},variants:{size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5",p:"0.5"},lg:{boxSize:"6",p:"0.5"}},variant:{solid:{borderColor:"border.emphasized","&:is([data-state=checked], [data-state=indeterminate])":{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},outline:{borderColor:"border","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg",borderColor:"colorPalette.solid"}},subtle:{bg:"colorPalette.muted",borderColor:"colorPalette.muted","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},plain:{"&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},inverted:{borderColor:"border",color:"colorPalette.fg","&:is([data-state=checked], [data-state=indeterminate])":{borderColor:"colorPalette.solid"}}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),{variants:dw,defaultVariants:hw}=Ia,fw=fe({className:"chakra-code",base:{fontFamily:"mono",alignItems:"center",display:"inline-flex",borderRadius:"l2"},variants:dw,defaultVariants:hw}),Bh=fe({className:"color-swatch",base:{boxSize:"var(--swatch-size)",shadow:"inset 0 0 0 1px rgba(0, 0, 0, 0.1)","--checker-size":"8px","--checker-bg":"colors.bg","--checker-fg":"colors.bg.emphasized",background:"linear-gradient(var(--color), var(--color)), repeating-conic-gradient(var(--checker-fg) 0%, var(--checker-fg) 25%, var(--checker-bg) 0%, var(--checker-bg) 50%) 0% 50% / var(--checker-size) var(--checker-size) !important",display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},variants:{size:{"2xs":{"--swatch-size":"sizes.3.5"},xs:{"--swatch-size":"sizes.4"},sm:{"--swatch-size":"sizes.4.5"},md:{"--swatch-size":"sizes.5"},lg:{"--swatch-size":"sizes.6"},xl:{"--swatch-size":"sizes.7"},"2xl":{"--swatch-size":"sizes.8"},inherit:{"--swatch-size":"inherit"},full:{"--swatch-size":"100%"}},shape:{square:{borderRadius:"none"},circle:{borderRadius:"full"},rounded:{borderRadius:"l1"}}},defaultVariants:{size:"md",shape:"rounded"}}),gw=fe({className:"chakra-container",base:{position:"relative",maxWidth:"8xl",w:"100%",mx:"auto",px:{base:"4",md:"6",lg:"8"}},variants:{centerContent:{true:{display:"flex",flexDirection:"column",alignItems:"center"}},fluid:{true:{maxWidth:"full"}}}}),pw=fe({className:"chakra-heading",base:{fontFamily:"heading",fontWeight:"semibold"},variants:{size:{xs:{textStyle:"xs"},sm:{textStyle:"sm"},md:{textStyle:"md"},lg:{textStyle:"lg"},xl:{textStyle:"xl"},"2xl":{textStyle:"2xl"},"3xl":{textStyle:"3xl"},"4xl":{textStyle:"4xl"},"5xl":{textStyle:"5xl"},"6xl":{textStyle:"6xl"},"7xl":{textStyle:"7xl"}}},defaultVariants:{size:"xl"}}),mw=fe({className:"chakra-icon",base:{display:"inline-block",lineHeight:"1em",flexShrink:"0",color:"currentcolor",verticalAlign:"middle"},variants:{size:{inherit:{},xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"},xl:{boxSize:"7"},"2xl":{boxSize:"8"}}},defaultVariants:{size:"inherit"}}),he=fe({className:"chakra-input",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},height:"var(--input-height)",minW:"var(--input-height)","--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{"2xs":{textStyle:"xs",px:"2","--input-height":"sizes.7"},xs:{textStyle:"xs",px:"2","--input-height":"sizes.8"},sm:{textStyle:"sm",px:"2.5","--input-height":"sizes.9"},md:{textStyle:"sm",px:"3","--input-height":"sizes.10"},lg:{textStyle:"md",px:"4","--input-height":"sizes.11"},xl:{textStyle:"md",px:"4.5","--input-height":"sizes.12"},"2xl":{textStyle:"lg",px:"5","--input-height":"sizes.16"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)",_invalid:{borderColor:"var(--error-color)",boxShadow:"0px 1px 0px 0px var(--error-color)"}}}}},defaultVariants:{size:"md",variant:"outline"}}),vw=fe({className:"chakra-input-addon",base:{flex:"0 0 auto",width:"auto",display:"flex",alignItems:"center",whiteSpace:"nowrap",alignSelf:"stretch",borderRadius:"l2"},variants:{size:he.variants.size,variant:{outline:{borderWidth:"1px",borderColor:"border",bg:"bg.muted"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.emphasized"},flushed:{borderBottom:"1px solid",borderColor:"inherit",borderRadius:"0",px:"0",bg:"transparent"}}},defaultVariants:{size:"md",variant:"outline"}}),bw=fe({className:"chakra-kbd",base:{display:"inline-flex",alignItems:"center",fontWeight:"medium",fontFamily:"mono",flexShrink:"0",whiteSpace:"nowrap",wordSpacing:"-0.5em",userSelect:"none",px:"1",borderRadius:"l2"},variants:{variant:{raised:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderWidth:"1px",borderBottomWidth:"2px",borderColor:"colorPalette.muted"},outline:{borderWidth:"1px",color:"colorPalette.fg"},subtle:{bg:"colorPalette.muted",color:"colorPalette.fg"},plain:{color:"colorPalette.fg"}},size:{sm:{textStyle:"xs",height:"4.5"},md:{textStyle:"sm",height:"5"},lg:{textStyle:"md",height:"6"}}},defaultVariants:{size:"md",variant:"raised"}}),yw=fe({className:"chakra-link",base:{display:"inline-flex",alignItems:"center",outline:"none",gap:"1.5",cursor:"pointer",borderRadius:"l1",focusRing:"outside"},variants:{variant:{underline:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"},plain:{color:"colorPalette.fg",_hover:{textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"}}}},defaultVariants:{variant:"plain"}}),xw=fe({className:"chakra-mark",base:{bg:"transparent",color:"inherit",whiteSpace:"nowrap"},variants:{variant:{subtle:{bg:"colorPalette.subtle",color:"inherit"},solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},text:{fontWeight:"medium"},plain:{}}}}),Ee=fe({className:"chakra-radiomark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:0,verticalAlign:"top",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"full",cursor:"radio",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing",outlineOffset:"2px"},_invalid:{colorPalette:"red",borderColor:"red.500"},_disabled:{opacity:"0.5",cursor:"disabled"},"& .dot":{height:"100%",width:"100%",borderRadius:"full",bg:"currentColor",scale:"0.4"}},variants:{variant:{solid:{borderWidth:"1px",borderColor:"border.emphasized",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},subtle:{borderWidth:"1px",bg:"colorPalette.muted",borderColor:"colorPalette.muted",color:"transparent",_checked:{color:"colorPalette.fg"}},outline:{borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.fg",borderColor:"colorPalette.solid"},"& .dot":{scale:"0.6"}},inverted:{bg:"bg",borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.solid",borderColor:"currentcolor"}}},size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),kw=fe({className:"chakra-separator",base:{display:"block",borderColor:"border"},variants:{variant:{solid:{borderStyle:"solid"},dashed:{borderStyle:"dashed"},dotted:{borderStyle:"dotted"}},orientation:{vertical:{borderInlineStartWidth:"var(--separator-thickness)"},horizontal:{borderTopWidth:"var(--separator-thickness)"}},size:{xs:{"--separator-thickness":"0.5px"},sm:{"--separator-thickness":"1px"},md:{"--separator-thickness":"2px"},lg:{"--separator-thickness":"3px"}}},defaultVariants:{size:"sm",variant:"solid",orientation:"horizontal"}}),Cw=fe({className:"chakra-skeleton",base:{},variants:{loading:{true:{borderRadius:"l2",boxShadow:"none",backgroundClip:"padding-box",cursor:"default",color:"transparent",pointerEvents:"none",userSelect:"none",flexShrink:"0","&::before, &::after, *":{visibility:"hidden"}},false:{background:"unset",animation:"fade-in var(--fade-duration, 0.1s) ease-out !important"}},variant:{pulse:{background:"bg.emphasized",animation:"pulse",animationDuration:"var(--duration, 1.2s)"},shine:{"--animate-from":"200%","--animate-to":"-200%","--start-color":"colors.bg.muted","--end-color":"colors.bg.emphasized",backgroundImage:"linear-gradient(270deg,var(--start-color),var(--end-color),var(--end-color),var(--start-color))",backgroundSize:"400% 100%",animation:"bg-position var(--duration, 5s) ease-in-out infinite"},none:{animation:"none"}}},defaultVariants:{variant:"pulse",loading:!0}}),ww=fe({className:"chakra-skip-nav",base:{display:"inline-flex",bg:"bg.panel",padding:"2.5",borderRadius:"l2",fontWeight:"semibold",focusVisibleRing:"outside",textStyle:"sm",userSelect:"none",border:"0",height:"1px",width:"1px",margin:"-1px",outline:"0",overflow:"hidden",position:"absolute",clip:"rect(0 0 0 0)",_focusVisible:{clip:"auto",width:"auto",height:"auto",position:"fixed",top:"6",insetStart:"6"}}}),Sw=fe({className:"chakra-spinner",base:{display:"inline-block",borderColor:"currentColor",borderStyle:"solid",borderWidth:"2px",borderRadius:"full",width:"var(--spinner-size)",height:"var(--spinner-size)",animation:"spin",animationDuration:"slowest","--spinner-track-color":"transparent",borderBottomColor:"var(--spinner-track-color)",borderInlineStartColor:"var(--spinner-track-color)"},variants:{size:{inherit:{"--spinner-size":"1em"},xs:{"--spinner-size":"sizes.3"},sm:{"--spinner-size":"sizes.4"},md:{"--spinner-size":"sizes.5"},lg:{"--spinner-size":"sizes.8"},xl:{"--spinner-size":"sizes.10"}}},defaultVariants:{size:"md"}}),Ew=fe({className:"chakra-textarea",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{xs:{textStyle:"xs",px:"2",py:"1.5",scrollPaddingBottom:"1.5"},sm:{textStyle:"sm",px:"2.5",py:"2",scrollPaddingBottom:"2"},md:{textStyle:"sm",px:"3",py:"2",scrollPaddingBottom:"2"},lg:{textStyle:"md",px:"4",py:"3",scrollPaddingBottom:"3"},xl:{textStyle:"md",px:"4.5",py:"3.5",scrollPaddingBottom:"3.5"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}}}},defaultVariants:{size:"md",variant:"outline"}}),Ow={badge:Ia,button:uw,code:fw,container:gw,heading:pw,input:he,inputAddon:vw,kbd:bw,link:yw,mark:xw,separator:kw,skeleton:Cw,skipNavLink:ww,spinner:Sw,textarea:Ew,icon:mw,checkmark:Se,radiomark:Ee,colorSwatch:Bh},Pw=xa.colors({bg:{DEFAULT:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},emphasized:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},inverted:{value:{_light:"{colors.black}",_dark:"{colors.white}"}},panel:{value:{_light:"{colors.white}",_dark:"{colors.gray.950}"}},error:{value:{_light:"{colors.red.50}",_dark:"{colors.red.950}"}},warning:{value:{_light:"{colors.orange.50}",_dark:"{colors.orange.950}"}},success:{value:{_light:"{colors.green.50}",_dark:"{colors.green.950}"}},info:{value:{_light:"{colors.blue.50}",_dark:"{colors.blue.950}"}}},fg:{DEFAULT:{value:{_light:"{colors.black}",_dark:"{colors.gray.50}"}},muted:{value:{_light:"{colors.gray.600}",_dark:"{colors.gray.400}"}},subtle:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.500}"}},inverted:{value:{_light:"{colors.gray.50}",_dark:"{colors.black}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.300}"}},success:{value:{_light:"{colors.green.600}",_dark:"{colors.green.300}"}},info:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.300}"}}},border:{DEFAULT:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},inverted:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.400}"}},success:{value:{_light:"{colors.green.500}",_dark:"{colors.green.400}"}},info:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.400}"}}},gray:{contrast:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},fg:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},subtle:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},muted:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},solid:{value:{_light:"{colors.gray.900}",_dark:"{colors.white}"}},focusRing:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.400}"}}},red:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.red.700}",_dark:"{colors.red.300}"}},subtle:{value:{_light:"{colors.red.100}",_dark:"{colors.red.900}"}},muted:{value:{_light:"{colors.red.200}",_dark:"{colors.red.800}"}},emphasized:{value:{_light:"{colors.red.300}",_dark:"{colors.red.700}"}},solid:{value:{_light:"{colors.red.600}",_dark:"{colors.red.600}"}},focusRing:{value:{_light:"{colors.red.500}",_dark:"{colors.red.500}"}}},orange:{contrast:{value:{_light:"white",_dark:"black"}},fg:{value:{_light:"{colors.orange.700}",_dark:"{colors.orange.300}"}},subtle:{value:{_light:"{colors.orange.100}",_dark:"{colors.orange.900}"}},muted:{value:{_light:"{colors.orange.200}",_dark:"{colors.orange.800}"}},emphasized:{value:{_light:"{colors.orange.300}",_dark:"{colors.orange.700}"}},solid:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.500}"}},focusRing:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.500}"}}},green:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.green.700}",_dark:"{colors.green.300}"}},subtle:{value:{_light:"{colors.green.100}",_dark:"{colors.green.900}"}},muted:{value:{_light:"{colors.green.200}",_dark:"{colors.green.800}"}},emphasized:{value:{_light:"{colors.green.300}",_dark:"{colors.green.700}"}},solid:{value:{_light:"{colors.green.600}",_dark:"{colors.green.600}"}},focusRing:{value:{_light:"{colors.green.500}",_dark:"{colors.green.500}"}}},blue:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.blue.700}",_dark:"{colors.blue.300}"}},subtle:{value:{_light:"{colors.blue.100}",_dark:"{colors.blue.900}"}},muted:{value:{_light:"{colors.blue.200}",_dark:"{colors.blue.800}"}},emphasized:{value:{_light:"{colors.blue.300}",_dark:"{colors.blue.700}"}},solid:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.600}"}},focusRing:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.500}"}}},yellow:{contrast:{value:{_light:"black",_dark:"black"}},fg:{value:{_light:"{colors.yellow.800}",_dark:"{colors.yellow.300}"}},subtle:{value:{_light:"{colors.yellow.100}",_dark:"{colors.yellow.900}"}},muted:{value:{_light:"{colors.yellow.200}",_dark:"{colors.yellow.800}"}},emphasized:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.700}"}},solid:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.300}"}},focusRing:{value:{_light:"{colors.yellow.500}",_dark:"{colors.yellow.500}"}}},teal:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.teal.700}",_dark:"{colors.teal.300}"}},subtle:{value:{_light:"{colors.teal.100}",_dark:"{colors.teal.900}"}},muted:{value:{_light:"{colors.teal.200}",_dark:"{colors.teal.800}"}},emphasized:{value:{_light:"{colors.teal.300}",_dark:"{colors.teal.700}"}},solid:{value:{_light:"{colors.teal.600}",_dark:"{colors.teal.600}"}},focusRing:{value:{_light:"{colors.teal.500}",_dark:"{colors.teal.500}"}}},purple:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.purple.700}",_dark:"{colors.purple.300}"}},subtle:{value:{_light:"{colors.purple.100}",_dark:"{colors.purple.900}"}},muted:{value:{_light:"{colors.purple.200}",_dark:"{colors.purple.800}"}},emphasized:{value:{_light:"{colors.purple.300}",_dark:"{colors.purple.700}"}},solid:{value:{_light:"{colors.purple.600}",_dark:"{colors.purple.600}"}},focusRing:{value:{_light:"{colors.purple.500}",_dark:"{colors.purple.500}"}}},pink:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.pink.700}",_dark:"{colors.pink.300}"}},subtle:{value:{_light:"{colors.pink.100}",_dark:"{colors.pink.900}"}},muted:{value:{_light:"{colors.pink.200}",_dark:"{colors.pink.800}"}},emphasized:{value:{_light:"{colors.pink.300}",_dark:"{colors.pink.700}"}},solid:{value:{_light:"{colors.pink.600}",_dark:"{colors.pink.600}"}},focusRing:{value:{_light:"{colors.pink.500}",_dark:"{colors.pink.500}"}}},cyan:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.cyan.700}",_dark:"{colors.cyan.300}"}},subtle:{value:{_light:"{colors.cyan.100}",_dark:"{colors.cyan.900}"}},muted:{value:{_light:"{colors.cyan.200}",_dark:"{colors.cyan.800}"}},emphasized:{value:{_light:"{colors.cyan.300}",_dark:"{colors.cyan.700}"}},solid:{value:{_light:"{colors.cyan.600}",_dark:"{colors.cyan.600}"}},focusRing:{value:{_light:"{colors.cyan.500}",_dark:"{colors.cyan.500}"}}}}),Rw=xa.radii({l1:{value:"{radii.xs}"},l2:{value:"{radii.sm}"},l3:{value:"{radii.md}"}}),Iw=xa.shadows({xs:{value:{_light:"0px 1px 2px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/20}",_dark:"0px 1px 1px {black/64}, 0px 0px 1px inset {colors.gray.300/20}"}},sm:{value:{_light:"0px 2px 4px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 2px 4px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},md:{value:{_light:"0px 4px 8px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 4px 8px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},lg:{value:{_light:"0px 8px 16px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 8px 16px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},xl:{value:{_light:"0px 16px 24px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 16px 24px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},"2xl":{value:{_light:"0px 24px 40px {colors.gray.900/16}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 24px 40px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},inner:{value:{_light:"inset 0 2px 4px 0 {black/5}",_dark:"inset 0 2px 4px 0 black"}},inset:{value:{_light:"inset 0 0 0 1px {black/5}",_dark:"inset 0 0 0 1px {colors.gray.300/5}"}}}),Tw=xu.extendWith("itemBody"),Nw=j("action-bar").parts("positioner","content","separator","selectionTrigger","closeTrigger"),Aw=j("alert").parts("title","description","root","indicator","content"),_w=j("breadcrumb").parts("link","currentLink","item","list","root","ellipsis","separator"),Vw=j("blockquote").parts("root","icon","content","caption"),Fw=j("card").parts("root","header","body","footer","title","description"),Lw=j("checkbox-card",["root","control","label","description","addon","indicator","content"]),Dw=j("data-list").parts("root","item","itemLabel","itemValue"),zw=ea.extendWith("header","body","footer","backdrop"),Mw=ea.extendWith("header","body","footer","backdrop"),jw=ld.extendWith("textarea"),$w=j("empty-state",["root","content","indicator","title","description"]),Bw=ud.extendWith("requiredIndicator"),Ww=hd.extendWith("content"),Hw=fd.extendWith("itemContent","dropzoneContent","fileText"),Uw=j("list").parts("root","item","indicator"),Gw=xd.extendWith("itemCommand"),qw=j("select").parts("root","field","indicator"),Kw=Dd.extendWith("header","body","footer"),Wh=fa.extendWith("itemAddon","itemIndicator"),Yw=Wh.extendWith("itemContent","itemDescription"),Xw=Md.extendWith("itemIndicator"),Qw=Bd.extendWith("indicatorGroup"),Jw=s1.extendWith("indicatorGroup","empty"),Zw=Gd.extendWith("markerIndicator"),eS=j("stat").parts("root","label","helpText","valueText","valueUnit","indicator"),tS=j("status").parts("root","indicator"),nS=j("steps",["root","list","item","trigger","indicator","separator","content","title","description","nextTrigger","prevTrigger","progress"]),rS=qd.extendWith("indicator"),oS=j("table").parts("root","header","body","row","columnHeader","cell","footer","caption"),iS=j("toast").parts("root","title","description","indicator","closeTrigger","actionTrigger"),sS=j("tabs").parts("root","trigger","list","content","contentGroup","indicator"),aS=j("tag").parts("root","label","closeTrigger","startElement","endElement"),lS=j("timeline").parts("root","item","content","separator","indicator","connector","title","description"),cS=wx.extendWith("channelText"),uS=j("code-block",["root","content","title","header","footer","control","overlay","code","codeText","copyTrigger","copyIndicator","collapseTrigger","collapseIndicator","collapseText"]);Tu.extendWith("valueText");const dS=V1,hS=B({className:"chakra-accordion",slots:Tw.keys(),base:{root:{width:"full","--accordion-radius":"radii.l2"},item:{overflowAnchor:"none"},itemTrigger:{display:"flex",alignItems:"center",textAlign:"start",width:"full",outline:"0",gap:"3",fontWeight:"medium",borderRadius:"var(--accordion-radius)",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{layerStyle:"disabled"}},itemBody:{pt:"var(--accordion-padding-y)",pb:"calc(var(--accordion-padding-y) * 2)"},itemContent:{overflow:"hidden",borderRadius:"var(--accordion-radius)",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}},itemIndicator:{transition:"rotate 0.2s",transformOrigin:"center",color:"fg.subtle",_open:{rotate:"180deg"},_icon:{width:"1.2em",height:"1.2em"}}},variants:{variant:{outline:{item:{borderBottomWidth:"1px"}},subtle:{itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{borderRadius:"var(--accordion-radius)",_open:{bg:"colorPalette.subtle"}}},enclosed:{root:{borderWidth:"1px",borderRadius:"var(--accordion-radius)",divideY:"1px",overflow:"hidden"},itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{_open:{bg:"bg.subtle"}}},plain:{}},size:{sm:{root:{"--accordion-padding-x":"spacing.3","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"sm",py:"var(--accordion-padding-y)"}},md:{root:{"--accordion-padding-x":"spacing.4","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"md",py:"var(--accordion-padding-y)"}},lg:{root:{"--accordion-padding-x":"spacing.4.5","--accordion-padding-y":"spacing.2.5"},itemTrigger:{textStyle:"lg",py:"var(--accordion-padding-y)"}}}},defaultVariants:{size:"md",variant:"outline"}}),fS=B({className:"chakra-action-bar",slots:Nw.keys(),base:{positioner:{position:"fixed",display:"flex",justifyContent:"center",pointerEvents:"none",insetInline:"0",top:"unset",bottom:"calc(env(safe-area-inset-bottom) + 20px)"},content:{bg:"bg.panel",shadow:"md",display:"flex",alignItems:"center",gap:"3",borderRadius:"l3",py:"2.5",px:"3",pointerEvents:"auto",translate:"calc(-1 * var(--scrollbar-width) / 2) 0px",_open:{animationName:"slide-from-bottom, fade-in",animationDuration:"moderate"},_closed:{animationName:"slide-to-bottom, fade-out",animationDuration:"faster"}},separator:{width:"1px",height:"5",bg:"border"},selectionTrigger:{display:"inline-flex",alignItems:"center",gap:"2",alignSelf:"stretch",textStyle:"sm",px:"4",py:"1",borderRadius:"l2",borderWidth:"1px",borderStyle:"dashed"}}}),gS=B({slots:Aw.keys(),className:"chakra-alert",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",borderRadius:"l3"},title:{fontWeight:"medium"},description:{display:"inline"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",width:"1em",height:"1em",_icon:{boxSize:"full"}},content:{display:"flex",flex:"1",gap:"1"}},variants:{status:{info:{root:{colorPalette:"blue"}},warning:{root:{colorPalette:"orange"}},success:{root:{colorPalette:"green"}},error:{root:{colorPalette:"red"}},neutral:{root:{colorPalette:"gray"}}},inline:{true:{content:{display:"inline-flex",flexDirection:"row",alignItems:"center"}},false:{content:{display:"flex",flexDirection:"column"}}},variant:{subtle:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg"}},surface:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"},indicator:{color:"colorPalette.contrast"}}},size:{sm:{root:{gap:"2",px:"3",py:"3",textStyle:"xs"},indicator:{textStyle:"lg"}},md:{root:{gap:"3",px:"4",py:"4",textStyle:"sm"},indicator:{textStyle:"xl"}},lg:{root:{gap:"3",px:"4",py:"4",textStyle:"md"},indicator:{textStyle:"2xl"}}}},defaultVariants:{status:"info",variant:"subtle",size:"md",inline:!1}}),pS=B({slots:wu.keys(),className:"chakra-avatar",base:{root:{display:"inline-flex",alignItems:"center",justifyContent:"center",fontWeight:"medium",position:"relative",verticalAlign:"top",flexShrink:"0",userSelect:"none",width:"var(--avatar-size)",height:"var(--avatar-size)",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)","&[data-group-item]":{borderWidth:"2px",borderColor:"bg"}},image:{width:"100%",height:"100%",objectFit:"cover",borderRadius:"var(--avatar-radius)"},fallback:{lineHeight:"1",textTransform:"uppercase",fontWeight:"medium",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)"}},variants:{size:{full:{root:{"--avatar-size":"100%","--avatar-font-size":"100%"}},"2xs":{root:{"--avatar-font-size":"fontSizes.2xs","--avatar-size":"sizes.6"}},xs:{root:{"--avatar-font-size":"fontSizes.xs","--avatar-size":"sizes.8"}},sm:{root:{"--avatar-font-size":"fontSizes.sm","--avatar-size":"sizes.9"}},md:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.10"}},lg:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.11"}},xl:{root:{"--avatar-font-size":"fontSizes.lg","--avatar-size":"sizes.12"}},"2xl":{root:{"--avatar-font-size":"fontSizes.xl","--avatar-size":"sizes.16"}}},variant:{solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},subtle:{root:{bg:"colorPalette.muted",color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",borderWidth:"1px",borderColor:"colorPalette.muted"}}},shape:{square:{},rounded:{root:{"--avatar-radius":"radii.l3"}},full:{root:{"--avatar-radius":"radii.full"}}},borderless:{true:{root:{"&[data-group-item]":{borderWidth:"0px"}}}}},defaultVariants:{size:"md",shape:"full",variant:"subtle"}}),mS=B({className:"chakra-blockquote",slots:Vw.keys(),base:{root:{position:"relative",display:"flex",flexDirection:"column",gap:"2"},caption:{textStyle:"sm",color:"fg.muted"},icon:{boxSize:"5"}},variants:{justify:{start:{root:{alignItems:"flex-start",textAlign:"start"}},center:{root:{alignItems:"center",textAlign:"center"}},end:{root:{alignItems:"flex-end",textAlign:"end"}}},variant:{subtle:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.muted"},icon:{color:"colorPalette.fg"}},solid:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.solid"},icon:{color:"colorPalette.solid"}},plain:{root:{paddingX:"5"},icon:{color:"colorPalette.solid"}}}},defaultVariants:{variant:"subtle",justify:"start"}}),vS=B({className:"chakra-breadcrumb",slots:_w.keys(),base:{list:{display:"flex",alignItems:"center",wordBreak:"break-word",color:"fg.muted",listStyle:"none"},link:{outline:"0",textDecoration:"none",borderRadius:"l1",focusRing:"outside",display:"inline-flex",alignItems:"center",gap:"2"},item:{display:"inline-flex",alignItems:"center"},separator:{color:"fg.muted",opacity:"0.8",_icon:{boxSize:"1em"},_rtl:{rotate:"180deg"}},ellipsis:{display:"inline-flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"1em"}}},variants:{variant:{underline:{link:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"0.2em",textDecorationColor:"colorPalette.muted"},currentLink:{color:"colorPalette.fg"}},plain:{link:{color:"fg.muted",_hover:{color:"fg"}},currentLink:{color:"fg"}}},size:{sm:{list:{gap:"1",textStyle:"xs"}},md:{list:{gap:"1.5",textStyle:"sm"}},lg:{list:{gap:"2",textStyle:"md"}}}},defaultVariants:{variant:"plain",size:"md"}}),bS=B({className:"chakra-card",slots:Fw.keys(),base:{root:{display:"flex",flexDirection:"column",position:"relative",minWidth:"0",wordWrap:"break-word",borderRadius:"l3",color:"fg",textAlign:"start"},title:{fontWeight:"semibold"},description:{color:"fg.muted",fontSize:"sm"},header:{paddingInline:"var(--card-padding)",paddingTop:"var(--card-padding)",display:"flex",flexDirection:"column",gap:"1.5"},body:{padding:"var(--card-padding)",flex:"1",display:"flex",flexDirection:"column"},footer:{display:"flex",alignItems:"center",gap:"2",paddingInline:"var(--card-padding)",paddingBottom:"var(--card-padding)"}},variants:{size:{sm:{root:{"--card-padding":"spacing.4"},title:{textStyle:"md"}},md:{root:{"--card-padding":"spacing.6"},title:{textStyle:"lg"}},lg:{root:{"--card-padding":"spacing.7"},title:{textStyle:"xl"}}},variant:{elevated:{root:{bg:"bg.panel",boxShadow:"md"}},outline:{root:{bg:"bg.panel",borderWidth:"1px",borderColor:"border"}},subtle:{root:{bg:"bg.muted"}}}},defaultVariants:{variant:"outline",size:"md"}}),yS=B({slots:Cx.keys(),className:"chakra-checkbox",base:{root:{display:"inline-flex",gap:"2",alignItems:"center",verticalAlign:"top",position:"relative"},control:Se.base,label:{fontWeight:"medium",userSelect:"none",_disabled:{opacity:"0.5"}}},variants:{size:{xs:{root:{gap:"1.5"},label:{textStyle:"xs"},control:Se.variants?.size?.xs},sm:{root:{gap:"2"},label:{textStyle:"sm"},control:Se.variants?.size?.sm},md:{root:{gap:"2.5"},label:{textStyle:"sm"},control:Se.variants?.size?.md},lg:{root:{gap:"3"},label:{textStyle:"md"},control:Se.variants?.size?.lg}},variant:{outline:{control:Se.variants?.variant?.outline},solid:{control:Se.variants?.variant?.solid},subtle:{control:Se.variants?.variant?.subtle}}},defaultVariants:{variant:"solid",size:"md"}}),xS=B({slots:Lw.keys(),className:"chakra-checkbox-card",base:{root:{display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",flex:"1",focusVisibleRing:"outside",_disabled:{opacity:"0.8"},_invalid:{outline:"2px solid",outlineColor:"border.error"}},control:{display:"inline-flex",flex:"1",position:"relative",borderRadius:"inherit",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"},label:{fontWeight:"medium",display:"flex",alignItems:"center",gap:"2",flex:"1",_disabled:{opacity:"0.5"}},description:{opacity:"0.64",textStyle:"sm",_disabled:{opacity:"0.5"}},addon:{_disabled:{opacity:"0.5"}},indicator:Se.base,content:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"}},variants:{size:{sm:{root:{textStyle:"sm"},control:{padding:"3",gap:"1.5"},addon:{px:"3",py:"1.5",borderTopWidth:"1px"},indicator:Se.variants?.size.sm},md:{root:{textStyle:"sm"},control:{padding:"4",gap:"2.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.md},lg:{root:{textStyle:"md"},control:{padding:"4",gap:"3.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.lg}},variant:{surface:{root:{borderWidth:"1px",borderColor:"border",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"},_disabled:{bg:"bg.muted"}},indicator:Se.variants?.variant.solid},subtle:{root:{bg:"bg.muted"},control:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},indicator:Se.variants?.variant.plain},outline:{root:{borderWidth:"1px",borderColor:"border",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.solid},solid:{root:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.inverted}},justify:{start:{root:{"--checkbox-card-justify":"flex-start"}},end:{root:{"--checkbox-card-justify":"flex-end"}},center:{root:{"--checkbox-card-justify":"center"}}},align:{start:{root:{"--checkbox-card-align":"flex-start"},content:{textAlign:"start"}},end:{root:{"--checkbox-card-align":"flex-end"},content:{textAlign:"end"}},center:{root:{"--checkbox-card-align":"center"},content:{textAlign:"center"}}},orientation:{vertical:{control:{flexDirection:"column"}},horizontal:{control:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),kS=B({slots:uS.keys(),className:"code-block",base:{root:{colorPalette:"gray",rounded:"var(--code-block-radius)",overflow:"hidden",bg:"bg",color:"fg",borderWidth:"1px","--code-block-max-height":"320px","--code-block-bg":"colors.bg","--code-block-fg":"colors.fg","--code-block-obscured-opacity":"0.5","--code-block-obscured-blur":"1px","--code-block-line-number-width":"sizes.3","--code-block-line-number-margin":"spacing.4","--code-block-highlight-bg":"{colors.teal.focusRing/20}","--code-block-highlight-border":"colors.teal.focusRing","--code-block-highlight-added-bg":"{colors.green.focusRing/20}","--code-block-highlight-added-border":"colors.green.focusRing","--code-block-highlight-removed-bg":"{colors.red.focusRing/20}","--code-block-highlight-removed-border":"colors.red.focusRing"},header:{display:"flex",alignItems:"center",gap:"2",position:"relative",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)",mb:"calc(var(--code-block-padding) / 2 * -1)"},title:{display:"inline-flex",alignItems:"center",gap:"1.5",flex:"1",color:"fg.muted"},control:{gap:"1.5",display:"inline-flex",alignItems:"center"},footer:{display:"flex",alignItems:"center",justifyContent:"center",gap:"2",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)"},content:{position:"relative",colorScheme:"dark",overflowX:"auto",overflowY:"hidden",borderBottomRadius:"var(--code-block-radius)",maxHeight:"var(--code-block-max-height)","& ::selection":{bg:"blue.500/40"},_expanded:{maxHeight:"unset"}},overlay:{"--bg":"{colors.black/50}",display:"flex",alignItems:"flex-end",justifyContent:"center",padding:"4",bgImage:"linear-gradient(0deg,var(--bg) 25%,transparent 100%)",color:"white",minH:"5rem",pos:"absolute",bottom:"0",insetInline:"0",zIndex:"1",fontWeight:"medium",_expanded:{display:"none"}},code:{fontFamily:"mono",lineHeight:"tall",whiteSpace:"pre",counterReset:"line 0"},codeText:{px:"var(--code-block-padding)",py:"var(--code-block-padding)",position:"relative",display:"block",width:"100%","&[data-has-focused]":{"& [data-line]:not([data-focused])":{transitionProperty:"opacity, filter",transitionDuration:"moderate",transitionTimingFunction:"ease-in-out",opacity:"var(--code-block-obscured-opacity)",filter:"blur(var(--code-block-obscured-blur))"},"&:hover":{"--code-block-obscured-opacity":"1","--code-block-obscured-blur":"0px"}},"&[data-has-line-numbers][data-plaintext]":{paddingInlineStart:"calc(var(--code-block-line-number-width) + var(--code-block-line-number-margin) + var(--code-block-padding))"},"& [data-line]":{position:"relative","--highlight-bg":"var(--code-block-highlight-bg)","--highlight-border":"var(--code-block-highlight-border)","&[data-highlight], &[data-diff]":{display:"inline-block",width:"full","&:after":{content:"''",display:"block",position:"absolute",insetStart:"calc(var(--code-block-padding) * -1)",insetEnd:"0px",width:"calc(100% + var(--code-block-padding) * 2)",height:"100%",bg:"var(--highlight-bg)",borderStartWidth:"2px",borderStartColor:"var(--highlight-border)"}},"&[data-diff='added']":{"--highlight-bg":"var(--code-block-highlight-added-bg)","--highlight-border":"var(--code-block-highlight-added-border)"},"&[data-diff='removed']":{"--highlight-bg":"var(--code-block-highlight-removed-bg)","--highlight-border":"var(--code-block-highlight-removed-border)"}},"&[data-word-wrap]":{"&[data-plaintext], & [data-line]":{whiteSpace:"pre-wrap",wordBreak:"break-all"}},"&[data-has-line-numbers]":{"--content":"counter(line)","& [data-line]:before":{content:"var(--content)",counterIncrement:"line",width:"var(--code-block-line-number-width)",marginRight:"var(--code-block-line-number-margin)",display:"inline-block",textAlign:"end",userSelect:"none",whiteSpace:"nowrap",opacity:.4},"& [data-diff='added']:before":{content:"'+'"},"& [data-diff='removed']:before":{content:"'-'"}}}},variants:{size:{sm:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.md","--code-block-header-height":"sizes.8"},title:{textStyle:"xs"},code:{fontSize:"xs"}},md:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.lg","--code-block-header-height":"sizes.10"},title:{textStyle:"xs"},code:{fontSize:"sm"}},lg:{root:{"--code-block-padding":"spacing.5","--code-block-radius":"radii.xl","--code-block-header-height":"sizes.12"},title:{textStyle:"sm"},code:{fontSize:"sm"}}}},defaultVariants:{size:"md"}}),CS=B({slots:Ac.keys(),className:"chakra-collapsible",base:{content:{overflow:"hidden",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}),wS=B({className:"colorPicker",slots:cS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5"},label:{color:"fg",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},valueText:{textAlign:"start"},control:{display:"flex",alignItems:"center",flexDirection:"row",gap:"2",position:"relative"},swatchTrigger:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{display:"flex",alignItems:"center",justifyContent:"center",flexDirection:"row",flexShrink:"0",gap:"2",textStyle:"sm",minH:"var(--input-height)",minW:"var(--input-height)",px:"1",rounded:"l2",_disabled:{opacity:"0.5"},"--focus-color":"colors.colorPalette.focusRing","&:focus-visible":{borderColor:"var(--focus-color)",outline:"1px solid var(--focus-color)"},"&[data-fit-content]":{"--input-height":"unset",px:"0",border:"0"}},content:{display:"flex",flexDirection:"column",bg:"bg.panel",borderRadius:"l3",boxShadow:"lg",width:"64",p:"4",gap:"3",zIndex:"dropdown",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},area:{height:"180px",borderRadius:"l2",overflow:"hidden"},areaThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",focusVisibleRing:"mixed",focusRingColor:"white"},areaBackground:{height:"full"},channelSlider:{borderRadius:"l2",flex:"1"},channelSliderTrack:{height:"var(--slider-height)",borderRadius:"inherit",boxShadow:"inset 0 0 0 1px rgba(0,0,0,0.1)"},channelText:{textStyle:"xs",color:"fg.muted",fontWeight:"medium",textTransform:"capitalize"},swatchGroup:{display:"flex",flexDirection:"row",flexWrap:"wrap",gap:"2"},swatch:{...Bh.base,borderRadius:"l1"},swatchIndicator:{color:"white",rounded:"full"},channelSliderThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",transform:"translate(-50%, -50%)",focusVisibleRing:"outside",focusRingOffset:"1px"},channelInput:{...he.base,"&::-webkit-inner-spin-button, &::-webkit-outer-spin-button":{WebkitAppearance:"none",margin:0}},formatSelect:{textStyle:"xs",textTransform:"uppercase",borderWidth:"1px",minH:"6",focusRing:"inside",rounded:"l2"},transparencyGrid:{borderRadius:"l2"},view:{display:"flex",flexDirection:"column",gap:"2"}},variants:{size:{"2xs":{channelInput:he.variants?.size?.["2xs"],swatch:{"--swatch-size":"sizes.4.5"},trigger:{"--input-height":"sizes.7"},area:{"--thumb-size":"sizes.3"},channelSlider:{"--slider-height":"sizes.3","--thumb-size":"sizes.3"}},xs:{channelInput:he.variants?.size?.xs,swatch:{"--swatch-size":"sizes.5"},trigger:{"--input-height":"sizes.8"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},sm:{channelInput:he.variants?.size?.sm,swatch:{"--swatch-size":"sizes.6"},trigger:{"--input-height":"sizes.9"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},md:{channelInput:he.variants?.size?.md,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.10"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},lg:{channelInput:he.variants?.size?.lg,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.11"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},xl:{channelInput:he.variants?.size?.xl,swatch:{"--swatch-size":"sizes.8"},trigger:{"--input-height":"sizes.12"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},"2xl":{channelInput:he.variants?.size?.["2xl"],swatch:{"--swatch-size":"sizes.10"},trigger:{"--input-height":"sizes.16"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}}},variant:{outline:{channelInput:he.variants?.variant?.outline,trigger:{borderWidth:"1px"}},subtle:{channelInput:he.variants?.variant?.subtle,trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}}},defaultVariants:{size:"md",variant:"outline"}}),SS=B({className:"chakra-combobox",slots:Jw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},input:{display:"flex",alignItems:"center",justifyContent:"space-between",background:"bg.panel",width:"full",minH:"var(--combobox-input-height)",px:"var(--combobox-input-padding-x)","--input-height":"var(--combobox-input-height)",borderRadius:"l2",outline:0,userSelect:"none",textAlign:"start",_placeholderShown:{color:"fg.muted"},_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},trigger:{display:"inline-flex",alignItems:"center",justifyContent:"center","--input-height":"var(--combobox-input-height)"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"},control:{pos:"relative"},indicatorGroup:{display:"flex",alignItems:"center",justifyContent:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--combobox-input-padding-x)",_icon:{boxSize:"var(--combobox-indicator-size)"},"[data-disabled] &":{opacity:.5}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"0s"},"&[data-empty]:not(:has([data-scope=combobox][data-part=empty]))":{opacity:0}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{boxSize:"var(--combobox-indicator-size)"}},empty:{py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{pb:"var(--combobox-item-padding-y)",_last:{pb:"0"}},itemGroupLabel:{fontWeight:"medium",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"}},variants:{variant:{outline:{input:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"}},subtle:{input:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"}},flushed:{input:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}},indicatorGroup:{px:"0"}}},size:{xs:{root:{"--combobox-input-height":"sizes.8","--combobox-input-padding-x":"spacing.2","--combobox-indicator-size":"sizes.3.5"},input:{textStyle:"xs"},content:{"--combobox-item-padding-x":"spacing.1.5","--combobox-item-padding-y":"spacing.1","--combobox-indicator-size":"sizes.3.5",p:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"}},sm:{root:{"--combobox-input-height":"sizes.9","--combobox-input-padding-x":"spacing.2.5","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"}},md:{root:{"--combobox-input-height":"sizes.10","--combobox-input-padding-x":"spacing.3","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{textStyle:"sm",gap:"2"}},lg:{root:{"--combobox-input-height":"sizes.12","--combobox-input-padding-x":"spacing.4","--combobox-indicator-size":"sizes.5"},input:{textStyle:"md"},content:{"--combobox-item-padding-y":"spacing.2","--combobox-item-padding-x":"spacing.3","--combobox-indicator-size":"sizes.5",p:"1.5",textStyle:"md"},trigger:{textStyle:"md",py:"3",gap:"2"}}}},defaultVariants:{size:"md",variant:"outline"}}),ES=B({slots:Dw.keys(),className:"chakra-data-list",base:{itemLabel:{display:"flex",alignItems:"center",gap:"1"},itemValue:{display:"flex",minWidth:"0",flex:"1"}},variants:{orientation:{horizontal:{root:{display:"flex",flexDirection:"column"},item:{display:"inline-flex",alignItems:"center",gap:"4"},itemLabel:{minWidth:"120px"}},vertical:{root:{display:"flex",flexDirection:"column"},item:{display:"flex",flexDirection:"column",gap:"1"}}},size:{sm:{root:{gap:"3"},item:{textStyle:"xs"}},md:{root:{gap:"4"},item:{textStyle:"sm"}},lg:{root:{gap:"5"},item:{textStyle:"md"}}},variant:{subtle:{itemLabel:{color:"fg.muted"}},bold:{itemLabel:{fontWeight:"medium"},itemValue:{color:"fg.muted"}}}},defaultVariants:{size:"md",orientation:"vertical",variant:"subtle"}}),OS=B({slots:zw.keys(),className:"chakra-dialog",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",left:0,top:0,w:"100dvw",h:"100dvh",zIndex:"var(--z-index)",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100dvw",height:"100dvh",position:"fixed",left:0,top:0,"--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",justifyContent:"center",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,borderRadius:"l3",textStyle:"sm",my:"var(--dialog-margin, var(--dialog-base-margin))","--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"moderate"},_closed:{animationDuration:"faster"}},header:{display:"flex",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{flex:"1",px:"6",pt:"2",pb:"6"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"2",insetEnd:"2"}},variants:{placement:{center:{positioner:{alignItems:"center"},content:{"--dialog-base-margin":"auto",mx:"auto"}},top:{positioner:{alignItems:"flex-start"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}},bottom:{positioner:{alignItems:"flex-end"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}}},scrollBehavior:{inside:{positioner:{overflow:"hidden"},content:{maxH:"calc(100% - 7.5rem)"},body:{overflow:"auto"}},outside:{positioner:{overflow:"auto",pointerEvents:"auto"}}},size:{xs:{content:{maxW:"sm"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},cover:{positioner:{padding:"10"},content:{width:"100%",height:"100%","--dialog-margin":"0"}},full:{content:{maxW:"100dvw",minH:"100dvh","--dialog-margin":"0",borderRadius:"0"}}},motionPreset:{scale:{content:{_open:{animationName:"scale-in, fade-in"},_closed:{animationName:"scale-out, fade-out"}}},"slide-in-bottom":{content:{_open:{animationName:"slide-from-bottom, fade-in"},_closed:{animationName:"slide-to-bottom, fade-out"}}},"slide-in-top":{content:{_open:{animationName:"slide-from-top, fade-in"},_closed:{animationName:"slide-to-top, fade-out"}}},"slide-in-left":{content:{_open:{animationName:"slide-from-left, fade-in"},_closed:{animationName:"slide-to-left, fade-out"}}},"slide-in-right":{content:{_open:{animationName:"slide-from-right, fade-in"},_closed:{animationName:"slide-to-right, fade-out"}}},none:{}}},defaultVariants:{size:"md",scrollBehavior:"outside",placement:"top",motionPreset:"scale"}}),PS=B({slots:Mw.keys(),className:"chakra-drawer",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",insetInlineStart:0,top:0,w:"100vw",h:"100dvh",zIndex:"overlay",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100vw",height:"100dvh",position:"fixed",insetInlineStart:0,top:0,zIndex:"modal",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,zIndex:"modal",textStyle:"sm",maxH:"100dvh",color:"inherit",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"slowest",animationTimingFunction:"ease-in-smooth"},_closed:{animationDuration:"slower",animationTimingFunction:"ease-in-smooth"}},header:{display:"flex",alignItems:"center",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{px:"6",py:"2",flex:"1",overflow:"auto"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{flex:"1",textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"3",insetEnd:"2"}},variants:{size:{xs:{content:{maxW:"xs"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},full:{content:{maxW:"100vw",h:"100dvh"}}},placement:{start:{positioner:{justifyContent:"flex-start",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-left-full, fade-in",_rtl:"slide-from-right-full, fade-in"}},_closed:{animationName:{base:"slide-to-left-full, fade-out",_rtl:"slide-to-right-full, fade-out"}}}},end:{positioner:{justifyContent:"flex-end",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-right-full, fade-in",_rtl:"slide-from-left-full, fade-in"}},_closed:{animationName:{base:"slide-to-right-full, fade-out",_rtl:"slide-to-left-full, fade-out"}}}},top:{positioner:{justifyContent:"stretch",alignItems:"flex-start"},content:{maxW:"100%",_open:{animationName:"slide-from-top-full, fade-in"},_closed:{animationName:"slide-to-top-full, fade-out"}}},bottom:{positioner:{justifyContent:"stretch",alignItems:"flex-end"},content:{maxW:"100%",_open:{animationName:"slide-from-bottom-full, fade-in"},_closed:{animationName:"slide-to-bottom-full, fade-out"}}}},contained:{true:{positioner:{padding:"4"},content:{borderRadius:"l3"}}}},defaultVariants:{size:"xs",placement:"end"}}),Hh=tr({fontSize:"inherit",fontWeight:"inherit",textAlign:"inherit",bg:"transparent",borderRadius:"l2"}),RS=B({slots:jw.keys(),className:"chakra-editable",base:{root:{display:"inline-flex",alignItems:"center",position:"relative",gap:"1.5",width:"full"},preview:{...Hh,py:"1",px:"1",display:"inline-flex",alignItems:"center",transitionProperty:"common",transitionDuration:"moderate",cursor:"text",_hover:{bg:"bg.muted"},_disabled:{userSelect:"none"}},input:{...Hh,outline:"0",py:"1",px:"1",transitionProperty:"common",transitionDuration:"normal",width:"full",focusVisibleRing:"inside",focusRingWidth:"2px",_placeholder:{opacity:.6}},control:{display:"inline-flex",alignItems:"center",gap:"1.5"}},variants:{size:{sm:{root:{textStyle:"sm"},preview:{minH:"8"},input:{minH:"8"}},md:{root:{textStyle:"sm"},preview:{minH:"9"},input:{minH:"9"}},lg:{root:{textStyle:"md"},preview:{minH:"10"},input:{minH:"10"}}}},defaultVariants:{size:"md"}}),IS=B({slots:$w.keys(),className:"chakra-empty-state",base:{root:{width:"full"},content:{display:"flex",flexDirection:"column",alignItems:"center",justifyContent:"center"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:"fg.subtle",_icon:{boxSize:"1em"}},title:{fontWeight:"semibold"},description:{textStyle:"sm",color:"fg.muted"}},variants:{size:{sm:{root:{px:"4",py:"6"},title:{textStyle:"md"},content:{gap:"4"},indicator:{textStyle:"2xl"}},md:{root:{px:"8",py:"12"},title:{textStyle:"lg"},content:{gap:"6"},indicator:{textStyle:"4xl"}},lg:{root:{px:"12",py:"16"},title:{textStyle:"xl"},content:{gap:"8"},indicator:{textStyle:"6xl"}}}},defaultVariants:{size:"md"}}),TS=B({className:"chakra-field",slots:Bw.keys(),base:{requiredIndicator:{color:"fg.error",lineHeight:"1"},root:{display:"flex",width:"100%",position:"relative",gap:"1.5"},label:{display:"flex",alignItems:"center",textAlign:"start",textStyle:"sm",fontWeight:"medium",gap:"1",userSelect:"none",_disabled:{opacity:"0.5"}},errorText:{display:"inline-flex",alignItems:"center",fontWeight:"medium",gap:"1",color:"fg.error",textStyle:"xs"},helperText:{color:"fg.muted",textStyle:"xs"}},variants:{orientation:{vertical:{root:{flexDirection:"column",alignItems:"flex-start"}},horizontal:{root:{flexDirection:"row",alignItems:"center",justifyContent:"space-between"},label:{flex:"0 0 var(--field-label-width, 80px)"}}}},defaultVariants:{orientation:"vertical"}}),NS=B({className:"fieldset",slots:Ww.keys(),base:{root:{display:"flex",flexDirection:"column",width:"full"},content:{display:"flex",flexDirection:"column",width:"full"},legend:{color:"fg",fontWeight:"medium",_disabled:{opacity:"0.5"}},helperText:{color:"fg.muted",textStyle:"sm"},errorText:{display:"inline-flex",alignItems:"center",color:"fg.error",gap:"2",fontWeight:"medium",textStyle:"sm"}},variants:{size:{sm:{root:{spaceY:"2"},content:{gap:"1.5"},legend:{textStyle:"sm"}},md:{root:{spaceY:"4"},content:{gap:"4"},legend:{textStyle:"sm"}},lg:{root:{spaceY:"6"},content:{gap:"4"},legend:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),AS=B({className:"chakra-file-upload",slots:Hw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"4",width:"100%",alignItems:"flex-start"},label:{fontWeight:"medium",textStyle:"sm"},dropzone:{background:"bg",borderRadius:"l3",borderWidth:"2px",borderStyle:"dashed",display:"flex",alignItems:"center",flexDirection:"column",gap:"4",justifyContent:"center",minHeight:"2xs",px:"3",py:"2",transition:"backgrounds",focusVisibleRing:"outside",_hover:{bg:"bg.subtle"},_dragging:{bg:"colorPalette.subtle",borderStyle:"solid",borderColor:"colorPalette.solid"}},dropzoneContent:{display:"flex",flexDirection:"column",alignItems:"center",textAlign:"center",gap:"1",textStyle:"sm"},item:{pos:"relative",textStyle:"sm",animationName:"fade-in",animationDuration:"moderate",background:"bg",borderRadius:"l2",borderWidth:"1px",width:"100%",display:"flex",alignItems:"center",gap:"3",p:"4"},itemGroup:{width:"100%",display:"flex",flexDirection:"column",gap:"3",_empty:{display:"none"}},itemName:{color:"fg",fontWeight:"medium",lineClamp:"1"},itemContent:{display:"flex",flexDirection:"column",gap:"0.5",flex:"1"},itemSizeText:{color:"fg.muted",textStyle:"xs"},itemDeleteTrigger:{display:"flex",alignItems:"center",justifyContent:"center",alignSelf:"flex-start",boxSize:"5",p:"2px",color:"fg.muted",cursor:"button"},itemPreview:{color:"fg.muted",_icon:{boxSize:"4.5"}}},defaultVariants:{}}),_S=B({className:"chakra-hover-card",slots:gd.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--hovercard-bg":"colors.bg.panel",bg:"var(--hovercard-bg)",boxShadow:"lg",maxWidth:"80",borderRadius:"l3",zIndex:"popover",transformOrigin:"var(--transform-origin)",outline:"0",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--hovercard-bg)"},arrowTip:{borderTopWidth:"0.5px",borderInlineStartWidth:"0.5px"}},variants:{size:{xs:{content:{padding:"3"}},sm:{content:{padding:"4"}},md:{content:{padding:"5"}},lg:{content:{padding:"6"}}}},defaultVariants:{size:"md"}}),VS=B({className:"chakra-list",slots:Uw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"var(--list-gap)","& :where(ul, ol)":{marginTop:"var(--list-gap)"}},item:{whiteSpace:"normal",display:"list-item"},indicator:{marginEnd:"2",minHeight:"1lh",flexShrink:0,display:"inline-block",verticalAlign:"middle"}},variants:{variant:{marker:{root:{listStyle:"revert"},item:{_marker:{color:"fg.subtle"}}},plain:{item:{alignItems:"flex-start",display:"inline-flex"}}},align:{center:{item:{alignItems:"center"}},start:{item:{alignItems:"flex-start"}},end:{item:{alignItems:"flex-end"}}}},defaultVariants:{variant:"marker"}}),FS=B({className:"chakra-listbox",slots:dS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},content:{display:"flex",maxH:"96",p:"1",gap:"1",textStyle:"sm",outline:"none",scrollPadding:"1",_horizontal:{flexDirection:"row",overflowX:"auto"},_vertical:{flexDirection:"column",overflowY:"auto"},"--listbox-item-padding-x":"spacing.2","--listbox-item-padding-y":"spacing.1.5"},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"pointer",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)",_highlighted:{outline:"2px solid",outlineColor:"border.emphasized"},_disabled:{pointerEvents:"none",opacity:"0.5"}},empty:{py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{mt:"1.5",_first:{mt:"0"}},itemGroupLabel:{py:"1.5",px:"2",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"4"}}},variants:{variant:{subtle:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_hover:{bg:"bg.emphasized/60"},_selected:{bg:"bg.muted"}}},solid:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_selected:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}},plain:{}}},defaultVariants:{variant:"subtle"}}),LS=B({className:"chakra-menu",slots:Gw.keys(),base:{content:{outline:0,bg:"bg.panel",boxShadow:"lg",color:"fg",maxHeight:"var(--available-height)","--menu-z-index":"zIndex.dropdown",zIndex:"calc(var(--menu-z-index) + var(--layer-index, 0))",borderRadius:"l2",overflow:"hidden",overflowY:"auto",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},item:{textDecoration:"none",color:"fg",userSelect:"none",borderRadius:"l1",width:"100%",display:"flex",cursor:"menuitem",alignItems:"center",textAlign:"start",position:"relative",flex:"0 0 auto",outline:0,_disabled:{layerStyle:"disabled"},"&[data-type]":{ps:"8"}},itemText:{flex:"1"},itemIndicator:{position:"absolute",insetStart:"2",transform:"translateY(-50%)",top:"50%"},itemGroupLabel:{px:"2",py:"1.5",fontWeight:"semibold",textStyle:"sm"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},itemCommand:{opacity:"0.6",textStyle:"xs",ms:"auto",ps:"4",letterSpacing:"widest",fontFamily:"inherit"},separator:{height:"1px",bg:"bg.muted",my:"1",mx:"-1"}},variants:{variant:{subtle:{item:{_highlighted:{bg:"bg.emphasized/60"}}},solid:{item:{_highlighted:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}}},size:{sm:{content:{minW:"8rem",padding:"1",scrollPadding:"1"},item:{gap:"1",textStyle:"xs",py:"1",px:"1.5"}},md:{content:{minW:"8rem",padding:"1.5",scrollPadding:"1.5"},item:{gap:"2",textStyle:"sm",py:"1.5",px:"2"}}}},defaultVariants:{size:"md",variant:"subtle"}}),di=B({className:"chakra-select",slots:Qw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},trigger:{display:"flex",alignItems:"center",justifyContent:"space-between",width:"full",minH:"var(--select-trigger-height)","--input-height":"var(--select-trigger-height)",px:"var(--select-trigger-padding-x)",borderRadius:"l2",userSelect:"none",textAlign:"start",focusVisibleRing:"inside",_placeholderShown:{color:"fg.muted/80"},_disabled:{layerStyle:"disabled"},_invalid:{borderColor:"border.error"}},indicatorGroup:{display:"flex",alignItems:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--select-trigger-padding-x)",pointerEvents:"none"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:{base:"fg.muted",_disabled:"fg.subtle",_invalid:"fg.error"}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"fastest"}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{width:"4",height:"4"}},control:{pos:"relative"},itemText:{flex:"1"},itemGroup:{_first:{mt:"0"}},itemGroupLabel:{py:"1",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"}},variants:{variant:{outline:{trigger:{bg:"transparent",borderWidth:"1px",borderColor:"border",_expanded:{borderColor:"border.emphasized"}}},subtle:{trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}},size:{xs:{root:{"--select-trigger-height":"sizes.8","--select-trigger-padding-x":"spacing.2"},content:{p:"1",gap:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"},item:{py:"1",px:"2"},itemGroupLabel:{py:"1",px:"2"},indicator:{_icon:{width:"3.5",height:"3.5"}}},sm:{root:{"--select-trigger-height":"sizes.9","--select-trigger-padding-x":"spacing.2.5"},content:{p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"},indicator:{_icon:{width:"4",height:"4"}},item:{py:"1",px:"1.5"},itemGroup:{mt:"1"},itemGroupLabel:{py:"1",px:"1.5"}},md:{root:{"--select-trigger-height":"sizes.10","--select-trigger-padding-x":"spacing.3"},content:{p:"1",textStyle:"sm"},itemGroup:{mt:"1.5"},item:{py:"1.5",px:"2"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},itemGroupLabel:{py:"1.5",px:"2"},trigger:{textStyle:"sm",gap:"2"},indicator:{_icon:{width:"4",height:"4"}}},lg:{root:{"--select-trigger-height":"sizes.12","--select-trigger-padding-x":"spacing.4"},content:{p:"1.5",textStyle:"md"},itemGroup:{mt:"2"},item:{py:"2",px:"3"},itemGroupLabel:{py:"2",px:"3"},trigger:{textStyle:"md",py:"3",gap:"2"},indicator:{_icon:{width:"5",height:"5"}}}}},defaultVariants:{size:"md",variant:"outline"}}),DS=B({className:"chakra-native-select",slots:qw.keys(),base:{root:{height:"fit-content",display:"flex",width:"100%",position:"relative"},field:{width:"100%",minWidth:"0",outline:"0",appearance:"none",borderRadius:"l2","--error-color":"colors.border.error","--input-height":"var(--select-field-height)",height:"var(--select-field-height)",_disabled:{layerStyle:"disabled"},_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"},focusVisibleRing:"inside",lineHeight:"normal","& > option, & > optgroup":{bg:"bg"}},indicator:{position:"absolute",display:"inline-flex",alignItems:"center",justifyContent:"center",pointerEvents:"none",top:"50%",transform:"translateY(-50%)",height:"100%",color:"fg.muted",_disabled:{opacity:"0.5"},_invalid:{color:"fg.error"},_icon:{width:"1em",height:"1em"}}},variants:{variant:{outline:{field:di.variants?.variant.outline.trigger},subtle:{field:di.variants?.variant.subtle.trigger},plain:{field:{bg:"transparent",color:"fg",focusRingWidth:"2px"}}},size:{xs:{root:{"--select-field-height":"sizes.8"},field:{textStyle:"xs",ps:"2",pe:"6"},indicator:{textStyle:"sm",insetEnd:"1.5"}},sm:{root:{"--select-field-height":"sizes.9"},field:{textStyle:"sm",ps:"2.5",pe:"8"},indicator:{textStyle:"md",insetEnd:"2"}},md:{root:{"--select-field-height":"sizes.10"},field:{textStyle:"sm",ps:"3",pe:"8"},indicator:{textStyle:"lg",insetEnd:"2"}},lg:{root:{"--select-field-height":"sizes.11"},field:{textStyle:"md",ps:"4",pe:"8"},indicator:{textStyle:"xl",insetEnd:"3"}},xl:{root:{"--select-field-height":"sizes.12"},field:{textStyle:"md",ps:"4.5",pe:"10"},indicator:{textStyle:"xl",insetEnd:"3"}}}},defaultVariants:di.defaultVariants});function Ta(e,t){const n={};for(const r in e){const o=t(r,e[r]);n[o[0]]=o[1]}return n}const Uh=tr({display:"flex",justifyContent:"center",alignItems:"center",flex:"1",userSelect:"none",cursor:"button",lineHeight:"1",color:"fg.muted","--stepper-base-radius":"radii.l1","--stepper-radius":"calc(var(--stepper-base-radius) + 1px)",_icon:{boxSize:"1em"},_disabled:{opacity:"0.5"},_hover:{bg:"bg.muted"},_active:{bg:"bg.emphasized"}}),zS=B({className:"chakra-number-input",slots:Nd.keys(),base:{root:{position:"relative",zIndex:"0",isolation:"isolate"},input:{...he.base,verticalAlign:"top",pe:"calc(var(--stepper-width) + 0.5rem)"},control:{display:"flex",flexDirection:"column",position:"absolute",top:"0",insetEnd:"0px",margin:"1px",width:"var(--stepper-width)",height:"calc(100% - 2px)",zIndex:"1",borderStartWidth:"1px",divideY:"1px"},incrementTrigger:{...Uh,borderTopEndRadius:"var(--stepper-radius)"},decrementTrigger:{...Uh,borderBottomEndRadius:"var(--stepper-radius)"},valueText:{fontWeight:"medium",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums"}},variants:{size:{xs:{input:he.variants.size.xs,control:{fontSize:"2xs","--stepper-width":"sizes.4"}},sm:{input:he.variants.size.sm,control:{fontSize:"xs","--stepper-width":"sizes.5"}},md:{input:he.variants.size.md,control:{fontSize:"sm","--stepper-width":"sizes.6"}},lg:{input:he.variants.size.lg,control:{fontSize:"sm","--stepper-width":"sizes.6"}}},variant:Ta(he.variants.variant,(e,t)=>[e,{input:t}])},defaultVariants:{size:"md",variant:"outline"}}),{variants:Gh,defaultVariants:MS}=he,jS=B({className:"chakra-pin-input",slots:Ld.keys(),base:{input:{...he.base,textAlign:"center",width:"var(--input-height)"},control:{display:"inline-flex",gap:"2",isolation:"isolate"}},variants:{size:Ta(Gh.size,(e,t)=>[e,{input:{...t,px:"1"}}]),variant:Ta(Gh.variant,(e,t)=>[e,{input:t}]),attached:{true:{control:{gap:"0",spaceX:"-1px"},input:{_notFirst:{borderStartRadius:"0"},_notLast:{borderEndRadius:"0"},_focusVisible:{zIndex:"1"}}}}},defaultVariants:MS}),$S=B({className:"chakra-popover",slots:Kw.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--popover-bg":"colors.bg.panel",bg:"var(--popover-bg)",boxShadow:"lg","--popover-size":"sizes.xs","--popover-mobile-size":"calc(100dvw - 1rem)",width:{base:"min(var(--popover-mobile-size), var(--popover-size))",sm:"var(--popover-size)"},borderRadius:"l3","--popover-z-index":"zIndex.popover",zIndex:"calc(var(--popover-z-index) + var(--layer-index, 0))",outline:"0",transformOrigin:"var(--transform-origin)",maxHeight:"var(--available-height)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"faster"}},header:{paddingInline:"var(--popover-padding)",paddingTop:"var(--popover-padding)"},body:{padding:"var(--popover-padding)",flex:"1"},footer:{display:"flex",alignItems:"center",paddingInline:"var(--popover-padding)",paddingBottom:"var(--popover-padding)"},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--popover-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px"}},variants:{size:{xs:{content:{"--popover-padding":"spacing.3"}},sm:{content:{"--popover-padding":"spacing.4"}},md:{content:{"--popover-padding":"spacing.5"}},lg:{content:{"--popover-padding":"spacing.6"}}}},defaultVariants:{size:"md"}}),BS=B({slots:ha.keys(),className:"chakra-progress",base:{root:{textStyle:"sm",position:"relative"},track:{overflow:"hidden",position:"relative"},range:{display:"flex",alignItems:"center",justifyContent:"center",transitionProperty:"width, height",transitionDuration:"slow",height:"100%",bgColor:"var(--track-color)",_indeterminate:{"--animate-from-x":"-40%","--animate-to-x":"100%",position:"absolute",willChange:"left",minWidth:"50%",animation:"position 1s ease infinite normal none running",backgroundImage:"linear-gradient(to right, transparent 0%, var(--track-color) 50%, transparent 100%)"}},label:{display:"inline-flex",fontWeight:"medium",alignItems:"center",gap:"1"},valueText:{textStyle:"xs",lineHeight:"1",fontWeight:"medium"}},variants:{variant:{outline:{track:{shadow:"inset",bgColor:"bg.muted"},range:{bgColor:"colorPalette.solid"}},subtle:{track:{bgColor:"colorPalette.muted"},range:{bgColor:"colorPalette.solid/72"}}},shape:{square:{},rounded:{track:{borderRadius:"l1"}},full:{track:{borderRadius:"full"}}},striped:{true:{range:{backgroundImage:"linear-gradient(45deg, var(--stripe-color) 25%, transparent 25%, transparent 50%, var(--stripe-color) 50%, var(--stripe-color) 75%, transparent 75%, transparent)",backgroundSize:"var(--stripe-size) var(--stripe-size)","--stripe-size":"1rem","--stripe-color":{_light:"rgba(255, 255, 255, 0.3)",_dark:"rgba(0, 0, 0, 0.3)"}}}},animated:{true:{range:{"--animate-from":"var(--stripe-size)",animation:"bg-position 1s linear infinite"}}},size:{xs:{track:{h:"1.5"}},sm:{track:{h:"2"}},md:{track:{h:"2.5"}},lg:{track:{h:"3"}},xl:{track:{h:"4"}}}},defaultVariants:{variant:"outline",size:"md",shape:"rounded"}}),WS=B({className:"chakra-progress-circle",slots:ha.keys(),base:{root:{display:"inline-flex",textStyle:"sm",position:"relative"},circle:{_indeterminate:{animation:"spin 2s linear infinite"}},circleTrack:{"--track-color":"colors.colorPalette.muted",stroke:"var(--track-color)"},circleRange:{stroke:"colorPalette.solid",transitionProperty:"stroke-dashoffset, stroke-dasharray",transitionDuration:"0.6s",_indeterminate:{animation:"circular-progress 1.5s linear infinite"}},label:{display:"inline-flex"},valueText:{lineHeight:"1",fontWeight:"medium",letterSpacing:"tight",fontVariantNumeric:"tabular-nums"}},variants:{size:{xs:{circle:{"--size":"24px","--thickness":"4px"},valueText:{textStyle:"2xs"}},sm:{circle:{"--size":"32px","--thickness":"5px"},valueText:{textStyle:"2xs"}},md:{circle:{"--size":"40px","--thickness":"6px"},valueText:{textStyle:"xs"}},lg:{circle:{"--size":"48px","--thickness":"7px"},valueText:{textStyle:"sm"}},xl:{circle:{"--size":"64px","--thickness":"8px"},valueText:{textStyle:"sm"}}}},defaultVariants:{size:"md"}}),HS=B({slots:zd.keys(),className:"chakra-qr-code",base:{root:{position:"relative",width:"fit-content","--qr-code-overlay-size":"calc(var(--qr-code-size) / 3)"},frame:{width:"var(--qr-code-size)",height:"var(--qr-code-size)",fill:"currentColor"},overlay:{display:"flex",alignItems:"center",justifyContent:"center",width:"var(--qr-code-overlay-size)",height:"var(--qr-code-overlay-size)",padding:"1",bg:"bg",rounded:"l1"}},variants:{size:{"2xs":{root:{"--qr-code-size":"40px"}},xs:{root:{"--qr-code-size":"64px"}},sm:{root:{"--qr-code-size":"80px"}},md:{root:{"--qr-code-size":"120px"}},lg:{root:{"--qr-code-size":"160px"}},xl:{root:{"--qr-code-size":"200px"}},"2xl":{root:{"--qr-code-size":"240px"}},full:{root:{"--qr-code-size":"100%"}}}},defaultVariants:{size:"md"}}),US=B({className:"chakra-radio-card",slots:Yw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",isolation:"isolate"},item:{flex:"1",display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",_focus:{bg:"colorPalette.muted/20"},_disabled:{opacity:"0.8",borderColor:"border.disabled"},_checked:{zIndex:"1"}},label:{display:"inline-flex",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},itemText:{fontWeight:"medium",flex:"1"},itemDescription:{opacity:"0.64",textStyle:"sm"},itemControl:{display:"inline-flex",flex:"1",pos:"relative",rounded:"inherit",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)",_disabled:{bg:"bg.muted"}},itemIndicator:Ee.base,itemAddon:{roundedBottom:"inherit",_disabled:{color:"fg.muted"}},itemContent:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)"}},variants:{size:{sm:{item:{textStyle:"sm"},itemControl:{padding:"3",gap:"1.5"},itemAddon:{px:"3",py:"1.5",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.sm},md:{item:{textStyle:"sm"},itemControl:{padding:"4",gap:"2.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.md},lg:{item:{textStyle:"md"},itemControl:{padding:"4",gap:"3.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.lg}},variant:{surface:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"}},itemIndicator:Ee.variants?.variant.solid},subtle:{item:{bg:"bg.muted"},itemControl:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},itemIndicator:Ee.variants?.variant.outline},outline:{item:{borderWidth:"1px",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.solid},solid:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.inverted}},justify:{start:{item:{"--radio-card-justify":"flex-start"}},end:{item:{"--radio-card-justify":"flex-end"}},center:{item:{"--radio-card-justify":"center"}}},align:{start:{item:{"--radio-card-align":"flex-start"},itemControl:{textAlign:"start"}},end:{item:{"--radio-card-align":"flex-end"},itemControl:{textAlign:"end"}},center:{item:{"--radio-card-align":"center"},itemControl:{textAlign:"center"}}},orientation:{vertical:{itemControl:{flexDirection:"column"}},horizontal:{itemControl:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),GS=B({className:"chakra-radio-group",slots:Wh.keys(),base:{item:{display:"inline-flex",alignItems:"center",position:"relative",fontWeight:"medium",_disabled:{cursor:"disabled"}},itemControl:Ee.base,label:{userSelect:"none",textStyle:"sm",_disabled:{opacity:"0.5"}}},variants:{variant:{outline:{itemControl:Ee.variants?.variant?.outline},subtle:{itemControl:Ee.variants?.variant?.subtle},solid:{itemControl:Ee.variants?.variant?.solid}},size:{xs:{item:{textStyle:"xs",gap:"1.5"},itemControl:Ee.variants?.size?.xs},sm:{item:{textStyle:"sm",gap:"2"},itemControl:Ee.variants?.size?.sm},md:{item:{textStyle:"sm",gap:"2.5"},itemControl:Ee.variants?.size?.md},lg:{item:{textStyle:"md",gap:"3"},itemControl:Ee.variants?.size?.lg}}},defaultVariants:{size:"md",variant:"solid"}}),qS=B({className:"chakra-rating-group",slots:Xw.keys(),base:{root:{display:"inline-flex"},control:{display:"inline-flex",alignItems:"center"},item:{display:"inline-flex",alignItems:"center",justifyContent:"center",userSelect:"none"},itemIndicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",width:"1em",height:"1em",position:"relative","--clip-path":{base:"inset(0 50% 0 0)",_rtl:"inset(0 0 0 50%)"},_icon:{stroke:"currentColor",width:"100%",height:"100%",display:"inline-block",flexShrink:0,position:"absolute",left:0,top:0},"& [data-bg]":{color:"bg.emphasized"},"& [data-fg]":{color:"transparent"},"&[data-highlighted]:not([data-half])":{"& [data-fg]":{color:"colorPalette.solid"}},"&[data-half]":{"& [data-fg]":{color:"colorPalette.solid",clipPath:"var(--clip-path)"}}}},variants:{size:{xs:{item:{textStyle:"sm"}},sm:{item:{textStyle:"md"}},md:{item:{textStyle:"xl"}},lg:{item:{textStyle:"2xl"}}}},defaultVariants:{size:"md"}}),KS=B({className:"chakra-scroll-area",slots:jd.keys(),base:{root:{display:"flex",flexDirection:"column",width:"100%",height:"100%",position:"relative",overflow:"hidden","--scrollbar-margin":"2px","--scrollbar-click-area":"calc(var(--scrollbar-size) + calc(var(--scrollbar-margin) * 2))"},viewport:{display:"flex",flexDirection:"column",height:"100%",width:"100%",borderRadius:"inherit",WebkitOverflowScrolling:"touch",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}},content:{minWidth:"100%"},scrollbar:{display:"flex",userSelect:"none",touchAction:"none",borderRadius:"full",colorPalette:"gray",transition:"opacity 150ms 300ms",position:"relative",margin:"var(--scrollbar-margin)","&:not([data-overflow-x], [data-overflow-y])":{display:"none"},bg:"{colors.colorPalette.solid/10}","--thumb-bg":"{colors.colorPalette.solid/25}","&:is(:hover, :active)":{"--thumb-bg":"{colors.colorPalette.solid/50}"},_before:{content:'""',position:"absolute"},_vertical:{width:"var(--scrollbar-size)",flexDirection:"column","&::before":{width:"var(--scrollbar-click-area)",height:"100%",insetInlineStart:"calc(var(--scrollbar-margin) * -1)"}},_horizontal:{height:"var(--scrollbar-size)",flexDirection:"row","&::before":{height:"var(--scrollbar-click-area)",width:"100%",top:"calc(var(--scrollbar-margin) * -1)"}}},thumb:{borderRadius:"inherit",bg:"var(--thumb-bg)",transition:"backgrounds",_vertical:{width:"full"},_horizontal:{height:"full"}},corner:{bg:"bg.muted",margin:"var(--scrollbar-margin)",opacity:0,transition:"opacity 150ms 300ms","&[data-hover]":{transitionDelay:"0ms",opacity:1}}},variants:{variant:{hover:{scrollbar:{opacity:"0","&[data-hover], &[data-scrolling]":{opacity:"1",transitionDuration:"faster",transitionDelay:"0ms"}}},always:{scrollbar:{opacity:"1"}}},size:{xs:{root:{"--scrollbar-size":"sizes.1"}},sm:{root:{"--scrollbar-size":"sizes.1.5"}},md:{root:{"--scrollbar-size":"sizes.2"}},lg:{root:{"--scrollbar-size":"sizes.3"}}}},defaultVariants:{size:"md",variant:"hover"}}),YS=B({className:"chakra-segment-group",slots:$d.keys(),base:{root:{"--segment-radius":"radii.l2",borderRadius:"l2",display:"inline-flex",boxShadow:"inset",minW:"max-content",textAlign:"center",position:"relative",isolation:"isolate",bg:"bg.muted",_vertical:{flexDirection:"column"}},item:{display:"flex",alignItems:"center",justifyContent:"center",userSelect:"none",fontSize:"sm",position:"relative",color:"fg",borderRadius:"var(--segment-radius)",_disabled:{opacity:"0.5"},"&:has(input:focus-visible)":{focusRing:"outside"},_before:{content:'""',position:"absolute",bg:"border",transition:"opacity 0.2s"},_horizontal:{_before:{insetInlineStart:0,insetBlock:"1.5",width:"1px"}},_vertical:{_before:{insetBlockStart:0,insetInline:"1.5",height:"1px"}},"& + &[data-state=checked], &[data-state=checked] + &, &:first-of-type":{_before:{opacity:"0"}},"&[data-state=checked][data-ssr]":{shadow:"sm",bg:"bg",borderRadius:"var(--segment-radius)"}},indicator:{shadow:"sm",pos:"absolute",bg:{_light:"bg",_dark:"bg.emphasized"},width:"var(--width)",height:"var(--height)",top:"var(--top)",left:"var(--left)",zIndex:-1,borderRadius:"var(--segment-radius)"}},variants:{size:{xs:{item:{textStyle:"xs",px:"3",gap:"1",height:"6"}},sm:{item:{textStyle:"sm",px:"4",gap:"2",height:"8"}},md:{item:{textStyle:"sm",px:"4",gap:"2",height:"10"}},lg:{item:{textStyle:"md",px:"4.5",gap:"3",height:"11"}}}},defaultVariants:{size:"md"}}),XS=B({className:"chakra-slider",slots:Zw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",textStyle:"sm",position:"relative",isolation:"isolate",touchAction:"none"},label:{fontWeight:"medium",textStyle:"sm"},control:{display:"inline-flex",alignItems:"center",position:"relative"},track:{overflow:"hidden",borderRadius:"full",flex:"1"},range:{width:"inherit",height:"inherit",_disabled:{bg:"border.emphasized!"}},markerGroup:{position:"absolute!",zIndex:"1"},marker:{"--marker-bg":{base:"white",_underValue:"colors.bg"},display:"flex",alignItems:"center",gap:"calc(var(--slider-thumb-size) / 2)",color:"fg.muted",textStyle:"xs"},markerIndicator:{width:"var(--slider-marker-size)",height:"var(--slider-marker-size)",borderRadius:"full",bg:"var(--marker-bg)"},thumb:{width:"var(--slider-thumb-size)",height:"var(--slider-thumb-size)",display:"flex",alignItems:"center",justifyContent:"center",outline:0,zIndex:"2",borderRadius:"full",_focusVisible:{ring:"2px",ringColor:"colorPalette.focusRing",ringOffset:"2px",ringOffsetColor:"bg"}}},variants:{size:{sm:{root:{"--slider-thumb-size":"sizes.4","--slider-track-size":"sizes.1.5","--slider-marker-center":"6px","--slider-marker-size":"sizes.1","--slider-marker-inset":"3px"}},md:{root:{"--slider-thumb-size":"sizes.5","--slider-track-size":"sizes.2","--slider-marker-center":"8px","--slider-marker-size":"sizes.1","--slider-marker-inset":"4px"}},lg:{root:{"--slider-thumb-size":"sizes.6","--slider-track-size":"sizes.2.5","--slider-marker-center":"9px","--slider-marker-size":"sizes.1.5","--slider-marker-inset":"5px"}}},variant:{outline:{track:{shadow:"inset",bg:"bg.emphasized/72"},range:{bg:"colorPalette.solid"},thumb:{borderWidth:"2px",borderColor:"colorPalette.solid",bg:"bg",_disabled:{bg:"border.emphasized",borderColor:"border.emphasized"}}},solid:{track:{bg:"colorPalette.subtle",_disabled:{bg:"bg.muted"}},range:{bg:"colorPalette.solid"},thumb:{bg:"colorPalette.solid",_disabled:{bg:"border.emphasized"}}}},orientation:{vertical:{root:{display:"inline-flex"},control:{flexDirection:"column",height:"100%",minWidth:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginEnd:"4"}},track:{width:"var(--slider-track-size)"},thumb:{left:"50%",translate:"-50% 0"},markerGroup:{insetStart:"var(--slider-marker-center)",insetBlock:"var(--slider-marker-inset)"},marker:{flexDirection:"row"}},horizontal:{control:{flexDirection:"row",width:"100%",minHeight:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginBottom:"4"}},track:{height:"var(--slider-track-size)"},thumb:{top:"50%",translate:"0 -50%"},markerGroup:{top:"var(--slider-marker-center)",insetInline:"var(--slider-marker-inset)"},marker:{flexDirection:"column"}}}},defaultVariants:{size:"md",variant:"outline",orientation:"horizontal"}}),QS=B({className:"chakra-stat",slots:eS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",position:"relative",flex:"1"},label:{display:"inline-flex",gap:"1.5",alignItems:"center",color:"fg.muted",textStyle:"sm"},helpText:{color:"fg.muted",textStyle:"xs"},valueUnit:{color:"fg.muted",textStyle:"xs",fontWeight:"initial",letterSpacing:"initial"},valueText:{verticalAlign:"baseline",fontWeight:"semibold",letterSpacing:"tight",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums",display:"inline-flex",gap:"1"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",marginEnd:1,"& :where(svg)":{w:"1em",h:"1em"},"&[data-type=up]":{color:"fg.success"},"&[data-type=down]":{color:"fg.error"}}},variants:{size:{sm:{valueText:{textStyle:"xl"}},md:{valueText:{textStyle:"2xl"}},lg:{valueText:{textStyle:"3xl"}}}},defaultVariants:{size:"md"}}),JS=B({className:"chakra-status",slots:tS.keys(),base:{root:{display:"inline-flex",alignItems:"center",gap:"2"},indicator:{width:"0.64em",height:"0.64em",flexShrink:0,borderRadius:"full",forcedColorAdjust:"none",bg:"colorPalette.solid"}},variants:{size:{sm:{root:{textStyle:"xs"}},md:{root:{textStyle:"sm"}},lg:{root:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),ZS=B({className:"chakra-steps",slots:nS.keys(),base:{root:{display:"flex",width:"full"},list:{display:"flex",justifyContent:"space-between","--steps-gutter":"spacing.3","--steps-thickness":"2px"},title:{fontWeight:"medium",color:"fg"},description:{color:"fg.muted"},separator:{bg:"border",flex:"1"},indicator:{display:"flex",justifyContent:"center",alignItems:"center",flexShrink:"0",borderRadius:"full",fontWeight:"medium",width:"var(--steps-size)",height:"var(--steps-size)",_icon:{flexShrink:"0",width:"var(--steps-icon-size)",height:"var(--steps-icon-size)"}},item:{position:"relative",display:"flex",gap:"3",flex:"1 0 0","&:last-of-type":{flex:"initial","& [data-part=separator]":{display:"none"}}},trigger:{display:"flex",alignItems:"center",gap:"3",textAlign:"start",focusVisibleRing:"outside",borderRadius:"l2"},content:{focusVisibleRing:"outside"}},variants:{orientation:{vertical:{root:{flexDirection:"row",height:"100%"},list:{flexDirection:"column",alignItems:"flex-start"},separator:{position:"absolute",width:"var(--steps-thickness)",height:"100%",maxHeight:"calc(100% - var(--steps-size) - var(--steps-gutter) * 2)",top:"calc(var(--steps-size) + var(--steps-gutter))",insetStart:"calc(var(--steps-size) / 2 - 1px)"},item:{alignItems:"flex-start"}},horizontal:{root:{flexDirection:"column",width:"100%"},list:{flexDirection:"row",alignItems:"center"},separator:{width:"100%",height:"var(--steps-thickness)",marginX:"var(--steps-gutter)"},item:{alignItems:"center"}}},variant:{solid:{indicator:{_incomplete:{borderWidth:"var(--steps-thickness)"},_current:{bg:"colorPalette.muted",borderWidth:"var(--steps-thickness)",borderColor:"colorPalette.solid",color:"colorPalette.fg"},_complete:{bg:"colorPalette.solid",borderColor:"colorPalette.solid",color:"colorPalette.contrast"}},separator:{_complete:{bg:"colorPalette.solid"}}},subtle:{indicator:{_incomplete:{bg:"bg.muted"},_current:{bg:"colorPalette.muted",color:"colorPalette.fg"},_complete:{bg:"colorPalette.emphasized",color:"colorPalette.fg"}},separator:{_complete:{bg:"colorPalette.emphasized"}}}},size:{xs:{root:{gap:"2.5"},list:{"--steps-size":"sizes.6","--steps-icon-size":"sizes.3.5",textStyle:"xs"},title:{textStyle:"sm"}},sm:{root:{gap:"3"},list:{"--steps-size":"sizes.8","--steps-icon-size":"sizes.4",textStyle:"xs"},title:{textStyle:"sm"}},md:{root:{gap:"4"},list:{"--steps-size":"sizes.10","--steps-icon-size":"sizes.4",textStyle:"sm"},title:{textStyle:"sm"}},lg:{root:{gap:"6"},list:{"--steps-size":"sizes.11","--steps-icon-size":"sizes.5",textStyle:"md"},title:{textStyle:"md"}}}},defaultVariants:{size:"md",variant:"solid",orientation:"horizontal"}}),eE=B({slots:rS.keys(),className:"chakra-switch",base:{root:{display:"inline-flex",gap:"2.5",alignItems:"center",position:"relative",verticalAlign:"middle","--switch-diff":"calc(var(--switch-width) - var(--switch-height))","--switch-x":{base:"var(--switch-diff)",_rtl:"calc(var(--switch-diff) * -1)"}},label:{lineHeight:"1",userSelect:"none",fontSize:"sm",fontWeight:"medium",_disabled:{opacity:"0.5"}},indicator:{position:"absolute",height:"var(--switch-height)",width:"var(--switch-height)",fontSize:"var(--switch-indicator-font-size)",fontWeight:"medium",flexShrink:0,userSelect:"none",display:"grid",placeContent:"center",transition:"inset-inline-start 0.12s ease",insetInlineStart:"calc(var(--switch-x) - 2px)",_checked:{insetInlineStart:"2px"}},control:{display:"inline-flex",gap:"0.5rem",flexShrink:0,justifyContent:"flex-start",cursor:"switch",borderRadius:"full",position:"relative",width:"var(--switch-width)",height:"var(--switch-height)",transition:"backgrounds",_disabled:{opacity:"0.5",cursor:"not-allowed"},_invalid:{outline:"2px solid",outlineColor:"border.error",outlineOffset:"2px"}},thumb:{display:"flex",alignItems:"center",justifyContent:"center",flexShrink:0,transitionProperty:"translate",transitionDuration:"fast",borderRadius:"inherit",_checked:{translate:"var(--switch-x) 0"}}},variants:{variant:{solid:{control:{borderRadius:"full",bg:"bg.emphasized",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}},thumb:{bg:"white",width:"var(--switch-height)",height:"var(--switch-height)",scale:"0.8",boxShadow:"sm",_checked:{bg:"colorPalette.contrast"}}},raised:{control:{borderRadius:"full",height:"calc(var(--switch-height) / 2)",bg:"bg.muted",boxShadow:"inset",_checked:{bg:"colorPalette.solid/60"}},thumb:{width:"var(--switch-height)",height:"var(--switch-height)",position:"relative",top:"calc(var(--switch-height) * -0.25)",bg:"white",boxShadow:"xs",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}}}},size:{xs:{root:{"--switch-width":"sizes.6","--switch-height":"sizes.3","--switch-indicator-font-size":"fontSizes.xs"}},sm:{root:{"--switch-width":"sizes.8","--switch-height":"sizes.4","--switch-indicator-font-size":"fontSizes.xs"}},md:{root:{"--switch-width":"sizes.10","--switch-height":"sizes.5","--switch-indicator-font-size":"fontSizes.sm"}},lg:{root:{"--switch-width":"sizes.12","--switch-height":"sizes.6","--switch-indicator-font-size":"fontSizes.md"}}}},defaultVariants:{variant:"solid",size:"md"}}),tE=B({className:"chakra-table",slots:oS.keys(),base:{root:{fontVariantNumeric:"lining-nums tabular-nums",borderCollapse:"collapse",width:"full",textAlign:"start",verticalAlign:"top"},row:{_selected:{bg:"colorPalette.subtle"}},cell:{textAlign:"start",alignItems:"center"},columnHeader:{fontWeight:"medium",textAlign:"start",color:"fg"},caption:{fontWeight:"medium",textStyle:"xs"},footer:{fontWeight:"medium"}},variants:{interactive:{true:{body:{"& tr":{_hover:{bg:"colorPalette.subtle"}}}}},stickyHeader:{true:{header:{"& :where(tr)":{top:"var(--table-sticky-offset, 0)",position:"sticky",zIndex:1}}}},striped:{true:{row:{"&:nth-of-type(odd) td":{bg:"bg.muted"}}}},showColumnBorder:{true:{columnHeader:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}},cell:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}}}},variant:{line:{columnHeader:{borderBottomWidth:"1px"},cell:{borderBottomWidth:"1px"},row:{bg:"bg"}},outline:{root:{boxShadow:"0 0 0 1px {colors.border}",overflow:"hidden"},columnHeader:{borderBottomWidth:"1px"},header:{bg:"bg.muted"},row:{"&:not(:last-of-type)":{borderBottomWidth:"1px"}},footer:{borderTopWidth:"1px"}}},size:{sm:{root:{textStyle:"sm"},columnHeader:{px:"2",py:"2"},cell:{px:"2",py:"2"}},md:{root:{textStyle:"sm"},columnHeader:{px:"3",py:"3"},cell:{px:"3",py:"3"}},lg:{root:{textStyle:"md"},columnHeader:{px:"4",py:"3"},cell:{px:"4",py:"3"}}}},defaultVariants:{variant:"line",size:"md"}}),nE=B({slots:sS.keys(),className:"chakra-tabs",base:{root:{"--tabs-trigger-radius":"radii.l2",position:"relative",_horizontal:{display:"block"},_vertical:{display:"flex"}},list:{display:"inline-flex",position:"relative",isolation:"isolate","--tabs-indicator-shadow":"shadows.xs","--tabs-indicator-bg":"colors.bg",minH:"var(--tabs-height)",_horizontal:{flexDirection:"row"},_vertical:{flexDirection:"column"}},trigger:{outline:"0",minW:"var(--tabs-height)",height:"var(--tabs-height)",display:"flex",alignItems:"center",fontWeight:"medium",position:"relative",cursor:"button",gap:"2",_focusVisible:{zIndex:1,outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{cursor:"not-allowed",opacity:.5}},content:{focusVisibleRing:"inside",_horizontal:{width:"100%",pt:"var(--tabs-content-padding)"},_vertical:{height:"100%",ps:"var(--tabs-content-padding)"}},indicator:{width:"var(--width)",height:"var(--height)",borderRadius:"var(--tabs-indicator-radius)",bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",zIndex:-1}},variants:{fitted:{true:{list:{display:"flex"},trigger:{flex:1,textAlign:"center",justifyContent:"center"}}},justify:{start:{list:{justifyContent:"flex-start"}},center:{list:{justifyContent:"center"}},end:{list:{justifyContent:"flex-end"}}},size:{sm:{root:{"--tabs-height":"sizes.9","--tabs-content-padding":"spacing.3"},trigger:{py:"1",px:"3",textStyle:"sm"}},md:{root:{"--tabs-height":"sizes.10","--tabs-content-padding":"spacing.4"},trigger:{py:"2",px:"4",textStyle:"sm"}},lg:{root:{"--tabs-height":"sizes.11","--tabs-content-padding":"spacing.4.5"},trigger:{py:"2",px:"4.5",textStyle:"md"}}},variant:{line:{list:{display:"flex",borderColor:"border",_horizontal:{borderBottomWidth:"1px"},_vertical:{borderEndWidth:"1px"}},trigger:{color:"fg.muted",_disabled:{_active:{bg:"initial"}},_selected:{color:"fg",_horizontal:{layerStyle:"indicator.bottom","--indicator-offset-y":"-1px","--indicator-color":"colors.colorPalette.solid"},_vertical:{layerStyle:"indicator.end","--indicator-offset-x":"-1px"}}}},subtle:{trigger:{borderRadius:"var(--tabs-trigger-radius)",color:"fg.muted",_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}},enclosed:{list:{bg:"bg.muted",padding:"1",borderRadius:"l3",minH:"calc(var(--tabs-height) - 4px)"},trigger:{justifyContent:"center",color:"fg.muted",borderRadius:"var(--tabs-trigger-radius)",_selected:{bg:"bg",color:"colorPalette.fg",shadow:"xs"}}},outline:{list:{"--line-thickness":"1px","--line-offset":"calc(var(--line-thickness) * -1)",borderColor:"border",display:"flex",_horizontal:{_before:{content:'""',position:"absolute",bottom:"0px",width:"100%",borderBottomWidth:"var(--line-thickness)",borderBottomColor:"border"}},_vertical:{_before:{content:'""',position:"absolute",insetInline:"var(--line-offset)",height:"calc(100% - calc(var(--line-thickness) * 2))",borderEndWidth:"var(--line-thickness)",borderEndColor:"border"}}},trigger:{color:"fg.muted",borderWidth:"1px",borderColor:"transparent",_selected:{bg:"currentBg",color:"colorPalette.fg"},_horizontal:{borderTopRadius:"var(--tabs-trigger-radius)",marginBottom:"var(--line-offset)",marginEnd:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderBottomColor:"transparent"}},_vertical:{borderStartRadius:"var(--tabs-trigger-radius)",marginEnd:"var(--line-offset)",marginBottom:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderEndColor:"transparent"}}}},plain:{trigger:{color:"fg.muted",_selected:{color:"colorPalette.fg"},borderRadius:"var(--tabs-trigger-radius)","&[data-selected][data-ssr]":{bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",borderRadius:"var(--tabs-indicator-radius)"}}}}},defaultVariants:{size:"md",variant:"line"}}),hi=Ia.variants?.variant,rE=B({slots:aS.keys(),className:"chakra-tag",base:{root:{display:"inline-flex",alignItems:"center",verticalAlign:"top",maxWidth:"100%",userSelect:"none",borderRadius:"l2",focusVisibleRing:"outside"},label:{lineClamp:"1"},closeTrigger:{display:"flex",alignItems:"center",justifyContent:"center",outline:"0",borderRadius:"l1",color:"currentColor",focusVisibleRing:"inside",focusRingWidth:"2px"},startElement:{flexShrink:0,boxSize:"var(--tag-element-size)",ms:"var(--tag-element-offset)","&:has([data-scope=avatar])":{boxSize:"var(--tag-avatar-size)",ms:"calc(var(--tag-element-offset) * 1.5)"},_icon:{boxSize:"100%"}},endElement:{flexShrink:0,boxSize:"var(--tag-element-size)",me:"var(--tag-element-offset)",_icon:{boxSize:"100%"},"&:has(button)":{ms:"calc(var(--tag-element-offset) * -1)"}}},variants:{size:{sm:{root:{px:"1.5",minH:"4.5",gap:"1","--tag-avatar-size":"spacing.3","--tag-element-size":"spacing.3","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},md:{root:{px:"1.5",minH:"5",gap:"1","--tag-avatar-size":"spacing.3.5","--tag-element-size":"spacing.3.5","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},lg:{root:{px:"2",minH:"6",gap:"1.5","--tag-avatar-size":"spacing.4.5","--tag-element-size":"spacing.4","--tag-element-offset":"-3px"},label:{textStyle:"sm"}},xl:{root:{px:"2.5",minH:"8",gap:"1.5","--tag-avatar-size":"spacing.6","--tag-element-size":"spacing.4.5","--tag-element-offset":"-4px"},label:{textStyle:"sm"}}},variant:{subtle:{root:hi?.subtle},solid:{root:hi?.solid},outline:{root:hi?.outline},surface:{root:hi?.surface}}},defaultVariants:{size:"md",variant:"surface"}}),oE=B({slots:lS.keys(),className:"chakra-timeline",base:{root:{display:"flex",flexDirection:"column",width:"full","--timeline-thickness":"1px","--timeline-gutter":"4px"},item:{display:"flex",position:"relative",alignItems:"flex-start",flexShrink:0,gap:"4",_last:{"& :where(.chakra-timeline__separator)":{display:"none"}}},separator:{position:"absolute",borderStartWidth:"var(--timeline-thickness)",ms:"calc(-1 * var(--timeline-thickness) / 2)",insetInlineStart:"calc(var(--timeline-indicator-size) / 2)",insetBlock:"0",borderColor:"border"},indicator:{outline:"2px solid {colors.bg}",position:"relative",flexShrink:"0",boxSize:"var(--timeline-indicator-size)",fontSize:"var(--timeline-font-size)",display:"flex",alignItems:"center",justifyContent:"center",borderRadius:"full",fontWeight:"medium"},connector:{alignSelf:"stretch",position:"relative"},content:{pb:"6",display:"flex",flexDirection:"column",width:"full",gap:"2"},title:{display:"flex",fontWeight:"medium",flexWrap:"wrap",gap:"1.5",alignItems:"center",mt:"var(--timeline-margin)"},description:{color:"fg.muted",textStyle:"xs"}},variants:{variant:{subtle:{indicator:{bg:"colorPalette.muted"}},solid:{indicator:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},outline:{indicator:{bg:"currentBg",borderWidth:"1px",borderColor:"colorPalette.muted"}},plain:{}},size:{sm:{root:{"--timeline-indicator-size":"sizes.4","--timeline-font-size":"fontSizes.2xs"},title:{textStyle:"xs"}},md:{root:{"--timeline-indicator-size":"sizes.5","--timeline-font-size":"fontSizes.xs"},title:{textStyle:"sm"}},lg:{root:{"--timeline-indicator-size":"sizes.6","--timeline-font-size":"fontSizes.xs"},title:{mt:"0.5",textStyle:"sm"}},xl:{root:{"--timeline-indicator-size":"sizes.8","--timeline-font-size":"fontSizes.sm"},title:{mt:"1.5",textStyle:"sm"}}}},defaultVariants:{size:"md",variant:"solid"}}),iE=B({slots:iS.keys(),className:"chakra-toast",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",gap:"3",py:"4",ps:"4",pe:"6",borderRadius:"l2",translate:"var(--x) var(--y)",scale:"var(--scale)",zIndex:"var(--z-index)",height:"var(--height)",opacity:"var(--opacity)",willChange:"translate, opacity, scale",transition:"translate 400ms, scale 400ms, opacity 400ms, height 400ms, box-shadow 200ms",transitionTimingFunction:"cubic-bezier(0.21, 1.02, 0.73, 1)",_closed:{transition:"translate 400ms, scale 400ms, opacity 200ms",transitionTimingFunction:"cubic-bezier(0.06, 0.71, 0.55, 1)"},bg:"bg.panel",color:"fg",boxShadow:"xl","--toast-trigger-bg":"colors.bg.muted","&[data-type=warning]":{bg:"orange.solid",color:"orange.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=success]":{bg:"green.solid",color:"green.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=error]":{bg:"red.solid",color:"red.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"}},title:{fontWeight:"medium",textStyle:"sm",marginEnd:"2"},description:{display:"inline",textStyle:"sm",opacity:"0.8"},indicator:{flexShrink:"0",boxSize:"5"},actionTrigger:{textStyle:"sm",fontWeight:"medium",height:"8",px:"3",borderRadius:"l2",alignSelf:"center",borderWidth:"1px",borderColor:"var(--toast-border-color, inherit)",transition:"background 200ms",_hover:{bg:"var(--toast-trigger-bg)"}},closeTrigger:{position:"absolute",top:"1",insetEnd:"1",padding:"1",display:"inline-flex",alignItems:"center",justifyContent:"center",color:"{currentColor/60}",borderRadius:"l2",textStyle:"md",transition:"background 200ms",_icon:{boxSize:"1em"}}}}),sE=B({slots:Kd.keys(),className:"chakra-tooltip",base:{content:{"--tooltip-bg":"colors.bg.inverted",bg:"var(--tooltip-bg)",color:"fg.inverted",px:"2.5",py:"1",borderRadius:"l2",fontWeight:"medium",textStyle:"xs",boxShadow:"md",maxW:"xs",zIndex:"tooltip",transformOrigin:"var(--transform-origin)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"fast"}},arrow:{"--arrow-size":"sizes.2","--arrow-background":"var(--tooltip-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px",borderColor:"var(--tooltip-bg)"}}}),qh=tr({display:"flex",alignItems:"center",gap:"var(--tree-item-gap)",rounded:"l2",userSelect:"none",position:"relative","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-icon-offset":"calc(var(--tree-icon-size) * var(--tree-depth) * 0.5)","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset) + var(--tree-icon-offset))",ps:"var(--tree-offset)",pe:"var(--tree-padding-inline)",py:"var(--tree-padding-block)",focusVisibleRing:"inside",focusRingColor:"border.emphasized",focusRingWidth:"2px","&:hover, &:focus-visible":{bg:"bg.muted"},_disabled:{layerStyle:"disabled"}}),Kh=tr({flex:"1"}),Yh=tr({_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}),Xh=tr({_selected:{layerStyle:"fill.solid"}}),aE=B({slots:vd.keys(),className:"chakra-tree-view",base:{root:{width:"full",display:"flex",flexDirection:"column",gap:"2"},tree:{display:"flex",flexDirection:"column","--tree-item-gap":"spacing.2",_icon:{boxSize:"var(--tree-icon-size)"}},label:{fontWeight:"medium",textStyle:"sm"},branch:{position:"relative"},branchContent:{position:"relative"},branchIndentGuide:{height:"100%",width:"1px",bg:"border",position:"absolute","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset))","--tree-icon-offset":"calc(var(--tree-icon-size) * 0.5 * var(--depth))",insetInlineStart:"calc(var(--tree-offset) + var(--tree-icon-offset))",zIndex:"1"},branchIndicator:{color:"fg.muted",transformOrigin:"center",transitionDuration:"normal",transitionProperty:"transform",transitionTimingFunction:"default",_open:{transform:"rotate(90deg)"}},branchTrigger:{display:"inline-flex",alignItems:"center",justifyContent:"center"},branchControl:qh,item:qh,itemText:Kh,branchText:Kh,nodeCheckbox:{display:"inline-flex"}},variants:{size:{md:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1.5","--tree-icon-size":"spacing.4"}},sm:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}},xs:{tree:{textStyle:"xs","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.2","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}}},variant:{subtle:{branchControl:Yh,item:Yh},solid:{branchControl:Xh,item:Xh}},animateContent:{true:{branchContent:{_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}},defaultVariants:{size:"md",variant:"subtle"}}),lE={accordion:hS,actionBar:fS,alert:gS,avatar:pS,blockquote:mS,breadcrumb:vS,card:bS,checkbox:yS,checkboxCard:xS,codeBlock:kS,collapsible:CS,dataList:ES,dialog:OS,drawer:PS,editable:RS,emptyState:IS,field:TS,fieldset:NS,fileUpload:AS,hoverCard:_S,list:VS,listbox:FS,menu:LS,nativeSelect:DS,numberInput:zS,pinInput:jS,popover:$S,progress:BS,progressCircle:WS,radioCard:US,radioGroup:GS,ratingGroup:qS,scrollArea:KS,segmentGroup:YS,select:di,combobox:SS,slider:XS,stat:QS,steps:ZS,switch:eE,table:tE,tabs:nE,tag:rE,toast:iE,tooltip:sE,status:JS,timeline:oE,colorPicker:wS,qrCode:HS,treeView:aE},cE=Hk({"2xs":{value:{fontSize:"2xs",lineHeight:"0.75rem"}},xs:{value:{fontSize:"xs",lineHeight:"1rem"}},sm:{value:{fontSize:"sm",lineHeight:"1.25rem"}},md:{value:{fontSize:"md",lineHeight:"1.5rem"}},lg:{value:{fontSize:"lg",lineHeight:"1.75rem"}},xl:{value:{fontSize:"xl",lineHeight:"1.875rem"}},"2xl":{value:{fontSize:"2xl",lineHeight:"2rem"}},"3xl":{value:{fontSize:"3xl",lineHeight:"2.375rem"}},"4xl":{value:{fontSize:"4xl",lineHeight:"2.75rem",letterSpacing:"-0.025em"}},"5xl":{value:{fontSize:"5xl",lineHeight:"3.75rem",letterSpacing:"-0.025em"}},"6xl":{value:{fontSize:"6xl",lineHeight:"4.5rem",letterSpacing:"-0.025em"}},"7xl":{value:{fontSize:"7xl",lineHeight:"5.75rem",letterSpacing:"-0.025em"}},none:{value:{}},label:{value:{fontSize:"sm",lineHeight:"1.25rem",fontWeight:"medium"}}}),uE=de.animations({spin:{value:"spin 1s linear infinite"},ping:{value:"ping 1s cubic-bezier(0, 0, 0.2, 1) infinite"},pulse:{value:"pulse 2s cubic-bezier(0.4, 0, 0.6, 1) infinite"},bounce:{value:"bounce 1s infinite"}}),dE=de.aspectRatios({square:{value:"1 / 1"},landscape:{value:"4 / 3"},portrait:{value:"3 / 4"},wide:{value:"16 / 9"},ultrawide:{value:"18 / 5"},golden:{value:"1.618 / 1"}}),hE=de.blurs({none:{value:" "},sm:{value:"4px"},md:{value:"8px"},lg:{value:"12px"},xl:{value:"16px"},"2xl":{value:"24px"},"3xl":{value:"40px"},"4xl":{value:"64px"}}),fE=de.borders({xs:{value:"0.5px solid"},sm:{value:"1px solid"},md:{value:"2px solid"},lg:{value:"4px solid"},xl:{value:"8px solid"}}),gE=de.colors({transparent:{value:"transparent"},current:{value:"currentColor"},black:{value:"#09090B"},white:{value:"#FFFFFF"},whiteAlpha:{50:{value:"rgba(255, 255, 255, 0.04)"},100:{value:"rgba(255, 255, 255, 0.06)"},200:{value:"rgba(255, 255, 255, 0.08)"},300:{value:"rgba(255, 255, 255, 0.16)"},400:{value:"rgba(255, 255, 255, 0.24)"},500:{value:"rgba(255, 255, 255, 0.36)"},600:{value:"rgba(255, 255, 255, 0.48)"},700:{value:"rgba(255, 255, 255, 0.64)"},800:{value:"rgba(255, 255, 255, 0.80)"},900:{value:"rgba(255, 255, 255, 0.92)"},950:{value:"rgba(255, 255, 255, 0.95)"}},blackAlpha:{50:{value:"rgba(0, 0, 0, 0.04)"},100:{value:"rgba(0, 0, 0, 0.06)"},200:{value:"rgba(0, 0, 0, 0.08)"},300:{value:"rgba(0, 0, 0, 0.16)"},400:{value:"rgba(0, 0, 0, 0.24)"},500:{value:"rgba(0, 0, 0, 0.36)"},600:{value:"rgba(0, 0, 0, 0.48)"},700:{value:"rgba(0, 0, 0, 0.64)"},800:{value:"rgba(0, 0, 0, 0.80)"},900:{value:"rgba(0, 0, 0, 0.92)"},950:{value:"rgba(0, 0, 0, 0.95)"}},gray:{50:{value:"#fafafa"},100:{value:"#f4f4f5"},200:{value:"#e4e4e7"},300:{value:"#d4d4d8"},400:{value:"#a1a1aa"},500:{value:"#71717a"},600:{value:"#52525b"},700:{value:"#3f3f46"},800:{value:"#27272a"},900:{value:"#18181b"},950:{value:"#111111"}},red:{50:{value:"#fef2f2"},100:{value:"#fee2e2"},200:{value:"#fecaca"},300:{value:"#fca5a5"},400:{value:"#f87171"},500:{value:"#ef4444"},600:{value:"#dc2626"},700:{value:"#991919"},800:{value:"#511111"},900:{value:"#300c0c"},950:{value:"#1f0808"}},orange:{50:{value:"#fff7ed"},100:{value:"#ffedd5"},200:{value:"#fed7aa"},300:{value:"#fdba74"},400:{value:"#fb923c"},500:{value:"#f97316"},600:{value:"#ea580c"},700:{value:"#92310a"},800:{value:"#6c2710"},900:{value:"#3b1106"},950:{value:"#220a04"}},yellow:{50:{value:"#fefce8"},100:{value:"#fef9c3"},200:{value:"#fef08a"},300:{value:"#fde047"},400:{value:"#facc15"},500:{value:"#eab308"},600:{value:"#ca8a04"},700:{value:"#845209"},800:{value:"#713f12"},900:{value:"#422006"},950:{value:"#281304"}},green:{50:{value:"#f0fdf4"},100:{value:"#dcfce7"},200:{value:"#bbf7d0"},300:{value:"#86efac"},400:{value:"#4ade80"},500:{value:"#22c55e"},600:{value:"#16a34a"},700:{value:"#116932"},800:{value:"#124a28"},900:{value:"#042713"},950:{value:"#03190c"}},teal:{50:{value:"#f0fdfa"},100:{value:"#ccfbf1"},200:{value:"#99f6e4"},300:{value:"#5eead4"},400:{value:"#2dd4bf"},500:{value:"#14b8a6"},600:{value:"#0d9488"},700:{value:"#0c5d56"},800:{value:"#114240"},900:{value:"#032726"},950:{value:"#021716"}},blue:{50:{value:"#eff6ff"},100:{value:"#dbeafe"},200:{value:"#bfdbfe"},300:{value:"#a3cfff"},400:{value:"#60a5fa"},500:{value:"#3b82f6"},600:{value:"#2563eb"},700:{value:"#173da6"},800:{value:"#1a3478"},900:{value:"#14204a"},950:{value:"#0c142e"}},cyan:{50:{value:"#ecfeff"},100:{value:"#cffafe"},200:{value:"#a5f3fc"},300:{value:"#67e8f9"},400:{value:"#22d3ee"},500:{value:"#06b6d4"},600:{value:"#0891b2"},700:{value:"#0c5c72"},800:{value:"#134152"},900:{value:"#072a38"},950:{value:"#051b24"}},purple:{50:{value:"#faf5ff"},100:{value:"#f3e8ff"},200:{value:"#e9d5ff"},300:{value:"#d8b4fe"},400:{value:"#c084fc"},500:{value:"#a855f7"},600:{value:"#9333ea"},700:{value:"#641ba3"},800:{value:"#4a1772"},900:{value:"#2f0553"},950:{value:"#1a032e"}},pink:{50:{value:"#fdf2f8"},100:{value:"#fce7f3"},200:{value:"#fbcfe8"},300:{value:"#f9a8d4"},400:{value:"#f472b6"},500:{value:"#ec4899"},600:{value:"#db2777"},700:{value:"#a41752"},800:{value:"#6d0e34"},900:{value:"#45061f"},950:{value:"#2c0514"}}}),pE=de.cursor({button:{value:"pointer"},checkbox:{value:"default"},disabled:{value:"not-allowed"},menuitem:{value:"default"},option:{value:"default"},radio:{value:"default"},slider:{value:"default"},switch:{value:"pointer"}}),mE=de.durations({fastest:{value:"50ms"},faster:{value:"100ms"},fast:{value:"150ms"},moderate:{value:"200ms"},slow:{value:"300ms"},slower:{value:"400ms"},slowest:{value:"500ms"}}),vE=de.easings({"ease-in":{value:"cubic-bezier(0.42, 0, 1, 1)"},"ease-out":{value:"cubic-bezier(0, 0, 0.58, 1)"},"ease-in-out":{value:"cubic-bezier(0.42, 0, 0.58, 1)"},"ease-in-smooth":{value:"cubic-bezier(0.32, 0.72, 0, 1)"}}),bE=de.fontSizes({"2xs":{value:"0.625rem"},xs:{value:"0.75rem"},sm:{value:"0.875rem"},md:{value:"1rem"},lg:{value:"1.125rem"},xl:{value:"1.25rem"},"2xl":{value:"1.5rem"},"3xl":{value:"1.875rem"},"4xl":{value:"2.25rem"},"5xl":{value:"3rem"},"6xl":{value:"3.75rem"},"7xl":{value:"4.5rem"},"8xl":{value:"6rem"},"9xl":{value:"8rem"}}),yE=de.fontWeights({thin:{value:"100"},extralight:{value:"200"},light:{value:"300"},normal:{value:"400"},medium:{value:"500"},semibold:{value:"600"},bold:{value:"700"},extrabold:{value:"800"},black:{value:"900"}}),Qh='-apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol"',xE=de.fonts({heading:{value:`Inter, ${Qh}`},body:{value:`Inter, ${Qh}`},mono:{value:'SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace'}}),kE=Bk({spin:{"0%":{transform:"rotate(0deg)"},"100%":{transform:"rotate(360deg)"}},pulse:{"50%":{opacity:"0.5"}},ping:{"75%, 100%":{transform:"scale(2)",opacity:"0"}},bounce:{"0%, 100%":{transform:"translateY(-25%)",animationTimingFunction:"cubic-bezier(0.8,0,1,1)"},"50%":{transform:"none",animationTimingFunction:"cubic-bezier(0,0,0.2,1)"}},"bg-position":{from:{backgroundPosition:"var(--animate-from, 1rem) 0"},to:{backgroundPosition:"var(--animate-to, 0) 0"}},position:{from:{insetInlineStart:"var(--animate-from-x)",insetBlockStart:"var(--animate-from-y)"},to:{insetInlineStart:"var(--animate-to-x)",insetBlockStart:"var(--animate-to-y)"}},"circular-progress":{"0%":{strokeDasharray:"1, 400",strokeDashoffset:"0"},"50%":{strokeDasharray:"400, 400",strokeDashoffset:"-100%"},"100%":{strokeDasharray:"400, 400",strokeDashoffset:"-260%"}},"expand-height":{from:{height:"0"},to:{height:"var(--height)"}},"collapse-height":{from:{height:"var(--height)"},to:{height:"0"}},"expand-width":{from:{width:"0"},to:{width:"var(--width)"}},"collapse-width":{from:{height:"var(--width)"},to:{height:"0"}},"fade-in":{from:{opacity:0},to:{opacity:1}},"fade-out":{from:{opacity:1},to:{opacity:0}},"slide-from-left-full":{from:{translate:"-100% 0"},to:{translate:"0 0"}},"slide-from-right-full":{from:{translate:"100% 0"},to:{translate:"0 0"}},"slide-from-top-full":{from:{translate:"0 -100%"},to:{translate:"0 0"}},"slide-from-bottom-full":{from:{translate:"0 100%"},to:{translate:"0 0"}},"slide-to-left-full":{from:{translate:"0 0"},to:{translate:"-100% 0"}},"slide-to-right-full":{from:{translate:"0 0"},to:{translate:"100% 0"}},"slide-to-top-full":{from:{translate:"0 0"},to:{translate:"0 -100%"}},"slide-to-bottom-full":{from:{translate:"0 0"},to:{translate:"0 100%"}},"slide-from-top":{"0%":{translate:"0 -0.5rem"},to:{translate:"0"}},"slide-from-bottom":{"0%":{translate:"0 0.5rem"},to:{translate:"0"}},"slide-from-left":{"0%":{translate:"-0.5rem 0"},to:{translate:"0"}},"slide-from-right":{"0%":{translate:"0.5rem 0"},to:{translate:"0"}},"slide-to-top":{"0%":{translate:"0"},to:{translate:"0 -0.5rem"}},"slide-to-bottom":{"0%":{translate:"0"},to:{translate:"0 0.5rem"}},"slide-to-left":{"0%":{translate:"0"},to:{translate:"-0.5rem 0"}},"slide-to-right":{"0%":{translate:"0"},to:{translate:"0.5rem 0"}},"scale-in":{from:{scale:"0.95"},to:{scale:"1"}},"scale-out":{from:{scale:"1"},to:{scale:"0.95"}}}),CE=de.letterSpacings({tighter:{value:"-0.05em"},tight:{value:"-0.025em"},wide:{value:"0.025em"},wider:{value:"0.05em"},widest:{value:"0.1em"}}),wE=de.lineHeights({shorter:{value:1.25},short:{value:1.375},moderate:{value:1.5},tall:{value:1.625},taller:{value:2}}),SE=de.radii({none:{value:"0"},"2xs":{value:"0.0625rem"},xs:{value:"0.125rem"},sm:{value:"0.25rem"},md:{value:"0.375rem"},lg:{value:"0.5rem"},xl:{value:"0.75rem"},"2xl":{value:"1rem"},"3xl":{value:"1.5rem"},"4xl":{value:"2rem"},full:{value:"9999px"}}),Jh=de.spacing({.5:{value:"0.125rem"},1:{value:"0.25rem"},1.5:{value:"0.375rem"},2:{value:"0.5rem"},2.5:{value:"0.625rem"},3:{value:"0.75rem"},3.5:{value:"0.875rem"},4:{value:"1rem"},4.5:{value:"1.125rem"},5:{value:"1.25rem"},6:{value:"1.5rem"},7:{value:"1.75rem"},8:{value:"2rem"},9:{value:"2.25rem"},10:{value:"2.5rem"},11:{value:"2.75rem"},12:{value:"3rem"},14:{value:"3.5rem"},16:{value:"4rem"},20:{value:"5rem"},24:{value:"6rem"},28:{value:"7rem"},32:{value:"8rem"},36:{value:"9rem"},40:{value:"10rem"},44:{value:"11rem"},48:{value:"12rem"},52:{value:"13rem"},56:{value:"14rem"},60:{value:"15rem"},64:{value:"16rem"},72:{value:"18rem"},80:{value:"20rem"},96:{value:"24rem"}}),EE=de.sizes({"3xs":{value:"14rem"},"2xs":{value:"16rem"},xs:{value:"20rem"},sm:{value:"24rem"},md:{value:"28rem"},lg:{value:"32rem"},xl:{value:"36rem"},"2xl":{value:"42rem"},"3xl":{value:"48rem"},"4xl":{value:"56rem"},"5xl":{value:"64rem"},"6xl":{value:"72rem"},"7xl":{value:"80rem"},"8xl":{value:"90rem"}}),OE=de.sizes({max:{value:"max-content"},min:{value:"min-content"},fit:{value:"fit-content"},prose:{value:"60ch"},full:{value:"100%"},dvh:{value:"100dvh"},svh:{value:"100svh"},lvh:{value:"100lvh"},dvw:{value:"100dvw"},svw:{value:"100svw"},lvw:{value:"100lvw"},vw:{value:"100vw"},vh:{value:"100vh"}}),PE=de.sizes({"1/2":{value:"50%"},"1/3":{value:"33.333333%"},"2/3":{value:"66.666667%"},"1/4":{value:"25%"},"3/4":{value:"75%"},"1/5":{value:"20%"},"2/5":{value:"40%"},"3/5":{value:"60%"},"4/5":{value:"80%"},"1/6":{value:"16.666667%"},"2/6":{value:"33.333333%"},"3/6":{value:"50%"},"4/6":{value:"66.666667%"},"5/6":{value:"83.333333%"},"1/12":{value:"8.333333%"},"2/12":{value:"16.666667%"},"3/12":{value:"25%"},"4/12":{value:"33.333333%"},"5/12":{value:"41.666667%"},"6/12":{value:"50%"},"7/12":{value:"58.333333%"},"8/12":{value:"66.666667%"},"9/12":{value:"75%"},"10/12":{value:"83.333333%"},"11/12":{value:"91.666667%"}}),RE=de.sizes({...EE,...Jh,...PE,...OE}),IE=de.zIndex({hide:{value:-1},base:{value:0},docked:{value:10},dropdown:{value:1e3},sticky:{value:1100},banner:{value:1200},overlay:{value:1300},modal:{value:1400},popover:{value:1500},skipNav:{value:1600},toast:{value:1700},tooltip:{value:1800},max:{value:2147483647}}),TE=ka({preflight:!0,cssVarsPrefix:"chakra",cssVarsRoot:":where(html, .chakra-theme)",globalCss:aw,theme:{breakpoints:sw,keyframes:kE,tokens:{aspectRatios:dE,animations:uE,blurs:hE,borders:fE,colors:gE,durations:mE,easings:vE,fonts:xE,fontSizes:bE,fontWeights:yE,letterSpacings:CE,lineHeights:wE,radii:SE,spacing:Jh,sizes:RE,zIndex:IE,cursor:pE},semanticTokens:{colors:Pw,shadows:Iw,radii:Rw},recipes:Ow,slotRecipes:lE,textStyles:cE,layerStyles:lw,animationStyles:cw}}),Zh=Zd(eC,TE);zh(Zh);function NE(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getSlotRecipe(t):{});return r.sva(structuredClone(o))},[t,n,r])}const AE=e=>e.charAt(0).toUpperCase()+e.slice(1),fi=e=>{const{key:t,recipe:n}=e,r=AE(t||n.className||"Component"),[o,i]=Fn({name:`${r}StylesContext`,errorMessage:`use${r}Styles returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `}),[s,a]=Fn({name:`${r}ClassNameContext`,errorMessage:`use${r}ClassNames returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `,strict:!1}),[l,c]=Fn({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`,defaultValue:{}});function u(f){const{unstyled:p,...v}=f,b=NE({key:t,recipe:v.recipe||n}),[y,x]=P.useMemo(()=>b.splitVariantProps(v),[v,b]);return{styles:P.useMemo(()=>p?$m:b(y),[p,y,b]),classNames:b.classNameMap,props:x}}function h(f,p={}){const{defaultProps:v}=p,b=y=>{const x=c(),w=P.useMemo(()=>br(v,x,y),[x,y]),{styles:k,classNames:O,props:R}=u(w);return g.jsx(o,{value:k,children:g.jsx(s,{value:O,children:g.jsx(f,{...R})})})};return b.displayName=f.displayName||f.name,b}return{StylesProvider:o,ClassNamesProvider:s,PropsProvider:l,usePropsContext:c,useRecipeResult:u,withProvider:(f,p,v)=>{const{defaultProps:b,...y}=v??{},x=ve(f,{},y),w=P.forwardRef((k,O)=>{const R=c(),I=P.useMemo(()=>br(b??{},R,k),[R,k]),{styles:N,props:A,classNames:T}=u(I),S=T[p],_=g.jsx(o,{value:N,children:g.jsx(s,{value:T,children:g.jsx(x,{ref:O,...A,css:[N[p],I.css],className:Ge(I.className,S)})})});return v?.wrapElement?.(_,I)??_});return w.displayName=f.displayName||f.name,w},withContext:(f,p,v)=>{const b=ve(f,{},v),y=P.forwardRef((x,w)=>{const{unstyled:k,...O}=x,R=i(),N=a()?.[p];return g.jsx(b,{...O,css:[!k&&p?R[p]:void 0,x.css],ref:w,className:Ge(x.className,N)})});return y.displayName=f.displayName||f.name,y},withRootProvider:h,useStyles:i,useClassNames:a}},ef=ve("div",{base:{position:"absolute",display:"flex",alignItems:"center",justifyContent:"center"},variants:{axis:{horizontal:{insetStart:"50%",translate:"-50%",_rtl:{translate:"50%"}},vertical:{top:"50%",translate:"0 -50%"},both:{insetStart:"50%",top:"50%",translate:"-50% -50%",_rtl:{translate:"50% -50%"}}}},defaultVariants:{axis:"both"}});ef.displayName="AbsoluteCenter";const _E=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11.0026 16L6.75999 11.7574L8.17421 10.3431L11.0026 13.1716L16.6595 7.51472L18.0737 8.92893L11.0026 16Z"})}),tf=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 15H13V17H11V15ZM11 7H13V13H11V7Z"})}),nf=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",stroke:"currentColor",strokeWidth:"0",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 7H13V9H11V7ZM11 11H13V17H11V11Z"})}),VE=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",...e,children:g.jsx("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M18.7071 6.70711C19.0976 6.31658 19.0976 5.68342 18.7071 5.29289C18.3166 4.90237 17.6834 4.90237 17.2929 5.29289L12 10.5858L6.70711 5.29289C6.31658 4.90237 5.68342 4.90237 5.29289 5.29289C4.90237 5.68342 4.90237 6.31658 5.29289 6.70711L10.5858 12L5.29289 17.2929C4.90237 17.6834 4.90237 18.3166 5.29289 18.7071C5.68342 19.0976 6.31658 19.0976 6.70711 18.7071L12 13.4142L17.2929 18.7071C17.6834 19.0976 18.3166 19.0976 18.7071 18.7071C19.0976 18.3166 19.0976 17.6834 18.7071 17.2929L13.4142 12L18.7071 6.70711Z"})}),[FE,LE]=Fn({name:"AlertStatusContext",hookName:"useAlertStatusContext",providerName:""}),{withProvider:DE,withContext:Na,useStyles:zE}=fi({key:"alert"}),ME=DE("div","root",{forwardAsChild:!0,wrapElement(e,t){return g.jsx(FE,{value:{status:t.status||"info"},children:e})}}),rf=Na("div","title"),jE=Na("div","description"),$E=Na("div","content"),BE={info:nf,warning:tf,success:_E,error:tf,neutral:nf},WE=P.forwardRef(function(t,n){const r=LE(),o=zE(),i=typeof r.status=="string"?BE[r.status]:P.Fragment,{children:s=g.jsx(i,{}),...a}=t;return g.jsx(ve.span,{ref:n,...a,css:[o.indicator,t.css],children:s})}),HE=e=>e?"":void 0,{withContext:UE}=Lt({key:"badge"}),of=UE("span"),{withContext:GE}=Lt({key:"spinner"}),qE=GE("span"),KE=C.forwardRef(function(t,n){const{spinner:r=g.jsx(qE,{size:"inherit",borderWidth:"0.125em",color:"inherit"}),spinnerPlacement:o="start",children:i,text:s,visible:a=!0,...l}=t;return a?s?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[o==="start"&&r,s,o==="end"&&r]}):r?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[g.jsx(ef,{display:"inline-flex",children:r}),g.jsx(Vo,{visibility:"hidden",display:"contents",children:i})]}):g.jsx(Vo,{ref:n,display:"contents",...l,children:i}):i}),{useRecipeResult:YE,usePropsContext:XE}=Lt({key:"button"}),Re=P.forwardRef(function(t,n){const r=XE(),o=P.useMemo(()=>br(r,t),[r,t]),i=YE(o),{loading:s,loadingText:a,children:l,spinner:c,spinnerPlacement:u,...h}=i.props;return g.jsx(ve.button,{type:"button",ref:n,...h,"data-loading":HE(s),disabled:s||h.disabled,className:Ge(i.className,o.className),css:[i.styles,o.css],children:!o.asChild&&s?g.jsx(KE,{spinner:c,text:a,spinnerPlacement:u,children:l}):l})}),Et=P.forwardRef(function(t,n){return g.jsx(Re,{px:"0",py:"0",_icon:{fontSize:"1.2em"},ref:n,...t})}),fn=C.forwardRef(function(t,n){return g.jsx(Et,{variant:"ghost","aria-label":"Close",ref:n,...t,children:t.children??g.jsx(VE,{})})}),sf=ve("div",{base:{display:"flex",alignItems:"center",justifyContent:"center"},variants:{inline:{true:{display:"inline-flex"}}}});sf.displayName="Center";const{withContext:QE}=Lt({key:"code"}),JE=QE("code");function ZE(e){const{gap:t,direction:n}=e,r={column:{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},"column-reverse":{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},row:{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0},"row-reverse":{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0}};return{"&":Jd(n,o=>r[o])}}function e2(e){return P.Children.toArray(e).filter(t=>P.isValidElement(t))}const af=P.forwardRef(function(t,n){const{direction:r="column",align:o,justify:i,gap:s="0.5rem",wrap:a,children:l,separator:c,className:u,...h}=t,d=P.useMemo(()=>ZE({gap:s,direction:r}),[s,r]),m=P.useMemo(()=>P.isValidElement(c)?e2(l).map((f,p,v)=>{const b=typeof f.key<"u"?f.key:p,y=c,x=P.cloneElement(y,{css:[d,y.props.css]});return g.jsxs(P.Fragment,{children:[f,p===v.length-1?null:x]},b)}):l,[l,c,d]);return g.jsx(ve.div,{ref:n,display:"flex",alignItems:o,justifyContent:i,flexDirection:r,flexWrap:a,gap:c?void 0:s,className:Ge("chakra-stack",u),...h,children:m})}),{withRootProvider:lf,withContext:pt}=fi({key:"dialog"});lf(O1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});const gn=lf(E1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});pt(ad,"trigger",{forwardAsChild:!0});const pn=pt(Zu,"positioner",{forwardAsChild:!0}),mn=pt(Qu,"content",{forwardAsChild:!0});pt(Ju,"description",{forwardAsChild:!0});const vn=pt(sd,"title",{forwardAsChild:!0}),bn=pt(Xu,"closeTrigger",{forwardAsChild:!0}),yn=P.forwardRef(function(t,n){const r=Ht();return g.jsx(ve.button,{...t,ref:n,onClick:()=>r.setOpen(!1)})}),xn=pt(Yu,"backdrop",{forwardAsChild:!0}),kn=pt("div","body"),Cn=pt("div","footer"),wn=pt("div","header");function t2(e){const{each:t,fallback:n,children:r}=e;return t?.length===0?n||null:t?.map(r)}const gi=P.forwardRef(function(t,n){const{direction:r,align:o,justify:i,wrap:s,basis:a,grow:l,shrink:c,inline:u,...h}=t;return g.jsx(ve.div,{ref:n,...h,css:{display:u?"inline-flex":"flex",flexDirection:r,alignItems:o,justifyContent:i,flexWrap:s,flexBasis:a,flexGrow:l,flexShrink:c,...t.css}})}),{withContext:n2}=Lt({key:"input"}),r2=n2(cd),{withContext:o2}=Lt({key:"link"}),Br=o2("a"),{withProvider:i2,withContext:cf}=fi({key:"list"}),uf=i2("ul","root",{defaultProps:{role:"list"}}),df=cf("li","item");cf("span","indicator");const s2=P.forwardRef(function(t,n){return g.jsx(af,{align:"center",...t,direction:"row",ref:n})}),pi=P.forwardRef(function(t,n){return g.jsx(af,{align:"center",...t,direction:"column",ref:n})}),{StylesProvider:a2,ClassNamesProvider:l2,useRecipeResult:c2,withContext:Ot}=fi({key:"table"}),hf=P.forwardRef(function({native:t,...n},r){const{styles:o,props:i,classNames:s}=c2(n),a=P.useMemo(()=>t?{...o.root,"& thead":o.header,"& tbody":o.body,"& tfoot":o.footer,"& thead th":o.columnHeader,"& tr":o.row,"& td":o.cell,"& caption":o.caption}:o.root,[o,t]);return g.jsx(l2,{value:s,children:g.jsx(a2,{value:o,children:g.jsx(ve.table,{ref:r,...i,css:[a,n.css],className:Ge(s?.root,n.className)})})})}),mi=Ot("tr","row");ve("div",{base:{display:"block",whiteSpace:"nowrap",WebkitOverflowScrolling:"touch",overflow:"auto",maxWidth:"100%"}});const ff=Ot("thead","header");Ot("tfoot","footer");const ke=Ot("th","columnHeader"),Ce=Ot("td","cell");Ot("caption","caption",{defaultProps:{captionSide:"bottom"}});const gf=Ot("tbody","body");Ot("colgroup"),Ot("col");const{withContext:u2}=Lt({key:"textarea"}),pf=u2(dd);var ir=class{constructor(){this.listeners=new Set,this.subscribe=this.subscribe.bind(this)}subscribe(e){return this.listeners.add(e),this.onSubscribe(),()=>{this.listeners.delete(e),this.onUnsubscribe()}}hasListeners(){return this.listeners.size>0}onSubscribe(){}onUnsubscribe(){}},d2={setTimeout:(e,t)=>setTimeout(e,t),clearTimeout:e=>clearTimeout(e),setInterval:(e,t)=>setInterval(e,t),clearInterval:e=>clearInterval(e)},h2=class{#e=d2;#t=!1;setTimeoutProvider(e){this.#e=e}setTimeout(e,t){return this.#e.setTimeout(e,t)}clearTimeout(e){this.#e.clearTimeout(e)}setInterval(e,t){return this.#e.setInterval(e,t)}clearInterval(e){this.#e.clearInterval(e)}},Sn=new h2;function f2(e){setTimeout(e,0)}var En=typeof window>"u"||"Deno"in globalThis;function Ne(){}function g2(e,t){return typeof e=="function"?e(t):e}function Aa(e){return typeof e=="number"&&e>=0&&e!==1/0}function mf(e,t){return Math.max(e+(t||0)-Date.now(),0)}function qt(e,t){return typeof e=="function"?e(t):e}function Qe(e,t){return typeof e=="function"?e(t):e}function vf(e,t){const{type:n="all",exact:r,fetchStatus:o,predicate:i,queryKey:s,stale:a}=e;if(s){if(r){if(t.queryHash!==_a(s,t.options))return!1}else if(!Wr(t.queryKey,s))return!1}if(n!=="all"){const l=t.isActive();if(n==="active"&&!l||n==="inactive"&&l)return!1}return!(typeof a=="boolean"&&t.isStale()!==a||o&&o!==t.state.fetchStatus||i&&!i(t))}function bf(e,t){const{exact:n,status:r,predicate:o,mutationKey:i}=e;if(i){if(!t.options.mutationKey)return!1;if(n){if(On(t.options.mutationKey)!==On(i))return!1}else if(!Wr(t.options.mutationKey,i))return!1}return!(r&&t.state.status!==r||o&&!o(t))}function _a(e,t){return(t?.queryKeyHashFn||On)(e)}function On(e){return JSON.stringify(e,(t,n)=>Va(n)?Object.keys(n).sort().reduce((r,o)=>(r[o]=n[o],r),{}):n)}function Wr(e,t){return e===t?!0:typeof e!=typeof t?!1:e&&t&&typeof e=="object"&&typeof t=="object"?Object.keys(t).every(n=>Wr(e[n],t[n])):!1}var p2=Object.prototype.hasOwnProperty;function yf(e,t){if(e===t)return e;const n=xf(e)&&xf(t);if(!n&&!(Va(e)&&Va(t)))return t;const o=(n?e:Object.keys(e)).length,i=n?t:Object.keys(t),s=i.length,a=n?new Array(s):{};let l=0;for(let c=0;c{Sn.setTimeout(t,e)})}function Fa(e,t,n){return typeof n.structuralSharing=="function"?n.structuralSharing(e,t):n.structuralSharing!==!1?yf(e,t):t}function v2(e,t,n=0){const r=[...e,t];return n&&r.length>n?r.slice(1):r}function b2(e,t,n=0){const r=[t,...e];return n&&r.length>n?r.slice(0,-1):r}var La=Symbol();function Cf(e,t){return!e.queryFn&&t?.initialPromise?()=>t.initialPromise:!e.queryFn||e.queryFn===La?()=>Promise.reject(new Error(`Missing queryFn: '${e.queryHash}'`)):e.queryFn}function wf(e,t){return typeof e=="function"?e(...t):!!e}var y2=class extends ir{#e;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e();return window.addEventListener("visibilitychange",t,!1),()=>{window.removeEventListener("visibilitychange",t)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(t=>{typeof t=="boolean"?this.setFocused(t):this.onFocus()})}setFocused(e){this.#e!==e&&(this.#e=e,this.onFocus())}onFocus(){const e=this.isFocused();this.listeners.forEach(t=>{t(e)})}isFocused(){return typeof this.#e=="boolean"?this.#e:globalThis.document?.visibilityState!=="hidden"}},Da=new y2;function za(){let e,t;const n=new Promise((o,i)=>{e=o,t=i});n.status="pending",n.catch(()=>{});function r(o){Object.assign(n,o),delete n.resolve,delete n.reject}return n.resolve=o=>{r({status:"fulfilled",value:o}),e(o)},n.reject=o=>{r({status:"rejected",reason:o}),t(o)},n}var x2=f2;function k2(){let e=[],t=0,n=a=>{a()},r=a=>{a()},o=x2;const i=a=>{t?e.push(a):o(()=>{n(a)})},s=()=>{const a=e;e=[],a.length&&o(()=>{r(()=>{a.forEach(l=>{n(l)})})})};return{batch:a=>{let l;t++;try{l=a()}finally{t--,t||s()}return l},batchCalls:a=>(...l)=>{i(()=>{a(...l)})},schedule:i,setNotifyFunction:a=>{n=a},setBatchNotifyFunction:a=>{r=a},setScheduler:a=>{o=a}}}var ge=k2(),C2=class extends ir{#e=!0;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e(!0),n=()=>e(!1);return window.addEventListener("online",t,!1),window.addEventListener("offline",n,!1),()=>{window.removeEventListener("online",t),window.removeEventListener("offline",n)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(this.setOnline.bind(this))}setOnline(e){this.#e!==e&&(this.#e=e,this.listeners.forEach(n=>{n(e)}))}isOnline(){return this.#e}},bi=new C2;function w2(e){return Math.min(1e3*2**e,3e4)}function Sf(e){return(e??"online")==="online"?bi.isOnline():!0}var Ma=class extends Error{constructor(e){super("CancelledError"),this.revert=e?.revert,this.silent=e?.silent}};function Ef(e){let t=!1,n=0,r;const o=za(),i=()=>o.status!=="pending",s=p=>{if(!i()){const v=new Ma(p);d(v),e.onCancel?.(v)}},a=()=>{t=!0},l=()=>{t=!1},c=()=>Da.isFocused()&&(e.networkMode==="always"||bi.isOnline())&&e.canRun(),u=()=>Sf(e.networkMode)&&e.canRun(),h=p=>{i()||(r?.(),o.resolve(p))},d=p=>{i()||(r?.(),o.reject(p))},m=()=>new Promise(p=>{r=v=>{(i()||c())&&p(v)},e.onPause?.()}).then(()=>{r=void 0,i()||e.onContinue?.()}),f=()=>{if(i())return;let p;const v=n===0?e.initialPromise:void 0;try{p=v??e.fn()}catch(b){p=Promise.reject(b)}Promise.resolve(p).then(h).catch(b=>{if(i())return;const y=e.retry??(En?0:3),x=e.retryDelay??w2,w=typeof x=="function"?x(n,b):x,k=y===!0||typeof y=="number"&&nc()?void 0:m()).then(()=>{t?d(b):f()})})};return{promise:o,status:()=>o.status,cancel:s,continue:()=>(r?.(),o),cancelRetry:a,continueRetry:l,canStart:u,start:()=>(u()?f():m().then(f),o)}}var Of=class{#e;destroy(){this.clearGcTimeout()}scheduleGc(){this.clearGcTimeout(),Aa(this.gcTime)&&(this.#e=Sn.setTimeout(()=>{this.optionalRemove()},this.gcTime))}updateGcTime(e){this.gcTime=Math.max(this.gcTime||0,e??(En?1/0:300*1e3))}clearGcTimeout(){this.#e&&(Sn.clearTimeout(this.#e),this.#e=void 0)}},S2=class extends Of{#e;#t;#n;#o;#r;#s;#a;constructor(e){super(),this.#a=!1,this.#s=e.defaultOptions,this.setOptions(e.options),this.observers=[],this.#o=e.client,this.#n=this.#o.getQueryCache(),this.queryKey=e.queryKey,this.queryHash=e.queryHash,this.#e=Rf(this.options),this.state=e.state??this.#e,this.scheduleGc()}get meta(){return this.options.meta}get promise(){return this.#r?.promise}setOptions(e){if(this.options={...this.#s,...e},this.updateGcTime(this.options.gcTime),this.state&&this.state.data===void 0){const t=Rf(this.options);t.data!==void 0&&(this.setData(t.data,{updatedAt:t.dataUpdatedAt,manual:!0}),this.#e=t)}}optionalRemove(){!this.observers.length&&this.state.fetchStatus==="idle"&&this.#n.remove(this)}setData(e,t){const n=Fa(this.state.data,e,this.options);return this.#i({data:n,type:"success",dataUpdatedAt:t?.updatedAt,manual:t?.manual}),n}setState(e,t){this.#i({type:"setState",state:e,setStateOptions:t})}cancel(e){const t=this.#r?.promise;return this.#r?.cancel(e),t?t.then(Ne).catch(Ne):Promise.resolve()}destroy(){super.destroy(),this.cancel({silent:!0})}reset(){this.destroy(),this.setState(this.#e)}isActive(){return this.observers.some(e=>Qe(e.options.enabled,this)!==!1)}isDisabled(){return this.getObserversCount()>0?!this.isActive():this.options.queryFn===La||this.state.dataUpdateCount+this.state.errorUpdateCount===0}isStatic(){return this.getObserversCount()>0?this.observers.some(e=>qt(e.options.staleTime,this)==="static"):!1}isStale(){return this.getObserversCount()>0?this.observers.some(e=>e.getCurrentResult().isStale):this.state.data===void 0||this.state.isInvalidated}isStaleByTime(e=0){return this.state.data===void 0?!0:e==="static"?!1:this.state.isInvalidated?!0:!mf(this.state.dataUpdatedAt,e)}onFocus(){this.observers.find(t=>t.shouldFetchOnWindowFocus())?.refetch({cancelRefetch:!1}),this.#r?.continue()}onOnline(){this.observers.find(t=>t.shouldFetchOnReconnect())?.refetch({cancelRefetch:!1}),this.#r?.continue()}addObserver(e){this.observers.includes(e)||(this.observers.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",query:this,observer:e}))}removeObserver(e){this.observers.includes(e)&&(this.observers=this.observers.filter(t=>t!==e),this.observers.length||(this.#r&&(this.#a?this.#r.cancel({revert:!0}):this.#r.cancelRetry()),this.scheduleGc()),this.#n.notify({type:"observerRemoved",query:this,observer:e}))}getObserversCount(){return this.observers.length}invalidate(){this.state.isInvalidated||this.#i({type:"invalidate"})}async fetch(e,t){if(this.state.fetchStatus!=="idle"&&this.#r?.status()!=="rejected"){if(this.state.data!==void 0&&t?.cancelRefetch)this.cancel({silent:!0});else if(this.#r)return this.#r.continueRetry(),this.#r.promise}if(e&&this.setOptions(e),!this.options.queryFn){const a=this.observers.find(l=>l.options.queryFn);a&&this.setOptions(a.options)}const n=new AbortController,r=a=>{Object.defineProperty(a,"signal",{enumerable:!0,get:()=>(this.#a=!0,n.signal)})},o=()=>{const a=Cf(this.options,t),c=(()=>{const u={client:this.#o,queryKey:this.queryKey,meta:this.meta};return r(u),u})();return this.#a=!1,this.options.persister?this.options.persister(a,c,this):a(c)},s=(()=>{const a={fetchOptions:t,options:this.options,queryKey:this.queryKey,client:this.#o,state:this.state,fetchFn:o};return r(a),a})();this.options.behavior?.onFetch(s,this),this.#t=this.state,(this.state.fetchStatus==="idle"||this.state.fetchMeta!==s.fetchOptions?.meta)&&this.#i({type:"fetch",meta:s.fetchOptions?.meta}),this.#r=Ef({initialPromise:t?.initialPromise,fn:s.fetchFn,onCancel:a=>{a instanceof Ma&&a.revert&&this.setState({...this.#t,fetchStatus:"idle"}),n.abort()},onFail:(a,l)=>{this.#i({type:"failed",failureCount:a,error:l})},onPause:()=>{this.#i({type:"pause"})},onContinue:()=>{this.#i({type:"continue"})},retry:s.options.retry,retryDelay:s.options.retryDelay,networkMode:s.options.networkMode,canRun:()=>!0});try{const a=await this.#r.start();if(a===void 0)throw new Error(`${this.queryHash} data is undefined`);return this.setData(a),this.#n.config.onSuccess?.(a,this),this.#n.config.onSettled?.(a,this.state.error,this),a}catch(a){if(a instanceof Ma){if(a.silent)return this.#r.promise;if(a.revert){if(this.state.data===void 0)throw a;return this.state.data}}throw this.#i({type:"error",error:a}),this.#n.config.onError?.(a,this),this.#n.config.onSettled?.(this.state.data,a,this),a}finally{this.scheduleGc()}}#i(e){const t=n=>{switch(e.type){case"failed":return{...n,fetchFailureCount:e.failureCount,fetchFailureReason:e.error};case"pause":return{...n,fetchStatus:"paused"};case"continue":return{...n,fetchStatus:"fetching"};case"fetch":return{...n,...Pf(n.data,this.options),fetchMeta:e.meta??null};case"success":const r={...n,data:e.data,dataUpdateCount:n.dataUpdateCount+1,dataUpdatedAt:e.dataUpdatedAt??Date.now(),error:null,isInvalidated:!1,status:"success",...!e.manual&&{fetchStatus:"idle",fetchFailureCount:0,fetchFailureReason:null}};return this.#t=e.manual?r:void 0,r;case"error":const o=e.error;return{...n,error:o,errorUpdateCount:n.errorUpdateCount+1,errorUpdatedAt:Date.now(),fetchFailureCount:n.fetchFailureCount+1,fetchFailureReason:o,fetchStatus:"idle",status:"error"};case"invalidate":return{...n,isInvalidated:!0};case"setState":return{...n,...e.state}}};this.state=t(this.state),ge.batch(()=>{this.observers.forEach(n=>{n.onQueryUpdate()}),this.#n.notify({query:this,type:"updated",action:e})})}};function Pf(e,t){return{fetchFailureCount:0,fetchFailureReason:null,fetchStatus:Sf(t.networkMode)?"fetching":"paused",...e===void 0&&{error:null,status:"pending"}}}function Rf(e){const t=typeof e.initialData=="function"?e.initialData():e.initialData,n=t!==void 0,r=n?typeof e.initialDataUpdatedAt=="function"?e.initialDataUpdatedAt():e.initialDataUpdatedAt:0;return{data:t,dataUpdateCount:0,dataUpdatedAt:n?r??Date.now():0,error:null,errorUpdateCount:0,errorUpdatedAt:0,fetchFailureCount:0,fetchFailureReason:null,fetchMeta:null,isInvalidated:!1,status:n?"success":"pending",fetchStatus:"idle"}}var E2=class extends ir{constructor(e,t){super(),this.options=t,this.#e=e,this.#i=null,this.#a=za(),this.bindMethods(),this.setOptions(t)}#e;#t=void 0;#n=void 0;#o=void 0;#r;#s;#a;#i;#p;#h;#f;#c;#u;#l;#g=new Set;bindMethods(){this.refetch=this.refetch.bind(this)}onSubscribe(){this.listeners.size===1&&(this.#t.addObserver(this),If(this.#t,this.options)?this.#d():this.updateResult(),this.#y())}onUnsubscribe(){this.hasListeners()||this.destroy()}shouldFetchOnReconnect(){return ja(this.#t,this.options,this.options.refetchOnReconnect)}shouldFetchOnWindowFocus(){return ja(this.#t,this.options,this.options.refetchOnWindowFocus)}destroy(){this.listeners=new Set,this.#x(),this.#k(),this.#t.removeObserver(this)}setOptions(e){const t=this.options,n=this.#t;if(this.options=this.#e.defaultQueryOptions(e),this.options.enabled!==void 0&&typeof this.options.enabled!="boolean"&&typeof this.options.enabled!="function"&&typeof Qe(this.options.enabled,this.#t)!="boolean")throw new Error("Expected enabled to be a boolean or a callback that returns a boolean");this.#C(),this.#t.setOptions(this.options),t._defaulted&&!vi(this.options,t)&&this.#e.getQueryCache().notify({type:"observerOptionsUpdated",query:this.#t,observer:this});const r=this.hasListeners();r&&Tf(this.#t,n,this.options,t)&&this.#d(),this.updateResult(),r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||qt(this.options.staleTime,this.#t)!==qt(t.staleTime,this.#t))&&this.#m();const o=this.#v();r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||o!==this.#l)&&this.#b(o)}getOptimisticResult(e){const t=this.#e.getQueryCache().build(this.#e,e),n=this.createResult(t,e);return P2(this,n)&&(this.#o=n,this.#s=this.options,this.#r=this.#t.state),n}getCurrentResult(){return this.#o}trackResult(e,t){return new Proxy(e,{get:(n,r)=>(this.trackProp(r),t?.(r),r==="promise"&&!this.options.experimental_prefetchInRender&&this.#a.status==="pending"&&this.#a.reject(new Error("experimental_prefetchInRender feature flag is not enabled")),Reflect.get(n,r))})}trackProp(e){this.#g.add(e)}getCurrentQuery(){return this.#t}refetch({...e}={}){return this.fetch({...e})}fetchOptimistic(e){const t=this.#e.defaultQueryOptions(e),n=this.#e.getQueryCache().build(this.#e,t);return n.fetch().then(()=>this.createResult(n,t))}fetch(e){return this.#d({...e,cancelRefetch:e.cancelRefetch??!0}).then(()=>(this.updateResult(),this.#o))}#d(e){this.#C();let t=this.#t.fetch(this.options,e);return e?.throwOnError||(t=t.catch(Ne)),t}#m(){this.#x();const e=qt(this.options.staleTime,this.#t);if(En||this.#o.isStale||!Aa(e))return;const n=mf(this.#o.dataUpdatedAt,e)+1;this.#c=Sn.setTimeout(()=>{this.#o.isStale||this.updateResult()},n)}#v(){return(typeof this.options.refetchInterval=="function"?this.options.refetchInterval(this.#t):this.options.refetchInterval)??!1}#b(e){this.#k(),this.#l=e,!(En||Qe(this.options.enabled,this.#t)===!1||!Aa(this.#l)||this.#l===0)&&(this.#u=Sn.setInterval(()=>{(this.options.refetchIntervalInBackground||Da.isFocused())&&this.#d()},this.#l))}#y(){this.#m(),this.#b(this.#v())}#x(){this.#c&&(Sn.clearTimeout(this.#c),this.#c=void 0)}#k(){this.#u&&(Sn.clearInterval(this.#u),this.#u=void 0)}createResult(e,t){const n=this.#t,r=this.options,o=this.#o,i=this.#r,s=this.#s,l=e!==n?e.state:this.#n,{state:c}=e;let u={...c},h=!1,d;if(t._optimisticResults){const I=this.hasListeners(),N=!I&&If(e,t),A=I&&Tf(e,n,t,r);(N||A)&&(u={...u,...Pf(c.data,e.options)}),t._optimisticResults==="isRestoring"&&(u.fetchStatus="idle")}let{error:m,errorUpdatedAt:f,status:p}=u;d=u.data;let v=!1;if(t.placeholderData!==void 0&&d===void 0&&p==="pending"){let I;o?.isPlaceholderData&&t.placeholderData===s?.placeholderData?(I=o.data,v=!0):I=typeof t.placeholderData=="function"?t.placeholderData(this.#f?.state.data,this.#f):t.placeholderData,I!==void 0&&(p="success",d=Fa(o?.data,I,t),h=!0)}if(t.select&&d!==void 0&&!v)if(o&&d===i?.data&&t.select===this.#p)d=this.#h;else try{this.#p=t.select,d=t.select(d),d=Fa(o?.data,d,t),this.#h=d,this.#i=null}catch(I){this.#i=I}this.#i&&(m=this.#i,d=this.#h,f=Date.now(),p="error");const b=u.fetchStatus==="fetching",y=p==="pending",x=p==="error",w=y&&b,k=d!==void 0,R={status:p,fetchStatus:u.fetchStatus,isPending:y,isSuccess:p==="success",isError:x,isInitialLoading:w,isLoading:w,data:d,dataUpdatedAt:u.dataUpdatedAt,error:m,errorUpdatedAt:f,failureCount:u.fetchFailureCount,failureReason:u.fetchFailureReason,errorUpdateCount:u.errorUpdateCount,isFetched:u.dataUpdateCount>0||u.errorUpdateCount>0,isFetchedAfterMount:u.dataUpdateCount>l.dataUpdateCount||u.errorUpdateCount>l.errorUpdateCount,isFetching:b,isRefetching:b&&!y,isLoadingError:x&&!k,isPaused:u.fetchStatus==="paused",isPlaceholderData:h,isRefetchError:x&&k,isStale:$a(e,t),refetch:this.refetch,promise:this.#a,isEnabled:Qe(t.enabled,e)!==!1};if(this.options.experimental_prefetchInRender){const I=T=>{R.status==="error"?T.reject(R.error):R.data!==void 0&&T.resolve(R.data)},N=()=>{const T=this.#a=R.promise=za();I(T)},A=this.#a;switch(A.status){case"pending":e.queryHash===n.queryHash&&I(A);break;case"fulfilled":(R.status==="error"||R.data!==A.value)&&N();break;case"rejected":(R.status!=="error"||R.error!==A.reason)&&N();break}}return R}updateResult(){const e=this.#o,t=this.createResult(this.#t,this.options);if(this.#r=this.#t.state,this.#s=this.options,this.#r.data!==void 0&&(this.#f=this.#t),vi(t,e))return;this.#o=t;const n=()=>{if(!e)return!0;const{notifyOnChangeProps:r}=this.options,o=typeof r=="function"?r():r;if(o==="all"||!o&&!this.#g.size)return!0;const i=new Set(o??this.#g);return this.options.throwOnError&&i.add("error"),Object.keys(this.#o).some(s=>{const a=s;return this.#o[a]!==e[a]&&i.has(a)})};this.#w({listeners:n()})}#C(){const e=this.#e.getQueryCache().build(this.#e,this.options);if(e===this.#t)return;const t=this.#t;this.#t=e,this.#n=e.state,this.hasListeners()&&(t?.removeObserver(this),e.addObserver(this))}onQueryUpdate(){this.updateResult(),this.hasListeners()&&this.#y()}#w(e){ge.batch(()=>{e.listeners&&this.listeners.forEach(t=>{t(this.#o)}),this.#e.getQueryCache().notify({query:this.#t,type:"observerResultsUpdated"})})}};function O2(e,t){return Qe(t.enabled,e)!==!1&&e.state.data===void 0&&!(e.state.status==="error"&&t.retryOnMount===!1)}function If(e,t){return O2(e,t)||e.state.data!==void 0&&ja(e,t,t.refetchOnMount)}function ja(e,t,n){if(Qe(t.enabled,e)!==!1&&qt(t.staleTime,e)!=="static"){const r=typeof n=="function"?n(e):n;return r==="always"||r!==!1&&$a(e,t)}return!1}function Tf(e,t,n,r){return(e!==t||Qe(r.enabled,e)===!1)&&(!n.suspense||e.state.status!=="error")&&$a(e,n)}function $a(e,t){return Qe(t.enabled,e)!==!1&&e.isStaleByTime(qt(t.staleTime,e))}function P2(e,t){return!vi(e.getCurrentResult(),t)}function Nf(e){return{onFetch:(t,n)=>{const r=t.options,o=t.fetchOptions?.meta?.fetchMore?.direction,i=t.state.data?.pages||[],s=t.state.data?.pageParams||[];let a={pages:[],pageParams:[]},l=0;const c=async()=>{let u=!1;const h=f=>{Object.defineProperty(f,"signal",{enumerable:!0,get:()=>(t.signal.aborted?u=!0:t.signal.addEventListener("abort",()=>{u=!0}),t.signal)})},d=Cf(t.options,t.fetchOptions),m=async(f,p,v)=>{if(u)return Promise.reject();if(p==null&&f.pages.length)return Promise.resolve(f);const y=(()=>{const O={client:t.client,queryKey:t.queryKey,pageParam:p,direction:v?"backward":"forward",meta:t.options.meta};return h(O),O})(),x=await d(y),{maxPages:w}=t.options,k=v?b2:v2;return{pages:k(f.pages,x,w),pageParams:k(f.pageParams,p,w)}};if(o&&i.length){const f=o==="backward",p=f?R2:Af,v={pages:i,pageParams:s},b=p(r,v);a=await m(v,b,f)}else{const f=e??i.length;do{const p=l===0?s[0]??r.initialPageParam:Af(r,a);if(l>0&&p==null)break;a=await m(a,p),l++}while(lt.options.persister?.(c,{client:t.client,queryKey:t.queryKey,meta:t.options.meta,signal:t.signal},n):t.fetchFn=c}}}function Af(e,{pages:t,pageParams:n}){const r=t.length-1;return t.length>0?e.getNextPageParam(t[r],t,n[r],n):void 0}function R2(e,{pages:t,pageParams:n}){return t.length>0?e.getPreviousPageParam?.(t[0],t,n[0],n):void 0}var I2=class extends Of{#e;#t;#n;#o;constructor(e){super(),this.#e=e.client,this.mutationId=e.mutationId,this.#n=e.mutationCache,this.#t=[],this.state=e.state||_f(),this.setOptions(e.options),this.scheduleGc()}setOptions(e){this.options=e,this.updateGcTime(this.options.gcTime)}get meta(){return this.options.meta}addObserver(e){this.#t.includes(e)||(this.#t.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",mutation:this,observer:e}))}removeObserver(e){this.#t=this.#t.filter(t=>t!==e),this.scheduleGc(),this.#n.notify({type:"observerRemoved",mutation:this,observer:e})}optionalRemove(){this.#t.length||(this.state.status==="pending"?this.scheduleGc():this.#n.remove(this))}continue(){return this.#o?.continue()??this.execute(this.state.variables)}async execute(e){const t=()=>{this.#r({type:"continue"})},n={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};this.#o=Ef({fn:()=>this.options.mutationFn?this.options.mutationFn(e,n):Promise.reject(new Error("No mutationFn found")),onFail:(i,s)=>{this.#r({type:"failed",failureCount:i,error:s})},onPause:()=>{this.#r({type:"pause"})},onContinue:t,retry:this.options.retry??0,retryDelay:this.options.retryDelay,networkMode:this.options.networkMode,canRun:()=>this.#n.canRun(this)});const r=this.state.status==="pending",o=!this.#o.canStart();try{if(r)t();else{this.#r({type:"pending",variables:e,isPaused:o}),await this.#n.config.onMutate?.(e,this,n);const s=await this.options.onMutate?.(e,n);s!==this.state.context&&this.#r({type:"pending",context:s,variables:e,isPaused:o})}const i=await this.#o.start();return await this.#n.config.onSuccess?.(i,e,this.state.context,this,n),await this.options.onSuccess?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(i,null,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(i,null,e,this.state.context,n),this.#r({type:"success",data:i}),i}catch(i){try{throw await this.#n.config.onError?.(i,e,this.state.context,this,n),await this.options.onError?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(void 0,i,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(void 0,i,e,this.state.context,n),i}finally{this.#r({type:"error",error:i})}}finally{this.#n.runNext(this)}}#r(e){const t=n=>{switch(e.type){case"failed":return{...n,failureCount:e.failureCount,failureReason:e.error};case"pause":return{...n,isPaused:!0};case"continue":return{...n,isPaused:!1};case"pending":return{...n,context:e.context,data:void 0,failureCount:0,failureReason:null,error:null,isPaused:e.isPaused,status:"pending",variables:e.variables,submittedAt:Date.now()};case"success":return{...n,data:e.data,failureCount:0,failureReason:null,error:null,status:"success",isPaused:!1};case"error":return{...n,data:void 0,error:e.error,failureCount:n.failureCount+1,failureReason:e.error,isPaused:!1,status:"error"}}};this.state=t(this.state),ge.batch(()=>{this.#t.forEach(n=>{n.onMutationUpdate(e)}),this.#n.notify({mutation:this,type:"updated",action:e})})}};function _f(){return{context:void 0,data:void 0,error:null,failureCount:0,failureReason:null,isPaused:!1,status:"idle",variables:void 0,submittedAt:0}}var T2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Set,this.#t=new Map,this.#n=0}#e;#t;#n;build(e,t,n){const r=new I2({client:e,mutationCache:this,mutationId:++this.#n,options:e.defaultMutationOptions(t),state:n});return this.add(r),r}add(e){this.#e.add(e);const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);n?n.push(e):this.#t.set(t,[e])}this.notify({type:"added",mutation:e})}remove(e){if(this.#e.delete(e)){const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);if(n)if(n.length>1){const r=n.indexOf(e);r!==-1&&n.splice(r,1)}else n[0]===e&&this.#t.delete(t)}}this.notify({type:"removed",mutation:e})}canRun(e){const t=yi(e);if(typeof t=="string"){const r=this.#t.get(t)?.find(o=>o.state.status==="pending");return!r||r===e}else return!0}runNext(e){const t=yi(e);return typeof t=="string"?this.#t.get(t)?.find(r=>r!==e&&r.state.isPaused)?.continue()??Promise.resolve():Promise.resolve()}clear(){ge.batch(()=>{this.#e.forEach(e=>{this.notify({type:"removed",mutation:e})}),this.#e.clear(),this.#t.clear()})}getAll(){return Array.from(this.#e)}find(e){const t={exact:!0,...e};return this.getAll().find(n=>bf(t,n))}findAll(e={}){return this.getAll().filter(t=>bf(e,t))}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}resumePausedMutations(){const e=this.getAll().filter(t=>t.state.isPaused);return ge.batch(()=>Promise.all(e.map(t=>t.continue().catch(Ne))))}};function yi(e){return e.options.scope?.id}var N2=class extends ir{#e;#t=void 0;#n;#o;constructor(e,t){super(),this.#e=e,this.setOptions(t),this.bindMethods(),this.#r()}bindMethods(){this.mutate=this.mutate.bind(this),this.reset=this.reset.bind(this)}setOptions(e){const t=this.options;this.options=this.#e.defaultMutationOptions(e),vi(this.options,t)||this.#e.getMutationCache().notify({type:"observerOptionsUpdated",mutation:this.#n,observer:this}),t?.mutationKey&&this.options.mutationKey&&On(t.mutationKey)!==On(this.options.mutationKey)?this.reset():this.#n?.state.status==="pending"&&this.#n.setOptions(this.options)}onUnsubscribe(){this.hasListeners()||this.#n?.removeObserver(this)}onMutationUpdate(e){this.#r(),this.#s(e)}getCurrentResult(){return this.#t}reset(){this.#n?.removeObserver(this),this.#n=void 0,this.#r(),this.#s()}mutate(e,t){return this.#o=t,this.#n?.removeObserver(this),this.#n=this.#e.getMutationCache().build(this.#e,this.options),this.#n.addObserver(this),this.#n.execute(e)}#r(){const e=this.#n?.state??_f();this.#t={...e,isPending:e.status==="pending",isSuccess:e.status==="success",isError:e.status==="error",isIdle:e.status==="idle",mutate:this.mutate,reset:this.reset}}#s(e){ge.batch(()=>{if(this.#o&&this.hasListeners()){const t=this.#t.variables,n=this.#t.context,r={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};e?.type==="success"?(this.#o.onSuccess?.(e.data,t,n,r),this.#o.onSettled?.(e.data,null,t,n,r)):e?.type==="error"&&(this.#o.onError?.(e.error,t,n,r),this.#o.onSettled?.(void 0,e.error,t,n,r))}this.listeners.forEach(t=>{t(this.#t)})})}},A2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Map}#e;build(e,t,n){const r=t.queryKey,o=t.queryHash??_a(r,t);let i=this.get(o);return i||(i=new S2({client:e,queryKey:r,queryHash:o,options:e.defaultQueryOptions(t),state:n,defaultOptions:e.getQueryDefaults(r)}),this.add(i)),i}add(e){this.#e.has(e.queryHash)||(this.#e.set(e.queryHash,e),this.notify({type:"added",query:e}))}remove(e){const t=this.#e.get(e.queryHash);t&&(e.destroy(),t===e&&this.#e.delete(e.queryHash),this.notify({type:"removed",query:e}))}clear(){ge.batch(()=>{this.getAll().forEach(e=>{this.remove(e)})})}get(e){return this.#e.get(e)}getAll(){return[...this.#e.values()]}find(e){const t={exact:!0,...e};return this.getAll().find(n=>vf(t,n))}findAll(e={}){const t=this.getAll();return Object.keys(e).length>0?t.filter(n=>vf(e,n)):t}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}onFocus(){ge.batch(()=>{this.getAll().forEach(e=>{e.onFocus()})})}onOnline(){ge.batch(()=>{this.getAll().forEach(e=>{e.onOnline()})})}},_2=class{#e;#t;#n;#o;#r;#s;#a;#i;constructor(e={}){this.#e=e.queryCache||new A2,this.#t=e.mutationCache||new T2,this.#n=e.defaultOptions||{},this.#o=new Map,this.#r=new Map,this.#s=0}mount(){this.#s++,this.#s===1&&(this.#a=Da.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onFocus())}),this.#i=bi.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onOnline())}))}unmount(){this.#s--,this.#s===0&&(this.#a?.(),this.#a=void 0,this.#i?.(),this.#i=void 0)}isFetching(e){return this.#e.findAll({...e,fetchStatus:"fetching"}).length}isMutating(e){return this.#t.findAll({...e,status:"pending"}).length}getQueryData(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state.data}ensureQueryData(e){const t=this.defaultQueryOptions(e),n=this.#e.build(this,t),r=n.state.data;return r===void 0?this.fetchQuery(e):(e.revalidateIfStale&&n.isStaleByTime(qt(t.staleTime,n))&&this.prefetchQuery(t),Promise.resolve(r))}getQueriesData(e){return this.#e.findAll(e).map(({queryKey:t,state:n})=>{const r=n.data;return[t,r]})}setQueryData(e,t,n){const r=this.defaultQueryOptions({queryKey:e}),i=this.#e.get(r.queryHash)?.state.data,s=g2(t,i);if(s!==void 0)return this.#e.build(this,r).setData(s,{...n,manual:!0})}setQueriesData(e,t,n){return ge.batch(()=>this.#e.findAll(e).map(({queryKey:r})=>[r,this.setQueryData(r,t,n)]))}getQueryState(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state}removeQueries(e){const t=this.#e;ge.batch(()=>{t.findAll(e).forEach(n=>{t.remove(n)})})}resetQueries(e,t){const n=this.#e;return ge.batch(()=>(n.findAll(e).forEach(r=>{r.reset()}),this.refetchQueries({type:"active",...e},t)))}cancelQueries(e,t={}){const n={revert:!0,...t},r=ge.batch(()=>this.#e.findAll(e).map(o=>o.cancel(n)));return Promise.all(r).then(Ne).catch(Ne)}invalidateQueries(e,t={}){return ge.batch(()=>(this.#e.findAll(e).forEach(n=>{n.invalidate()}),e?.refetchType==="none"?Promise.resolve():this.refetchQueries({...e,type:e?.refetchType??e?.type??"active"},t)))}refetchQueries(e,t={}){const n={...t,cancelRefetch:t.cancelRefetch??!0},r=ge.batch(()=>this.#e.findAll(e).filter(o=>!o.isDisabled()&&!o.isStatic()).map(o=>{let i=o.fetch(void 0,n);return n.throwOnError||(i=i.catch(Ne)),o.state.fetchStatus==="paused"?Promise.resolve():i}));return Promise.all(r).then(Ne)}fetchQuery(e){const t=this.defaultQueryOptions(e);t.retry===void 0&&(t.retry=!1);const n=this.#e.build(this,t);return n.isStaleByTime(qt(t.staleTime,n))?n.fetch(t):Promise.resolve(n.state.data)}prefetchQuery(e){return this.fetchQuery(e).then(Ne).catch(Ne)}fetchInfiniteQuery(e){return e.behavior=Nf(e.pages),this.fetchQuery(e)}prefetchInfiniteQuery(e){return this.fetchInfiniteQuery(e).then(Ne).catch(Ne)}ensureInfiniteQueryData(e){return e.behavior=Nf(e.pages),this.ensureQueryData(e)}resumePausedMutations(){return bi.isOnline()?this.#t.resumePausedMutations():Promise.resolve()}getQueryCache(){return this.#e}getMutationCache(){return this.#t}getDefaultOptions(){return this.#n}setDefaultOptions(e){this.#n=e}setQueryDefaults(e,t){this.#o.set(On(e),{queryKey:e,defaultOptions:t})}getQueryDefaults(e){const t=[...this.#o.values()],n={};return t.forEach(r=>{Wr(e,r.queryKey)&&Object.assign(n,r.defaultOptions)}),n}setMutationDefaults(e,t){this.#r.set(On(e),{mutationKey:e,defaultOptions:t})}getMutationDefaults(e){const t=[...this.#r.values()],n={};return t.forEach(r=>{Wr(e,r.mutationKey)&&Object.assign(n,r.defaultOptions)}),n}defaultQueryOptions(e){if(e._defaulted)return e;const t={...this.#n.queries,...this.getQueryDefaults(e.queryKey),...e,_defaulted:!0};return t.queryHash||(t.queryHash=_a(t.queryKey,t)),t.refetchOnReconnect===void 0&&(t.refetchOnReconnect=t.networkMode!=="always"),t.throwOnError===void 0&&(t.throwOnError=!!t.suspense),!t.networkMode&&t.persister&&(t.networkMode="offlineFirst"),t.queryFn===La&&(t.enabled=!1),t}defaultMutationOptions(e){return e?._defaulted?e:{...this.#n.mutations,...e?.mutationKey&&this.getMutationDefaults(e.mutationKey),...e,_defaulted:!0}}clear(){this.#e.clear(),this.#t.clear()}},Vf=C.createContext(void 0),Ff=e=>{const t=C.useContext(Vf);if(!t)throw new Error("No QueryClient set, use QueryClientProvider to set one");return t},V2=({client:e,children:t})=>(C.useEffect(()=>(e.mount(),()=>{e.unmount()}),[e]),g.jsx(Vf.Provider,{value:e,children:t})),Lf=C.createContext(!1),F2=()=>C.useContext(Lf);Lf.Provider;function L2(){let e=!1;return{clearReset:()=>{e=!1},reset:()=>{e=!0},isReset:()=>e}}var D2=C.createContext(L2()),z2=()=>C.useContext(D2),M2=(e,t)=>{(e.suspense||e.throwOnError||e.experimental_prefetchInRender)&&(t.isReset()||(e.retryOnMount=!1))},j2=e=>{C.useEffect(()=>{e.clearReset()},[e])},$2=({result:e,errorResetBoundary:t,throwOnError:n,query:r,suspense:o})=>e.isError&&!t.isReset()&&!e.isFetching&&r&&(o&&e.data===void 0||wf(n,[e.error,r])),B2=e=>{if(e.suspense){const n=o=>o==="static"?o:Math.max(o??1e3,1e3),r=e.staleTime;e.staleTime=typeof r=="function"?(...o)=>n(r(...o)):n(r),typeof e.gcTime=="number"&&(e.gcTime=Math.max(e.gcTime,1e3))}},W2=(e,t)=>e.isLoading&&e.isFetching&&!t,H2=(e,t)=>e?.suspense&&t.isPending,Df=(e,t,n)=>t.fetchOptimistic(e).catch(()=>{n.clearReset()});function U2(e,t,n){const r=F2(),o=z2(),i=Ff(),s=i.defaultQueryOptions(e);i.getDefaultOptions().queries?._experimental_beforeQuery?.(s),s._optimisticResults=r?"isRestoring":"optimistic",B2(s),M2(s,o),j2(o);const a=!i.getQueryCache().get(s.queryHash),[l]=C.useState(()=>new t(i,s)),c=l.getOptimisticResult(s),u=!r&&e.subscribed!==!1;if(C.useSyncExternalStore(C.useCallback(h=>{const d=u?l.subscribe(ge.batchCalls(h)):Ne;return l.updateResult(),d},[l,u]),()=>l.getCurrentResult(),()=>l.getCurrentResult()),C.useEffect(()=>{l.setOptions(s)},[s,l]),H2(s,c))throw Df(s,l,o);if($2({result:c,errorResetBoundary:o,throwOnError:s.throwOnError,query:i.getQueryCache().get(s.queryHash),suspense:s.suspense}))throw c.error;return i.getDefaultOptions().queries?._experimental_afterQuery?.(s,c),s.experimental_prefetchInRender&&!En&&W2(c,r)&&(a?Df(s,l,o):i.getQueryCache().get(s.queryHash)?.promise)?.catch(Ne).finally(()=>{l.updateResult()}),s.notifyOnChangeProps?c:l.trackResult(c)}function zf(e,t){return U2(e,E2)}function Pn(e,t){const n=Ff(),[r]=C.useState(()=>new N2(n,e));C.useEffect(()=>{r.setOptions(e)},[r,e]);const o=C.useSyncExternalStore(C.useCallback(s=>r.subscribe(ge.batchCalls(s)),[r]),()=>r.getCurrentResult(),()=>r.getCurrentResult()),i=C.useCallback((s,a)=>{r.mutate(s,a).catch(Ne)},[r]);if(o.error&&wf(r.options.throwOnError,[o.error]))throw o.error;return{...o,mutate:i,mutateAsync:o.mutate}}function Mf(e,t){return function(){return e.apply(t,arguments)}}const{toString:G2}=Object.prototype,{getPrototypeOf:Ba}=Object,{iterator:xi,toStringTag:jf}=Symbol,ki=(e=>t=>{const n=G2.call(t);return e[n]||(e[n]=n.slice(8,-1).toLowerCase())})(Object.create(null)),ot=e=>(e=e.toLowerCase(),t=>ki(t)===e),Ci=e=>t=>typeof t===e,{isArray:sr}=Array,ar=Ci("undefined");function Hr(e){return e!==null&&!ar(e)&&e.constructor!==null&&!ar(e.constructor)&&ze(e.constructor.isBuffer)&&e.constructor.isBuffer(e)}const $f=ot("ArrayBuffer");function q2(e){let t;return typeof ArrayBuffer<"u"&&ArrayBuffer.isView?t=ArrayBuffer.isView(e):t=e&&e.buffer&&$f(e.buffer),t}const K2=Ci("string"),ze=Ci("function"),Bf=Ci("number"),Ur=e=>e!==null&&typeof e=="object",Y2=e=>e===!0||e===!1,wi=e=>{if(ki(e)!=="object")return!1;const t=Ba(e);return(t===null||t===Object.prototype||Object.getPrototypeOf(t)===null)&&!(jf in e)&&!(xi in e)},X2=e=>{if(!Ur(e)||Hr(e))return!1;try{return Object.keys(e).length===0&&Object.getPrototypeOf(e)===Object.prototype}catch{return!1}},Q2=ot("Date"),J2=ot("File"),Z2=ot("Blob"),eO=ot("FileList"),tO=e=>Ur(e)&&ze(e.pipe),nO=e=>{let t;return e&&(typeof FormData=="function"&&e instanceof FormData||ze(e.append)&&((t=ki(e))==="formdata"||t==="object"&&ze(e.toString)&&e.toString()==="[object FormData]"))},rO=ot("URLSearchParams"),[oO,iO,sO,aO]=["ReadableStream","Request","Response","Headers"].map(ot),lO=e=>e.trim?e.trim():e.replace(/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,"");function Gr(e,t,{allOwnKeys:n=!1}={}){if(e===null||typeof e>"u")return;let r,o;if(typeof e!="object"&&(e=[e]),sr(e))for(r=0,o=e.length;r0;)if(o=n[r],t===o.toLowerCase())return o;return null}const Rn=typeof globalThis<"u"?globalThis:typeof self<"u"?self:typeof window<"u"?window:globalThis,Hf=e=>!ar(e)&&e!==Rn;function Wa(){const{caseless:e,skipUndefined:t}=Hf(this)&&this||{},n={},r=(o,i)=>{const s=e&&Wf(n,i)||i;wi(n[s])&&wi(o)?n[s]=Wa(n[s],o):wi(o)?n[s]=Wa({},o):sr(o)?n[s]=o.slice():(!t||!ar(o))&&(n[s]=o)};for(let o=0,i=arguments.length;o(Gr(t,(o,i)=>{n&&ze(o)?e[i]=Mf(o,n):e[i]=o},{allOwnKeys:r}),e),uO=e=>(e.charCodeAt(0)===65279&&(e=e.slice(1)),e),dO=(e,t,n,r)=>{e.prototype=Object.create(t.prototype,r),e.prototype.constructor=e,Object.defineProperty(e,"super",{value:t.prototype}),n&&Object.assign(e.prototype,n)},hO=(e,t,n,r)=>{let o,i,s;const a={};if(t=t||{},e==null)return t;do{for(o=Object.getOwnPropertyNames(e),i=o.length;i-- >0;)s=o[i],(!r||r(s,e,t))&&!a[s]&&(t[s]=e[s],a[s]=!0);e=n!==!1&&Ba(e)}while(e&&(!n||n(e,t))&&e!==Object.prototype);return t},fO=(e,t,n)=>{e=String(e),(n===void 0||n>e.length)&&(n=e.length),n-=t.length;const r=e.indexOf(t,n);return r!==-1&&r===n},gO=e=>{if(!e)return null;if(sr(e))return e;let t=e.length;if(!Bf(t))return null;const n=new Array(t);for(;t-- >0;)n[t]=e[t];return n},pO=(e=>t=>e&&t instanceof e)(typeof Uint8Array<"u"&&Ba(Uint8Array)),mO=(e,t)=>{const r=(e&&e[xi]).call(e);let o;for(;(o=r.next())&&!o.done;){const i=o.value;t.call(e,i[0],i[1])}},vO=(e,t)=>{let n;const r=[];for(;(n=e.exec(t))!==null;)r.push(n);return r},bO=ot("HTMLFormElement"),yO=e=>e.toLowerCase().replace(/[-_\s]([a-z\d])(\w*)/g,function(n,r,o){return r.toUpperCase()+o}),Uf=(({hasOwnProperty:e})=>(t,n)=>e.call(t,n))(Object.prototype),xO=ot("RegExp"),Gf=(e,t)=>{const n=Object.getOwnPropertyDescriptors(e),r={};Gr(n,(o,i)=>{let s;(s=t(o,i,e))!==!1&&(r[i]=s||o)}),Object.defineProperties(e,r)},kO=e=>{Gf(e,(t,n)=>{if(ze(e)&&["arguments","caller","callee"].indexOf(n)!==-1)return!1;const r=e[n];if(ze(r)){if(t.enumerable=!1,"writable"in t){t.writable=!1;return}t.set||(t.set=()=>{throw Error("Can not rewrite read-only method '"+n+"'")})}})},CO=(e,t)=>{const n={},r=o=>{o.forEach(i=>{n[i]=!0})};return sr(e)?r(e):r(String(e).split(t)),n},wO=()=>{},SO=(e,t)=>e!=null&&Number.isFinite(e=+e)?e:t;function EO(e){return!!(e&&ze(e.append)&&e[jf]==="FormData"&&e[xi])}const OO=e=>{const t=new Array(10),n=(r,o)=>{if(Ur(r)){if(t.indexOf(r)>=0)return;if(Hr(r))return r;if(!("toJSON"in r)){t[o]=r;const i=sr(r)?[]:{};return Gr(r,(s,a)=>{const l=n(s,o+1);!ar(l)&&(i[a]=l)}),t[o]=void 0,i}}return r};return n(e,0)},PO=ot("AsyncFunction"),RO=e=>e&&(Ur(e)||ze(e))&&ze(e.then)&&ze(e.catch),qf=((e,t)=>e?setImmediate:t?((n,r)=>(Rn.addEventListener("message",({source:o,data:i})=>{o===Rn&&i===n&&r.length&&r.shift()()},!1),o=>{r.push(o),Rn.postMessage(n,"*")}))(`axios@${Math.random()}`,[]):n=>setTimeout(n))(typeof setImmediate=="function",ze(Rn.postMessage)),IO=typeof queueMicrotask<"u"?queueMicrotask.bind(Rn):typeof process<"u"&&process.nextTick||qf,E={isArray:sr,isArrayBuffer:$f,isBuffer:Hr,isFormData:nO,isArrayBufferView:q2,isString:K2,isNumber:Bf,isBoolean:Y2,isObject:Ur,isPlainObject:wi,isEmptyObject:X2,isReadableStream:oO,isRequest:iO,isResponse:sO,isHeaders:aO,isUndefined:ar,isDate:Q2,isFile:J2,isBlob:Z2,isRegExp:xO,isFunction:ze,isStream:tO,isURLSearchParams:rO,isTypedArray:pO,isFileList:eO,forEach:Gr,merge:Wa,extend:cO,trim:lO,stripBOM:uO,inherits:dO,toFlatObject:hO,kindOf:ki,kindOfTest:ot,endsWith:fO,toArray:gO,forEachEntry:mO,matchAll:vO,isHTMLForm:bO,hasOwnProperty:Uf,hasOwnProp:Uf,reduceDescriptors:Gf,freezeMethods:kO,toObjectSet:CO,toCamelCase:yO,noop:wO,toFiniteNumber:SO,findKey:Wf,global:Rn,isContextDefined:Hf,isSpecCompliantForm:EO,toJSONObject:OO,isAsyncFn:PO,isThenable:RO,setImmediate:qf,asap:IO,isIterable:e=>e!=null&&ze(e[xi])};function H(e,t,n,r,o){Error.call(this),Error.captureStackTrace?Error.captureStackTrace(this,this.constructor):this.stack=new Error().stack,this.message=e,this.name="AxiosError",t&&(this.code=t),n&&(this.config=n),r&&(this.request=r),o&&(this.response=o,this.status=o.status?o.status:null)}E.inherits(H,Error,{toJSON:function(){return{message:this.message,name:this.name,description:this.description,number:this.number,fileName:this.fileName,lineNumber:this.lineNumber,columnNumber:this.columnNumber,stack:this.stack,config:E.toJSONObject(this.config),code:this.code,status:this.status}}});const Kf=H.prototype,Yf={};["ERR_BAD_OPTION_VALUE","ERR_BAD_OPTION","ECONNABORTED","ETIMEDOUT","ERR_NETWORK","ERR_FR_TOO_MANY_REDIRECTS","ERR_DEPRECATED","ERR_BAD_RESPONSE","ERR_BAD_REQUEST","ERR_CANCELED","ERR_NOT_SUPPORT","ERR_INVALID_URL"].forEach(e=>{Yf[e]={value:e}}),Object.defineProperties(H,Yf),Object.defineProperty(Kf,"isAxiosError",{value:!0}),H.from=(e,t,n,r,o,i)=>{const s=Object.create(Kf);E.toFlatObject(e,s,function(u){return u!==Error.prototype},c=>c!=="isAxiosError");const a=e&&e.message?e.message:"Error",l=t==null&&e?e.code:t;return H.call(s,a,l,n,r,o),e&&s.cause==null&&Object.defineProperty(s,"cause",{value:e,configurable:!0}),s.name=e&&e.name||"Error",i&&Object.assign(s,i),s};const TO=null;function Ha(e){return E.isPlainObject(e)||E.isArray(e)}function Xf(e){return E.endsWith(e,"[]")?e.slice(0,-2):e}function Qf(e,t,n){return e?e.concat(t).map(function(o,i){return o=Xf(o),!n&&i?"["+o+"]":o}).join(n?".":""):t}function NO(e){return E.isArray(e)&&!e.some(Ha)}const AO=E.toFlatObject(E,{},null,function(t){return/^is[A-Z]/.test(t)});function Si(e,t,n){if(!E.isObject(e))throw new TypeError("target must be an object");t=t||new FormData,n=E.toFlatObject(n,{metaTokens:!0,dots:!1,indexes:!1},!1,function(p,v){return!E.isUndefined(v[p])});const r=n.metaTokens,o=n.visitor||u,i=n.dots,s=n.indexes,l=(n.Blob||typeof Blob<"u"&&Blob)&&E.isSpecCompliantForm(t);if(!E.isFunction(o))throw new TypeError("visitor must be a function");function c(f){if(f===null)return"";if(E.isDate(f))return f.toISOString();if(E.isBoolean(f))return f.toString();if(!l&&E.isBlob(f))throw new H("Blob is not supported. Use a Buffer instead.");return E.isArrayBuffer(f)||E.isTypedArray(f)?l&&typeof Blob=="function"?new Blob([f]):Buffer.from(f):f}function u(f,p,v){let b=f;if(f&&!v&&typeof f=="object"){if(E.endsWith(p,"{}"))p=r?p:p.slice(0,-2),f=JSON.stringify(f);else if(E.isArray(f)&&NO(f)||(E.isFileList(f)||E.endsWith(p,"[]"))&&(b=E.toArray(f)))return p=Xf(p),b.forEach(function(x,w){!(E.isUndefined(x)||x===null)&&t.append(s===!0?Qf([p],w,i):s===null?p:p+"[]",c(x))}),!1}return Ha(f)?!0:(t.append(Qf(v,p,i),c(f)),!1)}const h=[],d=Object.assign(AO,{defaultVisitor:u,convertValue:c,isVisitable:Ha});function m(f,p){if(!E.isUndefined(f)){if(h.indexOf(f)!==-1)throw Error("Circular reference detected in "+p.join("."));h.push(f),E.forEach(f,function(b,y){(!(E.isUndefined(b)||b===null)&&o.call(t,b,E.isString(y)?y.trim():y,p,d))===!0&&m(b,p?p.concat(y):[y])}),h.pop()}}if(!E.isObject(e))throw new TypeError("data must be an object");return m(e),t}function Jf(e){const t={"!":"%21","'":"%27","(":"%28",")":"%29","~":"%7E","%20":"+","%00":"\0"};return encodeURIComponent(e).replace(/[!'()~]|%20|%00/g,function(r){return t[r]})}function Ua(e,t){this._pairs=[],e&&Si(e,this,t)}const Zf=Ua.prototype;Zf.append=function(t,n){this._pairs.push([t,n])},Zf.toString=function(t){const n=t?function(r){return t.call(this,r,Jf)}:Jf;return this._pairs.map(function(o){return n(o[0])+"="+n(o[1])},"").join("&")};function _O(e){return encodeURIComponent(e).replace(/%3A/gi,":").replace(/%24/g,"$").replace(/%2C/gi,",").replace(/%20/g,"+")}function eg(e,t,n){if(!t)return e;const r=n&&n.encode||_O;E.isFunction(n)&&(n={serialize:n});const o=n&&n.serialize;let i;if(o?i=o(t,n):i=E.isURLSearchParams(t)?t.toString():new Ua(t,n).toString(r),i){const s=e.indexOf("#");s!==-1&&(e=e.slice(0,s)),e+=(e.indexOf("?")===-1?"?":"&")+i}return e}class tg{constructor(){this.handlers=[]}use(t,n,r){return this.handlers.push({fulfilled:t,rejected:n,synchronous:r?r.synchronous:!1,runWhen:r?r.runWhen:null}),this.handlers.length-1}eject(t){this.handlers[t]&&(this.handlers[t]=null)}clear(){this.handlers&&(this.handlers=[])}forEach(t){E.forEach(this.handlers,function(r){r!==null&&t(r)})}}const ng={silentJSONParsing:!0,forcedJSONParsing:!0,clarifyTimeoutError:!1},VO={isBrowser:!0,classes:{URLSearchParams:typeof URLSearchParams<"u"?URLSearchParams:Ua,FormData:typeof FormData<"u"?FormData:null,Blob:typeof Blob<"u"?Blob:null},protocols:["http","https","file","blob","url","data"]},Ga=typeof window<"u"&&typeof document<"u",qa=typeof navigator=="object"&&navigator||void 0,FO=Ga&&(!qa||["ReactNative","NativeScript","NS"].indexOf(qa.product)<0),LO=typeof WorkerGlobalScope<"u"&&self instanceof WorkerGlobalScope&&typeof self.importScripts=="function",DO=Ga&&window.location.href||"http://localhost",Ie={...Object.freeze(Object.defineProperty({__proto__:null,hasBrowserEnv:Ga,hasStandardBrowserEnv:FO,hasStandardBrowserWebWorkerEnv:LO,navigator:qa,origin:DO},Symbol.toStringTag,{value:"Module"})),...VO};function zO(e,t){return Si(e,new Ie.classes.URLSearchParams,{visitor:function(n,r,o,i){return Ie.isNode&&E.isBuffer(n)?(this.append(r,n.toString("base64")),!1):i.defaultVisitor.apply(this,arguments)},...t})}function MO(e){return E.matchAll(/\w+|\[(\w*)]/g,e).map(t=>t[0]==="[]"?"":t[1]||t[0])}function jO(e){const t={},n=Object.keys(e);let r;const o=n.length;let i;for(r=0;r=n.length;return s=!s&&E.isArray(o)?o.length:s,l?(E.hasOwnProp(o,s)?o[s]=[o[s],r]:o[s]=r,!a):((!o[s]||!E.isObject(o[s]))&&(o[s]=[]),t(n,r,o[s],i)&&E.isArray(o[s])&&(o[s]=jO(o[s])),!a)}if(E.isFormData(e)&&E.isFunction(e.entries)){const n={};return E.forEachEntry(e,(r,o)=>{t(MO(r),o,n,0)}),n}return null}function $O(e,t,n){if(E.isString(e))try{return(t||JSON.parse)(e),E.trim(e)}catch(r){if(r.name!=="SyntaxError")throw r}return(n||JSON.stringify)(e)}const qr={transitional:ng,adapter:["xhr","http","fetch"],transformRequest:[function(t,n){const r=n.getContentType()||"",o=r.indexOf("application/json")>-1,i=E.isObject(t);if(i&&E.isHTMLForm(t)&&(t=new FormData(t)),E.isFormData(t))return o?JSON.stringify(rg(t)):t;if(E.isArrayBuffer(t)||E.isBuffer(t)||E.isStream(t)||E.isFile(t)||E.isBlob(t)||E.isReadableStream(t))return t;if(E.isArrayBufferView(t))return t.buffer;if(E.isURLSearchParams(t))return n.setContentType("application/x-www-form-urlencoded;charset=utf-8",!1),t.toString();let a;if(i){if(r.indexOf("application/x-www-form-urlencoded")>-1)return zO(t,this.formSerializer).toString();if((a=E.isFileList(t))||r.indexOf("multipart/form-data")>-1){const l=this.env&&this.env.FormData;return Si(a?{"files[]":t}:t,l&&new l,this.formSerializer)}}return i||o?(n.setContentType("application/json",!1),$O(t)):t}],transformResponse:[function(t){const n=this.transitional||qr.transitional,r=n&&n.forcedJSONParsing,o=this.responseType==="json";if(E.isResponse(t)||E.isReadableStream(t))return t;if(t&&E.isString(t)&&(r&&!this.responseType||o)){const s=!(n&&n.silentJSONParsing)&&o;try{return JSON.parse(t,this.parseReviver)}catch(a){if(s)throw a.name==="SyntaxError"?H.from(a,H.ERR_BAD_RESPONSE,this,null,this.response):a}}return t}],timeout:0,xsrfCookieName:"XSRF-TOKEN",xsrfHeaderName:"X-XSRF-TOKEN",maxContentLength:-1,maxBodyLength:-1,env:{FormData:Ie.classes.FormData,Blob:Ie.classes.Blob},validateStatus:function(t){return t>=200&&t<300},headers:{common:{Accept:"application/json, text/plain, */*","Content-Type":void 0}}};E.forEach(["delete","get","head","post","put","patch"],e=>{qr.headers[e]={}});const BO=E.toObjectSet(["age","authorization","content-length","content-type","etag","expires","from","host","if-modified-since","if-unmodified-since","last-modified","location","max-forwards","proxy-authorization","referer","retry-after","user-agent"]),WO=e=>{const t={};let n,r,o;return e&&e.split(` +`).forEach(function(s){o=s.indexOf(":"),n=s.substring(0,o).trim().toLowerCase(),r=s.substring(o+1).trim(),!(!n||t[n]&&BO[n])&&(n==="set-cookie"?t[n]?t[n].push(r):t[n]=[r]:t[n]=t[n]?t[n]+", "+r:r)}),t},og=Symbol("internals");function Kr(e){return e&&String(e).trim().toLowerCase()}function Ei(e){return e===!1||e==null?e:E.isArray(e)?e.map(Ei):String(e)}function HO(e){const t=Object.create(null),n=/([^\s,;=]+)\s*(?:=\s*([^,;]+))?/g;let r;for(;r=n.exec(e);)t[r[1]]=r[2];return t}const UO=e=>/^[-_a-zA-Z0-9^`|~,!#$%&'*+.]+$/.test(e.trim());function Ka(e,t,n,r,o){if(E.isFunction(r))return r.call(this,t,n);if(o&&(t=n),!!E.isString(t)){if(E.isString(r))return t.indexOf(r)!==-1;if(E.isRegExp(r))return r.test(t)}}function GO(e){return e.trim().toLowerCase().replace(/([a-z\d])(\w*)/g,(t,n,r)=>n.toUpperCase()+r)}function qO(e,t){const n=E.toCamelCase(" "+t);["get","set","has"].forEach(r=>{Object.defineProperty(e,r+n,{value:function(o,i,s){return this[r].call(this,t,o,i,s)},configurable:!0})})}let Me=class{constructor(t){t&&this.set(t)}set(t,n,r){const o=this;function i(a,l,c){const u=Kr(l);if(!u)throw new Error("header name must be a non-empty string");const h=E.findKey(o,u);(!h||o[h]===void 0||c===!0||c===void 0&&o[h]!==!1)&&(o[h||l]=Ei(a))}const s=(a,l)=>E.forEach(a,(c,u)=>i(c,u,l));if(E.isPlainObject(t)||t instanceof this.constructor)s(t,n);else if(E.isString(t)&&(t=t.trim())&&!UO(t))s(WO(t),n);else if(E.isObject(t)&&E.isIterable(t)){let a={},l,c;for(const u of t){if(!E.isArray(u))throw TypeError("Object iterator must return a key-value pair");a[c=u[0]]=(l=a[c])?E.isArray(l)?[...l,u[1]]:[l,u[1]]:u[1]}s(a,n)}else t!=null&&i(n,t,r);return this}get(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);if(r){const o=this[r];if(!n)return o;if(n===!0)return HO(o);if(E.isFunction(n))return n.call(this,o,r);if(E.isRegExp(n))return n.exec(o);throw new TypeError("parser must be boolean|regexp|function")}}}has(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);return!!(r&&this[r]!==void 0&&(!n||Ka(this,this[r],r,n)))}return!1}delete(t,n){const r=this;let o=!1;function i(s){if(s=Kr(s),s){const a=E.findKey(r,s);a&&(!n||Ka(r,r[a],a,n))&&(delete r[a],o=!0)}}return E.isArray(t)?t.forEach(i):i(t),o}clear(t){const n=Object.keys(this);let r=n.length,o=!1;for(;r--;){const i=n[r];(!t||Ka(this,this[i],i,t,!0))&&(delete this[i],o=!0)}return o}normalize(t){const n=this,r={};return E.forEach(this,(o,i)=>{const s=E.findKey(r,i);if(s){n[s]=Ei(o),delete n[i];return}const a=t?GO(i):String(i).trim();a!==i&&delete n[i],n[a]=Ei(o),r[a]=!0}),this}concat(...t){return this.constructor.concat(this,...t)}toJSON(t){const n=Object.create(null);return E.forEach(this,(r,o)=>{r!=null&&r!==!1&&(n[o]=t&&E.isArray(r)?r.join(", "):r)}),n}[Symbol.iterator](){return Object.entries(this.toJSON())[Symbol.iterator]()}toString(){return Object.entries(this.toJSON()).map(([t,n])=>t+": "+n).join(` +`)}getSetCookie(){return this.get("set-cookie")||[]}get[Symbol.toStringTag](){return"AxiosHeaders"}static from(t){return t instanceof this?t:new this(t)}static concat(t,...n){const r=new this(t);return n.forEach(o=>r.set(o)),r}static accessor(t){const r=(this[og]=this[og]={accessors:{}}).accessors,o=this.prototype;function i(s){const a=Kr(s);r[a]||(qO(o,s),r[a]=!0)}return E.isArray(t)?t.forEach(i):i(t),this}};Me.accessor(["Content-Type","Content-Length","Accept","Accept-Encoding","User-Agent","Authorization"]),E.reduceDescriptors(Me.prototype,({value:e},t)=>{let n=t[0].toUpperCase()+t.slice(1);return{get:()=>e,set(r){this[n]=r}}}),E.freezeMethods(Me);function Ya(e,t){const n=this||qr,r=t||n,o=Me.from(r.headers);let i=r.data;return E.forEach(e,function(a){i=a.call(n,i,o.normalize(),t?t.status:void 0)}),o.normalize(),i}function ig(e){return!!(e&&e.__CANCEL__)}function lr(e,t,n){H.call(this,e??"canceled",H.ERR_CANCELED,t,n),this.name="CanceledError"}E.inherits(lr,H,{__CANCEL__:!0});function sg(e,t,n){const r=n.config.validateStatus;!n.status||!r||r(n.status)?e(n):t(new H("Request failed with status code "+n.status,[H.ERR_BAD_REQUEST,H.ERR_BAD_RESPONSE][Math.floor(n.status/100)-4],n.config,n.request,n))}function KO(e){const t=/^([-+\w]{1,25})(:?\/\/|:)/.exec(e);return t&&t[1]||""}function YO(e,t){e=e||10;const n=new Array(e),r=new Array(e);let o=0,i=0,s;return t=t!==void 0?t:1e3,function(l){const c=Date.now(),u=r[i];s||(s=c),n[o]=l,r[o]=c;let h=i,d=0;for(;h!==o;)d+=n[h++],h=h%e;if(o=(o+1)%e,o===i&&(i=(i+1)%e),c-s{n=u,o=null,i&&(clearTimeout(i),i=null),e(...c)};return[(...c)=>{const u=Date.now(),h=u-n;h>=r?s(c,u):(o=c,i||(i=setTimeout(()=>{i=null,s(o)},r-h)))},()=>o&&s(o)]}const Oi=(e,t,n=3)=>{let r=0;const o=YO(50,250);return XO(i=>{const s=i.loaded,a=i.lengthComputable?i.total:void 0,l=s-r,c=o(l),u=s<=a;r=s;const h={loaded:s,total:a,progress:a?s/a:void 0,bytes:l,rate:c||void 0,estimated:c&&a&&u?(a-s)/c:void 0,event:i,lengthComputable:a!=null,[t?"download":"upload"]:!0};e(h)},n)},ag=(e,t)=>{const n=e!=null;return[r=>t[0]({lengthComputable:n,total:e,loaded:r}),t[1]]},lg=e=>(...t)=>E.asap(()=>e(...t)),QO=Ie.hasStandardBrowserEnv?((e,t)=>n=>(n=new URL(n,Ie.origin),e.protocol===n.protocol&&e.host===n.host&&(t||e.port===n.port)))(new URL(Ie.origin),Ie.navigator&&/(msie|trident)/i.test(Ie.navigator.userAgent)):()=>!0,JO=Ie.hasStandardBrowserEnv?{write(e,t,n,r,o,i){const s=[e+"="+encodeURIComponent(t)];E.isNumber(n)&&s.push("expires="+new Date(n).toGMTString()),E.isString(r)&&s.push("path="+r),E.isString(o)&&s.push("domain="+o),i===!0&&s.push("secure"),document.cookie=s.join("; ")},read(e){const t=document.cookie.match(new RegExp("(^|;\\s*)("+e+")=([^;]*)"));return t?decodeURIComponent(t[3]):null},remove(e){this.write(e,"",Date.now()-864e5)}}:{write(){},read(){return null},remove(){}};function ZO(e){return/^([a-z][a-z\d+\-.]*:)?\/\//i.test(e)}function eP(e,t){return t?e.replace(/\/?\/$/,"")+"/"+t.replace(/^\/+/,""):e}function cg(e,t,n){let r=!ZO(t);return e&&(r||n==!1)?eP(e,t):t}const ug=e=>e instanceof Me?{...e}:e;function In(e,t){t=t||{};const n={};function r(c,u,h,d){return E.isPlainObject(c)&&E.isPlainObject(u)?E.merge.call({caseless:d},c,u):E.isPlainObject(u)?E.merge({},u):E.isArray(u)?u.slice():u}function o(c,u,h,d){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c,h,d)}else return r(c,u,h,d)}function i(c,u){if(!E.isUndefined(u))return r(void 0,u)}function s(c,u){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c)}else return r(void 0,u)}function a(c,u,h){if(h in t)return r(c,u);if(h in e)return r(void 0,c)}const l={url:i,method:i,data:i,baseURL:s,transformRequest:s,transformResponse:s,paramsSerializer:s,timeout:s,timeoutMessage:s,withCredentials:s,withXSRFToken:s,adapter:s,responseType:s,xsrfCookieName:s,xsrfHeaderName:s,onUploadProgress:s,onDownloadProgress:s,decompress:s,maxContentLength:s,maxBodyLength:s,beforeRedirect:s,transport:s,httpAgent:s,httpsAgent:s,cancelToken:s,socketPath:s,responseEncoding:s,validateStatus:a,headers:(c,u,h)=>o(ug(c),ug(u),h,!0)};return E.forEach(Object.keys({...e,...t}),function(u){const h=l[u]||o,d=h(e[u],t[u],u);E.isUndefined(d)&&h!==a||(n[u]=d)}),n}const dg=e=>{const t=In({},e);let{data:n,withXSRFToken:r,xsrfHeaderName:o,xsrfCookieName:i,headers:s,auth:a}=t;if(t.headers=s=Me.from(s),t.url=eg(cg(t.baseURL,t.url,t.allowAbsoluteUrls),e.params,e.paramsSerializer),a&&s.set("Authorization","Basic "+btoa((a.username||"")+":"+(a.password?unescape(encodeURIComponent(a.password)):""))),E.isFormData(n)){if(Ie.hasStandardBrowserEnv||Ie.hasStandardBrowserWebWorkerEnv)s.setContentType(void 0);else if(E.isFunction(n.getHeaders)){const l=n.getHeaders(),c=["content-type","content-length"];Object.entries(l).forEach(([u,h])=>{c.includes(u.toLowerCase())&&s.set(u,h)})}}if(Ie.hasStandardBrowserEnv&&(r&&E.isFunction(r)&&(r=r(t)),r||r!==!1&&QO(t.url))){const l=o&&i&&JO.read(i);l&&s.set(o,l)}return t},tP=typeof XMLHttpRequest<"u"&&function(e){return new Promise(function(n,r){const o=dg(e);let i=o.data;const s=Me.from(o.headers).normalize();let{responseType:a,onUploadProgress:l,onDownloadProgress:c}=o,u,h,d,m,f;function p(){m&&m(),f&&f(),o.cancelToken&&o.cancelToken.unsubscribe(u),o.signal&&o.signal.removeEventListener("abort",u)}let v=new XMLHttpRequest;v.open(o.method.toUpperCase(),o.url,!0),v.timeout=o.timeout;function b(){if(!v)return;const x=Me.from("getAllResponseHeaders"in v&&v.getAllResponseHeaders()),k={data:!a||a==="text"||a==="json"?v.responseText:v.response,status:v.status,statusText:v.statusText,headers:x,config:e,request:v};sg(function(R){n(R),p()},function(R){r(R),p()},k),v=null}"onloadend"in v?v.onloadend=b:v.onreadystatechange=function(){!v||v.readyState!==4||v.status===0&&!(v.responseURL&&v.responseURL.indexOf("file:")===0)||setTimeout(b)},v.onabort=function(){v&&(r(new H("Request aborted",H.ECONNABORTED,e,v)),v=null)},v.onerror=function(w){const k=w&&w.message?w.message:"Network Error",O=new H(k,H.ERR_NETWORK,e,v);O.event=w||null,r(O),v=null},v.ontimeout=function(){let w=o.timeout?"timeout of "+o.timeout+"ms exceeded":"timeout exceeded";const k=o.transitional||ng;o.timeoutErrorMessage&&(w=o.timeoutErrorMessage),r(new H(w,k.clarifyTimeoutError?H.ETIMEDOUT:H.ECONNABORTED,e,v)),v=null},i===void 0&&s.setContentType(null),"setRequestHeader"in v&&E.forEach(s.toJSON(),function(w,k){v.setRequestHeader(k,w)}),E.isUndefined(o.withCredentials)||(v.withCredentials=!!o.withCredentials),a&&a!=="json"&&(v.responseType=o.responseType),c&&([d,f]=Oi(c,!0),v.addEventListener("progress",d)),l&&v.upload&&([h,m]=Oi(l),v.upload.addEventListener("progress",h),v.upload.addEventListener("loadend",m)),(o.cancelToken||o.signal)&&(u=x=>{v&&(r(!x||x.type?new lr(null,e,v):x),v.abort(),v=null)},o.cancelToken&&o.cancelToken.subscribe(u),o.signal&&(o.signal.aborted?u():o.signal.addEventListener("abort",u)));const y=KO(o.url);if(y&&Ie.protocols.indexOf(y)===-1){r(new H("Unsupported protocol "+y+":",H.ERR_BAD_REQUEST,e));return}v.send(i||null)})},nP=(e,t)=>{const{length:n}=e=e?e.filter(Boolean):[];if(t||n){let r=new AbortController,o;const i=function(c){if(!o){o=!0,a();const u=c instanceof Error?c:this.reason;r.abort(u instanceof H?u:new lr(u instanceof Error?u.message:u))}};let s=t&&setTimeout(()=>{s=null,i(new H(`timeout ${t} of ms exceeded`,H.ETIMEDOUT))},t);const a=()=>{e&&(s&&clearTimeout(s),s=null,e.forEach(c=>{c.unsubscribe?c.unsubscribe(i):c.removeEventListener("abort",i)}),e=null)};e.forEach(c=>c.addEventListener("abort",i));const{signal:l}=r;return l.unsubscribe=()=>E.asap(a),l}},rP=function*(e,t){let n=e.byteLength;if(n{const o=oP(e,t);let i=0,s,a=l=>{s||(s=!0,r&&r(l))};return new ReadableStream({async pull(l){try{const{done:c,value:u}=await o.next();if(c){a(),l.close();return}let h=u.byteLength;if(n){let d=i+=h;n(d)}l.enqueue(new Uint8Array(u))}catch(c){throw a(c),c}},cancel(l){return a(l),o.return()}},{highWaterMark:2})},fg=64*1024,{isFunction:Pi}=E,sP=(({Request:e,Response:t})=>({Request:e,Response:t}))(E.global),{ReadableStream:gg,TextEncoder:pg}=E.global,mg=(e,...t)=>{try{return!!e(...t)}catch{return!1}},aP=e=>{e=E.merge.call({skipUndefined:!0},sP,e);const{fetch:t,Request:n,Response:r}=e,o=t?Pi(t):typeof fetch=="function",i=Pi(n),s=Pi(r);if(!o)return!1;const a=o&&Pi(gg),l=o&&(typeof pg=="function"?(f=>p=>f.encode(p))(new pg):async f=>new Uint8Array(await new n(f).arrayBuffer())),c=i&&a&&mg(()=>{let f=!1;const p=new n(Ie.origin,{body:new gg,method:"POST",get duplex(){return f=!0,"half"}}).headers.has("Content-Type");return f&&!p}),u=s&&a&&mg(()=>E.isReadableStream(new r("").body)),h={stream:u&&(f=>f.body)};o&&["text","arrayBuffer","blob","formData","stream"].forEach(f=>{!h[f]&&(h[f]=(p,v)=>{let b=p&&p[f];if(b)return b.call(p);throw new H(`Response type '${f}' is not supported`,H.ERR_NOT_SUPPORT,v)})});const d=async f=>{if(f==null)return 0;if(E.isBlob(f))return f.size;if(E.isSpecCompliantForm(f))return(await new n(Ie.origin,{method:"POST",body:f}).arrayBuffer()).byteLength;if(E.isArrayBufferView(f)||E.isArrayBuffer(f))return f.byteLength;if(E.isURLSearchParams(f)&&(f=f+""),E.isString(f))return(await l(f)).byteLength},m=async(f,p)=>{const v=E.toFiniteNumber(f.getContentLength());return v??d(p)};return async f=>{let{url:p,method:v,data:b,signal:y,cancelToken:x,timeout:w,onDownloadProgress:k,onUploadProgress:O,responseType:R,headers:I,withCredentials:N="same-origin",fetchOptions:A}=dg(f),T=t||fetch;R=R?(R+"").toLowerCase():"text";let S=nP([y,x&&x.toAbortSignal()],w),_=null;const D=S&&S.unsubscribe&&(()=>{S.unsubscribe()});let z;try{if(O&&c&&v!=="get"&&v!=="head"&&(z=await m(I,b))!==0){let Z=new n(p,{method:"POST",body:b,duplex:"half"}),pe;if(E.isFormData(b)&&(pe=Z.headers.get("content-type"))&&I.setContentType(pe),Z.body){const[Je,yt]=ag(z,Oi(lg(O)));b=hg(Z.body,fg,Je,yt)}}E.isString(N)||(N=N?"include":"omit");const W=i&&"credentials"in n.prototype,J={...A,signal:S,method:v.toUpperCase(),headers:I.normalize().toJSON(),body:b,duplex:"half",credentials:W?N:void 0};_=i&&new n(p,J);let ee=await(i?T(_,A):T(p,J));const $=u&&(R==="stream"||R==="response");if(u&&(k||$&&D)){const Z={};["status","statusText","headers"].forEach(_t=>{Z[_t]=ee[_t]});const pe=E.toFiniteNumber(ee.headers.get("content-length")),[Je,yt]=k&&ag(pe,Oi(lg(k),!0))||[];ee=new r(hg(ee.body,fg,Je,()=>{yt&&yt(),D&&D()}),Z)}R=R||"text";let q=await h[E.findKey(h,R)||"text"](ee,f);return!$&&D&&D(),await new Promise((Z,pe)=>{sg(Z,pe,{data:q,headers:Me.from(ee.headers),status:ee.status,statusText:ee.statusText,config:f,request:_})})}catch(W){throw D&&D(),W&&W.name==="TypeError"&&/Load failed|fetch/i.test(W.message)?Object.assign(new H("Network Error",H.ERR_NETWORK,f,_),{cause:W.cause||W}):H.from(W,W&&W.code,f,_)}}},lP=new Map,vg=e=>{let t=e?e.env:{};const{fetch:n,Request:r,Response:o}=t,i=[r,o,n];let s=i.length,a=s,l,c,u=lP;for(;a--;)l=i[a],c=u.get(l),c===void 0&&u.set(l,c=a?new Map:aP(t)),u=c;return c};vg();const Xa={http:TO,xhr:tP,fetch:{get:vg}};E.forEach(Xa,(e,t)=>{if(e){try{Object.defineProperty(e,"name",{value:t})}catch{}Object.defineProperty(e,"adapterName",{value:t})}});const bg=e=>`- ${e}`,cP=e=>E.isFunction(e)||e===null||e===!1,yg={getAdapter:(e,t)=>{e=E.isArray(e)?e:[e];const{length:n}=e;let r,o;const i={};for(let s=0;s`adapter ${l} `+(c===!1?"is not supported by the environment":"is not available in the build"));let a=n?s.length>1?`since : +`+s.map(bg).join(` +`):" "+bg(s[0]):"as no adapter specified";throw new H("There is no suitable adapter to dispatch the request "+a,"ERR_NOT_SUPPORT")}return o},adapters:Xa};function Qa(e){if(e.cancelToken&&e.cancelToken.throwIfRequested(),e.signal&&e.signal.aborted)throw new lr(null,e)}function xg(e){return Qa(e),e.headers=Me.from(e.headers),e.data=Ya.call(e,e.transformRequest),["post","put","patch"].indexOf(e.method)!==-1&&e.headers.setContentType("application/x-www-form-urlencoded",!1),yg.getAdapter(e.adapter||qr.adapter,e)(e).then(function(r){return Qa(e),r.data=Ya.call(e,e.transformResponse,r),r.headers=Me.from(r.headers),r},function(r){return ig(r)||(Qa(e),r&&r.response&&(r.response.data=Ya.call(e,e.transformResponse,r.response),r.response.headers=Me.from(r.response.headers))),Promise.reject(r)})}const kg="1.12.2",Ri={};["object","boolean","number","function","string","symbol"].forEach((e,t)=>{Ri[e]=function(r){return typeof r===e||"a"+(t<1?"n ":" ")+e}});const Cg={};Ri.transitional=function(t,n,r){function o(i,s){return"[Axios v"+kg+"] Transitional option '"+i+"'"+s+(r?". "+r:"")}return(i,s,a)=>{if(t===!1)throw new H(o(s," has been removed"+(n?" in "+n:"")),H.ERR_DEPRECATED);return n&&!Cg[s]&&(Cg[s]=!0,console.warn(o(s," has been deprecated since v"+n+" and will be removed in the near future"))),t?t(i,s,a):!0}},Ri.spelling=function(t){return(n,r)=>(console.warn(`${r} is likely a misspelling of ${t}`),!0)};function uP(e,t,n){if(typeof e!="object")throw new H("options must be an object",H.ERR_BAD_OPTION_VALUE);const r=Object.keys(e);let o=r.length;for(;o-- >0;){const i=r[o],s=t[i];if(s){const a=e[i],l=a===void 0||s(a,i,e);if(l!==!0)throw new H("option "+i+" must be "+l,H.ERR_BAD_OPTION_VALUE);continue}if(n!==!0)throw new H("Unknown option "+i,H.ERR_BAD_OPTION)}}const Ii={assertOptions:uP,validators:Ri},mt=Ii.validators;let Tn=class{constructor(t){this.defaults=t||{},this.interceptors={request:new tg,response:new tg}}async request(t,n){try{return await this._request(t,n)}catch(r){if(r instanceof Error){let o={};Error.captureStackTrace?Error.captureStackTrace(o):o=new Error;const i=o.stack?o.stack.replace(/^.+\n/,""):"";try{r.stack?i&&!String(r.stack).endsWith(i.replace(/^.+\n.+\n/,""))&&(r.stack+=` +`+i):r.stack=i}catch{}}throw r}}_request(t,n){typeof t=="string"?(n=n||{},n.url=t):n=t||{},n=In(this.defaults,n);const{transitional:r,paramsSerializer:o,headers:i}=n;r!==void 0&&Ii.assertOptions(r,{silentJSONParsing:mt.transitional(mt.boolean),forcedJSONParsing:mt.transitional(mt.boolean),clarifyTimeoutError:mt.transitional(mt.boolean)},!1),o!=null&&(E.isFunction(o)?n.paramsSerializer={serialize:o}:Ii.assertOptions(o,{encode:mt.function,serialize:mt.function},!0)),n.allowAbsoluteUrls!==void 0||(this.defaults.allowAbsoluteUrls!==void 0?n.allowAbsoluteUrls=this.defaults.allowAbsoluteUrls:n.allowAbsoluteUrls=!0),Ii.assertOptions(n,{baseUrl:mt.spelling("baseURL"),withXsrfToken:mt.spelling("withXSRFToken")},!0),n.method=(n.method||this.defaults.method||"get").toLowerCase();let s=i&&E.merge(i.common,i[n.method]);i&&E.forEach(["delete","get","head","post","put","patch","common"],f=>{delete i[f]}),n.headers=Me.concat(s,i);const a=[];let l=!0;this.interceptors.request.forEach(function(p){typeof p.runWhen=="function"&&p.runWhen(n)===!1||(l=l&&p.synchronous,a.unshift(p.fulfilled,p.rejected))});const c=[];this.interceptors.response.forEach(function(p){c.push(p.fulfilled,p.rejected)});let u,h=0,d;if(!l){const f=[xg.bind(this),void 0];for(f.unshift(...a),f.push(...c),d=f.length,u=Promise.resolve(n);h{if(!r._listeners)return;let i=r._listeners.length;for(;i-- >0;)r._listeners[i](o);r._listeners=null}),this.promise.then=o=>{let i;const s=new Promise(a=>{r.subscribe(a),i=a}).then(o);return s.cancel=function(){r.unsubscribe(i)},s},t(function(i,s,a){r.reason||(r.reason=new lr(i,s,a),n(r.reason))})}throwIfRequested(){if(this.reason)throw this.reason}subscribe(t){if(this.reason){t(this.reason);return}this._listeners?this._listeners.push(t):this._listeners=[t]}unsubscribe(t){if(!this._listeners)return;const n=this._listeners.indexOf(t);n!==-1&&this._listeners.splice(n,1)}toAbortSignal(){const t=new AbortController,n=r=>{t.abort(r)};return this.subscribe(n),t.signal.unsubscribe=()=>this.unsubscribe(n),t.signal}static source(){let t;return{token:new wp(function(o){t=o}),cancel:t}}};function hP(e){return function(n){return e.apply(null,n)}}function fP(e){return E.isObject(e)&&e.isAxiosError===!0}const Ja={Continue:100,SwitchingProtocols:101,Processing:102,EarlyHints:103,Ok:200,Created:201,Accepted:202,NonAuthoritativeInformation:203,NoContent:204,ResetContent:205,PartialContent:206,MultiStatus:207,AlreadyReported:208,ImUsed:226,MultipleChoices:300,MovedPermanently:301,Found:302,SeeOther:303,NotModified:304,UseProxy:305,Unused:306,TemporaryRedirect:307,PermanentRedirect:308,BadRequest:400,Unauthorized:401,PaymentRequired:402,Forbidden:403,NotFound:404,MethodNotAllowed:405,NotAcceptable:406,ProxyAuthenticationRequired:407,RequestTimeout:408,Conflict:409,Gone:410,LengthRequired:411,PreconditionFailed:412,PayloadTooLarge:413,UriTooLong:414,UnsupportedMediaType:415,RangeNotSatisfiable:416,ExpectationFailed:417,ImATeapot:418,MisdirectedRequest:421,UnprocessableEntity:422,Locked:423,FailedDependency:424,TooEarly:425,UpgradeRequired:426,PreconditionRequired:428,TooManyRequests:429,RequestHeaderFieldsTooLarge:431,UnavailableForLegalReasons:451,InternalServerError:500,NotImplemented:501,BadGateway:502,ServiceUnavailable:503,GatewayTimeout:504,HttpVersionNotSupported:505,VariantAlsoNegotiates:506,InsufficientStorage:507,LoopDetected:508,NotExtended:510,NetworkAuthenticationRequired:511};Object.entries(Ja).forEach(([e,t])=>{Ja[t]=e});function wg(e){const t=new Tn(e),n=Mf(Tn.prototype.request,t);return E.extend(n,Tn.prototype,t,{allOwnKeys:!0}),E.extend(n,t,null,{allOwnKeys:!0}),n.create=function(o){return wg(In(e,o))},n}const ce=wg(qr);ce.Axios=Tn,ce.CanceledError=lr,ce.CancelToken=dP,ce.isCancel=ig,ce.VERSION=kg,ce.toFormData=Si,ce.AxiosError=H,ce.Cancel=ce.CanceledError,ce.all=function(t){return Promise.all(t)},ce.spread=hP,ce.isAxiosError=fP,ce.mergeConfig=In,ce.AxiosHeaders=Me,ce.formToJSON=e=>rg(E.isHTMLForm(e)?new FormData(e):e),ce.getAdapter=yg.getAdapter,ce.HttpStatusCode=Ja,ce.default=ce;const{Axios:HT,AxiosError:UT,CanceledError:GT,isCancel:qT,CancelToken:KT,VERSION:YT,all:XT,Cancel:QT,isAxiosError:JT,spread:ZT,toFormData:e5,AxiosHeaders:t5,HttpStatusCode:n5,formToJSON:r5,getAdapter:o5,mergeConfig:i5}=ce;var gP=(e,t,n,r,o,i,s,a)=>{let l=document.documentElement,c=["light","dark"];function u(m){(Array.isArray(e)?e:[e]).forEach(f=>{let p=f==="class",v=p&&i?o.map(b=>i[b]||b):o;p?(l.classList.remove(...v),l.classList.add(i&&i[m]?i[m]:m)):l.setAttribute(f,m)}),h(m)}function h(m){a&&c.includes(m)&&(l.style.colorScheme=m)}function d(){return window.matchMedia("(prefers-color-scheme: dark)").matches?"dark":"light"}if(r)u(r);else try{let m=localStorage.getItem(t)||n,f=s&&m==="system"?d():m;u(f)}catch{}},Sg=["light","dark"],Eg="(prefers-color-scheme: dark)",pP=typeof window>"u",Og=C.createContext(void 0),mP=e=>C.useContext(Og)?C.createElement(C.Fragment,null,e.children):C.createElement(bP,{...e}),vP=["light","dark"],bP=({forcedTheme:e,disableTransitionOnChange:t=!1,enableSystem:n=!0,enableColorScheme:r=!0,storageKey:o="theme",themes:i=vP,defaultTheme:s=n?"system":"light",attribute:a="data-theme",value:l,children:c,nonce:u,scriptProps:h})=>{let[d,m]=C.useState(()=>xP(o,s)),[f,p]=C.useState(()=>d==="system"?Za():d),v=l?Object.values(l):i,b=C.useCallback(k=>{let O=k;if(!O)return;k==="system"&&n&&(O=Za());let R=l?l[O]:O,I=t?kP(u):null,N=document.documentElement,A=T=>{T==="class"?(N.classList.remove(...v),R&&N.classList.add(R)):T.startsWith("data-")&&(R?N.setAttribute(T,R):N.removeAttribute(T))};if(Array.isArray(a)?a.forEach(A):A(a),r){let T=Sg.includes(s)?s:null,S=Sg.includes(O)?O:T;N.style.colorScheme=S}I?.()},[u]),y=C.useCallback(k=>{let O=typeof k=="function"?k(d):k;m(O);try{localStorage.setItem(o,O)}catch{}},[d]),x=C.useCallback(k=>{let O=Za(k);p(O),d==="system"&&n&&!e&&b("system")},[d,e]);C.useEffect(()=>{let k=window.matchMedia(Eg);return k.addListener(x),x(k),()=>k.removeListener(x)},[x]),C.useEffect(()=>{let k=O=>{O.key===o&&(O.newValue?m(O.newValue):y(s))};return window.addEventListener("storage",k),()=>window.removeEventListener("storage",k)},[y]),C.useEffect(()=>{b(e??d)},[e,d]);let w=C.useMemo(()=>({theme:d,setTheme:y,forcedTheme:e,resolvedTheme:d==="system"?f:d,themes:n?[...i,"system"]:i,systemTheme:n?f:void 0}),[d,y,e,f,n,i]);return C.createElement(Og.Provider,{value:w},C.createElement(yP,{forcedTheme:e,storageKey:o,attribute:a,enableSystem:n,enableColorScheme:r,defaultTheme:s,value:l,themes:i,nonce:u,scriptProps:h}),c)},yP=C.memo(({forcedTheme:e,storageKey:t,attribute:n,enableSystem:r,enableColorScheme:o,defaultTheme:i,value:s,themes:a,nonce:l,scriptProps:c})=>{let u=JSON.stringify([n,t,i,e,a,s,r,o]).slice(1,-1);return C.createElement("script",{...c,suppressHydrationWarning:!0,nonce:typeof window>"u"?l:"",dangerouslySetInnerHTML:{__html:`(${gP.toString()})(${u})`}})}),xP=(e,t)=>{if(pP)return;let n;try{n=localStorage.getItem(e)||void 0}catch{}return n||t},kP=e=>{let t=document.createElement("style");return e&&t.setAttribute("nonce",e),t.appendChild(document.createTextNode("*,*::before,*::after{-webkit-transition:none!important;-moz-transition:none!important;-o-transition:none!important;-ms-transition:none!important;transition:none!important}")),document.head.appendChild(t),()=>{window.getComputedStyle(document.body),setTimeout(()=>{document.head.removeChild(t)},1)}},Za=e=>(e||(e=window.matchMedia(Eg)),e.matches?"dark":"light");const CP=e=>g.jsx(mP,{attribute:"class",disableTransitionOnChange:!0,...e});/** + * react-router v7.9.2 * * Copyright (c) Remix Software Inc. * @@ -35,25 +35,9 @@ * LICENSE.md file in the root directory of this source tree. * * @license MIT - */function qr(){return qr=Object.assign?Object.assign.bind():function(e){for(var t=1;t"u")throw new Error(t)}function yg(e,t){if(!e){typeof console<"u"&&console.warn(t);try{throw new Error(t)}catch{}}}function gP(){return Math.random().toString(36).substr(2,8)}function xg(e,t){return{usr:e.state,key:e.key,idx:t}}function Xa(e,t,n,r){return n===void 0&&(n=null),qr({pathname:typeof e=="string"?e:e.pathname,search:"",hash:""},typeof t=="string"?ar(t):t,{state:n,key:t&&t.key||r||gP()})}function Pi(e){let{pathname:t="/",search:n="",hash:r=""}=e;return n&&n!=="?"&&(t+=n.charAt(0)==="?"?n:"?"+n),r&&r!=="#"&&(t+=r.charAt(0)==="#"?r:"#"+r),t}function ar(e){let t={};if(e){let n=e.indexOf("#");n>=0&&(t.hash=e.substr(n),e=e.substr(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substr(r),e=e.substr(0,r)),e&&(t.pathname=e)}return t}function pP(e,t,n,r){r===void 0&&(r={});let{window:o=document.defaultView,v5Compat:i=!1}=r,s=o.history,a=Wt.Pop,l=null,c=u();c==null&&(c=0,s.replaceState(qr({},s.state,{idx:c}),""));function u(){return(s.state||{idx:null}).idx}function d(){a=Wt.Pop;let v=u(),y=v==null?null:v-c;c=v,l&&l({action:a,location:p.location,delta:y})}function h(v,y){a=Wt.Push;let x=Xa(p.location,v,y);c=u()+1;let b=xg(x,c),C=p.createHref(x);try{s.pushState(b,"",C)}catch(k){if(k instanceof DOMException&&k.name==="DataCloneError")throw k;o.location.assign(C)}i&&l&&l({action:a,location:p.location,delta:1})}function m(v,y){a=Wt.Replace;let x=Xa(p.location,v,y);c=u();let b=xg(x,c),C=p.createHref(x);s.replaceState(b,"",C),i&&l&&l({action:a,location:p.location,delta:0})}function g(v){let y=o.location.origin!=="null"?o.location.origin:o.location.href,x=typeof v=="string"?v:Pi(v);return x=x.replace(/ $/,"%20"),ce(y,"No window.location.(origin|href) available to create URL for href: "+x),new URL(x,y)}let p={get action(){return a},get location(){return e(o,s)},listen(v){if(l)throw new Error("A history only accepts one active listener");return o.addEventListener(bg,d),l=v,()=>{o.removeEventListener(bg,d),l=null}},createHref(v){return t(o,v)},createURL:g,encodeLocation(v){let y=g(v);return{pathname:y.pathname,search:y.search,hash:y.hash}},push:h,replace:m,go(v){return s.go(v)}};return p}var kg;(function(e){e.data="data",e.deferred="deferred",e.redirect="redirect",e.error="error"})(kg||(kg={}));function mP(e,t,n){return n===void 0&&(n="/"),vP(e,t,n)}function vP(e,t,n,r){let o=typeof t=="string"?ar(t):t,i=lr(o.pathname||"/",n);if(i==null)return null;let s=Cg(e);bP(s);let a=null;for(let l=0;a==null&&l{let l={relativePath:a===void 0?i.path||"":a,caseSensitive:i.caseSensitive===!0,childrenIndex:s,route:i};l.relativePath.startsWith("/")&&(ce(l.relativePath.startsWith(r),'Absolute route path "'+l.relativePath+'" nested under path '+('"'+r+'" is not valid. An absolute child route path ')+"must start with the combined path of all its parent routes."),l.relativePath=l.relativePath.slice(r.length));let c=Ht([r,l.relativePath]),u=n.concat(l);i.children&&i.children.length>0&&(ce(i.index!==!0,"Index routes must not have child routes. Please remove "+('all child routes from route path "'+c+'".')),Cg(i.children,t,u,c)),!(i.path==null&&!i.index)&&t.push({path:c,score:EP(c,i.index),routesMeta:u})};return e.forEach((i,s)=>{var a;if(i.path===""||!((a=i.path)!=null&&a.includes("?")))o(i,s);else for(let l of Sg(i.path))o(i,s,l)}),t}function Sg(e){let t=e.split("/");if(t.length===0)return[];let[n,...r]=t,o=n.endsWith("?"),i=n.replace(/\?$/,"");if(r.length===0)return o?[i,""]:[i];let s=Sg(r.join("/")),a=[];return a.push(...s.map(l=>l===""?i:[i,l].join("/"))),o&&a.push(...s),a.map(l=>e.startsWith("/")&&l===""?"/":l)}function bP(e){e.sort((t,n)=>t.score!==n.score?n.score-t.score:OP(t.routesMeta.map(r=>r.childrenIndex),n.routesMeta.map(r=>r.childrenIndex)))}const yP=/^:[\w-]+$/,xP=3,kP=2,CP=1,SP=10,wP=-2,wg=e=>e==="*";function EP(e,t){let n=e.split("/"),r=n.length;return n.some(wg)&&(r+=wP),t&&(r+=kP),n.filter(o=>!wg(o)).reduce((o,i)=>o+(yP.test(i)?xP:i===""?CP:SP),r)}function OP(e,t){return e.length===t.length&&e.slice(0,-1).every((r,o)=>r===t[o])?e[e.length-1]-t[t.length-1]:0}function PP(e,t,n){let{routesMeta:r}=e,o={},i="/",s=[];for(let a=0;a{let{paramName:h,isOptional:m}=u;if(h==="*"){let p=a[d]||"";s=i.slice(0,i.length-p.length).replace(/(.)\/+$/,"$1")}const g=a[d];return m&&!g?c[h]=void 0:c[h]=(g||"").replace(/%2F/g,"/"),c},{}),pathname:i,pathnameBase:s,pattern:e}}function IP(e,t,n){t===void 0&&(t=!1),n===void 0&&(n=!0),yg(e==="*"||!e.endsWith("*")||e.endsWith("/*"),'Route path "'+e+'" will be treated as if it were '+('"'+e.replace(/\*$/,"/*")+'" because the `*` character must ')+"always follow a `/` in the pattern. To get rid of this warning, "+('please change the route path to "'+e.replace(/\*$/,"/*")+'".'));let r=[],o="^"+e.replace(/\/*\*?$/,"").replace(/^\/*/,"/").replace(/[\\.*+^${}|()[\]]/g,"\\$&").replace(/\/:([\w-]+)(\?)?/g,(s,a,l)=>(r.push({paramName:a,isOptional:l!=null}),l?"/?([^\\/]+)?":"/([^\\/]+)"));return e.endsWith("*")?(r.push({paramName:"*"}),o+=e==="*"||e==="/*"?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?o+="\\/*$":e!==""&&e!=="/"&&(o+="(?:(?=\\/|$))"),[new RegExp(o,t?void 0:"i"),r]}function RP(e){try{return e.split("/").map(t=>decodeURIComponent(t).replace(/\//g,"%2F")).join("/")}catch(t){return yg(!1,'The URL path "'+e+'" could not be decoded because it is is a malformed URL segment. This is probably due to a bad percent '+("encoding ("+t+").")),e}}function lr(e,t){if(t==="/")return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&r!=="/"?null:e.slice(n)||"/"}function TP(e,t){t===void 0&&(t="/");let{pathname:n,search:r="",hash:o=""}=typeof e=="string"?ar(e):e;return{pathname:n?n.startsWith("/")?n:NP(n,t):t,search:VP(r),hash:FP(o)}}function NP(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach(o=>{o===".."?n.length>1&&n.pop():o!=="."&&n.push(o)}),n.length>1?n.join("/"):"/"}function Ja(e,t,n,r){return"Cannot include a '"+e+"' character in a manually specified "+("`to."+t+"` field ["+JSON.stringify(r)+"]. Please separate it out to the ")+("`to."+n+"` field. Alternatively you may provide the full path as ")+'a string in and the router will parse it for you.'}function AP(e){return e.filter((t,n)=>n===0||t.route.path&&t.route.path.length>0)}function Eg(e,t){let n=AP(e);return t?n.map((r,o)=>o===n.length-1?r.pathname:r.pathnameBase):n.map(r=>r.pathnameBase)}function Og(e,t,n,r){r===void 0&&(r=!1);let o;typeof e=="string"?o=ar(e):(o=qr({},e),ce(!o.pathname||!o.pathname.includes("?"),Ja("?","pathname","search",o)),ce(!o.pathname||!o.pathname.includes("#"),Ja("#","pathname","hash",o)),ce(!o.search||!o.search.includes("#"),Ja("#","search","hash",o)));let i=e===""||o.pathname==="",s=i?"/":o.pathname,a;if(s==null)a=n;else{let d=t.length-1;if(!r&&s.startsWith("..")){let h=s.split("/");for(;h[0]==="..";)h.shift(),d-=1;o.pathname=h.join("/")}a=d>=0?t[d]:"/"}let l=TP(o,a),c=s&&s!=="/"&&s.endsWith("/"),u=(i||s===".")&&n.endsWith("/");return!l.pathname.endsWith("/")&&(c||u)&&(l.pathname+="/"),l}const Ht=e=>e.join("/").replace(/\/\/+/g,"/"),_P=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),VP=e=>!e||e==="?"?"":e.startsWith("?")?e:"?"+e,FP=e=>!e||e==="#"?"":e.startsWith("#")?e:"#"+e;function LP(e){return e!=null&&typeof e.status=="number"&&typeof e.statusText=="string"&&typeof e.internal=="boolean"&&"data"in e}const Pg=["post","put","patch","delete"];new Set(Pg);const DP=["get",...Pg];new Set(DP);/** - * React Router v6.30.1 - * - * Copyright (c) Remix Software Inc. - * - * This source code is licensed under the MIT license found in the - * LICENSE.md file in the root directory of this source tree. - * - * @license MIT - */function Kr(){return Kr=Object.assign?Object.assign.bind():function(e){for(var t=1;t{a.current=!0}),O.useCallback(function(c,u){if(u===void 0&&(u={}),!a.current)return;if(typeof c=="number"){r.go(c);return}let d=Og(c,JSON.parse(s),i,u.relative==="path");e==null&&t!=="/"&&(d.pathname=d.pathname==="/"?t:Ht([t,d.pathname])),(u.replace?r.replace:r.push)(d,u.state,u)},[t,r,s,i,e])}function Ti(e,t){let{relative:n}=t===void 0?{}:t,{future:r}=O.useContext(Ut),{matches:o}=O.useContext(Rn),{pathname:i}=Xr(),s=JSON.stringify(Eg(o,r.v7_relativeSplatPath));return O.useMemo(()=>Og(e,JSON.parse(s),i,n==="path"),[e,s,i,n])}function $P(e,t){return BP(e,t)}function BP(e,t,n,r){Yr()||ce(!1);let{navigator:o}=O.useContext(Ut),{matches:i}=O.useContext(Rn),s=i[i.length-1],a=s?s.params:{};s&&s.pathname;let l=s?s.pathnameBase:"/";s&&s.route;let c=Xr(),u;if(t){var d;let v=typeof t=="string"?ar(t):t;l==="/"||(d=v.pathname)!=null&&d.startsWith(l)||ce(!1),u=v}else u=c;let h=u.pathname||"/",m=h;if(l!=="/"){let v=l.replace(/^\//,"").split("/");m="/"+h.replace(/^\//,"").split("/").slice(v.length).join("/")}let g=mP(e,{pathname:m}),p=qP(g&&g.map(v=>Object.assign({},v,{params:Object.assign({},a,v.params),pathname:Ht([l,o.encodeLocation?o.encodeLocation(v.pathname).pathname:v.pathname]),pathnameBase:v.pathnameBase==="/"?l:Ht([l,o.encodeLocation?o.encodeLocation(v.pathnameBase).pathname:v.pathnameBase])})),i,n,r);return t&&p?O.createElement(Ri.Provider,{value:{location:Kr({pathname:"/",search:"",hash:"",state:null,key:"default"},u),navigationType:Wt.Pop}},p):p}function WP(){let e=QP(),t=LP(e)?e.status+" "+e.statusText:e instanceof Error?e.message:JSON.stringify(e),n=e instanceof Error?e.stack:null,o={padding:"0.5rem",backgroundColor:"rgba(200,200,200, 0.5)"};return O.createElement(O.Fragment,null,O.createElement("h2",null,"Unexpected Application Error!"),O.createElement("h3",{style:{fontStyle:"italic"}},t),n?O.createElement("pre",{style:o},n):null,null)}const HP=O.createElement(WP,null);class UP extends O.Component{constructor(t){super(t),this.state={location:t.location,revalidation:t.revalidation,error:t.error}}static getDerivedStateFromError(t){return{error:t}}static getDerivedStateFromProps(t,n){return n.location!==t.location||n.revalidation!=="idle"&&t.revalidation==="idle"?{error:t.error,location:t.location,revalidation:t.revalidation}:{error:t.error!==void 0?t.error:n.error,location:n.location,revalidation:t.revalidation||n.revalidation}}componentDidCatch(t,n){console.error("React Router caught the following error during render",t,n)}render(){return this.state.error!==void 0?O.createElement(Rn.Provider,{value:this.props.routeContext},O.createElement(Rg.Provider,{value:this.state.error,children:this.props.component})):this.props.children}}function GP(e){let{routeContext:t,match:n,children:r}=e,o=O.useContext(Ii);return o&&o.static&&o.staticContext&&(n.route.errorElement||n.route.ErrorBoundary)&&(o.staticContext._deepestRenderedBoundaryId=n.route.id),O.createElement(Rn.Provider,{value:t},r)}function qP(e,t,n,r){var o;if(t===void 0&&(t=[]),n===void 0&&(n=null),r===void 0&&(r=null),e==null){var i;if(!n)return null;if(n.errors)e=n.matches;else if((i=r)!=null&&i.v7_partialHydration&&t.length===0&&!n.initialized&&n.matches.length>0)e=n.matches;else return null}let s=e,a=(o=n)==null?void 0:o.errors;if(a!=null){let u=s.findIndex(d=>d.route.id&&a?.[d.route.id]!==void 0);u>=0||ce(!1),s=s.slice(0,Math.min(s.length,u+1))}let l=!1,c=-1;if(n&&r&&r.v7_partialHydration)for(let u=0;u=0?s=s.slice(0,c+1):s=[s[0]];break}}}return s.reduceRight((u,d,h)=>{let m,g=!1,p=null,v=null;n&&(m=a&&d.route.id?a[d.route.id]:void 0,p=d.route.errorElement||HP,l&&(c<0&&h===0?(ZP("route-fallback"),g=!0,v=null):c===h&&(g=!0,v=d.route.hydrateFallbackElement||null)));let y=t.concat(s.slice(0,h+1)),x=()=>{let b;return m?b=p:g?b=v:d.route.Component?b=O.createElement(d.route.Component,null):d.route.element?b=d.route.element:b=u,O.createElement(GP,{match:d,routeContext:{outlet:u,matches:y,isDataRoute:n!=null},children:b})};return n&&(d.route.ErrorBoundary||d.route.errorElement||h===0)?O.createElement(UP,{location:n.location,revalidation:n.revalidation,component:p,error:m,children:x(),routeContext:{outlet:null,matches:y,isDataRoute:!0}}):x()},null)}var Ng=(function(e){return e.UseBlocker="useBlocker",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e})(Ng||{}),Ag=(function(e){return e.UseBlocker="useBlocker",e.UseLoaderData="useLoaderData",e.UseActionData="useActionData",e.UseRouteError="useRouteError",e.UseNavigation="useNavigation",e.UseRouteLoaderData="useRouteLoaderData",e.UseMatches="useMatches",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e.UseRouteId="useRouteId",e})(Ag||{});function KP(e){let t=O.useContext(Ii);return t||ce(!1),t}function YP(e){let t=O.useContext(Ig);return t||ce(!1),t}function XP(e){let t=O.useContext(Rn);return t||ce(!1),t}function _g(e){let t=XP(),n=t.matches[t.matches.length-1];return n.route.id||ce(!1),n.route.id}function QP(){var e;let t=O.useContext(Rg),n=YP(),r=_g();return t!==void 0?t:(e=n.errors)==null?void 0:e[r]}function JP(){let{router:e}=KP(Ng.UseNavigateStable),t=_g(Ag.UseNavigateStable),n=O.useRef(!1);return Tg(()=>{n.current=!0}),O.useCallback(function(o,i){i===void 0&&(i={}),n.current&&(typeof o=="number"?e.navigate(o):e.navigate(o,Kr({fromRouteId:t},i)))},[e,t])}const Vg={};function ZP(e,t,n){Vg[e]||(Vg[e]=!0)}function eI(e,t){e?.v7_startTransition,e?.v7_relativeSplatPath}function Za(e){ce(!1)}function tI(e){let{basename:t="/",children:n=null,location:r,navigationType:o=Wt.Pop,navigator:i,static:s=!1,future:a}=e;Yr()&&ce(!1);let l=t.replace(/^\/*/,"/"),c=O.useMemo(()=>({basename:l,navigator:i,static:s,future:Kr({v7_relativeSplatPath:!1},a)}),[l,a,i,s]);typeof r=="string"&&(r=ar(r));let{pathname:u="/",search:d="",hash:h="",state:m=null,key:g="default"}=r,p=O.useMemo(()=>{let v=lr(u,l);return v==null?null:{location:{pathname:v,search:d,hash:h,state:m,key:g},navigationType:o}},[l,u,d,h,m,g,o]);return p==null?null:O.createElement(Ut.Provider,{value:c},O.createElement(Ri.Provider,{children:n,value:p}))}function nI(e){let{children:t,location:n}=e;return $P(el(t),n)}new Promise(()=>{});function el(e,t){t===void 0&&(t=[]);let n=[];return O.Children.forEach(e,(r,o)=>{if(!O.isValidElement(r))return;let i=[...t,o];if(r.type===O.Fragment){n.push.apply(n,el(r.props.children,i));return}r.type!==Za&&ce(!1),!r.props.index||!r.props.children||ce(!1);let s={id:r.props.id||i.join("-"),caseSensitive:r.props.caseSensitive,element:r.props.element,Component:r.props.Component,index:r.props.index,path:r.props.path,loader:r.props.loader,action:r.props.action,errorElement:r.props.errorElement,ErrorBoundary:r.props.ErrorBoundary,hasErrorBoundary:r.props.ErrorBoundary!=null||r.props.errorElement!=null,shouldRevalidate:r.props.shouldRevalidate,handle:r.props.handle,lazy:r.props.lazy};r.props.children&&(s.children=el(r.props.children,i)),n.push(s)}),n}/** - * React Router DOM v6.30.1 - * - * Copyright (c) Remix Software Inc. - * - * This source code is licensed under the MIT license found in the - * LICENSE.md file in the root directory of this source tree. - * - * @license MIT - */function Ni(){return Ni=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0)&&(n[o]=e[o]);return n}function rI(e){return!!(e.metaKey||e.altKey||e.ctrlKey||e.shiftKey)}function oI(e,t){return e.button===0&&(!t||t==="_self")&&!rI(e)}const iI=["onClick","relative","reloadDocument","replace","state","target","to","preventScrollReset","viewTransition"],sI=["aria-current","caseSensitive","className","end","style","to","viewTransition","children"],aI="6";try{window.__reactRouterVersion=aI}catch{}const lI=O.createContext({isTransitioning:!1}),Lg=O["startTransition"];function cI(e){let{basename:t,children:n,future:r,window:o}=e,i=O.useRef();i.current==null&&(i.current=fP({window:o,v5Compat:!0}));let s=i.current,[a,l]=O.useState({action:s.action,location:s.location}),{v7_startTransition:c}=r||{},u=O.useCallback(d=>{c&&Lg?Lg(()=>l(d)):l(d)},[l,c]);return O.useLayoutEffect(()=>s.listen(u),[s,u]),O.useEffect(()=>eI(r),[r]),O.createElement(tI,{basename:t,children:n,location:a.location,navigationType:a.action,navigator:s,future:r})}const uI=typeof window<"u"&&typeof window.document<"u"&&typeof window.document.createElement<"u",dI=/^(?:[a-z][a-z0-9+.-]*:|\/\/)/i,Dg=O.forwardRef(function(t,n){let{onClick:r,relative:o,reloadDocument:i,replace:s,state:a,target:l,to:c,preventScrollReset:u,viewTransition:d}=t,h=Fg(t,iI),{basename:m}=O.useContext(Ut),g,p=!1;if(typeof c=="string"&&dI.test(c)&&(g=c,uI))try{let b=new URL(window.location.href),C=c.startsWith("//")?new URL(b.protocol+c):new URL(c),k=lr(C.pathname,m);C.origin===b.origin&&k!=null?c=k+C.search+C.hash:p=!0}catch{}let v=zP(c,{relative:o}),y=gI(c,{replace:s,state:a,target:l,preventScrollReset:u,relative:o,viewTransition:d});function x(b){r&&r(b),b.defaultPrevented||y(b)}return O.createElement("a",Ni({},h,{href:g||v,onClick:p||i?r:x,ref:n,target:l}))}),hI=O.forwardRef(function(t,n){let{"aria-current":r="page",caseSensitive:o=!1,className:i="",end:s=!1,style:a,to:l,viewTransition:c,children:u}=t,d=Fg(t,sI),h=Ti(l,{relative:d.relative}),m=Xr(),g=O.useContext(Ig),{navigator:p,basename:v}=O.useContext(Ut),y=g!=null&&pI(h)&&c===!0,x=p.encodeLocation?p.encodeLocation(h).pathname:h.pathname,b=m.pathname,C=g&&g.navigation&&g.navigation.location?g.navigation.location.pathname:null;o||(b=b.toLowerCase(),C=C?C.toLowerCase():null,x=x.toLowerCase()),C&&v&&(C=lr(C,v)||C);const k=x!=="/"&&x.endsWith("/")?x.length-1:x.length;let E=b===x||!s&&b.startsWith(x)&&b.charAt(k)==="/",I=C!=null&&(C===x||!s&&C.startsWith(x)&&C.charAt(x.length)==="/"),R={isActive:E,isPending:I,isTransitioning:y},_=E?r:void 0,A;typeof i=="function"?A=i(R):A=[i,E?"active":null,I?"pending":null,y?"transitioning":null].filter(Boolean).join(" ");let T=typeof a=="function"?a(R):a;return O.createElement(Dg,Ni({},d,{"aria-current":_,className:A,ref:n,style:T,to:l,viewTransition:c}),typeof u=="function"?u(R):u)});var tl;(function(e){e.UseScrollRestoration="useScrollRestoration",e.UseSubmit="useSubmit",e.UseSubmitFetcher="useSubmitFetcher",e.UseFetcher="useFetcher",e.useViewTransitionState="useViewTransitionState"})(tl||(tl={}));var zg;(function(e){e.UseFetcher="useFetcher",e.UseFetchers="useFetchers",e.UseScrollRestoration="useScrollRestoration"})(zg||(zg={}));function fI(e){let t=O.useContext(Ii);return t||ce(!1),t}function gI(e,t){let{target:n,replace:r,state:o,preventScrollReset:i,relative:s,viewTransition:a}=t===void 0?{}:t,l=MP(),c=Xr(),u=Ti(e,{relative:s});return O.useCallback(d=>{if(oI(d,n)){d.preventDefault();let h=r!==void 0?r:Pi(c)===Pi(u);l(e,{replace:h,state:o,preventScrollReset:i,relative:s,viewTransition:a})}},[c,l,u,r,o,n,e,i,s,a])}function pI(e,t){t===void 0&&(t={});let n=O.useContext(lI);n==null&&ce(!1);let{basename:r}=fI(tl.useViewTransitionState),o=Ti(e,{relative:t.relative});if(!n.isTransitioning)return!1;let i=lr(n.currentLocation.pathname,r)||n.currentLocation.pathname,s=lr(n.nextLocation.pathname,r)||n.nextLocation.pathname;return Qa(o.pathname,s)!=null||Qa(o.pathname,i)!=null}const mI="UiServiceWorker",vI=e=>[mI],bI="UiServiceJobs",yI=e=>[bI];class Mg{_fns;constructor(){this._fns=[]}eject(t){const n=this._fns.indexOf(t);n!==-1&&(this._fns=[...this._fns.slice(0,n),...this._fns.slice(n+1)])}use(t){this._fns=[...this._fns,t]}}const Et={BASE:"",CREDENTIALS:"include",ENCODE_PATH:void 0,HEADERS:void 0,PASSWORD:void 0,TOKEN:void 0,USERNAME:void 0,VERSION:"0.1.0",WITH_CREDENTIALS:!1,interceptors:{request:new Mg,response:new Mg}};class jg extends Error{url;status;statusText;body;request;constructor(t,n,r){super(r),this.name="ApiError",this.url=n.url,this.status=n.status,this.statusText=n.statusText,this.body=n.body,this.request=t}}class xI extends Error{constructor(t){super(t),this.name="CancelError"}get isCancelled(){return!0}}class kI{_isResolved;_isRejected;_isCancelled;cancelHandlers;promise;_resolve;_reject;constructor(t){this._isResolved=!1,this._isRejected=!1,this._isCancelled=!1,this.cancelHandlers=[],this.promise=new Promise((n,r)=>{this._resolve=n,this._reject=r;const o=a=>{this._isResolved||this._isRejected||this._isCancelled||(this._isResolved=!0,this._resolve&&this._resolve(a))},i=a=>{this._isResolved||this._isRejected||this._isCancelled||(this._isRejected=!0,this._reject&&this._reject(a))},s=a=>{this._isResolved||this._isRejected||this._isCancelled||this.cancelHandlers.push(a)};return Object.defineProperty(s,"isResolved",{get:()=>this._isResolved}),Object.defineProperty(s,"isRejected",{get:()=>this._isRejected}),Object.defineProperty(s,"isCancelled",{get:()=>this._isCancelled}),t(o,i,s)})}get[Symbol.toStringTag](){return"Cancellable Promise"}then(t,n){return this.promise.then(t,n)}catch(t){return this.promise.catch(t)}finally(t){return this.promise.finally(t)}cancel(){if(!(this._isResolved||this._isRejected||this._isCancelled)){if(this._isCancelled=!0,this.cancelHandlers.length)try{for(const t of this.cancelHandlers)t()}catch(t){console.warn("Cancellation threw an error",t);return}this.cancelHandlers.length=0,this._reject&&this._reject(new xI("Request aborted"))}}get isCancelled(){return this._isCancelled}}const Ai=e=>typeof e=="string",nl=e=>Ai(e)&&e!=="",$g=e=>e instanceof Blob,CI=e=>e instanceof FormData,Bg=e=>e>=200&&e<300,SI=e=>{try{return btoa(e)}catch{return Buffer.from(e).toString("base64")}},wI=e=>{const t=[],n=(o,i)=>{t.push(`${encodeURIComponent(o)}=${encodeURIComponent(String(i))}`)},r=(o,i)=>{i!=null&&(i instanceof Date?n(o,i.toISOString()):Array.isArray(i)?i.forEach(s=>r(o,s)):typeof i=="object"?Object.entries(i).forEach(([s,a])=>r(`${o}[${s}]`,a)):n(o,i))};return Object.entries(e).forEach(([o,i])=>r(o,i)),t.length?`?${t.join("&")}`:""},EI=(e,t)=>{const n=encodeURI,r=t.url.replace("{api-version}",e.VERSION).replace(/{(.*?)}/g,(i,s)=>t.path?.hasOwnProperty(s)?n(String(t.path[s])):i),o=e.BASE+r;return t.query?o+wI(t.query):o},OI=e=>{if(e.formData){const t=new FormData,n=(r,o)=>{Ai(o)||$g(o)?t.append(r,o):t.append(r,JSON.stringify(o))};return Object.entries(e.formData).filter(([,r])=>r!=null).forEach(([r,o])=>{Array.isArray(o)?o.forEach(i=>n(r,i)):n(r,o)}),t}},_i=async(e,t)=>t,PI=async(e,t)=>{const[n,r,o,i]=await Promise.all([_i(t,e.TOKEN),_i(t,e.USERNAME),_i(t,e.PASSWORD),_i(t,e.HEADERS)]),s=Object.entries({Accept:"application/json",...i,...t.headers}).filter(([,a])=>a!=null).reduce((a,[l,c])=>({...a,[l]:String(c)}),{});if(nl(n)&&(s.Authorization=`Bearer ${n}`),nl(r)&&nl(o)){const a=SI(`${r}:${o}`);s.Authorization=`Basic ${a}`}return t.body!==void 0?t.mediaType?s["Content-Type"]=t.mediaType:$g(t.body)?s["Content-Type"]=t.body.type||"application/octet-stream":Ai(t.body)?s["Content-Type"]="text/plain":CI(t.body)||(s["Content-Type"]="application/json"):t.formData!==void 0&&t.mediaType&&(s["Content-Type"]=t.mediaType),s},II=e=>{if(e.body)return e.body},RI=async(e,t,n,r,o,i,s,a)=>{const l=new AbortController;let c={data:r??o,headers:i,method:t.method,signal:l.signal,url:n,withCredentials:e.WITH_CREDENTIALS};s(()=>l.abort());for(const u of e.interceptors.request._fns)c=await u(c);try{return await a.request(c)}catch(u){const d=u;if(d.response)return d.response;throw u}},TI=(e,t)=>{if(t){const n=e.headers[t];if(Ai(n))return n}},NI=e=>{if(e.status!==204)return e.data},AI=(e,t)=>{const r={400:"Bad Request",401:"Unauthorized",402:"Payment Required",403:"Forbidden",404:"Not Found",405:"Method Not Allowed",406:"Not Acceptable",407:"Proxy Authentication Required",408:"Request Timeout",409:"Conflict",410:"Gone",411:"Length Required",412:"Precondition Failed",413:"Payload Too Large",414:"URI Too Long",415:"Unsupported Media Type",416:"Range Not Satisfiable",417:"Expectation Failed",418:"Im a teapot",421:"Misdirected Request",422:"Unprocessable Content",423:"Locked",424:"Failed Dependency",425:"Too Early",426:"Upgrade Required",428:"Precondition Required",429:"Too Many Requests",431:"Request Header Fields Too Large",451:"Unavailable For Legal Reasons",500:"Internal Server Error",501:"Not Implemented",502:"Bad Gateway",503:"Service Unavailable",504:"Gateway Timeout",505:"HTTP Version Not Supported",506:"Variant Also Negotiates",507:"Insufficient Storage",508:"Loop Detected",510:"Not Extended",511:"Network Authentication Required",...e.errors}[t.status];if(r)throw new jg(e,t,r);if(!t.ok){const o=t.status??"unknown",i=t.statusText??"unknown",s=(()=>{try{return JSON.stringify(t.body,null,2)}catch{return}})();throw new jg(e,t,`Generic Error: status: ${o}; status text: ${i}; body: ${s}`)}},Ot=(e,t,n=le)=>new kI(async(r,o,i)=>{try{const s=EI(e,t),a=OI(t),l=II(t),c=await PI(e,t);if(!i.isCancelled){let u=await RI(e,t,s,l,a,c,i,n);for(const p of e.interceptors.response._fns)u=await p(u);const d=NI(u),h=TI(u,t.responseHeader);let m=d;t.responseTransformer&&Bg(u.status)&&(m=await t.responseTransformer(d));const g={url:s,ok:Bg(u.status),status:u.status,statusText:u.statusText,body:h??m};AI(t,g),r(g.body)}}catch(s){o(s)}});class Pt{static worker(){return Ot(Et,{method:"GET",url:"/edge_worker/ui/worker"})}static jobs(){return Ot(Et,{method:"GET",url:"/edge_worker/ui/jobs"})}static requestWorkerMaintenance(t){return Ot(Et,{method:"POST",url:"/edge_worker/ui/worker/{worker_name}/maintenance",path:{worker_name:t.workerName},body:t.requestBody,mediaType:"application/json",errors:{422:"Validation Error"}})}static updateWorkerMaintenance(t){return Ot(Et,{method:"PATCH",url:"/edge_worker/ui/worker/{worker_name}/maintenance",path:{worker_name:t.workerName},body:t.requestBody,mediaType:"application/json",errors:{422:"Validation Error"}})}static exitWorkerMaintenance(t){return Ot(Et,{method:"DELETE",url:"/edge_worker/ui/worker/{worker_name}/maintenance",path:{worker_name:t.workerName},errors:{422:"Validation Error"}})}static requestWorkerShutdown(t){return Ot(Et,{method:"POST",url:"/edge_worker/ui/worker/{worker_name}/shutdown",path:{worker_name:t.workerName},errors:{422:"Validation Error"}})}static deleteWorker(t){return Ot(Et,{method:"DELETE",url:"/edge_worker/ui/worker/{worker_name}",path:{worker_name:t.workerName},errors:{422:"Validation Error"}})}static addWorkerQueue(t){return Ot(Et,{method:"PUT",url:"/edge_worker/ui/worker/{worker_name}/queues/{queue_name}",path:{worker_name:t.workerName,queue_name:t.queueName},errors:{422:"Validation Error"}})}static removeWorkerQueue(t){return Ot(Et,{method:"DELETE",url:"/edge_worker/ui/worker/{worker_name}/queues/{queue_name}",path:{worker_name:t.workerName,queue_name:t.queueName},errors:{422:"Validation Error"}})}}const _I=(e,t)=>Rf({queryKey:vI(),queryFn:()=>Pt.worker(),...t}),VI=(e,t)=>Rf({queryKey:yI(),queryFn:()=>Pt.jobs(),...t}),FI=e=>En({mutationFn:({requestBody:t,workerName:n})=>Pt.requestWorkerMaintenance({requestBody:t,workerName:n}),...e}),LI=e=>En({mutationFn:({workerName:t})=>Pt.requestWorkerShutdown({workerName:t}),...e}),DI=e=>En({mutationFn:({queueName:t,workerName:n})=>Pt.addWorkerQueue({queueName:t,workerName:n}),...e}),zI=e=>En({mutationFn:({requestBody:t,workerName:n})=>Pt.updateWorkerMaintenance({requestBody:t,workerName:n}),...e}),MI=e=>En({mutationFn:({workerName:t})=>Pt.exitWorkerMaintenance({workerName:t}),...e}),jI=e=>En({mutationFn:({workerName:t})=>Pt.deleteWorker({workerName:t}),...e}),$I=e=>En({mutationFn:({queueName:t,workerName:n})=>Pt.removeWorkerQueue({queueName:t,workerName:n}),...e});function Vi(e){const t=new Date(e);if(!Number.isNaN(t.valueOf()))return t;const n=String(e).match(/\d+/g);if(n==null||n.length<=2)return t;{const[r,o,...i]=n.map(l=>parseInt(l)),s=[r,o-1,...i];return new Date(Date.UTC(...s))}}const Qr=(e,t,n)=>{const r=e!==1?t+"s":t;return e+" "+r+" "+n};function rl(){return rl=Object.assign?Object.assign.bind():function(e){for(var t=1;tDate.now();function Li({date:e,formatter:t,component:n="time",live:r=!0,minPeriod:o=0,maxPeriod:i=Fi,title:s,now:a=BI,...l}){const[c,u]=P.useState(a());P.useEffect(()=>{if(!r)return;const I=(()=>{const R=Vi(e).valueOf();if(!R)return console.warn("[react-timeago] Invalid Date provided"),0;const _=Math.round(Math.abs(c-R)/1e3),A=_{u(a())},T):0})();return()=>{I&&clearTimeout(I)}},[e,r,i,o,a,c]),P.useEffect(()=>{u(a())},[e]);const d=n,h=Vi(e).valueOf();if(!h)return null;const m=Math.round(Math.abs(c-h)/1e3),g=h"u"?typeof e=="string"?e:Vi(e).toISOString().substring(0,16).replace("T"," "):s,x=d==="time"?{...l,dateTime:Vi(e).toISOString()}:l,b=(E=E,I=I,R=R,_=h,A=Qr,T=T)=>Qr(E,I,R),C=t||Qr;let k;try{k=C(p,v,g,h,b,a),k||(k=Qr(p,v,g,h,b,a))}catch(E){console.error("[react-timeago] Formatter threw an error:",E),k=Qr(p,v,g)}return O.createElement(d,rl({},x,{title:y}),k)}var Ug={color:void 0,size:void 0,className:void 0,style:void 0,attr:void 0},Gg=P.createContext&&P.createContext(Ug),WI=["attr","size","title"];function HI(e,t){if(e==null)return{};var n=UI(e,t),r,o;if(Object.getOwnPropertySymbols){var i=Object.getOwnPropertySymbols(e);for(o=0;o=0)&&Object.prototype.propertyIsEnumerable.call(e,r)&&(n[r]=e[r])}return n}function UI(e,t){if(e==null)return{};var n={};for(var r in e)if(Object.prototype.hasOwnProperty.call(e,r)){if(t.indexOf(r)>=0)continue;n[r]=e[r]}return n}function Di(){return Di=Object.assign?Object.assign.bind():function(e){for(var t=1;tP.createElement(t.tag,zi({key:n},t.attr),Kg(t.child)))}function re(e){return t=>P.createElement(YI,Di({attr:zi({},e.attr)},t),Kg(e.child))}function YI(e){var t=n=>{var{attr:r,size:o,title:i}=e,s=HI(e,WI),a=o||n.size||"1em",l;return n.className&&(l=n.className),e.className&&(l=(l?l+" ":"")+e.className),P.createElement("svg",Di({stroke:"currentColor",fill:"currentColor",strokeWidth:"0"},n.attr,r,s,{className:l,style:zi(zi({color:e.color||n.color},n.style),e.style),height:a,width:a,xmlns:"http://www.w3.org/2000/svg"}),i&&P.createElement("title",null,i),e.children)};return Gg!==void 0?P.createElement(Gg.Consumer,null,n=>t(n)):t(Ug)}function XI(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M11 10v4h4"},child:[]},{tag:"path",attr:{d:"m11 14 1.535-1.605a5 5 0 0 1 8 1.5"},child:[]},{tag:"path",attr:{d:"M16 2v4"},child:[]},{tag:"path",attr:{d:"m21 18-1.535 1.605a5 5 0 0 1-8-1.5"},child:[]},{tag:"path",attr:{d:"M21 22v-4h-4"},child:[]},{tag:"path",attr:{d:"M21 8.5V6a2 2 0 0 0-2-2H5a2 2 0 0 0-2 2v14a2 2 0 0 0 2 2h4.3"},child:[]},{tag:"path",attr:{d:"M3 10h4"},child:[]},{tag:"path",attr:{d:"M8 2v4"},child:[]}]})(e)}function QI(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M20 6 9 17l-5-5"},child:[]}]})(e)}function Yg(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M10.1 2.182a10 10 0 0 1 3.8 0"},child:[]},{tag:"path",attr:{d:"M13.9 21.818a10 10 0 0 1-3.8 0"},child:[]},{tag:"path",attr:{d:"M17.609 3.721a10 10 0 0 1 2.69 2.7"},child:[]},{tag:"path",attr:{d:"M2.182 13.9a10 10 0 0 1 0-3.8"},child:[]},{tag:"path",attr:{d:"M20.279 17.609a10 10 0 0 1-2.7 2.69"},child:[]},{tag:"path",attr:{d:"M21.818 10.1a10 10 0 0 1 0 3.8"},child:[]},{tag:"path",attr:{d:"M3.721 6.391a10 10 0 0 1 2.7-2.69"},child:[]},{tag:"path",attr:{d:"M6.391 20.279a10 10 0 0 1-2.69-2.7"},child:[]}]})(e)}function JI(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M12 2a10 10 0 0 1 7.38 16.75"},child:[]},{tag:"path",attr:{d:"m16 12-4-4-4 4"},child:[]},{tag:"path",attr:{d:"M12 16V8"},child:[]},{tag:"path",attr:{d:"M2.5 8.875a10 10 0 0 0-.5 3"},child:[]},{tag:"path",attr:{d:"M2.83 16a10 10 0 0 0 2.43 3.4"},child:[]},{tag:"path",attr:{d:"M4.636 5.235a10 10 0 0 1 .891-.857"},child:[]},{tag:"path",attr:{d:"M8.644 21.42a10 10 0 0 0 7.631-.38"},child:[]}]})(e)}function ZI(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M15 3h6v6"},child:[]},{tag:"path",attr:{d:"M10 14 21 3"},child:[]},{tag:"path",attr:{d:"M18 13v6a2 2 0 0 1-2 2H5a2 2 0 0 1-2-2V8a2 2 0 0 1 2-2h6"},child:[]}]})(e)}function Xg(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M11 12H3"},child:[]},{tag:"path",attr:{d:"M16 6H3"},child:[]},{tag:"path",attr:{d:"M16 18H3"},child:[]},{tag:"path",attr:{d:"M21 12h-6"},child:[]}]})(e)}function Qg(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M11 12H3"},child:[]},{tag:"path",attr:{d:"M16 6H3"},child:[]},{tag:"path",attr:{d:"M16 18H3"},child:[]},{tag:"path",attr:{d:"M18 9v6"},child:[]},{tag:"path",attr:{d:"M21 12h-6"},child:[]}]})(e)}function eR(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"m15 14 5-5-5-5"},child:[]},{tag:"path",attr:{d:"M20 9H9.5A5.5 5.5 0 0 0 4 14.5A5.5 5.5 0 0 0 9.5 20H13"},child:[]}]})(e)}function tR(e){return re({attr:{viewBox:"0 0 24 24",fill:"none",stroke:"currentColor",strokeWidth:"2",strokeLinecap:"round",strokeLinejoin:"round"},child:[{tag:"path",attr:{d:"M18 6 6 18"},child:[]},{tag:"path",attr:{d:"m6 6 12 12"},child:[]}]})(e)}const nR=P.forwardRef((e,t)=>f.jsx(St,{"aria-label":"Close",ref:t,variant:"ghost",...e,children:e.children??f.jsx(tR,{})})),rR=P.forwardRef((e,t)=>{const{children:n,closable:r,endElement:o,icon:i,onClose:s,startElement:a,title:l,...c}=e;return f.jsxs(RE,{ref:t,...c,alignItems:"center",children:[a??f.jsx(_E,{children:i}),n?f.jsxs(NE,{children:[f.jsx(Kh,{children:l}),f.jsx(TE,{children:n})]}):f.jsx(Kh,{flex:"1",children:l}),o,r?f.jsx(nR,{alignSelf:"flex-start",insetEnd:"-2",onClick:s,pos:"relative",size:"sm",top:"-2"}):void 0]})});/*! + */var Pg="popstate";function wP(e={}){function t(r,o){let{pathname:i,search:s,hash:a}=r.location;return el("",{pathname:i,search:s,hash:a},o.state&&o.state.usr||null,o.state&&o.state.key||"default")}function n(r,o){return typeof o=="string"?o:Yr(o)}return EP(t,n,null,e)}function le(e,t){if(e===!1||e===null||typeof e>"u")throw new Error(t)}function vt(e,t){if(!e){typeof console<"u"&&console.warn(t);try{throw new Error(t)}catch{}}}function SP(){return Math.random().toString(36).substring(2,10)}function Rg(e,t){return{usr:e.state,key:e.key,idx:t}}function el(e,t,n=null,r){return{pathname:typeof e=="string"?e:e.pathname,search:"",hash:"",...typeof t=="string"?cr(t):t,state:n,key:t&&t.key||r||SP()}}function Yr({pathname:e="/",search:t="",hash:n=""}){return t&&t!=="?"&&(e+=t.charAt(0)==="?"?t:"?"+t),n&&n!=="#"&&(e+=n.charAt(0)==="#"?n:"#"+n),e}function cr(e){let t={};if(e){let n=e.indexOf("#");n>=0&&(t.hash=e.substring(n),e=e.substring(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substring(r),e=e.substring(0,r)),e&&(t.pathname=e)}return t}function EP(e,t,n,r={}){let{window:o=document.defaultView,v5Compat:i=!1}=r,s=o.history,a="POP",l=null,c=u();c==null&&(c=0,s.replaceState({...s.state,idx:c},""));function u(){return(s.state||{idx:null}).idx}function h(){a="POP";let v=u(),b=v==null?null:v-c;c=v,l&&l({action:a,location:p.location,delta:b})}function d(v,b){a="PUSH";let y=el(p.location,v,b);c=u()+1;let x=Rg(y,c),w=p.createHref(y);try{s.pushState(x,"",w)}catch(k){if(k instanceof DOMException&&k.name==="DataCloneError")throw k;o.location.assign(w)}i&&l&&l({action:a,location:p.location,delta:1})}function m(v,b){a="REPLACE";let y=el(p.location,v,b);c=u();let x=Rg(y,c),w=p.createHref(y);s.replaceState(x,"",w),i&&l&&l({action:a,location:p.location,delta:0})}function f(v){return OP(v)}let p={get action(){return a},get location(){return e(o,s)},listen(v){if(l)throw new Error("A history only accepts one active listener");return o.addEventListener(Pg,h),l=v,()=>{o.removeEventListener(Pg,h),l=null}},createHref(v){return t(o,v)},createURL:f,encodeLocation(v){let b=f(v);return{pathname:b.pathname,search:b.search,hash:b.hash}},push:d,replace:m,go(v){return s.go(v)}};return p}function OP(e,t=!1){let n="http://localhost";typeof window<"u"&&(n=window.location.origin!=="null"?window.location.origin:window.location.href),le(n,"No window.location.(origin|href) available to create URL");let r=typeof e=="string"?e:Yr(e);return r=r.replace(/ $/,"%20"),!t&&r.startsWith("//")&&(r=n+r),new URL(r,n)}function Ig(e,t,n="/"){return PP(e,t,n,!1)}function PP(e,t,n,r){let o=typeof t=="string"?cr(t):t,i=Pt(o.pathname||"/",n);if(i==null)return null;let s=Tg(e);RP(s);let a=null;for(let l=0;a==null&&l{let u={relativePath:c===void 0?s.path||"":c,caseSensitive:s.caseSensitive===!0,childrenIndex:a,route:s};if(u.relativePath.startsWith("/")){if(!u.relativePath.startsWith(r)&&l)return;le(u.relativePath.startsWith(r),`Absolute route path "${u.relativePath}" nested under path "${r}" is not valid. An absolute child route path must start with the combined path of all its parent routes.`),u.relativePath=u.relativePath.slice(r.length)}let h=Rt([r,u.relativePath]),d=n.concat(u);s.children&&s.children.length>0&&(le(s.index!==!0,`Index routes must not have child routes. Please remove all child routes from route path "${h}".`),Tg(s.children,t,d,h,l)),!(s.path==null&&!s.index)&&t.push({path:h,score:FP(h,s.index),routesMeta:d})};return e.forEach((s,a)=>{if(s.path===""||!s.path?.includes("?"))i(s,a);else for(let l of Ng(s.path))i(s,a,!0,l)}),t}function Ng(e){let t=e.split("/");if(t.length===0)return[];let[n,...r]=t,o=n.endsWith("?"),i=n.replace(/\?$/,"");if(r.length===0)return o?[i,""]:[i];let s=Ng(r.join("/")),a=[];return a.push(...s.map(l=>l===""?i:[i,l].join("/"))),o&&a.push(...s),a.map(l=>e.startsWith("/")&&l===""?"/":l)}function RP(e){e.sort((t,n)=>t.score!==n.score?n.score-t.score:LP(t.routesMeta.map(r=>r.childrenIndex),n.routesMeta.map(r=>r.childrenIndex)))}var IP=/^:[\w-]+$/,TP=3,NP=2,AP=1,_P=10,VP=-2,Ag=e=>e==="*";function FP(e,t){let n=e.split("/"),r=n.length;return n.some(Ag)&&(r+=VP),t&&(r+=NP),n.filter(o=>!Ag(o)).reduce((o,i)=>o+(IP.test(i)?TP:i===""?AP:_P),r)}function LP(e,t){return e.length===t.length&&e.slice(0,-1).every((r,o)=>r===t[o])?e[e.length-1]-t[t.length-1]:0}function DP(e,t,n=!1){let{routesMeta:r}=e,o={},i="/",s=[];for(let a=0;a{if(u==="*"){let f=a[d]||"";s=i.slice(0,i.length-f.length).replace(/(.)\/+$/,"$1")}const m=a[d];return h&&!m?c[u]=void 0:c[u]=(m||"").replace(/%2F/g,"/"),c},{}),pathname:i,pathnameBase:s,pattern:e}}function zP(e,t=!1,n=!0){vt(e==="*"||!e.endsWith("*")||e.endsWith("/*"),`Route path "${e}" will be treated as if it were "${e.replace(/\*$/,"/*")}" because the \`*\` character must always follow a \`/\` in the pattern. To get rid of this warning, please change the route path to "${e.replace(/\*$/,"/*")}".`);let r=[],o="^"+e.replace(/\/*\*?$/,"").replace(/^\/*/,"/").replace(/[\\.*+^${}|()[\]]/g,"\\$&").replace(/\/:([\w-]+)(\?)?/g,(s,a,l)=>(r.push({paramName:a,isOptional:l!=null}),l?"/?([^\\/]+)?":"/([^\\/]+)")).replace(/\/([\w-]+)\?(\/|$)/g,"(/$1)?$2");return e.endsWith("*")?(r.push({paramName:"*"}),o+=e==="*"||e==="/*"?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?o+="\\/*$":e!==""&&e!=="/"&&(o+="(?:(?=\\/|$))"),[new RegExp(o,t?void 0:"i"),r]}function MP(e){try{return e.split("/").map(t=>decodeURIComponent(t).replace(/\//g,"%2F")).join("/")}catch(t){return vt(!1,`The URL path "${e}" could not be decoded because it is a malformed URL segment. This is probably due to a bad percent encoding (${t}).`),e}}function Pt(e,t){if(t==="/")return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&r!=="/"?null:e.slice(n)||"/"}function jP(e,t="/"){let{pathname:n,search:r="",hash:o=""}=typeof e=="string"?cr(e):e;return{pathname:n?n.startsWith("/")?n:$P(n,t):t,search:HP(r),hash:UP(o)}}function $P(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach(o=>{o===".."?n.length>1&&n.pop():o!=="."&&n.push(o)}),n.length>1?n.join("/"):"/"}function tl(e,t,n,r){return`Cannot include a '${e}' character in a manually specified \`to.${t}\` field [${JSON.stringify(r)}]. Please separate it out to the \`to.${n}\` field. Alternatively you may provide the full path as a string in and the router will parse it for you.`}function BP(e){return e.filter((t,n)=>n===0||t.route.path&&t.route.path.length>0)}function _g(e){let t=BP(e);return t.map((n,r)=>r===t.length-1?n.pathname:n.pathnameBase)}function Vg(e,t,n,r=!1){let o;typeof e=="string"?o=cr(e):(o={...e},le(!o.pathname||!o.pathname.includes("?"),tl("?","pathname","search",o)),le(!o.pathname||!o.pathname.includes("#"),tl("#","pathname","hash",o)),le(!o.search||!o.search.includes("#"),tl("#","search","hash",o)));let i=e===""||o.pathname==="",s=i?"/":o.pathname,a;if(s==null)a=n;else{let h=t.length-1;if(!r&&s.startsWith("..")){let d=s.split("/");for(;d[0]==="..";)d.shift(),h-=1;o.pathname=d.join("/")}a=h>=0?t[h]:"/"}let l=jP(o,a),c=s&&s!=="/"&&s.endsWith("/"),u=(i||s===".")&&n.endsWith("/");return!l.pathname.endsWith("/")&&(c||u)&&(l.pathname+="/"),l}var Rt=e=>e.join("/").replace(/\/\/+/g,"/"),WP=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),HP=e=>!e||e==="?"?"":e.startsWith("?")?e:"?"+e,UP=e=>!e||e==="#"?"":e.startsWith("#")?e:"#"+e;function GP(e){return e!=null&&typeof e.status=="number"&&typeof e.statusText=="string"&&typeof e.internal=="boolean"&&"data"in e}var Fg=["POST","PUT","PATCH","DELETE"];new Set(Fg);var qP=["GET",...Fg];new Set(qP);var ur=C.createContext(null);ur.displayName="DataRouter";var Ni=C.createContext(null);Ni.displayName="DataRouterState",C.createContext(!1);var Lg=C.createContext({isTransitioning:!1});Lg.displayName="ViewTransition";var KP=C.createContext(new Map);KP.displayName="Fetchers";var YP=C.createContext(null);YP.displayName="Await";var bt=C.createContext(null);bt.displayName="Navigation";var Xr=C.createContext(null);Xr.displayName="Location";var It=C.createContext({outlet:null,matches:[],isDataRoute:!1});It.displayName="Route";var nl=C.createContext(null);nl.displayName="RouteError";function XP(e,{relative:t}={}){le(Qr(),"useHref() may be used only in the context of a component.");let{basename:n,navigator:r}=C.useContext(bt),{hash:o,pathname:i,search:s}=Jr(e,{relative:t}),a=i;return n!=="/"&&(a=i==="/"?n:Rt([n,i])),r.createHref({pathname:a,search:s,hash:o})}function Qr(){return C.useContext(Xr)!=null}function Nn(){return le(Qr(),"useLocation() may be used only in the context of a component."),C.useContext(Xr).location}var Dg="You should call navigate() in a React.useEffect(), not when your component is first rendered.";function zg(e){C.useContext(bt).static||C.useLayoutEffect(e)}function QP(){let{isDataRoute:e}=C.useContext(It);return e?uR():JP()}function JP(){le(Qr(),"useNavigate() may be used only in the context of a component.");let e=C.useContext(ur),{basename:t,navigator:n}=C.useContext(bt),{matches:r}=C.useContext(It),{pathname:o}=Nn(),i=JSON.stringify(_g(r)),s=C.useRef(!1);return zg(()=>{s.current=!0}),C.useCallback((l,c={})=>{if(vt(s.current,Dg),!s.current)return;if(typeof l=="number"){n.go(l);return}let u=Vg(l,JSON.parse(i),o,c.relative==="path");e==null&&t!=="/"&&(u.pathname=u.pathname==="/"?t:Rt([t,u.pathname])),(c.replace?n.replace:n.push)(u,c.state,c)},[t,n,i,o,e])}C.createContext(null);function Jr(e,{relative:t}={}){let{matches:n}=C.useContext(It),{pathname:r}=Nn(),o=JSON.stringify(_g(n));return C.useMemo(()=>Vg(e,JSON.parse(o),r,t==="path"),[e,o,r,t])}function ZP(e,t){return Mg(e,t)}function Mg(e,t,n,r,o){le(Qr(),"useRoutes() may be used only in the context of a component.");let{navigator:i}=C.useContext(bt),{matches:s}=C.useContext(It),a=s[s.length-1],l=a?a.params:{},c=a?a.pathname:"/",u=a?a.pathnameBase:"/",h=a&&a.route;{let y=h&&h.path||"";$g(c,!h||y.endsWith("*")||y.endsWith("*?"),`You rendered descendant (or called \`useRoutes()\`) at "${c}" (under ) but the parent route path has no trailing "*". This means if you navigate deeper, the parent won't match anymore and therefore the child routes will never render. + +Please change the parent to .`)}let d=Nn(),m;if(t){let y=typeof t=="string"?cr(t):t;le(u==="/"||y.pathname?.startsWith(u),`When overriding the location using \`\` or \`useRoutes(routes, location)\`, the location pathname must begin with the portion of the URL pathname that was matched by all parent routes. The current pathname base is "${u}" but pathname "${y.pathname}" was given in the \`location\` prop.`),m=y}else m=d;let f=m.pathname||"/",p=f;if(u!=="/"){let y=u.replace(/^\//,"").split("/");p="/"+f.replace(/^\//,"").split("/").slice(y.length).join("/")}let v=Ig(e,{pathname:p});vt(h||v!=null,`No routes matched location "${m.pathname}${m.search}${m.hash}" `),vt(v==null||v[v.length-1].route.element!==void 0||v[v.length-1].route.Component!==void 0||v[v.length-1].route.lazy!==void 0,`Matched leaf route at location "${m.pathname}${m.search}${m.hash}" does not have an element or Component. This means it will render an with a null value by default resulting in an "empty" page.`);let b=oR(v&&v.map(y=>Object.assign({},y,{params:Object.assign({},l,y.params),pathname:Rt([u,i.encodeLocation?i.encodeLocation(y.pathname.replace(/\?/g,"%3F").replace(/#/g,"%23")).pathname:y.pathname]),pathnameBase:y.pathnameBase==="/"?u:Rt([u,i.encodeLocation?i.encodeLocation(y.pathnameBase.replace(/\?/g,"%3F").replace(/#/g,"%23")).pathname:y.pathnameBase])})),s,n,r,o);return t&&b?C.createElement(Xr.Provider,{value:{location:{pathname:"/",search:"",hash:"",state:null,key:"default",...m},navigationType:"POP"}},b):b}function eR(){let e=cR(),t=GP(e)?`${e.status} ${e.statusText}`:e instanceof Error?e.message:JSON.stringify(e),n=e instanceof Error?e.stack:null,r="rgba(200,200,200, 0.5)",o={padding:"0.5rem",backgroundColor:r},i={padding:"2px 4px",backgroundColor:r},s=null;return console.error("Error handled by React Router default ErrorBoundary:",e),s=C.createElement(C.Fragment,null,C.createElement("p",null,"💿 Hey developer 👋"),C.createElement("p",null,"You can provide a way better UX than this when your app throws errors by providing your own ",C.createElement("code",{style:i},"ErrorBoundary")," or"," ",C.createElement("code",{style:i},"errorElement")," prop on your route.")),C.createElement(C.Fragment,null,C.createElement("h2",null,"Unexpected Application Error!"),C.createElement("h3",{style:{fontStyle:"italic"}},t),n?C.createElement("pre",{style:o},n):null,s)}var tR=C.createElement(eR,null),nR=class extends C.Component{constructor(e){super(e),this.state={location:e.location,revalidation:e.revalidation,error:e.error}}static getDerivedStateFromError(e){return{error:e}}static getDerivedStateFromProps(e,t){return t.location!==e.location||t.revalidation!=="idle"&&e.revalidation==="idle"?{error:e.error,location:e.location,revalidation:e.revalidation}:{error:e.error!==void 0?e.error:t.error,location:t.location,revalidation:e.revalidation||t.revalidation}}componentDidCatch(e,t){this.props.unstable_onError?this.props.unstable_onError(e,t):console.error("React Router caught the following error during render",e)}render(){return this.state.error!==void 0?C.createElement(It.Provider,{value:this.props.routeContext},C.createElement(nl.Provider,{value:this.state.error,children:this.props.component})):this.props.children}};function rR({routeContext:e,match:t,children:n}){let r=C.useContext(ur);return r&&r.static&&r.staticContext&&(t.route.errorElement||t.route.ErrorBoundary)&&(r.staticContext._deepestRenderedBoundaryId=t.route.id),C.createElement(It.Provider,{value:e},n)}function oR(e,t=[],n=null,r=null,o=null){if(e==null){if(!n)return null;if(n.errors)e=n.matches;else if(t.length===0&&!n.initialized&&n.matches.length>0)e=n.matches;else return null}let i=e,s=n?.errors;if(s!=null){let c=i.findIndex(u=>u.route.id&&s?.[u.route.id]!==void 0);le(c>=0,`Could not find a matching route for errors on route IDs: ${Object.keys(s).join(",")}`),i=i.slice(0,Math.min(i.length,c+1))}let a=!1,l=-1;if(n)for(let c=0;c=0?i=i.slice(0,l+1):i=[i[0]];break}}}return i.reduceRight((c,u,h)=>{let d,m=!1,f=null,p=null;n&&(d=s&&u.route.id?s[u.route.id]:void 0,f=u.route.errorElement||tR,a&&(l<0&&h===0?($g("route-fallback",!1,"No `HydrateFallback` element provided to render during initial hydration"),m=!0,p=null):l===h&&(m=!0,p=u.route.hydrateFallbackElement||null)));let v=t.concat(i.slice(0,h+1)),b=()=>{let y;return d?y=f:m?y=p:u.route.Component?y=C.createElement(u.route.Component,null):u.route.element?y=u.route.element:y=c,C.createElement(rR,{match:u,routeContext:{outlet:c,matches:v,isDataRoute:n!=null},children:y})};return n&&(u.route.ErrorBoundary||u.route.errorElement||h===0)?C.createElement(nR,{location:n.location,revalidation:n.revalidation,component:f,error:d,children:b(),routeContext:{outlet:null,matches:v,isDataRoute:!0},unstable_onError:r}):b()},null)}function rl(e){return`${e} must be used within a data router. See https://reactrouter.com/en/main/routers/picking-a-router.`}function iR(e){let t=C.useContext(ur);return le(t,rl(e)),t}function sR(e){let t=C.useContext(Ni);return le(t,rl(e)),t}function aR(e){let t=C.useContext(It);return le(t,rl(e)),t}function ol(e){let t=aR(e),n=t.matches[t.matches.length-1];return le(n.route.id,`${e} can only be used on routes that contain a unique "id"`),n.route.id}function lR(){return ol("useRouteId")}function cR(){let e=C.useContext(nl),t=sR("useRouteError"),n=ol("useRouteError");return e!==void 0?e:t.errors?.[n]}function uR(){let{router:e}=iR("useNavigate"),t=ol("useNavigate"),n=C.useRef(!1);return zg(()=>{n.current=!0}),C.useCallback(async(o,i={})=>{vt(n.current,Dg),n.current&&(typeof o=="number"?e.navigate(o):await e.navigate(o,{fromRouteId:t,...i}))},[e,t])}var jg={};function $g(e,t,n){!t&&!jg[e]&&(jg[e]=!0,vt(!1,n))}C.memo(dR);function dR({routes:e,future:t,state:n,unstable_onError:r}){return Mg(e,void 0,n,r,t)}function il(e){le(!1,"A is only ever to be used as the child of element, never rendered directly. Please wrap your in a .")}function hR({basename:e="/",children:t=null,location:n,navigationType:r="POP",navigator:o,static:i=!1}){le(!Qr(),"You cannot render a inside another . You should never have more than one in your app.");let s=e.replace(/^\/*/,"/"),a=C.useMemo(()=>({basename:s,navigator:o,static:i,future:{}}),[s,o,i]);typeof n=="string"&&(n=cr(n));let{pathname:l="/",search:c="",hash:u="",state:h=null,key:d="default"}=n,m=C.useMemo(()=>{let f=Pt(l,s);return f==null?null:{location:{pathname:f,search:c,hash:u,state:h,key:d},navigationType:r}},[s,l,c,u,h,d,r]);return vt(m!=null,` is not able to match the URL "${l}${c}${u}" because it does not start with the basename, so the won't render anything.`),m==null?null:C.createElement(bt.Provider,{value:a},C.createElement(Xr.Provider,{children:t,value:m}))}function fR({children:e,location:t}){return ZP(sl(e),t)}function sl(e,t=[]){let n=[];return C.Children.forEach(e,(r,o)=>{if(!C.isValidElement(r))return;let i=[...t,o];if(r.type===C.Fragment){n.push.apply(n,sl(r.props.children,i));return}le(r.type===il,`[${typeof r.type=="string"?r.type:r.type.name}] is not a component. All component children of must be a or `),le(!r.props.index||!r.props.children,"An index route cannot have child routes.");let s={id:r.props.id||i.join("-"),caseSensitive:r.props.caseSensitive,element:r.props.element,Component:r.props.Component,index:r.props.index,path:r.props.path,middleware:r.props.middleware,loader:r.props.loader,action:r.props.action,hydrateFallbackElement:r.props.hydrateFallbackElement,HydrateFallback:r.props.HydrateFallback,errorElement:r.props.errorElement,ErrorBoundary:r.props.ErrorBoundary,hasErrorBoundary:r.props.hasErrorBoundary===!0||r.props.ErrorBoundary!=null||r.props.errorElement!=null,shouldRevalidate:r.props.shouldRevalidate,handle:r.props.handle,lazy:r.props.lazy};r.props.children&&(s.children=sl(r.props.children,i)),n.push(s)}),n}var Ai="get",_i="application/x-www-form-urlencoded";function Vi(e){return e!=null&&typeof e.tagName=="string"}function gR(e){return Vi(e)&&e.tagName.toLowerCase()==="button"}function pR(e){return Vi(e)&&e.tagName.toLowerCase()==="form"}function mR(e){return Vi(e)&&e.tagName.toLowerCase()==="input"}function vR(e){return!!(e.metaKey||e.altKey||e.ctrlKey||e.shiftKey)}function bR(e,t){return e.button===0&&(!t||t==="_self")&&!vR(e)}var Fi=null;function yR(){if(Fi===null)try{new FormData(document.createElement("form"),0),Fi=!1}catch{Fi=!0}return Fi}var xR=new Set(["application/x-www-form-urlencoded","multipart/form-data","text/plain"]);function al(e){return e!=null&&!xR.has(e)?(vt(!1,`"${e}" is not a valid \`encType\` for \`
\`/\`\` and will default to "${_i}"`),null):e}function kR(e,t){let n,r,o,i,s;if(pR(e)){let a=e.getAttribute("action");r=a?Pt(a,t):null,n=e.getAttribute("method")||Ai,o=al(e.getAttribute("enctype"))||_i,i=new FormData(e)}else if(gR(e)||mR(e)&&(e.type==="submit"||e.type==="image")){let a=e.form;if(a==null)throw new Error('Cannot submit a )} diff --git a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/TaskNames.tsx b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/TaskNames.tsx index c807458c77196..da76cfc8595b8 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/TaskNames.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/TaskNames.tsx @@ -117,7 +117,7 @@ export const TaskNames = ({ nodes, onRowClick }: Props) => { px={1} > diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/AdminButton.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/AdminButton.tsx index 349a270e886ed..ddc01d9db28ee 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/AdminButton.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/AdminButton.tsx @@ -79,7 +79,7 @@ export const AdminButton = ({ return ( - } title={translate("nav.admin")} /> + } title={translate("nav.admin")} /> {menuItems} diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/BrowseButton.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/BrowseButton.tsx index e24700e045172..170f89a94f5fb 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/BrowseButton.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/BrowseButton.tsx @@ -70,7 +70,7 @@ export const BrowseButton = ({ return ( - } title={translate("nav.browse")} /> + } title={translate("nav.browse")} /> {menuItems} diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/DocsButton.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/DocsButton.tsx index a33cc3724e9f3..d73b323b3e122 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/DocsButton.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/DocsButton.tsx @@ -61,7 +61,7 @@ export const DocsButton = ({ return ( - } title={translate("nav.docs")} /> + } title={translate("nav.docs")} /> {links diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/Nav.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/Nav.tsx index 2d843b1556c02..b313a4ff63484 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/Nav.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/Nav.tsx @@ -151,16 +151,16 @@ export const Nav = () => { - } title={translate("nav.home")} to="/" /> + } title={translate("nav.home")} to="/" /> } + icon={} title={translate("nav.dags")} to="dags" /> } + icon={} title={translate("nav.assets")} to="assets" /> diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/PluginMenuItem.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/PluginMenuItem.tsx index d28df1dff2342..c7372703237a5 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/PluginMenuItem.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/PluginMenuItem.tsx @@ -46,11 +46,11 @@ export const PluginMenuItem = ({ const displayIcon = colorMode === "dark" && typeof iconDarkMode === "string" ? iconDarkMode : icon; const pluginIcon = typeof displayIcon === "string" ? ( - + ) : urlRoute === "legacy-fab-views" ? ( - + ) : ( - + ); const isExternal = urlRoute === undefined || urlRoute === null; diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/SecurityButton.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/SecurityButton.tsx index 38f67ddd9f631..6d9fdca68f0e2 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/SecurityButton.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/SecurityButton.tsx @@ -36,7 +36,7 @@ export const SecurityButton = () => { return ( - } title={translate("nav.security")} /> + } title={translate("nav.security")} /> {authLinks.extra_menu_items.map(({ text }) => { diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/TimezoneMenuItem.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/TimezoneMenuItem.tsx index 360560ca199f9..596cf79bb1ba1 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/TimezoneMenuItem.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/TimezoneMenuItem.tsx @@ -48,7 +48,7 @@ export const TimezoneMenuItem = ({ onOpen }: { readonly onOpen: () => void }) => return ( - + {translate("timezone")}: {dayjs(time).tz(selectedTimezone).format("HH:mm z (Z)")} ); diff --git a/airflow-core/src/airflow/ui/src/layouts/Nav/UserSettingsButton.tsx b/airflow-core/src/airflow/ui/src/layouts/Nav/UserSettingsButton.tsx index c8cc8108945a8..31e2dd0078a60 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Nav/UserSettingsButton.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Nav/UserSettingsButton.tsx @@ -67,21 +67,21 @@ export const UserSettingsButton = ({ externalViews }: { readonly externalViews: return ( - } title={translate("user")} /> + } title={translate("user")} /> - + {translate("selectLanguage")} - + {translate("appearance.appearance")} {isRTL ? ( - + ) : ( - + )} @@ -90,17 +90,17 @@ export const UserSettingsButton = ({ externalViews }: { readonly externalViews: value={theme} > - + {translate("appearance.lightMode")} - + {translate("appearance.darkMode")} - + {translate("appearance.systemMode")} @@ -113,12 +113,12 @@ export const UserSettingsButton = ({ externalViews }: { readonly externalViews: > {dagView === "grid" ? ( <> - + {translate("defaultToGraphView")} ) : ( <> - + {translate("defaultToGridView")} )} @@ -129,7 +129,7 @@ export const UserSettingsButton = ({ externalViews }: { readonly externalViews: ))} {translate("logout")} diff --git a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/DAGImportErrors.tsx b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/DAGImportErrors.tsx index e6fc9d1f3e267..87b9bf917e0c6 100644 --- a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/DAGImportErrors.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/DAGImportErrors.tsx @@ -55,7 +55,7 @@ export const DAGImportErrors = ({ iconOnly = false }: { readonly iconOnly?: bool onClick={onOpen} title={translate("importErrors.dagImportError", { count: importErrorsCount })} > - + {importErrorsCount} ) : ( diff --git a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/PluginImportErrors.tsx b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/PluginImportErrors.tsx index cea92bed6d794..ed9643c35c1da 100644 --- a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/PluginImportErrors.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/PluginImportErrors.tsx @@ -58,7 +58,7 @@ export const PluginImportErrors = ({ iconOnly = false }: { readonly iconOnly?: b onClick={onOpen} title={translate("plugins.importError", { count: importErrorsCount })} > - + {importErrorsCount} ) : ( From 5d9e2c5eaf6f8ce6856393a295c859d91c223c23 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Thu, 2 Oct 2025 10:28:19 -0400 Subject: [PATCH 178/338] Update recent runs bar chart and improve responsiveness (#56314) --- .../ui/src/pages/DagsList/RecentRuns.tsx | 25 +++++++++++-------- .../FavoriteDags/FavoriteDagCard.tsx | 5 ++-- .../Dashboard/FavoriteDags/FavoriteDags.tsx | 6 ++--- .../ui/src/pages/Dashboard/Stats/Stats.tsx | 6 ++--- 4 files changed, 23 insertions(+), 19 deletions(-) diff --git a/airflow-core/src/airflow/ui/src/pages/DagsList/RecentRuns.tsx b/airflow-core/src/airflow/ui/src/pages/DagsList/RecentRuns.tsx index d52ca59f53631..8048e5182d034 100644 --- a/airflow-core/src/airflow/ui/src/pages/DagsList/RecentRuns.tsx +++ b/airflow-core/src/airflow/ui/src/pages/DagsList/RecentRuns.tsx @@ -23,6 +23,7 @@ import { useTranslation } from "react-i18next"; import { Link } from "react-router-dom"; import type { DAGWithLatestDagRunsResponse } from "openapi/requests/types.gen"; +import { StateIcon } from "src/components/StateIcon"; import Time from "src/components/Time"; import { Tooltip } from "src/components/ui"; import { renderDuration } from "src/utils"; @@ -48,7 +49,7 @@ export const RecentRuns = ({ ); return ( - + {latestRuns.map((run) => ( - - - + + + ))} diff --git a/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDagCard.tsx b/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDagCard.tsx index 964be0a41bc5c..85c5cba322300 100644 --- a/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDagCard.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDagCard.tsx @@ -38,14 +38,13 @@ export const FavoriteDagCard = ({ dagId, dagName, latestRuns }: FavoriteDagProps borderWidth="1px" display="flex" flexDirection="column" - height="100%" justifyContent="center" + maxWidth="200px" overflow="hidden" px={4} py={3} - width="100%" > - + {latestRuns.length > 0 ? ( ) : ( diff --git a/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDags.tsx b/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDags.tsx index e17b111f5dd98..4451cd88b9676 100644 --- a/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDags.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Dashboard/FavoriteDags/FavoriteDags.tsx @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -import { Box, Flex, Heading, SimpleGrid, Text } from "@chakra-ui/react"; +import { Box, Flex, Heading, Text } from "@chakra-ui/react"; import { useTranslation } from "react-i18next"; import { FiStar } from "react-icons/fi"; @@ -47,7 +47,7 @@ export const FavoriteDags = () => { {translate("favorite.noFavoriteDags")} ) : ( - + {favorites.dags.map((dag) => ( { latestRuns={dag.latest_dag_runs} /> ))} - + )} ); diff --git a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/Stats.tsx b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/Stats.tsx index e1027f050b59b..4192ad7b88652 100644 --- a/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/Stats.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Dashboard/Stats/Stats.tsx @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -import { Box, Flex, Heading, HStack } from "@chakra-ui/react"; +import { Box, Flex, Heading } from "@chakra-ui/react"; import { useTranslation } from "react-i18next"; import { FiClipboard, FiZap } from "react-icons/fi"; @@ -51,7 +51,7 @@ export const Stats = () => { - + { label={translate("stats.activeDags")} link="dags?paused=false" /> - + ); }; From cf03ed733f590d4bc60515bdef5bd622ca7a28ba Mon Sep 17 00:00:00 2001 From: Kevin Yang <85313829+sjyangkevin@users.noreply.github.com> Date: Thu, 2 Oct 2025 10:55:47 -0400 Subject: [PATCH 179/338] attempt to update 3.0.1 release notes for serialization interface change (#56318) * attempt to update 3.0.1 release notes for serialization interface change * minor update to make it concise --- RELEASE_NOTES.rst | 16 ++++++++++++++++ reproducible_build.yaml | 4 ++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst index b9c07b55cb176..8e5f492fa29c9 100644 --- a/RELEASE_NOTES.rst +++ b/RELEASE_NOTES.rst @@ -319,6 +319,22 @@ You can also use ``structlog`` loggers at the top level of modules etc, and ``st (You can't add arbitrary key/value pairs to ``stdlib``, but the normal ``percent-formatter`` approaches still work fine.) +Serialization Interface Changes +""""""""""""""""""""""""""""""" + +The deserializer interface in ``airflow.serialization.serializers`` has changed for improved security. + +**Before 3.1.0:** + +``def deserialize(classname: str, version: int, data: Any)`` + +**Starting with 3.1.0:** + +``def deserialize(cls: type, version: int, data: Any)`` + +The class loading is now handled in ``serde.py``, and the deserializer receives the loaded class directly rather than a ``classname`` string. +This update avoids the use of ``import_string`` in the deserializer, making deserialization more secure. + New Features ^^^^^^^^^^^^ diff --git a/reproducible_build.yaml b/reproducible_build.yaml index fc6fff85bb2e5..15ac7f8ea66ed 100644 --- a/reproducible_build.yaml +++ b/reproducible_build.yaml @@ -1,2 +1,2 @@ -release-notes-hash: 07082f3225cc822b8f456a377336d689 -source-date-epoch: 1759336130 +release-notes-hash: decf692f4614345c187317f600697668 +source-date-epoch: 1759372174 From a71927f8f5c48c71b29fc7ae041a16cdb43f7300 Mon Sep 17 00:00:00 2001 From: Pierre Jeambrun Date: Thu, 2 Oct 2025 17:29:53 +0200 Subject: [PATCH 180/338] Expand and collapse group component (#56293) * Extract component for expand and collapse button group * Refactor toggle groups buttons --- .../src/components/ExpandCollapseButtons.tsx | 60 +++++++++++++++++++ .../ui/src/layouts/Details/ToggleGroups.tsx | 33 ++++------ .../airflow/ui/src/pages/Events/Events.tsx | 28 +++------ .../src/airflow/ui/src/pages/XCom/XCom.tsx | 26 +++----- 4 files changed, 87 insertions(+), 60 deletions(-) create mode 100644 airflow-core/src/airflow/ui/src/components/ExpandCollapseButtons.tsx diff --git a/airflow-core/src/airflow/ui/src/components/ExpandCollapseButtons.tsx b/airflow-core/src/airflow/ui/src/components/ExpandCollapseButtons.tsx new file mode 100644 index 0000000000000..fac305a7624a0 --- /dev/null +++ b/airflow-core/src/airflow/ui/src/components/ExpandCollapseButtons.tsx @@ -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. + */ +import { ButtonGroup, IconButton } from "@chakra-ui/react"; +import { MdCompress, MdExpand } from "react-icons/md"; + +type Props = { + readonly collapseLabel: string; + readonly expandLabel: string; + readonly isCollapseDisabled?: boolean; + readonly isExpandDisabled?: boolean; + readonly onCollapse: () => void; + readonly onExpand: () => void; +}; + +export const ExpandCollapseButtons = ({ + collapseLabel, + expandLabel, + isCollapseDisabled, + isExpandDisabled, + onCollapse, + onExpand, + ...rest +}: Props) => ( + + + + + + + + +); diff --git a/airflow-core/src/airflow/ui/src/layouts/Details/ToggleGroups.tsx b/airflow-core/src/airflow/ui/src/layouts/Details/ToggleGroups.tsx index c55446c1c56be..8abaa3cca3e90 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Details/ToggleGroups.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Details/ToggleGroups.tsx @@ -16,10 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -import { type ButtonGroupProps, IconButton, ButtonGroup } from "@chakra-ui/react"; +import type { ButtonGroupProps } from "@chakra-ui/react"; import { useTranslation } from "react-i18next"; -import { MdExpand, MdCompress } from "react-icons/md"; +import { ExpandCollapseButtons } from "src/components/ExpandCollapseButtons"; import { useOpenGroups } from "src/context/openGroups"; export const ToggleGroups = (props: ButtonGroupProps) => { @@ -46,25 +46,14 @@ export const ToggleGroups = (props: ButtonGroupProps) => { const collapseLabel = translate("dag:taskGroups.collapseAll"); return ( - - - - - - - - + ); }; diff --git a/airflow-core/src/airflow/ui/src/pages/Events/Events.tsx b/airflow-core/src/airflow/ui/src/pages/Events/Events.tsx index ca19dd3653447..2a3bbefbb3a74 100644 --- a/airflow-core/src/airflow/ui/src/pages/Events/Events.tsx +++ b/airflow-core/src/airflow/ui/src/pages/Events/Events.tsx @@ -16,11 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -import { ButtonGroup, Code, Flex, Heading, IconButton, useDisclosure, VStack } from "@chakra-ui/react"; +import { Code, Flex, Heading, useDisclosure, VStack } from "@chakra-ui/react"; import type { ColumnDef } from "@tanstack/react-table"; import dayjs from "dayjs"; import { useTranslation } from "react-i18next"; -import { MdCompress, MdExpand } from "react-icons/md"; import { useParams, useSearchParams } from "react-router-dom"; import { useEventLogServiceGetEventLogs } from "openapi/queries"; @@ -28,6 +27,7 @@ import type { EventLogResponse } from "openapi/requests/types.gen"; import { DataTable } from "src/components/DataTable"; import { useTableURLState } from "src/components/DataTable/useTableUrlState"; import { ErrorAlert } from "src/components/ErrorAlert"; +import { ExpandCollapseButtons } from "src/components/ExpandCollapseButtons"; import RenderedJsonField from "src/components/RenderedJsonField"; import Time from "src/components/Time"; import { SearchParamsKeys, type SearchParamsKeysType } from "src/constants/searchParams"; @@ -214,24 +214,12 @@ export const Events = () => { ) : undefined} - - - - - - - - + diff --git a/airflow-core/src/airflow/ui/src/pages/XCom/XCom.tsx b/airflow-core/src/airflow/ui/src/pages/XCom/XCom.tsx index 68f3fa7f4df36..0e66988437693 100644 --- a/airflow-core/src/airflow/ui/src/pages/XCom/XCom.tsx +++ b/airflow-core/src/airflow/ui/src/pages/XCom/XCom.tsx @@ -16,10 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -import { Box, Heading, Link, Flex, ButtonGroup, IconButton, useDisclosure } from "@chakra-ui/react"; +import { Box, Heading, Link, Flex, useDisclosure } from "@chakra-ui/react"; import type { ColumnDef } from "@tanstack/react-table"; import { useTranslation } from "react-i18next"; -import { MdCompress, MdExpand } from "react-icons/md"; import { Link as RouterLink, useParams, useSearchParams } from "react-router-dom"; import { useXcomServiceGetXcomEntries } from "openapi/queries"; @@ -27,6 +26,7 @@ import type { XComResponse } from "openapi/requests/types.gen"; import { DataTable } from "src/components/DataTable"; import { useTableURLState } from "src/components/DataTable/useTableUrlState"; import { ErrorAlert } from "src/components/ErrorAlert"; +import { ExpandCollapseButtons } from "src/components/ExpandCollapseButtons"; import { TruncatedText } from "src/components/TruncatedText"; import { SearchParamsKeys, type SearchParamsKeysType } from "src/constants/searchParams"; import { getTaskInstanceLink } from "src/utils/links"; @@ -162,22 +162,12 @@ export const XCom = () => { - - - - - - - - + From 5ccacacd4cc1e3608a994361d65b4b819d4f90c6 Mon Sep 17 00:00:00 2001 From: Xch1 <34787736+xchwan@users.noreply.github.com> Date: Fri, 3 Oct 2025 02:36:28 +0800 Subject: [PATCH 181/338] Enable PT011 rule to prvoider tests (#56237) * fix test_cloud_sql Signed-off-by: Xch1 * fix test_cloud_logging Signed-off-by: Xch1 * fix hooks/test_bigquery Signed-off-by: Xch1 * fix assets/test_gcs Signed-off-by: Xch1 * fix assets/test_bigquery Signed-off-by: Xch1 --------- Signed-off-by: Xch1 --- providers/google/tests/unit/google/assets/test_bigquery.py | 2 +- providers/google/tests/unit/google/assets/test_gcs.py | 2 +- .../google/tests/unit/google/cloud/hooks/test_bigquery.py | 4 +--- .../tests/unit/google/cloud/hooks/test_cloud_logging.py | 2 +- .../google/tests/unit/google/cloud/hooks/test_cloud_sql.py | 6 +++--- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/providers/google/tests/unit/google/assets/test_bigquery.py b/providers/google/tests/unit/google/assets/test_bigquery.py index acc3adb9ef7cb..c2f4277944475 100644 --- a/providers/google/tests/unit/google/assets/test_bigquery.py +++ b/providers/google/tests/unit/google/assets/test_bigquery.py @@ -42,7 +42,7 @@ def test_sanitize_uri_pass() -> None: ) def test_sanitize_uri_fail(value: str) -> None: uri_i = urllib.parse.urlsplit(value) - with pytest.raises(ValueError): + with pytest.raises(ValueError, match="URI format bigquery:// must contain"): sanitize_uri(uri_i) diff --git a/providers/google/tests/unit/google/assets/test_gcs.py b/providers/google/tests/unit/google/assets/test_gcs.py index e9920302b0e0a..9ee93d9f6e37f 100644 --- a/providers/google/tests/unit/google/assets/test_gcs.py +++ b/providers/google/tests/unit/google/assets/test_gcs.py @@ -33,7 +33,7 @@ def test_sanitize_uri(): def test_sanitize_uri_no_netloc(): - with pytest.raises(ValueError): + with pytest.raises(ValueError, match="URI format gs:// must contain"): sanitize_uri(urllib.parse.urlsplit("gs://")) diff --git a/providers/google/tests/unit/google/cloud/hooks/test_bigquery.py b/providers/google/tests/unit/google/cloud/hooks/test_bigquery.py index 650eed7a0359c..9267497af3064 100644 --- a/providers/google/tests/unit/google/cloud/hooks/test_bigquery.py +++ b/providers/google/tests/unit/google/cloud/hooks/test_bigquery.py @@ -1550,11 +1550,9 @@ async def test_cancel_job_failure(self, mock_job, mock_auth_default): project_id = "test_project" location = "US" - with pytest.raises(Exception) as excinfo: + with pytest.raises(Exception, match="Cancellation failed"): await hook.cancel_job(job_id=job_id, project_id=project_id, location=location) - assert "Cancellation failed" in str(excinfo.value), "Exception message not passed correctly" - mock_job_instance.cancel.assert_called_once() @pytest.mark.asyncio diff --git a/providers/google/tests/unit/google/cloud/hooks/test_cloud_logging.py b/providers/google/tests/unit/google/cloud/hooks/test_cloud_logging.py index afaea5185ff97..d4d83088a008f 100644 --- a/providers/google/tests/unit/google/cloud/hooks/test_cloud_logging.py +++ b/providers/google/tests/unit/google/cloud/hooks/test_cloud_logging.py @@ -160,7 +160,7 @@ def test_create_sink_dict_input(self, mock_config_client): mock_config_client.return_value.create_sink.assert_called_once_with(request=expected_request) def test_update_sink_invalid_dict_format(self): - with pytest.raises(ValueError): + with pytest.raises(ValueError, match="Unknown field for LogSink: invalid_key"): self.hook.update_sink( sink_name=SINK_NAME, sink={"invalid_key": "value"}, diff --git a/providers/google/tests/unit/google/cloud/hooks/test_cloud_sql.py b/providers/google/tests/unit/google/cloud/hooks/test_cloud_sql.py index f1a332eb4fb7a..52d1190fbc300 100644 --- a/providers/google/tests/unit/google/cloud/hooks/test_cloud_sql.py +++ b/providers/google/tests/unit/google/cloud/hooks/test_cloud_sql.py @@ -974,10 +974,10 @@ def test_cloudsql_database_hook_get_sqlproxy_runner_no_proxy(self, get_connectio hook = CloudSQLDatabaseHook( gcp_cloudsql_conn_id="cloudsql_connection", default_gcp_project_id="google_connection" ) - with pytest.raises(ValueError) as ctx: + with pytest.raises( + ValueError, match="Proxy runner can only be retrieved in case of use_proxy = True" + ): hook.get_sqlproxy_runner() - err = ctx.value - assert "Proxy runner can only be retrieved in case of use_proxy = True" in str(err) @mock.patch("airflow.providers.google.cloud.hooks.cloud_sql.CloudSQLDatabaseHook.get_connection") def test_cloudsql_database_hook_get_sqlproxy_runner(self, get_connection): From 719cd0c05c2f7a627007e5e75f675512f25c6701 Mon Sep 17 00:00:00 2001 From: meher1993 Date: Thu, 2 Oct 2025 14:21:30 -0500 Subject: [PATCH 182/338] Add if_not_exists to index creation in migrations (#56328) --- .../versions/0001_1_4_0_create_ab_tables_if_missing.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/providers/fab/src/airflow/providers/fab/migrations/versions/0001_1_4_0_create_ab_tables_if_missing.py b/providers/fab/src/airflow/providers/fab/migrations/versions/0001_1_4_0_create_ab_tables_if_missing.py index e281437c9c6e1..a7ad1301c4af2 100644 --- a/providers/fab/src/airflow/providers/fab/migrations/versions/0001_1_4_0_create_ab_tables_if_missing.py +++ b/providers/fab/src/airflow/providers/fab/migrations/versions/0001_1_4_0_create_ab_tables_if_missing.py @@ -139,8 +139,8 @@ def upgrade() -> None: if_not_exists=True, ) with op.batch_alter_table("ab_group_role", schema=None) as batch_op: - batch_op.create_index("idx_group_id", ["group_id"], unique=False) - batch_op.create_index("idx_group_role_id", ["role_id"], unique=False) + batch_op.create_index("idx_group_id", ["group_id"], unique=False, if_not_exists=True) + batch_op.create_index("idx_group_role_id", ["role_id"], unique=False, if_not_exists=True) op.create_table( "ab_permission_view", @@ -175,8 +175,8 @@ def upgrade() -> None: if_not_exists=True, ) with op.batch_alter_table("ab_user_group", schema=None) as batch_op: - batch_op.create_index("idx_user_group_id", ["group_id"], unique=False) - batch_op.create_index("idx_user_id", ["user_id"], unique=False) + batch_op.create_index("idx_user_group_id", ["group_id"], unique=False, if_not_exists=True) + batch_op.create_index("idx_user_id", ["user_id"], unique=False, if_not_exists=True) op.create_table( "ab_user_role", From 11afbbba48b575efec568ba0799e31cb790a98d1 Mon Sep 17 00:00:00 2001 From: VladaZakharova Date: Thu, 2 Oct 2025 22:19:59 +0000 Subject: [PATCH 183/338] Fix dataflow java system test + link (#56286) --- .../providers/apache/beam/operators/beam.py | 14 +++- .../providers/apache/beam/triggers/beam.py | 10 ++- .../unit/apache/beam/operators/test_beam.py | 14 +++- .../unit/apache/beam/triggers/test_beam.py | 77 +++++++++++++++---- .../dataflow/example_dataflow_native_java.py | 4 +- 5 files changed, 93 insertions(+), 26 deletions(-) diff --git a/providers/apache/beam/src/airflow/providers/apache/beam/operators/beam.py b/providers/apache/beam/src/airflow/providers/apache/beam/operators/beam.py index 8d4d893c9196a..7713bbf067b1c 100644 --- a/providers/apache/beam/src/airflow/providers/apache/beam/operators/beam.py +++ b/providers/apache/beam/src/airflow/providers/apache/beam/operators/beam.py @@ -458,7 +458,12 @@ def execute_on_dataflow(self, context: Context): ) location = self.dataflow_config.location or DEFAULT_DATAFLOW_LOCATION - DataflowJobLink.persist(context=context, region=location) + DataflowJobLink.persist( + context=context, + region=self.dataflow_config.location, + job_id=self.dataflow_job_id, + project_id=self.dataflow_config.project_id, + ) if self.deferrable: trigger_args = { @@ -648,7 +653,12 @@ def execute_on_dataflow(self, context: Context): is_dataflow_job_id_exist_callback=self.is_dataflow_job_id_exist_callback, ) if self.dataflow_job_name and self.dataflow_config.location: - DataflowJobLink.persist(context=context) + DataflowJobLink.persist( + context=context, + region=self.dataflow_config.location, + job_id=self.dataflow_job_id, + project_id=self.dataflow_config.project_id, + ) if self.deferrable: trigger_args = { "job_id": self.dataflow_job_id, diff --git a/providers/apache/beam/src/airflow/providers/apache/beam/triggers/beam.py b/providers/apache/beam/src/airflow/providers/apache/beam/triggers/beam.py index 5778a12666614..6b2464eb37fb2 100644 --- a/providers/apache/beam/src/airflow/providers/apache/beam/triggers/beam.py +++ b/providers/apache/beam/src/airflow/providers/apache/beam/triggers/beam.py @@ -39,23 +39,25 @@ def file_has_gcs_path(file_path: str): @staticmethod async def provide_gcs_tempfile(gcs_file, gcp_conn_id): try: - from airflow.providers.google.cloud.hooks.gcs import GCSHook + from airflow.providers.google.cloud.hooks.gcs import GCSAsyncHook except ImportError: from airflow.exceptions import AirflowOptionalProviderFeatureException raise AirflowOptionalProviderFeatureException( - "Failed to import GCSHook. To use the GCSHook functionality, please install the " + "Failed to import GCSAsyncHook. To use the GCSAsyncHook functionality, please install the " "apache-airflow-google-provider." ) - gcs_hook = GCSHook(gcp_conn_id=gcp_conn_id) + async_gcs_hook = GCSAsyncHook(gcp_conn_id=gcp_conn_id) + sync_gcs_hook = await async_gcs_hook.get_sync_hook() + loop = asyncio.get_running_loop() # Running synchronous `enter_context()` method in a separate # thread using the default executor `None`. The `run_in_executor()` function returns the # file object, which is created using gcs function `provide_file()`, asynchronously. # This means we can perform asynchronous operations with this file. - create_tmp_file_call = gcs_hook.provide_file(object_url=gcs_file) + create_tmp_file_call = sync_gcs_hook.provide_file(object_url=gcs_file) tmp_gcs_file: IO[str] = await loop.run_in_executor( None, contextlib.ExitStack().enter_context, diff --git a/providers/apache/beam/tests/unit/apache/beam/operators/test_beam.py b/providers/apache/beam/tests/unit/apache/beam/operators/test_beam.py index 626800de7cac6..0727d472f5ef6 100644 --- a/providers/apache/beam/tests/unit/apache/beam/operators/test_beam.py +++ b/providers/apache/beam/tests/unit/apache/beam/operators/test_beam.py @@ -247,7 +247,12 @@ def test_exec_dataflow_runner( } gcs_provide_file.assert_any_call(object_url=PY_FILE) gcs_provide_file.assert_any_call(object_url=REQURIEMENTS_FILE) - persist_link_mock.assert_called_once_with(context={}, region="us-central1") + persist_link_mock.assert_called_once_with( + context={}, + region="us-central1", + job_id=None, + project_id=dataflow_hook_mock.return_value.project_id, + ) beam_hook_mock.return_value.start_python_pipeline.assert_called_once_with( variables=expected_options, py_file=gcs_provide_file.return_value.__enter__.return_value.name, @@ -468,7 +473,12 @@ def test_exec_dataflow_runner(self, gcs_hook, dataflow_hook_mock, beam_hook_mock "output": "gs://test/output", "impersonateServiceAccount": TEST_IMPERSONATION_ACCOUNT, } - persist_link_mock.assert_called_once_with(context={}) + persist_link_mock.assert_called_once_with( + context={}, + region="us-central1", + job_id=None, + project_id=dataflow_hook_mock.return_value.project_id, + ) beam_hook_mock.return_value.start_java_pipeline.assert_called_once_with( variables=expected_options, jar=gcs_provide_file.return_value.__enter__.return_value.name, diff --git a/providers/apache/beam/tests/unit/apache/beam/triggers/test_beam.py b/providers/apache/beam/tests/unit/apache/beam/triggers/test_beam.py index aed634c223be7..543b1d7fbe590 100644 --- a/providers/apache/beam/tests/unit/apache/beam/triggers/test_beam.py +++ b/providers/apache/beam/tests/unit/apache/beam/triggers/test_beam.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import asyncio from unittest import mock import pytest @@ -134,17 +135,41 @@ async def test_beam_trigger_exception_should_execute_successfully( assert TriggerEvent({"status": "error", "message": "Test exception"}) == actual @pytest.mark.asyncio - async def test_beam_trigger_gcs_provide_file_should_execute_successfully(self, python_trigger): + async def test_beam_trigger_gcs_provide_file_should_execute_successfully( + self, python_trigger, monkeypatch + ): """ - Test that BeamPythonPipelineTrigger downloads GCS provide file correct. + Test that BeamPythonPipelineTrigger downloads GCS provide file correctly with GCSAsyncHook. """ + TEST_GCS_PY_FILE = "gs://bucket/path/file.py" python_trigger.py_file = TEST_GCS_PY_FILE - with mock.patch("airflow.providers.google.cloud.hooks.gcs.GCSHook") as mock_gcs_hook: - mock_gcs_hook.return_value.provide_file.return_value = "mocked_temp_file" - generator = python_trigger.run() - await generator.asend(None) - mock_gcs_hook.assert_called_once_with(gcp_conn_id=python_trigger.gcp_conn_id) - mock_gcs_hook.return_value.provide_file.assert_called_once_with(object_url=TEST_GCS_PY_FILE) + + with mock.patch("airflow.providers.google.cloud.hooks.gcs.GCSAsyncHook") as MockAsyncHook: + async_hook_instance = MockAsyncHook.return_value + + class DummyCM: + def __enter__(self): + return "mocked_temp_file" + + def __exit__(self, exc_type, exc, tb): + return False + + sync_hook = mock.Mock(name="SyncGCSHook") + sync_hook.provide_file.return_value = DummyCM() + + async_hook_instance.get_sync_hook = mock.AsyncMock(return_value=sync_hook) + + fake_loop = mock.Mock() + fake_loop.run_in_executor = mock.AsyncMock(return_value="mocked_temp_file") + monkeypatch.setattr(asyncio, "get_running_loop", lambda: fake_loop) + + gen = python_trigger.run() + await gen.asend(None) + + MockAsyncHook.assert_called_once_with(gcp_conn_id=python_trigger.gcp_conn_id) + async_hook_instance.get_sync_hook.assert_awaited_once() + sync_hook.provide_file.assert_called_once_with(object_url=TEST_GCS_PY_FILE) + fake_loop.run_in_executor.assert_awaited_once() class TestBeamJavaPipelineTrigger: @@ -211,15 +236,35 @@ async def test_beam_trigger_exception_should_execute_successfully( assert TriggerEvent({"status": "error", "message": "Test exception"}) == actual @pytest.mark.asyncio - async def test_beam_trigger_gcs_provide_file_should_execute_successfully(self, java_trigger): + async def test_beam_trigger_gcs_provide_file_should_execute_successfully(self, java_trigger, monkeypatch): """ - Test that BeamJavaPipelineTrigger downloads GCS provide file correct. + Test that BeamJavaPipelineTrigger downloads GCS provide file correctly with GCSAsyncHook. """ java_trigger.jar = TEST_GCS_JAR_FILE - with mock.patch("airflow.providers.google.cloud.hooks.gcs.GCSHook") as mock_gcs_hook: - mock_gcs_hook.return_value.provide_file.return_value = "mocked_temp_file" - generator = java_trigger.run() - await generator.asend(None) - mock_gcs_hook.assert_called_once_with(gcp_conn_id=java_trigger.gcp_conn_id) - mock_gcs_hook.return_value.provide_file.assert_called_once_with(object_url=TEST_GCS_JAR_FILE) + with mock.patch("airflow.providers.google.cloud.hooks.gcs.GCSAsyncHook") as MockAsyncHook: + async_hook_instance = MockAsyncHook.return_value + + class DummyCM: + def __enter__(self): + return "mocked_temp_file" + + def __exit__(self, exc_type, exc, tb): + return False + + sync_hook = mock.Mock(name="SyncGCSHook") + sync_hook.provide_file.return_value = DummyCM() + + async_hook_instance.get_sync_hook = mock.AsyncMock(return_value=sync_hook) + + fake_loop = mock.Mock() + fake_loop.run_in_executor = mock.AsyncMock(return_value="mocked_temp_file") + monkeypatch.setattr(asyncio, "get_running_loop", lambda: fake_loop) + + gen = java_trigger.run() + await gen.asend(None) + + MockAsyncHook.assert_called_once_with(gcp_conn_id=java_trigger.gcp_conn_id) + async_hook_instance.get_sync_hook.assert_awaited_once() + sync_hook.provide_file.assert_called_once_with(object_url=TEST_GCS_JAR_FILE) + fake_loop.run_in_executor.assert_awaited_once() diff --git a/providers/google/tests/system/google/cloud/dataflow/example_dataflow_native_java.py b/providers/google/tests/system/google/cloud/dataflow/example_dataflow_native_java.py index 550b0381e6a62..ed74ede79d472 100644 --- a/providers/google/tests/system/google/cloud/dataflow/example_dataflow_native_java.py +++ b/providers/google/tests/system/google/cloud/dataflow/example_dataflow_native_java.py @@ -87,7 +87,7 @@ # [START howto_operator_start_java_job_local_jar] start_java_job_direct = BeamRunJavaPipelineOperator( task_id="start_java_job_direct", - jar=LOCAL_JAR, + jar=GCS_JAR, pipeline_options={ "output": GCS_OUTPUT, }, @@ -102,7 +102,7 @@ start_java_job_direct_deferrable = BeamRunJavaPipelineOperator( task_id="start_java_job_direct_deferrable", - jar=GCS_JAR, + jar=LOCAL_JAR, pipeline_options={ "output": GCS_OUTPUT, }, From b59cedddeed94ca22a685271fd81c3e487cebb82 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Fri, 3 Oct 2025 11:45:43 +0530 Subject: [PATCH 184/338] Bump ruff to 0.13.3 and prek to 0.2.3 (#56355) --- .github/actions/install-prek/action.yml | 2 +- .pre-commit-config.yaml | 2 +- Dockerfile.ci | 2 +- airflow-core/docs/best-practices.rst | 2 +- dev/breeze/README.md | 2 +- dev/breeze/doc/ci/02_images.md | 2 +- dev/breeze/pyproject.toml | 2 +- .../commands/release_management_commands.py | 2 +- dev/breeze/uv.lock | 1580 ++++++++--------- devel-common/pyproject.toml | 2 +- scripts/ci/prek/check_imports_in_providers.py | 2 +- scripts/ci/prek/ruff_format.py | 2 +- 12 files changed, 801 insertions(+), 801 deletions(-) diff --git a/.github/actions/install-prek/action.yml b/.github/actions/install-prek/action.yml index 4d8e002f0ecea..ecf2ac7a55cae 100644 --- a/.github/actions/install-prek/action.yml +++ b/.github/actions/install-prek/action.yml @@ -27,7 +27,7 @@ inputs: default: "0.8.22" # Keep this comment to allow automatic replacement of uv version prek-version: description: 'prek version to use' - default: "0.2.1" # Keep this comment to allow automatic replacement of prek version + default: "0.2.3" # Keep this comment to allow automatic replacement of prek version skip-prek-hooks: description: "Skip some prek hooks from installation" default: "" diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index bd8fc3f90725d..dd1ff6006daa7 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -412,7 +412,7 @@ repos: types_or: [python, pyi] args: [--fix] require_serial: true - additional_dependencies: ['ruff==0.13.2'] + additional_dependencies: ['ruff==0.13.3'] exclude: ^airflow-core/tests/unit/dags/test_imports\.py$|^performance/tests/test_.*\.py$ - id: ruff-format name: Run 'ruff format' diff --git a/Dockerfile.ci b/Dockerfile.ci index 1260d680ba78e..3ddf9297ebec9 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1678,7 +1678,7 @@ COPY --from=scripts common.sh install_packaging_tools.sh install_additional_depe ARG AIRFLOW_PIP_VERSION=25.2 # ARG AIRFLOW_PIP_VERSION="git+https://github.com/pypa/pip.git@main" ARG AIRFLOW_UV_VERSION=0.8.22 -ARG AIRFLOW_PREK_VERSION="0.2.1" +ARG AIRFLOW_PREK_VERSION="0.2.3" # UV_LINK_MODE=copy is needed since we are using cache mounted from the host ENV AIRFLOW_PIP_VERSION=${AIRFLOW_PIP_VERSION} \ diff --git a/airflow-core/docs/best-practices.rst b/airflow-core/docs/best-practices.rst index 368b5c1363941..297884a918573 100644 --- a/airflow-core/docs/best-practices.rst +++ b/airflow-core/docs/best-practices.rst @@ -310,7 +310,7 @@ Installing and Using ruff .. code-block:: bash - pip install "ruff>=0.13.2" + pip install "ruff>=0.13.3" 2. **Running ruff**: Execute ``ruff`` to check your Dags for potential issues: diff --git a/dev/breeze/README.md b/dev/breeze/README.md index 8c79ee50a5781..c9699263cd9a9 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -135,6 +135,6 @@ PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PREK. --------------------------------------------------------------------------------------------------------- -Package config hash: 500727c0d31e88142b441b8e58c9db20a495af2994d65a4cc1a3e1e7d6eab8466a5e87242fbf361b888c5dd271861284dfb286409adb26b8f50e25d5fdfec034 +Package config hash: 71b4a2063359e3487de35d5f6b6acc11096799ccf64bf9e4497a3c7d258cba5fca384e8e199b5933bb1c46c262bdf3ff636167c8863b48905cb31cd10b3312c5 --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/doc/ci/02_images.md b/dev/breeze/doc/ci/02_images.md index cbfb37da27b65..4cd1ceaa77399 100644 --- a/dev/breeze/doc/ci/02_images.md +++ b/dev/breeze/doc/ci/02_images.md @@ -444,7 +444,7 @@ can be used for CI images: | `ADDITIONAL_DEV_APT_ENV` | | Additional env variables defined when installing dev deps | | `AIRFLOW_PIP_VERSION` | `25.2` | `pip` version used. | | `AIRFLOW_UV_VERSION` | `0.8.22` | `uv` version used. | -| `AIRFLOW_PREK_VERSION` | `0.2.1` | `prek` version used. | +| `AIRFLOW_PREK_VERSION` | `0.2.3` | `prek` version used. | | `AIRFLOW_USE_UV` | `true` | Whether to use UV for installation. | | `PIP_PROGRESS_BAR` | `on` | Progress bar for PIP installation | diff --git a/dev/breeze/pyproject.toml b/dev/breeze/pyproject.toml index 82dc1a9a1e666..97d37c583666e 100644 --- a/dev/breeze/pyproject.toml +++ b/dev/breeze/pyproject.toml @@ -60,7 +60,7 @@ dependencies = [ "jinja2>=3.1.5", "jsonschema>=4.19.1", "packaging>=25.0", - "prek>=0.2.1", + "prek>=0.2.3", "psutil>=5.9.6", "pygithub>=2.1.1", "pytest-xdist>=3.3.1", diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py index fb942c01d3737..142787b5cec19 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py @@ -264,7 +264,7 @@ class VersionedFile(NamedTuple): AIRFLOW_USE_UV = False GITPYTHON_VERSION = "3.1.45" RICH_VERSION = "14.1.0" -PREK_VERSION = "0.2.1" +PREK_VERSION = "0.2.3" HATCH_VERSION = "1.14.2" PYYAML_VERSION = "6.0.3" diff --git a/dev/breeze/uv.lock b/dev/breeze/uv.lock index e85ead2d289d2..83dfe2038ac1e 100644 --- a/dev/breeze/uv.lock +++ b/dev/breeze/uv.lock @@ -1,5 +1,5 @@ version = 1 -revision = 1 +revision = 3 requires-python = ">=3.10, !=3.13" resolution-markers = [ "python_full_version >= '3.13'", @@ -16,9 +16,9 @@ dependencies = [ { name = "sniffio" }, { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/f1/b4/636b3b65173d3ce9a38ef5f0522789614e590dab6a8d505340a4efe4c567/anyio-4.10.0.tar.gz", hash = "sha256:3f3fae35c96039744587aa5b8371e7e8e603c0702999535961dd336026973ba6", size = 213252 } +sdist = { url = "https://files.pythonhosted.org/packages/f1/b4/636b3b65173d3ce9a38ef5f0522789614e590dab6a8d505340a4efe4c567/anyio-4.10.0.tar.gz", hash = "sha256:3f3fae35c96039744587aa5b8371e7e8e603c0702999535961dd336026973ba6", size = 213252, upload-time = "2025-08-04T08:54:26.451Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/6f/12/e5e0282d673bb9746bacfb6e2dba8719989d3660cdb2ea79aee9a9651afb/anyio-4.10.0-py3-none-any.whl", hash = "sha256:60e474ac86736bbfd6f210f7a61218939c318f43f9972497381f1c5e930ed3d1", size = 107213 }, + { url = "https://files.pythonhosted.org/packages/6f/12/e5e0282d673bb9746bacfb6e2dba8719989d3660cdb2ea79aee9a9651afb/anyio-4.10.0-py3-none-any.whl", hash = "sha256:60e474ac86736bbfd6f210f7a61218939c318f43f9972497381f1c5e930ed3d1", size = 107213, upload-time = "2025-08-04T08:54:24.882Z" }, ] [[package]] @@ -75,7 +75,7 @@ requires-dist = [ { name = "jinja2", specifier = ">=3.1.5" }, { name = "jsonschema", specifier = ">=4.19.1" }, { name = "packaging", specifier = ">=25.0" }, - { name = "prek", specifier = ">=0.2.1" }, + { name = "prek", specifier = ">=0.2.3" }, { name = "psutil", specifier = ">=5.9.6" }, { name = "pygithub", specifier = ">=2.1.1" }, { name = "pytest", specifier = ">=8.3.3" }, @@ -96,18 +96,18 @@ requires-dist = [ name = "attrs" version = "25.3.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/5a/b0/1367933a8532ee6ff8d63537de4f1177af4bff9f3e829baf7331f595bb24/attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b", size = 812032 } +sdist = { url = "https://files.pythonhosted.org/packages/5a/b0/1367933a8532ee6ff8d63537de4f1177af4bff9f3e829baf7331f595bb24/attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b", size = 812032, upload-time = "2025-03-13T11:10:22.779Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/77/06/bb80f5f86020c4551da315d78b3ab75e8228f89f0162f2c3a819e407941a/attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3", size = 63815 }, + { url = "https://files.pythonhosted.org/packages/77/06/bb80f5f86020c4551da315d78b3ab75e8228f89f0162f2c3a819e407941a/attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3", size = 63815, upload-time = "2025-03-13T11:10:21.14Z" }, ] [[package]] name = "backports-tarfile" version = "1.2.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/86/72/cd9b395f25e290e633655a100af28cb253e4393396264a98bd5f5951d50f/backports_tarfile-1.2.0.tar.gz", hash = "sha256:d75e02c268746e1b8144c278978b6e98e85de6ad16f8e4b0844a154557eca991", size = 86406 } +sdist = { url = "https://files.pythonhosted.org/packages/86/72/cd9b395f25e290e633655a100af28cb253e4393396264a98bd5f5951d50f/backports_tarfile-1.2.0.tar.gz", hash = "sha256:d75e02c268746e1b8144c278978b6e98e85de6ad16f8e4b0844a154557eca991", size = 86406, upload-time = "2024-05-28T17:01:54.731Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b9/fa/123043af240e49752f1c4bd24da5053b6bd00cad78c2be53c0d1e8b975bc/backports.tarfile-1.2.0-py3-none-any.whl", hash = "sha256:77e284d754527b01fb1e6fa8a1afe577858ebe4e9dad8919e34c862cb399bc34", size = 30181 }, + { url = "https://files.pythonhosted.org/packages/b9/fa/123043af240e49752f1c4bd24da5053b6bd00cad78c2be53c0d1e8b975bc/backports.tarfile-1.2.0-py3-none-any.whl", hash = "sha256:77e284d754527b01fb1e6fa8a1afe577858ebe4e9dad8919e34c862cb399bc34", size = 30181, upload-time = "2024-05-28T17:01:53.112Z" }, ] [[package]] @@ -123,25 +123,25 @@ dependencies = [ { name = "tomli", marker = "python_full_version < '3.11'" }, { name = "typing-extensions", marker = "python_full_version < '3.11'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/94/49/26a7b0f3f35da4b5a65f081943b7bcd22d7002f5f0fb8098ec1ff21cb6ef/black-25.1.0.tar.gz", hash = "sha256:33496d5cd1222ad73391352b4ae8da15253c5de89b93a80b3e2c8d9a19ec2666", size = 649449 } +sdist = { url = "https://files.pythonhosted.org/packages/94/49/26a7b0f3f35da4b5a65f081943b7bcd22d7002f5f0fb8098ec1ff21cb6ef/black-25.1.0.tar.gz", hash = "sha256:33496d5cd1222ad73391352b4ae8da15253c5de89b93a80b3e2c8d9a19ec2666", size = 649449, upload-time = "2025-01-29T04:15:40.373Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/4d/3b/4ba3f93ac8d90410423fdd31d7541ada9bcee1df32fb90d26de41ed40e1d/black-25.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:759e7ec1e050a15f89b770cefbf91ebee8917aac5c20483bc2d80a6c3a04df32", size = 1629419 }, - { url = "https://files.pythonhosted.org/packages/b4/02/0bde0485146a8a5e694daed47561785e8b77a0466ccc1f3e485d5ef2925e/black-25.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0e519ecf93120f34243e6b0054db49c00a35f84f195d5bce7e9f5cfc578fc2da", size = 1461080 }, - { url = "https://files.pythonhosted.org/packages/52/0e/abdf75183c830eaca7589144ff96d49bce73d7ec6ad12ef62185cc0f79a2/black-25.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:055e59b198df7ac0b7efca5ad7ff2516bca343276c466be72eb04a3bcc1f82d7", size = 1766886 }, - { url = "https://files.pythonhosted.org/packages/dc/a6/97d8bb65b1d8a41f8a6736222ba0a334db7b7b77b8023ab4568288f23973/black-25.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:db8ea9917d6f8fc62abd90d944920d95e73c83a5ee3383493e35d271aca872e9", size = 1419404 }, - { url = "https://files.pythonhosted.org/packages/7e/4f/87f596aca05c3ce5b94b8663dbfe242a12843caaa82dd3f85f1ffdc3f177/black-25.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a39337598244de4bae26475f77dda852ea00a93bd4c728e09eacd827ec929df0", size = 1614372 }, - { url = "https://files.pythonhosted.org/packages/e7/d0/2c34c36190b741c59c901e56ab7f6e54dad8df05a6272a9747ecef7c6036/black-25.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:96c1c7cd856bba8e20094e36e0f948718dc688dba4a9d78c3adde52b9e6c2299", size = 1442865 }, - { url = "https://files.pythonhosted.org/packages/21/d4/7518c72262468430ead45cf22bd86c883a6448b9eb43672765d69a8f1248/black-25.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:bce2e264d59c91e52d8000d507eb20a9aca4a778731a08cfff7e5ac4a4bb7096", size = 1749699 }, - { url = "https://files.pythonhosted.org/packages/58/db/4f5beb989b547f79096e035c4981ceb36ac2b552d0ac5f2620e941501c99/black-25.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:172b1dbff09f86ce6f4eb8edf9dede08b1fce58ba194c87d7a4f1a5aa2f5b3c2", size = 1428028 }, - { url = "https://files.pythonhosted.org/packages/83/71/3fe4741df7adf015ad8dfa082dd36c94ca86bb21f25608eb247b4afb15b2/black-25.1.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:4b60580e829091e6f9238c848ea6750efed72140b91b048770b64e74fe04908b", size = 1650988 }, - { url = "https://files.pythonhosted.org/packages/13/f3/89aac8a83d73937ccd39bbe8fc6ac8860c11cfa0af5b1c96d081facac844/black-25.1.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1e2978f6df243b155ef5fa7e558a43037c3079093ed5d10fd84c43900f2d8ecc", size = 1453985 }, - { url = "https://files.pythonhosted.org/packages/6f/22/b99efca33f1f3a1d2552c714b1e1b5ae92efac6c43e790ad539a163d1754/black-25.1.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:3b48735872ec535027d979e8dcb20bf4f70b5ac75a8ea99f127c106a7d7aba9f", size = 1783816 }, - { url = "https://files.pythonhosted.org/packages/18/7e/a27c3ad3822b6f2e0e00d63d58ff6299a99a5b3aee69fa77cd4b0076b261/black-25.1.0-cp312-cp312-win_amd64.whl", hash = "sha256:ea0213189960bda9cf99be5b8c8ce66bb054af5e9e861249cd23471bd7b0b3ba", size = 1440860 }, - { url = "https://files.pythonhosted.org/packages/98/87/0edf98916640efa5d0696e1abb0a8357b52e69e82322628f25bf14d263d1/black-25.1.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:8f0b18a02996a836cc9c9c78e5babec10930862827b1b724ddfe98ccf2f2fe4f", size = 1650673 }, - { url = "https://files.pythonhosted.org/packages/52/e5/f7bf17207cf87fa6e9b676576749c6b6ed0d70f179a3d812c997870291c3/black-25.1.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:afebb7098bfbc70037a053b91ae8437c3857482d3a690fefc03e9ff7aa9a5fd3", size = 1453190 }, - { url = "https://files.pythonhosted.org/packages/e3/ee/adda3d46d4a9120772fae6de454c8495603c37c4c3b9c60f25b1ab6401fe/black-25.1.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:030b9759066a4ee5e5aca28c3c77f9c64789cdd4de8ac1df642c40b708be6171", size = 1782926 }, - { url = "https://files.pythonhosted.org/packages/cc/64/94eb5f45dcb997d2082f097a3944cfc7fe87e071907f677e80788a2d7b7a/black-25.1.0-cp313-cp313-win_amd64.whl", hash = "sha256:a22f402b410566e2d1c950708c77ebf5ebd5d0d88a6a2e87c86d9fb48afa0d18", size = 1442613 }, - { url = "https://files.pythonhosted.org/packages/09/71/54e999902aed72baf26bca0d50781b01838251a462612966e9fc4891eadd/black-25.1.0-py3-none-any.whl", hash = "sha256:95e8176dae143ba9097f351d174fdaf0ccd29efb414b362ae3fd72bf0f710717", size = 207646 }, + { url = "https://files.pythonhosted.org/packages/4d/3b/4ba3f93ac8d90410423fdd31d7541ada9bcee1df32fb90d26de41ed40e1d/black-25.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:759e7ec1e050a15f89b770cefbf91ebee8917aac5c20483bc2d80a6c3a04df32", size = 1629419, upload-time = "2025-01-29T05:37:06.642Z" }, + { url = "https://files.pythonhosted.org/packages/b4/02/0bde0485146a8a5e694daed47561785e8b77a0466ccc1f3e485d5ef2925e/black-25.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0e519ecf93120f34243e6b0054db49c00a35f84f195d5bce7e9f5cfc578fc2da", size = 1461080, upload-time = "2025-01-29T05:37:09.321Z" }, + { url = "https://files.pythonhosted.org/packages/52/0e/abdf75183c830eaca7589144ff96d49bce73d7ec6ad12ef62185cc0f79a2/black-25.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:055e59b198df7ac0b7efca5ad7ff2516bca343276c466be72eb04a3bcc1f82d7", size = 1766886, upload-time = "2025-01-29T04:18:24.432Z" }, + { url = "https://files.pythonhosted.org/packages/dc/a6/97d8bb65b1d8a41f8a6736222ba0a334db7b7b77b8023ab4568288f23973/black-25.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:db8ea9917d6f8fc62abd90d944920d95e73c83a5ee3383493e35d271aca872e9", size = 1419404, upload-time = "2025-01-29T04:19:04.296Z" }, + { url = "https://files.pythonhosted.org/packages/7e/4f/87f596aca05c3ce5b94b8663dbfe242a12843caaa82dd3f85f1ffdc3f177/black-25.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a39337598244de4bae26475f77dda852ea00a93bd4c728e09eacd827ec929df0", size = 1614372, upload-time = "2025-01-29T05:37:11.71Z" }, + { url = "https://files.pythonhosted.org/packages/e7/d0/2c34c36190b741c59c901e56ab7f6e54dad8df05a6272a9747ecef7c6036/black-25.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:96c1c7cd856bba8e20094e36e0f948718dc688dba4a9d78c3adde52b9e6c2299", size = 1442865, upload-time = "2025-01-29T05:37:14.309Z" }, + { url = "https://files.pythonhosted.org/packages/21/d4/7518c72262468430ead45cf22bd86c883a6448b9eb43672765d69a8f1248/black-25.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:bce2e264d59c91e52d8000d507eb20a9aca4a778731a08cfff7e5ac4a4bb7096", size = 1749699, upload-time = "2025-01-29T04:18:17.688Z" }, + { url = "https://files.pythonhosted.org/packages/58/db/4f5beb989b547f79096e035c4981ceb36ac2b552d0ac5f2620e941501c99/black-25.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:172b1dbff09f86ce6f4eb8edf9dede08b1fce58ba194c87d7a4f1a5aa2f5b3c2", size = 1428028, upload-time = "2025-01-29T04:18:51.711Z" }, + { url = "https://files.pythonhosted.org/packages/83/71/3fe4741df7adf015ad8dfa082dd36c94ca86bb21f25608eb247b4afb15b2/black-25.1.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:4b60580e829091e6f9238c848ea6750efed72140b91b048770b64e74fe04908b", size = 1650988, upload-time = "2025-01-29T05:37:16.707Z" }, + { url = "https://files.pythonhosted.org/packages/13/f3/89aac8a83d73937ccd39bbe8fc6ac8860c11cfa0af5b1c96d081facac844/black-25.1.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1e2978f6df243b155ef5fa7e558a43037c3079093ed5d10fd84c43900f2d8ecc", size = 1453985, upload-time = "2025-01-29T05:37:18.273Z" }, + { url = "https://files.pythonhosted.org/packages/6f/22/b99efca33f1f3a1d2552c714b1e1b5ae92efac6c43e790ad539a163d1754/black-25.1.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:3b48735872ec535027d979e8dcb20bf4f70b5ac75a8ea99f127c106a7d7aba9f", size = 1783816, upload-time = "2025-01-29T04:18:33.823Z" }, + { url = "https://files.pythonhosted.org/packages/18/7e/a27c3ad3822b6f2e0e00d63d58ff6299a99a5b3aee69fa77cd4b0076b261/black-25.1.0-cp312-cp312-win_amd64.whl", hash = "sha256:ea0213189960bda9cf99be5b8c8ce66bb054af5e9e861249cd23471bd7b0b3ba", size = 1440860, upload-time = "2025-01-29T04:19:12.944Z" }, + { url = "https://files.pythonhosted.org/packages/98/87/0edf98916640efa5d0696e1abb0a8357b52e69e82322628f25bf14d263d1/black-25.1.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:8f0b18a02996a836cc9c9c78e5babec10930862827b1b724ddfe98ccf2f2fe4f", size = 1650673, upload-time = "2025-01-29T05:37:20.574Z" }, + { url = "https://files.pythonhosted.org/packages/52/e5/f7bf17207cf87fa6e9b676576749c6b6ed0d70f179a3d812c997870291c3/black-25.1.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:afebb7098bfbc70037a053b91ae8437c3857482d3a690fefc03e9ff7aa9a5fd3", size = 1453190, upload-time = "2025-01-29T05:37:22.106Z" }, + { url = "https://files.pythonhosted.org/packages/e3/ee/adda3d46d4a9120772fae6de454c8495603c37c4c3b9c60f25b1ab6401fe/black-25.1.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:030b9759066a4ee5e5aca28c3c77f9c64789cdd4de8ac1df642c40b708be6171", size = 1782926, upload-time = "2025-01-29T04:18:58.564Z" }, + { url = "https://files.pythonhosted.org/packages/cc/64/94eb5f45dcb997d2082f097a3944cfc7fe87e071907f677e80788a2d7b7a/black-25.1.0-cp313-cp313-win_amd64.whl", hash = "sha256:a22f402b410566e2d1c950708c77ebf5ebd5d0d88a6a2e87c86d9fb48afa0d18", size = 1442613, upload-time = "2025-01-29T04:19:27.63Z" }, + { url = "https://files.pythonhosted.org/packages/09/71/54e999902aed72baf26bca0d50781b01838251a462612966e9fc4891eadd/black-25.1.0-py3-none-any.whl", hash = "sha256:95e8176dae143ba9097f351d174fdaf0ccd29efb414b362ae3fd72bf0f710717", size = 207646, upload-time = "2025-01-29T04:15:38.082Z" }, ] [[package]] @@ -153,9 +153,9 @@ dependencies = [ { name = "jmespath" }, { name = "s3transfer" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/1c/92/3ffa75ed0594ef289dde3dde9e1d62a496515313f11caee499a5dfd2241d/boto3-1.40.11.tar.gz", hash = "sha256:0c03da130467d51c6b940d19be295c56314e14ce0f0464cc86145e98d3c9e983", size = 112060 } +sdist = { url = "https://files.pythonhosted.org/packages/1c/92/3ffa75ed0594ef289dde3dde9e1d62a496515313f11caee499a5dfd2241d/boto3-1.40.11.tar.gz", hash = "sha256:0c03da130467d51c6b940d19be295c56314e14ce0f0464cc86145e98d3c9e983", size = 112060, upload-time = "2025-08-15T19:26:03.724Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/51/4a/5d33b6046d425c9b39d36a1171ea87a9c3b297ba116952b81033eae61260/boto3-1.40.11-py3-none-any.whl", hash = "sha256:9d2d211d9cb3efc9a2b2ceec3c510b4e62e389618fd5c871e74d2cbca4561ff5", size = 140072 }, + { url = "https://files.pythonhosted.org/packages/51/4a/5d33b6046d425c9b39d36a1171ea87a9c3b297ba116952b81033eae61260/boto3-1.40.11-py3-none-any.whl", hash = "sha256:9d2d211d9cb3efc9a2b2ceec3c510b4e62e389618fd5c871e74d2cbca4561ff5", size = 140072, upload-time = "2025-08-15T19:26:02.09Z" }, ] [[package]] @@ -167,27 +167,27 @@ dependencies = [ { name = "python-dateutil" }, { name = "urllib3" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/34/b2/23e4dc97d941dad612959664029f2eb843fd65ce58cc7b3c02f996b6357c/botocore-1.40.11.tar.gz", hash = "sha256:95af22e1b2230bdd5faa9d1c87e8b147028b14b531770a1148bf495967ccba5e", size = 14339310 } +sdist = { url = "https://files.pythonhosted.org/packages/34/b2/23e4dc97d941dad612959664029f2eb843fd65ce58cc7b3c02f996b6357c/botocore-1.40.11.tar.gz", hash = "sha256:95af22e1b2230bdd5faa9d1c87e8b147028b14b531770a1148bf495967ccba5e", size = 14339310, upload-time = "2025-08-15T19:25:54.286Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2d/f9/400e0da61cbbcea7868458f3a447d1191a62ae5e2852d2acdfd4d51b2843/botocore-1.40.11-py3-none-any.whl", hash = "sha256:4beca0c5f92201da1bf1bc0a55038538ad2defded32ab0638cb68f5631dcc665", size = 14005730 }, + { url = "https://files.pythonhosted.org/packages/2d/f9/400e0da61cbbcea7868458f3a447d1191a62ae5e2852d2acdfd4d51b2843/botocore-1.40.11-py3-none-any.whl", hash = "sha256:4beca0c5f92201da1bf1bc0a55038538ad2defded32ab0638cb68f5631dcc665", size = 14005730, upload-time = "2025-08-15T19:25:49.793Z" }, ] [[package]] name = "cachetools" version = "5.5.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/6c/81/3747dad6b14fa2cf53fcf10548cf5aea6913e96fab41a3c198676f8948a5/cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4", size = 28380 } +sdist = { url = "https://files.pythonhosted.org/packages/6c/81/3747dad6b14fa2cf53fcf10548cf5aea6913e96fab41a3c198676f8948a5/cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4", size = 28380, upload-time = "2025-02-20T21:01:19.524Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/72/76/20fa66124dbe6be5cafeb312ece67de6b61dd91a0247d1ea13db4ebb33c2/cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a", size = 10080 }, + { url = "https://files.pythonhosted.org/packages/72/76/20fa66124dbe6be5cafeb312ece67de6b61dd91a0247d1ea13db4ebb33c2/cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a", size = 10080, upload-time = "2025-02-20T21:01:16.647Z" }, ] [[package]] name = "certifi" version = "2025.8.3" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/dc/67/960ebe6bf230a96cda2e0abcf73af550ec4f090005363542f0765df162e0/certifi-2025.8.3.tar.gz", hash = "sha256:e564105f78ded564e3ae7c923924435e1daa7463faeab5bb932bc53ffae63407", size = 162386 } +sdist = { url = "https://files.pythonhosted.org/packages/dc/67/960ebe6bf230a96cda2e0abcf73af550ec4f090005363542f0765df162e0/certifi-2025.8.3.tar.gz", hash = "sha256:e564105f78ded564e3ae7c923924435e1daa7463faeab5bb932bc53ffae63407", size = 162386, upload-time = "2025-08-03T03:07:47.08Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e5/48/1549795ba7742c948d2ad169c1c8cdbae65bc450d6cd753d124b17c8cd32/certifi-2025.8.3-py3-none-any.whl", hash = "sha256:f6c12493cfb1b06ba2ff328595af9350c65d6644968e5d3a2ffd78699af217a5", size = 161216 }, + { url = "https://files.pythonhosted.org/packages/e5/48/1549795ba7742c948d2ad169c1c8cdbae65bc450d6cd753d124b17c8cd32/certifi-2025.8.3-py3-none-any.whl", hash = "sha256:f6c12493cfb1b06ba2ff328595af9350c65d6644968e5d3a2ffd78699af217a5", size = 161216, upload-time = "2025-08-03T03:07:45.777Z" }, ] [[package]] @@ -197,118 +197,118 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pycparser" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/90/07/f44ca684db4e4f08a3fdc6eeb9a0d15dc6883efc7b8c90357fdbf74e186c/cffi-1.17.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:df8b1c11f177bc2313ec4b2d46baec87a5f3e71fc8b45dab2ee7cae86d9aba14", size = 182191 }, - { url = "https://files.pythonhosted.org/packages/08/fd/cc2fedbd887223f9f5d170c96e57cbf655df9831a6546c1727ae13fa977a/cffi-1.17.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8f2cdc858323644ab277e9bb925ad72ae0e67f69e804f4898c070998d50b1a67", size = 178592 }, - { url = "https://files.pythonhosted.org/packages/de/cc/4635c320081c78d6ffc2cab0a76025b691a91204f4aa317d568ff9280a2d/cffi-1.17.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:edae79245293e15384b51f88b00613ba9f7198016a5948b5dddf4917d4d26382", size = 426024 }, - { url = "https://files.pythonhosted.org/packages/b6/7b/3b2b250f3aab91abe5f8a51ada1b717935fdaec53f790ad4100fe2ec64d1/cffi-1.17.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45398b671ac6d70e67da8e4224a065cec6a93541bb7aebe1b198a61b58c7b702", size = 448188 }, - { url = "https://files.pythonhosted.org/packages/d3/48/1b9283ebbf0ec065148d8de05d647a986c5f22586b18120020452fff8f5d/cffi-1.17.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad9413ccdeda48c5afdae7e4fa2192157e991ff761e7ab8fdd8926f40b160cc3", size = 455571 }, - { url = "https://files.pythonhosted.org/packages/40/87/3b8452525437b40f39ca7ff70276679772ee7e8b394934ff60e63b7b090c/cffi-1.17.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5da5719280082ac6bd9aa7becb3938dc9f9cbd57fac7d2871717b1feb0902ab6", size = 436687 }, - { url = "https://files.pythonhosted.org/packages/8d/fb/4da72871d177d63649ac449aec2e8a29efe0274035880c7af59101ca2232/cffi-1.17.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bb1a08b8008b281856e5971307cc386a8e9c5b625ac297e853d36da6efe9c17", size = 446211 }, - { url = "https://files.pythonhosted.org/packages/ab/a0/62f00bcb411332106c02b663b26f3545a9ef136f80d5df746c05878f8c4b/cffi-1.17.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:045d61c734659cc045141be4bae381a41d89b741f795af1dd018bfb532fd0df8", size = 461325 }, - { url = "https://files.pythonhosted.org/packages/36/83/76127035ed2e7e27b0787604d99da630ac3123bfb02d8e80c633f218a11d/cffi-1.17.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6883e737d7d9e4899a8a695e00ec36bd4e5e4f18fabe0aca0efe0a4b44cdb13e", size = 438784 }, - { url = "https://files.pythonhosted.org/packages/21/81/a6cd025db2f08ac88b901b745c163d884641909641f9b826e8cb87645942/cffi-1.17.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:6b8b4a92e1c65048ff98cfe1f735ef8f1ceb72e3d5f0c25fdb12087a23da22be", size = 461564 }, - { url = "https://files.pythonhosted.org/packages/f8/fe/4d41c2f200c4a457933dbd98d3cf4e911870877bd94d9656cc0fcb390681/cffi-1.17.1-cp310-cp310-win32.whl", hash = "sha256:c9c3d058ebabb74db66e431095118094d06abf53284d9c81f27300d0e0d8bc7c", size = 171804 }, - { url = "https://files.pythonhosted.org/packages/d1/b6/0b0f5ab93b0df4acc49cae758c81fe4e5ef26c3ae2e10cc69249dfd8b3ab/cffi-1.17.1-cp310-cp310-win_amd64.whl", hash = "sha256:0f048dcf80db46f0098ccac01132761580d28e28bc0f78ae0d58048063317e15", size = 181299 }, - { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264 }, - { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651 }, - { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259 }, - { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200 }, - { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235 }, - { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721 }, - { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242 }, - { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999 }, - { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242 }, - { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604 }, - { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727 }, - { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400 }, - { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178 }, - { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840 }, - { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803 }, - { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850 }, - { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729 }, - { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256 }, - { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424 }, - { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568 }, - { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736 }, - { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448 }, - { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976 }, - { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989 }, - { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802 }, - { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792 }, - { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893 }, - { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810 }, - { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200 }, - { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447 }, - { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358 }, - { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469 }, - { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475 }, - { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009 }, +sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621, upload-time = "2024-09-04T20:45:21.852Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/90/07/f44ca684db4e4f08a3fdc6eeb9a0d15dc6883efc7b8c90357fdbf74e186c/cffi-1.17.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:df8b1c11f177bc2313ec4b2d46baec87a5f3e71fc8b45dab2ee7cae86d9aba14", size = 182191, upload-time = "2024-09-04T20:43:30.027Z" }, + { url = "https://files.pythonhosted.org/packages/08/fd/cc2fedbd887223f9f5d170c96e57cbf655df9831a6546c1727ae13fa977a/cffi-1.17.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8f2cdc858323644ab277e9bb925ad72ae0e67f69e804f4898c070998d50b1a67", size = 178592, upload-time = "2024-09-04T20:43:32.108Z" }, + { url = "https://files.pythonhosted.org/packages/de/cc/4635c320081c78d6ffc2cab0a76025b691a91204f4aa317d568ff9280a2d/cffi-1.17.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:edae79245293e15384b51f88b00613ba9f7198016a5948b5dddf4917d4d26382", size = 426024, upload-time = "2024-09-04T20:43:34.186Z" }, + { url = "https://files.pythonhosted.org/packages/b6/7b/3b2b250f3aab91abe5f8a51ada1b717935fdaec53f790ad4100fe2ec64d1/cffi-1.17.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45398b671ac6d70e67da8e4224a065cec6a93541bb7aebe1b198a61b58c7b702", size = 448188, upload-time = "2024-09-04T20:43:36.286Z" }, + { url = "https://files.pythonhosted.org/packages/d3/48/1b9283ebbf0ec065148d8de05d647a986c5f22586b18120020452fff8f5d/cffi-1.17.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad9413ccdeda48c5afdae7e4fa2192157e991ff761e7ab8fdd8926f40b160cc3", size = 455571, upload-time = "2024-09-04T20:43:38.586Z" }, + { url = "https://files.pythonhosted.org/packages/40/87/3b8452525437b40f39ca7ff70276679772ee7e8b394934ff60e63b7b090c/cffi-1.17.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5da5719280082ac6bd9aa7becb3938dc9f9cbd57fac7d2871717b1feb0902ab6", size = 436687, upload-time = "2024-09-04T20:43:40.084Z" }, + { url = "https://files.pythonhosted.org/packages/8d/fb/4da72871d177d63649ac449aec2e8a29efe0274035880c7af59101ca2232/cffi-1.17.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bb1a08b8008b281856e5971307cc386a8e9c5b625ac297e853d36da6efe9c17", size = 446211, upload-time = "2024-09-04T20:43:41.526Z" }, + { url = "https://files.pythonhosted.org/packages/ab/a0/62f00bcb411332106c02b663b26f3545a9ef136f80d5df746c05878f8c4b/cffi-1.17.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:045d61c734659cc045141be4bae381a41d89b741f795af1dd018bfb532fd0df8", size = 461325, upload-time = "2024-09-04T20:43:43.117Z" }, + { url = "https://files.pythonhosted.org/packages/36/83/76127035ed2e7e27b0787604d99da630ac3123bfb02d8e80c633f218a11d/cffi-1.17.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6883e737d7d9e4899a8a695e00ec36bd4e5e4f18fabe0aca0efe0a4b44cdb13e", size = 438784, upload-time = "2024-09-04T20:43:45.256Z" }, + { url = "https://files.pythonhosted.org/packages/21/81/a6cd025db2f08ac88b901b745c163d884641909641f9b826e8cb87645942/cffi-1.17.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:6b8b4a92e1c65048ff98cfe1f735ef8f1ceb72e3d5f0c25fdb12087a23da22be", size = 461564, upload-time = "2024-09-04T20:43:46.779Z" }, + { url = "https://files.pythonhosted.org/packages/f8/fe/4d41c2f200c4a457933dbd98d3cf4e911870877bd94d9656cc0fcb390681/cffi-1.17.1-cp310-cp310-win32.whl", hash = "sha256:c9c3d058ebabb74db66e431095118094d06abf53284d9c81f27300d0e0d8bc7c", size = 171804, upload-time = "2024-09-04T20:43:48.186Z" }, + { url = "https://files.pythonhosted.org/packages/d1/b6/0b0f5ab93b0df4acc49cae758c81fe4e5ef26c3ae2e10cc69249dfd8b3ab/cffi-1.17.1-cp310-cp310-win_amd64.whl", hash = "sha256:0f048dcf80db46f0098ccac01132761580d28e28bc0f78ae0d58048063317e15", size = 181299, upload-time = "2024-09-04T20:43:49.812Z" }, + { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264, upload-time = "2024-09-04T20:43:51.124Z" }, + { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651, upload-time = "2024-09-04T20:43:52.872Z" }, + { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259, upload-time = "2024-09-04T20:43:56.123Z" }, + { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200, upload-time = "2024-09-04T20:43:57.891Z" }, + { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235, upload-time = "2024-09-04T20:44:00.18Z" }, + { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721, upload-time = "2024-09-04T20:44:01.585Z" }, + { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242, upload-time = "2024-09-04T20:44:03.467Z" }, + { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999, upload-time = "2024-09-04T20:44:05.023Z" }, + { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242, upload-time = "2024-09-04T20:44:06.444Z" }, + { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604, upload-time = "2024-09-04T20:44:08.206Z" }, + { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727, upload-time = "2024-09-04T20:44:09.481Z" }, + { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400, upload-time = "2024-09-04T20:44:10.873Z" }, + { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178, upload-time = "2024-09-04T20:44:12.232Z" }, + { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840, upload-time = "2024-09-04T20:44:13.739Z" }, + { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803, upload-time = "2024-09-04T20:44:15.231Z" }, + { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850, upload-time = "2024-09-04T20:44:17.188Z" }, + { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729, upload-time = "2024-09-04T20:44:18.688Z" }, + { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256, upload-time = "2024-09-04T20:44:20.248Z" }, + { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424, upload-time = "2024-09-04T20:44:21.673Z" }, + { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568, upload-time = "2024-09-04T20:44:23.245Z" }, + { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736, upload-time = "2024-09-04T20:44:24.757Z" }, + { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448, upload-time = "2024-09-04T20:44:26.208Z" }, + { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976, upload-time = "2024-09-04T20:44:27.578Z" }, + { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989, upload-time = "2024-09-04T20:44:28.956Z" }, + { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802, upload-time = "2024-09-04T20:44:30.289Z" }, + { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792, upload-time = "2024-09-04T20:44:32.01Z" }, + { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893, upload-time = "2024-09-04T20:44:33.606Z" }, + { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810, upload-time = "2024-09-04T20:44:35.191Z" }, + { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200, upload-time = "2024-09-04T20:44:36.743Z" }, + { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447, upload-time = "2024-09-04T20:44:38.492Z" }, + { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358, upload-time = "2024-09-04T20:44:40.046Z" }, + { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469, upload-time = "2024-09-04T20:44:41.616Z" }, + { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475, upload-time = "2024-09-04T20:44:43.733Z" }, + { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009, upload-time = "2024-09-04T20:44:45.309Z" }, ] [[package]] name = "charset-normalizer" version = "3.4.3" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/83/2d/5fd176ceb9b2fc619e63405525573493ca23441330fcdaee6bef9460e924/charset_normalizer-3.4.3.tar.gz", hash = "sha256:6fce4b8500244f6fcb71465d4a4930d132ba9ab8e71a7859e6a5d59851068d14", size = 122371 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/d6/98/f3b8013223728a99b908c9344da3aa04ee6e3fa235f19409033eda92fb78/charset_normalizer-3.4.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:fb7f67a1bfa6e40b438170ebdc8158b78dc465a5a67b6dde178a46987b244a72", size = 207695 }, - { url = "https://files.pythonhosted.org/packages/21/40/5188be1e3118c82dcb7c2a5ba101b783822cfb413a0268ed3be0468532de/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:cc9370a2da1ac13f0153780040f465839e6cccb4a1e44810124b4e22483c93fe", size = 147153 }, - { url = "https://files.pythonhosted.org/packages/37/60/5d0d74bc1e1380f0b72c327948d9c2aca14b46a9efd87604e724260f384c/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:07a0eae9e2787b586e129fdcbe1af6997f8d0e5abaa0bc98c0e20e124d67e601", size = 160428 }, - { url = "https://files.pythonhosted.org/packages/85/9a/d891f63722d9158688de58d050c59dc3da560ea7f04f4c53e769de5140f5/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:74d77e25adda8581ffc1c720f1c81ca082921329452eba58b16233ab1842141c", size = 157627 }, - { url = "https://files.pythonhosted.org/packages/65/1a/7425c952944a6521a9cfa7e675343f83fd82085b8af2b1373a2409c683dc/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d0e909868420b7049dafd3a31d45125b31143eec59235311fc4c57ea26a4acd2", size = 152388 }, - { url = "https://files.pythonhosted.org/packages/f0/c9/a2c9c2a355a8594ce2446085e2ec97fd44d323c684ff32042e2a6b718e1d/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:c6f162aabe9a91a309510d74eeb6507fab5fff92337a15acbe77753d88d9dcf0", size = 150077 }, - { url = "https://files.pythonhosted.org/packages/3b/38/20a1f44e4851aa1c9105d6e7110c9d020e093dfa5836d712a5f074a12bf7/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:4ca4c094de7771a98d7fbd67d9e5dbf1eb73efa4f744a730437d8a3a5cf994f0", size = 161631 }, - { url = "https://files.pythonhosted.org/packages/a4/fa/384d2c0f57edad03d7bec3ebefb462090d8905b4ff5a2d2525f3bb711fac/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:02425242e96bcf29a49711b0ca9f37e451da7c70562bc10e8ed992a5a7a25cc0", size = 159210 }, - { url = "https://files.pythonhosted.org/packages/33/9e/eca49d35867ca2db336b6ca27617deed4653b97ebf45dfc21311ce473c37/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:78deba4d8f9590fe4dae384aeff04082510a709957e968753ff3c48399f6f92a", size = 153739 }, - { url = "https://files.pythonhosted.org/packages/2a/91/26c3036e62dfe8de8061182d33be5025e2424002125c9500faff74a6735e/charset_normalizer-3.4.3-cp310-cp310-win32.whl", hash = "sha256:d79c198e27580c8e958906f803e63cddb77653731be08851c7df0b1a14a8fc0f", size = 99825 }, - { url = "https://files.pythonhosted.org/packages/e2/c6/f05db471f81af1fa01839d44ae2a8bfeec8d2a8b4590f16c4e7393afd323/charset_normalizer-3.4.3-cp310-cp310-win_amd64.whl", hash = "sha256:c6e490913a46fa054e03699c70019ab869e990270597018cef1d8562132c2669", size = 107452 }, - { url = "https://files.pythonhosted.org/packages/7f/b5/991245018615474a60965a7c9cd2b4efbaabd16d582a5547c47ee1c7730b/charset_normalizer-3.4.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b256ee2e749283ef3ddcff51a675ff43798d92d746d1a6e4631bf8c707d22d0b", size = 204483 }, - { url = "https://files.pythonhosted.org/packages/c7/2a/ae245c41c06299ec18262825c1569c5d3298fc920e4ddf56ab011b417efd/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:13faeacfe61784e2559e690fc53fa4c5ae97c6fcedb8eb6fb8d0a15b475d2c64", size = 145520 }, - { url = "https://files.pythonhosted.org/packages/3a/a4/b3b6c76e7a635748c4421d2b92c7b8f90a432f98bda5082049af37ffc8e3/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:00237675befef519d9af72169d8604a067d92755e84fe76492fef5441db05b91", size = 158876 }, - { url = "https://files.pythonhosted.org/packages/e2/e6/63bb0e10f90a8243c5def74b5b105b3bbbfb3e7bb753915fe333fb0c11ea/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:585f3b2a80fbd26b048a0be90c5aae8f06605d3c92615911c3a2b03a8a3b796f", size = 156083 }, - { url = "https://files.pythonhosted.org/packages/87/df/b7737ff046c974b183ea9aa111b74185ac8c3a326c6262d413bd5a1b8c69/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0e78314bdc32fa80696f72fa16dc61168fda4d6a0c014e0380f9d02f0e5d8a07", size = 150295 }, - { url = "https://files.pythonhosted.org/packages/61/f1/190d9977e0084d3f1dc169acd060d479bbbc71b90bf3e7bf7b9927dec3eb/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:96b2b3d1a83ad55310de8c7b4a2d04d9277d5591f40761274856635acc5fcb30", size = 148379 }, - { url = "https://files.pythonhosted.org/packages/4c/92/27dbe365d34c68cfe0ca76f1edd70e8705d82b378cb54ebbaeabc2e3029d/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:939578d9d8fd4299220161fdd76e86c6a251987476f5243e8864a7844476ba14", size = 160018 }, - { url = "https://files.pythonhosted.org/packages/99/04/baae2a1ea1893a01635d475b9261c889a18fd48393634b6270827869fa34/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:fd10de089bcdcd1be95a2f73dbe6254798ec1bda9f450d5828c96f93e2536b9c", size = 157430 }, - { url = "https://files.pythonhosted.org/packages/2f/36/77da9c6a328c54d17b960c89eccacfab8271fdaaa228305330915b88afa9/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1e8ac75d72fa3775e0b7cb7e4629cec13b7514d928d15ef8ea06bca03ef01cae", size = 151600 }, - { url = "https://files.pythonhosted.org/packages/64/d4/9eb4ff2c167edbbf08cdd28e19078bf195762e9bd63371689cab5ecd3d0d/charset_normalizer-3.4.3-cp311-cp311-win32.whl", hash = "sha256:6cf8fd4c04756b6b60146d98cd8a77d0cdae0e1ca20329da2ac85eed779b6849", size = 99616 }, - { url = "https://files.pythonhosted.org/packages/f4/9c/996a4a028222e7761a96634d1820de8a744ff4327a00ada9c8942033089b/charset_normalizer-3.4.3-cp311-cp311-win_amd64.whl", hash = "sha256:31a9a6f775f9bcd865d88ee350f0ffb0e25936a7f930ca98995c05abf1faf21c", size = 107108 }, - { url = "https://files.pythonhosted.org/packages/e9/5e/14c94999e418d9b87682734589404a25854d5f5d0408df68bc15b6ff54bb/charset_normalizer-3.4.3-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:e28e334d3ff134e88989d90ba04b47d84382a828c061d0d1027b1b12a62b39b1", size = 205655 }, - { url = "https://files.pythonhosted.org/packages/7d/a8/c6ec5d389672521f644505a257f50544c074cf5fc292d5390331cd6fc9c3/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:0cacf8f7297b0c4fcb74227692ca46b4a5852f8f4f24b3c766dd94a1075c4884", size = 146223 }, - { url = "https://files.pythonhosted.org/packages/fc/eb/a2ffb08547f4e1e5415fb69eb7db25932c52a52bed371429648db4d84fb1/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c6fd51128a41297f5409deab284fecbe5305ebd7e5a1f959bee1c054622b7018", size = 159366 }, - { url = "https://files.pythonhosted.org/packages/82/10/0fd19f20c624b278dddaf83b8464dcddc2456cb4b02bb902a6da126b87a1/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:3cfb2aad70f2c6debfbcb717f23b7eb55febc0bb23dcffc0f076009da10c6392", size = 157104 }, - { url = "https://files.pythonhosted.org/packages/16/ab/0233c3231af734f5dfcf0844aa9582d5a1466c985bbed6cedab85af9bfe3/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1606f4a55c0fd363d754049cdf400175ee96c992b1f8018b993941f221221c5f", size = 151830 }, - { url = "https://files.pythonhosted.org/packages/ae/02/e29e22b4e02839a0e4a06557b1999d0a47db3567e82989b5bb21f3fbbd9f/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:027b776c26d38b7f15b26a5da1044f376455fb3766df8fc38563b4efbc515154", size = 148854 }, - { url = "https://files.pythonhosted.org/packages/05/6b/e2539a0a4be302b481e8cafb5af8792da8093b486885a1ae4d15d452bcec/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:42e5088973e56e31e4fa58eb6bd709e42fc03799c11c42929592889a2e54c491", size = 160670 }, - { url = "https://files.pythonhosted.org/packages/31/e7/883ee5676a2ef217a40ce0bffcc3d0dfbf9e64cbcfbdf822c52981c3304b/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:cc34f233c9e71701040d772aa7490318673aa7164a0efe3172b2981218c26d93", size = 158501 }, - { url = "https://files.pythonhosted.org/packages/c1/35/6525b21aa0db614cf8b5792d232021dca3df7f90a1944db934efa5d20bb1/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:320e8e66157cc4e247d9ddca8e21f427efc7a04bbd0ac8a9faf56583fa543f9f", size = 153173 }, - { url = "https://files.pythonhosted.org/packages/50/ee/f4704bad8201de513fdc8aac1cabc87e38c5818c93857140e06e772b5892/charset_normalizer-3.4.3-cp312-cp312-win32.whl", hash = "sha256:fb6fecfd65564f208cbf0fba07f107fb661bcd1a7c389edbced3f7a493f70e37", size = 99822 }, - { url = "https://files.pythonhosted.org/packages/39/f5/3b3836ca6064d0992c58c7561c6b6eee1b3892e9665d650c803bd5614522/charset_normalizer-3.4.3-cp312-cp312-win_amd64.whl", hash = "sha256:86df271bf921c2ee3818f0522e9a5b8092ca2ad8b065ece5d7d9d0e9f4849bcc", size = 107543 }, - { url = "https://files.pythonhosted.org/packages/65/ca/2135ac97709b400c7654b4b764daf5c5567c2da45a30cdd20f9eefe2d658/charset_normalizer-3.4.3-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:14c2a87c65b351109f6abfc424cab3927b3bdece6f706e4d12faaf3d52ee5efe", size = 205326 }, - { url = "https://files.pythonhosted.org/packages/71/11/98a04c3c97dd34e49c7d247083af03645ca3730809a5509443f3c37f7c99/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:41d1fc408ff5fdfb910200ec0e74abc40387bccb3252f3f27c0676731df2b2c8", size = 146008 }, - { url = "https://files.pythonhosted.org/packages/60/f5/4659a4cb3c4ec146bec80c32d8bb16033752574c20b1252ee842a95d1a1e/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:1bb60174149316da1c35fa5233681f7c0f9f514509b8e399ab70fea5f17e45c9", size = 159196 }, - { url = "https://files.pythonhosted.org/packages/86/9e/f552f7a00611f168b9a5865a1414179b2c6de8235a4fa40189f6f79a1753/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:30d006f98569de3459c2fc1f2acde170b7b2bd265dc1943e87e1a4efe1b67c31", size = 156819 }, - { url = "https://files.pythonhosted.org/packages/7e/95/42aa2156235cbc8fa61208aded06ef46111c4d3f0de233107b3f38631803/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:416175faf02e4b0810f1f38bcb54682878a4af94059a1cd63b8747244420801f", size = 151350 }, - { url = "https://files.pythonhosted.org/packages/c2/a9/3865b02c56f300a6f94fc631ef54f0a8a29da74fb45a773dfd3dcd380af7/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6aab0f181c486f973bc7262a97f5aca3ee7e1437011ef0c2ec04b5a11d16c927", size = 148644 }, - { url = "https://files.pythonhosted.org/packages/77/d9/cbcf1a2a5c7d7856f11e7ac2d782aec12bdfea60d104e60e0aa1c97849dc/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:fdabf8315679312cfa71302f9bd509ded4f2f263fb5b765cf1433b39106c3cc9", size = 160468 }, - { url = "https://files.pythonhosted.org/packages/f6/42/6f45efee8697b89fda4d50580f292b8f7f9306cb2971d4b53f8914e4d890/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:bd28b817ea8c70215401f657edef3a8aa83c29d447fb0b622c35403780ba11d5", size = 158187 }, - { url = "https://files.pythonhosted.org/packages/70/99/f1c3bdcfaa9c45b3ce96f70b14f070411366fa19549c1d4832c935d8e2c3/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:18343b2d246dc6761a249ba1fb13f9ee9a2bcd95decc767319506056ea4ad4dc", size = 152699 }, - { url = "https://files.pythonhosted.org/packages/a3/ad/b0081f2f99a4b194bcbb1934ef3b12aa4d9702ced80a37026b7607c72e58/charset_normalizer-3.4.3-cp313-cp313-win32.whl", hash = "sha256:6fb70de56f1859a3f71261cbe41005f56a7842cc348d3aeb26237560bfa5e0ce", size = 99580 }, - { url = "https://files.pythonhosted.org/packages/9a/8f/ae790790c7b64f925e5c953b924aaa42a243fb778fed9e41f147b2a5715a/charset_normalizer-3.4.3-cp313-cp313-win_amd64.whl", hash = "sha256:cf1ebb7d78e1ad8ec2a8c4732c7be2e736f6e5123a4146c5b89c9d1f585f8cef", size = 107366 }, - { url = "https://files.pythonhosted.org/packages/8e/91/b5a06ad970ddc7a0e513112d40113e834638f4ca1120eb727a249fb2715e/charset_normalizer-3.4.3-cp314-cp314-macosx_10_13_universal2.whl", hash = "sha256:3cd35b7e8aedeb9e34c41385fda4f73ba609e561faedfae0a9e75e44ac558a15", size = 204342 }, - { url = "https://files.pythonhosted.org/packages/ce/ec/1edc30a377f0a02689342f214455c3f6c2fbedd896a1d2f856c002fc3062/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b89bc04de1d83006373429975f8ef9e7932534b8cc9ca582e4db7d20d91816db", size = 145995 }, - { url = "https://files.pythonhosted.org/packages/17/e5/5e67ab85e6d22b04641acb5399c8684f4d37caf7558a53859f0283a650e9/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:2001a39612b241dae17b4687898843f254f8748b796a2e16f1051a17078d991d", size = 158640 }, - { url = "https://files.pythonhosted.org/packages/f1/e5/38421987f6c697ee3722981289d554957c4be652f963d71c5e46a262e135/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:8dcfc373f888e4fb39a7bc57e93e3b845e7f462dacc008d9749568b1c4ece096", size = 156636 }, - { url = "https://files.pythonhosted.org/packages/a0/e4/5a075de8daa3ec0745a9a3b54467e0c2967daaaf2cec04c845f73493e9a1/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:18b97b8404387b96cdbd30ad660f6407799126d26a39ca65729162fd810a99aa", size = 150939 }, - { url = "https://files.pythonhosted.org/packages/02/f7/3611b32318b30974131db62b4043f335861d4d9b49adc6d57c1149cc49d4/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:ccf600859c183d70eb47e05a44cd80a4ce77394d1ac0f79dbd2dd90a69a3a049", size = 148580 }, - { url = "https://files.pythonhosted.org/packages/7e/61/19b36f4bd67f2793ab6a99b979b4e4f3d8fc754cbdffb805335df4337126/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:53cd68b185d98dde4ad8990e56a58dea83a4162161b1ea9272e5c9182ce415e0", size = 159870 }, - { url = "https://files.pythonhosted.org/packages/06/57/84722eefdd338c04cf3030ada66889298eaedf3e7a30a624201e0cbe424a/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:30a96e1e1f865f78b030d65241c1ee850cdf422d869e9028e2fc1d5e4db73b92", size = 157797 }, - { url = "https://files.pythonhosted.org/packages/72/2a/aff5dd112b2f14bcc3462c312dce5445806bfc8ab3a7328555da95330e4b/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:d716a916938e03231e86e43782ca7878fb602a125a91e7acb8b5112e2e96ac16", size = 152224 }, - { url = "https://files.pythonhosted.org/packages/b7/8c/9839225320046ed279c6e839d51f028342eb77c91c89b8ef2549f951f3ec/charset_normalizer-3.4.3-cp314-cp314-win32.whl", hash = "sha256:c6dbd0ccdda3a2ba7c2ecd9d77b37f3b5831687d8dc1b6ca5f56a4880cc7b7ce", size = 100086 }, - { url = "https://files.pythonhosted.org/packages/ee/7a/36fbcf646e41f710ce0a563c1c9a343c6edf9be80786edeb15b6f62e17db/charset_normalizer-3.4.3-cp314-cp314-win_amd64.whl", hash = "sha256:73dc19b562516fc9bcf6e5d6e596df0b4eb98d87e4f79f3ae71840e6ed21361c", size = 107400 }, - { url = "https://files.pythonhosted.org/packages/8a/1f/f041989e93b001bc4e44bb1669ccdcf54d3f00e628229a85b08d330615c5/charset_normalizer-3.4.3-py3-none-any.whl", hash = "sha256:ce571ab16d890d23b5c278547ba694193a45011ff86a9162a71307ed9f86759a", size = 53175 }, +sdist = { url = "https://files.pythonhosted.org/packages/83/2d/5fd176ceb9b2fc619e63405525573493ca23441330fcdaee6bef9460e924/charset_normalizer-3.4.3.tar.gz", hash = "sha256:6fce4b8500244f6fcb71465d4a4930d132ba9ab8e71a7859e6a5d59851068d14", size = 122371, upload-time = "2025-08-09T07:57:28.46Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d6/98/f3b8013223728a99b908c9344da3aa04ee6e3fa235f19409033eda92fb78/charset_normalizer-3.4.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:fb7f67a1bfa6e40b438170ebdc8158b78dc465a5a67b6dde178a46987b244a72", size = 207695, upload-time = "2025-08-09T07:55:36.452Z" }, + { url = "https://files.pythonhosted.org/packages/21/40/5188be1e3118c82dcb7c2a5ba101b783822cfb413a0268ed3be0468532de/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:cc9370a2da1ac13f0153780040f465839e6cccb4a1e44810124b4e22483c93fe", size = 147153, upload-time = "2025-08-09T07:55:38.467Z" }, + { url = "https://files.pythonhosted.org/packages/37/60/5d0d74bc1e1380f0b72c327948d9c2aca14b46a9efd87604e724260f384c/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:07a0eae9e2787b586e129fdcbe1af6997f8d0e5abaa0bc98c0e20e124d67e601", size = 160428, upload-time = "2025-08-09T07:55:40.072Z" }, + { url = "https://files.pythonhosted.org/packages/85/9a/d891f63722d9158688de58d050c59dc3da560ea7f04f4c53e769de5140f5/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:74d77e25adda8581ffc1c720f1c81ca082921329452eba58b16233ab1842141c", size = 157627, upload-time = "2025-08-09T07:55:41.706Z" }, + { url = "https://files.pythonhosted.org/packages/65/1a/7425c952944a6521a9cfa7e675343f83fd82085b8af2b1373a2409c683dc/charset_normalizer-3.4.3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d0e909868420b7049dafd3a31d45125b31143eec59235311fc4c57ea26a4acd2", size = 152388, upload-time = "2025-08-09T07:55:43.262Z" }, + { url = "https://files.pythonhosted.org/packages/f0/c9/a2c9c2a355a8594ce2446085e2ec97fd44d323c684ff32042e2a6b718e1d/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:c6f162aabe9a91a309510d74eeb6507fab5fff92337a15acbe77753d88d9dcf0", size = 150077, upload-time = "2025-08-09T07:55:44.903Z" }, + { url = "https://files.pythonhosted.org/packages/3b/38/20a1f44e4851aa1c9105d6e7110c9d020e093dfa5836d712a5f074a12bf7/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:4ca4c094de7771a98d7fbd67d9e5dbf1eb73efa4f744a730437d8a3a5cf994f0", size = 161631, upload-time = "2025-08-09T07:55:46.346Z" }, + { url = "https://files.pythonhosted.org/packages/a4/fa/384d2c0f57edad03d7bec3ebefb462090d8905b4ff5a2d2525f3bb711fac/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:02425242e96bcf29a49711b0ca9f37e451da7c70562bc10e8ed992a5a7a25cc0", size = 159210, upload-time = "2025-08-09T07:55:47.539Z" }, + { url = "https://files.pythonhosted.org/packages/33/9e/eca49d35867ca2db336b6ca27617deed4653b97ebf45dfc21311ce473c37/charset_normalizer-3.4.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:78deba4d8f9590fe4dae384aeff04082510a709957e968753ff3c48399f6f92a", size = 153739, upload-time = "2025-08-09T07:55:48.744Z" }, + { url = "https://files.pythonhosted.org/packages/2a/91/26c3036e62dfe8de8061182d33be5025e2424002125c9500faff74a6735e/charset_normalizer-3.4.3-cp310-cp310-win32.whl", hash = "sha256:d79c198e27580c8e958906f803e63cddb77653731be08851c7df0b1a14a8fc0f", size = 99825, upload-time = "2025-08-09T07:55:50.305Z" }, + { url = "https://files.pythonhosted.org/packages/e2/c6/f05db471f81af1fa01839d44ae2a8bfeec8d2a8b4590f16c4e7393afd323/charset_normalizer-3.4.3-cp310-cp310-win_amd64.whl", hash = "sha256:c6e490913a46fa054e03699c70019ab869e990270597018cef1d8562132c2669", size = 107452, upload-time = "2025-08-09T07:55:51.461Z" }, + { url = "https://files.pythonhosted.org/packages/7f/b5/991245018615474a60965a7c9cd2b4efbaabd16d582a5547c47ee1c7730b/charset_normalizer-3.4.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b256ee2e749283ef3ddcff51a675ff43798d92d746d1a6e4631bf8c707d22d0b", size = 204483, upload-time = "2025-08-09T07:55:53.12Z" }, + { url = "https://files.pythonhosted.org/packages/c7/2a/ae245c41c06299ec18262825c1569c5d3298fc920e4ddf56ab011b417efd/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:13faeacfe61784e2559e690fc53fa4c5ae97c6fcedb8eb6fb8d0a15b475d2c64", size = 145520, upload-time = "2025-08-09T07:55:54.712Z" }, + { url = "https://files.pythonhosted.org/packages/3a/a4/b3b6c76e7a635748c4421d2b92c7b8f90a432f98bda5082049af37ffc8e3/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:00237675befef519d9af72169d8604a067d92755e84fe76492fef5441db05b91", size = 158876, upload-time = "2025-08-09T07:55:56.024Z" }, + { url = "https://files.pythonhosted.org/packages/e2/e6/63bb0e10f90a8243c5def74b5b105b3bbbfb3e7bb753915fe333fb0c11ea/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:585f3b2a80fbd26b048a0be90c5aae8f06605d3c92615911c3a2b03a8a3b796f", size = 156083, upload-time = "2025-08-09T07:55:57.582Z" }, + { url = "https://files.pythonhosted.org/packages/87/df/b7737ff046c974b183ea9aa111b74185ac8c3a326c6262d413bd5a1b8c69/charset_normalizer-3.4.3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0e78314bdc32fa80696f72fa16dc61168fda4d6a0c014e0380f9d02f0e5d8a07", size = 150295, upload-time = "2025-08-09T07:55:59.147Z" }, + { url = "https://files.pythonhosted.org/packages/61/f1/190d9977e0084d3f1dc169acd060d479bbbc71b90bf3e7bf7b9927dec3eb/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:96b2b3d1a83ad55310de8c7b4a2d04d9277d5591f40761274856635acc5fcb30", size = 148379, upload-time = "2025-08-09T07:56:00.364Z" }, + { url = "https://files.pythonhosted.org/packages/4c/92/27dbe365d34c68cfe0ca76f1edd70e8705d82b378cb54ebbaeabc2e3029d/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:939578d9d8fd4299220161fdd76e86c6a251987476f5243e8864a7844476ba14", size = 160018, upload-time = "2025-08-09T07:56:01.678Z" }, + { url = "https://files.pythonhosted.org/packages/99/04/baae2a1ea1893a01635d475b9261c889a18fd48393634b6270827869fa34/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:fd10de089bcdcd1be95a2f73dbe6254798ec1bda9f450d5828c96f93e2536b9c", size = 157430, upload-time = "2025-08-09T07:56:02.87Z" }, + { url = "https://files.pythonhosted.org/packages/2f/36/77da9c6a328c54d17b960c89eccacfab8271fdaaa228305330915b88afa9/charset_normalizer-3.4.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1e8ac75d72fa3775e0b7cb7e4629cec13b7514d928d15ef8ea06bca03ef01cae", size = 151600, upload-time = "2025-08-09T07:56:04.089Z" }, + { url = "https://files.pythonhosted.org/packages/64/d4/9eb4ff2c167edbbf08cdd28e19078bf195762e9bd63371689cab5ecd3d0d/charset_normalizer-3.4.3-cp311-cp311-win32.whl", hash = "sha256:6cf8fd4c04756b6b60146d98cd8a77d0cdae0e1ca20329da2ac85eed779b6849", size = 99616, upload-time = "2025-08-09T07:56:05.658Z" }, + { url = "https://files.pythonhosted.org/packages/f4/9c/996a4a028222e7761a96634d1820de8a744ff4327a00ada9c8942033089b/charset_normalizer-3.4.3-cp311-cp311-win_amd64.whl", hash = "sha256:31a9a6f775f9bcd865d88ee350f0ffb0e25936a7f930ca98995c05abf1faf21c", size = 107108, upload-time = "2025-08-09T07:56:07.176Z" }, + { url = "https://files.pythonhosted.org/packages/e9/5e/14c94999e418d9b87682734589404a25854d5f5d0408df68bc15b6ff54bb/charset_normalizer-3.4.3-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:e28e334d3ff134e88989d90ba04b47d84382a828c061d0d1027b1b12a62b39b1", size = 205655, upload-time = "2025-08-09T07:56:08.475Z" }, + { url = "https://files.pythonhosted.org/packages/7d/a8/c6ec5d389672521f644505a257f50544c074cf5fc292d5390331cd6fc9c3/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:0cacf8f7297b0c4fcb74227692ca46b4a5852f8f4f24b3c766dd94a1075c4884", size = 146223, upload-time = "2025-08-09T07:56:09.708Z" }, + { url = "https://files.pythonhosted.org/packages/fc/eb/a2ffb08547f4e1e5415fb69eb7db25932c52a52bed371429648db4d84fb1/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:c6fd51128a41297f5409deab284fecbe5305ebd7e5a1f959bee1c054622b7018", size = 159366, upload-time = "2025-08-09T07:56:11.326Z" }, + { url = "https://files.pythonhosted.org/packages/82/10/0fd19f20c624b278dddaf83b8464dcddc2456cb4b02bb902a6da126b87a1/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:3cfb2aad70f2c6debfbcb717f23b7eb55febc0bb23dcffc0f076009da10c6392", size = 157104, upload-time = "2025-08-09T07:56:13.014Z" }, + { url = "https://files.pythonhosted.org/packages/16/ab/0233c3231af734f5dfcf0844aa9582d5a1466c985bbed6cedab85af9bfe3/charset_normalizer-3.4.3-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1606f4a55c0fd363d754049cdf400175ee96c992b1f8018b993941f221221c5f", size = 151830, upload-time = "2025-08-09T07:56:14.428Z" }, + { url = "https://files.pythonhosted.org/packages/ae/02/e29e22b4e02839a0e4a06557b1999d0a47db3567e82989b5bb21f3fbbd9f/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:027b776c26d38b7f15b26a5da1044f376455fb3766df8fc38563b4efbc515154", size = 148854, upload-time = "2025-08-09T07:56:16.051Z" }, + { url = "https://files.pythonhosted.org/packages/05/6b/e2539a0a4be302b481e8cafb5af8792da8093b486885a1ae4d15d452bcec/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:42e5088973e56e31e4fa58eb6bd709e42fc03799c11c42929592889a2e54c491", size = 160670, upload-time = "2025-08-09T07:56:17.314Z" }, + { url = "https://files.pythonhosted.org/packages/31/e7/883ee5676a2ef217a40ce0bffcc3d0dfbf9e64cbcfbdf822c52981c3304b/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:cc34f233c9e71701040d772aa7490318673aa7164a0efe3172b2981218c26d93", size = 158501, upload-time = "2025-08-09T07:56:18.641Z" }, + { url = "https://files.pythonhosted.org/packages/c1/35/6525b21aa0db614cf8b5792d232021dca3df7f90a1944db934efa5d20bb1/charset_normalizer-3.4.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:320e8e66157cc4e247d9ddca8e21f427efc7a04bbd0ac8a9faf56583fa543f9f", size = 153173, upload-time = "2025-08-09T07:56:20.289Z" }, + { url = "https://files.pythonhosted.org/packages/50/ee/f4704bad8201de513fdc8aac1cabc87e38c5818c93857140e06e772b5892/charset_normalizer-3.4.3-cp312-cp312-win32.whl", hash = "sha256:fb6fecfd65564f208cbf0fba07f107fb661bcd1a7c389edbced3f7a493f70e37", size = 99822, upload-time = "2025-08-09T07:56:21.551Z" }, + { url = "https://files.pythonhosted.org/packages/39/f5/3b3836ca6064d0992c58c7561c6b6eee1b3892e9665d650c803bd5614522/charset_normalizer-3.4.3-cp312-cp312-win_amd64.whl", hash = "sha256:86df271bf921c2ee3818f0522e9a5b8092ca2ad8b065ece5d7d9d0e9f4849bcc", size = 107543, upload-time = "2025-08-09T07:56:23.115Z" }, + { url = "https://files.pythonhosted.org/packages/65/ca/2135ac97709b400c7654b4b764daf5c5567c2da45a30cdd20f9eefe2d658/charset_normalizer-3.4.3-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:14c2a87c65b351109f6abfc424cab3927b3bdece6f706e4d12faaf3d52ee5efe", size = 205326, upload-time = "2025-08-09T07:56:24.721Z" }, + { url = "https://files.pythonhosted.org/packages/71/11/98a04c3c97dd34e49c7d247083af03645ca3730809a5509443f3c37f7c99/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:41d1fc408ff5fdfb910200ec0e74abc40387bccb3252f3f27c0676731df2b2c8", size = 146008, upload-time = "2025-08-09T07:56:26.004Z" }, + { url = "https://files.pythonhosted.org/packages/60/f5/4659a4cb3c4ec146bec80c32d8bb16033752574c20b1252ee842a95d1a1e/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:1bb60174149316da1c35fa5233681f7c0f9f514509b8e399ab70fea5f17e45c9", size = 159196, upload-time = "2025-08-09T07:56:27.25Z" }, + { url = "https://files.pythonhosted.org/packages/86/9e/f552f7a00611f168b9a5865a1414179b2c6de8235a4fa40189f6f79a1753/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:30d006f98569de3459c2fc1f2acde170b7b2bd265dc1943e87e1a4efe1b67c31", size = 156819, upload-time = "2025-08-09T07:56:28.515Z" }, + { url = "https://files.pythonhosted.org/packages/7e/95/42aa2156235cbc8fa61208aded06ef46111c4d3f0de233107b3f38631803/charset_normalizer-3.4.3-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:416175faf02e4b0810f1f38bcb54682878a4af94059a1cd63b8747244420801f", size = 151350, upload-time = "2025-08-09T07:56:29.716Z" }, + { url = "https://files.pythonhosted.org/packages/c2/a9/3865b02c56f300a6f94fc631ef54f0a8a29da74fb45a773dfd3dcd380af7/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6aab0f181c486f973bc7262a97f5aca3ee7e1437011ef0c2ec04b5a11d16c927", size = 148644, upload-time = "2025-08-09T07:56:30.984Z" }, + { url = "https://files.pythonhosted.org/packages/77/d9/cbcf1a2a5c7d7856f11e7ac2d782aec12bdfea60d104e60e0aa1c97849dc/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:fdabf8315679312cfa71302f9bd509ded4f2f263fb5b765cf1433b39106c3cc9", size = 160468, upload-time = "2025-08-09T07:56:32.252Z" }, + { url = "https://files.pythonhosted.org/packages/f6/42/6f45efee8697b89fda4d50580f292b8f7f9306cb2971d4b53f8914e4d890/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:bd28b817ea8c70215401f657edef3a8aa83c29d447fb0b622c35403780ba11d5", size = 158187, upload-time = "2025-08-09T07:56:33.481Z" }, + { url = "https://files.pythonhosted.org/packages/70/99/f1c3bdcfaa9c45b3ce96f70b14f070411366fa19549c1d4832c935d8e2c3/charset_normalizer-3.4.3-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:18343b2d246dc6761a249ba1fb13f9ee9a2bcd95decc767319506056ea4ad4dc", size = 152699, upload-time = "2025-08-09T07:56:34.739Z" }, + { url = "https://files.pythonhosted.org/packages/a3/ad/b0081f2f99a4b194bcbb1934ef3b12aa4d9702ced80a37026b7607c72e58/charset_normalizer-3.4.3-cp313-cp313-win32.whl", hash = "sha256:6fb70de56f1859a3f71261cbe41005f56a7842cc348d3aeb26237560bfa5e0ce", size = 99580, upload-time = "2025-08-09T07:56:35.981Z" }, + { url = "https://files.pythonhosted.org/packages/9a/8f/ae790790c7b64f925e5c953b924aaa42a243fb778fed9e41f147b2a5715a/charset_normalizer-3.4.3-cp313-cp313-win_amd64.whl", hash = "sha256:cf1ebb7d78e1ad8ec2a8c4732c7be2e736f6e5123a4146c5b89c9d1f585f8cef", size = 107366, upload-time = "2025-08-09T07:56:37.339Z" }, + { url = "https://files.pythonhosted.org/packages/8e/91/b5a06ad970ddc7a0e513112d40113e834638f4ca1120eb727a249fb2715e/charset_normalizer-3.4.3-cp314-cp314-macosx_10_13_universal2.whl", hash = "sha256:3cd35b7e8aedeb9e34c41385fda4f73ba609e561faedfae0a9e75e44ac558a15", size = 204342, upload-time = "2025-08-09T07:56:38.687Z" }, + { url = "https://files.pythonhosted.org/packages/ce/ec/1edc30a377f0a02689342f214455c3f6c2fbedd896a1d2f856c002fc3062/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b89bc04de1d83006373429975f8ef9e7932534b8cc9ca582e4db7d20d91816db", size = 145995, upload-time = "2025-08-09T07:56:40.048Z" }, + { url = "https://files.pythonhosted.org/packages/17/e5/5e67ab85e6d22b04641acb5399c8684f4d37caf7558a53859f0283a650e9/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:2001a39612b241dae17b4687898843f254f8748b796a2e16f1051a17078d991d", size = 158640, upload-time = "2025-08-09T07:56:41.311Z" }, + { url = "https://files.pythonhosted.org/packages/f1/e5/38421987f6c697ee3722981289d554957c4be652f963d71c5e46a262e135/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:8dcfc373f888e4fb39a7bc57e93e3b845e7f462dacc008d9749568b1c4ece096", size = 156636, upload-time = "2025-08-09T07:56:43.195Z" }, + { url = "https://files.pythonhosted.org/packages/a0/e4/5a075de8daa3ec0745a9a3b54467e0c2967daaaf2cec04c845f73493e9a1/charset_normalizer-3.4.3-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:18b97b8404387b96cdbd30ad660f6407799126d26a39ca65729162fd810a99aa", size = 150939, upload-time = "2025-08-09T07:56:44.819Z" }, + { url = "https://files.pythonhosted.org/packages/02/f7/3611b32318b30974131db62b4043f335861d4d9b49adc6d57c1149cc49d4/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:ccf600859c183d70eb47e05a44cd80a4ce77394d1ac0f79dbd2dd90a69a3a049", size = 148580, upload-time = "2025-08-09T07:56:46.684Z" }, + { url = "https://files.pythonhosted.org/packages/7e/61/19b36f4bd67f2793ab6a99b979b4e4f3d8fc754cbdffb805335df4337126/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_ppc64le.whl", hash = "sha256:53cd68b185d98dde4ad8990e56a58dea83a4162161b1ea9272e5c9182ce415e0", size = 159870, upload-time = "2025-08-09T07:56:47.941Z" }, + { url = "https://files.pythonhosted.org/packages/06/57/84722eefdd338c04cf3030ada66889298eaedf3e7a30a624201e0cbe424a/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_s390x.whl", hash = "sha256:30a96e1e1f865f78b030d65241c1ee850cdf422d869e9028e2fc1d5e4db73b92", size = 157797, upload-time = "2025-08-09T07:56:49.756Z" }, + { url = "https://files.pythonhosted.org/packages/72/2a/aff5dd112b2f14bcc3462c312dce5445806bfc8ab3a7328555da95330e4b/charset_normalizer-3.4.3-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:d716a916938e03231e86e43782ca7878fb602a125a91e7acb8b5112e2e96ac16", size = 152224, upload-time = "2025-08-09T07:56:51.369Z" }, + { url = "https://files.pythonhosted.org/packages/b7/8c/9839225320046ed279c6e839d51f028342eb77c91c89b8ef2549f951f3ec/charset_normalizer-3.4.3-cp314-cp314-win32.whl", hash = "sha256:c6dbd0ccdda3a2ba7c2ecd9d77b37f3b5831687d8dc1b6ca5f56a4880cc7b7ce", size = 100086, upload-time = "2025-08-09T07:56:52.722Z" }, + { url = "https://files.pythonhosted.org/packages/ee/7a/36fbcf646e41f710ce0a563c1c9a343c6edf9be80786edeb15b6f62e17db/charset_normalizer-3.4.3-cp314-cp314-win_amd64.whl", hash = "sha256:73dc19b562516fc9bcf6e5d6e596df0b4eb98d87e4f79f3ae71840e6ed21361c", size = 107400, upload-time = "2025-08-09T07:56:55.172Z" }, + { url = "https://files.pythonhosted.org/packages/8a/1f/f041989e93b001bc4e44bb1669ccdcf54d3f00e628229a85b08d330615c5/charset_normalizer-3.4.3-py3-none-any.whl", hash = "sha256:ce571ab16d890d23b5c278547ba694193a45011ff86a9162a71307ed9f86759a", size = 53175, upload-time = "2025-08-09T07:57:26.864Z" }, ] [[package]] @@ -318,18 +318,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "colorama", marker = "sys_platform == 'win32'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b9/2e/0090cbf739cee7d23781ad4b89a9894a41538e4fcf4c31dcdd705b78eb8b/click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a", size = 226593 } +sdist = { url = "https://files.pythonhosted.org/packages/b9/2e/0090cbf739cee7d23781ad4b89a9894a41538e4fcf4c31dcdd705b78eb8b/click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a", size = 226593, upload-time = "2024-12-21T18:38:44.339Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/7e/d4/7ebdbd03970677812aac39c869717059dbb71a4cfc033ca6e5221787892c/click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2", size = 98188 }, + { url = "https://files.pythonhosted.org/packages/7e/d4/7ebdbd03970677812aac39c869717059dbb71a4cfc033ca6e5221787892c/click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2", size = 98188, upload-time = "2024-12-21T18:38:41.666Z" }, ] [[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 } +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 }, + { 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]] @@ -339,62 +339,62 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "cffi", marker = "platform_python_implementation != 'PyPy'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/d6/0d/d13399c94234ee8f3df384819dc67e0c5ce215fb751d567a55a1f4b028c7/cryptography-45.0.6.tar.gz", hash = "sha256:5c966c732cf6e4a276ce83b6e4c729edda2df6929083a952cc7da973c539c719", size = 744949 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/8c/29/2793d178d0eda1ca4a09a7c4e09a5185e75738cc6d526433e8663b460ea6/cryptography-45.0.6-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:048e7ad9e08cf4c0ab07ff7f36cc3115924e22e2266e034450a890d9e312dd74", size = 7042702 }, - { url = "https://files.pythonhosted.org/packages/b3/b6/cabd07410f222f32c8d55486c464f432808abaa1f12af9afcbe8f2f19030/cryptography-45.0.6-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:44647c5d796f5fc042bbc6d61307d04bf29bccb74d188f18051b635f20a9c75f", size = 4206483 }, - { url = "https://files.pythonhosted.org/packages/8b/9e/f9c7d36a38b1cfeb1cc74849aabe9bf817990f7603ff6eb485e0d70e0b27/cryptography-45.0.6-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:e40b80ecf35ec265c452eea0ba94c9587ca763e739b8e559c128d23bff7ebbbf", size = 4429679 }, - { url = "https://files.pythonhosted.org/packages/9c/2a/4434c17eb32ef30b254b9e8b9830cee4e516f08b47fdd291c5b1255b8101/cryptography-45.0.6-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:00e8724bdad672d75e6f069b27970883179bd472cd24a63f6e620ca7e41cc0c5", size = 4210553 }, - { url = "https://files.pythonhosted.org/packages/ef/1d/09a5df8e0c4b7970f5d1f3aff1b640df6d4be28a64cae970d56c6cf1c772/cryptography-45.0.6-cp311-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:7a3085d1b319d35296176af31c90338eeb2ddac8104661df79f80e1d9787b8b2", size = 3894499 }, - { url = "https://files.pythonhosted.org/packages/79/62/120842ab20d9150a9d3a6bdc07fe2870384e82f5266d41c53b08a3a96b34/cryptography-45.0.6-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:1b7fa6a1c1188c7ee32e47590d16a5a0646270921f8020efc9a511648e1b2e08", size = 4458484 }, - { url = "https://files.pythonhosted.org/packages/fd/80/1bc3634d45ddfed0871bfba52cf8f1ad724761662a0c792b97a951fb1b30/cryptography-45.0.6-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:275ba5cc0d9e320cd70f8e7b96d9e59903c815ca579ab96c1e37278d231fc402", size = 4210281 }, - { url = "https://files.pythonhosted.org/packages/7d/fe/ffb12c2d83d0ee625f124880a1f023b5878f79da92e64c37962bbbe35f3f/cryptography-45.0.6-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:f4028f29a9f38a2025abedb2e409973709c660d44319c61762202206ed577c42", size = 4456890 }, - { url = "https://files.pythonhosted.org/packages/8c/8e/b3f3fe0dc82c77a0deb5f493b23311e09193f2268b77196ec0f7a36e3f3e/cryptography-45.0.6-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:ee411a1b977f40bd075392c80c10b58025ee5c6b47a822a33c1198598a7a5f05", size = 4333247 }, - { url = "https://files.pythonhosted.org/packages/b3/a6/c3ef2ab9e334da27a1d7b56af4a2417d77e7806b2e0f90d6267ce120d2e4/cryptography-45.0.6-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:e2a21a8eda2d86bb604934b6b37691585bd095c1f788530c1fcefc53a82b3453", size = 4565045 }, - { url = "https://files.pythonhosted.org/packages/31/c3/77722446b13fa71dddd820a5faab4ce6db49e7e0bf8312ef4192a3f78e2f/cryptography-45.0.6-cp311-abi3-win32.whl", hash = "sha256:d063341378d7ee9c91f9d23b431a3502fc8bfacd54ef0a27baa72a0843b29159", size = 2928923 }, - { url = "https://files.pythonhosted.org/packages/38/63/a025c3225188a811b82932a4dcc8457a26c3729d81578ccecbcce2cb784e/cryptography-45.0.6-cp311-abi3-win_amd64.whl", hash = "sha256:833dc32dfc1e39b7376a87b9a6a4288a10aae234631268486558920029b086ec", size = 3403805 }, - { url = "https://files.pythonhosted.org/packages/5b/af/bcfbea93a30809f126d51c074ee0fac5bd9d57d068edf56c2a73abedbea4/cryptography-45.0.6-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:3436128a60a5e5490603ab2adbabc8763613f638513ffa7d311c900a8349a2a0", size = 7020111 }, - { url = "https://files.pythonhosted.org/packages/98/c6/ea5173689e014f1a8470899cd5beeb358e22bb3cf5a876060f9d1ca78af4/cryptography-45.0.6-cp37-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:0d9ef57b6768d9fa58e92f4947cea96ade1233c0e236db22ba44748ffedca394", size = 4198169 }, - { url = "https://files.pythonhosted.org/packages/ba/73/b12995edc0c7e2311ffb57ebd3b351f6b268fed37d93bfc6f9856e01c473/cryptography-45.0.6-cp37-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ea3c42f2016a5bbf71825537c2ad753f2870191134933196bee408aac397b3d9", size = 4421273 }, - { url = "https://files.pythonhosted.org/packages/f7/6e/286894f6f71926bc0da67408c853dd9ba953f662dcb70993a59fd499f111/cryptography-45.0.6-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:20ae4906a13716139d6d762ceb3e0e7e110f7955f3bc3876e3a07f5daadec5f3", size = 4199211 }, - { url = "https://files.pythonhosted.org/packages/de/34/a7f55e39b9623c5cb571d77a6a90387fe557908ffc44f6872f26ca8ae270/cryptography-45.0.6-cp37-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:2dac5ec199038b8e131365e2324c03d20e97fe214af051d20c49db129844e8b3", size = 3883732 }, - { url = "https://files.pythonhosted.org/packages/f9/b9/c6d32edbcba0cd9f5df90f29ed46a65c4631c4fbe11187feb9169c6ff506/cryptography-45.0.6-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:18f878a34b90d688982e43f4b700408b478102dd58b3e39de21b5ebf6509c301", size = 4450655 }, - { url = "https://files.pythonhosted.org/packages/77/2d/09b097adfdee0227cfd4c699b3375a842080f065bab9014248933497c3f9/cryptography-45.0.6-cp37-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:5bd6020c80c5b2b2242d6c48487d7b85700f5e0038e67b29d706f98440d66eb5", size = 4198956 }, - { url = "https://files.pythonhosted.org/packages/55/66/061ec6689207d54effdff535bbdf85cc380d32dd5377173085812565cf38/cryptography-45.0.6-cp37-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:eccddbd986e43014263eda489abbddfbc287af5cddfd690477993dbb31e31016", size = 4449859 }, - { url = "https://files.pythonhosted.org/packages/41/ff/e7d5a2ad2d035e5a2af116e1a3adb4d8fcd0be92a18032917a089c6e5028/cryptography-45.0.6-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:550ae02148206beb722cfe4ef0933f9352bab26b087af00e48fdfb9ade35c5b3", size = 4320254 }, - { url = "https://files.pythonhosted.org/packages/82/27/092d311af22095d288f4db89fcaebadfb2f28944f3d790a4cf51fe5ddaeb/cryptography-45.0.6-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:5b64e668fc3528e77efa51ca70fadcd6610e8ab231e3e06ae2bab3b31c2b8ed9", size = 4554815 }, - { url = "https://files.pythonhosted.org/packages/7e/01/aa2f4940262d588a8fdf4edabe4cda45854d00ebc6eaac12568b3a491a16/cryptography-45.0.6-cp37-abi3-win32.whl", hash = "sha256:780c40fb751c7d2b0c6786ceee6b6f871e86e8718a8ff4bc35073ac353c7cd02", size = 2912147 }, - { url = "https://files.pythonhosted.org/packages/0a/bc/16e0276078c2de3ceef6b5a34b965f4436215efac45313df90d55f0ba2d2/cryptography-45.0.6-cp37-abi3-win_amd64.whl", hash = "sha256:20d15aed3ee522faac1a39fbfdfee25d17b1284bafd808e1640a74846d7c4d1b", size = 3390459 }, - { url = "https://files.pythonhosted.org/packages/56/d2/4482d97c948c029be08cb29854a91bd2ae8da7eb9c4152461f1244dcea70/cryptography-45.0.6-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:705bb7c7ecc3d79a50f236adda12ca331c8e7ecfbea51edd931ce5a7a7c4f012", size = 3576812 }, - { url = "https://files.pythonhosted.org/packages/ec/24/55fc238fcaa122855442604b8badb2d442367dfbd5a7ca4bb0bd346e263a/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:826b46dae41a1155a0c0e66fafba43d0ede1dc16570b95e40c4d83bfcf0a451d", size = 4141694 }, - { url = "https://files.pythonhosted.org/packages/f9/7e/3ea4fa6fbe51baf3903806a0241c666b04c73d2358a3ecce09ebee8b9622/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:cc4d66f5dc4dc37b89cfef1bd5044387f7a1f6f0abb490815628501909332d5d", size = 4375010 }, - { url = "https://files.pythonhosted.org/packages/50/42/ec5a892d82d2a2c29f80fc19ced4ba669bca29f032faf6989609cff1f8dc/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:f68f833a9d445cc49f01097d95c83a850795921b3f7cc6488731e69bde3288da", size = 4141377 }, - { url = "https://files.pythonhosted.org/packages/e7/d7/246c4c973a22b9c2931999da953a2c19cae7c66b9154c2d62ffed811225e/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:3b5bf5267e98661b9b888a9250d05b063220dfa917a8203744454573c7eb79db", size = 4374609 }, - { url = "https://files.pythonhosted.org/packages/78/6d/c49ccf243f0a1b0781c2a8de8123ee552f0c8a417c6367a24d2ecb7c11b3/cryptography-45.0.6-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:2384f2ab18d9be88a6e4f8972923405e2dbb8d3e16c6b43f15ca491d7831bd18", size = 3322156 }, - { url = "https://files.pythonhosted.org/packages/61/69/c252de4ec047ba2f567ecb53149410219577d408c2aea9c989acae7eafce/cryptography-45.0.6-pp311-pypy311_pp73-macosx_10_9_x86_64.whl", hash = "sha256:fc022c1fa5acff6def2fc6d7819bbbd31ccddfe67d075331a65d9cfb28a20983", size = 3584669 }, - { url = "https://files.pythonhosted.org/packages/e3/fe/deea71e9f310a31fe0a6bfee670955152128d309ea2d1c79e2a5ae0f0401/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:3de77e4df42ac8d4e4d6cdb342d989803ad37707cf8f3fbf7b088c9cbdd46427", size = 4153022 }, - { url = "https://files.pythonhosted.org/packages/60/45/a77452f5e49cb580feedba6606d66ae7b82c128947aa754533b3d1bd44b0/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:599c8d7df950aa68baa7e98f7b73f4f414c9f02d0e8104a30c0182a07732638b", size = 4386802 }, - { url = "https://files.pythonhosted.org/packages/a3/b9/a2f747d2acd5e3075fdf5c145c7c3568895daaa38b3b0c960ef830db6cdc/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:31a2b9a10530a1cb04ffd6aa1cd4d3be9ed49f7d77a4dafe198f3b382f41545c", size = 4152706 }, - { url = "https://files.pythonhosted.org/packages/81/ec/381b3e8d0685a3f3f304a382aa3dfce36af2d76467da0fd4bb21ddccc7b2/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:e5b3dda1b00fb41da3af4c5ef3f922a200e33ee5ba0f0bc9ecf0b0c173958385", size = 4386740 }, - { url = "https://files.pythonhosted.org/packages/0a/76/cf8d69da8d0b5ecb0db406f24a63a3f69ba5e791a11b782aeeefef27ccbb/cryptography-45.0.6-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:629127cfdcdc6806dfe234734d7cb8ac54edaf572148274fa377a7d3405b0043", size = 3331874 }, +sdist = { url = "https://files.pythonhosted.org/packages/d6/0d/d13399c94234ee8f3df384819dc67e0c5ce215fb751d567a55a1f4b028c7/cryptography-45.0.6.tar.gz", hash = "sha256:5c966c732cf6e4a276ce83b6e4c729edda2df6929083a952cc7da973c539c719", size = 744949, upload-time = "2025-08-05T23:59:27.93Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8c/29/2793d178d0eda1ca4a09a7c4e09a5185e75738cc6d526433e8663b460ea6/cryptography-45.0.6-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:048e7ad9e08cf4c0ab07ff7f36cc3115924e22e2266e034450a890d9e312dd74", size = 7042702, upload-time = "2025-08-05T23:58:23.464Z" }, + { url = "https://files.pythonhosted.org/packages/b3/b6/cabd07410f222f32c8d55486c464f432808abaa1f12af9afcbe8f2f19030/cryptography-45.0.6-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:44647c5d796f5fc042bbc6d61307d04bf29bccb74d188f18051b635f20a9c75f", size = 4206483, upload-time = "2025-08-05T23:58:27.132Z" }, + { url = "https://files.pythonhosted.org/packages/8b/9e/f9c7d36a38b1cfeb1cc74849aabe9bf817990f7603ff6eb485e0d70e0b27/cryptography-45.0.6-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:e40b80ecf35ec265c452eea0ba94c9587ca763e739b8e559c128d23bff7ebbbf", size = 4429679, upload-time = "2025-08-05T23:58:29.152Z" }, + { url = "https://files.pythonhosted.org/packages/9c/2a/4434c17eb32ef30b254b9e8b9830cee4e516f08b47fdd291c5b1255b8101/cryptography-45.0.6-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:00e8724bdad672d75e6f069b27970883179bd472cd24a63f6e620ca7e41cc0c5", size = 4210553, upload-time = "2025-08-05T23:58:30.596Z" }, + { url = "https://files.pythonhosted.org/packages/ef/1d/09a5df8e0c4b7970f5d1f3aff1b640df6d4be28a64cae970d56c6cf1c772/cryptography-45.0.6-cp311-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:7a3085d1b319d35296176af31c90338eeb2ddac8104661df79f80e1d9787b8b2", size = 3894499, upload-time = "2025-08-05T23:58:32.03Z" }, + { url = "https://files.pythonhosted.org/packages/79/62/120842ab20d9150a9d3a6bdc07fe2870384e82f5266d41c53b08a3a96b34/cryptography-45.0.6-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:1b7fa6a1c1188c7ee32e47590d16a5a0646270921f8020efc9a511648e1b2e08", size = 4458484, upload-time = "2025-08-05T23:58:33.526Z" }, + { url = "https://files.pythonhosted.org/packages/fd/80/1bc3634d45ddfed0871bfba52cf8f1ad724761662a0c792b97a951fb1b30/cryptography-45.0.6-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:275ba5cc0d9e320cd70f8e7b96d9e59903c815ca579ab96c1e37278d231fc402", size = 4210281, upload-time = "2025-08-05T23:58:35.445Z" }, + { url = "https://files.pythonhosted.org/packages/7d/fe/ffb12c2d83d0ee625f124880a1f023b5878f79da92e64c37962bbbe35f3f/cryptography-45.0.6-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:f4028f29a9f38a2025abedb2e409973709c660d44319c61762202206ed577c42", size = 4456890, upload-time = "2025-08-05T23:58:36.923Z" }, + { url = "https://files.pythonhosted.org/packages/8c/8e/b3f3fe0dc82c77a0deb5f493b23311e09193f2268b77196ec0f7a36e3f3e/cryptography-45.0.6-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:ee411a1b977f40bd075392c80c10b58025ee5c6b47a822a33c1198598a7a5f05", size = 4333247, upload-time = "2025-08-05T23:58:38.781Z" }, + { url = "https://files.pythonhosted.org/packages/b3/a6/c3ef2ab9e334da27a1d7b56af4a2417d77e7806b2e0f90d6267ce120d2e4/cryptography-45.0.6-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:e2a21a8eda2d86bb604934b6b37691585bd095c1f788530c1fcefc53a82b3453", size = 4565045, upload-time = "2025-08-05T23:58:40.415Z" }, + { url = "https://files.pythonhosted.org/packages/31/c3/77722446b13fa71dddd820a5faab4ce6db49e7e0bf8312ef4192a3f78e2f/cryptography-45.0.6-cp311-abi3-win32.whl", hash = "sha256:d063341378d7ee9c91f9d23b431a3502fc8bfacd54ef0a27baa72a0843b29159", size = 2928923, upload-time = "2025-08-05T23:58:41.919Z" }, + { url = "https://files.pythonhosted.org/packages/38/63/a025c3225188a811b82932a4dcc8457a26c3729d81578ccecbcce2cb784e/cryptography-45.0.6-cp311-abi3-win_amd64.whl", hash = "sha256:833dc32dfc1e39b7376a87b9a6a4288a10aae234631268486558920029b086ec", size = 3403805, upload-time = "2025-08-05T23:58:43.792Z" }, + { url = "https://files.pythonhosted.org/packages/5b/af/bcfbea93a30809f126d51c074ee0fac5bd9d57d068edf56c2a73abedbea4/cryptography-45.0.6-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:3436128a60a5e5490603ab2adbabc8763613f638513ffa7d311c900a8349a2a0", size = 7020111, upload-time = "2025-08-05T23:58:45.316Z" }, + { url = "https://files.pythonhosted.org/packages/98/c6/ea5173689e014f1a8470899cd5beeb358e22bb3cf5a876060f9d1ca78af4/cryptography-45.0.6-cp37-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:0d9ef57b6768d9fa58e92f4947cea96ade1233c0e236db22ba44748ffedca394", size = 4198169, upload-time = "2025-08-05T23:58:47.121Z" }, + { url = "https://files.pythonhosted.org/packages/ba/73/b12995edc0c7e2311ffb57ebd3b351f6b268fed37d93bfc6f9856e01c473/cryptography-45.0.6-cp37-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ea3c42f2016a5bbf71825537c2ad753f2870191134933196bee408aac397b3d9", size = 4421273, upload-time = "2025-08-05T23:58:48.557Z" }, + { url = "https://files.pythonhosted.org/packages/f7/6e/286894f6f71926bc0da67408c853dd9ba953f662dcb70993a59fd499f111/cryptography-45.0.6-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:20ae4906a13716139d6d762ceb3e0e7e110f7955f3bc3876e3a07f5daadec5f3", size = 4199211, upload-time = "2025-08-05T23:58:50.139Z" }, + { url = "https://files.pythonhosted.org/packages/de/34/a7f55e39b9623c5cb571d77a6a90387fe557908ffc44f6872f26ca8ae270/cryptography-45.0.6-cp37-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:2dac5ec199038b8e131365e2324c03d20e97fe214af051d20c49db129844e8b3", size = 3883732, upload-time = "2025-08-05T23:58:52.253Z" }, + { url = "https://files.pythonhosted.org/packages/f9/b9/c6d32edbcba0cd9f5df90f29ed46a65c4631c4fbe11187feb9169c6ff506/cryptography-45.0.6-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:18f878a34b90d688982e43f4b700408b478102dd58b3e39de21b5ebf6509c301", size = 4450655, upload-time = "2025-08-05T23:58:53.848Z" }, + { url = "https://files.pythonhosted.org/packages/77/2d/09b097adfdee0227cfd4c699b3375a842080f065bab9014248933497c3f9/cryptography-45.0.6-cp37-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:5bd6020c80c5b2b2242d6c48487d7b85700f5e0038e67b29d706f98440d66eb5", size = 4198956, upload-time = "2025-08-05T23:58:55.209Z" }, + { url = "https://files.pythonhosted.org/packages/55/66/061ec6689207d54effdff535bbdf85cc380d32dd5377173085812565cf38/cryptography-45.0.6-cp37-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:eccddbd986e43014263eda489abbddfbc287af5cddfd690477993dbb31e31016", size = 4449859, upload-time = "2025-08-05T23:58:56.639Z" }, + { url = "https://files.pythonhosted.org/packages/41/ff/e7d5a2ad2d035e5a2af116e1a3adb4d8fcd0be92a18032917a089c6e5028/cryptography-45.0.6-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:550ae02148206beb722cfe4ef0933f9352bab26b087af00e48fdfb9ade35c5b3", size = 4320254, upload-time = "2025-08-05T23:58:58.833Z" }, + { url = "https://files.pythonhosted.org/packages/82/27/092d311af22095d288f4db89fcaebadfb2f28944f3d790a4cf51fe5ddaeb/cryptography-45.0.6-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:5b64e668fc3528e77efa51ca70fadcd6610e8ab231e3e06ae2bab3b31c2b8ed9", size = 4554815, upload-time = "2025-08-05T23:59:00.283Z" }, + { url = "https://files.pythonhosted.org/packages/7e/01/aa2f4940262d588a8fdf4edabe4cda45854d00ebc6eaac12568b3a491a16/cryptography-45.0.6-cp37-abi3-win32.whl", hash = "sha256:780c40fb751c7d2b0c6786ceee6b6f871e86e8718a8ff4bc35073ac353c7cd02", size = 2912147, upload-time = "2025-08-05T23:59:01.716Z" }, + { url = "https://files.pythonhosted.org/packages/0a/bc/16e0276078c2de3ceef6b5a34b965f4436215efac45313df90d55f0ba2d2/cryptography-45.0.6-cp37-abi3-win_amd64.whl", hash = "sha256:20d15aed3ee522faac1a39fbfdfee25d17b1284bafd808e1640a74846d7c4d1b", size = 3390459, upload-time = "2025-08-05T23:59:03.358Z" }, + { url = "https://files.pythonhosted.org/packages/56/d2/4482d97c948c029be08cb29854a91bd2ae8da7eb9c4152461f1244dcea70/cryptography-45.0.6-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:705bb7c7ecc3d79a50f236adda12ca331c8e7ecfbea51edd931ce5a7a7c4f012", size = 3576812, upload-time = "2025-08-05T23:59:04.833Z" }, + { url = "https://files.pythonhosted.org/packages/ec/24/55fc238fcaa122855442604b8badb2d442367dfbd5a7ca4bb0bd346e263a/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:826b46dae41a1155a0c0e66fafba43d0ede1dc16570b95e40c4d83bfcf0a451d", size = 4141694, upload-time = "2025-08-05T23:59:06.66Z" }, + { url = "https://files.pythonhosted.org/packages/f9/7e/3ea4fa6fbe51baf3903806a0241c666b04c73d2358a3ecce09ebee8b9622/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:cc4d66f5dc4dc37b89cfef1bd5044387f7a1f6f0abb490815628501909332d5d", size = 4375010, upload-time = "2025-08-05T23:59:08.14Z" }, + { url = "https://files.pythonhosted.org/packages/50/42/ec5a892d82d2a2c29f80fc19ced4ba669bca29f032faf6989609cff1f8dc/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:f68f833a9d445cc49f01097d95c83a850795921b3f7cc6488731e69bde3288da", size = 4141377, upload-time = "2025-08-05T23:59:09.584Z" }, + { url = "https://files.pythonhosted.org/packages/e7/d7/246c4c973a22b9c2931999da953a2c19cae7c66b9154c2d62ffed811225e/cryptography-45.0.6-pp310-pypy310_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:3b5bf5267e98661b9b888a9250d05b063220dfa917a8203744454573c7eb79db", size = 4374609, upload-time = "2025-08-05T23:59:11.923Z" }, + { url = "https://files.pythonhosted.org/packages/78/6d/c49ccf243f0a1b0781c2a8de8123ee552f0c8a417c6367a24d2ecb7c11b3/cryptography-45.0.6-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:2384f2ab18d9be88a6e4f8972923405e2dbb8d3e16c6b43f15ca491d7831bd18", size = 3322156, upload-time = "2025-08-05T23:59:13.597Z" }, + { url = "https://files.pythonhosted.org/packages/61/69/c252de4ec047ba2f567ecb53149410219577d408c2aea9c989acae7eafce/cryptography-45.0.6-pp311-pypy311_pp73-macosx_10_9_x86_64.whl", hash = "sha256:fc022c1fa5acff6def2fc6d7819bbbd31ccddfe67d075331a65d9cfb28a20983", size = 3584669, upload-time = "2025-08-05T23:59:15.431Z" }, + { url = "https://files.pythonhosted.org/packages/e3/fe/deea71e9f310a31fe0a6bfee670955152128d309ea2d1c79e2a5ae0f0401/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:3de77e4df42ac8d4e4d6cdb342d989803ad37707cf8f3fbf7b088c9cbdd46427", size = 4153022, upload-time = "2025-08-05T23:59:16.954Z" }, + { url = "https://files.pythonhosted.org/packages/60/45/a77452f5e49cb580feedba6606d66ae7b82c128947aa754533b3d1bd44b0/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:599c8d7df950aa68baa7e98f7b73f4f414c9f02d0e8104a30c0182a07732638b", size = 4386802, upload-time = "2025-08-05T23:59:18.55Z" }, + { url = "https://files.pythonhosted.org/packages/a3/b9/a2f747d2acd5e3075fdf5c145c7c3568895daaa38b3b0c960ef830db6cdc/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:31a2b9a10530a1cb04ffd6aa1cd4d3be9ed49f7d77a4dafe198f3b382f41545c", size = 4152706, upload-time = "2025-08-05T23:59:20.044Z" }, + { url = "https://files.pythonhosted.org/packages/81/ec/381b3e8d0685a3f3f304a382aa3dfce36af2d76467da0fd4bb21ddccc7b2/cryptography-45.0.6-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:e5b3dda1b00fb41da3af4c5ef3f922a200e33ee5ba0f0bc9ecf0b0c173958385", size = 4386740, upload-time = "2025-08-05T23:59:21.525Z" }, + { url = "https://files.pythonhosted.org/packages/0a/76/cf8d69da8d0b5ecb0db406f24a63a3f69ba5e791a11b782aeeefef27ccbb/cryptography-45.0.6-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:629127cfdcdc6806dfe234734d7cb8ac54edaf572148274fa377a7d3405b0043", size = 3331874, upload-time = "2025-08-05T23:59:23.017Z" }, ] [[package]] name = "distlib" version = "0.4.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/96/8e/709914eb2b5749865801041647dc7f4e6d00b549cfe88b65ca192995f07c/distlib-0.4.0.tar.gz", hash = "sha256:feec40075be03a04501a973d81f633735b4b69f98b05450592310c0f401a4e0d", size = 614605 } +sdist = { url = "https://files.pythonhosted.org/packages/96/8e/709914eb2b5749865801041647dc7f4e6d00b549cfe88b65ca192995f07c/distlib-0.4.0.tar.gz", hash = "sha256:feec40075be03a04501a973d81f633735b4b69f98b05450592310c0f401a4e0d", size = 614605, upload-time = "2025-07-17T16:52:00.465Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/33/6b/e0547afaf41bf2c42e52430072fa5658766e3d65bd4b03a563d1b6336f57/distlib-0.4.0-py2.py3-none-any.whl", hash = "sha256:9659f7d87e46584a30b5780e43ac7a2143098441670ff0a49d5f9034c54a6c16", size = 469047 }, + { url = "https://files.pythonhosted.org/packages/33/6b/e0547afaf41bf2c42e52430072fa5658766e3d65bd4b03a563d1b6336f57/distlib-0.4.0-py2.py3-none-any.whl", hash = "sha256:9659f7d87e46584a30b5780e43ac7a2143098441670ff0a49d5f9034c54a6c16", size = 469047, upload-time = "2025-07-17T16:51:58.613Z" }, ] [[package]] name = "docutils" version = "0.22" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/e9/86/5b41c32ecedcfdb4c77b28b6cb14234f252075f8cdb254531727a35547dd/docutils-0.22.tar.gz", hash = "sha256:ba9d57750e92331ebe7c08a1bbf7a7f8143b86c476acd51528b042216a6aad0f", size = 2277984 } +sdist = { url = "https://files.pythonhosted.org/packages/e9/86/5b41c32ecedcfdb4c77b28b6cb14234f252075f8cdb254531727a35547dd/docutils-0.22.tar.gz", hash = "sha256:ba9d57750e92331ebe7c08a1bbf7a7f8143b86c476acd51528b042216a6aad0f", size = 2277984, upload-time = "2025-07-29T15:20:31.06Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/44/57/8db39bc5f98f042e0153b1de9fb88e1a409a33cda4dd7f723c2ed71e01f6/docutils-0.22-py3-none-any.whl", hash = "sha256:4ed966a0e96a0477d852f7af31bdcb3adc049fbb35ccba358c2ea8a03287615e", size = 630709 }, + { url = "https://files.pythonhosted.org/packages/44/57/8db39bc5f98f042e0153b1de9fb88e1a409a33cda4dd7f723c2ed71e01f6/docutils-0.22-py3-none-any.whl", hash = "sha256:4ed966a0e96a0477d852f7af31bdcb3adc049fbb35ccba358c2ea8a03287615e", size = 630709, upload-time = "2025-07-29T15:20:28.335Z" }, ] [[package]] @@ -404,27 +404,27 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/0b/9f/a65090624ecf468cdca03533906e7c69ed7588582240cfe7cc9e770b50eb/exceptiongroup-1.3.0.tar.gz", hash = "sha256:b241f5885f560bc56a59ee63ca4c6a8bfa46ae4ad651af316d4e81817bb9fd88", size = 29749 } +sdist = { url = "https://files.pythonhosted.org/packages/0b/9f/a65090624ecf468cdca03533906e7c69ed7588582240cfe7cc9e770b50eb/exceptiongroup-1.3.0.tar.gz", hash = "sha256:b241f5885f560bc56a59ee63ca4c6a8bfa46ae4ad651af316d4e81817bb9fd88", size = 29749, upload-time = "2025-05-10T17:42:51.123Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/36/f4/c6e662dade71f56cd2f3735141b265c3c79293c109549c1e6933b0651ffc/exceptiongroup-1.3.0-py3-none-any.whl", hash = "sha256:4d111e6e0c13d0644cad6ddaa7ed0261a0b36971f6d23e7ec9b4b9097da78a10", size = 16674 }, + { url = "https://files.pythonhosted.org/packages/36/f4/c6e662dade71f56cd2f3735141b265c3c79293c109549c1e6933b0651ffc/exceptiongroup-1.3.0-py3-none-any.whl", hash = "sha256:4d111e6e0c13d0644cad6ddaa7ed0261a0b36971f6d23e7ec9b4b9097da78a10", size = 16674, upload-time = "2025-05-10T17:42:49.33Z" }, ] [[package]] name = "execnet" version = "2.1.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/bb/ff/b4c0dc78fbe20c3e59c0c7334de0c27eb4001a2b2017999af398bf730817/execnet-2.1.1.tar.gz", hash = "sha256:5189b52c6121c24feae288166ab41b32549c7e2348652736540b9e6e7d4e72e3", size = 166524 } +sdist = { url = "https://files.pythonhosted.org/packages/bb/ff/b4c0dc78fbe20c3e59c0c7334de0c27eb4001a2b2017999af398bf730817/execnet-2.1.1.tar.gz", hash = "sha256:5189b52c6121c24feae288166ab41b32549c7e2348652736540b9e6e7d4e72e3", size = 166524, upload-time = "2024-04-08T09:04:19.245Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/43/09/2aea36ff60d16dd8879bdb2f5b3ee0ba8d08cbbdcdfe870e695ce3784385/execnet-2.1.1-py3-none-any.whl", hash = "sha256:26dee51f1b80cebd6d0ca8e74dd8745419761d3bef34163928cbebbdc4749fdc", size = 40612 }, + { url = "https://files.pythonhosted.org/packages/43/09/2aea36ff60d16dd8879bdb2f5b3ee0ba8d08cbbdcdfe870e695ce3784385/execnet-2.1.1-py3-none-any.whl", hash = "sha256:26dee51f1b80cebd6d0ca8e74dd8745419761d3bef34163928cbebbdc4749fdc", size = 40612, upload-time = "2024-04-08T09:04:17.414Z" }, ] [[package]] name = "filelock" version = "3.19.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/40/bb/0ab3e58d22305b6f5440629d20683af28959bf793d98d11950e305c1c326/filelock-3.19.1.tar.gz", hash = "sha256:66eda1888b0171c998b35be2bcc0f6d75c388a7ce20c3f3f37aa8e96c2dddf58", size = 17687 } +sdist = { url = "https://files.pythonhosted.org/packages/40/bb/0ab3e58d22305b6f5440629d20683af28959bf793d98d11950e305c1c326/filelock-3.19.1.tar.gz", hash = "sha256:66eda1888b0171c998b35be2bcc0f6d75c388a7ce20c3f3f37aa8e96c2dddf58", size = 17687, upload-time = "2025-08-14T16:56:03.016Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/42/14/42b2651a2f46b022ccd948bca9f2d5af0fd8929c4eec235b8d6d844fbe67/filelock-3.19.1-py3-none-any.whl", hash = "sha256:d38e30481def20772f5baf097c122c3babc4fcdb7e14e57049eb9d88c6dc017d", size = 15988 }, + { url = "https://files.pythonhosted.org/packages/42/14/42b2651a2f46b022ccd948bca9f2d5af0fd8929c4eec235b8d6d844fbe67/filelock-3.19.1-py3-none-any.whl", hash = "sha256:d38e30481def20772f5baf097c122c3babc4fcdb7e14e57049eb9d88c6dc017d", size = 15988, upload-time = "2025-08-14T16:56:01.633Z" }, ] [[package]] @@ -438,18 +438,18 @@ dependencies = [ { name = "requests" }, { name = "tomli-w" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/50/9c/0608c91a5b6c013c63548515ae31cff6399cd9ce891bd9daee8c103da09b/flit-3.12.0.tar.gz", hash = "sha256:1c80f34dd96992e7758b40423d2809f48f640ca285d0b7821825e50745ec3740", size = 155038 } +sdist = { url = "https://files.pythonhosted.org/packages/50/9c/0608c91a5b6c013c63548515ae31cff6399cd9ce891bd9daee8c103da09b/flit-3.12.0.tar.gz", hash = "sha256:1c80f34dd96992e7758b40423d2809f48f640ca285d0b7821825e50745ec3740", size = 155038, upload-time = "2025-03-25T08:03:22.505Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/f5/82/ce1d3bb380b227e26e517655d1de7b32a72aad61fa21ff9bd91a2e2db6ee/flit-3.12.0-py3-none-any.whl", hash = "sha256:2b4e7171dc22881fa6adc2dbf083e5ecc72520be3cd7587d2a803da94d6ef431", size = 50657 }, + { url = "https://files.pythonhosted.org/packages/f5/82/ce1d3bb380b227e26e517655d1de7b32a72aad61fa21ff9bd91a2e2db6ee/flit-3.12.0-py3-none-any.whl", hash = "sha256:2b4e7171dc22881fa6adc2dbf083e5ecc72520be3cd7587d2a803da94d6ef431", size = 50657, upload-time = "2025-03-25T08:03:19.031Z" }, ] [[package]] name = "flit-core" version = "3.12.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/69/59/b6fc2188dfc7ea4f936cd12b49d707f66a1cb7a1d2c16172963534db741b/flit_core-3.12.0.tar.gz", hash = "sha256:18f63100d6f94385c6ed57a72073443e1a71a4acb4339491615d0f16d6ff01b2", size = 53690 } +sdist = { url = "https://files.pythonhosted.org/packages/69/59/b6fc2188dfc7ea4f936cd12b49d707f66a1cb7a1d2c16172963534db741b/flit_core-3.12.0.tar.gz", hash = "sha256:18f63100d6f94385c6ed57a72073443e1a71a4acb4339491615d0f16d6ff01b2", size = 53690, upload-time = "2025-03-25T08:03:23.969Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/f2/65/b6ba90634c984a4fcc02c7e3afe523fef500c4980fec67cc27536ee50acf/flit_core-3.12.0-py3-none-any.whl", hash = "sha256:e7a0304069ea895172e3c7bb703292e992c5d1555dd1233ab7b5621b5b69e62c", size = 45594 }, + { url = "https://files.pythonhosted.org/packages/f2/65/b6ba90634c984a4fcc02c7e3afe523fef500c4980fec67cc27536ee50acf/flit_core-3.12.0-py3-none-any.whl", hash = "sha256:e7a0304069ea895172e3c7bb703292e992c5d1555dd1233ab7b5621b5b69e62c", size = 45594, upload-time = "2025-03-25T08:03:20.772Z" }, ] [[package]] @@ -459,9 +459,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "smmap" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/72/94/63b0fc47eb32792c7ba1fe1b694daec9a63620db1e313033d18140c2320a/gitdb-4.0.12.tar.gz", hash = "sha256:5ef71f855d191a3326fcfbc0d5da835f26b13fbcba60c32c21091c349ffdb571", size = 394684 } +sdist = { url = "https://files.pythonhosted.org/packages/72/94/63b0fc47eb32792c7ba1fe1b694daec9a63620db1e313033d18140c2320a/gitdb-4.0.12.tar.gz", hash = "sha256:5ef71f855d191a3326fcfbc0d5da835f26b13fbcba60c32c21091c349ffdb571", size = 394684, upload-time = "2025-01-02T07:20:46.413Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/a0/61/5c78b91c3143ed5c14207f463aecfc8f9dbb5092fb2869baf37c273b2705/gitdb-4.0.12-py3-none-any.whl", hash = "sha256:67073e15955400952c6565cc3e707c554a4eea2e428946f7a4c162fab9bd9bcf", size = 62794 }, + { url = "https://files.pythonhosted.org/packages/a0/61/5c78b91c3143ed5c14207f463aecfc8f9dbb5092fb2869baf37c273b2705/gitdb-4.0.12-py3-none-any.whl", hash = "sha256:67073e15955400952c6565cc3e707c554a4eea2e428946f7a4c162fab9bd9bcf", size = 62794, upload-time = "2025-01-02T07:20:43.624Z" }, ] [[package]] @@ -471,9 +471,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "gitdb" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/9a/c8/dd58967d119baab745caec2f9d853297cec1989ec1d63f677d3880632b88/gitpython-3.1.45.tar.gz", hash = "sha256:85b0ee964ceddf211c41b9f27a49086010a190fd8132a24e21f362a4b36a791c", size = 215076 } +sdist = { url = "https://files.pythonhosted.org/packages/9a/c8/dd58967d119baab745caec2f9d853297cec1989ec1d63f677d3880632b88/gitpython-3.1.45.tar.gz", hash = "sha256:85b0ee964ceddf211c41b9f27a49086010a190fd8132a24e21f362a4b36a791c", size = 215076, upload-time = "2025-07-24T03:45:54.871Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/01/61/d4b89fec821f72385526e1b9d9a3a0385dda4a72b206d28049e2c7cd39b8/gitpython-3.1.45-py3-none-any.whl", hash = "sha256:8908cb2e02fb3b93b7eb0f2827125cb699869470432cc885f019b8fd0fccff77", size = 208168 }, + { url = "https://files.pythonhosted.org/packages/01/61/d4b89fec821f72385526e1b9d9a3a0385dda4a72b206d28049e2c7cd39b8/gitpython-3.1.45-py3-none-any.whl", hash = "sha256:8908cb2e02fb3b93b7eb0f2827125cb699869470432cc885f019b8fd0fccff77", size = 208168, upload-time = "2025-07-24T03:45:52.517Z" }, ] [[package]] @@ -487,9 +487,9 @@ dependencies = [ { name = "protobuf" }, { name = "requests" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/dc/21/e9d043e88222317afdbdb567165fdbc3b0aad90064c7e0c9eb0ad9955ad8/google_api_core-2.25.1.tar.gz", hash = "sha256:d2aaa0b13c78c61cb3f4282c464c046e45fbd75755683c9c525e6e8f7ed0a5e8", size = 165443 } +sdist = { url = "https://files.pythonhosted.org/packages/dc/21/e9d043e88222317afdbdb567165fdbc3b0aad90064c7e0c9eb0ad9955ad8/google_api_core-2.25.1.tar.gz", hash = "sha256:d2aaa0b13c78c61cb3f4282c464c046e45fbd75755683c9c525e6e8f7ed0a5e8", size = 165443, upload-time = "2025-06-12T20:52:20.439Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/14/4b/ead00905132820b623732b175d66354e9d3e69fcf2a5dcdab780664e7896/google_api_core-2.25.1-py3-none-any.whl", hash = "sha256:8a2a56c1fef82987a524371f99f3bd0143702fecc670c72e600c1cda6bf8dbb7", size = 160807 }, + { url = "https://files.pythonhosted.org/packages/14/4b/ead00905132820b623732b175d66354e9d3e69fcf2a5dcdab780664e7896/google_api_core-2.25.1-py3-none-any.whl", hash = "sha256:8a2a56c1fef82987a524371f99f3bd0143702fecc670c72e600c1cda6bf8dbb7", size = 160807, upload-time = "2025-06-12T20:52:19.334Z" }, ] [[package]] @@ -503,9 +503,9 @@ dependencies = [ { name = "httplib2" }, { name = "uritemplate" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/73/ed/6e7865324252ea0a9f7c8171a3a00439a1e8447a5dc08e6d6c483777bb38/google_api_python_client-2.179.0.tar.gz", hash = "sha256:76a774a49dd58af52e74ce7114db387e58f0aaf6760c9cf9201ab6d731d8bd8d", size = 13397672 } +sdist = { url = "https://files.pythonhosted.org/packages/73/ed/6e7865324252ea0a9f7c8171a3a00439a1e8447a5dc08e6d6c483777bb38/google_api_python_client-2.179.0.tar.gz", hash = "sha256:76a774a49dd58af52e74ce7114db387e58f0aaf6760c9cf9201ab6d731d8bd8d", size = 13397672, upload-time = "2025-08-13T18:45:28.838Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/42/d4/2568d5d907582cc145f3ffede43879746fd4b331308088a0fc57f7ecdbca/google_api_python_client-2.179.0-py3-none-any.whl", hash = "sha256:79ab5039d70c59dab874fd18333fca90fb469be51c96113cb133e5fc1f0b2a79", size = 13955142 }, + { url = "https://files.pythonhosted.org/packages/42/d4/2568d5d907582cc145f3ffede43879746fd4b331308088a0fc57f7ecdbca/google_api_python_client-2.179.0-py3-none-any.whl", hash = "sha256:79ab5039d70c59dab874fd18333fca90fb469be51c96113cb133e5fc1f0b2a79", size = 13955142, upload-time = "2025-08-13T18:45:25.944Z" }, ] [[package]] @@ -517,9 +517,9 @@ dependencies = [ { name = "pyasn1-modules" }, { name = "rsa" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/9e/9b/e92ef23b84fa10a64ce4831390b7a4c2e53c0132568d99d4ae61d04c8855/google_auth-2.40.3.tar.gz", hash = "sha256:500c3a29adedeb36ea9cf24b8d10858e152f2412e3ca37829b3fa18e33d63b77", size = 281029 } +sdist = { url = "https://files.pythonhosted.org/packages/9e/9b/e92ef23b84fa10a64ce4831390b7a4c2e53c0132568d99d4ae61d04c8855/google_auth-2.40.3.tar.gz", hash = "sha256:500c3a29adedeb36ea9cf24b8d10858e152f2412e3ca37829b3fa18e33d63b77", size = 281029, upload-time = "2025-06-04T18:04:57.577Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/17/63/b19553b658a1692443c62bd07e5868adaa0ad746a0751ba62c59568cd45b/google_auth-2.40.3-py2.py3-none-any.whl", hash = "sha256:1370d4593e86213563547f97a92752fc658456fe4514c809544f330fed45a7ca", size = 216137 }, + { url = "https://files.pythonhosted.org/packages/17/63/b19553b658a1692443c62bd07e5868adaa0ad746a0751ba62c59568cd45b/google_auth-2.40.3-py2.py3-none-any.whl", hash = "sha256:1370d4593e86213563547f97a92752fc658456fe4514c809544f330fed45a7ca", size = 216137, upload-time = "2025-06-04T18:04:55.573Z" }, ] [[package]] @@ -530,9 +530,9 @@ dependencies = [ { name = "google-auth" }, { name = "httplib2" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/56/be/217a598a818567b28e859ff087f347475c807a5649296fb5a817c58dacef/google-auth-httplib2-0.2.0.tar.gz", hash = "sha256:38aa7badf48f974f1eb9861794e9c0cb2a0511a4ec0679b1f886d108f5640e05", size = 10842 } +sdist = { url = "https://files.pythonhosted.org/packages/56/be/217a598a818567b28e859ff087f347475c807a5649296fb5a817c58dacef/google-auth-httplib2-0.2.0.tar.gz", hash = "sha256:38aa7badf48f974f1eb9861794e9c0cb2a0511a4ec0679b1f886d108f5640e05", size = 10842, upload-time = "2023-12-12T17:40:30.722Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/be/8a/fe34d2f3f9470a27b01c9e76226965863f153d5fbe276f83608562e49c04/google_auth_httplib2-0.2.0-py2.py3-none-any.whl", hash = "sha256:b65a0a2123300dd71281a7bf6e64d65a0759287df52729bdd1ae2e47dc311a3d", size = 9253 }, + { url = "https://files.pythonhosted.org/packages/be/8a/fe34d2f3f9470a27b01c9e76226965863f153d5fbe276f83608562e49c04/google_auth_httplib2-0.2.0-py2.py3-none-any.whl", hash = "sha256:b65a0a2123300dd71281a7bf6e64d65a0759287df52729bdd1ae2e47dc311a3d", size = 9253, upload-time = "2023-12-12T17:40:13.055Z" }, ] [[package]] @@ -543,9 +543,9 @@ dependencies = [ { name = "google-auth" }, { name = "requests-oauthlib" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/fb/87/e10bf24f7bcffc1421b84d6f9c3377c30ec305d082cd737ddaa6d8f77f7c/google_auth_oauthlib-1.2.2.tar.gz", hash = "sha256:11046fb8d3348b296302dd939ace8af0a724042e8029c1b872d87fabc9f41684", size = 20955 } +sdist = { url = "https://files.pythonhosted.org/packages/fb/87/e10bf24f7bcffc1421b84d6f9c3377c30ec305d082cd737ddaa6d8f77f7c/google_auth_oauthlib-1.2.2.tar.gz", hash = "sha256:11046fb8d3348b296302dd939ace8af0a724042e8029c1b872d87fabc9f41684", size = 20955, upload-time = "2025-04-22T16:40:29.172Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ac/84/40ee070be95771acd2f4418981edb834979424565c3eec3cd88b6aa09d24/google_auth_oauthlib-1.2.2-py3-none-any.whl", hash = "sha256:fd619506f4b3908b5df17b65f39ca8d66ea56986e5472eb5978fd8f3786f00a2", size = 19072 }, + { url = "https://files.pythonhosted.org/packages/ac/84/40ee070be95771acd2f4418981edb834979424565c3eec3cd88b6aa09d24/google_auth_oauthlib-1.2.2-py3-none-any.whl", hash = "sha256:fd619506f4b3908b5df17b65f39ca8d66ea56986e5472eb5978fd8f3786f00a2", size = 19072, upload-time = "2025-04-22T16:40:28.174Z" }, ] [[package]] @@ -555,18 +555,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "protobuf" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/39/24/33db22342cf4a2ea27c9955e6713140fedd51e8b141b5ce5260897020f1a/googleapis_common_protos-1.70.0.tar.gz", hash = "sha256:0e1b44e0ea153e6594f9f394fef15193a68aaaea2d843f83e2742717ca753257", size = 145903 } +sdist = { url = "https://files.pythonhosted.org/packages/39/24/33db22342cf4a2ea27c9955e6713140fedd51e8b141b5ce5260897020f1a/googleapis_common_protos-1.70.0.tar.gz", hash = "sha256:0e1b44e0ea153e6594f9f394fef15193a68aaaea2d843f83e2742717ca753257", size = 145903, upload-time = "2025-04-14T10:17:02.924Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/86/f1/62a193f0227cf15a920390abe675f386dec35f7ae3ffe6da582d3ade42c7/googleapis_common_protos-1.70.0-py3-none-any.whl", hash = "sha256:b8bfcca8c25a2bb253e0e0b0adaf8c00773e5e6af6fd92397576680b807e0fd8", size = 294530 }, + { url = "https://files.pythonhosted.org/packages/86/f1/62a193f0227cf15a920390abe675f386dec35f7ae3ffe6da582d3ade42c7/googleapis_common_protos-1.70.0-py3-none-any.whl", hash = "sha256:b8bfcca8c25a2bb253e0e0b0adaf8c00773e5e6af6fd92397576680b807e0fd8", size = 294530, upload-time = "2025-04-14T10:17:01.271Z" }, ] [[package]] name = "h11" version = "0.16.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/01/ee/02a2c011bdab74c6fb3c75474d40b3052059d95df7e73351460c8588d963/h11-0.16.0.tar.gz", hash = "sha256:4e35b956cf45792e4caa5885e69fba00bdbc6ffafbfa020300e549b208ee5ff1", size = 101250 } +sdist = { url = "https://files.pythonhosted.org/packages/01/ee/02a2c011bdab74c6fb3c75474d40b3052059d95df7e73351460c8588d963/h11-0.16.0.tar.gz", hash = "sha256:4e35b956cf45792e4caa5885e69fba00bdbc6ffafbfa020300e549b208ee5ff1", size = 101250, upload-time = "2025-04-24T03:35:25.427Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/04/4b/29cac41a4d98d144bf5f6d33995617b185d14b22401f75ca86f384e87ff1/h11-0.16.0-py3-none-any.whl", hash = "sha256:63cf8bbe7522de3bf65932fda1d9c2772064ffb3dae62d55932da54b31cb6c86", size = 37515 }, + { url = "https://files.pythonhosted.org/packages/04/4b/29cac41a4d98d144bf5f6d33995617b185d14b22401f75ca86f384e87ff1/h11-0.16.0-py3-none-any.whl", hash = "sha256:63cf8bbe7522de3bf65932fda1d9c2772064ffb3dae62d55932da54b31cb6c86", size = 37515, upload-time = "2025-04-24T03:35:24.344Z" }, ] [[package]] @@ -591,9 +591,9 @@ dependencies = [ { name = "virtualenv" }, { name = "zstandard" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/1f/43/c0b37db0e857a44ce5ffdb7e8a9b8fa6425d0b74dea698fafcd9bddb50d1/hatch-1.14.1.tar.gz", hash = "sha256:ca1aff788f8596b0dd1f8f8dfe776443d2724a86b1976fabaf087406ba3d0713", size = 5188180 } +sdist = { url = "https://files.pythonhosted.org/packages/1f/43/c0b37db0e857a44ce5ffdb7e8a9b8fa6425d0b74dea698fafcd9bddb50d1/hatch-1.14.1.tar.gz", hash = "sha256:ca1aff788f8596b0dd1f8f8dfe776443d2724a86b1976fabaf087406ba3d0713", size = 5188180, upload-time = "2025-04-07T04:16:04.522Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/a5/40/19c0935bf9f25808541a0e3144ac459de696c5b6b6d4511a98d456c69604/hatch-1.14.1-py3-none-any.whl", hash = "sha256:39cdaa59e47ce0c5505d88a951f4324a9c5aafa17e4a877e2fde79b36ab66c21", size = 125770 }, + { url = "https://files.pythonhosted.org/packages/a5/40/19c0935bf9f25808541a0e3144ac459de696c5b6b6d4511a98d456c69604/hatch-1.14.1-py3-none-any.whl", hash = "sha256:39cdaa59e47ce0c5505d88a951f4324a9c5aafa17e4a877e2fde79b36ab66c21", size = 125770, upload-time = "2025-04-07T04:16:02.525Z" }, ] [[package]] @@ -607,9 +607,9 @@ dependencies = [ { name = "tomli", marker = "python_full_version < '3.11'" }, { name = "trove-classifiers" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/8f/8a/cc1debe3514da292094f1c3a700e4ca25442489731ef7c0814358816bb03/hatchling-1.27.0.tar.gz", hash = "sha256:971c296d9819abb3811112fc52c7a9751c8d381898f36533bb16f9791e941fd6", size = 54983 } +sdist = { url = "https://files.pythonhosted.org/packages/8f/8a/cc1debe3514da292094f1c3a700e4ca25442489731ef7c0814358816bb03/hatchling-1.27.0.tar.gz", hash = "sha256:971c296d9819abb3811112fc52c7a9751c8d381898f36533bb16f9791e941fd6", size = 54983, upload-time = "2024-12-15T17:08:11.894Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/08/e7/ae38d7a6dfba0533684e0b2136817d667588ae3ec984c1a4e5df5eb88482/hatchling-1.27.0-py3-none-any.whl", hash = "sha256:d3a2f3567c4f926ea39849cdf924c7e99e6686c9c8e288ae1037c8fa2a5d937b", size = 75794 }, + { url = "https://files.pythonhosted.org/packages/08/e7/ae38d7a6dfba0533684e0b2136817d667588ae3ec984c1a4e5df5eb88482/hatchling-1.27.0-py3-none-any.whl", hash = "sha256:d3a2f3567c4f926ea39849cdf924c7e99e6686c9c8e288ae1037c8fa2a5d937b", size = 75794, upload-time = "2024-12-15T17:08:10.364Z" }, ] [[package]] @@ -620,9 +620,9 @@ dependencies = [ { name = "certifi" }, { name = "h11" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/06/94/82699a10bca87a5556c9c59b5963f2d039dbd239f25bc2a63907a05a14cb/httpcore-1.0.9.tar.gz", hash = "sha256:6e34463af53fd2ab5d807f399a9b45ea31c3dfa2276f15a2c3f00afff6e176e8", size = 85484 } +sdist = { url = "https://files.pythonhosted.org/packages/06/94/82699a10bca87a5556c9c59b5963f2d039dbd239f25bc2a63907a05a14cb/httpcore-1.0.9.tar.gz", hash = "sha256:6e34463af53fd2ab5d807f399a9b45ea31c3dfa2276f15a2c3f00afff6e176e8", size = 85484, upload-time = "2025-04-24T22:06:22.219Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/7e/f5/f66802a942d491edb555dd61e3a9961140fd64c90bce1eafd741609d334d/httpcore-1.0.9-py3-none-any.whl", hash = "sha256:2d400746a40668fc9dec9810239072b40b4484b640a8c38fd654a024c7a1bf55", size = 78784 }, + { url = "https://files.pythonhosted.org/packages/7e/f5/f66802a942d491edb555dd61e3a9961140fd64c90bce1eafd741609d334d/httpcore-1.0.9-py3-none-any.whl", hash = "sha256:2d400746a40668fc9dec9810239072b40b4484b640a8c38fd654a024c7a1bf55", size = 78784, upload-time = "2025-04-24T22:06:20.566Z" }, ] [[package]] @@ -632,9 +632,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pyparsing" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/3d/ad/2371116b22d616c194aa25ec410c9c6c37f23599dcd590502b74db197584/httplib2-0.22.0.tar.gz", hash = "sha256:d7a10bc5ef5ab08322488bde8c726eeee5c8618723fdb399597ec58f3d82df81", size = 351116 } +sdist = { url = "https://files.pythonhosted.org/packages/3d/ad/2371116b22d616c194aa25ec410c9c6c37f23599dcd590502b74db197584/httplib2-0.22.0.tar.gz", hash = "sha256:d7a10bc5ef5ab08322488bde8c726eeee5c8618723fdb399597ec58f3d82df81", size = 351116, upload-time = "2023-03-21T22:29:37.214Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/a8/6c/d2fbdaaa5959339d53ba38e94c123e4e84b8fbc4b84beb0e70d7c1608486/httplib2-0.22.0-py3-none-any.whl", hash = "sha256:14ae0a53c1ba8f3d37e9e27cf37eabb0fb9980f435ba405d546948b009dd64dc", size = 96854 }, + { url = "https://files.pythonhosted.org/packages/a8/6c/d2fbdaaa5959339d53ba38e94c123e4e84b8fbc4b84beb0e70d7c1608486/httplib2-0.22.0-py3-none-any.whl", hash = "sha256:14ae0a53c1ba8f3d37e9e27cf37eabb0fb9980f435ba405d546948b009dd64dc", size = 96854, upload-time = "2023-03-21T22:29:35.683Z" }, ] [[package]] @@ -647,9 +647,9 @@ dependencies = [ { name = "httpcore" }, { name = "idna" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b1/df/48c586a5fe32a0f01324ee087459e112ebb7224f646c0b5023f5e79e9956/httpx-0.28.1.tar.gz", hash = "sha256:75e98c5f16b0f35b567856f597f06ff2270a374470a5c2392242528e3e3e42fc", size = 141406 } +sdist = { url = "https://files.pythonhosted.org/packages/b1/df/48c586a5fe32a0f01324ee087459e112ebb7224f646c0b5023f5e79e9956/httpx-0.28.1.tar.gz", hash = "sha256:75e98c5f16b0f35b567856f597f06ff2270a374470a5c2392242528e3e3e42fc", size = 141406, upload-time = "2024-12-06T15:37:23.222Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2a/39/e50c7c3a983047577ee07d2a9e53faf5a69493943ec3f6a384bdc792deb2/httpx-0.28.1-py3-none-any.whl", hash = "sha256:d909fcccc110f8c7faf814ca82a9a4d816bc5a6dbfea25d6591d6985b8ba59ad", size = 73517 }, + { url = "https://files.pythonhosted.org/packages/2a/39/e50c7c3a983047577ee07d2a9e53faf5a69493943ec3f6a384bdc792deb2/httpx-0.28.1-py3-none-any.whl", hash = "sha256:d909fcccc110f8c7faf814ca82a9a4d816bc5a6dbfea25d6591d6985b8ba59ad", size = 73517, upload-time = "2024-12-06T15:37:21.509Z" }, ] [[package]] @@ -659,9 +659,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "idna" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/3a/51/1947bd81d75af87e3bb9e34593a4cf118115a8feb451ce7a69044ef1412e/hyperlink-21.0.0.tar.gz", hash = "sha256:427af957daa58bc909471c6c40f74c5450fa123dd093fc53efd2e91d2705a56b", size = 140743 } +sdist = { url = "https://files.pythonhosted.org/packages/3a/51/1947bd81d75af87e3bb9e34593a4cf118115a8feb451ce7a69044ef1412e/hyperlink-21.0.0.tar.gz", hash = "sha256:427af957daa58bc909471c6c40f74c5450fa123dd093fc53efd2e91d2705a56b", size = 140743, upload-time = "2021-01-08T05:51:20.972Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/6e/aa/8caf6a0a3e62863cbb9dab27135660acba46903b703e224f14f447e57934/hyperlink-21.0.0-py2.py3-none-any.whl", hash = "sha256:e6b14c37ecb73e89c77d78cdb4c2cc8f3fb59a885c5b3f819ff4ed80f25af1b4", size = 74638 }, + { url = "https://files.pythonhosted.org/packages/6e/aa/8caf6a0a3e62863cbb9dab27135660acba46903b703e224f14f447e57934/hyperlink-21.0.0-py2.py3-none-any.whl", hash = "sha256:e6b14c37ecb73e89c77d78cdb4c2cc8f3fb59a885c5b3f819ff4ed80f25af1b4", size = 74638, upload-time = "2021-01-08T05:51:22.906Z" }, ] [[package]] @@ -671,18 +671,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "requests" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/22/11/102da08f88412d875fa2f1a9a469ff7ad4c874b0ca6fed0048fe385bdb3d/id-1.5.0.tar.gz", hash = "sha256:292cb8a49eacbbdbce97244f47a97b4c62540169c976552e497fd57df0734c1d", size = 15237 } +sdist = { url = "https://files.pythonhosted.org/packages/22/11/102da08f88412d875fa2f1a9a469ff7ad4c874b0ca6fed0048fe385bdb3d/id-1.5.0.tar.gz", hash = "sha256:292cb8a49eacbbdbce97244f47a97b4c62540169c976552e497fd57df0734c1d", size = 15237, upload-time = "2024-12-04T19:53:05.575Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/9f/cb/18326d2d89ad3b0dd143da971e77afd1e6ca6674f1b1c3df4b6bec6279fc/id-1.5.0-py3-none-any.whl", hash = "sha256:f1434e1cef91f2cbb8a4ec64663d5a23b9ed43ef44c4c957d02583d61714c658", size = 13611 }, + { url = "https://files.pythonhosted.org/packages/9f/cb/18326d2d89ad3b0dd143da971e77afd1e6ca6674f1b1c3df4b6bec6279fc/id-1.5.0-py3-none-any.whl", hash = "sha256:f1434e1cef91f2cbb8a4ec64663d5a23b9ed43ef44c4c957d02583d61714c658", size = 13611, upload-time = "2024-12-04T19:53:03.02Z" }, ] [[package]] name = "idna" version = "3.10" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/f1/70/7703c29685631f5a7590aa73f1f1d3fa9a380e654b86af429e0934a32f7d/idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9", size = 190490 } +sdist = { url = "https://files.pythonhosted.org/packages/f1/70/7703c29685631f5a7590aa73f1f1d3fa9a380e654b86af429e0934a32f7d/idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9", size = 190490, upload-time = "2024-09-15T18:07:39.745Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/76/c6/c88e154df9c4e1a2a66ccf0005a88dfb2650c1dffb6f5ce603dfbd452ce3/idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3", size = 70442 }, + { url = "https://files.pythonhosted.org/packages/76/c6/c88e154df9c4e1a2a66ccf0005a88dfb2650c1dffb6f5ce603dfbd452ce3/idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3", size = 70442, upload-time = "2024-09-15T18:07:37.964Z" }, ] [[package]] @@ -692,18 +692,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "zipp", marker = "python_full_version < '3.13'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/76/66/650a33bd90f786193e4de4b3ad86ea60b53c89b669a5c7be931fac31cdb0/importlib_metadata-8.7.0.tar.gz", hash = "sha256:d13b81ad223b890aa16c5471f2ac3056cf76c5f10f82d6f9292f0b415f389000", size = 56641 } +sdist = { url = "https://files.pythonhosted.org/packages/76/66/650a33bd90f786193e4de4b3ad86ea60b53c89b669a5c7be931fac31cdb0/importlib_metadata-8.7.0.tar.gz", hash = "sha256:d13b81ad223b890aa16c5471f2ac3056cf76c5f10f82d6f9292f0b415f389000", size = 56641, upload-time = "2025-04-27T15:29:01.736Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/20/b0/36bd937216ec521246249be3bf9855081de4c5e06a0c9b4219dbeda50373/importlib_metadata-8.7.0-py3-none-any.whl", hash = "sha256:e5dd1551894c77868a30651cef00984d50e1002d06942a7101d34870c5f02afd", size = 27656 }, + { url = "https://files.pythonhosted.org/packages/20/b0/36bd937216ec521246249be3bf9855081de4c5e06a0c9b4219dbeda50373/importlib_metadata-8.7.0-py3-none-any.whl", hash = "sha256:e5dd1551894c77868a30651cef00984d50e1002d06942a7101d34870c5f02afd", size = 27656, upload-time = "2025-04-27T15:29:00.214Z" }, ] [[package]] name = "iniconfig" version = "2.1.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/f2/97/ebf4da567aa6827c909642694d71c9fcf53e5b504f2d96afea02718862f3/iniconfig-2.1.0.tar.gz", hash = "sha256:3abbd2e30b36733fee78f9c7f7308f2d0050e88f0087fd25c2645f63c773e1c7", size = 4793 } +sdist = { url = "https://files.pythonhosted.org/packages/f2/97/ebf4da567aa6827c909642694d71c9fcf53e5b504f2d96afea02718862f3/iniconfig-2.1.0.tar.gz", hash = "sha256:3abbd2e30b36733fee78f9c7f7308f2d0050e88f0087fd25c2645f63c773e1c7", size = 4793, upload-time = "2025-03-19T20:09:59.721Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2c/e1/e6716421ea10d38022b952c159d5161ca1193197fb744506875fbb87ea7b/iniconfig-2.1.0-py3-none-any.whl", hash = "sha256:9deba5723312380e77435581c6bf4935c94cbfab9b1ed33ef8d238ea168eb760", size = 6050 }, + { url = "https://files.pythonhosted.org/packages/2c/e1/e6716421ea10d38022b952c159d5161ca1193197fb744506875fbb87ea7b/iniconfig-2.1.0-py3-none-any.whl", hash = "sha256:9deba5723312380e77435581c6bf4935c94cbfab9b1ed33ef8d238ea168eb760", size = 6050, upload-time = "2025-03-19T20:10:01.071Z" }, ] [[package]] @@ -711,7 +711,7 @@ name = "inputimeout" version = "1.0.4" source = { registry = "https://pypi.org/simple" } wheels = [ - { url = "https://files.pythonhosted.org/packages/97/9c/1646ca469bc2dc299ac393c8d31136c6c22a35ca1e373fa462ac01100d37/inputimeout-1.0.4-py3-none-any.whl", hash = "sha256:f4e23d27753cfc25268eefc8d52a3edc46280ad831d226617c51882423475a43", size = 4639 }, + { url = "https://files.pythonhosted.org/packages/97/9c/1646ca469bc2dc299ac393c8d31136c6c22a35ca1e373fa462ac01100d37/inputimeout-1.0.4-py3-none-any.whl", hash = "sha256:f4e23d27753cfc25268eefc8d52a3edc46280ad831d226617c51882423475a43", size = 4639, upload-time = "2018-03-02T14:28:06.903Z" }, ] [[package]] @@ -721,9 +721,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "more-itertools" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/06/c0/ed4a27bc5571b99e3cff68f8a9fa5b56ff7df1c2251cc715a652ddd26402/jaraco.classes-3.4.0.tar.gz", hash = "sha256:47a024b51d0239c0dd8c8540c6c7f484be3b8fcf0b2d85c13825780d3b3f3acd", size = 11780 } +sdist = { url = "https://files.pythonhosted.org/packages/06/c0/ed4a27bc5571b99e3cff68f8a9fa5b56ff7df1c2251cc715a652ddd26402/jaraco.classes-3.4.0.tar.gz", hash = "sha256:47a024b51d0239c0dd8c8540c6c7f484be3b8fcf0b2d85c13825780d3b3f3acd", size = 11780, upload-time = "2024-03-31T07:27:36.643Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/7f/66/b15ce62552d84bbfcec9a4873ab79d993a1dd4edb922cbfccae192bd5b5f/jaraco.classes-3.4.0-py3-none-any.whl", hash = "sha256:f662826b6bed8cace05e7ff873ce0f9283b5c924470fe664fff1c2f00f581790", size = 6777 }, + { url = "https://files.pythonhosted.org/packages/7f/66/b15ce62552d84bbfcec9a4873ab79d993a1dd4edb922cbfccae192bd5b5f/jaraco.classes-3.4.0-py3-none-any.whl", hash = "sha256:f662826b6bed8cace05e7ff873ce0f9283b5c924470fe664fff1c2f00f581790", size = 6777, upload-time = "2024-03-31T07:27:34.792Z" }, ] [[package]] @@ -733,9 +733,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "backports-tarfile", marker = "python_full_version < '3.12'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/df/ad/f3777b81bf0b6e7bc7514a1656d3e637b2e8e15fab2ce3235730b3e7a4e6/jaraco_context-6.0.1.tar.gz", hash = "sha256:9bae4ea555cf0b14938dc0aee7c9f32ed303aa20a3b73e7dc80111628792d1b3", size = 13912 } +sdist = { url = "https://files.pythonhosted.org/packages/df/ad/f3777b81bf0b6e7bc7514a1656d3e637b2e8e15fab2ce3235730b3e7a4e6/jaraco_context-6.0.1.tar.gz", hash = "sha256:9bae4ea555cf0b14938dc0aee7c9f32ed303aa20a3b73e7dc80111628792d1b3", size = 13912, upload-time = "2024-08-20T03:39:27.358Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ff/db/0c52c4cf5e4bd9f5d7135ec7669a3a767af21b3a308e1ed3674881e52b62/jaraco.context-6.0.1-py3-none-any.whl", hash = "sha256:f797fc481b490edb305122c9181830a3a5b76d84ef6d1aef2fb9b47ab956f9e4", size = 6825 }, + { url = "https://files.pythonhosted.org/packages/ff/db/0c52c4cf5e4bd9f5d7135ec7669a3a767af21b3a308e1ed3674881e52b62/jaraco.context-6.0.1-py3-none-any.whl", hash = "sha256:f797fc481b490edb305122c9181830a3a5b76d84ef6d1aef2fb9b47ab956f9e4", size = 6825, upload-time = "2024-08-20T03:39:25.966Z" }, ] [[package]] @@ -745,18 +745,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "more-itertools" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/49/1c/831faaaa0f090b711c355c6d8b2abf277c72133aab472b6932b03322294c/jaraco_functools-4.2.1.tar.gz", hash = "sha256:be634abfccabce56fa3053f8c7ebe37b682683a4ee7793670ced17bab0087353", size = 19661 } +sdist = { url = "https://files.pythonhosted.org/packages/49/1c/831faaaa0f090b711c355c6d8b2abf277c72133aab472b6932b03322294c/jaraco_functools-4.2.1.tar.gz", hash = "sha256:be634abfccabce56fa3053f8c7ebe37b682683a4ee7793670ced17bab0087353", size = 19661, upload-time = "2025-06-21T19:22:03.201Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/f3/fd/179a20f832824514df39a90bb0e5372b314fea99f217f5ab942b10a8a4e8/jaraco_functools-4.2.1-py3-none-any.whl", hash = "sha256:590486285803805f4b1f99c60ca9e94ed348d4added84b74c7a12885561e524e", size = 10349 }, + { url = "https://files.pythonhosted.org/packages/f3/fd/179a20f832824514df39a90bb0e5372b314fea99f217f5ab942b10a8a4e8/jaraco_functools-4.2.1-py3-none-any.whl", hash = "sha256:590486285803805f4b1f99c60ca9e94ed348d4added84b74c7a12885561e524e", size = 10349, upload-time = "2025-06-21T19:22:02.039Z" }, ] [[package]] name = "jeepney" version = "0.9.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/7b/6f/357efd7602486741aa73ffc0617fb310a29b588ed0fd69c2399acbb85b0c/jeepney-0.9.0.tar.gz", hash = "sha256:cf0e9e845622b81e4a28df94c40345400256ec608d0e55bb8a3feaa9163f5732", size = 106758 } +sdist = { url = "https://files.pythonhosted.org/packages/7b/6f/357efd7602486741aa73ffc0617fb310a29b588ed0fd69c2399acbb85b0c/jeepney-0.9.0.tar.gz", hash = "sha256:cf0e9e845622b81e4a28df94c40345400256ec608d0e55bb8a3feaa9163f5732", size = 106758, upload-time = "2025-02-27T18:51:01.684Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b2/a3/e137168c9c44d18eff0376253da9f1e9234d0239e0ee230d2fee6cea8e55/jeepney-0.9.0-py3-none-any.whl", hash = "sha256:97e5714520c16fc0a45695e5365a2e11b81ea79bba796e26f9f1d178cb182683", size = 49010 }, + { url = "https://files.pythonhosted.org/packages/b2/a3/e137168c9c44d18eff0376253da9f1e9234d0239e0ee230d2fee6cea8e55/jeepney-0.9.0-py3-none-any.whl", hash = "sha256:97e5714520c16fc0a45695e5365a2e11b81ea79bba796e26f9f1d178cb182683", size = 49010, upload-time = "2025-02-27T18:51:00.104Z" }, ] [[package]] @@ -766,18 +766,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "markupsafe" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/df/bf/f7da0350254c0ed7c72f3e33cef02e048281fec7ecec5f032d4aac52226b/jinja2-3.1.6.tar.gz", hash = "sha256:0137fb05990d35f1275a587e9aee6d56da821fc83491a0fb838183be43f66d6d", size = 245115 } +sdist = { url = "https://files.pythonhosted.org/packages/df/bf/f7da0350254c0ed7c72f3e33cef02e048281fec7ecec5f032d4aac52226b/jinja2-3.1.6.tar.gz", hash = "sha256:0137fb05990d35f1275a587e9aee6d56da821fc83491a0fb838183be43f66d6d", size = 245115, upload-time = "2025-03-05T20:05:02.478Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/62/a1/3d680cbfd5f4b8f15abc1d571870c5fc3e594bb582bc3b64ea099db13e56/jinja2-3.1.6-py3-none-any.whl", hash = "sha256:85ece4451f492d0c13c5dd7c13a64681a86afae63a5f347908daf103ce6d2f67", size = 134899 }, + { url = "https://files.pythonhosted.org/packages/62/a1/3d680cbfd5f4b8f15abc1d571870c5fc3e594bb582bc3b64ea099db13e56/jinja2-3.1.6-py3-none-any.whl", hash = "sha256:85ece4451f492d0c13c5dd7c13a64681a86afae63a5f347908daf103ce6d2f67", size = 134899, upload-time = "2025-03-05T20:05:00.369Z" }, ] [[package]] name = "jmespath" version = "1.0.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/00/2a/e867e8531cf3e36b41201936b7fa7ba7b5702dbef42922193f05c8976cd6/jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe", size = 25843 } +sdist = { url = "https://files.pythonhosted.org/packages/00/2a/e867e8531cf3e36b41201936b7fa7ba7b5702dbef42922193f05c8976cd6/jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe", size = 25843, upload-time = "2022-06-17T18:00:12.224Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/31/b4/b9b800c45527aadd64d5b442f9b932b00648617eb5d63d2c7a6587b7cafc/jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980", size = 20256 }, + { url = "https://files.pythonhosted.org/packages/31/b4/b9b800c45527aadd64d5b442f9b932b00648617eb5d63d2c7a6587b7cafc/jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980", size = 20256, upload-time = "2022-06-17T18:00:10.251Z" }, ] [[package]] @@ -790,9 +790,9 @@ dependencies = [ { name = "referencing" }, { name = "rpds-py" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/d5/00/a297a868e9d0784450faa7365c2172a7d6110c763e30ba861867c32ae6a9/jsonschema-4.25.0.tar.gz", hash = "sha256:e63acf5c11762c0e6672ffb61482bdf57f0876684d8d249c0fe2d730d48bc55f", size = 356830 } +sdist = { url = "https://files.pythonhosted.org/packages/d5/00/a297a868e9d0784450faa7365c2172a7d6110c763e30ba861867c32ae6a9/jsonschema-4.25.0.tar.gz", hash = "sha256:e63acf5c11762c0e6672ffb61482bdf57f0876684d8d249c0fe2d730d48bc55f", size = 356830, upload-time = "2025-07-18T15:39:45.11Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/fe/54/c86cd8e011fe98803d7e382fd67c0df5ceab8d2b7ad8c5a81524f791551c/jsonschema-4.25.0-py3-none-any.whl", hash = "sha256:24c2e8da302de79c8b9382fee3e76b355e44d2a4364bb207159ce10b517bd716", size = 89184 }, + { url = "https://files.pythonhosted.org/packages/fe/54/c86cd8e011fe98803d7e382fd67c0df5ceab8d2b7ad8c5a81524f791551c/jsonschema-4.25.0-py3-none-any.whl", hash = "sha256:24c2e8da302de79c8b9382fee3e76b355e44d2a4364bb207159ce10b517bd716", size = 89184, upload-time = "2025-07-18T15:39:42.956Z" }, ] [[package]] @@ -802,9 +802,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "referencing" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/bf/ce/46fbd9c8119cfc3581ee5643ea49464d168028cfb5caff5fc0596d0cf914/jsonschema_specifications-2025.4.1.tar.gz", hash = "sha256:630159c9f4dbea161a6a2205c3011cc4f18ff381b189fff48bb39b9bf26ae608", size = 15513 } +sdist = { url = "https://files.pythonhosted.org/packages/bf/ce/46fbd9c8119cfc3581ee5643ea49464d168028cfb5caff5fc0596d0cf914/jsonschema_specifications-2025.4.1.tar.gz", hash = "sha256:630159c9f4dbea161a6a2205c3011cc4f18ff381b189fff48bb39b9bf26ae608", size = 15513, upload-time = "2025-04-23T12:34:07.418Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/01/0e/b27cdbaccf30b890c40ed1da9fd4a3593a5cf94dae54fb34f8a4b74fcd3f/jsonschema_specifications-2025.4.1-py3-none-any.whl", hash = "sha256:4653bffbd6584f7de83a67e0d620ef16900b390ddc7939d56684d6c81e33f1af", size = 18437 }, + { url = "https://files.pythonhosted.org/packages/01/0e/b27cdbaccf30b890c40ed1da9fd4a3593a5cf94dae54fb34f8a4b74fcd3f/jsonschema_specifications-2025.4.1-py3-none-any.whl", hash = "sha256:4653bffbd6584f7de83a67e0d620ef16900b390ddc7939d56684d6c81e33f1af", size = 18437, upload-time = "2025-04-23T12:34:05.422Z" }, ] [[package]] @@ -820,9 +820,9 @@ dependencies = [ { name = "pywin32-ctypes", marker = "sys_platform == 'win32'" }, { name = "secretstorage", marker = "sys_platform == 'linux'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/70/09/d904a6e96f76ff214be59e7aa6ef7190008f52a0ab6689760a98de0bf37d/keyring-25.6.0.tar.gz", hash = "sha256:0b39998aa941431eb3d9b0d4b2460bc773b9df6fed7621c2dfb291a7e0187a66", size = 62750 } +sdist = { url = "https://files.pythonhosted.org/packages/70/09/d904a6e96f76ff214be59e7aa6ef7190008f52a0ab6689760a98de0bf37d/keyring-25.6.0.tar.gz", hash = "sha256:0b39998aa941431eb3d9b0d4b2460bc773b9df6fed7621c2dfb291a7e0187a66", size = 62750, upload-time = "2024-12-25T15:26:45.782Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d3/32/da7f44bcb1105d3e88a0b74ebdca50c59121d2ddf71c9e34ba47df7f3a56/keyring-25.6.0-py3-none-any.whl", hash = "sha256:552a3f7af126ece7ed5c89753650eec89c7eaae8617d0aa4d9ad2b75111266bd", size = 39085 }, + { url = "https://files.pythonhosted.org/packages/d3/32/da7f44bcb1105d3e88a0b74ebdca50c59121d2ddf71c9e34ba47df7f3a56/keyring-25.6.0-py3-none-any.whl", hash = "sha256:552a3f7af126ece7ed5c89753650eec89c7eaae8617d0aa4d9ad2b75111266bd", size = 39085, upload-time = "2024-12-25T15:26:44.377Z" }, ] [[package]] @@ -832,154 +832,154 @@ 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 } +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 }, + { 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 = "markupsafe" version = "3.0.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/b2/97/5d42485e71dfc078108a86d6de8fa46db44a1a9295e89c5d6d4a06e23a62/markupsafe-3.0.2.tar.gz", hash = "sha256:ee55d3edf80167e48ea11a923c7386f4669df67d7994554387f84e7d8b0a2bf0", size = 20537 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/04/90/d08277ce111dd22f77149fd1a5d4653eeb3b3eaacbdfcbae5afb2600eebd/MarkupSafe-3.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7e94c425039cde14257288fd61dcfb01963e658efbc0ff54f5306b06054700f8", size = 14357 }, - { url = "https://files.pythonhosted.org/packages/04/e1/6e2194baeae0bca1fae6629dc0cbbb968d4d941469cbab11a3872edff374/MarkupSafe-3.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9e2d922824181480953426608b81967de705c3cef4d1af983af849d7bd619158", size = 12393 }, - { url = "https://files.pythonhosted.org/packages/1d/69/35fa85a8ece0a437493dc61ce0bb6d459dcba482c34197e3efc829aa357f/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38a9ef736c01fccdd6600705b09dc574584b89bea478200c5fbf112a6b0d5579", size = 21732 }, - { url = "https://files.pythonhosted.org/packages/22/35/137da042dfb4720b638d2937c38a9c2df83fe32d20e8c8f3185dbfef05f7/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbcb445fa71794da8f178f0f6d66789a28d7319071af7a496d4d507ed566270d", size = 20866 }, - { url = "https://files.pythonhosted.org/packages/29/28/6d029a903727a1b62edb51863232152fd335d602def598dade38996887f0/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:57cb5a3cf367aeb1d316576250f65edec5bb3be939e9247ae594b4bcbc317dfb", size = 20964 }, - { url = "https://files.pythonhosted.org/packages/cc/cd/07438f95f83e8bc028279909d9c9bd39e24149b0d60053a97b2bc4f8aa51/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3809ede931876f5b2ec92eef964286840ed3540dadf803dd570c3b7e13141a3b", size = 21977 }, - { url = "https://files.pythonhosted.org/packages/29/01/84b57395b4cc062f9c4c55ce0df7d3108ca32397299d9df00fedd9117d3d/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e07c3764494e3776c602c1e78e298937c3315ccc9043ead7e685b7f2b8d47b3c", size = 21366 }, - { url = "https://files.pythonhosted.org/packages/bd/6e/61ebf08d8940553afff20d1fb1ba7294b6f8d279df9fd0c0db911b4bbcfd/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b424c77b206d63d500bcb69fa55ed8d0e6a3774056bdc4839fc9298a7edca171", size = 21091 }, - { url = "https://files.pythonhosted.org/packages/11/23/ffbf53694e8c94ebd1e7e491de185124277964344733c45481f32ede2499/MarkupSafe-3.0.2-cp310-cp310-win32.whl", hash = "sha256:fcabf5ff6eea076f859677f5f0b6b5c1a51e70a376b0579e0eadef8db48c6b50", size = 15065 }, - { url = "https://files.pythonhosted.org/packages/44/06/e7175d06dd6e9172d4a69a72592cb3f7a996a9c396eee29082826449bbc3/MarkupSafe-3.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:6af100e168aa82a50e186c82875a5893c5597a0c1ccdb0d8b40240b1f28b969a", size = 15514 }, - { url = "https://files.pythonhosted.org/packages/6b/28/bbf83e3f76936960b850435576dd5e67034e200469571be53f69174a2dfd/MarkupSafe-3.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9025b4018f3a1314059769c7bf15441064b2207cb3f065e6ea1e7359cb46db9d", size = 14353 }, - { url = "https://files.pythonhosted.org/packages/6c/30/316d194b093cde57d448a4c3209f22e3046c5bb2fb0820b118292b334be7/MarkupSafe-3.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:93335ca3812df2f366e80509ae119189886b0f3c2b81325d39efdb84a1e2ae93", size = 12392 }, - { url = "https://files.pythonhosted.org/packages/f2/96/9cdafba8445d3a53cae530aaf83c38ec64c4d5427d975c974084af5bc5d2/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2cb8438c3cbb25e220c2ab33bb226559e7afb3baec11c4f218ffa7308603c832", size = 23984 }, - { url = "https://files.pythonhosted.org/packages/f1/a4/aefb044a2cd8d7334c8a47d3fb2c9f328ac48cb349468cc31c20b539305f/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a123e330ef0853c6e822384873bef7507557d8e4a082961e1defa947aa59ba84", size = 23120 }, - { url = "https://files.pythonhosted.org/packages/8d/21/5e4851379f88f3fad1de30361db501300d4f07bcad047d3cb0449fc51f8c/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e084f686b92e5b83186b07e8a17fc09e38fff551f3602b249881fec658d3eca", size = 23032 }, - { url = "https://files.pythonhosted.org/packages/00/7b/e92c64e079b2d0d7ddf69899c98842f3f9a60a1ae72657c89ce2655c999d/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d8213e09c917a951de9d09ecee036d5c7d36cb6cb7dbaece4c71a60d79fb9798", size = 24057 }, - { url = "https://files.pythonhosted.org/packages/f9/ac/46f960ca323037caa0a10662ef97d0a4728e890334fc156b9f9e52bcc4ca/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5b02fb34468b6aaa40dfc198d813a641e3a63b98c2b05a16b9f80b7ec314185e", size = 23359 }, - { url = "https://files.pythonhosted.org/packages/69/84/83439e16197337b8b14b6a5b9c2105fff81d42c2a7c5b58ac7b62ee2c3b1/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0bff5e0ae4ef2e1ae4fdf2dfd5b76c75e5c2fa4132d05fc1b0dabcd20c7e28c4", size = 23306 }, - { url = "https://files.pythonhosted.org/packages/9a/34/a15aa69f01e2181ed8d2b685c0d2f6655d5cca2c4db0ddea775e631918cd/MarkupSafe-3.0.2-cp311-cp311-win32.whl", hash = "sha256:6c89876f41da747c8d3677a2b540fb32ef5715f97b66eeb0c6b66f5e3ef6f59d", size = 15094 }, - { url = "https://files.pythonhosted.org/packages/da/b8/3a3bd761922d416f3dc5d00bfbed11f66b1ab89a0c2b6e887240a30b0f6b/MarkupSafe-3.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:70a87b411535ccad5ef2f1df5136506a10775d267e197e4cf531ced10537bd6b", size = 15521 }, - { url = "https://files.pythonhosted.org/packages/22/09/d1f21434c97fc42f09d290cbb6350d44eb12f09cc62c9476effdb33a18aa/MarkupSafe-3.0.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:9778bd8ab0a994ebf6f84c2b949e65736d5575320a17ae8984a77fab08db94cf", size = 14274 }, - { url = "https://files.pythonhosted.org/packages/6b/b0/18f76bba336fa5aecf79d45dcd6c806c280ec44538b3c13671d49099fdd0/MarkupSafe-3.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:846ade7b71e3536c4e56b386c2a47adf5741d2d8b94ec9dc3e92e5e1ee1e2225", size = 12348 }, - { url = "https://files.pythonhosted.org/packages/e0/25/dd5c0f6ac1311e9b40f4af06c78efde0f3b5cbf02502f8ef9501294c425b/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c99d261bd2d5f6b59325c92c73df481e05e57f19837bdca8413b9eac4bd8028", size = 24149 }, - { url = "https://files.pythonhosted.org/packages/f3/f0/89e7aadfb3749d0f52234a0c8c7867877876e0a20b60e2188e9850794c17/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e17c96c14e19278594aa4841ec148115f9c7615a47382ecb6b82bd8fea3ab0c8", size = 23118 }, - { url = "https://files.pythonhosted.org/packages/d5/da/f2eeb64c723f5e3777bc081da884b414671982008c47dcc1873d81f625b6/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:88416bd1e65dcea10bc7569faacb2c20ce071dd1f87539ca2ab364bf6231393c", size = 22993 }, - { url = "https://files.pythonhosted.org/packages/da/0e/1f32af846df486dce7c227fe0f2398dc7e2e51d4a370508281f3c1c5cddc/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:2181e67807fc2fa785d0592dc2d6206c019b9502410671cc905d132a92866557", size = 24178 }, - { url = "https://files.pythonhosted.org/packages/c4/f6/bb3ca0532de8086cbff5f06d137064c8410d10779c4c127e0e47d17c0b71/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:52305740fe773d09cffb16f8ed0427942901f00adedac82ec8b67752f58a1b22", size = 23319 }, - { url = "https://files.pythonhosted.org/packages/a2/82/8be4c96ffee03c5b4a034e60a31294daf481e12c7c43ab8e34a1453ee48b/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ad10d3ded218f1039f11a75f8091880239651b52e9bb592ca27de44eed242a48", size = 23352 }, - { url = "https://files.pythonhosted.org/packages/51/ae/97827349d3fcffee7e184bdf7f41cd6b88d9919c80f0263ba7acd1bbcb18/MarkupSafe-3.0.2-cp312-cp312-win32.whl", hash = "sha256:0f4ca02bea9a23221c0182836703cbf8930c5e9454bacce27e767509fa286a30", size = 15097 }, - { url = "https://files.pythonhosted.org/packages/c1/80/a61f99dc3a936413c3ee4e1eecac96c0da5ed07ad56fd975f1a9da5bc630/MarkupSafe-3.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:8e06879fc22a25ca47312fbe7c8264eb0b662f6db27cb2d3bbbc74b1df4b9b87", size = 15601 }, - { url = "https://files.pythonhosted.org/packages/83/0e/67eb10a7ecc77a0c2bbe2b0235765b98d164d81600746914bebada795e97/MarkupSafe-3.0.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ba9527cdd4c926ed0760bc301f6728ef34d841f405abf9d4f959c478421e4efd", size = 14274 }, - { url = "https://files.pythonhosted.org/packages/2b/6d/9409f3684d3335375d04e5f05744dfe7e9f120062c9857df4ab490a1031a/MarkupSafe-3.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f8b3d067f2e40fe93e1ccdd6b2e1d16c43140e76f02fb1319a05cf2b79d99430", size = 12352 }, - { url = "https://files.pythonhosted.org/packages/d2/f5/6eadfcd3885ea85fe2a7c128315cc1bb7241e1987443d78c8fe712d03091/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:569511d3b58c8791ab4c2e1285575265991e6d8f8700c7be0e88f86cb0672094", size = 24122 }, - { url = "https://files.pythonhosted.org/packages/0c/91/96cf928db8236f1bfab6ce15ad070dfdd02ed88261c2afafd4b43575e9e9/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15ab75ef81add55874e7ab7055e9c397312385bd9ced94920f2802310c930396", size = 23085 }, - { url = "https://files.pythonhosted.org/packages/c2/cf/c9d56af24d56ea04daae7ac0940232d31d5a8354f2b457c6d856b2057d69/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f3818cb119498c0678015754eba762e0d61e5b52d34c8b13d770f0719f7b1d79", size = 22978 }, - { url = "https://files.pythonhosted.org/packages/2a/9f/8619835cd6a711d6272d62abb78c033bda638fdc54c4e7f4272cf1c0962b/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:cdb82a876c47801bb54a690c5ae105a46b392ac6099881cdfb9f6e95e4014c6a", size = 24208 }, - { url = "https://files.pythonhosted.org/packages/f9/bf/176950a1792b2cd2102b8ffeb5133e1ed984547b75db47c25a67d3359f77/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:cabc348d87e913db6ab4aa100f01b08f481097838bdddf7c7a84b7575b7309ca", size = 23357 }, - { url = "https://files.pythonhosted.org/packages/ce/4f/9a02c1d335caabe5c4efb90e1b6e8ee944aa245c1aaaab8e8a618987d816/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:444dcda765c8a838eaae23112db52f1efaf750daddb2d9ca300bcae1039adc5c", size = 23344 }, - { url = "https://files.pythonhosted.org/packages/ee/55/c271b57db36f748f0e04a759ace9f8f759ccf22b4960c270c78a394f58be/MarkupSafe-3.0.2-cp313-cp313-win32.whl", hash = "sha256:bcf3e58998965654fdaff38e58584d8937aa3096ab5354d493c77d1fdd66d7a1", size = 15101 }, - { url = "https://files.pythonhosted.org/packages/29/88/07df22d2dd4df40aba9f3e402e6dc1b8ee86297dddbad4872bd5e7b0094f/MarkupSafe-3.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:e6a2a455bd412959b57a172ce6328d2dd1f01cb2135efda2e4576e8a23fa3b0f", size = 15603 }, - { url = "https://files.pythonhosted.org/packages/62/6a/8b89d24db2d32d433dffcd6a8779159da109842434f1dd2f6e71f32f738c/MarkupSafe-3.0.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:b5a6b3ada725cea8a5e634536b1b01c30bcdcd7f9c6fff4151548d5bf6b3a36c", size = 14510 }, - { url = "https://files.pythonhosted.org/packages/7a/06/a10f955f70a2e5a9bf78d11a161029d278eeacbd35ef806c3fd17b13060d/MarkupSafe-3.0.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a904af0a6162c73e3edcb969eeeb53a63ceeb5d8cf642fade7d39e7963a22ddb", size = 12486 }, - { url = "https://files.pythonhosted.org/packages/34/cf/65d4a571869a1a9078198ca28f39fba5fbb910f952f9dbc5220afff9f5e6/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4aa4e5faecf353ed117801a068ebab7b7e09ffb6e1d5e412dc852e0da018126c", size = 25480 }, - { url = "https://files.pythonhosted.org/packages/0c/e3/90e9651924c430b885468b56b3d597cabf6d72be4b24a0acd1fa0e12af67/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0ef13eaeee5b615fb07c9a7dadb38eac06a0608b41570d8ade51c56539e509d", size = 23914 }, - { url = "https://files.pythonhosted.org/packages/66/8c/6c7cf61f95d63bb866db39085150df1f2a5bd3335298f14a66b48e92659c/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d16a81a06776313e817c951135cf7340a3e91e8c1ff2fac444cfd75fffa04afe", size = 23796 }, - { url = "https://files.pythonhosted.org/packages/bb/35/cbe9238ec3f47ac9a7c8b3df7a808e7cb50fe149dc7039f5f454b3fba218/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:6381026f158fdb7c72a168278597a5e3a5222e83ea18f543112b2662a9b699c5", size = 25473 }, - { url = "https://files.pythonhosted.org/packages/e6/32/7621a4382488aa283cc05e8984a9c219abad3bca087be9ec77e89939ded9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3d79d162e7be8f996986c064d1c7c817f6df3a77fe3d6859f6f9e7be4b8c213a", size = 24114 }, - { url = "https://files.pythonhosted.org/packages/0d/80/0985960e4b89922cb5a0bac0ed39c5b96cbc1a536a99f30e8c220a996ed9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:131a3c7689c85f5ad20f9f6fb1b866f402c445b220c19fe4308c0b147ccd2ad9", size = 24098 }, - { url = "https://files.pythonhosted.org/packages/82/78/fedb03c7d5380df2427038ec8d973587e90561b2d90cd472ce9254cf348b/MarkupSafe-3.0.2-cp313-cp313t-win32.whl", hash = "sha256:ba8062ed2cf21c07a9e295d5b8a2a5ce678b913b45fdf68c32d95d6c1291e0b6", size = 15208 }, - { url = "https://files.pythonhosted.org/packages/4f/65/6079a46068dfceaeabb5dcad6d674f5f5c61a6fa5673746f42a9f4c233b3/MarkupSafe-3.0.2-cp313-cp313t-win_amd64.whl", hash = "sha256:e444a31f8db13eb18ada366ab3cf45fd4b31e4db1236a4448f68778c1d1a5a2f", size = 15739 }, +sdist = { url = "https://files.pythonhosted.org/packages/b2/97/5d42485e71dfc078108a86d6de8fa46db44a1a9295e89c5d6d4a06e23a62/markupsafe-3.0.2.tar.gz", hash = "sha256:ee55d3edf80167e48ea11a923c7386f4669df67d7994554387f84e7d8b0a2bf0", size = 20537, upload-time = "2024-10-18T15:21:54.129Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/04/90/d08277ce111dd22f77149fd1a5d4653eeb3b3eaacbdfcbae5afb2600eebd/MarkupSafe-3.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7e94c425039cde14257288fd61dcfb01963e658efbc0ff54f5306b06054700f8", size = 14357, upload-time = "2024-10-18T15:20:51.44Z" }, + { url = "https://files.pythonhosted.org/packages/04/e1/6e2194baeae0bca1fae6629dc0cbbb968d4d941469cbab11a3872edff374/MarkupSafe-3.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9e2d922824181480953426608b81967de705c3cef4d1af983af849d7bd619158", size = 12393, upload-time = "2024-10-18T15:20:52.426Z" }, + { url = "https://files.pythonhosted.org/packages/1d/69/35fa85a8ece0a437493dc61ce0bb6d459dcba482c34197e3efc829aa357f/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38a9ef736c01fccdd6600705b09dc574584b89bea478200c5fbf112a6b0d5579", size = 21732, upload-time = "2024-10-18T15:20:53.578Z" }, + { url = "https://files.pythonhosted.org/packages/22/35/137da042dfb4720b638d2937c38a9c2df83fe32d20e8c8f3185dbfef05f7/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbcb445fa71794da8f178f0f6d66789a28d7319071af7a496d4d507ed566270d", size = 20866, upload-time = "2024-10-18T15:20:55.06Z" }, + { url = "https://files.pythonhosted.org/packages/29/28/6d029a903727a1b62edb51863232152fd335d602def598dade38996887f0/MarkupSafe-3.0.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:57cb5a3cf367aeb1d316576250f65edec5bb3be939e9247ae594b4bcbc317dfb", size = 20964, upload-time = "2024-10-18T15:20:55.906Z" }, + { url = "https://files.pythonhosted.org/packages/cc/cd/07438f95f83e8bc028279909d9c9bd39e24149b0d60053a97b2bc4f8aa51/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3809ede931876f5b2ec92eef964286840ed3540dadf803dd570c3b7e13141a3b", size = 21977, upload-time = "2024-10-18T15:20:57.189Z" }, + { url = "https://files.pythonhosted.org/packages/29/01/84b57395b4cc062f9c4c55ce0df7d3108ca32397299d9df00fedd9117d3d/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e07c3764494e3776c602c1e78e298937c3315ccc9043ead7e685b7f2b8d47b3c", size = 21366, upload-time = "2024-10-18T15:20:58.235Z" }, + { url = "https://files.pythonhosted.org/packages/bd/6e/61ebf08d8940553afff20d1fb1ba7294b6f8d279df9fd0c0db911b4bbcfd/MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b424c77b206d63d500bcb69fa55ed8d0e6a3774056bdc4839fc9298a7edca171", size = 21091, upload-time = "2024-10-18T15:20:59.235Z" }, + { url = "https://files.pythonhosted.org/packages/11/23/ffbf53694e8c94ebd1e7e491de185124277964344733c45481f32ede2499/MarkupSafe-3.0.2-cp310-cp310-win32.whl", hash = "sha256:fcabf5ff6eea076f859677f5f0b6b5c1a51e70a376b0579e0eadef8db48c6b50", size = 15065, upload-time = "2024-10-18T15:21:00.307Z" }, + { url = "https://files.pythonhosted.org/packages/44/06/e7175d06dd6e9172d4a69a72592cb3f7a996a9c396eee29082826449bbc3/MarkupSafe-3.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:6af100e168aa82a50e186c82875a5893c5597a0c1ccdb0d8b40240b1f28b969a", size = 15514, upload-time = "2024-10-18T15:21:01.122Z" }, + { url = "https://files.pythonhosted.org/packages/6b/28/bbf83e3f76936960b850435576dd5e67034e200469571be53f69174a2dfd/MarkupSafe-3.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9025b4018f3a1314059769c7bf15441064b2207cb3f065e6ea1e7359cb46db9d", size = 14353, upload-time = "2024-10-18T15:21:02.187Z" }, + { url = "https://files.pythonhosted.org/packages/6c/30/316d194b093cde57d448a4c3209f22e3046c5bb2fb0820b118292b334be7/MarkupSafe-3.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:93335ca3812df2f366e80509ae119189886b0f3c2b81325d39efdb84a1e2ae93", size = 12392, upload-time = "2024-10-18T15:21:02.941Z" }, + { url = "https://files.pythonhosted.org/packages/f2/96/9cdafba8445d3a53cae530aaf83c38ec64c4d5427d975c974084af5bc5d2/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2cb8438c3cbb25e220c2ab33bb226559e7afb3baec11c4f218ffa7308603c832", size = 23984, upload-time = "2024-10-18T15:21:03.953Z" }, + { url = "https://files.pythonhosted.org/packages/f1/a4/aefb044a2cd8d7334c8a47d3fb2c9f328ac48cb349468cc31c20b539305f/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a123e330ef0853c6e822384873bef7507557d8e4a082961e1defa947aa59ba84", size = 23120, upload-time = "2024-10-18T15:21:06.495Z" }, + { url = "https://files.pythonhosted.org/packages/8d/21/5e4851379f88f3fad1de30361db501300d4f07bcad047d3cb0449fc51f8c/MarkupSafe-3.0.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e084f686b92e5b83186b07e8a17fc09e38fff551f3602b249881fec658d3eca", size = 23032, upload-time = "2024-10-18T15:21:07.295Z" }, + { url = "https://files.pythonhosted.org/packages/00/7b/e92c64e079b2d0d7ddf69899c98842f3f9a60a1ae72657c89ce2655c999d/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d8213e09c917a951de9d09ecee036d5c7d36cb6cb7dbaece4c71a60d79fb9798", size = 24057, upload-time = "2024-10-18T15:21:08.073Z" }, + { url = "https://files.pythonhosted.org/packages/f9/ac/46f960ca323037caa0a10662ef97d0a4728e890334fc156b9f9e52bcc4ca/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5b02fb34468b6aaa40dfc198d813a641e3a63b98c2b05a16b9f80b7ec314185e", size = 23359, upload-time = "2024-10-18T15:21:09.318Z" }, + { url = "https://files.pythonhosted.org/packages/69/84/83439e16197337b8b14b6a5b9c2105fff81d42c2a7c5b58ac7b62ee2c3b1/MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0bff5e0ae4ef2e1ae4fdf2dfd5b76c75e5c2fa4132d05fc1b0dabcd20c7e28c4", size = 23306, upload-time = "2024-10-18T15:21:10.185Z" }, + { url = "https://files.pythonhosted.org/packages/9a/34/a15aa69f01e2181ed8d2b685c0d2f6655d5cca2c4db0ddea775e631918cd/MarkupSafe-3.0.2-cp311-cp311-win32.whl", hash = "sha256:6c89876f41da747c8d3677a2b540fb32ef5715f97b66eeb0c6b66f5e3ef6f59d", size = 15094, upload-time = "2024-10-18T15:21:11.005Z" }, + { url = "https://files.pythonhosted.org/packages/da/b8/3a3bd761922d416f3dc5d00bfbed11f66b1ab89a0c2b6e887240a30b0f6b/MarkupSafe-3.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:70a87b411535ccad5ef2f1df5136506a10775d267e197e4cf531ced10537bd6b", size = 15521, upload-time = "2024-10-18T15:21:12.911Z" }, + { url = "https://files.pythonhosted.org/packages/22/09/d1f21434c97fc42f09d290cbb6350d44eb12f09cc62c9476effdb33a18aa/MarkupSafe-3.0.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:9778bd8ab0a994ebf6f84c2b949e65736d5575320a17ae8984a77fab08db94cf", size = 14274, upload-time = "2024-10-18T15:21:13.777Z" }, + { url = "https://files.pythonhosted.org/packages/6b/b0/18f76bba336fa5aecf79d45dcd6c806c280ec44538b3c13671d49099fdd0/MarkupSafe-3.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:846ade7b71e3536c4e56b386c2a47adf5741d2d8b94ec9dc3e92e5e1ee1e2225", size = 12348, upload-time = "2024-10-18T15:21:14.822Z" }, + { url = "https://files.pythonhosted.org/packages/e0/25/dd5c0f6ac1311e9b40f4af06c78efde0f3b5cbf02502f8ef9501294c425b/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c99d261bd2d5f6b59325c92c73df481e05e57f19837bdca8413b9eac4bd8028", size = 24149, upload-time = "2024-10-18T15:21:15.642Z" }, + { url = "https://files.pythonhosted.org/packages/f3/f0/89e7aadfb3749d0f52234a0c8c7867877876e0a20b60e2188e9850794c17/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e17c96c14e19278594aa4841ec148115f9c7615a47382ecb6b82bd8fea3ab0c8", size = 23118, upload-time = "2024-10-18T15:21:17.133Z" }, + { url = "https://files.pythonhosted.org/packages/d5/da/f2eeb64c723f5e3777bc081da884b414671982008c47dcc1873d81f625b6/MarkupSafe-3.0.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:88416bd1e65dcea10bc7569faacb2c20ce071dd1f87539ca2ab364bf6231393c", size = 22993, upload-time = "2024-10-18T15:21:18.064Z" }, + { url = "https://files.pythonhosted.org/packages/da/0e/1f32af846df486dce7c227fe0f2398dc7e2e51d4a370508281f3c1c5cddc/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:2181e67807fc2fa785d0592dc2d6206c019b9502410671cc905d132a92866557", size = 24178, upload-time = "2024-10-18T15:21:18.859Z" }, + { url = "https://files.pythonhosted.org/packages/c4/f6/bb3ca0532de8086cbff5f06d137064c8410d10779c4c127e0e47d17c0b71/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:52305740fe773d09cffb16f8ed0427942901f00adedac82ec8b67752f58a1b22", size = 23319, upload-time = "2024-10-18T15:21:19.671Z" }, + { url = "https://files.pythonhosted.org/packages/a2/82/8be4c96ffee03c5b4a034e60a31294daf481e12c7c43ab8e34a1453ee48b/MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ad10d3ded218f1039f11a75f8091880239651b52e9bb592ca27de44eed242a48", size = 23352, upload-time = "2024-10-18T15:21:20.971Z" }, + { url = "https://files.pythonhosted.org/packages/51/ae/97827349d3fcffee7e184bdf7f41cd6b88d9919c80f0263ba7acd1bbcb18/MarkupSafe-3.0.2-cp312-cp312-win32.whl", hash = "sha256:0f4ca02bea9a23221c0182836703cbf8930c5e9454bacce27e767509fa286a30", size = 15097, upload-time = "2024-10-18T15:21:22.646Z" }, + { url = "https://files.pythonhosted.org/packages/c1/80/a61f99dc3a936413c3ee4e1eecac96c0da5ed07ad56fd975f1a9da5bc630/MarkupSafe-3.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:8e06879fc22a25ca47312fbe7c8264eb0b662f6db27cb2d3bbbc74b1df4b9b87", size = 15601, upload-time = "2024-10-18T15:21:23.499Z" }, + { url = "https://files.pythonhosted.org/packages/83/0e/67eb10a7ecc77a0c2bbe2b0235765b98d164d81600746914bebada795e97/MarkupSafe-3.0.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ba9527cdd4c926ed0760bc301f6728ef34d841f405abf9d4f959c478421e4efd", size = 14274, upload-time = "2024-10-18T15:21:24.577Z" }, + { url = "https://files.pythonhosted.org/packages/2b/6d/9409f3684d3335375d04e5f05744dfe7e9f120062c9857df4ab490a1031a/MarkupSafe-3.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f8b3d067f2e40fe93e1ccdd6b2e1d16c43140e76f02fb1319a05cf2b79d99430", size = 12352, upload-time = "2024-10-18T15:21:25.382Z" }, + { url = "https://files.pythonhosted.org/packages/d2/f5/6eadfcd3885ea85fe2a7c128315cc1bb7241e1987443d78c8fe712d03091/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:569511d3b58c8791ab4c2e1285575265991e6d8f8700c7be0e88f86cb0672094", size = 24122, upload-time = "2024-10-18T15:21:26.199Z" }, + { url = "https://files.pythonhosted.org/packages/0c/91/96cf928db8236f1bfab6ce15ad070dfdd02ed88261c2afafd4b43575e9e9/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15ab75ef81add55874e7ab7055e9c397312385bd9ced94920f2802310c930396", size = 23085, upload-time = "2024-10-18T15:21:27.029Z" }, + { url = "https://files.pythonhosted.org/packages/c2/cf/c9d56af24d56ea04daae7ac0940232d31d5a8354f2b457c6d856b2057d69/MarkupSafe-3.0.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f3818cb119498c0678015754eba762e0d61e5b52d34c8b13d770f0719f7b1d79", size = 22978, upload-time = "2024-10-18T15:21:27.846Z" }, + { url = "https://files.pythonhosted.org/packages/2a/9f/8619835cd6a711d6272d62abb78c033bda638fdc54c4e7f4272cf1c0962b/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:cdb82a876c47801bb54a690c5ae105a46b392ac6099881cdfb9f6e95e4014c6a", size = 24208, upload-time = "2024-10-18T15:21:28.744Z" }, + { url = "https://files.pythonhosted.org/packages/f9/bf/176950a1792b2cd2102b8ffeb5133e1ed984547b75db47c25a67d3359f77/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:cabc348d87e913db6ab4aa100f01b08f481097838bdddf7c7a84b7575b7309ca", size = 23357, upload-time = "2024-10-18T15:21:29.545Z" }, + { url = "https://files.pythonhosted.org/packages/ce/4f/9a02c1d335caabe5c4efb90e1b6e8ee944aa245c1aaaab8e8a618987d816/MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:444dcda765c8a838eaae23112db52f1efaf750daddb2d9ca300bcae1039adc5c", size = 23344, upload-time = "2024-10-18T15:21:30.366Z" }, + { url = "https://files.pythonhosted.org/packages/ee/55/c271b57db36f748f0e04a759ace9f8f759ccf22b4960c270c78a394f58be/MarkupSafe-3.0.2-cp313-cp313-win32.whl", hash = "sha256:bcf3e58998965654fdaff38e58584d8937aa3096ab5354d493c77d1fdd66d7a1", size = 15101, upload-time = "2024-10-18T15:21:31.207Z" }, + { url = "https://files.pythonhosted.org/packages/29/88/07df22d2dd4df40aba9f3e402e6dc1b8ee86297dddbad4872bd5e7b0094f/MarkupSafe-3.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:e6a2a455bd412959b57a172ce6328d2dd1f01cb2135efda2e4576e8a23fa3b0f", size = 15603, upload-time = "2024-10-18T15:21:32.032Z" }, + { url = "https://files.pythonhosted.org/packages/62/6a/8b89d24db2d32d433dffcd6a8779159da109842434f1dd2f6e71f32f738c/MarkupSafe-3.0.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:b5a6b3ada725cea8a5e634536b1b01c30bcdcd7f9c6fff4151548d5bf6b3a36c", size = 14510, upload-time = "2024-10-18T15:21:33.625Z" }, + { url = "https://files.pythonhosted.org/packages/7a/06/a10f955f70a2e5a9bf78d11a161029d278eeacbd35ef806c3fd17b13060d/MarkupSafe-3.0.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a904af0a6162c73e3edcb969eeeb53a63ceeb5d8cf642fade7d39e7963a22ddb", size = 12486, upload-time = "2024-10-18T15:21:34.611Z" }, + { url = "https://files.pythonhosted.org/packages/34/cf/65d4a571869a1a9078198ca28f39fba5fbb910f952f9dbc5220afff9f5e6/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4aa4e5faecf353ed117801a068ebab7b7e09ffb6e1d5e412dc852e0da018126c", size = 25480, upload-time = "2024-10-18T15:21:35.398Z" }, + { url = "https://files.pythonhosted.org/packages/0c/e3/90e9651924c430b885468b56b3d597cabf6d72be4b24a0acd1fa0e12af67/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0ef13eaeee5b615fb07c9a7dadb38eac06a0608b41570d8ade51c56539e509d", size = 23914, upload-time = "2024-10-18T15:21:36.231Z" }, + { url = "https://files.pythonhosted.org/packages/66/8c/6c7cf61f95d63bb866db39085150df1f2a5bd3335298f14a66b48e92659c/MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d16a81a06776313e817c951135cf7340a3e91e8c1ff2fac444cfd75fffa04afe", size = 23796, upload-time = "2024-10-18T15:21:37.073Z" }, + { url = "https://files.pythonhosted.org/packages/bb/35/cbe9238ec3f47ac9a7c8b3df7a808e7cb50fe149dc7039f5f454b3fba218/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:6381026f158fdb7c72a168278597a5e3a5222e83ea18f543112b2662a9b699c5", size = 25473, upload-time = "2024-10-18T15:21:37.932Z" }, + { url = "https://files.pythonhosted.org/packages/e6/32/7621a4382488aa283cc05e8984a9c219abad3bca087be9ec77e89939ded9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3d79d162e7be8f996986c064d1c7c817f6df3a77fe3d6859f6f9e7be4b8c213a", size = 24114, upload-time = "2024-10-18T15:21:39.799Z" }, + { url = "https://files.pythonhosted.org/packages/0d/80/0985960e4b89922cb5a0bac0ed39c5b96cbc1a536a99f30e8c220a996ed9/MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:131a3c7689c85f5ad20f9f6fb1b866f402c445b220c19fe4308c0b147ccd2ad9", size = 24098, upload-time = "2024-10-18T15:21:40.813Z" }, + { url = "https://files.pythonhosted.org/packages/82/78/fedb03c7d5380df2427038ec8d973587e90561b2d90cd472ce9254cf348b/MarkupSafe-3.0.2-cp313-cp313t-win32.whl", hash = "sha256:ba8062ed2cf21c07a9e295d5b8a2a5ce678b913b45fdf68c32d95d6c1291e0b6", size = 15208, upload-time = "2024-10-18T15:21:41.814Z" }, + { url = "https://files.pythonhosted.org/packages/4f/65/6079a46068dfceaeabb5dcad6d674f5f5c61a6fa5673746f42a9f4c233b3/MarkupSafe-3.0.2-cp313-cp313t-win_amd64.whl", hash = "sha256:e444a31f8db13eb18ada366ab3cf45fd4b31e4db1236a4448f68778c1d1a5a2f", size = 15739, upload-time = "2024-10-18T15:21:42.784Z" }, ] [[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 } +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 }, + { 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 = "more-itertools" version = "10.7.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/ce/a0/834b0cebabbfc7e311f30b46c8188790a37f89fc8d756660346fe5abfd09/more_itertools-10.7.0.tar.gz", hash = "sha256:9fddd5403be01a94b204faadcff459ec3568cf110265d3c54323e1e866ad29d3", size = 127671 } +sdist = { url = "https://files.pythonhosted.org/packages/ce/a0/834b0cebabbfc7e311f30b46c8188790a37f89fc8d756660346fe5abfd09/more_itertools-10.7.0.tar.gz", hash = "sha256:9fddd5403be01a94b204faadcff459ec3568cf110265d3c54323e1e866ad29d3", size = 127671, upload-time = "2025-04-22T14:17:41.838Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2b/9f/7ba6f94fc1e9ac3d2b853fdff3035fb2fa5afbed898c4a72b8a020610594/more_itertools-10.7.0-py3-none-any.whl", hash = "sha256:d43980384673cb07d2f7d2d918c616b30c659c089ee23953f601d6609c67510e", size = 65278 }, + { url = "https://files.pythonhosted.org/packages/2b/9f/7ba6f94fc1e9ac3d2b853fdff3035fb2fa5afbed898c4a72b8a020610594/more_itertools-10.7.0-py3-none-any.whl", hash = "sha256:d43980384673cb07d2f7d2d918c616b30c659c089ee23953f601d6609c67510e", size = 65278, upload-time = "2025-04-22T14:17:40.49Z" }, ] [[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 } +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 }, + { 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 = "nh3" version = "0.3.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/c3/a4/96cff0977357f60f06ec4368c4c7a7a26cccfe7c9fcd54f5378bf0428fd3/nh3-0.3.0.tar.gz", hash = "sha256:d8ba24cb31525492ea71b6aac11a4adac91d828aadeff7c4586541bf5dc34d2f", size = 19655 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/b4/11/340b7a551916a4b2b68c54799d710f86cf3838a4abaad8e74d35360343bb/nh3-0.3.0-cp313-cp313t-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:a537ece1bf513e5a88d8cff8a872e12fe8d0f42ef71dd15a5e7520fecd191bbb", size = 1427992 }, - { url = "https://files.pythonhosted.org/packages/ad/7f/7c6b8358cf1222921747844ab0eef81129e9970b952fcb814df417159fb9/nh3-0.3.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7c915060a2c8131bef6a29f78debc29ba40859b6dbe2362ef9e5fd44f11487c2", size = 798194 }, - { url = "https://files.pythonhosted.org/packages/63/da/c5fd472b700ba37d2df630a9e0d8cc156033551ceb8b4c49cc8a5f606b68/nh3-0.3.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ba0caa8aa184196daa6e574d997a33867d6d10234018012d35f86d46024a2a95", size = 837884 }, - { url = "https://files.pythonhosted.org/packages/4c/3c/cba7b26ccc0ef150c81646478aa32f9c9535234f54845603c838a1dc955c/nh3-0.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:80fe20171c6da69c7978ecba33b638e951b85fb92059259edd285ff108b82a6d", size = 996365 }, - { url = "https://files.pythonhosted.org/packages/f3/ba/59e204d90727c25b253856e456ea61265ca810cda8ee802c35f3fadaab00/nh3-0.3.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:e90883f9f85288f423c77b3f5a6f4486375636f25f793165112679a7b6363b35", size = 1071042 }, - { url = "https://files.pythonhosted.org/packages/10/71/2fb1834c10fab6d9291d62c95192ea2f4c7518bd32ad6c46aab5d095cb87/nh3-0.3.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:0649464ac8eee018644aacbc103874ccbfac80e3035643c3acaab4287e36e7f5", size = 995737 }, - { url = "https://files.pythonhosted.org/packages/33/c1/8f8ccc2492a000b6156dce68a43253fcff8b4ce70ab4216d08f90a2ac998/nh3-0.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:1adeb1062a1c2974bc75b8d1ecb014c5fd4daf2df646bbe2831f7c23659793f9", size = 980552 }, - { url = "https://files.pythonhosted.org/packages/2f/d6/f1c6e091cbe8700401c736c2bc3980c46dca770a2cf6a3b48a175114058e/nh3-0.3.0-cp313-cp313t-win32.whl", hash = "sha256:7275fdffaab10cc5801bf026e3c089d8de40a997afc9e41b981f7ac48c5aa7d5", size = 593618 }, - { url = "https://files.pythonhosted.org/packages/23/1e/80a8c517655dd40bb13363fc4d9e66b2f13245763faab1a20f1df67165a7/nh3-0.3.0-cp313-cp313t-win_amd64.whl", hash = "sha256:423201bbdf3164a9e09aa01e540adbb94c9962cc177d5b1cbb385f5e1e79216e", size = 598948 }, - { url = "https://files.pythonhosted.org/packages/9a/e0/af86d2a974c87a4ba7f19bc3b44a8eaa3da480de264138fec82fe17b340b/nh3-0.3.0-cp313-cp313t-win_arm64.whl", hash = "sha256:16f8670201f7e8e0e05ed1a590eb84bfa51b01a69dd5caf1d3ea57733de6a52f", size = 580479 }, - { url = "https://files.pythonhosted.org/packages/0c/e0/cf1543e798ba86d838952e8be4cb8d18e22999be2a24b112a671f1c04fd6/nh3-0.3.0-cp38-abi3-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:ec6cfdd2e0399cb79ba4dcffb2332b94d9696c52272ff9d48a630c5dca5e325a", size = 1442218 }, - { url = "https://files.pythonhosted.org/packages/5c/86/a96b1453c107b815f9ab8fac5412407c33cc5c7580a4daf57aabeb41b774/nh3-0.3.0-cp38-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ce5e7185599f89b0e391e2f29cc12dc2e206167380cea49b33beda4891be2fe1", size = 823791 }, - { url = "https://files.pythonhosted.org/packages/97/33/11e7273b663839626f714cb68f6eb49899da5a0d9b6bc47b41fe870259c2/nh3-0.3.0-cp38-abi3-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:389d93d59b8214d51c400fb5b07866c2a4f79e4e14b071ad66c92184fec3a392", size = 811143 }, - { url = "https://files.pythonhosted.org/packages/6a/1b/b15bd1ce201a1a610aeb44afd478d55ac018b4475920a3118ffd806e2483/nh3-0.3.0-cp38-abi3-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:e9e6a7e4d38f7e8dda9edd1433af5170c597336c1a74b4693c5cb75ab2b30f2a", size = 1064661 }, - { url = "https://files.pythonhosted.org/packages/8f/14/079670fb2e848c4ba2476c5a7a2d1319826053f4f0368f61fca9bb4227ae/nh3-0.3.0-cp38-abi3-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7852f038a054e0096dac12b8141191e02e93e0b4608c4b993ec7d4ffafea4e49", size = 997061 }, - { url = "https://files.pythonhosted.org/packages/a3/e5/ac7fc565f5d8bce7f979d1afd68e8cb415020d62fa6507133281c7d49f91/nh3-0.3.0-cp38-abi3-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:af5aa8127f62bbf03d68f67a956627b1bd0469703a35b3dad28d0c1195e6c7fb", size = 924761 }, - { url = "https://files.pythonhosted.org/packages/39/2c/6394301428b2017a9d5644af25f487fa557d06bc8a491769accec7524d9a/nh3-0.3.0-cp38-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f416c35efee3e6a6c9ab7716d9e57aa0a49981be915963a82697952cba1353e1", size = 803959 }, - { url = "https://files.pythonhosted.org/packages/4e/9a/344b9f9c4bd1c2413a397f38ee6a3d5db30f1a507d4976e046226f12b297/nh3-0.3.0-cp38-abi3-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:37d3003d98dedca6cd762bf88f2e70b67f05100f6b949ffe540e189cc06887f9", size = 844073 }, - { url = "https://files.pythonhosted.org/packages/66/3f/cd37f76c8ca277b02a84aa20d7bd60fbac85b4e2cbdae77cb759b22de58b/nh3-0.3.0-cp38-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:634e34e6162e0408e14fb61d5e69dbaea32f59e847cfcfa41b66100a6b796f62", size = 1000680 }, - { url = "https://files.pythonhosted.org/packages/ee/db/7aa11b44bae4e7474feb1201d8dee04fabe5651c7cb51409ebda94a4ed67/nh3-0.3.0-cp38-abi3-musllinux_1_2_armv7l.whl", hash = "sha256:b0612ccf5de8a480cf08f047b08f9d3fecc12e63d2ee91769cb19d7290614c23", size = 1076613 }, - { url = "https://files.pythonhosted.org/packages/97/03/03f79f7e5178eb1ad5083af84faff471e866801beb980cc72943a4397368/nh3-0.3.0-cp38-abi3-musllinux_1_2_i686.whl", hash = "sha256:c7a32a7f0d89f7d30cb8f4a84bdbd56d1eb88b78a2434534f62c71dac538c450", size = 1001418 }, - { url = "https://files.pythonhosted.org/packages/ce/55/1974bcc16884a397ee699cebd3914e1f59be64ab305533347ca2d983756f/nh3-0.3.0-cp38-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:3f1b4f8a264a0c86ea01da0d0c390fe295ea0bcacc52c2103aca286f6884f518", size = 986499 }, - { url = "https://files.pythonhosted.org/packages/c9/50/76936ec021fe1f3270c03278b8af5f2079038116b5d0bfe8538ffe699d69/nh3-0.3.0-cp38-abi3-win32.whl", hash = "sha256:6d68fa277b4a3cf04e5c4b84dd0c6149ff7d56c12b3e3fab304c525b850f613d", size = 599000 }, - { url = "https://files.pythonhosted.org/packages/8c/ae/324b165d904dc1672eee5f5661c0a68d4bab5b59fbb07afb6d8d19a30b45/nh3-0.3.0-cp38-abi3-win_amd64.whl", hash = "sha256:bae63772408fd63ad836ec569a7c8f444dd32863d0c67f6e0b25ebbd606afa95", size = 604530 }, - { url = "https://files.pythonhosted.org/packages/5b/76/3165e84e5266d146d967a6cc784ff2fbf6ddd00985a55ec006b72bc39d5d/nh3-0.3.0-cp38-abi3-win_arm64.whl", hash = "sha256:d97d3efd61404af7e5721a0e74d81cdbfc6e5f97e11e731bb6d090e30a7b62b2", size = 585971 }, +sdist = { url = "https://files.pythonhosted.org/packages/c3/a4/96cff0977357f60f06ec4368c4c7a7a26cccfe7c9fcd54f5378bf0428fd3/nh3-0.3.0.tar.gz", hash = "sha256:d8ba24cb31525492ea71b6aac11a4adac91d828aadeff7c4586541bf5dc34d2f", size = 19655, upload-time = "2025-07-17T14:43:37.05Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b4/11/340b7a551916a4b2b68c54799d710f86cf3838a4abaad8e74d35360343bb/nh3-0.3.0-cp313-cp313t-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:a537ece1bf513e5a88d8cff8a872e12fe8d0f42ef71dd15a5e7520fecd191bbb", size = 1427992, upload-time = "2025-07-17T14:43:06.848Z" }, + { url = "https://files.pythonhosted.org/packages/ad/7f/7c6b8358cf1222921747844ab0eef81129e9970b952fcb814df417159fb9/nh3-0.3.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7c915060a2c8131bef6a29f78debc29ba40859b6dbe2362ef9e5fd44f11487c2", size = 798194, upload-time = "2025-07-17T14:43:08.263Z" }, + { url = "https://files.pythonhosted.org/packages/63/da/c5fd472b700ba37d2df630a9e0d8cc156033551ceb8b4c49cc8a5f606b68/nh3-0.3.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ba0caa8aa184196daa6e574d997a33867d6d10234018012d35f86d46024a2a95", size = 837884, upload-time = "2025-07-17T14:43:09.233Z" }, + { url = "https://files.pythonhosted.org/packages/4c/3c/cba7b26ccc0ef150c81646478aa32f9c9535234f54845603c838a1dc955c/nh3-0.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:80fe20171c6da69c7978ecba33b638e951b85fb92059259edd285ff108b82a6d", size = 996365, upload-time = "2025-07-17T14:43:10.243Z" }, + { url = "https://files.pythonhosted.org/packages/f3/ba/59e204d90727c25b253856e456ea61265ca810cda8ee802c35f3fadaab00/nh3-0.3.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:e90883f9f85288f423c77b3f5a6f4486375636f25f793165112679a7b6363b35", size = 1071042, upload-time = "2025-07-17T14:43:11.57Z" }, + { url = "https://files.pythonhosted.org/packages/10/71/2fb1834c10fab6d9291d62c95192ea2f4c7518bd32ad6c46aab5d095cb87/nh3-0.3.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:0649464ac8eee018644aacbc103874ccbfac80e3035643c3acaab4287e36e7f5", size = 995737, upload-time = "2025-07-17T14:43:12.659Z" }, + { url = "https://files.pythonhosted.org/packages/33/c1/8f8ccc2492a000b6156dce68a43253fcff8b4ce70ab4216d08f90a2ac998/nh3-0.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:1adeb1062a1c2974bc75b8d1ecb014c5fd4daf2df646bbe2831f7c23659793f9", size = 980552, upload-time = "2025-07-17T14:43:13.763Z" }, + { url = "https://files.pythonhosted.org/packages/2f/d6/f1c6e091cbe8700401c736c2bc3980c46dca770a2cf6a3b48a175114058e/nh3-0.3.0-cp313-cp313t-win32.whl", hash = "sha256:7275fdffaab10cc5801bf026e3c089d8de40a997afc9e41b981f7ac48c5aa7d5", size = 593618, upload-time = "2025-07-17T14:43:15.098Z" }, + { url = "https://files.pythonhosted.org/packages/23/1e/80a8c517655dd40bb13363fc4d9e66b2f13245763faab1a20f1df67165a7/nh3-0.3.0-cp313-cp313t-win_amd64.whl", hash = "sha256:423201bbdf3164a9e09aa01e540adbb94c9962cc177d5b1cbb385f5e1e79216e", size = 598948, upload-time = "2025-07-17T14:43:16.064Z" }, + { url = "https://files.pythonhosted.org/packages/9a/e0/af86d2a974c87a4ba7f19bc3b44a8eaa3da480de264138fec82fe17b340b/nh3-0.3.0-cp313-cp313t-win_arm64.whl", hash = "sha256:16f8670201f7e8e0e05ed1a590eb84bfa51b01a69dd5caf1d3ea57733de6a52f", size = 580479, upload-time = "2025-07-17T14:43:17.038Z" }, + { url = "https://files.pythonhosted.org/packages/0c/e0/cf1543e798ba86d838952e8be4cb8d18e22999be2a24b112a671f1c04fd6/nh3-0.3.0-cp38-abi3-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:ec6cfdd2e0399cb79ba4dcffb2332b94d9696c52272ff9d48a630c5dca5e325a", size = 1442218, upload-time = "2025-07-17T14:43:18.087Z" }, + { url = "https://files.pythonhosted.org/packages/5c/86/a96b1453c107b815f9ab8fac5412407c33cc5c7580a4daf57aabeb41b774/nh3-0.3.0-cp38-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ce5e7185599f89b0e391e2f29cc12dc2e206167380cea49b33beda4891be2fe1", size = 823791, upload-time = "2025-07-17T14:43:19.721Z" }, + { url = "https://files.pythonhosted.org/packages/97/33/11e7273b663839626f714cb68f6eb49899da5a0d9b6bc47b41fe870259c2/nh3-0.3.0-cp38-abi3-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:389d93d59b8214d51c400fb5b07866c2a4f79e4e14b071ad66c92184fec3a392", size = 811143, upload-time = "2025-07-17T14:43:20.779Z" }, + { url = "https://files.pythonhosted.org/packages/6a/1b/b15bd1ce201a1a610aeb44afd478d55ac018b4475920a3118ffd806e2483/nh3-0.3.0-cp38-abi3-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:e9e6a7e4d38f7e8dda9edd1433af5170c597336c1a74b4693c5cb75ab2b30f2a", size = 1064661, upload-time = "2025-07-17T14:43:21.839Z" }, + { url = "https://files.pythonhosted.org/packages/8f/14/079670fb2e848c4ba2476c5a7a2d1319826053f4f0368f61fca9bb4227ae/nh3-0.3.0-cp38-abi3-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7852f038a054e0096dac12b8141191e02e93e0b4608c4b993ec7d4ffafea4e49", size = 997061, upload-time = "2025-07-17T14:43:23.179Z" }, + { url = "https://files.pythonhosted.org/packages/a3/e5/ac7fc565f5d8bce7f979d1afd68e8cb415020d62fa6507133281c7d49f91/nh3-0.3.0-cp38-abi3-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:af5aa8127f62bbf03d68f67a956627b1bd0469703a35b3dad28d0c1195e6c7fb", size = 924761, upload-time = "2025-07-17T14:43:24.23Z" }, + { url = "https://files.pythonhosted.org/packages/39/2c/6394301428b2017a9d5644af25f487fa557d06bc8a491769accec7524d9a/nh3-0.3.0-cp38-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f416c35efee3e6a6c9ab7716d9e57aa0a49981be915963a82697952cba1353e1", size = 803959, upload-time = "2025-07-17T14:43:26.377Z" }, + { url = "https://files.pythonhosted.org/packages/4e/9a/344b9f9c4bd1c2413a397f38ee6a3d5db30f1a507d4976e046226f12b297/nh3-0.3.0-cp38-abi3-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:37d3003d98dedca6cd762bf88f2e70b67f05100f6b949ffe540e189cc06887f9", size = 844073, upload-time = "2025-07-17T14:43:27.375Z" }, + { url = "https://files.pythonhosted.org/packages/66/3f/cd37f76c8ca277b02a84aa20d7bd60fbac85b4e2cbdae77cb759b22de58b/nh3-0.3.0-cp38-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:634e34e6162e0408e14fb61d5e69dbaea32f59e847cfcfa41b66100a6b796f62", size = 1000680, upload-time = "2025-07-17T14:43:28.452Z" }, + { url = "https://files.pythonhosted.org/packages/ee/db/7aa11b44bae4e7474feb1201d8dee04fabe5651c7cb51409ebda94a4ed67/nh3-0.3.0-cp38-abi3-musllinux_1_2_armv7l.whl", hash = "sha256:b0612ccf5de8a480cf08f047b08f9d3fecc12e63d2ee91769cb19d7290614c23", size = 1076613, upload-time = "2025-07-17T14:43:30.031Z" }, + { url = "https://files.pythonhosted.org/packages/97/03/03f79f7e5178eb1ad5083af84faff471e866801beb980cc72943a4397368/nh3-0.3.0-cp38-abi3-musllinux_1_2_i686.whl", hash = "sha256:c7a32a7f0d89f7d30cb8f4a84bdbd56d1eb88b78a2434534f62c71dac538c450", size = 1001418, upload-time = "2025-07-17T14:43:31.429Z" }, + { url = "https://files.pythonhosted.org/packages/ce/55/1974bcc16884a397ee699cebd3914e1f59be64ab305533347ca2d983756f/nh3-0.3.0-cp38-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:3f1b4f8a264a0c86ea01da0d0c390fe295ea0bcacc52c2103aca286f6884f518", size = 986499, upload-time = "2025-07-17T14:43:32.459Z" }, + { url = "https://files.pythonhosted.org/packages/c9/50/76936ec021fe1f3270c03278b8af5f2079038116b5d0bfe8538ffe699d69/nh3-0.3.0-cp38-abi3-win32.whl", hash = "sha256:6d68fa277b4a3cf04e5c4b84dd0c6149ff7d56c12b3e3fab304c525b850f613d", size = 599000, upload-time = "2025-07-17T14:43:33.852Z" }, + { url = "https://files.pythonhosted.org/packages/8c/ae/324b165d904dc1672eee5f5661c0a68d4bab5b59fbb07afb6d8d19a30b45/nh3-0.3.0-cp38-abi3-win_amd64.whl", hash = "sha256:bae63772408fd63ad836ec569a7c8f444dd32863d0c67f6e0b25ebbd606afa95", size = 604530, upload-time = "2025-07-17T14:43:34.95Z" }, + { url = "https://files.pythonhosted.org/packages/5b/76/3165e84e5266d146d967a6cc784ff2fbf6ddd00985a55ec006b72bc39d5d/nh3-0.3.0-cp38-abi3-win_arm64.whl", hash = "sha256:d97d3efd61404af7e5721a0e74d81cdbfc6e5f97e11e731bb6d090e30a7b62b2", size = 585971, upload-time = "2025-07-17T14:43:35.936Z" }, ] [[package]] name = "oauthlib" version = "3.3.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/0b/5f/19930f824ffeb0ad4372da4812c50edbd1434f678c90c2733e1188edfc63/oauthlib-3.3.1.tar.gz", hash = "sha256:0f0f8aa759826a193cf66c12ea1af1637f87b9b4622d46e866952bb022e538c9", size = 185918 } +sdist = { url = "https://files.pythonhosted.org/packages/0b/5f/19930f824ffeb0ad4372da4812c50edbd1434f678c90c2733e1188edfc63/oauthlib-3.3.1.tar.gz", hash = "sha256:0f0f8aa759826a193cf66c12ea1af1637f87b9b4622d46e866952bb022e538c9", size = 185918, upload-time = "2025-06-19T22:48:08.269Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/be/9c/92789c596b8df838baa98fa71844d84283302f7604ed565dafe5a6b5041a/oauthlib-3.3.1-py3-none-any.whl", hash = "sha256:88119c938d2b8fb88561af5f6ee0eec8cc8d552b7bb1f712743136eb7523b7a1", size = 160065 }, + { url = "https://files.pythonhosted.org/packages/be/9c/92789c596b8df838baa98fa71844d84283302f7604ed565dafe5a6b5041a/oauthlib-3.3.1-py3-none-any.whl", hash = "sha256:88119c938d2b8fb88561af5f6ee0eec8cc8d552b7bb1f712743136eb7523b7a1", size = 160065, upload-time = "2025-06-19T22:48:06.508Z" }, ] [[package]] name = "packaging" version = "25.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a1/d4/1fc4078c65507b51b96ca8f8c3ba19e6a61c8253c72794544580a7b6c24d/packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f", size = 165727 } +sdist = { url = "https://files.pythonhosted.org/packages/a1/d4/1fc4078c65507b51b96ca8f8c3ba19e6a61c8253c72794544580a7b6c24d/packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f", size = 165727, upload-time = "2025-04-19T11:48:59.673Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/20/12/38679034af332785aac8774540895e234f4d07f7545804097de4b666afd8/packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484", size = 66469 }, + { url = "https://files.pythonhosted.org/packages/20/12/38679034af332785aac8774540895e234f4d07f7545804097de4b666afd8/packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484", size = 66469, upload-time = "2025-04-19T11:48:57.875Z" }, ] [[package]] name = "pathspec" version = "0.12.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/ca/bc/f35b8446f4531a7cb215605d100cd88b7ac6f44ab3fc94870c120ab3adbf/pathspec-0.12.1.tar.gz", hash = "sha256:a482d51503a1ab33b1c67a6c3813a26953dbdc71c31dacaef9a838c4e29f5712", size = 51043 } +sdist = { url = "https://files.pythonhosted.org/packages/ca/bc/f35b8446f4531a7cb215605d100cd88b7ac6f44ab3fc94870c120ab3adbf/pathspec-0.12.1.tar.gz", hash = "sha256:a482d51503a1ab33b1c67a6c3813a26953dbdc71c31dacaef9a838c4e29f5712", size = 51043, upload-time = "2023-12-10T22:30:45Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/cc/20/ff623b09d963f88bfde16306a54e12ee5ea43e9b597108672ff3a408aad6/pathspec-0.12.1-py3-none-any.whl", hash = "sha256:a0d503e138a4c123b27490a4f7beda6a01c6f288df0e4a8b79c7eb0dc7b4cc08", size = 31191 }, + { url = "https://files.pythonhosted.org/packages/cc/20/ff623b09d963f88bfde16306a54e12ee5ea43e9b597108672ff3a408aad6/pathspec-0.12.1-py3-none-any.whl", hash = "sha256:a0d503e138a4c123b27490a4f7beda6a01c6f288df0e4a8b79c7eb0dc7b4cc08", size = 31191, upload-time = "2023-12-10T22:30:43.14Z" }, ] [[package]] @@ -989,62 +989,62 @@ 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 } +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 }, + { 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 = "pip" version = "25.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/20/16/650289cd3f43d5a2fadfd98c68bd1e1e7f2550a1a5326768cddfbcedb2c5/pip-25.2.tar.gz", hash = "sha256:578283f006390f85bb6282dffb876454593d637f5d1be494b5202ce4877e71f2", size = 1840021 } +sdist = { url = "https://files.pythonhosted.org/packages/20/16/650289cd3f43d5a2fadfd98c68bd1e1e7f2550a1a5326768cddfbcedb2c5/pip-25.2.tar.gz", hash = "sha256:578283f006390f85bb6282dffb876454593d637f5d1be494b5202ce4877e71f2", size = 1840021, upload-time = "2025-07-30T21:50:15.401Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b7/3f/945ef7ab14dc4f9d7f40288d2df998d1837ee0888ec3659c813487572faa/pip-25.2-py3-none-any.whl", hash = "sha256:6d67a2b4e7f14d8b31b8b52648866fa717f45a1eb70e83002f4331d07e953717", size = 1752557 }, + { url = "https://files.pythonhosted.org/packages/b7/3f/945ef7ab14dc4f9d7f40288d2df998d1837ee0888ec3659c813487572faa/pip-25.2-py3-none-any.whl", hash = "sha256:6d67a2b4e7f14d8b31b8b52648866fa717f45a1eb70e83002f4331d07e953717", size = 1752557, upload-time = "2025-07-30T21:50:13.323Z" }, ] [[package]] name = "platformdirs" version = "4.3.8" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/fe/8b/3c73abc9c759ecd3f1f7ceff6685840859e8070c4d947c93fae71f6a0bf2/platformdirs-4.3.8.tar.gz", hash = "sha256:3d512d96e16bcb959a814c9f348431070822a6496326a4be0911c40b5a74c2bc", size = 21362 } +sdist = { url = "https://files.pythonhosted.org/packages/fe/8b/3c73abc9c759ecd3f1f7ceff6685840859e8070c4d947c93fae71f6a0bf2/platformdirs-4.3.8.tar.gz", hash = "sha256:3d512d96e16bcb959a814c9f348431070822a6496326a4be0911c40b5a74c2bc", size = 21362, upload-time = "2025-05-07T22:47:42.121Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/fe/39/979e8e21520d4e47a0bbe349e2713c0aac6f3d853d0e5b34d76206c439aa/platformdirs-4.3.8-py3-none-any.whl", hash = "sha256:ff7059bb7eb1179e2685604f4aaf157cfd9535242bd23742eadc3c13542139b4", size = 18567 }, + { url = "https://files.pythonhosted.org/packages/fe/39/979e8e21520d4e47a0bbe349e2713c0aac6f3d853d0e5b34d76206c439aa/platformdirs-4.3.8-py3-none-any.whl", hash = "sha256:ff7059bb7eb1179e2685604f4aaf157cfd9535242bd23742eadc3c13542139b4", size = 18567, upload-time = "2025-05-07T22:47:40.376Z" }, ] [[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 } +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 }, + { 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 = "prek" -version = "0.2.1" -source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/93/0a/b2dabb829aebc4f98a10b8f5fac0e6ca4746463d5183622457e5727f5705/prek-0.2.1.tar.gz", hash = "sha256:60543afbf72ad9ce27a5fc4301179f75f473dc4cc221ff53b2bad18ff6eac998", size = 2919831 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/50/f2/17fdccf51bef753a2ef50e936363779ff20a528ae4e197d9c9e84be5bc53/prek-0.2.1-py3-none-linux_armv6l.whl", hash = "sha256:bdd0e71ab6a63a9b81c268539ec249bfe1cd2b50fc39823727b256b0ba67d342", size = 4320246 }, - { url = "https://files.pythonhosted.org/packages/5e/fe/9a3b119ef905c8c52c17cfce1a4230d8f2f9ddfe6c0815ae52b7d616a038/prek-0.2.1-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:99c1611aea27c75cb76d23f7ca68440478b4372b2184e4c383af06a3ea9e8b8e", size = 4433105 }, - { url = "https://files.pythonhosted.org/packages/eb/93/a3cdfcd934dd2e661a2012efbed0e5ae8b2889ef313bd42f811826b5d259/prek-0.2.1-py3-none-macosx_11_0_arm64.whl", hash = "sha256:46ec9a2dc2f536ddc3381f9bc9461afbb48b767fe0159687b9a8302a0d910f9c", size = 4135455 }, - { url = "https://files.pythonhosted.org/packages/5d/d8/458d5dabe5b8671bd6054f46d8952f07b69be755d98370dc3fe80c797087/prek-0.2.1-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:dd97d0f8d9ec3f1749d3b1c8c3310d2d946976d1ad58e54b1dce3e177b09cbb7", size = 4301081 }, - { url = "https://files.pythonhosted.org/packages/e3/32/100086408852c008e0b5e44c6fe93b5f00cc968e072029f02c7f71375350/prek-0.2.1-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:38c22c63bd18a29b27d4c56804fe10e5ae7e2a2b9d1ec2bcdbaebda9d80b2f86", size = 4256669 }, - { url = "https://files.pythonhosted.org/packages/00/ad/dadcb1a781db3c0feed31ab8c299494fe31ef6d5cd4de4929dfe381dafb6/prek-0.2.1-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e198ddf56591cbe455ffa2106caa6aa8f0df11348a328a68e46e9a0e877f80f6", size = 4553470 }, - { url = "https://files.pythonhosted.org/packages/c4/a1/58ee7dc741ddc49816edf9ba5a0bf4cefa9d1ccd2fcdcf988d525773bdfc/prek-0.2.1-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:5be95c28c65249576e43f7894869f09db7ed62602b70b456c777881304abebf5", size = 4979408 }, - { url = "https://files.pythonhosted.org/packages/b0/96/178ebf0523c47e43263e1e0f1f719d0f0b6fe8932876243eb1c9584457e7/prek-0.2.1-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0355c6c67afe7aa376685cb5747a45e3ba9bcc091d82547ec3725ceafbe139da", size = 4913040 }, - { url = "https://files.pythonhosted.org/packages/da/3f/773f1cab7284b0dd4a2f26df68dadc8b3c5a4ff5eaa6f3baec9e0fabeac9/prek-0.2.1-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b784eb070ac3cfea9598f8590f8aafcdae5a871a9bcfd88535c6f6f71f315252", size = 5031851 }, - { url = "https://files.pythonhosted.org/packages/da/bb/c5205b9e8561ca8e3925fb0072dc81c5a6ab57863e216aacabafb7bee40d/prek-0.2.1-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:300ace6901a78a203030e76750d62cff1bdc9d64b08a998029a160a2435a5575", size = 4644291 }, - { url = "https://files.pythonhosted.org/packages/b1/d5/240376457cafc8a7e97ef72e2bbebd1c8d97dab2577c13a8b0485a8c4b49/prek-0.2.1-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:0b214b14d9c1cb4a3324206486cd2ea19e0c960bd2770785c49dbdaa2476a292", size = 4336230 }, - { url = "https://files.pythonhosted.org/packages/1d/70/a502fefb622186ce1d0f815f516fd368614fe5ecc596d0759d80e50da7d3/prek-0.2.1-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:a087ad088e08633469fe5b35411b6fcbc7c1bc78163f3fe2105142550917b581", size = 4424328 }, - { url = "https://files.pythonhosted.org/packages/97/d1/815eebdf760b8cdc129bd9818fd88b5ef91bddef4b1ce1dd1bae9aefbcb4/prek-0.2.1-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:e3b9f592af89d95b6a189f90f9f5c694086d340c79725ad165ae526a3be0be49", size = 4235667 }, - { url = "https://files.pythonhosted.org/packages/cc/6d/c86fceb0d2c4070a9218f8cb2b82b07cc2118acc910fa066ef8a78d1d15b/prek-0.2.1-py3-none-musllinux_1_1_i686.whl", hash = "sha256:194e7d898742f0e6565049f8e907e1237e5921b09c4c39ea572176e06296a745", size = 4449681 }, - { url = "https://files.pythonhosted.org/packages/97/a6/73678720f9f5a03288bac0ae7dc04fbe96fb01c4b31404480154a0776cc6/prek-0.2.1-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:483aa6f021874d887553cda5d651d989b2fa38f2334faffd53438a46999fc7ad", size = 4717583 }, - { url = "https://files.pythonhosted.org/packages/a2/1c/5a401e3c8db696f24bfa599022c743797af7a26bb1d5a8f79c5b0270afab/prek-0.2.1-py3-none-win32.whl", hash = "sha256:fbe488b478b47d9a7134d145b6564b90bfa1a1f7fec6bc7d7945aaa9a85f80cb", size = 4168471 }, - { url = "https://files.pythonhosted.org/packages/1d/70/cf844f271171e629dc29a496130317333867c6bcc02cc357eb80969ab328/prek-0.2.1-py3-none-win_amd64.whl", hash = "sha256:f066681ffb0f4de4c3a74dfb56cf417751ee2df2c38e2ed6af78d5e15262aa5d", size = 4719355 }, - { url = "https://files.pythonhosted.org/packages/20/63/1625310edb28937719eedfffcbd6e4728dd4d27f777a3f72fdf8c445976b/prek-0.2.1-py3-none-win_arm64.whl", hash = "sha256:97c8367e226930600dfad28405f091a0bc88cc601bfa66ace42ce1cd38e3ee92", size = 4414447 }, +version = "0.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a4/8e/ff52d55d27d3756e63f2b9e5b4c4435f7c7f485044df9bd874be01d4bac9/prek-0.2.3.tar.gz", hash = "sha256:a0df9d89618ea8060e766ec21f67bf6c0fac4f320fcbf3073919630b17494996", size = 3007716, upload-time = "2025-09-29T08:59:02.637Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/08/2b/bd188d222d55bd8c63c0bbf736f361b79559457b51553fe7d90ff9950839/prek-0.2.3-py3-none-linux_armv6l.whl", hash = "sha256:216c06989e421f79bf5a9eb3df1c470878438fac1bc0a636e03fc4d614bf219e", size = 4364783, upload-time = "2025-09-29T08:58:36.087Z" }, + { url = "https://files.pythonhosted.org/packages/c5/a4/24c2dea15242254e3b187f27f419039a88efb56e2d86658b7d2207dba637/prek-0.2.3-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:37467f2752ce0d4ca6451970d6863211db0ce3f390c74e91db790aad51357eb8", size = 4465162, upload-time = "2025-09-29T08:58:38.138Z" }, + { url = "https://files.pythonhosted.org/packages/f8/2f/1734a1ae08405ba303a31961b9f80b37d0d0ed6e61bb0df9a2ef4ef5f728/prek-0.2.3-py3-none-macosx_11_0_arm64.whl", hash = "sha256:2d24575deb40486a1f08799b1e5e17f3be685cbfe4b5071eb93911a9c2728841", size = 4161043, upload-time = "2025-09-29T08:58:39.273Z" }, + { url = "https://files.pythonhosted.org/packages/d3/c3/b26d307449d805bf8d27d3659f7257395f321399dfea9971a2db27f4e5f8/prek-0.2.3-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:3431ebc218d7a7fc0f708c9eec4027949a081870f4f9fba3a7d65c706db87b72", size = 4343619, upload-time = "2025-09-29T08:58:40.43Z" }, + { url = "https://files.pythonhosted.org/packages/6a/a2/d1850fb04ae63108e896c3f6e822ca70b76616b9aae4187e7dfb33c83588/prek-0.2.3-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:4520c1e0827775af9112787cf3b005a3deee6baeb5d6ebd71004a20b9a7ea73b", size = 4297871, upload-time = "2025-09-29T08:58:41.782Z" }, + { url = "https://files.pythonhosted.org/packages/86/a3/028aae3149f69441932ab0bb154068481bd77283502309880326b979480c/prek-0.2.3-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c8eab4a8d71b978f35a73b7a0e074bbe88c8982e2446a05bfc09fcc046b9a2c2", size = 4581855, upload-time = "2025-09-29T08:58:43.367Z" }, + { url = "https://files.pythonhosted.org/packages/5b/8e/aa4bd8ab2c4f365d02b963df2c4c1c1f6842b623f97c4411da25b2e6c880/prek-0.2.3-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:e5ba2b767f1ab011a41592dbc2b41ab9c4641e39b763ea0a0519e1d7b83f79ee", size = 5010892, upload-time = "2025-09-29T08:58:45.061Z" }, + { url = "https://files.pythonhosted.org/packages/81/eb/7f63d8b30fdecbf2722165406a92d56a819d824c0a2c51e308dcbecbbfc2/prek-0.2.3-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e4afc3876dc812a55dfe7d48c0878bdfadc035697292dbfd6a6bd819d845721b", size = 4938755, upload-time = "2025-09-29T08:58:46.682Z" }, + { url = "https://files.pythonhosted.org/packages/26/9d/0cc72bb823a078638b50dcca674e8eaa7bbd59dca5ee8f8d5311ec4295a5/prek-0.2.3-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2de997b640350a4653c267e6e6e681a0902ab436a8a7e659ab1bced213249f79", size = 5061621, upload-time = "2025-09-29T08:58:48.266Z" }, + { url = "https://files.pythonhosted.org/packages/e5/01/19c1bf227879cee63f8d0644d72728bce43ed96ad3ea355b74cdad77a2d1/prek-0.2.3-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:030a41c816d2558326f3ad9adab35e412280fc3183a81a23f450b86b9525aa1d", size = 4646050, upload-time = "2025-09-29T08:58:49.451Z" }, + { url = "https://files.pythonhosted.org/packages/a4/5c/aaa792519e01c7246a41c6a8983b2aa5d52d9b8c7d3d2a54bd0b528ce204/prek-0.2.3-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:7f567b7f2aab8b7dc09e23bab377df69f172b7ccc630bd98fadd103f04878a0e", size = 4356984, upload-time = "2025-09-29T08:58:51.069Z" }, + { url = "https://files.pythonhosted.org/packages/d4/36/6fe6aecc7302cc870e77d3c8084c62d3db8d3208d0c43aeb6e04de65fa78/prek-0.2.3-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:1193882d0e1fb21b757ccc53a34435d229a2f38ba441edd2c66289532b23d96a", size = 4454158, upload-time = "2025-09-29T08:58:52.628Z" }, + { url = "https://files.pythonhosted.org/packages/0c/d0/2c1a89a10f8fb08f7eb37bed360b25fef9db4b86a99e8181e77bffab0885/prek-0.2.3-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:b7a08dacd23791392e9807f4f42631aa1ac53b5907256e4c94af416833b23d00", size = 4276973, upload-time = "2025-09-29T08:58:54.472Z" }, + { url = "https://files.pythonhosted.org/packages/bf/a8/87e088e97badd0a5c9b79c6837a442a17aff9914d70425ef44ee0819c436/prek-0.2.3-py3-none-musllinux_1_1_i686.whl", hash = "sha256:466eb9ff44575c95b7442751ea86c2f0e9c8c188e7cb79a83134c3a768631c20", size = 4474593, upload-time = "2025-09-29T08:58:55.93Z" }, + { url = "https://files.pythonhosted.org/packages/62/9c/5a844812f37f7fec3087072fa1dac3da867ba29f71d2cab8df8de396a32e/prek-0.2.3-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:4f6f90c5adda349110a9ff6ca50cda8200d9fb10a6891ca9c89f179cb789c957", size = 4751816, upload-time = "2025-09-29T08:58:57.447Z" }, + { url = "https://files.pythonhosted.org/packages/1a/23/354fc3934cf09bc0e5d8fa3c52d40c5b686bf2b3faa314b9b6e8dd72d7f7/prek-0.2.3-py3-none-win32.whl", hash = "sha256:c90e15f8617a956a9d2b0c612783eec585a355da685b8f44d338af62ba667c55", size = 4183840, upload-time = "2025-09-29T08:58:58.636Z" }, + { url = "https://files.pythonhosted.org/packages/4a/07/8a285a062d9d1cf16bbafd1d3782e273e4c2863d521ad84013a1af7d746d/prek-0.2.3-py3-none-win_amd64.whl", hash = "sha256:21cb38ae352772477474cb4c3cd9e9056a43ba7779e634bc23826b6dd01941f5", size = 4748429, upload-time = "2025-09-29T08:58:59.924Z" }, + { url = "https://files.pythonhosted.org/packages/da/bd/916ccaee27bb3a9b018ad845da25d79594922085df77bcdef842379ad99a/prek-0.2.3-py3-none-win_arm64.whl", hash = "sha256:5c8bbdc6f4313d989327407a516b59e27624ff16c75f939c497c9cacf6e4fbba", size = 4436318, upload-time = "2025-09-29T08:59:01.418Z" }, ] [[package]] @@ -1054,56 +1054,56 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "protobuf" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/f4/ac/87285f15f7cce6d4a008f33f1757fb5a13611ea8914eb58c3d0d26243468/proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012", size = 56142 } +sdist = { url = "https://files.pythonhosted.org/packages/f4/ac/87285f15f7cce6d4a008f33f1757fb5a13611ea8914eb58c3d0d26243468/proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012", size = 56142, upload-time = "2025-03-10T15:54:38.843Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/4e/6d/280c4c2ce28b1593a19ad5239c8b826871fc6ec275c21afc8e1820108039/proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66", size = 50163 }, + { url = "https://files.pythonhosted.org/packages/4e/6d/280c4c2ce28b1593a19ad5239c8b826871fc6ec275c21afc8e1820108039/proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66", size = 50163, upload-time = "2025-03-10T15:54:37.335Z" }, ] [[package]] name = "protobuf" version = "6.32.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/c0/df/fb4a8eeea482eca989b51cffd274aac2ee24e825f0bf3cbce5281fa1567b/protobuf-6.32.0.tar.gz", hash = "sha256:a81439049127067fc49ec1d36e25c6ee1d1a2b7be930675f919258d03c04e7d2", size = 440614 } +sdist = { url = "https://files.pythonhosted.org/packages/c0/df/fb4a8eeea482eca989b51cffd274aac2ee24e825f0bf3cbce5281fa1567b/protobuf-6.32.0.tar.gz", hash = "sha256:a81439049127067fc49ec1d36e25c6ee1d1a2b7be930675f919258d03c04e7d2", size = 440614, upload-time = "2025-08-14T21:21:25.015Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/33/18/df8c87da2e47f4f1dcc5153a81cd6bca4e429803f4069a299e236e4dd510/protobuf-6.32.0-cp310-abi3-win32.whl", hash = "sha256:84f9e3c1ff6fb0308dbacb0950d8aa90694b0d0ee68e75719cb044b7078fe741", size = 424409 }, - { url = "https://files.pythonhosted.org/packages/e1/59/0a820b7310f8139bd8d5a9388e6a38e1786d179d6f33998448609296c229/protobuf-6.32.0-cp310-abi3-win_amd64.whl", hash = "sha256:a8bdbb2f009cfc22a36d031f22a625a38b615b5e19e558a7b756b3279723e68e", size = 435735 }, - { url = "https://files.pythonhosted.org/packages/cc/5b/0d421533c59c789e9c9894683efac582c06246bf24bb26b753b149bd88e4/protobuf-6.32.0-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:d52691e5bee6c860fff9a1c86ad26a13afbeb4b168cd4445c922b7e2cf85aaf0", size = 426449 }, - { url = "https://files.pythonhosted.org/packages/ec/7b/607764ebe6c7a23dcee06e054fd1de3d5841b7648a90fd6def9a3bb58c5e/protobuf-6.32.0-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:501fe6372fd1c8ea2a30b4d9be8f87955a64d6be9c88a973996cef5ef6f0abf1", size = 322869 }, - { url = "https://files.pythonhosted.org/packages/40/01/2e730bd1c25392fc32e3268e02446f0d77cb51a2c3a8486b1798e34d5805/protobuf-6.32.0-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:75a2aab2bd1aeb1f5dc7c5f33bcb11d82ea8c055c9becbb41c26a8c43fd7092c", size = 322009 }, - { url = "https://files.pythonhosted.org/packages/9c/f2/80ffc4677aac1bc3519b26bc7f7f5de7fce0ee2f7e36e59e27d8beb32dd1/protobuf-6.32.0-py3-none-any.whl", hash = "sha256:ba377e5b67b908c8f3072a57b63e2c6a4cbd18aea4ed98d2584350dbf46f2783", size = 169287 }, + { url = "https://files.pythonhosted.org/packages/33/18/df8c87da2e47f4f1dcc5153a81cd6bca4e429803f4069a299e236e4dd510/protobuf-6.32.0-cp310-abi3-win32.whl", hash = "sha256:84f9e3c1ff6fb0308dbacb0950d8aa90694b0d0ee68e75719cb044b7078fe741", size = 424409, upload-time = "2025-08-14T21:21:12.366Z" }, + { url = "https://files.pythonhosted.org/packages/e1/59/0a820b7310f8139bd8d5a9388e6a38e1786d179d6f33998448609296c229/protobuf-6.32.0-cp310-abi3-win_amd64.whl", hash = "sha256:a8bdbb2f009cfc22a36d031f22a625a38b615b5e19e558a7b756b3279723e68e", size = 435735, upload-time = "2025-08-14T21:21:15.046Z" }, + { url = "https://files.pythonhosted.org/packages/cc/5b/0d421533c59c789e9c9894683efac582c06246bf24bb26b753b149bd88e4/protobuf-6.32.0-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:d52691e5bee6c860fff9a1c86ad26a13afbeb4b168cd4445c922b7e2cf85aaf0", size = 426449, upload-time = "2025-08-14T21:21:16.687Z" }, + { url = "https://files.pythonhosted.org/packages/ec/7b/607764ebe6c7a23dcee06e054fd1de3d5841b7648a90fd6def9a3bb58c5e/protobuf-6.32.0-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:501fe6372fd1c8ea2a30b4d9be8f87955a64d6be9c88a973996cef5ef6f0abf1", size = 322869, upload-time = "2025-08-14T21:21:18.282Z" }, + { url = "https://files.pythonhosted.org/packages/40/01/2e730bd1c25392fc32e3268e02446f0d77cb51a2c3a8486b1798e34d5805/protobuf-6.32.0-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:75a2aab2bd1aeb1f5dc7c5f33bcb11d82ea8c055c9becbb41c26a8c43fd7092c", size = 322009, upload-time = "2025-08-14T21:21:19.893Z" }, + { url = "https://files.pythonhosted.org/packages/9c/f2/80ffc4677aac1bc3519b26bc7f7f5de7fce0ee2f7e36e59e27d8beb32dd1/protobuf-6.32.0-py3-none-any.whl", hash = "sha256:ba377e5b67b908c8f3072a57b63e2c6a4cbd18aea4ed98d2584350dbf46f2783", size = 169287, upload-time = "2025-08-14T21:21:23.515Z" }, ] [[package]] name = "psutil" version = "7.0.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003 } +sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003, upload-time = "2025-02-13T21:54:07.946Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051 }, - { url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535 }, - { url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004 }, - { url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986 }, - { url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544 }, - { url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053 }, - { url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885 }, + { url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051, upload-time = "2025-02-13T21:54:12.36Z" }, + { url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535, upload-time = "2025-02-13T21:54:16.07Z" }, + { url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004, upload-time = "2025-02-13T21:54:18.662Z" }, + { url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986, upload-time = "2025-02-13T21:54:21.811Z" }, + { url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544, upload-time = "2025-02-13T21:54:24.68Z" }, + { url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053, upload-time = "2025-02-13T21:54:34.31Z" }, + { url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885, upload-time = "2025-02-13T21:54:37.486Z" }, ] [[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 } +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 }, + { 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 = "pyasn1" version = "0.6.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/ba/e9/01f1a64245b89f039897cb0130016d79f77d52669aae6ee7b159a6c4c018/pyasn1-0.6.1.tar.gz", hash = "sha256:6f580d2bdd84365380830acf45550f2511469f673cb4a5ae3857a3170128b034", size = 145322 } +sdist = { url = "https://files.pythonhosted.org/packages/ba/e9/01f1a64245b89f039897cb0130016d79f77d52669aae6ee7b159a6c4c018/pyasn1-0.6.1.tar.gz", hash = "sha256:6f580d2bdd84365380830acf45550f2511469f673cb4a5ae3857a3170128b034", size = 145322, upload-time = "2024-09-10T22:41:42.55Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/c8/f1/d6a797abb14f6283c0ddff96bbdd46937f64122b8c925cab503dd37f8214/pyasn1-0.6.1-py3-none-any.whl", hash = "sha256:0d632f46f2ba09143da3a8afe9e33fb6f92fa2320ab7e886e2d0f7672af84629", size = 83135 }, + { url = "https://files.pythonhosted.org/packages/c8/f1/d6a797abb14f6283c0ddff96bbdd46937f64122b8c925cab503dd37f8214/pyasn1-0.6.1-py3-none-any.whl", hash = "sha256:0d632f46f2ba09143da3a8afe9e33fb6f92fa2320ab7e886e2d0f7672af84629", size = 83135, upload-time = "2024-09-11T16:00:36.122Z" }, ] [[package]] @@ -1113,18 +1113,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pyasn1" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/e9/e6/78ebbb10a8c8e4b61a59249394a4a594c1a7af95593dc933a349c8d00964/pyasn1_modules-0.4.2.tar.gz", hash = "sha256:677091de870a80aae844b1ca6134f54652fa2c8c5a52aa396440ac3106e941e6", size = 307892 } +sdist = { url = "https://files.pythonhosted.org/packages/e9/e6/78ebbb10a8c8e4b61a59249394a4a594c1a7af95593dc933a349c8d00964/pyasn1_modules-0.4.2.tar.gz", hash = "sha256:677091de870a80aae844b1ca6134f54652fa2c8c5a52aa396440ac3106e941e6", size = 307892, upload-time = "2025-03-28T02:41:22.17Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/47/8d/d529b5d697919ba8c11ad626e835d4039be708a35b0d22de83a269a6682c/pyasn1_modules-0.4.2-py3-none-any.whl", hash = "sha256:29253a9207ce32b64c3ac6600edc75368f98473906e8fd1043bd6b5b1de2c14a", size = 181259 }, + { url = "https://files.pythonhosted.org/packages/47/8d/d529b5d697919ba8c11ad626e835d4039be708a35b0d22de83a269a6682c/pyasn1_modules-0.4.2-py3-none-any.whl", hash = "sha256:29253a9207ce32b64c3ac6600edc75368f98473906e8fd1043bd6b5b1de2c14a", size = 181259, upload-time = "2025-03-28T02:41:19.028Z" }, ] [[package]] name = "pycparser" version = "2.22" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/1d/b2/31537cf4b1ca988837256c910a668b553fceb8f069bedc4b1c826024b52c/pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6", size = 172736 } +sdist = { url = "https://files.pythonhosted.org/packages/1d/b2/31537cf4b1ca988837256c910a668b553fceb8f069bedc4b1c826024b52c/pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6", size = 172736, upload-time = "2024-03-30T13:22:22.564Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/13/a3/a812df4e2dd5696d1f351d58b8fe16a405b234ad2886a0dab9183fb78109/pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc", size = 117552 }, + { url = "https://files.pythonhosted.org/packages/13/a3/a812df4e2dd5696d1f351d58b8fe16a405b234ad2886a0dab9183fb78109/pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc", size = 117552, upload-time = "2024-03-30T13:22:20.476Z" }, ] [[package]] @@ -1138,27 +1138,27 @@ dependencies = [ { name = "typing-extensions" }, { name = "urllib3" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/6a/a7/403e04aa96e2d94e1518d518d69718c2ba978c8d3ffa4ab3b101b94dbafa/pygithub-2.7.0.tar.gz", hash = "sha256:7cd6eafabb09b5369afba3586d86b1f1ad6f1326d2ff01bc47bb26615dce4cbb", size = 3707928 } +sdist = { url = "https://files.pythonhosted.org/packages/6a/a7/403e04aa96e2d94e1518d518d69718c2ba978c8d3ffa4ab3b101b94dbafa/pygithub-2.7.0.tar.gz", hash = "sha256:7cd6eafabb09b5369afba3586d86b1f1ad6f1326d2ff01bc47bb26615dce4cbb", size = 3707928, upload-time = "2025-07-31T11:52:53.714Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/57/76/d768dd31322173b3956692b75471ac37bf3759c7abb603152f6a9b6594a8/pygithub-2.7.0-py3-none-any.whl", hash = "sha256:40ecbfe26dc55cc34ab4b0ffa1d455e6f816ef9a2bc8d6f5ad18ce572f163700", size = 416514 }, + { url = "https://files.pythonhosted.org/packages/57/76/d768dd31322173b3956692b75471ac37bf3759c7abb603152f6a9b6594a8/pygithub-2.7.0-py3-none-any.whl", hash = "sha256:40ecbfe26dc55cc34ab4b0ffa1d455e6f816ef9a2bc8d6f5ad18ce572f163700", size = 416514, upload-time = "2025-07-31T11:52:51.909Z" }, ] [[package]] name = "pygments" version = "2.19.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/b0/77/a5b8c569bf593b0140bde72ea885a803b82086995367bf2037de0159d924/pygments-2.19.2.tar.gz", hash = "sha256:636cb2477cec7f8952536970bc533bc43743542f70392ae026374600add5b887", size = 4968631 } +sdist = { url = "https://files.pythonhosted.org/packages/b0/77/a5b8c569bf593b0140bde72ea885a803b82086995367bf2037de0159d924/pygments-2.19.2.tar.gz", hash = "sha256:636cb2477cec7f8952536970bc533bc43743542f70392ae026374600add5b887", size = 4968631, upload-time = "2025-06-21T13:39:12.283Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/c7/21/705964c7812476f378728bdf590ca4b771ec72385c533964653c68e86bdc/pygments-2.19.2-py3-none-any.whl", hash = "sha256:86540386c03d588bb81d44bc3928634ff26449851e99741617ecb9037ee5ec0b", size = 1225217 }, + { url = "https://files.pythonhosted.org/packages/c7/21/705964c7812476f378728bdf590ca4b771ec72385c533964653c68e86bdc/pygments-2.19.2-py3-none-any.whl", hash = "sha256:86540386c03d588bb81d44bc3928634ff26449851e99741617ecb9037ee5ec0b", size = 1225217, upload-time = "2025-06-21T13:39:07.939Z" }, ] [[package]] name = "pyjwt" version = "2.10.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/e7/46/bd74733ff231675599650d3e47f361794b22ef3e3770998dda30d3b63726/pyjwt-2.10.1.tar.gz", hash = "sha256:3cc5772eb20009233caf06e9d8a0577824723b44e6648ee0a2aedb6cf9381953", size = 87785 } +sdist = { url = "https://files.pythonhosted.org/packages/e7/46/bd74733ff231675599650d3e47f361794b22ef3e3770998dda30d3b63726/pyjwt-2.10.1.tar.gz", hash = "sha256:3cc5772eb20009233caf06e9d8a0577824723b44e6648ee0a2aedb6cf9381953", size = 87785, upload-time = "2024-11-28T03:43:29.933Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/61/ad/689f02752eeec26aed679477e80e632ef1b682313be70793d798c1d5fc8f/PyJWT-2.10.1-py3-none-any.whl", hash = "sha256:dcdd193e30abefd5debf142f9adfcdd2b58004e644f25406ffaebd50bd98dacb", size = 22997 }, + { url = "https://files.pythonhosted.org/packages/61/ad/689f02752eeec26aed679477e80e632ef1b682313be70793d798c1d5fc8f/PyJWT-2.10.1-py3-none-any.whl", hash = "sha256:dcdd193e30abefd5debf142f9adfcdd2b58004e644f25406ffaebd50bd98dacb", size = 22997, upload-time = "2024-11-28T03:43:27.893Z" }, ] [package.optional-dependencies] @@ -1173,26 +1173,26 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "cffi" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/a7/22/27582568be639dfe22ddb3902225f91f2f17ceff88ce80e4db396c8986da/PyNaCl-1.5.0.tar.gz", hash = "sha256:8ac7448f09ab85811607bdd21ec2464495ac8b7c66d146bf545b0f08fb9220ba", size = 3392854 } +sdist = { url = "https://files.pythonhosted.org/packages/a7/22/27582568be639dfe22ddb3902225f91f2f17ceff88ce80e4db396c8986da/PyNaCl-1.5.0.tar.gz", hash = "sha256:8ac7448f09ab85811607bdd21ec2464495ac8b7c66d146bf545b0f08fb9220ba", size = 3392854, upload-time = "2022-01-07T22:05:41.134Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ce/75/0b8ede18506041c0bf23ac4d8e2971b4161cd6ce630b177d0a08eb0d8857/PyNaCl-1.5.0-cp36-abi3-macosx_10_10_universal2.whl", hash = "sha256:401002a4aaa07c9414132aaed7f6836ff98f59277a234704ff66878c2ee4a0d1", size = 349920 }, - { url = "https://files.pythonhosted.org/packages/59/bb/fddf10acd09637327a97ef89d2a9d621328850a72f1fdc8c08bdf72e385f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_24_aarch64.whl", hash = "sha256:52cb72a79269189d4e0dc537556f4740f7f0a9ec41c1322598799b0bdad4ef92", size = 601722 }, - { url = "https://files.pythonhosted.org/packages/5d/70/87a065c37cca41a75f2ce113a5a2c2aa7533be648b184ade58971b5f7ccc/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a36d4a9dda1f19ce6e03c9a784a2921a4b726b02e1c736600ca9c22029474394", size = 680087 }, - { url = "https://files.pythonhosted.org/packages/ee/87/f1bb6a595f14a327e8285b9eb54d41fef76c585a0edef0a45f6fc95de125/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:0c84947a22519e013607c9be43706dd42513f9e6ae5d39d3613ca1e142fba44d", size = 856678 }, - { url = "https://files.pythonhosted.org/packages/66/28/ca86676b69bf9f90e710571b67450508484388bfce09acf8a46f0b8c785f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:06b8f6fa7f5de8d5d2f7573fe8c863c051225a27b61e6860fd047b1775807858", size = 1133660 }, - { url = "https://files.pythonhosted.org/packages/3d/85/c262db650e86812585e2bc59e497a8f59948a005325a11bbbc9ecd3fe26b/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:a422368fc821589c228f4c49438a368831cb5bbc0eab5ebe1d7fac9dded6567b", size = 663824 }, - { url = "https://files.pythonhosted.org/packages/fd/1a/cc308a884bd299b651f1633acb978e8596c71c33ca85e9dc9fa33a5399b9/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:61f642bf2378713e2c2e1de73444a3778e5f0a38be6fee0fe532fe30060282ff", size = 1117912 }, - { url = "https://files.pythonhosted.org/packages/25/2d/b7df6ddb0c2a33afdb358f8af6ea3b8c4d1196ca45497dd37a56f0c122be/PyNaCl-1.5.0-cp36-abi3-win32.whl", hash = "sha256:e46dae94e34b085175f8abb3b0aaa7da40767865ac82c928eeb9e57e1ea8a543", size = 204624 }, - { url = "https://files.pythonhosted.org/packages/5e/22/d3db169895faaf3e2eda892f005f433a62db2decbcfbc2f61e6517adfa87/PyNaCl-1.5.0-cp36-abi3-win_amd64.whl", hash = "sha256:20f42270d27e1b6a29f54032090b972d97f0a1b0948cc52392041ef7831fee93", size = 212141 }, + { url = "https://files.pythonhosted.org/packages/ce/75/0b8ede18506041c0bf23ac4d8e2971b4161cd6ce630b177d0a08eb0d8857/PyNaCl-1.5.0-cp36-abi3-macosx_10_10_universal2.whl", hash = "sha256:401002a4aaa07c9414132aaed7f6836ff98f59277a234704ff66878c2ee4a0d1", size = 349920, upload-time = "2022-01-07T22:05:49.156Z" }, + { url = "https://files.pythonhosted.org/packages/59/bb/fddf10acd09637327a97ef89d2a9d621328850a72f1fdc8c08bdf72e385f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_24_aarch64.whl", hash = "sha256:52cb72a79269189d4e0dc537556f4740f7f0a9ec41c1322598799b0bdad4ef92", size = 601722, upload-time = "2022-01-07T22:05:50.989Z" }, + { url = "https://files.pythonhosted.org/packages/5d/70/87a065c37cca41a75f2ce113a5a2c2aa7533be648b184ade58971b5f7ccc/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a36d4a9dda1f19ce6e03c9a784a2921a4b726b02e1c736600ca9c22029474394", size = 680087, upload-time = "2022-01-07T22:05:52.539Z" }, + { url = "https://files.pythonhosted.org/packages/ee/87/f1bb6a595f14a327e8285b9eb54d41fef76c585a0edef0a45f6fc95de125/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:0c84947a22519e013607c9be43706dd42513f9e6ae5d39d3613ca1e142fba44d", size = 856678, upload-time = "2022-01-07T22:05:54.251Z" }, + { url = "https://files.pythonhosted.org/packages/66/28/ca86676b69bf9f90e710571b67450508484388bfce09acf8a46f0b8c785f/PyNaCl-1.5.0-cp36-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:06b8f6fa7f5de8d5d2f7573fe8c863c051225a27b61e6860fd047b1775807858", size = 1133660, upload-time = "2022-01-07T22:05:56.056Z" }, + { url = "https://files.pythonhosted.org/packages/3d/85/c262db650e86812585e2bc59e497a8f59948a005325a11bbbc9ecd3fe26b/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:a422368fc821589c228f4c49438a368831cb5bbc0eab5ebe1d7fac9dded6567b", size = 663824, upload-time = "2022-01-07T22:05:57.434Z" }, + { url = "https://files.pythonhosted.org/packages/fd/1a/cc308a884bd299b651f1633acb978e8596c71c33ca85e9dc9fa33a5399b9/PyNaCl-1.5.0-cp36-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:61f642bf2378713e2c2e1de73444a3778e5f0a38be6fee0fe532fe30060282ff", size = 1117912, upload-time = "2022-01-07T22:05:58.665Z" }, + { url = "https://files.pythonhosted.org/packages/25/2d/b7df6ddb0c2a33afdb358f8af6ea3b8c4d1196ca45497dd37a56f0c122be/PyNaCl-1.5.0-cp36-abi3-win32.whl", hash = "sha256:e46dae94e34b085175f8abb3b0aaa7da40767865ac82c928eeb9e57e1ea8a543", size = 204624, upload-time = "2022-01-07T22:06:00.085Z" }, + { url = "https://files.pythonhosted.org/packages/5e/22/d3db169895faaf3e2eda892f005f433a62db2decbcfbc2f61e6517adfa87/PyNaCl-1.5.0-cp36-abi3-win_amd64.whl", hash = "sha256:20f42270d27e1b6a29f54032090b972d97f0a1b0948cc52392041ef7831fee93", size = 212141, upload-time = "2022-01-07T22:06:01.861Z" }, ] [[package]] name = "pyparsing" version = "3.2.3" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/bb/22/f1129e69d94ffff626bdb5c835506b3a5b4f3d070f17ea295e12c2c6f60f/pyparsing-3.2.3.tar.gz", hash = "sha256:b9c13f1ab8b3b542f72e28f634bad4de758ab3ce4546e4301970ad6fa77c38be", size = 1088608 } +sdist = { url = "https://files.pythonhosted.org/packages/bb/22/f1129e69d94ffff626bdb5c835506b3a5b4f3d070f17ea295e12c2c6f60f/pyparsing-3.2.3.tar.gz", hash = "sha256:b9c13f1ab8b3b542f72e28f634bad4de758ab3ce4546e4301970ad6fa77c38be", size = 1088608, upload-time = "2025-03-25T05:01:28.114Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/05/e7/df2285f3d08fee213f2d041540fa4fc9ca6c2d44cf36d3a035bf2a8d2bcc/pyparsing-3.2.3-py3-none-any.whl", hash = "sha256:a749938e02d6fd0b59b356ca504a24982314bb090c383e3cf201c95ef7e2bfcf", size = 111120 }, + { url = "https://files.pythonhosted.org/packages/05/e7/df2285f3d08fee213f2d041540fa4fc9ca6c2d44cf36d3a035bf2a8d2bcc/pyparsing-3.2.3-py3-none-any.whl", hash = "sha256:a749938e02d6fd0b59b356ca504a24982314bb090c383e3cf201c95ef7e2bfcf", size = 111120, upload-time = "2025-03-25T05:01:24.908Z" }, ] [[package]] @@ -1208,9 +1208,9 @@ dependencies = [ { name = "pygments" }, { name = "tomli", marker = "python_full_version < '3.11'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/08/ba/45911d754e8eba3d5a841a5ce61a65a685ff1798421ac054f85aa8747dfb/pytest-8.4.1.tar.gz", hash = "sha256:7c67fd69174877359ed9371ec3af8a3d2b04741818c51e5e99cc1742251fa93c", size = 1517714 } +sdist = { url = "https://files.pythonhosted.org/packages/08/ba/45911d754e8eba3d5a841a5ce61a65a685ff1798421ac054f85aa8747dfb/pytest-8.4.1.tar.gz", hash = "sha256:7c67fd69174877359ed9371ec3af8a3d2b04741818c51e5e99cc1742251fa93c", size = 1517714, upload-time = "2025-06-18T05:48:06.109Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/29/16/c8a903f4c4dffe7a12843191437d7cd8e32751d5de349d45d3fe69544e87/pytest-8.4.1-py3-none-any.whl", hash = "sha256:539c70ba6fcead8e78eebbf1115e8b589e7565830d7d006a8723f19ac8a0afb7", size = 365474 }, + { url = "https://files.pythonhosted.org/packages/29/16/c8a903f4c4dffe7a12843191437d7cd8e32751d5de349d45d3fe69544e87/pytest-8.4.1-py3-none-any.whl", hash = "sha256:539c70ba6fcead8e78eebbf1115e8b589e7565830d7d006a8723f19ac8a0afb7", size = 365474, upload-time = "2025-06-18T05:48:03.955Z" }, ] [[package]] @@ -1221,9 +1221,9 @@ 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 } +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 }, + { 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]] @@ -1233,82 +1233,82 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "six" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/66/c0/0c8b6ad9f17a802ee498c46e004a0eb49bc148f2fd230864601a86dcf6db/python-dateutil-2.9.0.post0.tar.gz", hash = "sha256:37dd54208da7e1cd875388217d5e00ebd4179249f90fb72437e91a35459a0ad3", size = 342432 } +sdist = { url = "https://files.pythonhosted.org/packages/66/c0/0c8b6ad9f17a802ee498c46e004a0eb49bc148f2fd230864601a86dcf6db/python-dateutil-2.9.0.post0.tar.gz", hash = "sha256:37dd54208da7e1cd875388217d5e00ebd4179249f90fb72437e91a35459a0ad3", size = 342432, upload-time = "2024-03-01T18:36:20.211Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ec/57/56b9bcc3c9c6a792fcbaf139543cee77261f3651ca9da0c93f5c1221264b/python_dateutil-2.9.0.post0-py2.py3-none-any.whl", hash = "sha256:a8b2bc7bffae282281c8140a97d3aa9c14da0b136dfe83f850eea9a5f7470427", size = 229892 }, + { url = "https://files.pythonhosted.org/packages/ec/57/56b9bcc3c9c6a792fcbaf139543cee77261f3651ca9da0c93f5c1221264b/python_dateutil-2.9.0.post0-py2.py3-none-any.whl", hash = "sha256:a8b2bc7bffae282281c8140a97d3aa9c14da0b136dfe83f850eea9a5f7470427", size = 229892, upload-time = "2024-03-01T18:36:18.57Z" }, ] [[package]] name = "pywin32-ctypes" version = "0.2.3" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/85/9f/01a1a99704853cb63f253eea009390c88e7131c67e66a0a02099a8c917cb/pywin32-ctypes-0.2.3.tar.gz", hash = "sha256:d162dc04946d704503b2edc4d55f3dba5c1d539ead017afa00142c38b9885755", size = 29471 } +sdist = { url = "https://files.pythonhosted.org/packages/85/9f/01a1a99704853cb63f253eea009390c88e7131c67e66a0a02099a8c917cb/pywin32-ctypes-0.2.3.tar.gz", hash = "sha256:d162dc04946d704503b2edc4d55f3dba5c1d539ead017afa00142c38b9885755", size = 29471, upload-time = "2024-08-14T10:15:34.626Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/de/3d/8161f7711c017e01ac9f008dfddd9410dff3674334c233bde66e7ba65bbf/pywin32_ctypes-0.2.3-py3-none-any.whl", hash = "sha256:8a1513379d709975552d202d942d9837758905c8d01eb82b8bcc30918929e7b8", size = 30756 }, + { url = "https://files.pythonhosted.org/packages/de/3d/8161f7711c017e01ac9f008dfddd9410dff3674334c233bde66e7ba65bbf/pywin32_ctypes-0.2.3-py3-none-any.whl", hash = "sha256:8a1513379d709975552d202d942d9837758905c8d01eb82b8bcc30918929e7b8", size = 30756, upload-time = "2024-08-14T10:15:33.187Z" }, ] [[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 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/f4/a0/39350dd17dd6d6c6507025c0e53aef67a9293a6d37d3511f23ea510d5800/pyyaml-6.0.3-cp310-cp310-macosx_10_13_x86_64.whl", hash = "sha256:214ed4befebe12df36bcc8bc2b64b396ca31be9304b8f59e25c11cf94a4c033b", size = 184227 }, - { url = "https://files.pythonhosted.org/packages/05/14/52d505b5c59ce73244f59c7a50ecf47093ce4765f116cdb98286a71eeca2/pyyaml-6.0.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:02ea2dfa234451bbb8772601d7b8e426c2bfa197136796224e50e35a78777956", size = 174019 }, - { url = "https://files.pythonhosted.org/packages/43/f7/0e6a5ae5599c838c696adb4e6330a59f463265bfa1e116cfd1fbb0abaaae/pyyaml-6.0.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b30236e45cf30d2b8e7b3e85881719e98507abed1011bf463a8fa23e9c3e98a8", size = 740646 }, - { url = "https://files.pythonhosted.org/packages/2f/3a/61b9db1d28f00f8fd0ae760459a5c4bf1b941baf714e207b6eb0657d2578/pyyaml-6.0.3-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:66291b10affd76d76f54fad28e22e51719ef9ba22b29e1d7d03d6777a9174198", size = 840793 }, - { url = "https://files.pythonhosted.org/packages/7a/1e/7acc4f0e74c4b3d9531e24739e0ab832a5edf40e64fbae1a9c01941cabd7/pyyaml-6.0.3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9c7708761fccb9397fe64bbc0395abcae8c4bf7b0eac081e12b809bf47700d0b", size = 770293 }, - { url = "https://files.pythonhosted.org/packages/8b/ef/abd085f06853af0cd59fa5f913d61a8eab65d7639ff2a658d18a25d6a89d/pyyaml-6.0.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:418cf3f2111bc80e0933b2cd8cd04f286338bb88bdc7bc8e6dd775ebde60b5e0", size = 732872 }, - { url = "https://files.pythonhosted.org/packages/1f/15/2bc9c8faf6450a8b3c9fc5448ed869c599c0a74ba2669772b1f3a0040180/pyyaml-6.0.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:5e0b74767e5f8c593e8c9b5912019159ed0533c70051e9cce3e8b6aa699fcd69", size = 758828 }, - { url = "https://files.pythonhosted.org/packages/a3/00/531e92e88c00f4333ce359e50c19b8d1de9fe8d581b1534e35ccfbc5f393/pyyaml-6.0.3-cp310-cp310-win32.whl", hash = "sha256:28c8d926f98f432f88adc23edf2e6d4921ac26fb084b028c733d01868d19007e", size = 142415 }, - { url = "https://files.pythonhosted.org/packages/2a/fa/926c003379b19fca39dd4634818b00dec6c62d87faf628d1394e137354d4/pyyaml-6.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:bdb2c67c6c1390b63c6ff89f210c8fd09d9a1217a465701eac7316313c915e4c", size = 158561 }, - { url = "https://files.pythonhosted.org/packages/6d/16/a95b6757765b7b031c9374925bb718d55e0a9ba8a1b6a12d25962ea44347/pyyaml-6.0.3-cp311-cp311-macosx_10_13_x86_64.whl", hash = "sha256:44edc647873928551a01e7a563d7452ccdebee747728c1080d881d68af7b997e", size = 185826 }, - { url = "https://files.pythonhosted.org/packages/16/19/13de8e4377ed53079ee996e1ab0a9c33ec2faf808a4647b7b4c0d46dd239/pyyaml-6.0.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:652cb6edd41e718550aad172851962662ff2681490a8a711af6a4d288dd96824", size = 175577 }, - { url = "https://files.pythonhosted.org/packages/0c/62/d2eb46264d4b157dae1275b573017abec435397aa59cbcdab6fc978a8af4/pyyaml-6.0.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:10892704fc220243f5305762e276552a0395f7beb4dbf9b14ec8fd43b57f126c", size = 775556 }, - { url = "https://files.pythonhosted.org/packages/10/cb/16c3f2cf3266edd25aaa00d6c4350381c8b012ed6f5276675b9eba8d9ff4/pyyaml-6.0.3-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:850774a7879607d3a6f50d36d04f00ee69e7fc816450e5f7e58d7f17f1ae5c00", size = 882114 }, - { url = "https://files.pythonhosted.org/packages/71/60/917329f640924b18ff085ab889a11c763e0b573da888e8404ff486657602/pyyaml-6.0.3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b8bb0864c5a28024fac8a632c443c87c5aa6f215c0b126c449ae1a150412f31d", size = 806638 }, - { url = "https://files.pythonhosted.org/packages/dd/6f/529b0f316a9fd167281a6c3826b5583e6192dba792dd55e3203d3f8e655a/pyyaml-6.0.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:1d37d57ad971609cf3c53ba6a7e365e40660e3be0e5175fa9f2365a379d6095a", size = 767463 }, - { url = "https://files.pythonhosted.org/packages/f2/6a/b627b4e0c1dd03718543519ffb2f1deea4a1e6d42fbab8021936a4d22589/pyyaml-6.0.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:37503bfbfc9d2c40b344d06b2199cf0e96e97957ab1c1b546fd4f87e53e5d3e4", size = 794986 }, - { url = "https://files.pythonhosted.org/packages/45/91/47a6e1c42d9ee337c4839208f30d9f09caa9f720ec7582917b264defc875/pyyaml-6.0.3-cp311-cp311-win32.whl", hash = "sha256:8098f252adfa6c80ab48096053f512f2321f0b998f98150cea9bd23d83e1467b", size = 142543 }, - { url = "https://files.pythonhosted.org/packages/da/e3/ea007450a105ae919a72393cb06f122f288ef60bba2dc64b26e2646fa315/pyyaml-6.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:9f3bfb4965eb874431221a3ff3fdcddc7e74e3b07799e0e84ca4a0f867d449bf", size = 158763 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { url = "https://files.pythonhosted.org/packages/74/93/7baea19427dcfbe1e5a372d81473250b379f04b1bd3c4c5ff825e2327202/pyyaml-6.0.3-cp312-cp312-win32.whl", hash = "sha256:96b533f0e99f6579b3d4d4995707cf36df9100d67e0c8303a0c55b27b5f99bc5", size = 137658 }, - { url = "https://files.pythonhosted.org/packages/86/bf/899e81e4cce32febab4fb42bb97dcdf66bc135272882d1987881a4b519e9/pyyaml-6.0.3-cp312-cp312-win_amd64.whl", hash = "sha256:5fcd34e47f6e0b794d17de1b4ff496c00986e1c83f7ab2fb8fcfe9616ff7477b", size = 154003 }, - { url = "https://files.pythonhosted.org/packages/1a/08/67bd04656199bbb51dbed1439b7f27601dfb576fb864099c7ef0c3e55531/pyyaml-6.0.3-cp312-cp312-win_arm64.whl", hash = "sha256:64386e5e707d03a7e172c0701abfb7e10f0fb753ee1d773128192742712a98fd", size = 140344 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { url = "https://files.pythonhosted.org/packages/de/94/980b50a6531b3019e45ddeada0626d45fa85cbe22300844a7983285bed3b/pyyaml-6.0.3-cp313-cp313-win32.whl", hash = "sha256:d0eae10f8159e8fdad514efdc92d74fd8d682c933a6dd088030f3834bc8e6b26", size = 137427 }, - { url = "https://files.pythonhosted.org/packages/97/c9/39d5b874e8b28845e4ec2202b5da735d0199dbe5b8fb85f91398814a9a46/pyyaml-6.0.3-cp313-cp313-win_amd64.whl", hash = "sha256:79005a0d97d5ddabfeeea4cf676af11e647e41d81c9a7722a193022accdb6b7c", size = 154090 }, - { url = "https://files.pythonhosted.org/packages/73/e8/2bdf3ca2090f68bb3d75b44da7bbc71843b19c9f2b9cb9b0f4ab7a5a4329/pyyaml-6.0.3-cp313-cp313-win_arm64.whl", hash = "sha256:5498cd1645aa724a7c71c8f378eb29ebe23da2fc0d7a08071d89469bf1d2defb", size = 140246 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { url = "https://files.pythonhosted.org/packages/23/20/bb6982b26a40bb43951265ba29d4c246ef0ff59c9fdcdf0ed04e0687de4d/pyyaml-6.0.3-cp314-cp314-win_amd64.whl", hash = "sha256:4a2e8cebe2ff6ab7d1050ecd59c25d4c8bd7e6f400f5f82b96557ac0abafd0ac", size = 156429 }, - { url = "https://files.pythonhosted.org/packages/f4/f4/a4541072bb9422c8a883ab55255f918fa378ecf083f5b85e87fc2b4eda1b/pyyaml-6.0.3-cp314-cp314-win_arm64.whl", hash = "sha256:93dda82c9c22deb0a405ea4dc5f2d0cda384168e466364dec6255b293923b2f3", size = 143912 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { 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 }, - { url = "https://files.pythonhosted.org/packages/f0/7a/1c7270340330e575b92f397352af856a8c06f230aa3e76f86b39d01b416a/pyyaml-6.0.3-cp314-cp314t-win_amd64.whl", hash = "sha256:4ad1906908f2f5ae4e5a8ddfce73c320c2a1429ec52eafd27138b7f1cbe341c9", size = 174062 }, - { url = "https://files.pythonhosted.org/packages/f1/12/de94a39c2ef588c7e6455cfbe7343d3b2dc9d6b6b2f40c4c6565744c873d/pyyaml-6.0.3-cp314-cp314t-win_arm64.whl", hash = "sha256:ebc55a14a21cb14062aa4162f906cd962b28e2e9ea38f9b4391244cd8de4ae0b", size = 149341 }, +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/f4/a0/39350dd17dd6d6c6507025c0e53aef67a9293a6d37d3511f23ea510d5800/pyyaml-6.0.3-cp310-cp310-macosx_10_13_x86_64.whl", hash = "sha256:214ed4befebe12df36bcc8bc2b64b396ca31be9304b8f59e25c11cf94a4c033b", size = 184227, upload-time = "2025-09-25T21:31:46.04Z" }, + { url = "https://files.pythonhosted.org/packages/05/14/52d505b5c59ce73244f59c7a50ecf47093ce4765f116cdb98286a71eeca2/pyyaml-6.0.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:02ea2dfa234451bbb8772601d7b8e426c2bfa197136796224e50e35a78777956", size = 174019, upload-time = "2025-09-25T21:31:47.706Z" }, + { url = "https://files.pythonhosted.org/packages/43/f7/0e6a5ae5599c838c696adb4e6330a59f463265bfa1e116cfd1fbb0abaaae/pyyaml-6.0.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b30236e45cf30d2b8e7b3e85881719e98507abed1011bf463a8fa23e9c3e98a8", size = 740646, upload-time = "2025-09-25T21:31:49.21Z" }, + { url = "https://files.pythonhosted.org/packages/2f/3a/61b9db1d28f00f8fd0ae760459a5c4bf1b941baf714e207b6eb0657d2578/pyyaml-6.0.3-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:66291b10affd76d76f54fad28e22e51719ef9ba22b29e1d7d03d6777a9174198", size = 840793, upload-time = "2025-09-25T21:31:50.735Z" }, + { url = "https://files.pythonhosted.org/packages/7a/1e/7acc4f0e74c4b3d9531e24739e0ab832a5edf40e64fbae1a9c01941cabd7/pyyaml-6.0.3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9c7708761fccb9397fe64bbc0395abcae8c4bf7b0eac081e12b809bf47700d0b", size = 770293, upload-time = "2025-09-25T21:31:51.828Z" }, + { url = "https://files.pythonhosted.org/packages/8b/ef/abd085f06853af0cd59fa5f913d61a8eab65d7639ff2a658d18a25d6a89d/pyyaml-6.0.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:418cf3f2111bc80e0933b2cd8cd04f286338bb88bdc7bc8e6dd775ebde60b5e0", size = 732872, upload-time = "2025-09-25T21:31:53.282Z" }, + { url = "https://files.pythonhosted.org/packages/1f/15/2bc9c8faf6450a8b3c9fc5448ed869c599c0a74ba2669772b1f3a0040180/pyyaml-6.0.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:5e0b74767e5f8c593e8c9b5912019159ed0533c70051e9cce3e8b6aa699fcd69", size = 758828, upload-time = "2025-09-25T21:31:54.807Z" }, + { url = "https://files.pythonhosted.org/packages/a3/00/531e92e88c00f4333ce359e50c19b8d1de9fe8d581b1534e35ccfbc5f393/pyyaml-6.0.3-cp310-cp310-win32.whl", hash = "sha256:28c8d926f98f432f88adc23edf2e6d4921ac26fb084b028c733d01868d19007e", size = 142415, upload-time = "2025-09-25T21:31:55.885Z" }, + { url = "https://files.pythonhosted.org/packages/2a/fa/926c003379b19fca39dd4634818b00dec6c62d87faf628d1394e137354d4/pyyaml-6.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:bdb2c67c6c1390b63c6ff89f210c8fd09d9a1217a465701eac7316313c915e4c", size = 158561, upload-time = "2025-09-25T21:31:57.406Z" }, + { url = "https://files.pythonhosted.org/packages/6d/16/a95b6757765b7b031c9374925bb718d55e0a9ba8a1b6a12d25962ea44347/pyyaml-6.0.3-cp311-cp311-macosx_10_13_x86_64.whl", hash = "sha256:44edc647873928551a01e7a563d7452ccdebee747728c1080d881d68af7b997e", size = 185826, upload-time = "2025-09-25T21:31:58.655Z" }, + { url = "https://files.pythonhosted.org/packages/16/19/13de8e4377ed53079ee996e1ab0a9c33ec2faf808a4647b7b4c0d46dd239/pyyaml-6.0.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:652cb6edd41e718550aad172851962662ff2681490a8a711af6a4d288dd96824", size = 175577, upload-time = "2025-09-25T21:32:00.088Z" }, + { url = "https://files.pythonhosted.org/packages/0c/62/d2eb46264d4b157dae1275b573017abec435397aa59cbcdab6fc978a8af4/pyyaml-6.0.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:10892704fc220243f5305762e276552a0395f7beb4dbf9b14ec8fd43b57f126c", size = 775556, upload-time = "2025-09-25T21:32:01.31Z" }, + { url = "https://files.pythonhosted.org/packages/10/cb/16c3f2cf3266edd25aaa00d6c4350381c8b012ed6f5276675b9eba8d9ff4/pyyaml-6.0.3-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:850774a7879607d3a6f50d36d04f00ee69e7fc816450e5f7e58d7f17f1ae5c00", size = 882114, upload-time = "2025-09-25T21:32:03.376Z" }, + { url = "https://files.pythonhosted.org/packages/71/60/917329f640924b18ff085ab889a11c763e0b573da888e8404ff486657602/pyyaml-6.0.3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b8bb0864c5a28024fac8a632c443c87c5aa6f215c0b126c449ae1a150412f31d", size = 806638, upload-time = "2025-09-25T21:32:04.553Z" }, + { url = "https://files.pythonhosted.org/packages/dd/6f/529b0f316a9fd167281a6c3826b5583e6192dba792dd55e3203d3f8e655a/pyyaml-6.0.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:1d37d57ad971609cf3c53ba6a7e365e40660e3be0e5175fa9f2365a379d6095a", size = 767463, upload-time = "2025-09-25T21:32:06.152Z" }, + { url = "https://files.pythonhosted.org/packages/f2/6a/b627b4e0c1dd03718543519ffb2f1deea4a1e6d42fbab8021936a4d22589/pyyaml-6.0.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:37503bfbfc9d2c40b344d06b2199cf0e96e97957ab1c1b546fd4f87e53e5d3e4", size = 794986, upload-time = "2025-09-25T21:32:07.367Z" }, + { url = "https://files.pythonhosted.org/packages/45/91/47a6e1c42d9ee337c4839208f30d9f09caa9f720ec7582917b264defc875/pyyaml-6.0.3-cp311-cp311-win32.whl", hash = "sha256:8098f252adfa6c80ab48096053f512f2321f0b998f98150cea9bd23d83e1467b", size = 142543, upload-time = "2025-09-25T21:32:08.95Z" }, + { url = "https://files.pythonhosted.org/packages/da/e3/ea007450a105ae919a72393cb06f122f288ef60bba2dc64b26e2646fa315/pyyaml-6.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:9f3bfb4965eb874431221a3ff3fdcddc7e74e3b07799e0e84ca4a0f867d449bf", size = 158763, upload-time = "2025-09-25T21:32:09.96Z" }, + { 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]] @@ -1320,9 +1320,9 @@ dependencies = [ { name = "nh3" }, { name = "pygments" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/5a/a9/104ec9234c8448c4379768221ea6df01260cd6c2ce13182d4eac531c8342/readme_renderer-44.0.tar.gz", hash = "sha256:8712034eabbfa6805cacf1402b4eeb2a73028f72d1166d6f5cb7f9c047c5d1e1", size = 32056 } +sdist = { url = "https://files.pythonhosted.org/packages/5a/a9/104ec9234c8448c4379768221ea6df01260cd6c2ce13182d4eac531c8342/readme_renderer-44.0.tar.gz", hash = "sha256:8712034eabbfa6805cacf1402b4eeb2a73028f72d1166d6f5cb7f9c047c5d1e1", size = 32056, upload-time = "2024-07-08T15:00:57.805Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e1/67/921ec3024056483db83953ae8e48079ad62b92db7880013ca77632921dd0/readme_renderer-44.0-py3-none-any.whl", hash = "sha256:2fbca89b81a08526aadf1357a8c2ae889ec05fb03f5da67f9769c9a592166151", size = 13310 }, + { url = "https://files.pythonhosted.org/packages/e1/67/921ec3024056483db83953ae8e48079ad62b92db7880013ca77632921dd0/readme_renderer-44.0-py3-none-any.whl", hash = "sha256:2fbca89b81a08526aadf1357a8c2ae889ec05fb03f5da67f9769c9a592166151", size = 13310, upload-time = "2024-07-08T15:00:56.577Z" }, ] [[package]] @@ -1334,9 +1334,9 @@ dependencies = [ { name = "rpds-py" }, { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/2f/db/98b5c277be99dd18bfd91dd04e1b759cad18d1a338188c936e92f921c7e2/referencing-0.36.2.tar.gz", hash = "sha256:df2e89862cd09deabbdba16944cc3f10feb6b3e6f18e902f7cc25609a34775aa", size = 74744 } +sdist = { url = "https://files.pythonhosted.org/packages/2f/db/98b5c277be99dd18bfd91dd04e1b759cad18d1a338188c936e92f921c7e2/referencing-0.36.2.tar.gz", hash = "sha256:df2e89862cd09deabbdba16944cc3f10feb6b3e6f18e902f7cc25609a34775aa", size = 74744, upload-time = "2025-01-25T08:48:16.138Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/c1/b1/3baf80dc6d2b7bc27a95a67752d0208e410351e3feb4eb78de5f77454d8d/referencing-0.36.2-py3-none-any.whl", hash = "sha256:e8699adbbf8b5c7de96d8ffa0eb5c158b3beafce084968e2ea8bb08c6794dcd0", size = 26775 }, + { url = "https://files.pythonhosted.org/packages/c1/b1/3baf80dc6d2b7bc27a95a67752d0208e410351e3feb4eb78de5f77454d8d/referencing-0.36.2-py3-none-any.whl", hash = "sha256:e8699adbbf8b5c7de96d8ffa0eb5c158b3beafce084968e2ea8bb08c6794dcd0", size = 26775, upload-time = "2025-01-25T08:48:14.241Z" }, ] [[package]] @@ -1349,9 +1349,9 @@ dependencies = [ { name = "idna" }, { name = "urllib3" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/e1/0a/929373653770d8a0d7ea76c37de6e41f11eb07559b103b1c02cafb3f7cf8/requests-2.32.4.tar.gz", hash = "sha256:27d0316682c8a29834d3264820024b62a36942083d52caf2f14c0591336d3422", size = 135258 } +sdist = { url = "https://files.pythonhosted.org/packages/e1/0a/929373653770d8a0d7ea76c37de6e41f11eb07559b103b1c02cafb3f7cf8/requests-2.32.4.tar.gz", hash = "sha256:27d0316682c8a29834d3264820024b62a36942083d52caf2f14c0591336d3422", size = 135258, upload-time = "2025-06-09T16:43:07.34Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/7c/e4/56027c4a6b4ae70ca9de302488c5ca95ad4a39e190093d6c1a8ace08341b/requests-2.32.4-py3-none-any.whl", hash = "sha256:27babd3cda2a6d50b30443204ee89830707d396671944c998b5975b031ac2b2c", size = 64847 }, + { url = "https://files.pythonhosted.org/packages/7c/e4/56027c4a6b4ae70ca9de302488c5ca95ad4a39e190093d6c1a8ace08341b/requests-2.32.4-py3-none-any.whl", hash = "sha256:27babd3cda2a6d50b30443204ee89830707d396671944c998b5975b031ac2b2c", size = 64847, upload-time = "2025-06-09T16:43:05.728Z" }, ] [[package]] @@ -1362,9 +1362,9 @@ dependencies = [ { name = "oauthlib" }, { name = "requests" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/42/f2/05f29bc3913aea15eb670be136045bf5c5bbf4b99ecb839da9b422bb2c85/requests-oauthlib-2.0.0.tar.gz", hash = "sha256:b3dffaebd884d8cd778494369603a9e7b58d29111bf6b41bdc2dcd87203af4e9", size = 55650 } +sdist = { url = "https://files.pythonhosted.org/packages/42/f2/05f29bc3913aea15eb670be136045bf5c5bbf4b99ecb839da9b422bb2c85/requests-oauthlib-2.0.0.tar.gz", hash = "sha256:b3dffaebd884d8cd778494369603a9e7b58d29111bf6b41bdc2dcd87203af4e9", size = 55650, upload-time = "2024-03-22T20:32:29.939Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/3b/5d/63d4ae3b9daea098d5d6f5da83984853c1bbacd5dc826764b249fe119d24/requests_oauthlib-2.0.0-py2.py3-none-any.whl", hash = "sha256:7dd8a5c40426b779b0868c404bdef9768deccf22749cde15852df527e6269b36", size = 24179 }, + { url = "https://files.pythonhosted.org/packages/3b/5d/63d4ae3b9daea098d5d6f5da83984853c1bbacd5dc826764b249fe119d24/requests_oauthlib-2.0.0-py2.py3-none-any.whl", hash = "sha256:7dd8a5c40426b779b0868c404bdef9768deccf22749cde15852df527e6269b36", size = 24179, upload-time = "2024-03-22T20:32:28.055Z" }, ] [[package]] @@ -1374,9 +1374,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "requests" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/f3/61/d7545dafb7ac2230c70d38d31cbfe4cc64f7144dc41f6e4e4b78ecd9f5bb/requests-toolbelt-1.0.0.tar.gz", hash = "sha256:7681a0a3d047012b5bdc0ee37d7f8f07ebe76ab08caeccfc3921ce23c88d5bc6", size = 206888 } +sdist = { url = "https://files.pythonhosted.org/packages/f3/61/d7545dafb7ac2230c70d38d31cbfe4cc64f7144dc41f6e4e4b78ecd9f5bb/requests-toolbelt-1.0.0.tar.gz", hash = "sha256:7681a0a3d047012b5bdc0ee37d7f8f07ebe76ab08caeccfc3921ce23c88d5bc6", size = 206888, upload-time = "2023-05-01T04:11:33.229Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/3f/51/d4db610ef29373b879047326cbf6fa98b6c1969d6f6dc423279de2b1be2c/requests_toolbelt-1.0.0-py2.py3-none-any.whl", hash = "sha256:cccfdd665f0a24fcf4726e690f65639d272bb0637b9b92dfd91a5568ccf6bd06", size = 54481 }, + { url = "https://files.pythonhosted.org/packages/3f/51/d4db610ef29373b879047326cbf6fa98b6c1969d6f6dc423279de2b1be2c/requests_toolbelt-1.0.0-py2.py3-none-any.whl", hash = "sha256:cccfdd665f0a24fcf4726e690f65639d272bb0637b9b92dfd91a5568ccf6bd06", size = 54481, upload-time = "2023-05-01T04:11:28.427Z" }, ] [[package]] @@ -1386,15 +1386,15 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "docutils" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/48/9c/6d8035cafa2d2d314f34e6cd9313a299de095b26e96f1c7312878f988eec/restructuredtext_lint-1.4.0.tar.gz", hash = "sha256:1b235c0c922341ab6c530390892eb9e92f90b9b75046063e047cacfb0f050c45", size = 16723 } +sdist = { url = "https://files.pythonhosted.org/packages/48/9c/6d8035cafa2d2d314f34e6cd9313a299de095b26e96f1c7312878f988eec/restructuredtext_lint-1.4.0.tar.gz", hash = "sha256:1b235c0c922341ab6c530390892eb9e92f90b9b75046063e047cacfb0f050c45", size = 16723, upload-time = "2022-02-24T05:51:10.907Z" } [[package]] name = "rfc3986" version = "2.0.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/85/40/1520d68bfa07ab5a6f065a186815fb6610c86fe957bc065754e47f7b0840/rfc3986-2.0.0.tar.gz", hash = "sha256:97aacf9dbd4bfd829baad6e6309fa6573aaf1be3f6fa735c8ab05e46cecb261c", size = 49026 } +sdist = { url = "https://files.pythonhosted.org/packages/85/40/1520d68bfa07ab5a6f065a186815fb6610c86fe957bc065754e47f7b0840/rfc3986-2.0.0.tar.gz", hash = "sha256:97aacf9dbd4bfd829baad6e6309fa6573aaf1be3f6fa735c8ab05e46cecb261c", size = 49026, upload-time = "2022-01-10T00:52:30.832Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ff/9a/9afaade874b2fa6c752c36f1548f718b5b83af81ed9b76628329dab81c1b/rfc3986-2.0.0-py2.py3-none-any.whl", hash = "sha256:50b1502b60e289cb37883f3dfd34532b8873c7de9f49bb546641ce9cbd256ebd", size = 31326 }, + { url = "https://files.pythonhosted.org/packages/ff/9a/9afaade874b2fa6c752c36f1548f718b5b83af81ed9b76628329dab81c1b/rfc3986-2.0.0-py2.py3-none-any.whl", hash = "sha256:50b1502b60e289cb37883f3dfd34532b8873c7de9f49bb546641ce9cbd256ebd", size = 31326, upload-time = "2022-01-10T00:52:29.594Z" }, ] [[package]] @@ -1405,9 +1405,9 @@ dependencies = [ { name = "markdown-it-py" }, { name = "pygments" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/fe/75/af448d8e52bf1d8fa6a9d089ca6c07ff4453d86c65c145d0a300bb073b9b/rich-14.1.0.tar.gz", hash = "sha256:e497a48b844b0320d45007cdebfeaeed8db2a4f4bcf49f15e455cfc4af11eaa8", size = 224441 } +sdist = { url = "https://files.pythonhosted.org/packages/fe/75/af448d8e52bf1d8fa6a9d089ca6c07ff4453d86c65c145d0a300bb073b9b/rich-14.1.0.tar.gz", hash = "sha256:e497a48b844b0320d45007cdebfeaeed8db2a4f4bcf49f15e455cfc4af11eaa8", size = 224441, upload-time = "2025-07-25T07:32:58.125Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e3/30/3c4d035596d3cf444529e0b2953ad0466f6049528a879d27534700580395/rich-14.1.0-py3-none-any.whl", hash = "sha256:536f5f1785986d6dbdea3c75205c473f970777b4a0d6c6dd1b696aa05a3fa04f", size = 243368 }, + { url = "https://files.pythonhosted.org/packages/e3/30/3c4d035596d3cf444529e0b2953ad0466f6049528a879d27534700580395/rich-14.1.0-py3-none-any.whl", hash = "sha256:536f5f1785986d6dbdea3c75205c473f970777b4a0d6c6dd1b696aa05a3fa04f", size = 243368, upload-time = "2025-07-25T07:32:56.73Z" }, ] [[package]] @@ -1419,144 +1419,144 @@ dependencies = [ { name = "rich" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b7/a8/dcc0a8ec9e91d76ecad9413a84b6d3a3310c6111cfe012d75ed385c78d96/rich_click-1.8.9.tar.gz", hash = "sha256:fd98c0ab9ddc1cf9c0b7463f68daf28b4d0033a74214ceb02f761b3ff2af3136", size = 39378 } +sdist = { url = "https://files.pythonhosted.org/packages/b7/a8/dcc0a8ec9e91d76ecad9413a84b6d3a3310c6111cfe012d75ed385c78d96/rich_click-1.8.9.tar.gz", hash = "sha256:fd98c0ab9ddc1cf9c0b7463f68daf28b4d0033a74214ceb02f761b3ff2af3136", size = 39378, upload-time = "2025-05-19T21:33:05.569Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b6/c2/9fce4c8a9587c4e90500114d742fe8ef0fd92d7bad29d136bb9941add271/rich_click-1.8.9-py3-none-any.whl", hash = "sha256:c3fa81ed8a671a10de65a9e20abf642cfdac6fdb882db1ef465ee33919fbcfe2", size = 36082 }, + { url = "https://files.pythonhosted.org/packages/b6/c2/9fce4c8a9587c4e90500114d742fe8ef0fd92d7bad29d136bb9941add271/rich_click-1.8.9-py3-none-any.whl", hash = "sha256:c3fa81ed8a671a10de65a9e20abf642cfdac6fdb882db1ef465ee33919fbcfe2", size = 36082, upload-time = "2025-05-19T21:33:04.195Z" }, ] [[package]] name = "rpds-py" version = "0.27.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/1e/d9/991a0dee12d9fc53ed027e26a26a64b151d77252ac477e22666b9688bc16/rpds_py-0.27.0.tar.gz", hash = "sha256:8b23cf252f180cda89220b378d917180f29d313cd6a07b2431c0d3b776aae86f", size = 27420 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/75/2d/ad2e37dee3f45580f7fa0066c412a521f9bee53d2718b0e9436d308a1ecd/rpds_py-0.27.0-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:130c1ffa5039a333f5926b09e346ab335f0d4ec393b030a18549a7c7e7c2cea4", size = 371511 }, - { url = "https://files.pythonhosted.org/packages/f5/67/57b4b2479193fde9dd6983a13c2550b5f9c3bcdf8912dffac2068945eb14/rpds_py-0.27.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a4cf32a26fa744101b67bfd28c55d992cd19438aff611a46cac7f066afca8fd4", size = 354718 }, - { url = "https://files.pythonhosted.org/packages/a3/be/c2b95ec4b813eb11f3a3c3d22f22bda8d3a48a074a0519cde968c4d102cf/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64a0fe3f334a40b989812de70160de6b0ec7e3c9e4a04c0bbc48d97c5d3600ae", size = 381518 }, - { url = "https://files.pythonhosted.org/packages/a5/d2/5a7279bc2b93b20bd50865a2269016238cee45f7dc3cc33402a7f41bd447/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9a0ff7ee28583ab30a52f371b40f54e7138c52ca67f8ca17ccb7ccf0b383cb5f", size = 396694 }, - { url = "https://files.pythonhosted.org/packages/65/e9/bac8b3714bd853c5bcb466e04acfb9a5da030d77e0ddf1dfad9afb791c31/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:15ea4d2e182345dd1b4286593601d766411b43f868924afe297570658c31a62b", size = 514813 }, - { url = "https://files.pythonhosted.org/packages/1d/aa/293115e956d7d13b7d2a9e9a4121f74989a427aa125f00ce4426ca8b7b28/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:36184b44bf60a480863e51021c26aca3dfe8dd2f5eeabb33622b132b9d8b8b54", size = 402246 }, - { url = "https://files.pythonhosted.org/packages/88/59/2d6789bb898fb3e2f0f7b82b7bcf27f579ebcb6cc36c24f4e208f7f58a5b/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b78430703cfcf5f5e86eb74027a1ed03a93509273d7c705babb547f03e60016", size = 383661 }, - { url = "https://files.pythonhosted.org/packages/0c/55/add13a593a7a81243a9eed56d618d3d427be5dc1214931676e3f695dfdc1/rpds_py-0.27.0-cp310-cp310-manylinux_2_31_riscv64.whl", hash = "sha256:dbd749cff1defbde270ca346b69b3baf5f1297213ef322254bf2a28537f0b046", size = 401691 }, - { url = "https://files.pythonhosted.org/packages/04/09/3e8b2aad494ffaca571e4e19611a12cc18fcfd756d9274f3871a2d822445/rpds_py-0.27.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6bde37765564cd22a676dd8101b657839a1854cfaa9c382c5abf6ff7accfd4ae", size = 416529 }, - { url = "https://files.pythonhosted.org/packages/a4/6d/bd899234728f1d8f72c9610f50fdf1c140ecd0a141320e1f1d0f6b20595d/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:1d66f45b9399036e890fb9c04e9f70c33857fd8f58ac8db9f3278cfa835440c3", size = 558673 }, - { url = "https://files.pythonhosted.org/packages/79/f4/f3e02def5193fb899d797c232f90d6f8f0f2b9eca2faef6f0d34cbc89b2e/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:d85d784c619370d9329bbd670f41ff5f2ae62ea4519761b679d0f57f0f0ee267", size = 588426 }, - { url = "https://files.pythonhosted.org/packages/e3/0c/88e716cd8fd760e5308835fe298255830de4a1c905fd51760b9bb40aa965/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:5df559e9e7644d9042f626f2c3997b555f347d7a855a15f170b253f6c5bfe358", size = 554552 }, - { url = "https://files.pythonhosted.org/packages/2b/a9/0a8243c182e7ac59b901083dff7e671feba6676a131bfff3f8d301cd2b36/rpds_py-0.27.0-cp310-cp310-win32.whl", hash = "sha256:b8a4131698b6992b2a56015f51646711ec5d893a0b314a4b985477868e240c87", size = 218081 }, - { url = "https://files.pythonhosted.org/packages/0f/e7/202ff35852312760148be9e08fe2ba6900aa28e7a46940a313eae473c10c/rpds_py-0.27.0-cp310-cp310-win_amd64.whl", hash = "sha256:cbc619e84a5e3ab2d452de831c88bdcad824414e9c2d28cd101f94dbdf26329c", size = 230077 }, - { url = "https://files.pythonhosted.org/packages/b4/c1/49d515434c1752e40f5e35b985260cf27af052593378580a2f139a5be6b8/rpds_py-0.27.0-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:dbc2ab5d10544eb485baa76c63c501303b716a5c405ff2469a1d8ceffaabf622", size = 371577 }, - { url = "https://files.pythonhosted.org/packages/e1/6d/bf2715b2fee5087fa13b752b5fd573f1a93e4134c74d275f709e38e54fe7/rpds_py-0.27.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7ec85994f96a58cf7ed288caa344b7fe31fd1d503bdf13d7331ead5f70ab60d5", size = 354959 }, - { url = "https://files.pythonhosted.org/packages/a3/5c/e7762808c746dd19733a81373c10da43926f6a6adcf4920a21119697a60a/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:190d7285cd3bb6d31d37a0534d7359c1ee191eb194c511c301f32a4afa5a1dd4", size = 381485 }, - { url = "https://files.pythonhosted.org/packages/40/51/0d308eb0b558309ca0598bcba4243f52c4cd20e15fe991b5bd75824f2e61/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:c10d92fb6d7fd827e44055fcd932ad93dac6a11e832d51534d77b97d1d85400f", size = 396816 }, - { url = "https://files.pythonhosted.org/packages/5c/aa/2d585ec911d78f66458b2c91252134ca0c7c70f687a72c87283173dc0c96/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dd2c1d27ebfe6a015cfa2005b7fe8c52d5019f7bbdd801bc6f7499aab9ae739e", size = 514950 }, - { url = "https://files.pythonhosted.org/packages/0b/ef/aced551cc1148179557aed84343073adadf252c91265263ee6203458a186/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4790c9d5dd565ddb3e9f656092f57268951398cef52e364c405ed3112dc7c7c1", size = 402132 }, - { url = "https://files.pythonhosted.org/packages/4b/ac/cf644803d8d417653fe2b3604186861d62ea6afaef1b2284045741baef17/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4300e15e7d03660f04be84a125d1bdd0e6b2f674bc0723bc0fd0122f1a4585dc", size = 383660 }, - { url = "https://files.pythonhosted.org/packages/c9/ec/caf47c55ce02b76cbaeeb2d3b36a73da9ca2e14324e3d75cf72b59dcdac5/rpds_py-0.27.0-cp311-cp311-manylinux_2_31_riscv64.whl", hash = "sha256:59195dc244fc183209cf8a93406889cadde47dfd2f0a6b137783aa9c56d67c85", size = 401730 }, - { url = "https://files.pythonhosted.org/packages/0b/71/c1f355afdcd5b99ffc253422aa4bdcb04ccf1491dcd1bda3688a0c07fd61/rpds_py-0.27.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fae4a01ef8c4cb2bbe92ef2063149596907dc4a881a8d26743b3f6b304713171", size = 416122 }, - { url = "https://files.pythonhosted.org/packages/38/0f/f4b5b1eda724ed0e04d2b26d8911cdc131451a7ee4c4c020a1387e5c6ded/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:e3dc8d4ede2dbae6c0fc2b6c958bf51ce9fd7e9b40c0f5b8835c3fde44f5807d", size = 558771 }, - { url = "https://files.pythonhosted.org/packages/93/c0/5f8b834db2289ab48d5cffbecbb75e35410103a77ac0b8da36bf9544ec1c/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c3782fb753aa825b4ccabc04292e07897e2fd941448eabf666856c5530277626", size = 587876 }, - { url = "https://files.pythonhosted.org/packages/d2/dd/1a1df02ab8eb970115cff2ae31a6f73916609b900dc86961dc382b8c2e5e/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:887ab1f12b0d227e9260558a4a2320024b20102207ada65c43e1ffc4546df72e", size = 554359 }, - { url = "https://files.pythonhosted.org/packages/a1/e4/95a014ab0d51ab6e3bebbdb476a42d992d2bbf9c489d24cff9fda998e925/rpds_py-0.27.0-cp311-cp311-win32.whl", hash = "sha256:5d6790ff400254137b81b8053b34417e2c46921e302d655181d55ea46df58cf7", size = 218084 }, - { url = "https://files.pythonhosted.org/packages/49/78/f8d5b71ec65a0376b0de31efcbb5528ce17a9b7fdd19c3763303ccfdedec/rpds_py-0.27.0-cp311-cp311-win_amd64.whl", hash = "sha256:e24d8031a2c62f34853756d9208eeafa6b940a1efcbfe36e8f57d99d52bb7261", size = 230085 }, - { url = "https://files.pythonhosted.org/packages/e7/d3/84429745184091e06b4cc70f8597408e314c2d2f7f5e13249af9ffab9e3d/rpds_py-0.27.0-cp311-cp311-win_arm64.whl", hash = "sha256:08680820d23df1df0a0260f714d12966bc6c42d02e8055a91d61e03f0c47dda0", size = 222112 }, - { url = "https://files.pythonhosted.org/packages/cd/17/e67309ca1ac993fa1888a0d9b2f5ccc1f67196ace32e76c9f8e1dbbbd50c/rpds_py-0.27.0-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:19c990fdf5acecbf0623e906ae2e09ce1c58947197f9bced6bbd7482662231c4", size = 362611 }, - { url = "https://files.pythonhosted.org/packages/93/2e/28c2fb84aa7aa5d75933d1862d0f7de6198ea22dfd9a0cca06e8a4e7509e/rpds_py-0.27.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:6c27a7054b5224710fcfb1a626ec3ff4f28bcb89b899148c72873b18210e446b", size = 347680 }, - { url = "https://files.pythonhosted.org/packages/44/3e/9834b4c8f4f5fe936b479e623832468aa4bd6beb8d014fecaee9eac6cdb1/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09965b314091829b378b60607022048953e25f0b396c2b70e7c4c81bcecf932e", size = 384600 }, - { url = "https://files.pythonhosted.org/packages/19/78/744123c7b38865a965cd9e6f691fde7ef989a00a256fa8bf15b75240d12f/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:14f028eb47f59e9169bfdf9f7ceafd29dd64902141840633683d0bad5b04ff34", size = 400697 }, - { url = "https://files.pythonhosted.org/packages/32/97/3c3d32fe7daee0a1f1a678b6d4dfb8c4dcf88197fa2441f9da7cb54a8466/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6168af0be75bba990a39f9431cdfae5f0ad501f4af32ae62e8856307200517b8", size = 517781 }, - { url = "https://files.pythonhosted.org/packages/b2/be/28f0e3e733680aa13ecec1212fc0f585928a206292f14f89c0b8a684cad1/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ab47fe727c13c09d0e6f508e3a49e545008e23bf762a245b020391b621f5b726", size = 406449 }, - { url = "https://files.pythonhosted.org/packages/95/ae/5d15c83e337c082d0367053baeb40bfba683f42459f6ebff63a2fd7e5518/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5fa01b3d5e3b7d97efab65bd3d88f164e289ec323a8c033c5c38e53ee25c007e", size = 386150 }, - { url = "https://files.pythonhosted.org/packages/bf/65/944e95f95d5931112829e040912b25a77b2e7ed913ea5fe5746aa5c1ce75/rpds_py-0.27.0-cp312-cp312-manylinux_2_31_riscv64.whl", hash = "sha256:6c135708e987f46053e0a1246a206f53717f9fadfba27174a9769ad4befba5c3", size = 406100 }, - { url = "https://files.pythonhosted.org/packages/21/a4/1664b83fae02894533cd11dc0b9f91d673797c2185b7be0f7496107ed6c5/rpds_py-0.27.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fc327f4497b7087d06204235199daf208fd01c82d80465dc5efa4ec9df1c5b4e", size = 421345 }, - { url = "https://files.pythonhosted.org/packages/7c/26/b7303941c2b0823bfb34c71378249f8beedce57301f400acb04bb345d025/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7e57906e38583a2cba67046a09c2637e23297618dc1f3caddbc493f2be97c93f", size = 561891 }, - { url = "https://files.pythonhosted.org/packages/9b/c8/48623d64d4a5a028fa99576c768a6159db49ab907230edddc0b8468b998b/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f4f69d7a4300fbf91efb1fb4916421bd57804c01ab938ab50ac9c4aa2212f03", size = 591756 }, - { url = "https://files.pythonhosted.org/packages/b3/51/18f62617e8e61cc66334c9fb44b1ad7baae3438662098efbc55fb3fda453/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:b4c4fbbcff474e1e5f38be1bf04511c03d492d42eec0babda5d03af3b5589374", size = 557088 }, - { url = "https://files.pythonhosted.org/packages/bd/4c/e84c3a276e2496a93d245516be6b49e20499aa8ca1c94d59fada0d79addc/rpds_py-0.27.0-cp312-cp312-win32.whl", hash = "sha256:27bac29bbbf39601b2aab474daf99dbc8e7176ca3389237a23944b17f8913d97", size = 221926 }, - { url = "https://files.pythonhosted.org/packages/83/89/9d0fbcef64340db0605eb0a0044f258076f3ae0a3b108983b2c614d96212/rpds_py-0.27.0-cp312-cp312-win_amd64.whl", hash = "sha256:8a06aa1197ec0281eb1d7daf6073e199eb832fe591ffa329b88bae28f25f5fe5", size = 233235 }, - { url = "https://files.pythonhosted.org/packages/c9/b0/e177aa9f39cbab060f96de4a09df77d494f0279604dc2f509263e21b05f9/rpds_py-0.27.0-cp312-cp312-win_arm64.whl", hash = "sha256:e14aab02258cb776a108107bd15f5b5e4a1bbaa61ef33b36693dfab6f89d54f9", size = 223315 }, - { url = "https://files.pythonhosted.org/packages/81/d2/dfdfd42565a923b9e5a29f93501664f5b984a802967d48d49200ad71be36/rpds_py-0.27.0-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:443d239d02d9ae55b74015234f2cd8eb09e59fbba30bf60baeb3123ad4c6d5ff", size = 362133 }, - { url = "https://files.pythonhosted.org/packages/ac/4a/0a2e2460c4b66021d349ce9f6331df1d6c75d7eea90df9785d333a49df04/rpds_py-0.27.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:b8a7acf04fda1f30f1007f3cc96d29d8cf0a53e626e4e1655fdf4eabc082d367", size = 347128 }, - { url = "https://files.pythonhosted.org/packages/35/8d/7d1e4390dfe09d4213b3175a3f5a817514355cb3524593380733204f20b9/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9d0f92b78cfc3b74a42239fdd8c1266f4715b573204c234d2f9fc3fc7a24f185", size = 384027 }, - { url = "https://files.pythonhosted.org/packages/c1/65/78499d1a62172891c8cd45de737b2a4b84a414b6ad8315ab3ac4945a5b61/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ce4ed8e0c7dbc5b19352b9c2c6131dd23b95fa8698b5cdd076307a33626b72dc", size = 399973 }, - { url = "https://files.pythonhosted.org/packages/10/a1/1c67c1d8cc889107b19570bb01f75cf49852068e95e6aee80d22915406fc/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:fde355b02934cc6b07200cc3b27ab0c15870a757d1a72fd401aa92e2ea3c6bfe", size = 515295 }, - { url = "https://files.pythonhosted.org/packages/df/27/700ec88e748436b6c7c4a2262d66e80f8c21ab585d5e98c45e02f13f21c0/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13bbc4846ae4c993f07c93feb21a24d8ec637573d567a924b1001e81c8ae80f9", size = 406737 }, - { url = "https://files.pythonhosted.org/packages/33/cc/6b0ee8f0ba3f2df2daac1beda17fde5cf10897a7d466f252bd184ef20162/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:be0744661afbc4099fef7f4e604e7f1ea1be1dd7284f357924af12a705cc7d5c", size = 385898 }, - { url = "https://files.pythonhosted.org/packages/e8/7e/c927b37d7d33c0a0ebf249cc268dc2fcec52864c1b6309ecb960497f2285/rpds_py-0.27.0-cp313-cp313-manylinux_2_31_riscv64.whl", hash = "sha256:069e0384a54f427bd65d7fda83b68a90606a3835901aaff42185fcd94f5a9295", size = 405785 }, - { url = "https://files.pythonhosted.org/packages/5b/d2/8ed50746d909dcf402af3fa58b83d5a590ed43e07251d6b08fad1a535ba6/rpds_py-0.27.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:4bc262ace5a1a7dc3e2eac2fa97b8257ae795389f688b5adf22c5db1e2431c43", size = 419760 }, - { url = "https://files.pythonhosted.org/packages/d3/60/2b2071aee781cb3bd49f94d5d35686990b925e9b9f3e3d149235a6f5d5c1/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:2fe6e18e5c8581f0361b35ae575043c7029d0a92cb3429e6e596c2cdde251432", size = 561201 }, - { url = "https://files.pythonhosted.org/packages/98/1f/27b67304272521aaea02be293fecedce13fa351a4e41cdb9290576fc6d81/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d93ebdb82363d2e7bec64eecdc3632b59e84bd270d74fe5be1659f7787052f9b", size = 591021 }, - { url = "https://files.pythonhosted.org/packages/db/9b/a2fadf823164dd085b1f894be6443b0762a54a7af6f36e98e8fcda69ee50/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:0954e3a92e1d62e83a54ea7b3fdc9efa5d61acef8488a8a3d31fdafbfb00460d", size = 556368 }, - { url = "https://files.pythonhosted.org/packages/24/f3/6d135d46a129cda2e3e6d4c5e91e2cc26ea0428c6cf152763f3f10b6dd05/rpds_py-0.27.0-cp313-cp313-win32.whl", hash = "sha256:2cff9bdd6c7b906cc562a505c04a57d92e82d37200027e8d362518df427f96cd", size = 221236 }, - { url = "https://files.pythonhosted.org/packages/c5/44/65d7494f5448ecc755b545d78b188440f81da98b50ea0447ab5ebfdf9bd6/rpds_py-0.27.0-cp313-cp313-win_amd64.whl", hash = "sha256:dc79d192fb76fc0c84f2c58672c17bbbc383fd26c3cdc29daae16ce3d927e8b2", size = 232634 }, - { url = "https://files.pythonhosted.org/packages/70/d9/23852410fadab2abb611733933401de42a1964ce6600a3badae35fbd573e/rpds_py-0.27.0-cp313-cp313-win_arm64.whl", hash = "sha256:5b3a5c8089eed498a3af23ce87a80805ff98f6ef8f7bdb70bd1b7dae5105f6ac", size = 222783 }, - { url = "https://files.pythonhosted.org/packages/15/75/03447917f78512b34463f4ef11066516067099a0c466545655503bed0c77/rpds_py-0.27.0-cp313-cp313t-macosx_10_12_x86_64.whl", hash = "sha256:90fb790138c1a89a2e58c9282fe1089638401f2f3b8dddd758499041bc6e0774", size = 359154 }, - { url = "https://files.pythonhosted.org/packages/6b/fc/4dac4fa756451f2122ddaf136e2c6aeb758dc6fdbe9ccc4bc95c98451d50/rpds_py-0.27.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:010c4843a3b92b54373e3d2291a7447d6c3fc29f591772cc2ea0e9f5c1da434b", size = 343909 }, - { url = "https://files.pythonhosted.org/packages/7b/81/723c1ed8e6f57ed9d8c0c07578747a2d3d554aaefc1ab89f4e42cfeefa07/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9ce7a9e967afc0a2af7caa0d15a3e9c1054815f73d6a8cb9225b61921b419bd", size = 379340 }, - { url = "https://files.pythonhosted.org/packages/98/16/7e3740413de71818ce1997df82ba5f94bae9fff90c0a578c0e24658e6201/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:aa0bf113d15e8abdfee92aa4db86761b709a09954083afcb5bf0f952d6065fdb", size = 391655 }, - { url = "https://files.pythonhosted.org/packages/e0/63/2a9f510e124d80660f60ecce07953f3f2d5f0b96192c1365443859b9c87f/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eb91d252b35004a84670dfeafadb042528b19842a0080d8b53e5ec1128e8f433", size = 513017 }, - { url = "https://files.pythonhosted.org/packages/2c/4e/cf6ff311d09776c53ea1b4f2e6700b9d43bb4e99551006817ade4bbd6f78/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:db8a6313dbac934193fc17fe7610f70cd8181c542a91382531bef5ed785e5615", size = 402058 }, - { url = "https://files.pythonhosted.org/packages/88/11/5e36096d474cb10f2a2d68b22af60a3bc4164fd8db15078769a568d9d3ac/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ce96ab0bdfcef1b8c371ada2100767ace6804ea35aacce0aef3aeb4f3f499ca8", size = 383474 }, - { url = "https://files.pythonhosted.org/packages/db/a2/3dff02805b06058760b5eaa6d8cb8db3eb3e46c9e452453ad5fc5b5ad9fe/rpds_py-0.27.0-cp313-cp313t-manylinux_2_31_riscv64.whl", hash = "sha256:7451ede3560086abe1aa27dcdcf55cd15c96b56f543fb12e5826eee6f721f858", size = 400067 }, - { url = "https://files.pythonhosted.org/packages/67/87/eed7369b0b265518e21ea836456a4ed4a6744c8c12422ce05bce760bb3cf/rpds_py-0.27.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:32196b5a99821476537b3f7732432d64d93a58d680a52c5e12a190ee0135d8b5", size = 412085 }, - { url = "https://files.pythonhosted.org/packages/8b/48/f50b2ab2fbb422fbb389fe296e70b7a6b5ea31b263ada5c61377e710a924/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:a029be818059870664157194e46ce0e995082ac49926f1423c1f058534d2aaa9", size = 555928 }, - { url = "https://files.pythonhosted.org/packages/98/41/b18eb51045d06887666c3560cd4bbb6819127b43d758f5adb82b5f56f7d1/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3841f66c1ffdc6cebce8aed64e36db71466f1dc23c0d9a5592e2a782a3042c79", size = 585527 }, - { url = "https://files.pythonhosted.org/packages/be/03/a3dd6470fc76499959b00ae56295b76b4bdf7c6ffc60d62006b1217567e1/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:42894616da0fc0dcb2ec08a77896c3f56e9cb2f4b66acd76fc8992c3557ceb1c", size = 554211 }, - { url = "https://files.pythonhosted.org/packages/bf/d1/ee5fd1be395a07423ac4ca0bcc05280bf95db2b155d03adefeb47d5ebf7e/rpds_py-0.27.0-cp313-cp313t-win32.whl", hash = "sha256:b1fef1f13c842a39a03409e30ca0bf87b39a1e2a305a9924deadb75a43105d23", size = 216624 }, - { url = "https://files.pythonhosted.org/packages/1c/94/4814c4c858833bf46706f87349c37ca45e154da7dbbec9ff09f1abeb08cc/rpds_py-0.27.0-cp313-cp313t-win_amd64.whl", hash = "sha256:183f5e221ba3e283cd36fdfbe311d95cd87699a083330b4f792543987167eff1", size = 230007 }, - { url = "https://files.pythonhosted.org/packages/0e/a5/8fffe1c7dc7c055aa02df310f9fb71cfc693a4d5ccc5de2d3456ea5fb022/rpds_py-0.27.0-cp314-cp314-macosx_10_12_x86_64.whl", hash = "sha256:f3cd110e02c5bf17d8fb562f6c9df5c20e73029d587cf8602a2da6c5ef1e32cb", size = 362595 }, - { url = "https://files.pythonhosted.org/packages/bc/c7/4e4253fd2d4bb0edbc0b0b10d9f280612ca4f0f990e3c04c599000fe7d71/rpds_py-0.27.0-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:8d0e09cf4863c74106b5265c2c310f36146e2b445ff7b3018a56799f28f39f6f", size = 347252 }, - { url = "https://files.pythonhosted.org/packages/f3/c8/3d1a954d30f0174dd6baf18b57c215da03cf7846a9d6e0143304e784cddc/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64f689ab822f9b5eb6dfc69893b4b9366db1d2420f7db1f6a2adf2a9ca15ad64", size = 384886 }, - { url = "https://files.pythonhosted.org/packages/e0/52/3c5835f2df389832b28f9276dd5395b5a965cea34226e7c88c8fbec2093c/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:e36c80c49853b3ffda7aa1831bf175c13356b210c73128c861f3aa93c3cc4015", size = 399716 }, - { url = "https://files.pythonhosted.org/packages/40/73/176e46992461a1749686a2a441e24df51ff86b99c2d34bf39f2a5273b987/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6de6a7f622860af0146cb9ee148682ff4d0cea0b8fd3ad51ce4d40efb2f061d0", size = 517030 }, - { url = "https://files.pythonhosted.org/packages/79/2a/7266c75840e8c6e70effeb0d38922a45720904f2cd695e68a0150e5407e2/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4045e2fc4b37ec4b48e8907a5819bdd3380708c139d7cc358f03a3653abedb89", size = 408448 }, - { url = "https://files.pythonhosted.org/packages/e6/5f/a7efc572b8e235093dc6cf39f4dbc8a7f08e65fdbcec7ff4daeb3585eef1/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9da162b718b12c4219eeeeb68a5b7552fbc7aadedf2efee440f88b9c0e54b45d", size = 387320 }, - { url = "https://files.pythonhosted.org/packages/a2/eb/9ff6bc92efe57cf5a2cb74dee20453ba444b6fdc85275d8c99e0d27239d1/rpds_py-0.27.0-cp314-cp314-manylinux_2_31_riscv64.whl", hash = "sha256:0665be515767dc727ffa5f74bd2ef60b0ff85dad6bb8f50d91eaa6b5fb226f51", size = 407414 }, - { url = "https://files.pythonhosted.org/packages/fb/bd/3b9b19b00d5c6e1bd0f418c229ab0f8d3b110ddf7ec5d9d689ef783d0268/rpds_py-0.27.0-cp314-cp314-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:203f581accef67300a942e49a37d74c12ceeef4514874c7cede21b012613ca2c", size = 420766 }, - { url = "https://files.pythonhosted.org/packages/17/6b/521a7b1079ce16258c70805166e3ac6ec4ee2139d023fe07954dc9b2d568/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:7873b65686a6471c0037139aa000d23fe94628e0daaa27b6e40607c90e3f5ec4", size = 562409 }, - { url = "https://files.pythonhosted.org/packages/8b/bf/65db5bfb14ccc55e39de8419a659d05a2a9cd232f0a699a516bb0991da7b/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:249ab91ceaa6b41abc5f19513cb95b45c6f956f6b89f1fe3d99c81255a849f9e", size = 590793 }, - { url = "https://files.pythonhosted.org/packages/db/b8/82d368b378325191ba7aae8f40f009b78057b598d4394d1f2cdabaf67b3f/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:d2f184336bc1d6abfaaa1262ed42739c3789b1e3a65a29916a615307d22ffd2e", size = 558178 }, - { url = "https://files.pythonhosted.org/packages/f6/ff/f270bddbfbc3812500f8131b1ebbd97afd014cd554b604a3f73f03133a36/rpds_py-0.27.0-cp314-cp314-win32.whl", hash = "sha256:d3c622c39f04d5751408f5b801ecb527e6e0a471b367f420a877f7a660d583f6", size = 222355 }, - { url = "https://files.pythonhosted.org/packages/bf/20/fdab055b1460c02ed356a0e0b0a78c1dd32dc64e82a544f7b31c9ac643dc/rpds_py-0.27.0-cp314-cp314-win_amd64.whl", hash = "sha256:cf824aceaeffff029ccfba0da637d432ca71ab21f13e7f6f5179cd88ebc77a8a", size = 234007 }, - { url = "https://files.pythonhosted.org/packages/4d/a8/694c060005421797a3be4943dab8347c76c2b429a9bef68fb2c87c9e70c7/rpds_py-0.27.0-cp314-cp314-win_arm64.whl", hash = "sha256:86aca1616922b40d8ac1b3073a1ead4255a2f13405e5700c01f7c8d29a03972d", size = 223527 }, - { url = "https://files.pythonhosted.org/packages/1e/f9/77f4c90f79d2c5ca8ce6ec6a76cb4734ee247de6b3a4f337e289e1f00372/rpds_py-0.27.0-cp314-cp314t-macosx_10_12_x86_64.whl", hash = "sha256:341d8acb6724c0c17bdf714319c393bb27f6d23d39bc74f94221b3e59fc31828", size = 359469 }, - { url = "https://files.pythonhosted.org/packages/c0/22/b97878d2f1284286fef4172069e84b0b42b546ea7d053e5fb7adb9ac6494/rpds_py-0.27.0-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:6b96b0b784fe5fd03beffff2b1533dc0d85e92bab8d1b2c24ef3a5dc8fac5669", size = 343960 }, - { url = "https://files.pythonhosted.org/packages/b1/b0/dfd55b5bb480eda0578ae94ef256d3061d20b19a0f5e18c482f03e65464f/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0c431bfb91478d7cbe368d0a699978050d3b112d7f1d440a41e90faa325557fd", size = 380201 }, - { url = "https://files.pythonhosted.org/packages/28/22/e1fa64e50d58ad2b2053077e3ec81a979147c43428de9e6de68ddf6aff4e/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:20e222a44ae9f507d0f2678ee3dd0c45ec1e930f6875d99b8459631c24058aec", size = 392111 }, - { url = "https://files.pythonhosted.org/packages/49/f9/43ab7a43e97aedf6cea6af70fdcbe18abbbc41d4ae6cdec1bfc23bbad403/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:184f0d7b342967f6cda94a07d0e1fae177d11d0b8f17d73e06e36ac02889f303", size = 515863 }, - { url = "https://files.pythonhosted.org/packages/38/9b/9bd59dcc636cd04d86a2d20ad967770bf348f5eb5922a8f29b547c074243/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a00c91104c173c9043bc46f7b30ee5e6d2f6b1149f11f545580f5d6fdff42c0b", size = 402398 }, - { url = "https://files.pythonhosted.org/packages/71/bf/f099328c6c85667aba6b66fa5c35a8882db06dcd462ea214be72813a0dd2/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7a37dd208f0d658e0487522078b1ed68cd6bce20ef4b5a915d2809b9094b410", size = 384665 }, - { url = "https://files.pythonhosted.org/packages/a9/c5/9c1f03121ece6634818490bd3c8be2c82a70928a19de03467fb25a3ae2a8/rpds_py-0.27.0-cp314-cp314t-manylinux_2_31_riscv64.whl", hash = "sha256:92f3b3ec3e6008a1fe00b7c0946a170f161ac00645cde35e3c9a68c2475e8156", size = 400405 }, - { url = "https://files.pythonhosted.org/packages/b5/b8/e25d54af3e63ac94f0c16d8fe143779fe71ff209445a0c00d0f6984b6b2c/rpds_py-0.27.0-cp314-cp314t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:a1b3db5fae5cbce2131b7420a3f83553d4d89514c03d67804ced36161fe8b6b2", size = 413179 }, - { url = "https://files.pythonhosted.org/packages/f9/d1/406b3316433fe49c3021546293a04bc33f1478e3ec7950215a7fce1a1208/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:5355527adaa713ab693cbce7c1e0ec71682f599f61b128cf19d07e5c13c9b1f1", size = 556895 }, - { url = "https://files.pythonhosted.org/packages/5f/bc/3697c0c21fcb9a54d46ae3b735eb2365eea0c2be076b8f770f98e07998de/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_i686.whl", hash = "sha256:fcc01c57ce6e70b728af02b2401c5bc853a9e14eb07deda30624374f0aebfe42", size = 585464 }, - { url = "https://files.pythonhosted.org/packages/63/09/ee1bb5536f99f42c839b177d552f6114aa3142d82f49cef49261ed28dbe0/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:3001013dae10f806380ba739d40dee11db1ecb91684febb8406a87c2ded23dae", size = 555090 }, - { url = "https://files.pythonhosted.org/packages/7d/2c/363eada9e89f7059199d3724135a86c47082cbf72790d6ba2f336d146ddb/rpds_py-0.27.0-cp314-cp314t-win32.whl", hash = "sha256:0f401c369186a5743694dd9fc08cba66cf70908757552e1f714bfc5219c655b5", size = 218001 }, - { url = "https://files.pythonhosted.org/packages/e2/3f/d6c216ed5199c9ef79e2a33955601f454ed1e7420a93b89670133bca5ace/rpds_py-0.27.0-cp314-cp314t-win_amd64.whl", hash = "sha256:8a1dca5507fa1337f75dcd5070218b20bc68cf8844271c923c1b79dfcbc20391", size = 230993 }, - { url = "https://files.pythonhosted.org/packages/47/55/287068956f9ba1cb40896d291213f09fdd4527630709058b45a592bc09dc/rpds_py-0.27.0-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:46f48482c1a4748ab2773f75fffbdd1951eb59794e32788834b945da857c47a8", size = 371566 }, - { url = "https://files.pythonhosted.org/packages/a2/fb/443af59cbe552e89680bb0f1d1ba47f6387b92083e28a45b8c8863b86c5a/rpds_py-0.27.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:419dd9c98bcc9fb0242be89e0c6e922df333b975d4268faa90d58499fd9c9ebe", size = 355781 }, - { url = "https://files.pythonhosted.org/packages/ad/f0/35f48bb073b5ca42b1dcc55cb148f4a3bd4411a3e584f6a18d26f0ea8832/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55d42a0ef2bdf6bc81e1cc2d49d12460f63c6ae1423c4f4851b828e454ccf6f1", size = 382575 }, - { url = "https://files.pythonhosted.org/packages/51/e1/5f5296a21d1189f0f116a938af2e346d83172bf814d373695e54004a936f/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:2e39169ac6aae06dd79c07c8a69d9da867cef6a6d7883a0186b46bb46ccfb0c3", size = 397435 }, - { url = "https://files.pythonhosted.org/packages/97/79/3af99b7852b2b55cad8a08863725cbe9dc14781bcf7dc6ecead0c3e1dc54/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:935afcdea4751b0ac918047a2df3f720212892347767aea28f5b3bf7be4f27c0", size = 514861 }, - { url = "https://files.pythonhosted.org/packages/df/3e/11fd6033708ed3ae0e6947bb94f762f56bb46bf59a1b16eef6944e8a62ee/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8de567dec6d451649a781633d36f5c7501711adee329d76c095be2178855b042", size = 402776 }, - { url = "https://files.pythonhosted.org/packages/b7/89/f9375ceaa996116de9cbc949874804c7874d42fb258c384c037a46d730b8/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:555ed147cbe8c8f76e72a4c6cd3b7b761cbf9987891b9448808148204aed74a5", size = 384665 }, - { url = "https://files.pythonhosted.org/packages/48/bf/0061e55c6f1f573a63c0f82306b8984ed3b394adafc66854a936d5db3522/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_31_riscv64.whl", hash = "sha256:d2cc2b34f9e1d31ce255174da82902ad75bd7c0d88a33df54a77a22f2ef421ee", size = 402518 }, - { url = "https://files.pythonhosted.org/packages/ae/dc/8d506676bfe87b3b683332ec8e6ab2b0be118a3d3595ed021e3274a63191/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:cb0702c12983be3b2fab98ead349ac63a98216d28dda6f518f52da5498a27a1b", size = 416247 }, - { url = "https://files.pythonhosted.org/packages/2e/02/9a89eea1b75c69e81632de7963076e455b1e00e1cfb46dfdabb055fa03e3/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:ba783541be46f27c8faea5a6645e193943c17ea2f0ffe593639d906a327a9bcc", size = 559456 }, - { url = "https://files.pythonhosted.org/packages/38/4a/0f3ac4351957847c0d322be6ec72f916e43804a2c1d04e9672ea4a67c315/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:2406d034635d1497c596c40c85f86ecf2bf9611c1df73d14078af8444fe48031", size = 587778 }, - { url = "https://files.pythonhosted.org/packages/c2/8e/39d0d7401095bed5a5ad5ef304fae96383f9bef40ca3f3a0807ff5b68d9d/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:dea0808153f1fbbad772669d906cddd92100277533a03845de6893cadeffc8be", size = 555247 }, - { url = "https://files.pythonhosted.org/packages/e0/04/6b8311e811e620b9eaca67cd80a118ff9159558a719201052a7b2abb88bf/rpds_py-0.27.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:d2a81bdcfde4245468f7030a75a37d50400ac2455c3a4819d9d550c937f90ab5", size = 230256 }, - { url = "https://files.pythonhosted.org/packages/59/64/72ab5b911fdcc48058359b0e786e5363e3fde885156116026f1a2ba9a5b5/rpds_py-0.27.0-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:e6491658dd2569f05860bad645569145c8626ac231877b0fb2d5f9bcb7054089", size = 371658 }, - { url = "https://files.pythonhosted.org/packages/6c/4b/90ff04b4da055db53d8fea57640d8d5d55456343a1ec9a866c0ecfe10fd1/rpds_py-0.27.0-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:bec77545d188f8bdd29d42bccb9191682a46fb2e655e3d1fb446d47c55ac3b8d", size = 355529 }, - { url = "https://files.pythonhosted.org/packages/a4/be/527491fb1afcd86fc5ce5812eb37bc70428ee017d77fee20de18155c3937/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:25a4aebf8ca02bbb90a9b3e7a463bbf3bee02ab1c446840ca07b1695a68ce424", size = 382822 }, - { url = "https://files.pythonhosted.org/packages/e0/a5/dcdb8725ce11e6d0913e6fcf782a13f4b8a517e8acc70946031830b98441/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:44524b96481a4c9b8e6c46d6afe43fa1fb485c261e359fbe32b63ff60e3884d8", size = 397233 }, - { url = "https://files.pythonhosted.org/packages/33/f9/0947920d1927e9f144660590cc38cadb0795d78fe0d9aae0ef71c1513b7c/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:45d04a73c54b6a5fd2bab91a4b5bc8b426949586e61340e212a8484919183859", size = 514892 }, - { url = "https://files.pythonhosted.org/packages/1d/ed/d1343398c1417c68f8daa1afce56ef6ce5cc587daaf98e29347b00a80ff2/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:343cf24de9ed6c728abefc5d5c851d5de06497caa7ac37e5e65dd572921ed1b5", size = 402733 }, - { url = "https://files.pythonhosted.org/packages/1d/0b/646f55442cd14014fb64d143428f25667a100f82092c90087b9ea7101c74/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7aed8118ae20515974650d08eb724150dc2e20c2814bcc307089569995e88a14", size = 384447 }, - { url = "https://files.pythonhosted.org/packages/4b/15/0596ef7529828e33a6c81ecf5013d1dd33a511a3e0be0561f83079cda227/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_31_riscv64.whl", hash = "sha256:af9d4fd79ee1cc8e7caf693ee02737daabfc0fcf2773ca0a4735b356c8ad6f7c", size = 402502 }, - { url = "https://files.pythonhosted.org/packages/c3/8d/986af3c42f8454a6cafff8729d99fb178ae9b08a9816325ac7a8fa57c0c0/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:f0396e894bd1e66c74ecbc08b4f6a03dc331140942c4b1d345dd131b68574a60", size = 416651 }, - { url = "https://files.pythonhosted.org/packages/e9/9a/b4ec3629b7b447e896eec574469159b5b60b7781d3711c914748bf32de05/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:59714ab0a5af25d723d8e9816638faf7f4254234decb7d212715c1aa71eee7be", size = 559460 }, - { url = "https://files.pythonhosted.org/packages/61/63/d1e127b40c3e4733b3a6f26ae7a063cdf2bc1caa5272c89075425c7d397a/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_i686.whl", hash = "sha256:88051c3b7d5325409f433c5a40328fcb0685fc04e5db49ff936e910901d10114", size = 588072 }, - { url = "https://files.pythonhosted.org/packages/04/7e/8ffc71a8f6833d9c9fb999f5b0ee736b8b159fd66968e05c7afc2dbcd57e/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:181bc29e59e5e5e6e9d63b143ff4d5191224d355e246b5a48c88ce6b35c4e466", size = 555083 }, +sdist = { url = "https://files.pythonhosted.org/packages/1e/d9/991a0dee12d9fc53ed027e26a26a64b151d77252ac477e22666b9688bc16/rpds_py-0.27.0.tar.gz", hash = "sha256:8b23cf252f180cda89220b378d917180f29d313cd6a07b2431c0d3b776aae86f", size = 27420, upload-time = "2025-08-07T08:26:39.624Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/75/2d/ad2e37dee3f45580f7fa0066c412a521f9bee53d2718b0e9436d308a1ecd/rpds_py-0.27.0-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:130c1ffa5039a333f5926b09e346ab335f0d4ec393b030a18549a7c7e7c2cea4", size = 371511, upload-time = "2025-08-07T08:23:06.205Z" }, + { url = "https://files.pythonhosted.org/packages/f5/67/57b4b2479193fde9dd6983a13c2550b5f9c3bcdf8912dffac2068945eb14/rpds_py-0.27.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a4cf32a26fa744101b67bfd28c55d992cd19438aff611a46cac7f066afca8fd4", size = 354718, upload-time = "2025-08-07T08:23:08.222Z" }, + { url = "https://files.pythonhosted.org/packages/a3/be/c2b95ec4b813eb11f3a3c3d22f22bda8d3a48a074a0519cde968c4d102cf/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64a0fe3f334a40b989812de70160de6b0ec7e3c9e4a04c0bbc48d97c5d3600ae", size = 381518, upload-time = "2025-08-07T08:23:09.696Z" }, + { url = "https://files.pythonhosted.org/packages/a5/d2/5a7279bc2b93b20bd50865a2269016238cee45f7dc3cc33402a7f41bd447/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9a0ff7ee28583ab30a52f371b40f54e7138c52ca67f8ca17ccb7ccf0b383cb5f", size = 396694, upload-time = "2025-08-07T08:23:11.105Z" }, + { url = "https://files.pythonhosted.org/packages/65/e9/bac8b3714bd853c5bcb466e04acfb9a5da030d77e0ddf1dfad9afb791c31/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:15ea4d2e182345dd1b4286593601d766411b43f868924afe297570658c31a62b", size = 514813, upload-time = "2025-08-07T08:23:12.215Z" }, + { url = "https://files.pythonhosted.org/packages/1d/aa/293115e956d7d13b7d2a9e9a4121f74989a427aa125f00ce4426ca8b7b28/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:36184b44bf60a480863e51021c26aca3dfe8dd2f5eeabb33622b132b9d8b8b54", size = 402246, upload-time = "2025-08-07T08:23:13.699Z" }, + { url = "https://files.pythonhosted.org/packages/88/59/2d6789bb898fb3e2f0f7b82b7bcf27f579ebcb6cc36c24f4e208f7f58a5b/rpds_py-0.27.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b78430703cfcf5f5e86eb74027a1ed03a93509273d7c705babb547f03e60016", size = 383661, upload-time = "2025-08-07T08:23:15.231Z" }, + { url = "https://files.pythonhosted.org/packages/0c/55/add13a593a7a81243a9eed56d618d3d427be5dc1214931676e3f695dfdc1/rpds_py-0.27.0-cp310-cp310-manylinux_2_31_riscv64.whl", hash = "sha256:dbd749cff1defbde270ca346b69b3baf5f1297213ef322254bf2a28537f0b046", size = 401691, upload-time = "2025-08-07T08:23:16.681Z" }, + { url = "https://files.pythonhosted.org/packages/04/09/3e8b2aad494ffaca571e4e19611a12cc18fcfd756d9274f3871a2d822445/rpds_py-0.27.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6bde37765564cd22a676dd8101b657839a1854cfaa9c382c5abf6ff7accfd4ae", size = 416529, upload-time = "2025-08-07T08:23:17.863Z" }, + { url = "https://files.pythonhosted.org/packages/a4/6d/bd899234728f1d8f72c9610f50fdf1c140ecd0a141320e1f1d0f6b20595d/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:1d66f45b9399036e890fb9c04e9f70c33857fd8f58ac8db9f3278cfa835440c3", size = 558673, upload-time = "2025-08-07T08:23:18.99Z" }, + { url = "https://files.pythonhosted.org/packages/79/f4/f3e02def5193fb899d797c232f90d6f8f0f2b9eca2faef6f0d34cbc89b2e/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:d85d784c619370d9329bbd670f41ff5f2ae62ea4519761b679d0f57f0f0ee267", size = 588426, upload-time = "2025-08-07T08:23:20.541Z" }, + { url = "https://files.pythonhosted.org/packages/e3/0c/88e716cd8fd760e5308835fe298255830de4a1c905fd51760b9bb40aa965/rpds_py-0.27.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:5df559e9e7644d9042f626f2c3997b555f347d7a855a15f170b253f6c5bfe358", size = 554552, upload-time = "2025-08-07T08:23:21.714Z" }, + { url = "https://files.pythonhosted.org/packages/2b/a9/0a8243c182e7ac59b901083dff7e671feba6676a131bfff3f8d301cd2b36/rpds_py-0.27.0-cp310-cp310-win32.whl", hash = "sha256:b8a4131698b6992b2a56015f51646711ec5d893a0b314a4b985477868e240c87", size = 218081, upload-time = "2025-08-07T08:23:23.273Z" }, + { url = "https://files.pythonhosted.org/packages/0f/e7/202ff35852312760148be9e08fe2ba6900aa28e7a46940a313eae473c10c/rpds_py-0.27.0-cp310-cp310-win_amd64.whl", hash = "sha256:cbc619e84a5e3ab2d452de831c88bdcad824414e9c2d28cd101f94dbdf26329c", size = 230077, upload-time = "2025-08-07T08:23:24.308Z" }, + { url = "https://files.pythonhosted.org/packages/b4/c1/49d515434c1752e40f5e35b985260cf27af052593378580a2f139a5be6b8/rpds_py-0.27.0-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:dbc2ab5d10544eb485baa76c63c501303b716a5c405ff2469a1d8ceffaabf622", size = 371577, upload-time = "2025-08-07T08:23:25.379Z" }, + { url = "https://files.pythonhosted.org/packages/e1/6d/bf2715b2fee5087fa13b752b5fd573f1a93e4134c74d275f709e38e54fe7/rpds_py-0.27.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7ec85994f96a58cf7ed288caa344b7fe31fd1d503bdf13d7331ead5f70ab60d5", size = 354959, upload-time = "2025-08-07T08:23:26.767Z" }, + { url = "https://files.pythonhosted.org/packages/a3/5c/e7762808c746dd19733a81373c10da43926f6a6adcf4920a21119697a60a/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:190d7285cd3bb6d31d37a0534d7359c1ee191eb194c511c301f32a4afa5a1dd4", size = 381485, upload-time = "2025-08-07T08:23:27.869Z" }, + { url = "https://files.pythonhosted.org/packages/40/51/0d308eb0b558309ca0598bcba4243f52c4cd20e15fe991b5bd75824f2e61/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:c10d92fb6d7fd827e44055fcd932ad93dac6a11e832d51534d77b97d1d85400f", size = 396816, upload-time = "2025-08-07T08:23:29.424Z" }, + { url = "https://files.pythonhosted.org/packages/5c/aa/2d585ec911d78f66458b2c91252134ca0c7c70f687a72c87283173dc0c96/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dd2c1d27ebfe6a015cfa2005b7fe8c52d5019f7bbdd801bc6f7499aab9ae739e", size = 514950, upload-time = "2025-08-07T08:23:30.576Z" }, + { url = "https://files.pythonhosted.org/packages/0b/ef/aced551cc1148179557aed84343073adadf252c91265263ee6203458a186/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4790c9d5dd565ddb3e9f656092f57268951398cef52e364c405ed3112dc7c7c1", size = 402132, upload-time = "2025-08-07T08:23:32.428Z" }, + { url = "https://files.pythonhosted.org/packages/4b/ac/cf644803d8d417653fe2b3604186861d62ea6afaef1b2284045741baef17/rpds_py-0.27.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4300e15e7d03660f04be84a125d1bdd0e6b2f674bc0723bc0fd0122f1a4585dc", size = 383660, upload-time = "2025-08-07T08:23:33.829Z" }, + { url = "https://files.pythonhosted.org/packages/c9/ec/caf47c55ce02b76cbaeeb2d3b36a73da9ca2e14324e3d75cf72b59dcdac5/rpds_py-0.27.0-cp311-cp311-manylinux_2_31_riscv64.whl", hash = "sha256:59195dc244fc183209cf8a93406889cadde47dfd2f0a6b137783aa9c56d67c85", size = 401730, upload-time = "2025-08-07T08:23:34.97Z" }, + { url = "https://files.pythonhosted.org/packages/0b/71/c1f355afdcd5b99ffc253422aa4bdcb04ccf1491dcd1bda3688a0c07fd61/rpds_py-0.27.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fae4a01ef8c4cb2bbe92ef2063149596907dc4a881a8d26743b3f6b304713171", size = 416122, upload-time = "2025-08-07T08:23:36.062Z" }, + { url = "https://files.pythonhosted.org/packages/38/0f/f4b5b1eda724ed0e04d2b26d8911cdc131451a7ee4c4c020a1387e5c6ded/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:e3dc8d4ede2dbae6c0fc2b6c958bf51ce9fd7e9b40c0f5b8835c3fde44f5807d", size = 558771, upload-time = "2025-08-07T08:23:37.478Z" }, + { url = "https://files.pythonhosted.org/packages/93/c0/5f8b834db2289ab48d5cffbecbb75e35410103a77ac0b8da36bf9544ec1c/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c3782fb753aa825b4ccabc04292e07897e2fd941448eabf666856c5530277626", size = 587876, upload-time = "2025-08-07T08:23:38.662Z" }, + { url = "https://files.pythonhosted.org/packages/d2/dd/1a1df02ab8eb970115cff2ae31a6f73916609b900dc86961dc382b8c2e5e/rpds_py-0.27.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:887ab1f12b0d227e9260558a4a2320024b20102207ada65c43e1ffc4546df72e", size = 554359, upload-time = "2025-08-07T08:23:39.897Z" }, + { url = "https://files.pythonhosted.org/packages/a1/e4/95a014ab0d51ab6e3bebbdb476a42d992d2bbf9c489d24cff9fda998e925/rpds_py-0.27.0-cp311-cp311-win32.whl", hash = "sha256:5d6790ff400254137b81b8053b34417e2c46921e302d655181d55ea46df58cf7", size = 218084, upload-time = "2025-08-07T08:23:41.086Z" }, + { url = "https://files.pythonhosted.org/packages/49/78/f8d5b71ec65a0376b0de31efcbb5528ce17a9b7fdd19c3763303ccfdedec/rpds_py-0.27.0-cp311-cp311-win_amd64.whl", hash = "sha256:e24d8031a2c62f34853756d9208eeafa6b940a1efcbfe36e8f57d99d52bb7261", size = 230085, upload-time = "2025-08-07T08:23:42.143Z" }, + { url = "https://files.pythonhosted.org/packages/e7/d3/84429745184091e06b4cc70f8597408e314c2d2f7f5e13249af9ffab9e3d/rpds_py-0.27.0-cp311-cp311-win_arm64.whl", hash = "sha256:08680820d23df1df0a0260f714d12966bc6c42d02e8055a91d61e03f0c47dda0", size = 222112, upload-time = "2025-08-07T08:23:43.233Z" }, + { url = "https://files.pythonhosted.org/packages/cd/17/e67309ca1ac993fa1888a0d9b2f5ccc1f67196ace32e76c9f8e1dbbbd50c/rpds_py-0.27.0-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:19c990fdf5acecbf0623e906ae2e09ce1c58947197f9bced6bbd7482662231c4", size = 362611, upload-time = "2025-08-07T08:23:44.773Z" }, + { url = "https://files.pythonhosted.org/packages/93/2e/28c2fb84aa7aa5d75933d1862d0f7de6198ea22dfd9a0cca06e8a4e7509e/rpds_py-0.27.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:6c27a7054b5224710fcfb1a626ec3ff4f28bcb89b899148c72873b18210e446b", size = 347680, upload-time = "2025-08-07T08:23:46.014Z" }, + { url = "https://files.pythonhosted.org/packages/44/3e/9834b4c8f4f5fe936b479e623832468aa4bd6beb8d014fecaee9eac6cdb1/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09965b314091829b378b60607022048953e25f0b396c2b70e7c4c81bcecf932e", size = 384600, upload-time = "2025-08-07T08:23:48Z" }, + { url = "https://files.pythonhosted.org/packages/19/78/744123c7b38865a965cd9e6f691fde7ef989a00a256fa8bf15b75240d12f/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:14f028eb47f59e9169bfdf9f7ceafd29dd64902141840633683d0bad5b04ff34", size = 400697, upload-time = "2025-08-07T08:23:49.407Z" }, + { url = "https://files.pythonhosted.org/packages/32/97/3c3d32fe7daee0a1f1a678b6d4dfb8c4dcf88197fa2441f9da7cb54a8466/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6168af0be75bba990a39f9431cdfae5f0ad501f4af32ae62e8856307200517b8", size = 517781, upload-time = "2025-08-07T08:23:50.557Z" }, + { url = "https://files.pythonhosted.org/packages/b2/be/28f0e3e733680aa13ecec1212fc0f585928a206292f14f89c0b8a684cad1/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ab47fe727c13c09d0e6f508e3a49e545008e23bf762a245b020391b621f5b726", size = 406449, upload-time = "2025-08-07T08:23:51.732Z" }, + { url = "https://files.pythonhosted.org/packages/95/ae/5d15c83e337c082d0367053baeb40bfba683f42459f6ebff63a2fd7e5518/rpds_py-0.27.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5fa01b3d5e3b7d97efab65bd3d88f164e289ec323a8c033c5c38e53ee25c007e", size = 386150, upload-time = "2025-08-07T08:23:52.822Z" }, + { url = "https://files.pythonhosted.org/packages/bf/65/944e95f95d5931112829e040912b25a77b2e7ed913ea5fe5746aa5c1ce75/rpds_py-0.27.0-cp312-cp312-manylinux_2_31_riscv64.whl", hash = "sha256:6c135708e987f46053e0a1246a206f53717f9fadfba27174a9769ad4befba5c3", size = 406100, upload-time = "2025-08-07T08:23:54.339Z" }, + { url = "https://files.pythonhosted.org/packages/21/a4/1664b83fae02894533cd11dc0b9f91d673797c2185b7be0f7496107ed6c5/rpds_py-0.27.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fc327f4497b7087d06204235199daf208fd01c82d80465dc5efa4ec9df1c5b4e", size = 421345, upload-time = "2025-08-07T08:23:55.832Z" }, + { url = "https://files.pythonhosted.org/packages/7c/26/b7303941c2b0823bfb34c71378249f8beedce57301f400acb04bb345d025/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7e57906e38583a2cba67046a09c2637e23297618dc1f3caddbc493f2be97c93f", size = 561891, upload-time = "2025-08-07T08:23:56.951Z" }, + { url = "https://files.pythonhosted.org/packages/9b/c8/48623d64d4a5a028fa99576c768a6159db49ab907230edddc0b8468b998b/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f4f69d7a4300fbf91efb1fb4916421bd57804c01ab938ab50ac9c4aa2212f03", size = 591756, upload-time = "2025-08-07T08:23:58.146Z" }, + { url = "https://files.pythonhosted.org/packages/b3/51/18f62617e8e61cc66334c9fb44b1ad7baae3438662098efbc55fb3fda453/rpds_py-0.27.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:b4c4fbbcff474e1e5f38be1bf04511c03d492d42eec0babda5d03af3b5589374", size = 557088, upload-time = "2025-08-07T08:23:59.6Z" }, + { url = "https://files.pythonhosted.org/packages/bd/4c/e84c3a276e2496a93d245516be6b49e20499aa8ca1c94d59fada0d79addc/rpds_py-0.27.0-cp312-cp312-win32.whl", hash = "sha256:27bac29bbbf39601b2aab474daf99dbc8e7176ca3389237a23944b17f8913d97", size = 221926, upload-time = "2025-08-07T08:24:00.695Z" }, + { url = "https://files.pythonhosted.org/packages/83/89/9d0fbcef64340db0605eb0a0044f258076f3ae0a3b108983b2c614d96212/rpds_py-0.27.0-cp312-cp312-win_amd64.whl", hash = "sha256:8a06aa1197ec0281eb1d7daf6073e199eb832fe591ffa329b88bae28f25f5fe5", size = 233235, upload-time = "2025-08-07T08:24:01.846Z" }, + { url = "https://files.pythonhosted.org/packages/c9/b0/e177aa9f39cbab060f96de4a09df77d494f0279604dc2f509263e21b05f9/rpds_py-0.27.0-cp312-cp312-win_arm64.whl", hash = "sha256:e14aab02258cb776a108107bd15f5b5e4a1bbaa61ef33b36693dfab6f89d54f9", size = 223315, upload-time = "2025-08-07T08:24:03.337Z" }, + { url = "https://files.pythonhosted.org/packages/81/d2/dfdfd42565a923b9e5a29f93501664f5b984a802967d48d49200ad71be36/rpds_py-0.27.0-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:443d239d02d9ae55b74015234f2cd8eb09e59fbba30bf60baeb3123ad4c6d5ff", size = 362133, upload-time = "2025-08-07T08:24:04.508Z" }, + { url = "https://files.pythonhosted.org/packages/ac/4a/0a2e2460c4b66021d349ce9f6331df1d6c75d7eea90df9785d333a49df04/rpds_py-0.27.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:b8a7acf04fda1f30f1007f3cc96d29d8cf0a53e626e4e1655fdf4eabc082d367", size = 347128, upload-time = "2025-08-07T08:24:05.695Z" }, + { url = "https://files.pythonhosted.org/packages/35/8d/7d1e4390dfe09d4213b3175a3f5a817514355cb3524593380733204f20b9/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9d0f92b78cfc3b74a42239fdd8c1266f4715b573204c234d2f9fc3fc7a24f185", size = 384027, upload-time = "2025-08-07T08:24:06.841Z" }, + { url = "https://files.pythonhosted.org/packages/c1/65/78499d1a62172891c8cd45de737b2a4b84a414b6ad8315ab3ac4945a5b61/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ce4ed8e0c7dbc5b19352b9c2c6131dd23b95fa8698b5cdd076307a33626b72dc", size = 399973, upload-time = "2025-08-07T08:24:08.143Z" }, + { url = "https://files.pythonhosted.org/packages/10/a1/1c67c1d8cc889107b19570bb01f75cf49852068e95e6aee80d22915406fc/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:fde355b02934cc6b07200cc3b27ab0c15870a757d1a72fd401aa92e2ea3c6bfe", size = 515295, upload-time = "2025-08-07T08:24:09.711Z" }, + { url = "https://files.pythonhosted.org/packages/df/27/700ec88e748436b6c7c4a2262d66e80f8c21ab585d5e98c45e02f13f21c0/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13bbc4846ae4c993f07c93feb21a24d8ec637573d567a924b1001e81c8ae80f9", size = 406737, upload-time = "2025-08-07T08:24:11.182Z" }, + { url = "https://files.pythonhosted.org/packages/33/cc/6b0ee8f0ba3f2df2daac1beda17fde5cf10897a7d466f252bd184ef20162/rpds_py-0.27.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:be0744661afbc4099fef7f4e604e7f1ea1be1dd7284f357924af12a705cc7d5c", size = 385898, upload-time = "2025-08-07T08:24:12.798Z" }, + { url = "https://files.pythonhosted.org/packages/e8/7e/c927b37d7d33c0a0ebf249cc268dc2fcec52864c1b6309ecb960497f2285/rpds_py-0.27.0-cp313-cp313-manylinux_2_31_riscv64.whl", hash = "sha256:069e0384a54f427bd65d7fda83b68a90606a3835901aaff42185fcd94f5a9295", size = 405785, upload-time = "2025-08-07T08:24:14.906Z" }, + { url = "https://files.pythonhosted.org/packages/5b/d2/8ed50746d909dcf402af3fa58b83d5a590ed43e07251d6b08fad1a535ba6/rpds_py-0.27.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:4bc262ace5a1a7dc3e2eac2fa97b8257ae795389f688b5adf22c5db1e2431c43", size = 419760, upload-time = "2025-08-07T08:24:16.129Z" }, + { url = "https://files.pythonhosted.org/packages/d3/60/2b2071aee781cb3bd49f94d5d35686990b925e9b9f3e3d149235a6f5d5c1/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:2fe6e18e5c8581f0361b35ae575043c7029d0a92cb3429e6e596c2cdde251432", size = 561201, upload-time = "2025-08-07T08:24:17.645Z" }, + { url = "https://files.pythonhosted.org/packages/98/1f/27b67304272521aaea02be293fecedce13fa351a4e41cdb9290576fc6d81/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d93ebdb82363d2e7bec64eecdc3632b59e84bd270d74fe5be1659f7787052f9b", size = 591021, upload-time = "2025-08-07T08:24:18.999Z" }, + { url = "https://files.pythonhosted.org/packages/db/9b/a2fadf823164dd085b1f894be6443b0762a54a7af6f36e98e8fcda69ee50/rpds_py-0.27.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:0954e3a92e1d62e83a54ea7b3fdc9efa5d61acef8488a8a3d31fdafbfb00460d", size = 556368, upload-time = "2025-08-07T08:24:20.54Z" }, + { url = "https://files.pythonhosted.org/packages/24/f3/6d135d46a129cda2e3e6d4c5e91e2cc26ea0428c6cf152763f3f10b6dd05/rpds_py-0.27.0-cp313-cp313-win32.whl", hash = "sha256:2cff9bdd6c7b906cc562a505c04a57d92e82d37200027e8d362518df427f96cd", size = 221236, upload-time = "2025-08-07T08:24:22.144Z" }, + { url = "https://files.pythonhosted.org/packages/c5/44/65d7494f5448ecc755b545d78b188440f81da98b50ea0447ab5ebfdf9bd6/rpds_py-0.27.0-cp313-cp313-win_amd64.whl", hash = "sha256:dc79d192fb76fc0c84f2c58672c17bbbc383fd26c3cdc29daae16ce3d927e8b2", size = 232634, upload-time = "2025-08-07T08:24:23.642Z" }, + { url = "https://files.pythonhosted.org/packages/70/d9/23852410fadab2abb611733933401de42a1964ce6600a3badae35fbd573e/rpds_py-0.27.0-cp313-cp313-win_arm64.whl", hash = "sha256:5b3a5c8089eed498a3af23ce87a80805ff98f6ef8f7bdb70bd1b7dae5105f6ac", size = 222783, upload-time = "2025-08-07T08:24:25.098Z" }, + { url = "https://files.pythonhosted.org/packages/15/75/03447917f78512b34463f4ef11066516067099a0c466545655503bed0c77/rpds_py-0.27.0-cp313-cp313t-macosx_10_12_x86_64.whl", hash = "sha256:90fb790138c1a89a2e58c9282fe1089638401f2f3b8dddd758499041bc6e0774", size = 359154, upload-time = "2025-08-07T08:24:26.249Z" }, + { url = "https://files.pythonhosted.org/packages/6b/fc/4dac4fa756451f2122ddaf136e2c6aeb758dc6fdbe9ccc4bc95c98451d50/rpds_py-0.27.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:010c4843a3b92b54373e3d2291a7447d6c3fc29f591772cc2ea0e9f5c1da434b", size = 343909, upload-time = "2025-08-07T08:24:27.405Z" }, + { url = "https://files.pythonhosted.org/packages/7b/81/723c1ed8e6f57ed9d8c0c07578747a2d3d554aaefc1ab89f4e42cfeefa07/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9ce7a9e967afc0a2af7caa0d15a3e9c1054815f73d6a8cb9225b61921b419bd", size = 379340, upload-time = "2025-08-07T08:24:28.714Z" }, + { url = "https://files.pythonhosted.org/packages/98/16/7e3740413de71818ce1997df82ba5f94bae9fff90c0a578c0e24658e6201/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:aa0bf113d15e8abdfee92aa4db86761b709a09954083afcb5bf0f952d6065fdb", size = 391655, upload-time = "2025-08-07T08:24:30.223Z" }, + { url = "https://files.pythonhosted.org/packages/e0/63/2a9f510e124d80660f60ecce07953f3f2d5f0b96192c1365443859b9c87f/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eb91d252b35004a84670dfeafadb042528b19842a0080d8b53e5ec1128e8f433", size = 513017, upload-time = "2025-08-07T08:24:31.446Z" }, + { url = "https://files.pythonhosted.org/packages/2c/4e/cf6ff311d09776c53ea1b4f2e6700b9d43bb4e99551006817ade4bbd6f78/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:db8a6313dbac934193fc17fe7610f70cd8181c542a91382531bef5ed785e5615", size = 402058, upload-time = "2025-08-07T08:24:32.613Z" }, + { url = "https://files.pythonhosted.org/packages/88/11/5e36096d474cb10f2a2d68b22af60a3bc4164fd8db15078769a568d9d3ac/rpds_py-0.27.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ce96ab0bdfcef1b8c371ada2100767ace6804ea35aacce0aef3aeb4f3f499ca8", size = 383474, upload-time = "2025-08-07T08:24:33.767Z" }, + { url = "https://files.pythonhosted.org/packages/db/a2/3dff02805b06058760b5eaa6d8cb8db3eb3e46c9e452453ad5fc5b5ad9fe/rpds_py-0.27.0-cp313-cp313t-manylinux_2_31_riscv64.whl", hash = "sha256:7451ede3560086abe1aa27dcdcf55cd15c96b56f543fb12e5826eee6f721f858", size = 400067, upload-time = "2025-08-07T08:24:35.021Z" }, + { url = "https://files.pythonhosted.org/packages/67/87/eed7369b0b265518e21ea836456a4ed4a6744c8c12422ce05bce760bb3cf/rpds_py-0.27.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:32196b5a99821476537b3f7732432d64d93a58d680a52c5e12a190ee0135d8b5", size = 412085, upload-time = "2025-08-07T08:24:36.267Z" }, + { url = "https://files.pythonhosted.org/packages/8b/48/f50b2ab2fbb422fbb389fe296e70b7a6b5ea31b263ada5c61377e710a924/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:a029be818059870664157194e46ce0e995082ac49926f1423c1f058534d2aaa9", size = 555928, upload-time = "2025-08-07T08:24:37.573Z" }, + { url = "https://files.pythonhosted.org/packages/98/41/b18eb51045d06887666c3560cd4bbb6819127b43d758f5adb82b5f56f7d1/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3841f66c1ffdc6cebce8aed64e36db71466f1dc23c0d9a5592e2a782a3042c79", size = 585527, upload-time = "2025-08-07T08:24:39.391Z" }, + { url = "https://files.pythonhosted.org/packages/be/03/a3dd6470fc76499959b00ae56295b76b4bdf7c6ffc60d62006b1217567e1/rpds_py-0.27.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:42894616da0fc0dcb2ec08a77896c3f56e9cb2f4b66acd76fc8992c3557ceb1c", size = 554211, upload-time = "2025-08-07T08:24:40.6Z" }, + { url = "https://files.pythonhosted.org/packages/bf/d1/ee5fd1be395a07423ac4ca0bcc05280bf95db2b155d03adefeb47d5ebf7e/rpds_py-0.27.0-cp313-cp313t-win32.whl", hash = "sha256:b1fef1f13c842a39a03409e30ca0bf87b39a1e2a305a9924deadb75a43105d23", size = 216624, upload-time = "2025-08-07T08:24:42.204Z" }, + { url = "https://files.pythonhosted.org/packages/1c/94/4814c4c858833bf46706f87349c37ca45e154da7dbbec9ff09f1abeb08cc/rpds_py-0.27.0-cp313-cp313t-win_amd64.whl", hash = "sha256:183f5e221ba3e283cd36fdfbe311d95cd87699a083330b4f792543987167eff1", size = 230007, upload-time = "2025-08-07T08:24:43.329Z" }, + { url = "https://files.pythonhosted.org/packages/0e/a5/8fffe1c7dc7c055aa02df310f9fb71cfc693a4d5ccc5de2d3456ea5fb022/rpds_py-0.27.0-cp314-cp314-macosx_10_12_x86_64.whl", hash = "sha256:f3cd110e02c5bf17d8fb562f6c9df5c20e73029d587cf8602a2da6c5ef1e32cb", size = 362595, upload-time = "2025-08-07T08:24:44.478Z" }, + { url = "https://files.pythonhosted.org/packages/bc/c7/4e4253fd2d4bb0edbc0b0b10d9f280612ca4f0f990e3c04c599000fe7d71/rpds_py-0.27.0-cp314-cp314-macosx_11_0_arm64.whl", hash = "sha256:8d0e09cf4863c74106b5265c2c310f36146e2b445ff7b3018a56799f28f39f6f", size = 347252, upload-time = "2025-08-07T08:24:45.678Z" }, + { url = "https://files.pythonhosted.org/packages/f3/c8/3d1a954d30f0174dd6baf18b57c215da03cf7846a9d6e0143304e784cddc/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64f689ab822f9b5eb6dfc69893b4b9366db1d2420f7db1f6a2adf2a9ca15ad64", size = 384886, upload-time = "2025-08-07T08:24:46.86Z" }, + { url = "https://files.pythonhosted.org/packages/e0/52/3c5835f2df389832b28f9276dd5395b5a965cea34226e7c88c8fbec2093c/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:e36c80c49853b3ffda7aa1831bf175c13356b210c73128c861f3aa93c3cc4015", size = 399716, upload-time = "2025-08-07T08:24:48.174Z" }, + { url = "https://files.pythonhosted.org/packages/40/73/176e46992461a1749686a2a441e24df51ff86b99c2d34bf39f2a5273b987/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6de6a7f622860af0146cb9ee148682ff4d0cea0b8fd3ad51ce4d40efb2f061d0", size = 517030, upload-time = "2025-08-07T08:24:49.52Z" }, + { url = "https://files.pythonhosted.org/packages/79/2a/7266c75840e8c6e70effeb0d38922a45720904f2cd695e68a0150e5407e2/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4045e2fc4b37ec4b48e8907a5819bdd3380708c139d7cc358f03a3653abedb89", size = 408448, upload-time = "2025-08-07T08:24:50.727Z" }, + { url = "https://files.pythonhosted.org/packages/e6/5f/a7efc572b8e235093dc6cf39f4dbc8a7f08e65fdbcec7ff4daeb3585eef1/rpds_py-0.27.0-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9da162b718b12c4219eeeeb68a5b7552fbc7aadedf2efee440f88b9c0e54b45d", size = 387320, upload-time = "2025-08-07T08:24:52.004Z" }, + { url = "https://files.pythonhosted.org/packages/a2/eb/9ff6bc92efe57cf5a2cb74dee20453ba444b6fdc85275d8c99e0d27239d1/rpds_py-0.27.0-cp314-cp314-manylinux_2_31_riscv64.whl", hash = "sha256:0665be515767dc727ffa5f74bd2ef60b0ff85dad6bb8f50d91eaa6b5fb226f51", size = 407414, upload-time = "2025-08-07T08:24:53.664Z" }, + { url = "https://files.pythonhosted.org/packages/fb/bd/3b9b19b00d5c6e1bd0f418c229ab0f8d3b110ddf7ec5d9d689ef783d0268/rpds_py-0.27.0-cp314-cp314-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:203f581accef67300a942e49a37d74c12ceeef4514874c7cede21b012613ca2c", size = 420766, upload-time = "2025-08-07T08:24:55.917Z" }, + { url = "https://files.pythonhosted.org/packages/17/6b/521a7b1079ce16258c70805166e3ac6ec4ee2139d023fe07954dc9b2d568/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:7873b65686a6471c0037139aa000d23fe94628e0daaa27b6e40607c90e3f5ec4", size = 562409, upload-time = "2025-08-07T08:24:57.17Z" }, + { url = "https://files.pythonhosted.org/packages/8b/bf/65db5bfb14ccc55e39de8419a659d05a2a9cd232f0a699a516bb0991da7b/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:249ab91ceaa6b41abc5f19513cb95b45c6f956f6b89f1fe3d99c81255a849f9e", size = 590793, upload-time = "2025-08-07T08:24:58.388Z" }, + { url = "https://files.pythonhosted.org/packages/db/b8/82d368b378325191ba7aae8f40f009b78057b598d4394d1f2cdabaf67b3f/rpds_py-0.27.0-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:d2f184336bc1d6abfaaa1262ed42739c3789b1e3a65a29916a615307d22ffd2e", size = 558178, upload-time = "2025-08-07T08:24:59.756Z" }, + { url = "https://files.pythonhosted.org/packages/f6/ff/f270bddbfbc3812500f8131b1ebbd97afd014cd554b604a3f73f03133a36/rpds_py-0.27.0-cp314-cp314-win32.whl", hash = "sha256:d3c622c39f04d5751408f5b801ecb527e6e0a471b367f420a877f7a660d583f6", size = 222355, upload-time = "2025-08-07T08:25:01.027Z" }, + { url = "https://files.pythonhosted.org/packages/bf/20/fdab055b1460c02ed356a0e0b0a78c1dd32dc64e82a544f7b31c9ac643dc/rpds_py-0.27.0-cp314-cp314-win_amd64.whl", hash = "sha256:cf824aceaeffff029ccfba0da637d432ca71ab21f13e7f6f5179cd88ebc77a8a", size = 234007, upload-time = "2025-08-07T08:25:02.268Z" }, + { url = "https://files.pythonhosted.org/packages/4d/a8/694c060005421797a3be4943dab8347c76c2b429a9bef68fb2c87c9e70c7/rpds_py-0.27.0-cp314-cp314-win_arm64.whl", hash = "sha256:86aca1616922b40d8ac1b3073a1ead4255a2f13405e5700c01f7c8d29a03972d", size = 223527, upload-time = "2025-08-07T08:25:03.45Z" }, + { url = "https://files.pythonhosted.org/packages/1e/f9/77f4c90f79d2c5ca8ce6ec6a76cb4734ee247de6b3a4f337e289e1f00372/rpds_py-0.27.0-cp314-cp314t-macosx_10_12_x86_64.whl", hash = "sha256:341d8acb6724c0c17bdf714319c393bb27f6d23d39bc74f94221b3e59fc31828", size = 359469, upload-time = "2025-08-07T08:25:04.648Z" }, + { url = "https://files.pythonhosted.org/packages/c0/22/b97878d2f1284286fef4172069e84b0b42b546ea7d053e5fb7adb9ac6494/rpds_py-0.27.0-cp314-cp314t-macosx_11_0_arm64.whl", hash = "sha256:6b96b0b784fe5fd03beffff2b1533dc0d85e92bab8d1b2c24ef3a5dc8fac5669", size = 343960, upload-time = "2025-08-07T08:25:05.863Z" }, + { url = "https://files.pythonhosted.org/packages/b1/b0/dfd55b5bb480eda0578ae94ef256d3061d20b19a0f5e18c482f03e65464f/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0c431bfb91478d7cbe368d0a699978050d3b112d7f1d440a41e90faa325557fd", size = 380201, upload-time = "2025-08-07T08:25:07.513Z" }, + { url = "https://files.pythonhosted.org/packages/28/22/e1fa64e50d58ad2b2053077e3ec81a979147c43428de9e6de68ddf6aff4e/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:20e222a44ae9f507d0f2678ee3dd0c45ec1e930f6875d99b8459631c24058aec", size = 392111, upload-time = "2025-08-07T08:25:09.149Z" }, + { url = "https://files.pythonhosted.org/packages/49/f9/43ab7a43e97aedf6cea6af70fdcbe18abbbc41d4ae6cdec1bfc23bbad403/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:184f0d7b342967f6cda94a07d0e1fae177d11d0b8f17d73e06e36ac02889f303", size = 515863, upload-time = "2025-08-07T08:25:10.431Z" }, + { url = "https://files.pythonhosted.org/packages/38/9b/9bd59dcc636cd04d86a2d20ad967770bf348f5eb5922a8f29b547c074243/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a00c91104c173c9043bc46f7b30ee5e6d2f6b1149f11f545580f5d6fdff42c0b", size = 402398, upload-time = "2025-08-07T08:25:11.819Z" }, + { url = "https://files.pythonhosted.org/packages/71/bf/f099328c6c85667aba6b66fa5c35a8882db06dcd462ea214be72813a0dd2/rpds_py-0.27.0-cp314-cp314t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7a37dd208f0d658e0487522078b1ed68cd6bce20ef4b5a915d2809b9094b410", size = 384665, upload-time = "2025-08-07T08:25:13.194Z" }, + { url = "https://files.pythonhosted.org/packages/a9/c5/9c1f03121ece6634818490bd3c8be2c82a70928a19de03467fb25a3ae2a8/rpds_py-0.27.0-cp314-cp314t-manylinux_2_31_riscv64.whl", hash = "sha256:92f3b3ec3e6008a1fe00b7c0946a170f161ac00645cde35e3c9a68c2475e8156", size = 400405, upload-time = "2025-08-07T08:25:14.417Z" }, + { url = "https://files.pythonhosted.org/packages/b5/b8/e25d54af3e63ac94f0c16d8fe143779fe71ff209445a0c00d0f6984b6b2c/rpds_py-0.27.0-cp314-cp314t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:a1b3db5fae5cbce2131b7420a3f83553d4d89514c03d67804ced36161fe8b6b2", size = 413179, upload-time = "2025-08-07T08:25:15.664Z" }, + { url = "https://files.pythonhosted.org/packages/f9/d1/406b3316433fe49c3021546293a04bc33f1478e3ec7950215a7fce1a1208/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:5355527adaa713ab693cbce7c1e0ec71682f599f61b128cf19d07e5c13c9b1f1", size = 556895, upload-time = "2025-08-07T08:25:17.061Z" }, + { url = "https://files.pythonhosted.org/packages/5f/bc/3697c0c21fcb9a54d46ae3b735eb2365eea0c2be076b8f770f98e07998de/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_i686.whl", hash = "sha256:fcc01c57ce6e70b728af02b2401c5bc853a9e14eb07deda30624374f0aebfe42", size = 585464, upload-time = "2025-08-07T08:25:18.406Z" }, + { url = "https://files.pythonhosted.org/packages/63/09/ee1bb5536f99f42c839b177d552f6114aa3142d82f49cef49261ed28dbe0/rpds_py-0.27.0-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:3001013dae10f806380ba739d40dee11db1ecb91684febb8406a87c2ded23dae", size = 555090, upload-time = "2025-08-07T08:25:20.461Z" }, + { url = "https://files.pythonhosted.org/packages/7d/2c/363eada9e89f7059199d3724135a86c47082cbf72790d6ba2f336d146ddb/rpds_py-0.27.0-cp314-cp314t-win32.whl", hash = "sha256:0f401c369186a5743694dd9fc08cba66cf70908757552e1f714bfc5219c655b5", size = 218001, upload-time = "2025-08-07T08:25:21.761Z" }, + { url = "https://files.pythonhosted.org/packages/e2/3f/d6c216ed5199c9ef79e2a33955601f454ed1e7420a93b89670133bca5ace/rpds_py-0.27.0-cp314-cp314t-win_amd64.whl", hash = "sha256:8a1dca5507fa1337f75dcd5070218b20bc68cf8844271c923c1b79dfcbc20391", size = 230993, upload-time = "2025-08-07T08:25:23.34Z" }, + { url = "https://files.pythonhosted.org/packages/47/55/287068956f9ba1cb40896d291213f09fdd4527630709058b45a592bc09dc/rpds_py-0.27.0-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:46f48482c1a4748ab2773f75fffbdd1951eb59794e32788834b945da857c47a8", size = 371566, upload-time = "2025-08-07T08:25:43.95Z" }, + { url = "https://files.pythonhosted.org/packages/a2/fb/443af59cbe552e89680bb0f1d1ba47f6387b92083e28a45b8c8863b86c5a/rpds_py-0.27.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:419dd9c98bcc9fb0242be89e0c6e922df333b975d4268faa90d58499fd9c9ebe", size = 355781, upload-time = "2025-08-07T08:25:45.256Z" }, + { url = "https://files.pythonhosted.org/packages/ad/f0/35f48bb073b5ca42b1dcc55cb148f4a3bd4411a3e584f6a18d26f0ea8832/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55d42a0ef2bdf6bc81e1cc2d49d12460f63c6ae1423c4f4851b828e454ccf6f1", size = 382575, upload-time = "2025-08-07T08:25:46.524Z" }, + { url = "https://files.pythonhosted.org/packages/51/e1/5f5296a21d1189f0f116a938af2e346d83172bf814d373695e54004a936f/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:2e39169ac6aae06dd79c07c8a69d9da867cef6a6d7883a0186b46bb46ccfb0c3", size = 397435, upload-time = "2025-08-07T08:25:48.204Z" }, + { url = "https://files.pythonhosted.org/packages/97/79/3af99b7852b2b55cad8a08863725cbe9dc14781bcf7dc6ecead0c3e1dc54/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:935afcdea4751b0ac918047a2df3f720212892347767aea28f5b3bf7be4f27c0", size = 514861, upload-time = "2025-08-07T08:25:49.814Z" }, + { url = "https://files.pythonhosted.org/packages/df/3e/11fd6033708ed3ae0e6947bb94f762f56bb46bf59a1b16eef6944e8a62ee/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8de567dec6d451649a781633d36f5c7501711adee329d76c095be2178855b042", size = 402776, upload-time = "2025-08-07T08:25:51.135Z" }, + { url = "https://files.pythonhosted.org/packages/b7/89/f9375ceaa996116de9cbc949874804c7874d42fb258c384c037a46d730b8/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:555ed147cbe8c8f76e72a4c6cd3b7b761cbf9987891b9448808148204aed74a5", size = 384665, upload-time = "2025-08-07T08:25:52.82Z" }, + { url = "https://files.pythonhosted.org/packages/48/bf/0061e55c6f1f573a63c0f82306b8984ed3b394adafc66854a936d5db3522/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_31_riscv64.whl", hash = "sha256:d2cc2b34f9e1d31ce255174da82902ad75bd7c0d88a33df54a77a22f2ef421ee", size = 402518, upload-time = "2025-08-07T08:25:54.073Z" }, + { url = "https://files.pythonhosted.org/packages/ae/dc/8d506676bfe87b3b683332ec8e6ab2b0be118a3d3595ed021e3274a63191/rpds_py-0.27.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:cb0702c12983be3b2fab98ead349ac63a98216d28dda6f518f52da5498a27a1b", size = 416247, upload-time = "2025-08-07T08:25:55.433Z" }, + { url = "https://files.pythonhosted.org/packages/2e/02/9a89eea1b75c69e81632de7963076e455b1e00e1cfb46dfdabb055fa03e3/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:ba783541be46f27c8faea5a6645e193943c17ea2f0ffe593639d906a327a9bcc", size = 559456, upload-time = "2025-08-07T08:25:56.866Z" }, + { url = "https://files.pythonhosted.org/packages/38/4a/0f3ac4351957847c0d322be6ec72f916e43804a2c1d04e9672ea4a67c315/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:2406d034635d1497c596c40c85f86ecf2bf9611c1df73d14078af8444fe48031", size = 587778, upload-time = "2025-08-07T08:25:58.202Z" }, + { url = "https://files.pythonhosted.org/packages/c2/8e/39d0d7401095bed5a5ad5ef304fae96383f9bef40ca3f3a0807ff5b68d9d/rpds_py-0.27.0-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:dea0808153f1fbbad772669d906cddd92100277533a03845de6893cadeffc8be", size = 555247, upload-time = "2025-08-07T08:25:59.707Z" }, + { url = "https://files.pythonhosted.org/packages/e0/04/6b8311e811e620b9eaca67cd80a118ff9159558a719201052a7b2abb88bf/rpds_py-0.27.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:d2a81bdcfde4245468f7030a75a37d50400ac2455c3a4819d9d550c937f90ab5", size = 230256, upload-time = "2025-08-07T08:26:01.07Z" }, + { url = "https://files.pythonhosted.org/packages/59/64/72ab5b911fdcc48058359b0e786e5363e3fde885156116026f1a2ba9a5b5/rpds_py-0.27.0-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:e6491658dd2569f05860bad645569145c8626ac231877b0fb2d5f9bcb7054089", size = 371658, upload-time = "2025-08-07T08:26:02.369Z" }, + { url = "https://files.pythonhosted.org/packages/6c/4b/90ff04b4da055db53d8fea57640d8d5d55456343a1ec9a866c0ecfe10fd1/rpds_py-0.27.0-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:bec77545d188f8bdd29d42bccb9191682a46fb2e655e3d1fb446d47c55ac3b8d", size = 355529, upload-time = "2025-08-07T08:26:03.83Z" }, + { url = "https://files.pythonhosted.org/packages/a4/be/527491fb1afcd86fc5ce5812eb37bc70428ee017d77fee20de18155c3937/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:25a4aebf8ca02bbb90a9b3e7a463bbf3bee02ab1c446840ca07b1695a68ce424", size = 382822, upload-time = "2025-08-07T08:26:05.52Z" }, + { url = "https://files.pythonhosted.org/packages/e0/a5/dcdb8725ce11e6d0913e6fcf782a13f4b8a517e8acc70946031830b98441/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:44524b96481a4c9b8e6c46d6afe43fa1fb485c261e359fbe32b63ff60e3884d8", size = 397233, upload-time = "2025-08-07T08:26:07.179Z" }, + { url = "https://files.pythonhosted.org/packages/33/f9/0947920d1927e9f144660590cc38cadb0795d78fe0d9aae0ef71c1513b7c/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:45d04a73c54b6a5fd2bab91a4b5bc8b426949586e61340e212a8484919183859", size = 514892, upload-time = "2025-08-07T08:26:08.622Z" }, + { url = "https://files.pythonhosted.org/packages/1d/ed/d1343398c1417c68f8daa1afce56ef6ce5cc587daaf98e29347b00a80ff2/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:343cf24de9ed6c728abefc5d5c851d5de06497caa7ac37e5e65dd572921ed1b5", size = 402733, upload-time = "2025-08-07T08:26:10.433Z" }, + { url = "https://files.pythonhosted.org/packages/1d/0b/646f55442cd14014fb64d143428f25667a100f82092c90087b9ea7101c74/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7aed8118ae20515974650d08eb724150dc2e20c2814bcc307089569995e88a14", size = 384447, upload-time = "2025-08-07T08:26:11.847Z" }, + { url = "https://files.pythonhosted.org/packages/4b/15/0596ef7529828e33a6c81ecf5013d1dd33a511a3e0be0561f83079cda227/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_31_riscv64.whl", hash = "sha256:af9d4fd79ee1cc8e7caf693ee02737daabfc0fcf2773ca0a4735b356c8ad6f7c", size = 402502, upload-time = "2025-08-07T08:26:13.537Z" }, + { url = "https://files.pythonhosted.org/packages/c3/8d/986af3c42f8454a6cafff8729d99fb178ae9b08a9816325ac7a8fa57c0c0/rpds_py-0.27.0-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:f0396e894bd1e66c74ecbc08b4f6a03dc331140942c4b1d345dd131b68574a60", size = 416651, upload-time = "2025-08-07T08:26:14.923Z" }, + { url = "https://files.pythonhosted.org/packages/e9/9a/b4ec3629b7b447e896eec574469159b5b60b7781d3711c914748bf32de05/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:59714ab0a5af25d723d8e9816638faf7f4254234decb7d212715c1aa71eee7be", size = 559460, upload-time = "2025-08-07T08:26:16.295Z" }, + { url = "https://files.pythonhosted.org/packages/61/63/d1e127b40c3e4733b3a6f26ae7a063cdf2bc1caa5272c89075425c7d397a/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_i686.whl", hash = "sha256:88051c3b7d5325409f433c5a40328fcb0685fc04e5db49ff936e910901d10114", size = 588072, upload-time = "2025-08-07T08:26:17.776Z" }, + { url = "https://files.pythonhosted.org/packages/04/7e/8ffc71a8f6833d9c9fb999f5b0ee736b8b159fd66968e05c7afc2dbcd57e/rpds_py-0.27.0-pp311-pypy311_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:181bc29e59e5e5e6e9d63b143ff4d5191224d355e246b5a48c88ce6b35c4e466", size = 555083, upload-time = "2025-08-07T08:26:19.301Z" }, ] [[package]] @@ -1566,9 +1566,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pyasn1" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/da/8a/22b7beea3ee0d44b1916c0c1cb0ee3af23b700b6da9f04991899d0c555d4/rsa-4.9.1.tar.gz", hash = "sha256:e7bdbfdb5497da4c07dfd35530e1a902659db6ff241e39d9953cad06ebd0ae75", size = 29034 } +sdist = { url = "https://files.pythonhosted.org/packages/da/8a/22b7beea3ee0d44b1916c0c1cb0ee3af23b700b6da9f04991899d0c555d4/rsa-4.9.1.tar.gz", hash = "sha256:e7bdbfdb5497da4c07dfd35530e1a902659db6ff241e39d9953cad06ebd0ae75", size = 29034, upload-time = "2025-04-16T09:51:18.218Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/64/8d/0133e4eb4beed9e425d9a98ed6e081a55d195481b7632472be1af08d2f6b/rsa-4.9.1-py3-none-any.whl", hash = "sha256:68635866661c6836b8d39430f97a996acbd61bfa49406748ea243539fe239762", size = 34696 }, + { url = "https://files.pythonhosted.org/packages/64/8d/0133e4eb4beed9e425d9a98ed6e081a55d195481b7632472be1af08d2f6b/rsa-4.9.1-py3-none-any.whl", hash = "sha256:68635866661c6836b8d39430f97a996acbd61bfa49406748ea243539fe239762", size = 34696, upload-time = "2025-04-16T09:51:17.142Z" }, ] [[package]] @@ -1578,9 +1578,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "botocore" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/6d/05/d52bf1e65044b4e5e27d4e63e8d1579dbdec54fce685908ae09bc3720030/s3transfer-0.13.1.tar.gz", hash = "sha256:c3fdba22ba1bd367922f27ec8032d6a1cf5f10c934fb5d68cf60fd5a23d936cf", size = 150589 } +sdist = { url = "https://files.pythonhosted.org/packages/6d/05/d52bf1e65044b4e5e27d4e63e8d1579dbdec54fce685908ae09bc3720030/s3transfer-0.13.1.tar.gz", hash = "sha256:c3fdba22ba1bd367922f27ec8032d6a1cf5f10c934fb5d68cf60fd5a23d936cf", size = 150589, upload-time = "2025-07-18T19:22:42.31Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/6d/4f/d073e09df851cfa251ef7840007d04db3293a0482ce607d2b993926089be/s3transfer-0.13.1-py3-none-any.whl", hash = "sha256:a981aa7429be23fe6dfc13e80e4020057cbab622b08c0315288758d67cabc724", size = 85308 }, + { url = "https://files.pythonhosted.org/packages/6d/4f/d073e09df851cfa251ef7840007d04db3293a0482ce607d2b993926089be/s3transfer-0.13.1-py3-none-any.whl", hash = "sha256:a981aa7429be23fe6dfc13e80e4020057cbab622b08c0315288758d67cabc724", size = 85308, upload-time = "2025-07-18T19:22:40.947Z" }, ] [[package]] @@ -1591,120 +1591,120 @@ dependencies = [ { name = "cryptography" }, { name = "jeepney" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/53/a4/f48c9d79cb507ed1373477dbceaba7401fd8a23af63b837fa61f1dcd3691/SecretStorage-3.3.3.tar.gz", hash = "sha256:2403533ef369eca6d2ba81718576c5e0f564d5cca1b58f73a8b23e7d4eeebd77", size = 19739 } +sdist = { url = "https://files.pythonhosted.org/packages/53/a4/f48c9d79cb507ed1373477dbceaba7401fd8a23af63b837fa61f1dcd3691/SecretStorage-3.3.3.tar.gz", hash = "sha256:2403533ef369eca6d2ba81718576c5e0f564d5cca1b58f73a8b23e7d4eeebd77", size = 19739, upload-time = "2022-08-13T16:22:46.976Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/54/24/b4293291fa1dd830f353d2cb163295742fa87f179fcc8a20a306a81978b7/SecretStorage-3.3.3-py3-none-any.whl", hash = "sha256:f356e6628222568e3af06f2eba8df495efa13b3b63081dafd4f7d9a7b7bc9f99", size = 15221 }, + { url = "https://files.pythonhosted.org/packages/54/24/b4293291fa1dd830f353d2cb163295742fa87f179fcc8a20a306a81978b7/SecretStorage-3.3.3-py3-none-any.whl", hash = "sha256:f356e6628222568e3af06f2eba8df495efa13b3b63081dafd4f7d9a7b7bc9f99", size = 15221, upload-time = "2022-08-13T16:22:44.457Z" }, ] [[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 } +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 }, + { 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 = "shellingham" version = "1.5.4" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/58/15/8b3609fd3830ef7b27b655beb4b4e9c62313a4e8da8c676e142cc210d58e/shellingham-1.5.4.tar.gz", hash = "sha256:8dbca0739d487e5bd35ab3ca4b36e11c4078f3a234bfce294b0a0291363404de", size = 10310 } +sdist = { url = "https://files.pythonhosted.org/packages/58/15/8b3609fd3830ef7b27b655beb4b4e9c62313a4e8da8c676e142cc210d58e/shellingham-1.5.4.tar.gz", hash = "sha256:8dbca0739d487e5bd35ab3ca4b36e11c4078f3a234bfce294b0a0291363404de", size = 10310, upload-time = "2023-10-24T04:13:40.426Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e0/f9/0595336914c5619e5f28a1fb793285925a8cd4b432c9da0a987836c7f822/shellingham-1.5.4-py2.py3-none-any.whl", hash = "sha256:7ecfff8f2fd72616f7481040475a65b2bf8af90a56c89140852d1120324e8686", size = 9755 }, + { url = "https://files.pythonhosted.org/packages/e0/f9/0595336914c5619e5f28a1fb793285925a8cd4b432c9da0a987836c7f822/shellingham-1.5.4-py2.py3-none-any.whl", hash = "sha256:7ecfff8f2fd72616f7481040475a65b2bf8af90a56c89140852d1120324e8686", size = 9755, upload-time = "2023-10-24T04:13:38.866Z" }, ] [[package]] name = "six" version = "1.17.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/94/e7/b2c673351809dca68a0e064b6af791aa332cf192da575fd474ed7d6f16a2/six-1.17.0.tar.gz", hash = "sha256:ff70335d468e7eb6ec65b95b99d3a2836546063f63acc5171de367e834932a81", size = 34031 } +sdist = { url = "https://files.pythonhosted.org/packages/94/e7/b2c673351809dca68a0e064b6af791aa332cf192da575fd474ed7d6f16a2/six-1.17.0.tar.gz", hash = "sha256:ff70335d468e7eb6ec65b95b99d3a2836546063f63acc5171de367e834932a81", size = 34031, upload-time = "2024-12-04T17:35:28.174Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b7/ce/149a00dd41f10bc29e5921b496af8b574d8413afcd5e30dfa0ed46c2cc5e/six-1.17.0-py2.py3-none-any.whl", hash = "sha256:4721f391ed90541fddacab5acf947aa0d3dc7d27b2e1e8eda2be8970586c3274", size = 11050 }, + { url = "https://files.pythonhosted.org/packages/b7/ce/149a00dd41f10bc29e5921b496af8b574d8413afcd5e30dfa0ed46c2cc5e/six-1.17.0-py2.py3-none-any.whl", hash = "sha256:4721f391ed90541fddacab5acf947aa0d3dc7d27b2e1e8eda2be8970586c3274", size = 11050, upload-time = "2024-12-04T17:35:26.475Z" }, ] [[package]] name = "smmap" version = "5.0.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/44/cd/a040c4b3119bbe532e5b0732286f805445375489fceaec1f48306068ee3b/smmap-5.0.2.tar.gz", hash = "sha256:26ea65a03958fa0c8a1c7e8c7a58fdc77221b8910f6be2131affade476898ad5", size = 22329 } +sdist = { url = "https://files.pythonhosted.org/packages/44/cd/a040c4b3119bbe532e5b0732286f805445375489fceaec1f48306068ee3b/smmap-5.0.2.tar.gz", hash = "sha256:26ea65a03958fa0c8a1c7e8c7a58fdc77221b8910f6be2131affade476898ad5", size = 22329, upload-time = "2025-01-02T07:14:40.909Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/04/be/d09147ad1ec7934636ad912901c5fd7667e1c858e19d355237db0d0cd5e4/smmap-5.0.2-py3-none-any.whl", hash = "sha256:b30115f0def7d7531d22a0fb6502488d879e75b260a9db4d0819cfb25403af5e", size = 24303 }, + { url = "https://files.pythonhosted.org/packages/04/be/d09147ad1ec7934636ad912901c5fd7667e1c858e19d355237db0d0cd5e4/smmap-5.0.2-py3-none-any.whl", hash = "sha256:b30115f0def7d7531d22a0fb6502488d879e75b260a9db4d0819cfb25403af5e", size = 24303, upload-time = "2025-01-02T07:14:38.724Z" }, ] [[package]] name = "sniffio" version = "1.3.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a2/87/a6771e1546d97e7e041b6ae58d80074f81b7d5121207425c964ddf5cfdbd/sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc", size = 20372 } +sdist = { url = "https://files.pythonhosted.org/packages/a2/87/a6771e1546d97e7e041b6ae58d80074f81b7d5121207425c964ddf5cfdbd/sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc", size = 20372, upload-time = "2024-02-25T23:20:04.057Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e9/44/75a9c9421471a6c4805dbf2356f7c181a29c1879239abab1ea2cc8f38b40/sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2", size = 10235 }, + { url = "https://files.pythonhosted.org/packages/e9/44/75a9c9421471a6c4805dbf2356f7c181a29c1879239abab1ea2cc8f38b40/sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2", size = 10235, upload-time = "2024-02-25T23:20:01.196Z" }, ] [[package]] name = "tabulate" version = "0.9.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/ec/fe/802052aecb21e3797b8f7902564ab6ea0d60ff8ca23952079064155d1ae1/tabulate-0.9.0.tar.gz", hash = "sha256:0095b12bf5966de529c0feb1fa08671671b3368eec77d7ef7ab114be2c068b3c", size = 81090 } +sdist = { url = "https://files.pythonhosted.org/packages/ec/fe/802052aecb21e3797b8f7902564ab6ea0d60ff8ca23952079064155d1ae1/tabulate-0.9.0.tar.gz", hash = "sha256:0095b12bf5966de529c0feb1fa08671671b3368eec77d7ef7ab114be2c068b3c", size = 81090, upload-time = "2022-10-06T17:21:48.54Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/40/44/4a5f08c96eb108af5cb50b41f76142f0afa346dfa99d5296fe7202a11854/tabulate-0.9.0-py3-none-any.whl", hash = "sha256:024ca478df22e9340661486f85298cff5f6dcdba14f3813e8830015b9ed1948f", size = 35252 }, + { url = "https://files.pythonhosted.org/packages/40/44/4a5f08c96eb108af5cb50b41f76142f0afa346dfa99d5296fe7202a11854/tabulate-0.9.0-py3-none-any.whl", hash = "sha256:024ca478df22e9340661486f85298cff5f6dcdba14f3813e8830015b9ed1948f", size = 35252, upload-time = "2022-10-06T17:21:44.262Z" }, ] [[package]] name = "tomli" version = "2.2.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/18/87/302344fed471e44a87289cf4967697d07e532f2421fdaf868a303cbae4ff/tomli-2.2.1.tar.gz", hash = "sha256:cd45e1dc79c835ce60f7404ec8119f2eb06d38b1deba146f07ced3bbc44505ff", size = 17175 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/43/ca/75707e6efa2b37c77dadb324ae7d9571cb424e61ea73fad7c56c2d14527f/tomli-2.2.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:678e4fa69e4575eb77d103de3df8a895e1591b48e740211bd1067378c69e8249", size = 131077 }, - { url = "https://files.pythonhosted.org/packages/c7/16/51ae563a8615d472fdbffc43a3f3d46588c264ac4f024f63f01283becfbb/tomli-2.2.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:023aa114dd824ade0100497eb2318602af309e5a55595f76b626d6d9f3b7b0a6", size = 123429 }, - { url = "https://files.pythonhosted.org/packages/f1/dd/4f6cd1e7b160041db83c694abc78e100473c15d54620083dbd5aae7b990e/tomli-2.2.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ece47d672db52ac607a3d9599a9d48dcb2f2f735c6c2d1f34130085bb12b112a", size = 226067 }, - { url = "https://files.pythonhosted.org/packages/a9/6b/c54ede5dc70d648cc6361eaf429304b02f2871a345bbdd51e993d6cdf550/tomli-2.2.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6972ca9c9cc9f0acaa56a8ca1ff51e7af152a9f87fb64623e31d5c83700080ee", size = 236030 }, - { url = "https://files.pythonhosted.org/packages/1f/47/999514fa49cfaf7a92c805a86c3c43f4215621855d151b61c602abb38091/tomli-2.2.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c954d2250168d28797dd4e3ac5cf812a406cd5a92674ee4c8f123c889786aa8e", size = 240898 }, - { url = "https://files.pythonhosted.org/packages/73/41/0a01279a7ae09ee1573b423318e7934674ce06eb33f50936655071d81a24/tomli-2.2.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8dd28b3e155b80f4d54beb40a441d366adcfe740969820caf156c019fb5c7ec4", size = 229894 }, - { url = "https://files.pythonhosted.org/packages/55/18/5d8bc5b0a0362311ce4d18830a5d28943667599a60d20118074ea1b01bb7/tomli-2.2.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e59e304978767a54663af13c07b3d1af22ddee3bb2fb0618ca1593e4f593a106", size = 245319 }, - { url = "https://files.pythonhosted.org/packages/92/a3/7ade0576d17f3cdf5ff44d61390d4b3febb8a9fc2b480c75c47ea048c646/tomli-2.2.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:33580bccab0338d00994d7f16f4c4ec25b776af3ffaac1ed74e0b3fc95e885a8", size = 238273 }, - { url = "https://files.pythonhosted.org/packages/72/6f/fa64ef058ac1446a1e51110c375339b3ec6be245af9d14c87c4a6412dd32/tomli-2.2.1-cp311-cp311-win32.whl", hash = "sha256:465af0e0875402f1d226519c9904f37254b3045fc5084697cefb9bdde1ff99ff", size = 98310 }, - { url = "https://files.pythonhosted.org/packages/6a/1c/4a2dcde4a51b81be3530565e92eda625d94dafb46dbeb15069df4caffc34/tomli-2.2.1-cp311-cp311-win_amd64.whl", hash = "sha256:2d0f2fdd22b02c6d81637a3c95f8cd77f995846af7414c5c4b8d0545afa1bc4b", size = 108309 }, - { url = "https://files.pythonhosted.org/packages/52/e1/f8af4c2fcde17500422858155aeb0d7e93477a0d59a98e56cbfe75070fd0/tomli-2.2.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:4a8f6e44de52d5e6c657c9fe83b562f5f4256d8ebbfe4ff922c495620a7f6cea", size = 132762 }, - { url = "https://files.pythonhosted.org/packages/03/b8/152c68bb84fc00396b83e7bbddd5ec0bd3dd409db4195e2a9b3e398ad2e3/tomli-2.2.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8d57ca8095a641b8237d5b079147646153d22552f1c637fd3ba7f4b0b29167a8", size = 123453 }, - { url = "https://files.pythonhosted.org/packages/c8/d6/fc9267af9166f79ac528ff7e8c55c8181ded34eb4b0e93daa767b8841573/tomli-2.2.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4e340144ad7ae1533cb897d406382b4b6fede8890a03738ff1683af800d54192", size = 233486 }, - { url = "https://files.pythonhosted.org/packages/5c/51/51c3f2884d7bab89af25f678447ea7d297b53b5a3b5730a7cb2ef6069f07/tomli-2.2.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:db2b95f9de79181805df90bedc5a5ab4c165e6ec3fe99f970d0e302f384ad222", size = 242349 }, - { url = "https://files.pythonhosted.org/packages/ab/df/bfa89627d13a5cc22402e441e8a931ef2108403db390ff3345c05253935e/tomli-2.2.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:40741994320b232529c802f8bc86da4e1aa9f413db394617b9a256ae0f9a7f77", size = 252159 }, - { url = "https://files.pythonhosted.org/packages/9e/6e/fa2b916dced65763a5168c6ccb91066f7639bdc88b48adda990db10c8c0b/tomli-2.2.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:400e720fe168c0f8521520190686ef8ef033fb19fc493da09779e592861b78c6", size = 237243 }, - { url = "https://files.pythonhosted.org/packages/b4/04/885d3b1f650e1153cbb93a6a9782c58a972b94ea4483ae4ac5cedd5e4a09/tomli-2.2.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:02abe224de6ae62c19f090f68da4e27b10af2b93213d36cf44e6e1c5abd19fdd", size = 259645 }, - { url = "https://files.pythonhosted.org/packages/9c/de/6b432d66e986e501586da298e28ebeefd3edc2c780f3ad73d22566034239/tomli-2.2.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:b82ebccc8c8a36f2094e969560a1b836758481f3dc360ce9a3277c65f374285e", size = 244584 }, - { url = "https://files.pythonhosted.org/packages/1c/9a/47c0449b98e6e7d1be6cbac02f93dd79003234ddc4aaab6ba07a9a7482e2/tomli-2.2.1-cp312-cp312-win32.whl", hash = "sha256:889f80ef92701b9dbb224e49ec87c645ce5df3fa2cc548664eb8a25e03127a98", size = 98875 }, - { url = "https://files.pythonhosted.org/packages/ef/60/9b9638f081c6f1261e2688bd487625cd1e660d0a85bd469e91d8db969734/tomli-2.2.1-cp312-cp312-win_amd64.whl", hash = "sha256:7fc04e92e1d624a4a63c76474610238576942d6b8950a2d7f908a340494e67e4", size = 109418 }, - { url = "https://files.pythonhosted.org/packages/04/90/2ee5f2e0362cb8a0b6499dc44f4d7d48f8fff06d28ba46e6f1eaa61a1388/tomli-2.2.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f4039b9cbc3048b2416cc57ab3bda989a6fcf9b36cf8937f01a6e731b64f80d7", size = 132708 }, - { url = "https://files.pythonhosted.org/packages/c0/ec/46b4108816de6b385141f082ba99e315501ccd0a2ea23db4a100dd3990ea/tomli-2.2.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:286f0ca2ffeeb5b9bd4fcc8d6c330534323ec51b2f52da063b11c502da16f30c", size = 123582 }, - { url = "https://files.pythonhosted.org/packages/a0/bd/b470466d0137b37b68d24556c38a0cc819e8febe392d5b199dcd7f578365/tomli-2.2.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a92ef1a44547e894e2a17d24e7557a5e85a9e1d0048b0b5e7541f76c5032cb13", size = 232543 }, - { url = "https://files.pythonhosted.org/packages/d9/e5/82e80ff3b751373f7cead2815bcbe2d51c895b3c990686741a8e56ec42ab/tomli-2.2.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9316dc65bed1684c9a98ee68759ceaed29d229e985297003e494aa825ebb0281", size = 241691 }, - { url = "https://files.pythonhosted.org/packages/05/7e/2a110bc2713557d6a1bfb06af23dd01e7dde52b6ee7dadc589868f9abfac/tomli-2.2.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e85e99945e688e32d5a35c1ff38ed0b3f41f43fad8df0bdf79f72b2ba7bc5272", size = 251170 }, - { url = "https://files.pythonhosted.org/packages/64/7b/22d713946efe00e0adbcdfd6d1aa119ae03fd0b60ebed51ebb3fa9f5a2e5/tomli-2.2.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:ac065718db92ca818f8d6141b5f66369833d4a80a9d74435a268c52bdfa73140", size = 236530 }, - { url = "https://files.pythonhosted.org/packages/38/31/3a76f67da4b0cf37b742ca76beaf819dca0ebef26d78fc794a576e08accf/tomli-2.2.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d920f33822747519673ee656a4b6ac33e382eca9d331c87770faa3eef562aeb2", size = 258666 }, - { url = "https://files.pythonhosted.org/packages/07/10/5af1293da642aded87e8a988753945d0cf7e00a9452d3911dd3bb354c9e2/tomli-2.2.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:a198f10c4d1b1375d7687bc25294306e551bf1abfa4eace6650070a5c1ae2744", size = 243954 }, - { url = "https://files.pythonhosted.org/packages/5b/b9/1ed31d167be802da0fc95020d04cd27b7d7065cc6fbefdd2f9186f60d7bd/tomli-2.2.1-cp313-cp313-win32.whl", hash = "sha256:d3f5614314d758649ab2ab3a62d4f2004c825922f9e370b29416484086b264ec", size = 98724 }, - { url = "https://files.pythonhosted.org/packages/c7/32/b0963458706accd9afcfeb867c0f9175a741bf7b19cd424230714d722198/tomli-2.2.1-cp313-cp313-win_amd64.whl", hash = "sha256:a38aa0308e754b0e3c67e344754dff64999ff9b513e691d0e786265c93583c69", size = 109383 }, - { url = "https://files.pythonhosted.org/packages/6e/c2/61d3e0f47e2b74ef40a68b9e6ad5984f6241a942f7cd3bbfbdbd03861ea9/tomli-2.2.1-py3-none-any.whl", hash = "sha256:cb55c73c5f4408779d0cf3eef9f762b9c9f147a77de7b258bef0a5628adc85cc", size = 14257 }, +sdist = { url = "https://files.pythonhosted.org/packages/18/87/302344fed471e44a87289cf4967697d07e532f2421fdaf868a303cbae4ff/tomli-2.2.1.tar.gz", hash = "sha256:cd45e1dc79c835ce60f7404ec8119f2eb06d38b1deba146f07ced3bbc44505ff", size = 17175, upload-time = "2024-11-27T22:38:36.873Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/43/ca/75707e6efa2b37c77dadb324ae7d9571cb424e61ea73fad7c56c2d14527f/tomli-2.2.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:678e4fa69e4575eb77d103de3df8a895e1591b48e740211bd1067378c69e8249", size = 131077, upload-time = "2024-11-27T22:37:54.956Z" }, + { url = "https://files.pythonhosted.org/packages/c7/16/51ae563a8615d472fdbffc43a3f3d46588c264ac4f024f63f01283becfbb/tomli-2.2.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:023aa114dd824ade0100497eb2318602af309e5a55595f76b626d6d9f3b7b0a6", size = 123429, upload-time = "2024-11-27T22:37:56.698Z" }, + { url = "https://files.pythonhosted.org/packages/f1/dd/4f6cd1e7b160041db83c694abc78e100473c15d54620083dbd5aae7b990e/tomli-2.2.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ece47d672db52ac607a3d9599a9d48dcb2f2f735c6c2d1f34130085bb12b112a", size = 226067, upload-time = "2024-11-27T22:37:57.63Z" }, + { url = "https://files.pythonhosted.org/packages/a9/6b/c54ede5dc70d648cc6361eaf429304b02f2871a345bbdd51e993d6cdf550/tomli-2.2.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6972ca9c9cc9f0acaa56a8ca1ff51e7af152a9f87fb64623e31d5c83700080ee", size = 236030, upload-time = "2024-11-27T22:37:59.344Z" }, + { url = "https://files.pythonhosted.org/packages/1f/47/999514fa49cfaf7a92c805a86c3c43f4215621855d151b61c602abb38091/tomli-2.2.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c954d2250168d28797dd4e3ac5cf812a406cd5a92674ee4c8f123c889786aa8e", size = 240898, upload-time = "2024-11-27T22:38:00.429Z" }, + { url = "https://files.pythonhosted.org/packages/73/41/0a01279a7ae09ee1573b423318e7934674ce06eb33f50936655071d81a24/tomli-2.2.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8dd28b3e155b80f4d54beb40a441d366adcfe740969820caf156c019fb5c7ec4", size = 229894, upload-time = "2024-11-27T22:38:02.094Z" }, + { url = "https://files.pythonhosted.org/packages/55/18/5d8bc5b0a0362311ce4d18830a5d28943667599a60d20118074ea1b01bb7/tomli-2.2.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e59e304978767a54663af13c07b3d1af22ddee3bb2fb0618ca1593e4f593a106", size = 245319, upload-time = "2024-11-27T22:38:03.206Z" }, + { url = "https://files.pythonhosted.org/packages/92/a3/7ade0576d17f3cdf5ff44d61390d4b3febb8a9fc2b480c75c47ea048c646/tomli-2.2.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:33580bccab0338d00994d7f16f4c4ec25b776af3ffaac1ed74e0b3fc95e885a8", size = 238273, upload-time = "2024-11-27T22:38:04.217Z" }, + { url = "https://files.pythonhosted.org/packages/72/6f/fa64ef058ac1446a1e51110c375339b3ec6be245af9d14c87c4a6412dd32/tomli-2.2.1-cp311-cp311-win32.whl", hash = "sha256:465af0e0875402f1d226519c9904f37254b3045fc5084697cefb9bdde1ff99ff", size = 98310, upload-time = "2024-11-27T22:38:05.908Z" }, + { url = "https://files.pythonhosted.org/packages/6a/1c/4a2dcde4a51b81be3530565e92eda625d94dafb46dbeb15069df4caffc34/tomli-2.2.1-cp311-cp311-win_amd64.whl", hash = "sha256:2d0f2fdd22b02c6d81637a3c95f8cd77f995846af7414c5c4b8d0545afa1bc4b", size = 108309, upload-time = "2024-11-27T22:38:06.812Z" }, + { url = "https://files.pythonhosted.org/packages/52/e1/f8af4c2fcde17500422858155aeb0d7e93477a0d59a98e56cbfe75070fd0/tomli-2.2.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:4a8f6e44de52d5e6c657c9fe83b562f5f4256d8ebbfe4ff922c495620a7f6cea", size = 132762, upload-time = "2024-11-27T22:38:07.731Z" }, + { url = "https://files.pythonhosted.org/packages/03/b8/152c68bb84fc00396b83e7bbddd5ec0bd3dd409db4195e2a9b3e398ad2e3/tomli-2.2.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8d57ca8095a641b8237d5b079147646153d22552f1c637fd3ba7f4b0b29167a8", size = 123453, upload-time = "2024-11-27T22:38:09.384Z" }, + { url = "https://files.pythonhosted.org/packages/c8/d6/fc9267af9166f79ac528ff7e8c55c8181ded34eb4b0e93daa767b8841573/tomli-2.2.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4e340144ad7ae1533cb897d406382b4b6fede8890a03738ff1683af800d54192", size = 233486, upload-time = "2024-11-27T22:38:10.329Z" }, + { url = "https://files.pythonhosted.org/packages/5c/51/51c3f2884d7bab89af25f678447ea7d297b53b5a3b5730a7cb2ef6069f07/tomli-2.2.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:db2b95f9de79181805df90bedc5a5ab4c165e6ec3fe99f970d0e302f384ad222", size = 242349, upload-time = "2024-11-27T22:38:11.443Z" }, + { url = "https://files.pythonhosted.org/packages/ab/df/bfa89627d13a5cc22402e441e8a931ef2108403db390ff3345c05253935e/tomli-2.2.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:40741994320b232529c802f8bc86da4e1aa9f413db394617b9a256ae0f9a7f77", size = 252159, upload-time = "2024-11-27T22:38:13.099Z" }, + { url = "https://files.pythonhosted.org/packages/9e/6e/fa2b916dced65763a5168c6ccb91066f7639bdc88b48adda990db10c8c0b/tomli-2.2.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:400e720fe168c0f8521520190686ef8ef033fb19fc493da09779e592861b78c6", size = 237243, upload-time = "2024-11-27T22:38:14.766Z" }, + { url = "https://files.pythonhosted.org/packages/b4/04/885d3b1f650e1153cbb93a6a9782c58a972b94ea4483ae4ac5cedd5e4a09/tomli-2.2.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:02abe224de6ae62c19f090f68da4e27b10af2b93213d36cf44e6e1c5abd19fdd", size = 259645, upload-time = "2024-11-27T22:38:15.843Z" }, + { url = "https://files.pythonhosted.org/packages/9c/de/6b432d66e986e501586da298e28ebeefd3edc2c780f3ad73d22566034239/tomli-2.2.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:b82ebccc8c8a36f2094e969560a1b836758481f3dc360ce9a3277c65f374285e", size = 244584, upload-time = "2024-11-27T22:38:17.645Z" }, + { url = "https://files.pythonhosted.org/packages/1c/9a/47c0449b98e6e7d1be6cbac02f93dd79003234ddc4aaab6ba07a9a7482e2/tomli-2.2.1-cp312-cp312-win32.whl", hash = "sha256:889f80ef92701b9dbb224e49ec87c645ce5df3fa2cc548664eb8a25e03127a98", size = 98875, upload-time = "2024-11-27T22:38:19.159Z" }, + { url = "https://files.pythonhosted.org/packages/ef/60/9b9638f081c6f1261e2688bd487625cd1e660d0a85bd469e91d8db969734/tomli-2.2.1-cp312-cp312-win_amd64.whl", hash = "sha256:7fc04e92e1d624a4a63c76474610238576942d6b8950a2d7f908a340494e67e4", size = 109418, upload-time = "2024-11-27T22:38:20.064Z" }, + { url = "https://files.pythonhosted.org/packages/04/90/2ee5f2e0362cb8a0b6499dc44f4d7d48f8fff06d28ba46e6f1eaa61a1388/tomli-2.2.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f4039b9cbc3048b2416cc57ab3bda989a6fcf9b36cf8937f01a6e731b64f80d7", size = 132708, upload-time = "2024-11-27T22:38:21.659Z" }, + { url = "https://files.pythonhosted.org/packages/c0/ec/46b4108816de6b385141f082ba99e315501ccd0a2ea23db4a100dd3990ea/tomli-2.2.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:286f0ca2ffeeb5b9bd4fcc8d6c330534323ec51b2f52da063b11c502da16f30c", size = 123582, upload-time = "2024-11-27T22:38:22.693Z" }, + { url = "https://files.pythonhosted.org/packages/a0/bd/b470466d0137b37b68d24556c38a0cc819e8febe392d5b199dcd7f578365/tomli-2.2.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a92ef1a44547e894e2a17d24e7557a5e85a9e1d0048b0b5e7541f76c5032cb13", size = 232543, upload-time = "2024-11-27T22:38:24.367Z" }, + { url = "https://files.pythonhosted.org/packages/d9/e5/82e80ff3b751373f7cead2815bcbe2d51c895b3c990686741a8e56ec42ab/tomli-2.2.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9316dc65bed1684c9a98ee68759ceaed29d229e985297003e494aa825ebb0281", size = 241691, upload-time = "2024-11-27T22:38:26.081Z" }, + { url = "https://files.pythonhosted.org/packages/05/7e/2a110bc2713557d6a1bfb06af23dd01e7dde52b6ee7dadc589868f9abfac/tomli-2.2.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e85e99945e688e32d5a35c1ff38ed0b3f41f43fad8df0bdf79f72b2ba7bc5272", size = 251170, upload-time = "2024-11-27T22:38:27.921Z" }, + { url = "https://files.pythonhosted.org/packages/64/7b/22d713946efe00e0adbcdfd6d1aa119ae03fd0b60ebed51ebb3fa9f5a2e5/tomli-2.2.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:ac065718db92ca818f8d6141b5f66369833d4a80a9d74435a268c52bdfa73140", size = 236530, upload-time = "2024-11-27T22:38:29.591Z" }, + { url = "https://files.pythonhosted.org/packages/38/31/3a76f67da4b0cf37b742ca76beaf819dca0ebef26d78fc794a576e08accf/tomli-2.2.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d920f33822747519673ee656a4b6ac33e382eca9d331c87770faa3eef562aeb2", size = 258666, upload-time = "2024-11-27T22:38:30.639Z" }, + { url = "https://files.pythonhosted.org/packages/07/10/5af1293da642aded87e8a988753945d0cf7e00a9452d3911dd3bb354c9e2/tomli-2.2.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:a198f10c4d1b1375d7687bc25294306e551bf1abfa4eace6650070a5c1ae2744", size = 243954, upload-time = "2024-11-27T22:38:31.702Z" }, + { url = "https://files.pythonhosted.org/packages/5b/b9/1ed31d167be802da0fc95020d04cd27b7d7065cc6fbefdd2f9186f60d7bd/tomli-2.2.1-cp313-cp313-win32.whl", hash = "sha256:d3f5614314d758649ab2ab3a62d4f2004c825922f9e370b29416484086b264ec", size = 98724, upload-time = "2024-11-27T22:38:32.837Z" }, + { url = "https://files.pythonhosted.org/packages/c7/32/b0963458706accd9afcfeb867c0f9175a741bf7b19cd424230714d722198/tomli-2.2.1-cp313-cp313-win_amd64.whl", hash = "sha256:a38aa0308e754b0e3c67e344754dff64999ff9b513e691d0e786265c93583c69", size = 109383, upload-time = "2024-11-27T22:38:34.455Z" }, + { url = "https://files.pythonhosted.org/packages/6e/c2/61d3e0f47e2b74ef40a68b9e6ad5984f6241a942f7cd3bbfbdbd03861ea9/tomli-2.2.1-py3-none-any.whl", hash = "sha256:cb55c73c5f4408779d0cf3eef9f762b9c9f147a77de7b258bef0a5628adc85cc", size = 14257, upload-time = "2024-11-27T22:38:35.385Z" }, ] [[package]] name = "tomli-w" version = "1.2.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/19/75/241269d1da26b624c0d5e110e8149093c759b7a286138f4efd61a60e75fe/tomli_w-1.2.0.tar.gz", hash = "sha256:2dd14fac5a47c27be9cd4c976af5a12d87fb1f0b4512f81d69cce3b35ae25021", size = 7184 } +sdist = { url = "https://files.pythonhosted.org/packages/19/75/241269d1da26b624c0d5e110e8149093c759b7a286138f4efd61a60e75fe/tomli_w-1.2.0.tar.gz", hash = "sha256:2dd14fac5a47c27be9cd4c976af5a12d87fb1f0b4512f81d69cce3b35ae25021", size = 7184, upload-time = "2025-01-15T12:07:24.262Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/c7/18/c86eb8e0202e32dd3df50d43d7ff9854f8e0603945ff398974c1d91ac1ef/tomli_w-1.2.0-py3-none-any.whl", hash = "sha256:188306098d013b691fcadc011abd66727d3c414c571bb01b1a174ba8c983cf90", size = 6675 }, + { url = "https://files.pythonhosted.org/packages/c7/18/c86eb8e0202e32dd3df50d43d7ff9854f8e0603945ff398974c1d91ac1ef/tomli_w-1.2.0-py3-none-any.whl", hash = "sha256:188306098d013b691fcadc011abd66727d3c414c571bb01b1a174ba8c983cf90", size = 6675, upload-time = "2025-01-15T12:07:22.074Z" }, ] [[package]] name = "tomlkit" version = "0.13.3" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/cc/18/0bbf3884e9eaa38819ebe46a7bd25dcd56b67434402b66a58c4b8e552575/tomlkit-0.13.3.tar.gz", hash = "sha256:430cf247ee57df2b94ee3fbe588e71d362a941ebb545dec29b53961d61add2a1", size = 185207 } +sdist = { url = "https://files.pythonhosted.org/packages/cc/18/0bbf3884e9eaa38819ebe46a7bd25dcd56b67434402b66a58c4b8e552575/tomlkit-0.13.3.tar.gz", hash = "sha256:430cf247ee57df2b94ee3fbe588e71d362a941ebb545dec29b53961d61add2a1", size = 185207, upload-time = "2025-06-05T07:13:44.947Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/bd/75/8539d011f6be8e29f339c42e633aae3cb73bffa95dd0f9adec09b9c58e85/tomlkit-0.13.3-py3-none-any.whl", hash = "sha256:c89c649d79ee40629a9fda55f8ace8c6a1b42deb912b2a8fd8d942ddadb606b0", size = 38901 }, + { url = "https://files.pythonhosted.org/packages/bd/75/8539d011f6be8e29f339c42e633aae3cb73bffa95dd0f9adec09b9c58e85/tomlkit-0.13.3-py3-none-any.whl", hash = "sha256:c89c649d79ee40629a9fda55f8ace8c6a1b42deb912b2a8fd8d942ddadb606b0", size = 38901, upload-time = "2025-06-05T07:13:43.546Z" }, ] [[package]] @@ -1714,18 +1714,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "colorama", marker = "sys_platform == 'win32'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/a8/4b/29b4ef32e036bb34e4ab51796dd745cdba7ed47ad142a9f4a1eb8e0c744d/tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2", size = 169737 } +sdist = { url = "https://files.pythonhosted.org/packages/a8/4b/29b4ef32e036bb34e4ab51796dd745cdba7ed47ad142a9f4a1eb8e0c744d/tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2", size = 169737, upload-time = "2024-11-24T20:12:22.481Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d0/30/dc54f88dd4a2b5dc8a0279bdd7270e735851848b762aeb1c1184ed1f6b14/tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2", size = 78540 }, + { url = "https://files.pythonhosted.org/packages/d0/30/dc54f88dd4a2b5dc8a0279bdd7270e735851848b762aeb1c1184ed1f6b14/tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2", size = 78540, upload-time = "2024-11-24T20:12:19.698Z" }, ] [[package]] name = "trove-classifiers" version = "2025.8.6.13" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/c3/21/707af14daa638b0df15b5d5700349e0abdd3e5140069f9ab6e0ccb922806/trove_classifiers-2025.8.6.13.tar.gz", hash = "sha256:5a0abad839d2ed810f213ab133d555d267124ddea29f1d8a50d6eca12a50ae6e", size = 16932 } +sdist = { url = "https://files.pythonhosted.org/packages/c3/21/707af14daa638b0df15b5d5700349e0abdd3e5140069f9ab6e0ccb922806/trove_classifiers-2025.8.6.13.tar.gz", hash = "sha256:5a0abad839d2ed810f213ab133d555d267124ddea29f1d8a50d6eca12a50ae6e", size = 16932, upload-time = "2025-08-06T13:26:26.479Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d5/44/323a87d78f04d5329092aada803af3612dd004a64b69ba8b13046601a8c9/trove_classifiers-2025.8.6.13-py3-none-any.whl", hash = "sha256:c4e7fc83012770d80b3ae95816111c32b085716374dccee0d3fbf5c235495f9f", size = 14121 }, + { url = "https://files.pythonhosted.org/packages/d5/44/323a87d78f04d5329092aada803af3612dd004a64b69ba8b13046601a8c9/trove_classifiers-2025.8.6.13-py3-none-any.whl", hash = "sha256:c4e7fc83012770d80b3ae95816111c32b085716374dccee0d3fbf5c235495f9f", size = 14121, upload-time = "2025-08-06T13:26:25.063Z" }, ] [[package]] @@ -1743,36 +1743,36 @@ dependencies = [ { name = "rich" }, { name = "urllib3" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/c8/a2/6df94fc5c8e2170d21d7134a565c3a8fb84f9797c1dd65a5976aaf714418/twine-6.1.0.tar.gz", hash = "sha256:be324f6272eff91d07ee93f251edf232fc647935dd585ac003539b42404a8dbd", size = 168404 } +sdist = { url = "https://files.pythonhosted.org/packages/c8/a2/6df94fc5c8e2170d21d7134a565c3a8fb84f9797c1dd65a5976aaf714418/twine-6.1.0.tar.gz", hash = "sha256:be324f6272eff91d07ee93f251edf232fc647935dd585ac003539b42404a8dbd", size = 168404, upload-time = "2025-01-21T18:45:26.758Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/7c/b6/74e927715a285743351233f33ea3c684528a0d374d2e43ff9ce9585b73fe/twine-6.1.0-py3-none-any.whl", hash = "sha256:a47f973caf122930bf0fbbf17f80b83bc1602c9ce393c7845f289a3001dc5384", size = 40791 }, + { url = "https://files.pythonhosted.org/packages/7c/b6/74e927715a285743351233f33ea3c684528a0d374d2e43ff9ce9585b73fe/twine-6.1.0-py3-none-any.whl", hash = "sha256:a47f973caf122930bf0fbbf17f80b83bc1602c9ce393c7845f289a3001dc5384", size = 40791, upload-time = "2025-01-21T18:45:24.584Z" }, ] [[package]] name = "typing-extensions" version = "4.14.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/98/5a/da40306b885cc8c09109dc2e1abd358d5684b1425678151cdaed4731c822/typing_extensions-4.14.1.tar.gz", hash = "sha256:38b39f4aeeab64884ce9f74c94263ef78f3c22467c8724005483154c26648d36", size = 107673 } +sdist = { url = "https://files.pythonhosted.org/packages/98/5a/da40306b885cc8c09109dc2e1abd358d5684b1425678151cdaed4731c822/typing_extensions-4.14.1.tar.gz", hash = "sha256:38b39f4aeeab64884ce9f74c94263ef78f3c22467c8724005483154c26648d36", size = 107673, upload-time = "2025-07-04T13:28:34.16Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/b5/00/d631e67a838026495268c2f6884f3711a15a9a2a96cd244fdaea53b823fb/typing_extensions-4.14.1-py3-none-any.whl", hash = "sha256:d1e1e3b58374dc93031d6eda2420a48ea44a36c2b4766a4fdeb3710755731d76", size = 43906 }, + { url = "https://files.pythonhosted.org/packages/b5/00/d631e67a838026495268c2f6884f3711a15a9a2a96cd244fdaea53b823fb/typing_extensions-4.14.1-py3-none-any.whl", hash = "sha256:d1e1e3b58374dc93031d6eda2420a48ea44a36c2b4766a4fdeb3710755731d76", size = 43906, upload-time = "2025-07-04T13:28:32.743Z" }, ] [[package]] name = "uritemplate" version = "4.2.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/98/60/f174043244c5306c9988380d2cb10009f91563fc4b31293d27e17201af56/uritemplate-4.2.0.tar.gz", hash = "sha256:480c2ed180878955863323eea31b0ede668795de182617fef9c6ca09e6ec9d0e", size = 33267 } +sdist = { url = "https://files.pythonhosted.org/packages/98/60/f174043244c5306c9988380d2cb10009f91563fc4b31293d27e17201af56/uritemplate-4.2.0.tar.gz", hash = "sha256:480c2ed180878955863323eea31b0ede668795de182617fef9c6ca09e6ec9d0e", size = 33267, upload-time = "2025-06-02T15:12:06.318Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/a9/99/3ae339466c9183ea5b8ae87b34c0b897eda475d2aec2307cae60e5cd4f29/uritemplate-4.2.0-py3-none-any.whl", hash = "sha256:962201ba1c4edcab02e60f9a0d3821e82dfc5d2d6662a21abd533879bdb8a686", size = 11488 }, + { url = "https://files.pythonhosted.org/packages/a9/99/3ae339466c9183ea5b8ae87b34c0b897eda475d2aec2307cae60e5cd4f29/uritemplate-4.2.0-py3-none-any.whl", hash = "sha256:962201ba1c4edcab02e60f9a0d3821e82dfc5d2d6662a21abd533879bdb8a686", size = 11488, upload-time = "2025-06-02T15:12:03.405Z" }, ] [[package]] name = "urllib3" version = "2.5.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/15/22/9ee70a2574a4f4599c47dd506532914ce044817c7752a79b6a51286319bc/urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760", size = 393185 } +sdist = { url = "https://files.pythonhosted.org/packages/15/22/9ee70a2574a4f4599c47dd506532914ce044817c7752a79b6a51286319bc/urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760", size = 393185, upload-time = "2025-06-18T14:07:41.644Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/a7/c2/fe1e52489ae3122415c51f387e221dd0773709bad6c6cdaa599e8a2c5185/urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc", size = 129795 }, + { url = "https://files.pythonhosted.org/packages/a7/c2/fe1e52489ae3122415c51f387e221dd0773709bad6c6cdaa599e8a2c5185/urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc", size = 129795, upload-time = "2025-06-18T14:07:40.39Z" }, ] [[package]] @@ -1782,35 +1782,35 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "click" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/d5/b7/30753098208505d7ff9be5b3a32112fb8a4cb3ddfccbbb7ba9973f2e29ff/userpath-1.9.2.tar.gz", hash = "sha256:6c52288dab069257cc831846d15d48133522455d4677ee69a9781f11dbefd815", size = 11140 } +sdist = { url = "https://files.pythonhosted.org/packages/d5/b7/30753098208505d7ff9be5b3a32112fb8a4cb3ddfccbbb7ba9973f2e29ff/userpath-1.9.2.tar.gz", hash = "sha256:6c52288dab069257cc831846d15d48133522455d4677ee69a9781f11dbefd815", size = 11140, upload-time = "2024-02-29T21:39:08.742Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/43/99/3ec6335ded5b88c2f7ed25c56ffd952546f7ed007ffb1e1539dc3b57015a/userpath-1.9.2-py3-none-any.whl", hash = "sha256:2cbf01a23d655a1ff8fc166dfb78da1b641d1ceabf0fe5f970767d380b14e89d", size = 9065 }, + { url = "https://files.pythonhosted.org/packages/43/99/3ec6335ded5b88c2f7ed25c56ffd952546f7ed007ffb1e1539dc3b57015a/userpath-1.9.2-py3-none-any.whl", hash = "sha256:2cbf01a23d655a1ff8fc166dfb78da1b641d1ceabf0fe5f970767d380b14e89d", size = 9065, upload-time = "2024-02-29T21:39:07.551Z" }, ] [[package]] name = "uv" version = "0.8.11" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a8/c1/765112567045a2219979d1a7038e4a2afbddd0637446556b089e77252528/uv-0.8.11.tar.gz", hash = "sha256:d98105244b895c6026e9f3d86f200b70039d39a5f4866022fae664ed935530f3", size = 3504312 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/4a/2f/6703896c45d29b44e5954bb283f00616387cef7ae80188226dac87aff93d/uv-0.8.11-py3-none-linux_armv6l.whl", hash = "sha256:1be7cbc874980dc3e5e0c40fdb3787013a35cce64485f7685fc4b0ee550f7c0c", size = 18497046 }, - { url = "https://files.pythonhosted.org/packages/61/fe/3ae518ea5a6c2e4fd3d0174486c841bd85e676b3971d9553445ab57319d9/uv-0.8.11-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:84c888cc7b3310aada6058ce964d9b48d4f7801add6f1236548adeb262c637bf", size = 18573000 }, - { url = "https://files.pythonhosted.org/packages/00/21/6a1cd01103aec916fdf2daa034e3a179a6b835b25db89f4f5e43117ac68c/uv-0.8.11-py3-none-macosx_11_0_arm64.whl", hash = "sha256:3e46395c7f2c7e52bf63f29f3fc1c6b357b011285d1df37d8af9c6f6f7cad36f", size = 17205164 }, - { url = "https://files.pythonhosted.org/packages/d0/b2/8a9e00d6e5c41a231f59f75c15b04626f7d4561364475962894a31b01fee/uv-0.8.11-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:d9d35783ac8600cd8e95e9afd007aa281edf3125803c570a4b3246138e2a304d", size = 17822163 }, - { url = "https://files.pythonhosted.org/packages/d9/e5/230f1ed3cbeae61d10ac8acc3d63b38a81c728161e7671fe3516aec72c76/uv-0.8.11-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ce267b13f498cebb9690c06461b727718bd11624679ddebb0a3998efe6b80ad7", size = 18152038 }, - { url = "https://files.pythonhosted.org/packages/95/be/7fd436adedd79c9afad14722135029010a972e17b05312795a976bc08854/uv-0.8.11-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c03aec1ad898642ae427b763cf5e5f90a678b91254f733ae08d01d15acd3672b", size = 18991855 }, - { url = "https://files.pythonhosted.org/packages/80/4e/2cca1be92fc3cdfddb5f2fa8d5650098948f357774cbe51810aaa5968da0/uv-0.8.11-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:83aa9c8b0085949542674301268e2b7b541f1108dc95664dedf50fffd1578f97", size = 20248085 }, - { url = "https://files.pythonhosted.org/packages/a5/9d/c4a5bbccfa45d8573d22da0d753329e572e72cd70796720dc0bc5c74e5c5/uv-0.8.11-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7e9506b3febbce3559290cb10cd1c84dbed32bc4f4b1062bc2fe4f093aa42aea", size = 19961250 }, - { url = "https://files.pythonhosted.org/packages/a4/f1/c1f9e59110fce261ee67cff854b4f95cae39a523d2a076c7566a704ebbe6/uv-0.8.11-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ba7bb038f0a263accefde1db68ecba7a756c85e6bcc25af161acef2711d6da19", size = 19314178 }, - { url = "https://files.pythonhosted.org/packages/fc/47/c398c3a9657a6f8c3a7b1938ae0b7061c4087e1fbb00f83a7a4f79005752/uv-0.8.11-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36eb184758f18347547045a3aa7cc87c98a75c773e437c8a85878eb004a31c2e", size = 19314121 }, - { url = "https://files.pythonhosted.org/packages/69/04/7ff94b68c33b93e89ec9920724b2a6d3992051584afd3410bf2604d2b93c/uv-0.8.11-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:0a7fcbe71cc5402b7c3d4c381f9b970a455d8ccc2a43ee2ce5ac2b617ec0534c", size = 18105431 }, - { url = "https://files.pythonhosted.org/packages/09/5a/aee6041cd0c9ab1c56da61ba1e9ac30b4ea7c1c85471e19cb0cc1e415c0a/uv-0.8.11-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:0da2c794dead209e660cb7df143ea9756c118ffa5874859e8a28a79101b5c760", size = 18984052 }, - { url = "https://files.pythonhosted.org/packages/05/cd/7b9926b676a3807312bfb91662813305b305c5218a05a9b651763b28267e/uv-0.8.11-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:0a95dc944d62db4ca282f7415c2d3c0fa3ead9e245a47d845515f5ddbd5a80ef", size = 18109344 }, - { url = "https://files.pythonhosted.org/packages/82/19/1e90e45fd84c4f5512dc9c8ad0ac3a4792677725047d3e7299f9dae41406/uv-0.8.11-py3-none-musllinux_1_1_i686.whl", hash = "sha256:0cd14f319e18a7b278238f0d87b18180282ec4d44d023f8b3ed2c8c091a14277", size = 18493945 }, - { url = "https://files.pythonhosted.org/packages/68/b8/e6b784ede573d3f1ba6fafe70dd317b4543146a6c2ca88a5f56923518552/uv-0.8.11-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:261d19395a211f980d1ebc861356cf73ba23ceece2392c0b36ade38f89fd16a6", size = 19398023 }, - { url = "https://files.pythonhosted.org/packages/65/5f/fd61ebec95bb5854c860d5268bc8ecbbca881465340f1e86302cacdd8234/uv-0.8.11-py3-none-win32.whl", hash = "sha256:0b922061f7b5915f224df23a849b6e1bfcace2e6b9fc0ee128868447873edb22", size = 18308608 }, - { url = "https://files.pythonhosted.org/packages/bb/57/84358ea67cee7ec029ed0d51e801a64c5929b7d647ae31cd5e5aea0c6f61/uv-0.8.11-py3-none-win_amd64.whl", hash = "sha256:fe01737f3ddd533903f31236219c29e09063541f17a060403acc51906ce0cfe8", size = 20214609 }, - { url = "https://files.pythonhosted.org/packages/e8/72/069a75703693d3297d95657957ea00d2f035896066f00a5692fbdce76d36/uv-0.8.11-py3-none-win_arm64.whl", hash = "sha256:cf3454d3407a5cac0d661b6033e3197643d0a6b5bb0e00869f6877ff7af907c9", size = 18878482 }, +sdist = { url = "https://files.pythonhosted.org/packages/a8/c1/765112567045a2219979d1a7038e4a2afbddd0637446556b089e77252528/uv-0.8.11.tar.gz", hash = "sha256:d98105244b895c6026e9f3d86f200b70039d39a5f4866022fae664ed935530f3", size = 3504312, upload-time = "2025-08-14T19:48:18.071Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4a/2f/6703896c45d29b44e5954bb283f00616387cef7ae80188226dac87aff93d/uv-0.8.11-py3-none-linux_armv6l.whl", hash = "sha256:1be7cbc874980dc3e5e0c40fdb3787013a35cce64485f7685fc4b0ee550f7c0c", size = 18497046, upload-time = "2025-08-14T19:47:28.18Z" }, + { url = "https://files.pythonhosted.org/packages/61/fe/3ae518ea5a6c2e4fd3d0174486c841bd85e676b3971d9553445ab57319d9/uv-0.8.11-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:84c888cc7b3310aada6058ce964d9b48d4f7801add6f1236548adeb262c637bf", size = 18573000, upload-time = "2025-08-14T19:47:32.156Z" }, + { url = "https://files.pythonhosted.org/packages/00/21/6a1cd01103aec916fdf2daa034e3a179a6b835b25db89f4f5e43117ac68c/uv-0.8.11-py3-none-macosx_11_0_arm64.whl", hash = "sha256:3e46395c7f2c7e52bf63f29f3fc1c6b357b011285d1df37d8af9c6f6f7cad36f", size = 17205164, upload-time = "2025-08-14T19:47:34.561Z" }, + { url = "https://files.pythonhosted.org/packages/d0/b2/8a9e00d6e5c41a231f59f75c15b04626f7d4561364475962894a31b01fee/uv-0.8.11-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:d9d35783ac8600cd8e95e9afd007aa281edf3125803c570a4b3246138e2a304d", size = 17822163, upload-time = "2025-08-14T19:47:37.111Z" }, + { url = "https://files.pythonhosted.org/packages/d9/e5/230f1ed3cbeae61d10ac8acc3d63b38a81c728161e7671fe3516aec72c76/uv-0.8.11-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ce267b13f498cebb9690c06461b727718bd11624679ddebb0a3998efe6b80ad7", size = 18152038, upload-time = "2025-08-14T19:47:39.951Z" }, + { url = "https://files.pythonhosted.org/packages/95/be/7fd436adedd79c9afad14722135029010a972e17b05312795a976bc08854/uv-0.8.11-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c03aec1ad898642ae427b763cf5e5f90a678b91254f733ae08d01d15acd3672b", size = 18991855, upload-time = "2025-08-14T19:47:42.664Z" }, + { url = "https://files.pythonhosted.org/packages/80/4e/2cca1be92fc3cdfddb5f2fa8d5650098948f357774cbe51810aaa5968da0/uv-0.8.11-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:83aa9c8b0085949542674301268e2b7b541f1108dc95664dedf50fffd1578f97", size = 20248085, upload-time = "2025-08-14T19:47:45.489Z" }, + { url = "https://files.pythonhosted.org/packages/a5/9d/c4a5bbccfa45d8573d22da0d753329e572e72cd70796720dc0bc5c74e5c5/uv-0.8.11-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7e9506b3febbce3559290cb10cd1c84dbed32bc4f4b1062bc2fe4f093aa42aea", size = 19961250, upload-time = "2025-08-14T19:47:47.963Z" }, + { url = "https://files.pythonhosted.org/packages/a4/f1/c1f9e59110fce261ee67cff854b4f95cae39a523d2a076c7566a704ebbe6/uv-0.8.11-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ba7bb038f0a263accefde1db68ecba7a756c85e6bcc25af161acef2711d6da19", size = 19314178, upload-time = "2025-08-14T19:47:50.469Z" }, + { url = "https://files.pythonhosted.org/packages/fc/47/c398c3a9657a6f8c3a7b1938ae0b7061c4087e1fbb00f83a7a4f79005752/uv-0.8.11-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36eb184758f18347547045a3aa7cc87c98a75c773e437c8a85878eb004a31c2e", size = 19314121, upload-time = "2025-08-14T19:47:54.17Z" }, + { url = "https://files.pythonhosted.org/packages/69/04/7ff94b68c33b93e89ec9920724b2a6d3992051584afd3410bf2604d2b93c/uv-0.8.11-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:0a7fcbe71cc5402b7c3d4c381f9b970a455d8ccc2a43ee2ce5ac2b617ec0534c", size = 18105431, upload-time = "2025-08-14T19:47:56.844Z" }, + { url = "https://files.pythonhosted.org/packages/09/5a/aee6041cd0c9ab1c56da61ba1e9ac30b4ea7c1c85471e19cb0cc1e415c0a/uv-0.8.11-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:0da2c794dead209e660cb7df143ea9756c118ffa5874859e8a28a79101b5c760", size = 18984052, upload-time = "2025-08-14T19:47:59.927Z" }, + { url = "https://files.pythonhosted.org/packages/05/cd/7b9926b676a3807312bfb91662813305b305c5218a05a9b651763b28267e/uv-0.8.11-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:0a95dc944d62db4ca282f7415c2d3c0fa3ead9e245a47d845515f5ddbd5a80ef", size = 18109344, upload-time = "2025-08-14T19:48:02.607Z" }, + { url = "https://files.pythonhosted.org/packages/82/19/1e90e45fd84c4f5512dc9c8ad0ac3a4792677725047d3e7299f9dae41406/uv-0.8.11-py3-none-musllinux_1_1_i686.whl", hash = "sha256:0cd14f319e18a7b278238f0d87b18180282ec4d44d023f8b3ed2c8c091a14277", size = 18493945, upload-time = "2025-08-14T19:48:05.112Z" }, + { url = "https://files.pythonhosted.org/packages/68/b8/e6b784ede573d3f1ba6fafe70dd317b4543146a6c2ca88a5f56923518552/uv-0.8.11-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:261d19395a211f980d1ebc861356cf73ba23ceece2392c0b36ade38f89fd16a6", size = 19398023, upload-time = "2025-08-14T19:48:07.993Z" }, + { url = "https://files.pythonhosted.org/packages/65/5f/fd61ebec95bb5854c860d5268bc8ecbbca881465340f1e86302cacdd8234/uv-0.8.11-py3-none-win32.whl", hash = "sha256:0b922061f7b5915f224df23a849b6e1bfcace2e6b9fc0ee128868447873edb22", size = 18308608, upload-time = "2025-08-14T19:48:10.847Z" }, + { url = "https://files.pythonhosted.org/packages/bb/57/84358ea67cee7ec029ed0d51e801a64c5929b7d647ae31cd5e5aea0c6f61/uv-0.8.11-py3-none-win_amd64.whl", hash = "sha256:fe01737f3ddd533903f31236219c29e09063541f17a060403acc51906ce0cfe8", size = 20214609, upload-time = "2025-08-14T19:48:13.368Z" }, + { url = "https://files.pythonhosted.org/packages/e8/72/069a75703693d3297d95657957ea00d2f035896066f00a5692fbdce76d36/uv-0.8.11-py3-none-win_arm64.whl", hash = "sha256:cf3454d3407a5cac0d661b6033e3197643d0a6b5bb0e00869f6877ff7af907c9", size = 18878482, upload-time = "2025-08-14T19:48:15.743Z" }, ] [[package]] @@ -1823,18 +1823,18 @@ dependencies = [ { name = "platformdirs" }, { name = "typing-extensions", marker = "python_full_version < '3.11'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/1c/14/37fcdba2808a6c615681cd216fecae00413c9dab44fb2e57805ecf3eaee3/virtualenv-20.34.0.tar.gz", hash = "sha256:44815b2c9dee7ed86e387b842a84f20b93f7f417f95886ca1996a72a4138eb1a", size = 6003808 } +sdist = { url = "https://files.pythonhosted.org/packages/1c/14/37fcdba2808a6c615681cd216fecae00413c9dab44fb2e57805ecf3eaee3/virtualenv-20.34.0.tar.gz", hash = "sha256:44815b2c9dee7ed86e387b842a84f20b93f7f417f95886ca1996a72a4138eb1a", size = 6003808, upload-time = "2025-08-13T14:24:07.464Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/76/06/04c8e804f813cf972e3262f3f8584c232de64f0cde9f703b46cf53a45090/virtualenv-20.34.0-py3-none-any.whl", hash = "sha256:341f5afa7eee943e4984a9207c025feedd768baff6753cd660c857ceb3e36026", size = 5983279 }, + { url = "https://files.pythonhosted.org/packages/76/06/04c8e804f813cf972e3262f3f8584c232de64f0cde9f703b46cf53a45090/virtualenv-20.34.0-py3-none-any.whl", hash = "sha256:341f5afa7eee943e4984a9207c025feedd768baff6753cd660c857ceb3e36026", size = 5983279, upload-time = "2025-08-13T14:24:05.111Z" }, ] [[package]] name = "zipp" version = "3.23.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/e3/02/0f2892c661036d50ede074e376733dca2ae7c6eb617489437771209d4180/zipp-3.23.0.tar.gz", hash = "sha256:a07157588a12518c9d4034df3fbbee09c814741a33ff63c05fa29d26a2404166", size = 25547 } +sdist = { url = "https://files.pythonhosted.org/packages/e3/02/0f2892c661036d50ede074e376733dca2ae7c6eb617489437771209d4180/zipp-3.23.0.tar.gz", hash = "sha256:a07157588a12518c9d4034df3fbbee09c814741a33ff63c05fa29d26a2404166", size = 25547, upload-time = "2025-06-08T17:06:39.4Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2e/54/647ade08bf0db230bfea292f893923872fd20be6ac6f53b2b936ba839d75/zipp-3.23.0-py3-none-any.whl", hash = "sha256:071652d6115ed432f5ce1d34c336c0adfd6a884660d1e9712a256d3d3bd4b14e", size = 10276 }, + { url = "https://files.pythonhosted.org/packages/2e/54/647ade08bf0db230bfea292f893923872fd20be6ac6f53b2b936ba839d75/zipp-3.23.0-py3-none-any.whl", hash = "sha256:071652d6115ed432f5ce1d34c336c0adfd6a884660d1e9712a256d3d3bd4b14e", size = 10276, upload-time = "2025-06-08T17:06:38.034Z" }, ] [[package]] @@ -1844,70 +1844,70 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "cffi", marker = "platform_python_implementation == 'PyPy'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/ed/f6/2ac0287b442160a89d726b17a9184a4c615bb5237db763791a7fd16d9df1/zstandard-0.23.0.tar.gz", hash = "sha256:b2d8c62d08e7255f68f7a740bae85b3c9b8e5466baa9cbf7f57f1cde0ac6bc09", size = 681701 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/2a/55/bd0487e86679db1823fc9ee0d8c9c78ae2413d34c0b461193b5f4c31d22f/zstandard-0.23.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bf0a05b6059c0528477fba9054d09179beb63744355cab9f38059548fedd46a9", size = 788701 }, - { url = "https://files.pythonhosted.org/packages/e1/8a/ccb516b684f3ad987dfee27570d635822e3038645b1a950c5e8022df1145/zstandard-0.23.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:fc9ca1c9718cb3b06634c7c8dec57d24e9438b2aa9a0f02b8bb36bf478538880", size = 633678 }, - { url = "https://files.pythonhosted.org/packages/12/89/75e633d0611c028e0d9af6df199423bf43f54bea5007e6718ab7132e234c/zstandard-0.23.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77da4c6bfa20dd5ea25cbf12c76f181a8e8cd7ea231c673828d0386b1740b8dc", size = 4941098 }, - { url = "https://files.pythonhosted.org/packages/4a/7a/bd7f6a21802de358b63f1ee636ab823711c25ce043a3e9f043b4fcb5ba32/zstandard-0.23.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b2170c7e0367dde86a2647ed5b6f57394ea7f53545746104c6b09fc1f4223573", size = 5308798 }, - { url = "https://files.pythonhosted.org/packages/79/3b/775f851a4a65013e88ca559c8ae42ac1352db6fcd96b028d0df4d7d1d7b4/zstandard-0.23.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c16842b846a8d2a145223f520b7e18b57c8f476924bda92aeee3a88d11cfc391", size = 5341840 }, - { url = "https://files.pythonhosted.org/packages/09/4f/0cc49570141dd72d4d95dd6fcf09328d1b702c47a6ec12fbed3b8aed18a5/zstandard-0.23.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:157e89ceb4054029a289fb504c98c6a9fe8010f1680de0201b3eb5dc20aa6d9e", size = 5440337 }, - { url = "https://files.pythonhosted.org/packages/e7/7c/aaa7cd27148bae2dc095191529c0570d16058c54c4597a7d118de4b21676/zstandard-0.23.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:203d236f4c94cd8379d1ea61db2fce20730b4c38d7f1c34506a31b34edc87bdd", size = 4861182 }, - { url = "https://files.pythonhosted.org/packages/ac/eb/4b58b5c071d177f7dc027129d20bd2a44161faca6592a67f8fcb0b88b3ae/zstandard-0.23.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:dc5d1a49d3f8262be192589a4b72f0d03b72dcf46c51ad5852a4fdc67be7b9e4", size = 4932936 }, - { url = "https://files.pythonhosted.org/packages/44/f9/21a5fb9bb7c9a274b05ad700a82ad22ce82f7ef0f485980a1e98ed6e8c5f/zstandard-0.23.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:752bf8a74412b9892f4e5b58f2f890a039f57037f52c89a740757ebd807f33ea", size = 5464705 }, - { url = "https://files.pythonhosted.org/packages/49/74/b7b3e61db3f88632776b78b1db597af3f44c91ce17d533e14a25ce6a2816/zstandard-0.23.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:80080816b4f52a9d886e67f1f96912891074903238fe54f2de8b786f86baded2", size = 4857882 }, - { url = "https://files.pythonhosted.org/packages/4a/7f/d8eb1cb123d8e4c541d4465167080bec88481ab54cd0b31eb4013ba04b95/zstandard-0.23.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:84433dddea68571a6d6bd4fbf8ff398236031149116a7fff6f777ff95cad3df9", size = 4697672 }, - { url = "https://files.pythonhosted.org/packages/5e/05/f7dccdf3d121309b60342da454d3e706453a31073e2c4dac8e1581861e44/zstandard-0.23.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:ab19a2d91963ed9e42b4e8d77cd847ae8381576585bad79dbd0a8837a9f6620a", size = 5206043 }, - { url = "https://files.pythonhosted.org/packages/86/9d/3677a02e172dccd8dd3a941307621c0cbd7691d77cb435ac3c75ab6a3105/zstandard-0.23.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:59556bf80a7094d0cfb9f5e50bb2db27fefb75d5138bb16fb052b61b0e0eeeb0", size = 5667390 }, - { url = "https://files.pythonhosted.org/packages/41/7e/0012a02458e74a7ba122cd9cafe491facc602c9a17f590367da369929498/zstandard-0.23.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:27d3ef2252d2e62476389ca8f9b0cf2bbafb082a3b6bfe9d90cbcbb5529ecf7c", size = 5198901 }, - { url = "https://files.pythonhosted.org/packages/65/3a/8f715b97bd7bcfc7342d8adcd99a026cb2fb550e44866a3b6c348e1b0f02/zstandard-0.23.0-cp310-cp310-win32.whl", hash = "sha256:5d41d5e025f1e0bccae4928981e71b2334c60f580bdc8345f824e7c0a4c2a813", size = 430596 }, - { url = "https://files.pythonhosted.org/packages/19/b7/b2b9eca5e5a01111e4fe8a8ffb56bdcdf56b12448a24effe6cfe4a252034/zstandard-0.23.0-cp310-cp310-win_amd64.whl", hash = "sha256:519fbf169dfac1222a76ba8861ef4ac7f0530c35dd79ba5727014613f91613d4", size = 495498 }, - { url = "https://files.pythonhosted.org/packages/9e/40/f67e7d2c25a0e2dc1744dd781110b0b60306657f8696cafb7ad7579469bd/zstandard-0.23.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:34895a41273ad33347b2fc70e1bff4240556de3c46c6ea430a7ed91f9042aa4e", size = 788699 }, - { url = "https://files.pythonhosted.org/packages/e8/46/66d5b55f4d737dd6ab75851b224abf0afe5774976fe511a54d2eb9063a41/zstandard-0.23.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:77ea385f7dd5b5676d7fd943292ffa18fbf5c72ba98f7d09fc1fb9e819b34c23", size = 633681 }, - { url = "https://files.pythonhosted.org/packages/63/b6/677e65c095d8e12b66b8f862b069bcf1f1d781b9c9c6f12eb55000d57583/zstandard-0.23.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:983b6efd649723474f29ed42e1467f90a35a74793437d0bc64a5bf482bedfa0a", size = 4944328 }, - { url = "https://files.pythonhosted.org/packages/59/cc/e76acb4c42afa05a9d20827116d1f9287e9c32b7ad58cc3af0721ce2b481/zstandard-0.23.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80a539906390591dd39ebb8d773771dc4db82ace6372c4d41e2d293f8e32b8db", size = 5311955 }, - { url = "https://files.pythonhosted.org/packages/78/e4/644b8075f18fc7f632130c32e8f36f6dc1b93065bf2dd87f03223b187f26/zstandard-0.23.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:445e4cb5048b04e90ce96a79b4b63140e3f4ab5f662321975679b5f6360b90e2", size = 5344944 }, - { url = "https://files.pythonhosted.org/packages/76/3f/dbafccf19cfeca25bbabf6f2dd81796b7218f768ec400f043edc767015a6/zstandard-0.23.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd30d9c67d13d891f2360b2a120186729c111238ac63b43dbd37a5a40670b8ca", size = 5442927 }, - { url = "https://files.pythonhosted.org/packages/0c/c3/d24a01a19b6733b9f218e94d1a87c477d523237e07f94899e1c10f6fd06c/zstandard-0.23.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d20fd853fbb5807c8e84c136c278827b6167ded66c72ec6f9a14b863d809211c", size = 4864910 }, - { url = "https://files.pythonhosted.org/packages/1c/a9/cf8f78ead4597264f7618d0875be01f9bc23c9d1d11afb6d225b867cb423/zstandard-0.23.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:ed1708dbf4d2e3a1c5c69110ba2b4eb6678262028afd6c6fbcc5a8dac9cda68e", size = 4935544 }, - { url = "https://files.pythonhosted.org/packages/2c/96/8af1e3731b67965fb995a940c04a2c20997a7b3b14826b9d1301cf160879/zstandard-0.23.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:be9b5b8659dff1f913039c2feee1aca499cfbc19e98fa12bc85e037c17ec6ca5", size = 5467094 }, - { url = "https://files.pythonhosted.org/packages/ff/57/43ea9df642c636cb79f88a13ab07d92d88d3bfe3e550b55a25a07a26d878/zstandard-0.23.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:65308f4b4890aa12d9b6ad9f2844b7ee42c7f7a4fd3390425b242ffc57498f48", size = 4860440 }, - { url = "https://files.pythonhosted.org/packages/46/37/edb78f33c7f44f806525f27baa300341918fd4c4af9472fbc2c3094be2e8/zstandard-0.23.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:98da17ce9cbf3bfe4617e836d561e433f871129e3a7ac16d6ef4c680f13a839c", size = 4700091 }, - { url = "https://files.pythonhosted.org/packages/c1/f1/454ac3962671a754f3cb49242472df5c2cced4eb959ae203a377b45b1a3c/zstandard-0.23.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:8ed7d27cb56b3e058d3cf684d7200703bcae623e1dcc06ed1e18ecda39fee003", size = 5208682 }, - { url = "https://files.pythonhosted.org/packages/85/b2/1734b0fff1634390b1b887202d557d2dd542de84a4c155c258cf75da4773/zstandard-0.23.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:b69bb4f51daf461b15e7b3db033160937d3ff88303a7bc808c67bbc1eaf98c78", size = 5669707 }, - { url = "https://files.pythonhosted.org/packages/52/5a/87d6971f0997c4b9b09c495bf92189fb63de86a83cadc4977dc19735f652/zstandard-0.23.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:034b88913ecc1b097f528e42b539453fa82c3557e414b3de9d5632c80439a473", size = 5201792 }, - { url = "https://files.pythonhosted.org/packages/79/02/6f6a42cc84459d399bd1a4e1adfc78d4dfe45e56d05b072008d10040e13b/zstandard-0.23.0-cp311-cp311-win32.whl", hash = "sha256:f2d4380bf5f62daabd7b751ea2339c1a21d1c9463f1feb7fc2bdcea2c29c3160", size = 430586 }, - { url = "https://files.pythonhosted.org/packages/be/a2/4272175d47c623ff78196f3c10e9dc7045c1b9caf3735bf041e65271eca4/zstandard-0.23.0-cp311-cp311-win_amd64.whl", hash = "sha256:62136da96a973bd2557f06ddd4e8e807f9e13cbb0bfb9cc06cfe6d98ea90dfe0", size = 495420 }, - { url = "https://files.pythonhosted.org/packages/7b/83/f23338c963bd9de687d47bf32efe9fd30164e722ba27fb59df33e6b1719b/zstandard-0.23.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:b4567955a6bc1b20e9c31612e615af6b53733491aeaa19a6b3b37f3b65477094", size = 788713 }, - { url = "https://files.pythonhosted.org/packages/5b/b3/1a028f6750fd9227ee0b937a278a434ab7f7fdc3066c3173f64366fe2466/zstandard-0.23.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1e172f57cd78c20f13a3415cc8dfe24bf388614324d25539146594c16d78fcc8", size = 633459 }, - { url = "https://files.pythonhosted.org/packages/26/af/36d89aae0c1f95a0a98e50711bc5d92c144939efc1f81a2fcd3e78d7f4c1/zstandard-0.23.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b0e166f698c5a3e914947388c162be2583e0c638a4703fc6a543e23a88dea3c1", size = 4945707 }, - { url = "https://files.pythonhosted.org/packages/cd/2e/2051f5c772f4dfc0aae3741d5fc72c3dcfe3aaeb461cc231668a4db1ce14/zstandard-0.23.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:12a289832e520c6bd4dcaad68e944b86da3bad0d339ef7989fb7e88f92e96072", size = 5306545 }, - { url = "https://files.pythonhosted.org/packages/0a/9e/a11c97b087f89cab030fa71206963090d2fecd8eb83e67bb8f3ffb84c024/zstandard-0.23.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d50d31bfedd53a928fed6707b15a8dbeef011bb6366297cc435accc888b27c20", size = 5337533 }, - { url = "https://files.pythonhosted.org/packages/fc/79/edeb217c57fe1bf16d890aa91a1c2c96b28c07b46afed54a5dcf310c3f6f/zstandard-0.23.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:72c68dda124a1a138340fb62fa21b9bf4848437d9ca60bd35db36f2d3345f373", size = 5436510 }, - { url = "https://files.pythonhosted.org/packages/81/4f/c21383d97cb7a422ddf1ae824b53ce4b51063d0eeb2afa757eb40804a8ef/zstandard-0.23.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:53dd9d5e3d29f95acd5de6802e909ada8d8d8cfa37a3ac64836f3bc4bc5512db", size = 4859973 }, - { url = "https://files.pythonhosted.org/packages/ab/15/08d22e87753304405ccac8be2493a495f529edd81d39a0870621462276ef/zstandard-0.23.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:6a41c120c3dbc0d81a8e8adc73312d668cd34acd7725f036992b1b72d22c1772", size = 4936968 }, - { url = "https://files.pythonhosted.org/packages/eb/fa/f3670a597949fe7dcf38119a39f7da49a8a84a6f0b1a2e46b2f71a0ab83f/zstandard-0.23.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:40b33d93c6eddf02d2c19f5773196068d875c41ca25730e8288e9b672897c105", size = 5467179 }, - { url = "https://files.pythonhosted.org/packages/4e/a9/dad2ab22020211e380adc477a1dbf9f109b1f8d94c614944843e20dc2a99/zstandard-0.23.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9206649ec587e6b02bd124fb7799b86cddec350f6f6c14bc82a2b70183e708ba", size = 4848577 }, - { url = "https://files.pythonhosted.org/packages/08/03/dd28b4484b0770f1e23478413e01bee476ae8227bbc81561f9c329e12564/zstandard-0.23.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:76e79bc28a65f467e0409098fa2c4376931fd3207fbeb6b956c7c476d53746dd", size = 4693899 }, - { url = "https://files.pythonhosted.org/packages/2b/64/3da7497eb635d025841e958bcd66a86117ae320c3b14b0ae86e9e8627518/zstandard-0.23.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:66b689c107857eceabf2cf3d3fc699c3c0fe8ccd18df2219d978c0283e4c508a", size = 5199964 }, - { url = "https://files.pythonhosted.org/packages/43/a4/d82decbab158a0e8a6ebb7fc98bc4d903266bce85b6e9aaedea1d288338c/zstandard-0.23.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9c236e635582742fee16603042553d276cca506e824fa2e6489db04039521e90", size = 5655398 }, - { url = "https://files.pythonhosted.org/packages/f2/61/ac78a1263bc83a5cf29e7458b77a568eda5a8f81980691bbc6eb6a0d45cc/zstandard-0.23.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a8fffdbd9d1408006baaf02f1068d7dd1f016c6bcb7538682622c556e7b68e35", size = 5191313 }, - { url = "https://files.pythonhosted.org/packages/e7/54/967c478314e16af5baf849b6ee9d6ea724ae5b100eb506011f045d3d4e16/zstandard-0.23.0-cp312-cp312-win32.whl", hash = "sha256:dc1d33abb8a0d754ea4763bad944fd965d3d95b5baef6b121c0c9013eaf1907d", size = 430877 }, - { url = "https://files.pythonhosted.org/packages/75/37/872d74bd7739639c4553bf94c84af7d54d8211b626b352bc57f0fd8d1e3f/zstandard-0.23.0-cp312-cp312-win_amd64.whl", hash = "sha256:64585e1dba664dc67c7cdabd56c1e5685233fbb1fc1966cfba2a340ec0dfff7b", size = 495595 }, - { url = "https://files.pythonhosted.org/packages/80/f1/8386f3f7c10261fe85fbc2c012fdb3d4db793b921c9abcc995d8da1b7a80/zstandard-0.23.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:576856e8594e6649aee06ddbfc738fec6a834f7c85bf7cadd1c53d4a58186ef9", size = 788975 }, - { url = "https://files.pythonhosted.org/packages/16/e8/cbf01077550b3e5dc86089035ff8f6fbbb312bc0983757c2d1117ebba242/zstandard-0.23.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:38302b78a850ff82656beaddeb0bb989a0322a8bbb1bf1ab10c17506681d772a", size = 633448 }, - { url = "https://files.pythonhosted.org/packages/06/27/4a1b4c267c29a464a161aeb2589aff212b4db653a1d96bffe3598f3f0d22/zstandard-0.23.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d2240ddc86b74966c34554c49d00eaafa8200a18d3a5b6ffbf7da63b11d74ee2", size = 4945269 }, - { url = "https://files.pythonhosted.org/packages/7c/64/d99261cc57afd9ae65b707e38045ed8269fbdae73544fd2e4a4d50d0ed83/zstandard-0.23.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2ef230a8fd217a2015bc91b74f6b3b7d6522ba48be29ad4ea0ca3a3775bf7dd5", size = 5306228 }, - { url = "https://files.pythonhosted.org/packages/7a/cf/27b74c6f22541f0263016a0fd6369b1b7818941de639215c84e4e94b2a1c/zstandard-0.23.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:774d45b1fac1461f48698a9d4b5fa19a69d47ece02fa469825b442263f04021f", size = 5336891 }, - { url = "https://files.pythonhosted.org/packages/fa/18/89ac62eac46b69948bf35fcd90d37103f38722968e2981f752d69081ec4d/zstandard-0.23.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6f77fa49079891a4aab203d0b1744acc85577ed16d767b52fc089d83faf8d8ed", size = 5436310 }, - { url = "https://files.pythonhosted.org/packages/a8/a8/5ca5328ee568a873f5118d5b5f70d1f36c6387716efe2e369010289a5738/zstandard-0.23.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ac184f87ff521f4840e6ea0b10c0ec90c6b1dcd0bad2f1e4a9a1b4fa177982ea", size = 4859912 }, - { url = "https://files.pythonhosted.org/packages/ea/ca/3781059c95fd0868658b1cf0440edd832b942f84ae60685d0cfdb808bca1/zstandard-0.23.0-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:c363b53e257246a954ebc7c488304b5592b9c53fbe74d03bc1c64dda153fb847", size = 4936946 }, - { url = "https://files.pythonhosted.org/packages/ce/11/41a58986f809532742c2b832c53b74ba0e0a5dae7e8ab4642bf5876f35de/zstandard-0.23.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:e7792606d606c8df5277c32ccb58f29b9b8603bf83b48639b7aedf6df4fe8171", size = 5466994 }, - { url = "https://files.pythonhosted.org/packages/83/e3/97d84fe95edd38d7053af05159465d298c8b20cebe9ccb3d26783faa9094/zstandard-0.23.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a0817825b900fcd43ac5d05b8b3079937073d2b1ff9cf89427590718b70dd840", size = 4848681 }, - { url = "https://files.pythonhosted.org/packages/6e/99/cb1e63e931de15c88af26085e3f2d9af9ce53ccafac73b6e48418fd5a6e6/zstandard-0.23.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:9da6bc32faac9a293ddfdcb9108d4b20416219461e4ec64dfea8383cac186690", size = 4694239 }, - { url = "https://files.pythonhosted.org/packages/ab/50/b1e703016eebbc6501fc92f34db7b1c68e54e567ef39e6e59cf5fb6f2ec0/zstandard-0.23.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:fd7699e8fd9969f455ef2926221e0233f81a2542921471382e77a9e2f2b57f4b", size = 5200149 }, - { url = "https://files.pythonhosted.org/packages/aa/e0/932388630aaba70197c78bdb10cce2c91fae01a7e553b76ce85471aec690/zstandard-0.23.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:d477ed829077cd945b01fc3115edd132c47e6540ddcd96ca169facff28173057", size = 5655392 }, - { url = "https://files.pythonhosted.org/packages/02/90/2633473864f67a15526324b007a9f96c96f56d5f32ef2a56cc12f9548723/zstandard-0.23.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:fa6ce8b52c5987b3e34d5674b0ab529a4602b632ebab0a93b07bfb4dfc8f8a33", size = 5191299 }, - { url = "https://files.pythonhosted.org/packages/b0/4c/315ca5c32da7e2dc3455f3b2caee5c8c2246074a61aac6ec3378a97b7136/zstandard-0.23.0-cp313-cp313-win32.whl", hash = "sha256:a9b07268d0c3ca5c170a385a0ab9fb7fdd9f5fd866be004c4ea39e44edce47dd", size = 430862 }, - { url = "https://files.pythonhosted.org/packages/a2/bf/c6aaba098e2d04781e8f4f7c0ba3c7aa73d00e4c436bcc0cf059a66691d1/zstandard-0.23.0-cp313-cp313-win_amd64.whl", hash = "sha256:f3513916e8c645d0610815c257cbfd3242adfd5c4cfa78be514e5a3ebb42a41b", size = 495578 }, +sdist = { url = "https://files.pythonhosted.org/packages/ed/f6/2ac0287b442160a89d726b17a9184a4c615bb5237db763791a7fd16d9df1/zstandard-0.23.0.tar.gz", hash = "sha256:b2d8c62d08e7255f68f7a740bae85b3c9b8e5466baa9cbf7f57f1cde0ac6bc09", size = 681701, upload-time = "2024-07-15T00:18:06.141Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/2a/55/bd0487e86679db1823fc9ee0d8c9c78ae2413d34c0b461193b5f4c31d22f/zstandard-0.23.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bf0a05b6059c0528477fba9054d09179beb63744355cab9f38059548fedd46a9", size = 788701, upload-time = "2024-07-15T00:13:27.351Z" }, + { url = "https://files.pythonhosted.org/packages/e1/8a/ccb516b684f3ad987dfee27570d635822e3038645b1a950c5e8022df1145/zstandard-0.23.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:fc9ca1c9718cb3b06634c7c8dec57d24e9438b2aa9a0f02b8bb36bf478538880", size = 633678, upload-time = "2024-07-15T00:13:30.24Z" }, + { url = "https://files.pythonhosted.org/packages/12/89/75e633d0611c028e0d9af6df199423bf43f54bea5007e6718ab7132e234c/zstandard-0.23.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77da4c6bfa20dd5ea25cbf12c76f181a8e8cd7ea231c673828d0386b1740b8dc", size = 4941098, upload-time = "2024-07-15T00:13:32.526Z" }, + { url = "https://files.pythonhosted.org/packages/4a/7a/bd7f6a21802de358b63f1ee636ab823711c25ce043a3e9f043b4fcb5ba32/zstandard-0.23.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b2170c7e0367dde86a2647ed5b6f57394ea7f53545746104c6b09fc1f4223573", size = 5308798, upload-time = "2024-07-15T00:13:34.925Z" }, + { url = "https://files.pythonhosted.org/packages/79/3b/775f851a4a65013e88ca559c8ae42ac1352db6fcd96b028d0df4d7d1d7b4/zstandard-0.23.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c16842b846a8d2a145223f520b7e18b57c8f476924bda92aeee3a88d11cfc391", size = 5341840, upload-time = "2024-07-15T00:13:37.376Z" }, + { url = "https://files.pythonhosted.org/packages/09/4f/0cc49570141dd72d4d95dd6fcf09328d1b702c47a6ec12fbed3b8aed18a5/zstandard-0.23.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:157e89ceb4054029a289fb504c98c6a9fe8010f1680de0201b3eb5dc20aa6d9e", size = 5440337, upload-time = "2024-07-15T00:13:39.772Z" }, + { url = "https://files.pythonhosted.org/packages/e7/7c/aaa7cd27148bae2dc095191529c0570d16058c54c4597a7d118de4b21676/zstandard-0.23.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:203d236f4c94cd8379d1ea61db2fce20730b4c38d7f1c34506a31b34edc87bdd", size = 4861182, upload-time = "2024-07-15T00:13:42.495Z" }, + { url = "https://files.pythonhosted.org/packages/ac/eb/4b58b5c071d177f7dc027129d20bd2a44161faca6592a67f8fcb0b88b3ae/zstandard-0.23.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:dc5d1a49d3f8262be192589a4b72f0d03b72dcf46c51ad5852a4fdc67be7b9e4", size = 4932936, upload-time = "2024-07-15T00:13:44.234Z" }, + { url = "https://files.pythonhosted.org/packages/44/f9/21a5fb9bb7c9a274b05ad700a82ad22ce82f7ef0f485980a1e98ed6e8c5f/zstandard-0.23.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:752bf8a74412b9892f4e5b58f2f890a039f57037f52c89a740757ebd807f33ea", size = 5464705, upload-time = "2024-07-15T00:13:46.822Z" }, + { url = "https://files.pythonhosted.org/packages/49/74/b7b3e61db3f88632776b78b1db597af3f44c91ce17d533e14a25ce6a2816/zstandard-0.23.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:80080816b4f52a9d886e67f1f96912891074903238fe54f2de8b786f86baded2", size = 4857882, upload-time = "2024-07-15T00:13:49.297Z" }, + { url = "https://files.pythonhosted.org/packages/4a/7f/d8eb1cb123d8e4c541d4465167080bec88481ab54cd0b31eb4013ba04b95/zstandard-0.23.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:84433dddea68571a6d6bd4fbf8ff398236031149116a7fff6f777ff95cad3df9", size = 4697672, upload-time = "2024-07-15T00:13:51.447Z" }, + { url = "https://files.pythonhosted.org/packages/5e/05/f7dccdf3d121309b60342da454d3e706453a31073e2c4dac8e1581861e44/zstandard-0.23.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:ab19a2d91963ed9e42b4e8d77cd847ae8381576585bad79dbd0a8837a9f6620a", size = 5206043, upload-time = "2024-07-15T00:13:53.587Z" }, + { url = "https://files.pythonhosted.org/packages/86/9d/3677a02e172dccd8dd3a941307621c0cbd7691d77cb435ac3c75ab6a3105/zstandard-0.23.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:59556bf80a7094d0cfb9f5e50bb2db27fefb75d5138bb16fb052b61b0e0eeeb0", size = 5667390, upload-time = "2024-07-15T00:13:56.137Z" }, + { url = "https://files.pythonhosted.org/packages/41/7e/0012a02458e74a7ba122cd9cafe491facc602c9a17f590367da369929498/zstandard-0.23.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:27d3ef2252d2e62476389ca8f9b0cf2bbafb082a3b6bfe9d90cbcbb5529ecf7c", size = 5198901, upload-time = "2024-07-15T00:13:58.584Z" }, + { url = "https://files.pythonhosted.org/packages/65/3a/8f715b97bd7bcfc7342d8adcd99a026cb2fb550e44866a3b6c348e1b0f02/zstandard-0.23.0-cp310-cp310-win32.whl", hash = "sha256:5d41d5e025f1e0bccae4928981e71b2334c60f580bdc8345f824e7c0a4c2a813", size = 430596, upload-time = "2024-07-15T00:14:00.693Z" }, + { url = "https://files.pythonhosted.org/packages/19/b7/b2b9eca5e5a01111e4fe8a8ffb56bdcdf56b12448a24effe6cfe4a252034/zstandard-0.23.0-cp310-cp310-win_amd64.whl", hash = "sha256:519fbf169dfac1222a76ba8861ef4ac7f0530c35dd79ba5727014613f91613d4", size = 495498, upload-time = "2024-07-15T00:14:02.741Z" }, + { url = "https://files.pythonhosted.org/packages/9e/40/f67e7d2c25a0e2dc1744dd781110b0b60306657f8696cafb7ad7579469bd/zstandard-0.23.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:34895a41273ad33347b2fc70e1bff4240556de3c46c6ea430a7ed91f9042aa4e", size = 788699, upload-time = "2024-07-15T00:14:04.909Z" }, + { url = "https://files.pythonhosted.org/packages/e8/46/66d5b55f4d737dd6ab75851b224abf0afe5774976fe511a54d2eb9063a41/zstandard-0.23.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:77ea385f7dd5b5676d7fd943292ffa18fbf5c72ba98f7d09fc1fb9e819b34c23", size = 633681, upload-time = "2024-07-15T00:14:13.99Z" }, + { url = "https://files.pythonhosted.org/packages/63/b6/677e65c095d8e12b66b8f862b069bcf1f1d781b9c9c6f12eb55000d57583/zstandard-0.23.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:983b6efd649723474f29ed42e1467f90a35a74793437d0bc64a5bf482bedfa0a", size = 4944328, upload-time = "2024-07-15T00:14:16.588Z" }, + { url = "https://files.pythonhosted.org/packages/59/cc/e76acb4c42afa05a9d20827116d1f9287e9c32b7ad58cc3af0721ce2b481/zstandard-0.23.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80a539906390591dd39ebb8d773771dc4db82ace6372c4d41e2d293f8e32b8db", size = 5311955, upload-time = "2024-07-15T00:14:19.389Z" }, + { url = "https://files.pythonhosted.org/packages/78/e4/644b8075f18fc7f632130c32e8f36f6dc1b93065bf2dd87f03223b187f26/zstandard-0.23.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:445e4cb5048b04e90ce96a79b4b63140e3f4ab5f662321975679b5f6360b90e2", size = 5344944, upload-time = "2024-07-15T00:14:22.173Z" }, + { url = "https://files.pythonhosted.org/packages/76/3f/dbafccf19cfeca25bbabf6f2dd81796b7218f768ec400f043edc767015a6/zstandard-0.23.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd30d9c67d13d891f2360b2a120186729c111238ac63b43dbd37a5a40670b8ca", size = 5442927, upload-time = "2024-07-15T00:14:24.825Z" }, + { url = "https://files.pythonhosted.org/packages/0c/c3/d24a01a19b6733b9f218e94d1a87c477d523237e07f94899e1c10f6fd06c/zstandard-0.23.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d20fd853fbb5807c8e84c136c278827b6167ded66c72ec6f9a14b863d809211c", size = 4864910, upload-time = "2024-07-15T00:14:26.982Z" }, + { url = "https://files.pythonhosted.org/packages/1c/a9/cf8f78ead4597264f7618d0875be01f9bc23c9d1d11afb6d225b867cb423/zstandard-0.23.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:ed1708dbf4d2e3a1c5c69110ba2b4eb6678262028afd6c6fbcc5a8dac9cda68e", size = 4935544, upload-time = "2024-07-15T00:14:29.582Z" }, + { url = "https://files.pythonhosted.org/packages/2c/96/8af1e3731b67965fb995a940c04a2c20997a7b3b14826b9d1301cf160879/zstandard-0.23.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:be9b5b8659dff1f913039c2feee1aca499cfbc19e98fa12bc85e037c17ec6ca5", size = 5467094, upload-time = "2024-07-15T00:14:40.126Z" }, + { url = "https://files.pythonhosted.org/packages/ff/57/43ea9df642c636cb79f88a13ab07d92d88d3bfe3e550b55a25a07a26d878/zstandard-0.23.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:65308f4b4890aa12d9b6ad9f2844b7ee42c7f7a4fd3390425b242ffc57498f48", size = 4860440, upload-time = "2024-07-15T00:14:42.786Z" }, + { url = "https://files.pythonhosted.org/packages/46/37/edb78f33c7f44f806525f27baa300341918fd4c4af9472fbc2c3094be2e8/zstandard-0.23.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:98da17ce9cbf3bfe4617e836d561e433f871129e3a7ac16d6ef4c680f13a839c", size = 4700091, upload-time = "2024-07-15T00:14:45.184Z" }, + { url = "https://files.pythonhosted.org/packages/c1/f1/454ac3962671a754f3cb49242472df5c2cced4eb959ae203a377b45b1a3c/zstandard-0.23.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:8ed7d27cb56b3e058d3cf684d7200703bcae623e1dcc06ed1e18ecda39fee003", size = 5208682, upload-time = "2024-07-15T00:14:47.407Z" }, + { url = "https://files.pythonhosted.org/packages/85/b2/1734b0fff1634390b1b887202d557d2dd542de84a4c155c258cf75da4773/zstandard-0.23.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:b69bb4f51daf461b15e7b3db033160937d3ff88303a7bc808c67bbc1eaf98c78", size = 5669707, upload-time = "2024-07-15T00:15:03.529Z" }, + { url = "https://files.pythonhosted.org/packages/52/5a/87d6971f0997c4b9b09c495bf92189fb63de86a83cadc4977dc19735f652/zstandard-0.23.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:034b88913ecc1b097f528e42b539453fa82c3557e414b3de9d5632c80439a473", size = 5201792, upload-time = "2024-07-15T00:15:28.372Z" }, + { url = "https://files.pythonhosted.org/packages/79/02/6f6a42cc84459d399bd1a4e1adfc78d4dfe45e56d05b072008d10040e13b/zstandard-0.23.0-cp311-cp311-win32.whl", hash = "sha256:f2d4380bf5f62daabd7b751ea2339c1a21d1c9463f1feb7fc2bdcea2c29c3160", size = 430586, upload-time = "2024-07-15T00:15:32.26Z" }, + { url = "https://files.pythonhosted.org/packages/be/a2/4272175d47c623ff78196f3c10e9dc7045c1b9caf3735bf041e65271eca4/zstandard-0.23.0-cp311-cp311-win_amd64.whl", hash = "sha256:62136da96a973bd2557f06ddd4e8e807f9e13cbb0bfb9cc06cfe6d98ea90dfe0", size = 495420, upload-time = "2024-07-15T00:15:34.004Z" }, + { url = "https://files.pythonhosted.org/packages/7b/83/f23338c963bd9de687d47bf32efe9fd30164e722ba27fb59df33e6b1719b/zstandard-0.23.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:b4567955a6bc1b20e9c31612e615af6b53733491aeaa19a6b3b37f3b65477094", size = 788713, upload-time = "2024-07-15T00:15:35.815Z" }, + { url = "https://files.pythonhosted.org/packages/5b/b3/1a028f6750fd9227ee0b937a278a434ab7f7fdc3066c3173f64366fe2466/zstandard-0.23.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1e172f57cd78c20f13a3415cc8dfe24bf388614324d25539146594c16d78fcc8", size = 633459, upload-time = "2024-07-15T00:15:37.995Z" }, + { url = "https://files.pythonhosted.org/packages/26/af/36d89aae0c1f95a0a98e50711bc5d92c144939efc1f81a2fcd3e78d7f4c1/zstandard-0.23.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b0e166f698c5a3e914947388c162be2583e0c638a4703fc6a543e23a88dea3c1", size = 4945707, upload-time = "2024-07-15T00:15:39.872Z" }, + { url = "https://files.pythonhosted.org/packages/cd/2e/2051f5c772f4dfc0aae3741d5fc72c3dcfe3aaeb461cc231668a4db1ce14/zstandard-0.23.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:12a289832e520c6bd4dcaad68e944b86da3bad0d339ef7989fb7e88f92e96072", size = 5306545, upload-time = "2024-07-15T00:15:41.75Z" }, + { url = "https://files.pythonhosted.org/packages/0a/9e/a11c97b087f89cab030fa71206963090d2fecd8eb83e67bb8f3ffb84c024/zstandard-0.23.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d50d31bfedd53a928fed6707b15a8dbeef011bb6366297cc435accc888b27c20", size = 5337533, upload-time = "2024-07-15T00:15:44.114Z" }, + { url = "https://files.pythonhosted.org/packages/fc/79/edeb217c57fe1bf16d890aa91a1c2c96b28c07b46afed54a5dcf310c3f6f/zstandard-0.23.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:72c68dda124a1a138340fb62fa21b9bf4848437d9ca60bd35db36f2d3345f373", size = 5436510, upload-time = "2024-07-15T00:15:46.509Z" }, + { url = "https://files.pythonhosted.org/packages/81/4f/c21383d97cb7a422ddf1ae824b53ce4b51063d0eeb2afa757eb40804a8ef/zstandard-0.23.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:53dd9d5e3d29f95acd5de6802e909ada8d8d8cfa37a3ac64836f3bc4bc5512db", size = 4859973, upload-time = "2024-07-15T00:15:49.939Z" }, + { url = "https://files.pythonhosted.org/packages/ab/15/08d22e87753304405ccac8be2493a495f529edd81d39a0870621462276ef/zstandard-0.23.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:6a41c120c3dbc0d81a8e8adc73312d668cd34acd7725f036992b1b72d22c1772", size = 4936968, upload-time = "2024-07-15T00:15:52.025Z" }, + { url = "https://files.pythonhosted.org/packages/eb/fa/f3670a597949fe7dcf38119a39f7da49a8a84a6f0b1a2e46b2f71a0ab83f/zstandard-0.23.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:40b33d93c6eddf02d2c19f5773196068d875c41ca25730e8288e9b672897c105", size = 5467179, upload-time = "2024-07-15T00:15:54.971Z" }, + { url = "https://files.pythonhosted.org/packages/4e/a9/dad2ab22020211e380adc477a1dbf9f109b1f8d94c614944843e20dc2a99/zstandard-0.23.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9206649ec587e6b02bd124fb7799b86cddec350f6f6c14bc82a2b70183e708ba", size = 4848577, upload-time = "2024-07-15T00:15:57.634Z" }, + { url = "https://files.pythonhosted.org/packages/08/03/dd28b4484b0770f1e23478413e01bee476ae8227bbc81561f9c329e12564/zstandard-0.23.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:76e79bc28a65f467e0409098fa2c4376931fd3207fbeb6b956c7c476d53746dd", size = 4693899, upload-time = "2024-07-15T00:16:00.811Z" }, + { url = "https://files.pythonhosted.org/packages/2b/64/3da7497eb635d025841e958bcd66a86117ae320c3b14b0ae86e9e8627518/zstandard-0.23.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:66b689c107857eceabf2cf3d3fc699c3c0fe8ccd18df2219d978c0283e4c508a", size = 5199964, upload-time = "2024-07-15T00:16:03.669Z" }, + { url = "https://files.pythonhosted.org/packages/43/a4/d82decbab158a0e8a6ebb7fc98bc4d903266bce85b6e9aaedea1d288338c/zstandard-0.23.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9c236e635582742fee16603042553d276cca506e824fa2e6489db04039521e90", size = 5655398, upload-time = "2024-07-15T00:16:06.694Z" }, + { url = "https://files.pythonhosted.org/packages/f2/61/ac78a1263bc83a5cf29e7458b77a568eda5a8f81980691bbc6eb6a0d45cc/zstandard-0.23.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a8fffdbd9d1408006baaf02f1068d7dd1f016c6bcb7538682622c556e7b68e35", size = 5191313, upload-time = "2024-07-15T00:16:09.758Z" }, + { url = "https://files.pythonhosted.org/packages/e7/54/967c478314e16af5baf849b6ee9d6ea724ae5b100eb506011f045d3d4e16/zstandard-0.23.0-cp312-cp312-win32.whl", hash = "sha256:dc1d33abb8a0d754ea4763bad944fd965d3d95b5baef6b121c0c9013eaf1907d", size = 430877, upload-time = "2024-07-15T00:16:11.758Z" }, + { url = "https://files.pythonhosted.org/packages/75/37/872d74bd7739639c4553bf94c84af7d54d8211b626b352bc57f0fd8d1e3f/zstandard-0.23.0-cp312-cp312-win_amd64.whl", hash = "sha256:64585e1dba664dc67c7cdabd56c1e5685233fbb1fc1966cfba2a340ec0dfff7b", size = 495595, upload-time = "2024-07-15T00:16:13.731Z" }, + { url = "https://files.pythonhosted.org/packages/80/f1/8386f3f7c10261fe85fbc2c012fdb3d4db793b921c9abcc995d8da1b7a80/zstandard-0.23.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:576856e8594e6649aee06ddbfc738fec6a834f7c85bf7cadd1c53d4a58186ef9", size = 788975, upload-time = "2024-07-15T00:16:16.005Z" }, + { url = "https://files.pythonhosted.org/packages/16/e8/cbf01077550b3e5dc86089035ff8f6fbbb312bc0983757c2d1117ebba242/zstandard-0.23.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:38302b78a850ff82656beaddeb0bb989a0322a8bbb1bf1ab10c17506681d772a", size = 633448, upload-time = "2024-07-15T00:16:17.897Z" }, + { url = "https://files.pythonhosted.org/packages/06/27/4a1b4c267c29a464a161aeb2589aff212b4db653a1d96bffe3598f3f0d22/zstandard-0.23.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d2240ddc86b74966c34554c49d00eaafa8200a18d3a5b6ffbf7da63b11d74ee2", size = 4945269, upload-time = "2024-07-15T00:16:20.136Z" }, + { url = "https://files.pythonhosted.org/packages/7c/64/d99261cc57afd9ae65b707e38045ed8269fbdae73544fd2e4a4d50d0ed83/zstandard-0.23.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2ef230a8fd217a2015bc91b74f6b3b7d6522ba48be29ad4ea0ca3a3775bf7dd5", size = 5306228, upload-time = "2024-07-15T00:16:23.398Z" }, + { url = "https://files.pythonhosted.org/packages/7a/cf/27b74c6f22541f0263016a0fd6369b1b7818941de639215c84e4e94b2a1c/zstandard-0.23.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:774d45b1fac1461f48698a9d4b5fa19a69d47ece02fa469825b442263f04021f", size = 5336891, upload-time = "2024-07-15T00:16:26.391Z" }, + { url = "https://files.pythonhosted.org/packages/fa/18/89ac62eac46b69948bf35fcd90d37103f38722968e2981f752d69081ec4d/zstandard-0.23.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6f77fa49079891a4aab203d0b1744acc85577ed16d767b52fc089d83faf8d8ed", size = 5436310, upload-time = "2024-07-15T00:16:29.018Z" }, + { url = "https://files.pythonhosted.org/packages/a8/a8/5ca5328ee568a873f5118d5b5f70d1f36c6387716efe2e369010289a5738/zstandard-0.23.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ac184f87ff521f4840e6ea0b10c0ec90c6b1dcd0bad2f1e4a9a1b4fa177982ea", size = 4859912, upload-time = "2024-07-15T00:16:31.871Z" }, + { url = "https://files.pythonhosted.org/packages/ea/ca/3781059c95fd0868658b1cf0440edd832b942f84ae60685d0cfdb808bca1/zstandard-0.23.0-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:c363b53e257246a954ebc7c488304b5592b9c53fbe74d03bc1c64dda153fb847", size = 4936946, upload-time = "2024-07-15T00:16:34.593Z" }, + { url = "https://files.pythonhosted.org/packages/ce/11/41a58986f809532742c2b832c53b74ba0e0a5dae7e8ab4642bf5876f35de/zstandard-0.23.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:e7792606d606c8df5277c32ccb58f29b9b8603bf83b48639b7aedf6df4fe8171", size = 5466994, upload-time = "2024-07-15T00:16:36.887Z" }, + { url = "https://files.pythonhosted.org/packages/83/e3/97d84fe95edd38d7053af05159465d298c8b20cebe9ccb3d26783faa9094/zstandard-0.23.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a0817825b900fcd43ac5d05b8b3079937073d2b1ff9cf89427590718b70dd840", size = 4848681, upload-time = "2024-07-15T00:16:39.709Z" }, + { url = "https://files.pythonhosted.org/packages/6e/99/cb1e63e931de15c88af26085e3f2d9af9ce53ccafac73b6e48418fd5a6e6/zstandard-0.23.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:9da6bc32faac9a293ddfdcb9108d4b20416219461e4ec64dfea8383cac186690", size = 4694239, upload-time = "2024-07-15T00:16:41.83Z" }, + { url = "https://files.pythonhosted.org/packages/ab/50/b1e703016eebbc6501fc92f34db7b1c68e54e567ef39e6e59cf5fb6f2ec0/zstandard-0.23.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:fd7699e8fd9969f455ef2926221e0233f81a2542921471382e77a9e2f2b57f4b", size = 5200149, upload-time = "2024-07-15T00:16:44.287Z" }, + { url = "https://files.pythonhosted.org/packages/aa/e0/932388630aaba70197c78bdb10cce2c91fae01a7e553b76ce85471aec690/zstandard-0.23.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:d477ed829077cd945b01fc3115edd132c47e6540ddcd96ca169facff28173057", size = 5655392, upload-time = "2024-07-15T00:16:46.423Z" }, + { url = "https://files.pythonhosted.org/packages/02/90/2633473864f67a15526324b007a9f96c96f56d5f32ef2a56cc12f9548723/zstandard-0.23.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:fa6ce8b52c5987b3e34d5674b0ab529a4602b632ebab0a93b07bfb4dfc8f8a33", size = 5191299, upload-time = "2024-07-15T00:16:49.053Z" }, + { url = "https://files.pythonhosted.org/packages/b0/4c/315ca5c32da7e2dc3455f3b2caee5c8c2246074a61aac6ec3378a97b7136/zstandard-0.23.0-cp313-cp313-win32.whl", hash = "sha256:a9b07268d0c3ca5c170a385a0ab9fb7fdd9f5fd866be004c4ea39e44edce47dd", size = 430862, upload-time = "2024-07-15T00:16:51.003Z" }, + { url = "https://files.pythonhosted.org/packages/a2/bf/c6aaba098e2d04781e8f4f7c0ba3c7aa73d00e4c436bcc0cf059a66691d1/zstandard-0.23.0-cp313-cp313-win_amd64.whl", hash = "sha256:f3513916e8c645d0610815c257cbfd3242adfd5c4cfa78be514e5a3ebb42a41b", size = 495578, upload-time = "2024-07-15T00:16:53.135Z" }, ] diff --git a/devel-common/pyproject.toml b/devel-common/pyproject.toml index 55b5ccca1176b..8b7b21c9e8270 100644 --- a/devel-common/pyproject.toml +++ b/devel-common/pyproject.toml @@ -35,7 +35,7 @@ dependencies = [ "kgb>=7.2.0", "requests_mock>=1.11.0", "rich>=13.6.0", - "ruff==0.13.2", + "ruff==0.13.3", "semver>=3.0.2", "time-machine>=2.15.0", "wheel>=0.42.0", diff --git a/scripts/ci/prek/check_imports_in_providers.py b/scripts/ci/prek/check_imports_in_providers.py index cd546b4607e43..08dbc92ed602a 100755 --- a/scripts/ci/prek/check_imports_in_providers.py +++ b/scripts/ci/prek/check_imports_in_providers.py @@ -19,7 +19,7 @@ # requires-python = ">=3.10" # dependencies = [ # "rich>=13.6.0", -# "ruff==0.13.2", +# "ruff==0.13.3", # ] # /// from __future__ import annotations diff --git a/scripts/ci/prek/ruff_format.py b/scripts/ci/prek/ruff_format.py index af8b2e5fa2228..1b38de4f3751b 100755 --- a/scripts/ci/prek/ruff_format.py +++ b/scripts/ci/prek/ruff_format.py @@ -18,7 +18,7 @@ # /// script # requires-python = ">=3.10" # dependencies = [ -# "ruff==0.13.2", +# "ruff==0.13.3", # ] # /// From 49ca3f27677342dc8d02e7bee1e1ab5dd0ed5e5b Mon Sep 17 00:00:00 2001 From: "Zhen-Lun (Kevin) Hong" Date: Fri, 3 Oct 2025 15:56:37 +0800 Subject: [PATCH 185/338] fix: allow mapped tasks to accept zero-length inputs on rerun (#56162) * fix: allow mapped tasks to accept zero-length inputs on rerun * fix: add test for rerun args of different length * chore: revise comments to align with the changes * chore: add comments before the task state check * fix: replace legacy query syntax --- airflow-core/src/airflow/models/dagrun.py | 10 ++-- airflow-core/tests/unit/models/test_dagrun.py | 55 ++++++++++++++++++- 2 files changed, 59 insertions(+), 6 deletions(-) diff --git a/airflow-core/src/airflow/models/dagrun.py b/airflow-core/src/airflow/models/dagrun.py index 56887614d4d1e..be7817cea1369 100644 --- a/airflow-core/src/airflow/models/dagrun.py +++ b/airflow-core/src/airflow/models/dagrun.py @@ -1517,10 +1517,6 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: return expanded_tis return () - def is_unmapped_task(ti: TI) -> bool: - # TODO: check why task is still MappedOperator even when not an unmapped task anymore - return isinstance(schedulable.task, MappedOperator) and schedulable.map_index == -1 - # Check dependencies. expansion_happened = False # Set of task ids for which was already done _revise_map_indexes_if_mapped @@ -1554,7 +1550,11 @@ def is_unmapped_task(ti: TI) -> bool: ) ) revised_map_index_task_ids.add(schedulable.task.task_id) - if not enable_lazy_task_expansion or not is_unmapped_task(schedulable): + + # _revise_map_indexes_if_mapped might mark the current task as REMOVED + # after calculating mapped task length, so we need to re-check + # the task state to ensure it's still schedulable + if schedulable.state in SCHEDULEABLE_STATES: ready_tis.append(schedulable) # Check if any ti changed state diff --git a/airflow-core/tests/unit/models/test_dagrun.py b/airflow-core/tests/unit/models/test_dagrun.py index 3813f89b3a37b..af80e153bf0e4 100644 --- a/airflow-core/tests/unit/models/test_dagrun.py +++ b/airflow-core/tests/unit/models/test_dagrun.py @@ -43,7 +43,7 @@ from airflow.providers.standard.operators.bash import BashOperator from airflow.providers.standard.operators.empty import EmptyOperator from airflow.providers.standard.operators.python import PythonOperator, ShortCircuitOperator -from airflow.sdk import DAG, BaseOperator, setup, task, task_group, teardown +from airflow.sdk import DAG, BaseOperator, get_current_context, setup, task, task_group, teardown from airflow.sdk.definitions.deadline import AsyncCallback, DeadlineAlert, DeadlineReference from airflow.serialization.serialized_objects import LazyDeserializedDAG, SerializedDAG from airflow.stats import Stats @@ -2253,6 +2253,59 @@ def tg(x, y): } +@pytest.mark.parametrize("rerun_length", [0, 1, 2, 3]) +def test_mapped_task_rerun_with_different_length_of_args(session, dag_maker, rerun_length): + @task + def generate_mapping_args(): + context = get_current_context() + if context["ti"].try_number == 0: + args = [i for i in range(2)] + else: + args = [i for i in range(rerun_length)] + return args + + @task + def mapped_print_value(arg): + return arg + + with dag_maker(session=session): + args = generate_mapping_args() + mapped_print_value.expand(arg=args) + + # First Run + dr = dag_maker.create_dagrun() + dag_maker.run_ti("generate_mapping_args", dr) + + decision = dr.task_instance_scheduling_decisions(session=session) + for ti in decision.schedulable_tis: + dag_maker.run_ti(ti.task_id, dr, map_index=ti.map_index) + + clear_task_instances(dr.get_task_instances(), session=session) + + # Second Run + ti = dr.get_task_instance(task_id="generate_mapping_args", session=session) + ti.try_number += 1 + session.merge(ti) + dag_maker.run_ti("generate_mapping_args", dr) + + # Check if the new mapped task instances are correctly scheduled + decision = dr.task_instance_scheduling_decisions(session=session) + assert len(decision.schedulable_tis) == rerun_length + assert all([ti.task_id == "mapped_print_value" for ti in decision.schedulable_tis]) + + # Check if mapped task rerun successfully + for ti in decision.schedulable_tis: + dag_maker.run_ti(ti.task_id, dr, map_index=ti.map_index) + query = select(TI).where( + TI.dag_id == dr.dag_id, + TI.run_id == dr.run_id, + TI.task_id == "mapped_print_value", + TI.state == TaskInstanceState.SUCCESS, + ) + success_tis = session.execute(query).all() + assert len(success_tis) == rerun_length + + def test_operator_mapped_task_group_receives_value(dag_maker, session): with dag_maker(session=session): From 6bace3d1cab3f175168f248fb94699c21be99561 Mon Sep 17 00:00:00 2001 From: Kirill Romanikhin <63454209+ido177@users.noreply.github.com> Date: Fri, 3 Oct 2025 10:58:25 +0300 Subject: [PATCH 186/338] Chart gitsync vars (#56245) * make the variable name in .Values.dags.gitSync.credentials secret dependent on the GitSync version * update gitsync tests --- chart/templates/_helpers.yaml | 22 +++++----- chart/values.yaml | 3 ++ .../airflow_aux/test_pod_template_file.py | 40 +++++++++++-------- .../other/test_git_sync_scheduler.py | 39 ++++++++++-------- 4 files changed, 62 insertions(+), 42 deletions(-) diff --git a/chart/templates/_helpers.yaml b/chart/templates/_helpers.yaml index b29a84cdcbe14..17f56e1343e5b 100644 --- a/chart/templates/_helpers.yaml +++ b/chart/templates/_helpers.yaml @@ -255,27 +255,31 @@ If release name contains chart name it will be used as a full name. - name: GITSYNC_SSH_KNOWN_HOSTS value: "false" {{- end }} - {{ else if .Values.dags.gitSync.credentialsSecret }} - - name: GIT_SYNC_USERNAME + {{- else if .Values.dags.gitSync.credentialsSecret }} + {{- $tag := trimPrefix "v" .Values.images.gitSync.tag }} + {{- if or (eq $tag "latest") (semverCompare ">=4.0.0" $tag) }} + - name: GITSYNC_USERNAME valueFrom: secretKeyRef: name: {{ .Values.dags.gitSync.credentialsSecret | quote }} - key: GIT_SYNC_USERNAME - - name: GITSYNC_USERNAME + key: GITSYNC_USERNAME + - name: GITSYNC_PASSWORD valueFrom: secretKeyRef: name: {{ .Values.dags.gitSync.credentialsSecret | quote }} - key: GITSYNC_USERNAME - - name: GIT_SYNC_PASSWORD + key: GITSYNC_PASSWORD + {{- else }} + - name: GIT_SYNC_USERNAME valueFrom: secretKeyRef: name: {{ .Values.dags.gitSync.credentialsSecret | quote }} - key: GIT_SYNC_PASSWORD - - name: GITSYNC_PASSWORD + key: GIT_SYNC_USERNAME + - name: GIT_SYNC_PASSWORD valueFrom: secretKeyRef: name: {{ .Values.dags.gitSync.credentialsSecret | quote }} - key: GITSYNC_PASSWORD + key: GIT_SYNC_PASSWORD + {{- end }} {{- end }} - name: GIT_SYNC_REV value: {{ .Values.dags.gitSync.rev | quote }} diff --git a/chart/values.yaml b/chart/values.yaml index 3a369bd954630..019b5ac4e55ed 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -124,6 +124,9 @@ images: gitSync: repository: registry.k8s.io/git-sync/git-sync tag: v4.4.2 + # NOTE: + # - If the tag is "v3.x.x" or any version < 4.0.0 - use GIT_SYNC_* env variables + # - If the tag is "v4.x.x" or "latest" - use GITSYNC_* env variables pullPolicy: IfNotPresent # Select certain nodes for airflow pods. diff --git a/helm-tests/tests/helm_tests/airflow_aux/test_pod_template_file.py b/helm-tests/tests/helm_tests/airflow_aux/test_pod_template_file.py index 6e2bdcc41c1db..540e7983a4f1e 100644 --- a/helm-tests/tests/helm_tests/airflow_aux/test_pod_template_file.py +++ b/helm-tests/tests/helm_tests/airflow_aux/test_pod_template_file.py @@ -254,7 +254,15 @@ def test_validate_if_ssh_known_hosts_are_added(self): "readOnly": True, } in jmespath.search("spec.initContainers[0].volumeMounts", docs[0]) - def test_should_set_username_and_pass_env_variables(self): + @pytest.mark.parametrize( + "tag,expected_prefix", + [ + ("v3.6.7", "GIT_SYNC_"), + ("v4.4.2", "GITSYNC_"), + ("latest", "GITSYNC_"), + ], + ) + def test_should_set_username_and_pass_env_variables(self, tag, expected_prefix): docs = render_chart( values={ "dags": { @@ -263,30 +271,28 @@ def test_should_set_username_and_pass_env_variables(self): "credentialsSecret": "user-pass-secret", "sshKeySecret": None, } - } + }, + "images": { + "gitSync": { + "tag": tag, + } + }, }, show_only=["templates/pod-template-file.yaml"], chart_dir=self.temp_chart_dir, ) - assert { - "name": "GIT_SYNC_USERNAME", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GIT_SYNC_USERNAME"}}, - } in jmespath.search("spec.initContainers[0].env", docs[0]) - assert { - "name": "GIT_SYNC_PASSWORD", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GIT_SYNC_PASSWORD"}}, - } in jmespath.search("spec.initContainers[0].env", docs[0]) + envs = jmespath.search("spec.initContainers[0].env", docs[0]) - # Testing git-sync v4 assert { - "name": "GITSYNC_USERNAME", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GITSYNC_USERNAME"}}, - } in jmespath.search("spec.initContainers[0].env", docs[0]) + "name": f"{expected_prefix}USERNAME", + "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": f"{expected_prefix}USERNAME"}}, + } in envs + assert { - "name": "GITSYNC_PASSWORD", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GITSYNC_PASSWORD"}}, - } in jmespath.search("spec.initContainers[0].env", docs[0]) + "name": f"{expected_prefix}PASSWORD", + "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": f"{expected_prefix}PASSWORD"}}, + } in envs def test_should_set_the_dags_volume_claim_correctly_when_using_an_existing_claim(self): docs = render_chart( diff --git a/helm-tests/tests/helm_tests/other/test_git_sync_scheduler.py b/helm-tests/tests/helm_tests/other/test_git_sync_scheduler.py index 82c9577f48505..83c214c99e477 100644 --- a/helm-tests/tests/helm_tests/other/test_git_sync_scheduler.py +++ b/helm-tests/tests/helm_tests/other/test_git_sync_scheduler.py @@ -17,6 +17,7 @@ from __future__ import annotations import jmespath +import pytest from chart_utils.helm_template_generator import render_chart @@ -266,7 +267,15 @@ def test_validate_sshkeysecret_not_added_when_persistence_is_enabled(self): ) assert "git-sync-ssh-key" not in jmespath.search("spec.template.spec.volumes[].name", docs[0]) - def test_should_set_username_and_pass_env_variables(self): + @pytest.mark.parametrize( + "tag,expected_prefix", + [ + ("v3.6.7", "GIT_SYNC_"), + ("v4.4.2", "GITSYNC_"), + ("latest", "GITSYNC_"), + ], + ) + def test_should_set_username_and_pass_env_variables_in_scheduler(self, tag, expected_prefix): docs = render_chart( values={ "airflowVersion": "2.10.5", @@ -277,28 +286,26 @@ def test_should_set_username_and_pass_env_variables(self): "sshKeySecret": None, } }, + "images": { + "gitSync": { + "tag": tag, + } + }, }, show_only=["templates/scheduler/scheduler-deployment.yaml"], ) - assert { - "name": "GIT_SYNC_USERNAME", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GIT_SYNC_USERNAME"}}, - } in jmespath.search("spec.template.spec.containers[1].env", docs[0]) - assert { - "name": "GIT_SYNC_PASSWORD", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GIT_SYNC_PASSWORD"}}, - } in jmespath.search("spec.template.spec.containers[1].env", docs[0]) + envs = jmespath.search("spec.template.spec.containers[1].env", docs[0]) - # Testing git-sync v4 assert { - "name": "GITSYNC_USERNAME", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GITSYNC_USERNAME"}}, - } in jmespath.search("spec.template.spec.containers[1].env", docs[0]) + "name": f"{expected_prefix}USERNAME", + "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": f"{expected_prefix}USERNAME"}}, + } in envs + assert { - "name": "GITSYNC_PASSWORD", - "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": "GITSYNC_PASSWORD"}}, - } in jmespath.search("spec.template.spec.containers[1].env", docs[0]) + "name": f"{expected_prefix}PASSWORD", + "valueFrom": {"secretKeyRef": {"name": "user-pass-secret", "key": f"{expected_prefix}PASSWORD"}}, + } in envs def test_should_set_the_volume_claim_correctly_when_using_an_existing_claim(self): docs = render_chart( From 9bbb34fd53db6e295248646d59b069a98053ded6 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 3 Oct 2025 01:45:08 -0700 Subject: [PATCH 187/338] Update provider's metadata (#56357) --- generated/provider_metadata.json | 252 ++++++++++++++++++++++++++++++- 1 file changed, 250 insertions(+), 2 deletions(-) diff --git a/generated/provider_metadata.json b/generated/provider_metadata.json index fe915f9c5600f..db42a9575845d 100644 --- a/generated/provider_metadata.json +++ b/generated/provider_metadata.json @@ -253,6 +253,10 @@ "3.2.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "3.2.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "amazon": { @@ -563,6 +567,14 @@ "9.12.0": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "9.13.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "9.14.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "apache.beam": { @@ -741,6 +753,14 @@ "6.1.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "6.1.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "6.1.5": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "apache.cassandra": { @@ -1355,6 +1375,10 @@ "4.10.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.10.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "apache.hive": { @@ -1763,6 +1787,14 @@ "1.10.2": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "1.10.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "1.10.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "apache.kylin": { @@ -2019,6 +2051,10 @@ "4.4.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "4.4.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "apache.pig": { @@ -2425,7 +2461,7 @@ }, "apache.tinkerpop": { "1.0.0": { - "associated_airflow_version": "2.11.0", + "associated_airflow_version": "3.1.0", "date_released": "2025-05-08T08:02:07Z" }, "1.0.1": { @@ -2437,7 +2473,7 @@ "date_released": "2025-08-02T06:58:37Z" }, "1.0.3": { - "associated_airflow_version": "3.0.5", + "associated_airflow_version": "3.1.0", "date_released": "2025-08-11T05:36:14Z" } }, @@ -2961,6 +2997,10 @@ "3.12.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "3.12.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "cloudant": { @@ -3393,6 +3433,14 @@ "10.7.0": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "10.8.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "10.8.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "cohere": { @@ -3513,6 +3561,10 @@ "1.7.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "1.7.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "common.io": { @@ -3587,6 +3639,10 @@ "1.6.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "1.6.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "common.messaging": { @@ -3613,6 +3669,10 @@ "1.0.5": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "2.0.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "common.sql": { @@ -3811,6 +3871,14 @@ "1.27.5": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "1.28.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "1.28.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "databricks": { @@ -4041,6 +4109,14 @@ "7.7.1": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "7.7.2": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "7.7.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "datadog": { @@ -4313,6 +4389,10 @@ "4.4.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.4.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "dingding": { @@ -4739,6 +4819,10 @@ "4.4.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.4.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "edge3": { @@ -4761,6 +4845,14 @@ "1.1.3": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "1.2.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "1.3.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "elasticsearch": { @@ -4963,6 +5055,10 @@ "6.3.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "6.3.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "exasol": { @@ -5133,6 +5229,10 @@ "4.8.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "4.8.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "fab": { @@ -5239,6 +5339,14 @@ "2.4.1": { "associated_airflow_version": "3.0.6", "date_released": "2025-08-17T18:26:38Z" + }, + "2.4.2": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "2.4.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "facebook": { @@ -5521,6 +5629,14 @@ "0.0.6": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "0.0.7": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "0.0.8": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "github": { @@ -5921,6 +6037,14 @@ "17.1.0": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "17.2.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "18.0.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "grpc": { @@ -6351,6 +6475,10 @@ "5.3.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "5.3.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "imap": { @@ -6713,6 +6841,10 @@ "5.2.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "5.2.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "jenkins": { @@ -6851,12 +6983,20 @@ "4.1.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.1.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "keycloak": { "0.0.1": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-17T18:26:38Z" + }, + "0.1.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "microsoft.azure": { @@ -7147,6 +7287,14 @@ "12.6.1": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "12.7.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "12.7.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "microsoft.mssql": { @@ -7395,6 +7543,10 @@ "3.1.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "3.1.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "microsoft.winrm": { @@ -7861,6 +8013,10 @@ "6.3.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "6.3.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "neo4j": { @@ -7979,6 +8135,10 @@ "3.10.0": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "3.10.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "odbc": { @@ -8419,6 +8579,14 @@ "2.6.1": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "2.7.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "2.7.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "opensearch": { @@ -8485,6 +8653,10 @@ "1.7.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "1.7.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "opsgenie": { @@ -8769,6 +8941,10 @@ "4.1.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.2.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "pagerduty": { @@ -9033,6 +9209,10 @@ "3.11.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "3.11.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "pgvector": { @@ -9141,6 +9321,10 @@ "2.3.3": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "2.3.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "postgres": { @@ -9339,6 +9523,10 @@ "6.2.3": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "6.3.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "presto": { @@ -9677,6 +9865,14 @@ "4.2.0": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "4.3.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "4.3.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "salesforce": { @@ -9827,6 +10023,10 @@ "5.11.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:38Z" + }, + "5.11.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "samba": { @@ -9937,6 +10137,10 @@ "4.10.3": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "4.11.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "segment": { @@ -10331,6 +10535,10 @@ "5.3.4": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "5.4.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "singularity": { @@ -10615,6 +10823,14 @@ "9.1.4": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "9.2.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "9.3.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "smtp": { @@ -10717,6 +10933,14 @@ "2.2.0": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "2.2.1": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "2.3.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "snowflake": { @@ -10979,6 +11203,14 @@ "6.5.2": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "6.5.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "6.5.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "sqlite": { @@ -11295,6 +11527,10 @@ "4.1.3": { "associated_airflow_version": "3.0.5", "date_released": "2025-08-11T05:36:14Z" + }, + "4.1.4": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "standard": { @@ -11361,6 +11597,14 @@ "1.6.0": { "associated_airflow_version": "3.0.6", "date_released": "2025-08-17T18:26:38Z" + }, + "1.7.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" + }, + "1.8.0": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-22T09:24:27Z" } }, "tableau": { @@ -12081,6 +12325,10 @@ "3.2.2": { "associated_airflow_version": "3.0.4", "date_released": "2025-08-02T06:58:37Z" + }, + "3.2.3": { + "associated_airflow_version": "3.1.0", + "date_released": "2025-09-09T03:51:12Z" } }, "yandex": { From b99e0ac9d100afeaf577436c7f2bba6d75e170a4 Mon Sep 17 00:00:00 2001 From: VladaZakharova Date: Fri, 3 Oct 2025 09:20:54 +0000 Subject: [PATCH 188/338] Fix install_java.sh (#56211) Co-authored-by: Ulada Zakharava --- scripts/in_container/bin/install_java.sh | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/scripts/in_container/bin/install_java.sh b/scripts/in_container/bin/install_java.sh index 8089e4338d97b..0aa15f8209518 100755 --- a/scripts/in_container/bin/install_java.sh +++ b/scripts/in_container/bin/install_java.sh @@ -25,6 +25,15 @@ BIN_PATH="/files/bin/java" if [[ $# != "0" && ${1} == "--reinstall" ]]; then rm -rf "${INSTALL_DIR}" rm -f "${BIN_PATH}" + mapfile -t files_to_delete < <(find /files/bin -type l -exec bash -c ' + for link; do + target=$(readlink "$link") + if [[ "$target" == /files/opt/java/* ]]; then + echo "$link" + fi + done + ' _ {} +) + rm -rf "${files_to_delete[@]}" fi hash -r @@ -34,7 +43,7 @@ if command -v java; then exit 1 fi -DOWNLOAD_URL='https://download.java.net/openjdk/jdk8u41/ri/openjdk-8u41-b04-linux-x64-14_jan_2020.tar.gz' +DOWNLOAD_URL="https://download.java.net/openjdk/jdk25/ri/openjdk-25+36_linux-x64_bin.tar.gz" if [[ -e ${INSTALL_DIR} ]]; then echo "The install directory (${INSTALL_DIR}) already exists. This may mean java is already installed." From d27513c9fb23cf5573b1da9d4d88bf21af257653 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 3 Oct 2025 03:23:10 -0700 Subject: [PATCH 189/338] Add new PL translations. (#56359) --- airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json | 2 ++ airflow-core/src/airflow/ui/public/i18n/locales/pl/dag.json | 2 ++ 2 files changed, 4 insertions(+) diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json index ccc627245dcf9..629aa2dd3ef98 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json @@ -113,6 +113,8 @@ }, "filter": "Filtr", "filters": { + "durationFrom": "Czas trwania od", + "durationTo": "Czas trwania do", "logicalDateFrom": "Data logiczna od", "logicalDateTo": "Data logiczna do", "runAfterFrom": "Uruchom po (od)", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/pl/dag.json b/airflow-core/src/airflow/ui/public/i18n/locales/pl/dag.json index 27276f595f68d..3a55fced65990 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/pl/dag.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/pl/dag.json @@ -10,6 +10,7 @@ "hourly": "Godzinowo", "legend": { "less": "Mniej", + "mixed": "Mieszane", "more": "Więcej" }, "navigation": { @@ -19,6 +20,7 @@ "previousYear": "Poprzedni rok" }, "noData": "Brak danych", + "noFailedRuns": "Brak nieudanych wykonań", "noRuns": "Brak wykonań", "totalRuns": "Łączna liczba wykonań", "week": "Tydzień {{weekNumber}}", From d8c152529fa9c8e098edd7574801036a3a3d8351 Mon Sep 17 00:00:00 2001 From: pawelgrochowicz Date: Fri, 3 Oct 2025 13:33:08 +0200 Subject: [PATCH 190/338] feature: Add OpenLineage support for CloudDataTransferServiceS3ToGCSOperator (#55946) --- .../CloudStorageTransferJobFacet.json | 68 ++++++++ .../CloudStorageTransferRunFacet.json | 60 +++++++ .../google/cloud/openlineage/facets.py | 82 ++++++++- .../cloud_storage_transfer_service.py | 64 ++++++- .../google/cloud/openlineage/test_facets.py | 58 ++++++- .../test_cloud_storage_transfer_service.py | 159 ++++++++++++++++++ 6 files changed, 483 insertions(+), 8 deletions(-) create mode 100644 providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferJobFacet.json create mode 100644 providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferRunFacet.json diff --git a/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferJobFacet.json b/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferJobFacet.json new file mode 100644 index 0000000000000..2f0f763af2f5c --- /dev/null +++ b/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferJobFacet.json @@ -0,0 +1,68 @@ +{ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "$defs": { + "CloudStorageTransferJobFacet": { + "allOf": [ + { + "$ref": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/JobFacet" + }, + { + "type": "object", + "properties": { + "jobName": { + "type": "string", + "description": "Transfer job name assigned by GCP Storage Transfer Service." + }, + "projectId": { + "type": "string", + "description": "GCP project ID." + }, + "description": { + "type": "string", + "description": "Optional description of the transfer job." + }, + "status": { + "type": "string", + "description": "Status of the transfer job (ENABLED, DISABLED)." + }, + "sourceBucket": { + "type": "string", + "description": "Source AWS S3 bucket." + }, + "sourcePath": { + "type": "string", + "description": "Prefix path inside the source bucket." + }, + "targetBucket": { + "type": "string", + "description": "Target GCS bucket." + }, + "targetPath": { + "type": "string", + "description": "Prefix path inside the target bucket." + }, + "objectConditions": { + "type": "object", + "description": "Filtering conditions for objects transferred." + }, + "transferOptions": { + "type": "object", + "description": "Transfer options such as overwrite or delete." + }, + "schedule": { + "type": "object", + "description": "Transfer schedule details." + } + } + } + ], + "type": "object" + } + }, + "type": "object", + "properties": { + "cloudStorageTransferJob": { + "$ref": "#/$defs/CloudStorageTransferJobFacet" + } + } +} diff --git a/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferRunFacet.json b/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferRunFacet.json new file mode 100644 index 0000000000000..8eea204b53aaf --- /dev/null +++ b/providers/google/src/airflow/providers/google/cloud/openlineage/CloudStorageTransferRunFacet.json @@ -0,0 +1,60 @@ +{ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "$defs": { + "CloudStorageTransferRunFacet": { + "allOf": [ + { + "$ref": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet" + }, + { + "type": "object", + "properties": { + "jobName": { + "type": "string", + "description": "Transfer job name associated with this run." + }, + "operationName": { + "type": "string", + "description": "Transfer operation name if available." + }, + "status": { + "type": "string", + "description": "Run status if available." + }, + "startTime": { + "type": "string", + "description": "Start time of the transfer operation." + }, + "endTime": { + "type": "string", + "description": "End time of the transfer operation." + }, + "wait": { + "type": "boolean", + "description": "Whether the operator waited for completion." + }, + "timeout": { + "type": ["number", "null"], + "description": "Timeout in seconds." + }, + "deferrable": { + "type": "boolean", + "description": "Whether the operator used deferrable mode." + }, + "deleteJobAfterCompletion": { + "type": "boolean", + "description": "Whether the transfer job was deleted after completion." + } + } + } + ], + "type": "object" + } + }, + "type": "object", + "properties": { + "cloudStorageTransferRun": { + "$ref": "#/$defs/CloudStorageTransferRunFacet" + } + } +} diff --git a/providers/google/src/airflow/providers/google/cloud/openlineage/facets.py b/providers/google/src/airflow/providers/google/cloud/openlineage/facets.py index 46177bb4de98b..645e8c51b07bc 100644 --- a/providers/google/src/airflow/providers/google/cloud/openlineage/facets.py +++ b/providers/google/src/airflow/providers/google/cloud/openlineage/facets.py @@ -24,13 +24,17 @@ from airflow.providers.google import __version__ as provider_version if TYPE_CHECKING: - from openlineage.client.generated.base import RunFacet + from openlineage.client.generated.base import JobFacet, RunFacet try: try: from openlineage.client.generated.base import RunFacet except ImportError: # Old OpenLineage client is used from openlineage.client.facet import BaseFacet as RunFacet # type: ignore[assignment] + try: + from openlineage.client.generated.base import JobFacet + except ImportError: # Old OpenLineage client is used + from openlineage.client.facet import BaseFacet as JobFacet # type: ignore[assignment] @define class BigQueryJobRunFacet(RunFacet): @@ -53,6 +57,80 @@ def _get_schema() -> str: f"providers-google/{provider_version}/airflow/providers/google/" "openlineage/BigQueryJobRunFacet.json" ) + + @define + class CloudStorageTransferJobFacet(JobFacet): + """ + Facet representing a Cloud Storage Transfer Service job configuration. + + :param jobName: Unique name of the transfer job. + :param projectId: GCP project where the transfer job is defined. + :param description: User-provided description of the transfer job. + :param status: Current status of the transfer job (e.g. "ENABLED", "DISABLED"). + :param sourceBucket: Name of the source bucket (e.g. AWS S3). + :param sourcePath: Prefix/path inside the source bucket. + :param targetBucket: Name of the destination bucket (e.g. GCS). + :param targetPath: Prefix/path inside the destination bucket. + :param objectConditions: Object selection rules (e.g. include/exclude prefixes). + :param transferOptions: Transfer options, such as overwrite behavior or whether to delete objects + from the source after transfer. + :param schedule: Schedule for the transfer job (if recurring). + """ + + jobName: str | None = field(default=None) + projectId: str | None = field(default=None) + description: str | None = field(default=None) + status: str | None = field(default=None) + sourceBucket: str | None = field(default=None) + sourcePath: str | None = field(default=None) + targetBucket: str | None = field(default=None) + targetPath: str | None = field(default=None) + objectConditions: dict | None = field(default=None) + transferOptions: dict | None = field(default=None) + schedule: dict | None = field(default=None) + + @staticmethod + def _get_schema() -> str: + return ( + "https://raw.githubusercontent.com/apache/airflow/" + f"providers-google/{provider_version}/airflow/providers/google/" + "openlineage/CloudStorageTransferJobFacet.json" + ) + + @define + class CloudStorageTransferRunFacet(RunFacet): + """ + Facet representing a Cloud Storage Transfer Service job execution run. + + :param jobName: Name of the transfer job being executed. + :param operationName: Name of the specific transfer operation instance. + :param status: Current status of the operation (e.g. "IN_PROGRESS", "SUCCESS", "FAILED"). + :param startTime: Time when the transfer job execution started (ISO 8601 format). + :param endTime: Time when the transfer job execution finished (ISO 8601 format). + :param wait: Whether the operator waits for the job to complete before finishing. + :param timeout: Timeout (in seconds) for the transfer run to complete. + :param deferrable: Whether the operator defers execution until job completion. + :param deleteJobAfterCompletion: Whether the operator deletes the transfer job after the run completes. + """ + + jobName: str | None = field(default=None) + operationName: str | None = field(default=None) + status: str | None = field(default=None) + startTime: str | None = field(default=None) + endTime: str | None = field(default=None) + wait: bool = field(default=True) + timeout: float | None = field(default=None) + deferrable: bool = field(default=False) + deleteJobAfterCompletion: bool = field(default=False) + + @staticmethod + def _get_schema() -> str: + return ( + "https://raw.githubusercontent.com/apache/airflow/" + f"providers-google/{provider_version}/airflow/providers/google/" + "openlineage/CloudStorageTransferRunFacet.json" + ) + except ImportError: # OpenLineage is not available def create_no_op(*_, **__) -> None: @@ -65,3 +143,5 @@ def create_no_op(*_, **__) -> None: return None BigQueryJobRunFacet = create_no_op # type: ignore[misc, assignment] + CloudStorageTransferJobFacet = create_no_op # type: ignore[misc, assignment] + CloudStorageTransferRunFacet = create_no_op # type: ignore[misc, assignment] diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py index 788a5822a71be..88307119aba9b 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py @@ -71,6 +71,7 @@ from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID if TYPE_CHECKING: + from airflow.providers.openlineage.extractors import OperatorLineage from airflow.utils.context import Context @@ -964,6 +965,7 @@ def __init__( self.aws_role_arn = aws_role_arn self.deferrable = deferrable self._validate_inputs() + self._transfer_job: dict[str, Any] | None = None def _validate_inputs(self) -> None: if self.delete_job_after_completion and not self.wait: @@ -978,19 +980,18 @@ def execute(self, context: Context) -> None: TransferJobPreprocessor(body=body, aws_conn_id=self.aws_conn_id, default_schedule=True).process_body() - job = hook.create_transfer_job(body=body) - + self._transfer_job = hook.create_transfer_job(body=body) if self.wait: if not self.deferrable: - hook.wait_for_transfer_job(job, timeout=self.timeout) + hook.wait_for_transfer_job(self._transfer_job, timeout=self.timeout) if self.delete_job_after_completion: - hook.delete_transfer_job(job_name=job[NAME], project_id=self.project_id) + hook.delete_transfer_job(job_name=self._transfer_job[NAME], project_id=self.project_id) else: self.defer( timeout=timedelta(seconds=self.timeout or 60), trigger=CloudStorageTransferServiceCheckJobStatusTrigger( - job_name=job[NAME], - project_id=job[PROJECT_ID], + job_name=self._transfer_job[NAME], + project_id=self._transfer_job[PROJECT_ID], gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.google_impersonation_chain, ), @@ -1040,6 +1041,57 @@ def _create_body(self) -> dict: return body + def get_openlineage_facets_on_complete(self, task_instance) -> OperatorLineage | None: + """Provide OpenLineage OperatorLineage for the S3->GCS transfer.""" + from airflow.providers.common.compat.openlineage.facet import Dataset + from airflow.providers.google.cloud.openlineage.facets import ( + CloudStorageTransferJobFacet, + CloudStorageTransferRunFacet, + ) + from airflow.providers.openlineage.extractors import OperatorLineage + + input_ds = Dataset( + namespace=f"s3://{self.s3_bucket}", + name=normalize_directory_path(self.s3_path) or "", + ) + + output_ds = Dataset( + namespace=f"gs://{self.gcs_bucket}", + name=normalize_directory_path(self.gcs_path) or "", + ) + + job = self._transfer_job or {} + job_facet = CloudStorageTransferJobFacet( + jobName=job.get(NAME), + projectId=job.get(PROJECT_ID, self.project_id), + description=job.get(DESCRIPTION, self.description), + status=job.get(STATUS), + sourceBucket=job.get(TRANSFER_SPEC, {}) + .get(AWS_S3_DATA_SOURCE, {}) + .get(BUCKET_NAME, self.s3_bucket), + sourcePath=job.get(TRANSFER_SPEC, {}).get(AWS_S3_DATA_SOURCE, {}).get(PATH, self.s3_path), + targetBucket=job.get(TRANSFER_SPEC, {}).get(GCS_DATA_SINK, {}).get(BUCKET_NAME, self.gcs_bucket), + targetPath=job.get(TRANSFER_SPEC, {}).get(GCS_DATA_SINK, {}).get(PATH, self.gcs_path), + objectConditions=job.get(TRANSFER_SPEC, {}).get("objectConditions", self.object_conditions), + transferOptions=job.get(TRANSFER_SPEC, {}).get("transferOptions", self.transfer_options), + schedule=job.get(SCHEDULE, self.schedule), + ) + + run_facet = CloudStorageTransferRunFacet( + jobName=job.get(NAME), + wait=self.wait, + timeout=self.timeout, + deferrable=self.deferrable, + deleteJobAfterCompletion=self.delete_job_after_completion, + ) + + return OperatorLineage( + inputs=[input_ds], + outputs=[output_ds], + job_facets={"cloudStorageTransferJob": job_facet}, + run_facets={"cloudStorageTransferRun": run_facet}, + ) + class CloudDataTransferServiceGCSToGCSOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/tests/unit/google/cloud/openlineage/test_facets.py b/providers/google/tests/unit/google/cloud/openlineage/test_facets.py index 4b45a9ee4b739..0aa1e4feebcb1 100644 --- a/providers/google/tests/unit/google/cloud/openlineage/test_facets.py +++ b/providers/google/tests/unit/google/cloud/openlineage/test_facets.py @@ -16,7 +16,11 @@ # under the License. from __future__ import annotations -from airflow.providers.google.cloud.openlineage.facets import BigQueryJobRunFacet +from airflow.providers.google.cloud.openlineage.facets import ( + BigQueryJobRunFacet, + CloudStorageTransferJobFacet, + CloudStorageTransferRunFacet, +) def test_bigquery_job_run_facet(): @@ -24,3 +28,55 @@ def test_bigquery_job_run_facet(): assert facet.cached is True assert facet.billedBytes == 123 assert facet.properties == "some_properties" + + +def test_cloud_storage_transfer_job_facet(): + facet = CloudStorageTransferJobFacet( + jobName="transferJobs/123", + projectId="test-project", + description="S3 to GCS transfer", + status="ENABLED", + sourceBucket="my-s3-bucket", + sourcePath="data/", + targetBucket="my-gcs-bucket", + targetPath="backup/", + objectConditions={"maxTimeElapsedSinceLastModification": "86400s"}, + transferOptions={"overwriteObjectsAlreadyExistingInSink": True}, + schedule={"scheduleStartDate": {"year": 2025, "month": 9, "day": 17}}, + ) + + assert facet.jobName == "transferJobs/123" + assert facet.projectId == "test-project" + assert facet.description == "S3 to GCS transfer" + assert facet.status == "ENABLED" + assert facet.sourceBucket == "my-s3-bucket" + assert facet.sourcePath == "data/" + assert facet.targetBucket == "my-gcs-bucket" + assert facet.targetPath == "backup/" + assert facet.objectConditions == {"maxTimeElapsedSinceLastModification": "86400s"} + assert facet.transferOptions == {"overwriteObjectsAlreadyExistingInSink": True} + assert facet.schedule == {"scheduleStartDate": {"year": 2025, "month": 9, "day": 17}} + + +def test_cloud_storage_transfer_run_facet(): + facet = CloudStorageTransferRunFacet( + jobName="transferJobs/123", + operationName="transferOperations/abc", + status="SUCCESS", + startTime="2025-09-17T10:00:00Z", + endTime="2025-09-17T10:05:00Z", + wait=True, + timeout=3600, + deferrable=False, + deleteJobAfterCompletion=True, + ) + + assert facet.jobName == "transferJobs/123" + assert facet.operationName == "transferOperations/abc" + assert facet.status == "SUCCESS" + assert facet.startTime == "2025-09-17T10:00:00Z" + assert facet.endTime == "2025-09-17T10:05:00Z" + assert facet.wait is True + assert facet.timeout == 3600 + assert facet.deferrable is False + assert facet.deleteJobAfterCompletion is True diff --git a/providers/google/tests/unit/google/cloud/operators/test_cloud_storage_transfer_service.py b/providers/google/tests/unit/google/cloud/operators/test_cloud_storage_transfer_service.py index e3f2201c8787a..60257593333ed 100644 --- a/providers/google/tests/unit/google/cloud/operators/test_cloud_storage_transfer_service.py +++ b/providers/google/tests/unit/google/cloud/operators/test_cloud_storage_transfer_service.py @@ -39,6 +39,7 @@ LIST_URL, NAME, PATH, + PROJECT_ID, SCHEDULE, SCHEDULE_END_DATE, SCHEDULE_START_DATE, @@ -47,6 +48,10 @@ STATUS, TRANSFER_SPEC, ) +from airflow.providers.google.cloud.openlineage.facets import ( + CloudStorageTransferJobFacet, + CloudStorageTransferRunFacet, +) from airflow.providers.google.cloud.operators.cloud_storage_transfer_service import ( CloudDataTransferServiceCancelOperationOperator, CloudDataTransferServiceCreateJobOperator, @@ -1018,6 +1023,160 @@ def test_async_execute_error(self, mock_aws_hook): context={}, event={"status": "error", "message": "test failure message"} ) + @pytest.mark.parametrize( + "wait, job_name", + [ + (True, "transferJobs/123"), + (False, "transferJobs/456"), + ], + ) + def test_get_openlineage_facets_on_complete_facets_run_and_job(self, wait, job_name): + op = CloudDataTransferServiceS3ToGCSOperator( + task_id=TASK_ID, + s3_bucket=AWS_BUCKET_NAME, + s3_path="raw/", + gcs_bucket=GCS_BUCKET_NAME, + gcs_path="processed/", + project_id=GCP_PROJECT_ID, + wait=wait, + description=DESCRIPTION, + ) + op._transfer_job = { + NAME: job_name, + PROJECT_ID: GCP_PROJECT_ID, + DESCRIPTION: DESCRIPTION, + STATUS: "ENABLED", + TRANSFER_SPEC: { + AWS_S3_DATA_SOURCE: {BUCKET_NAME: AWS_BUCKET_NAME, PATH: "raw/"}, + GCS_DATA_SINK: {BUCKET_NAME: GCS_BUCKET_NAME, PATH: "processed/"}, + }, + } + + result = op.get_openlineage_facets_on_complete(task_instance=mock.Mock()) + + assert result.inputs[0].namespace == f"s3://{AWS_BUCKET_NAME}" + assert result.inputs[0].name == "raw/" + assert result.outputs[0].namespace == f"gs://{GCS_BUCKET_NAME}" + assert result.outputs[0].name == "processed/" + + job_facet = result.job_facets["cloudStorageTransferJob"] + assert isinstance(job_facet, CloudStorageTransferJobFacet) + assert job_facet.jobName == job_name + assert job_facet.projectId == GCP_PROJECT_ID + assert job_facet.description == DESCRIPTION + assert job_facet.status == "ENABLED" + assert job_facet.sourceBucket == AWS_BUCKET_NAME + assert job_facet.sourcePath == "raw/" + assert job_facet.targetBucket == GCS_BUCKET_NAME + assert job_facet.targetPath == "processed/" + assert job_facet.objectConditions is None + assert job_facet.transferOptions is None + assert job_facet.schedule is None + + run_facet = result.run_facets["cloudStorageTransferRun"] + assert isinstance(run_facet, CloudStorageTransferRunFacet) + assert run_facet.jobName == job_name + assert run_facet.wait == wait + + @pytest.mark.parametrize( + "object_conditions, delete_source", + [ + ({"includePrefixes": ["2025/"]}, True), + (None, False), + ], + ) + def test_get_openlineage_facets_on_complete_job_facet_includes_object_conditions_and_options( + self, object_conditions, delete_source + ): + op = CloudDataTransferServiceS3ToGCSOperator( + task_id=TASK_ID, + s3_bucket=AWS_BUCKET_NAME, + gcs_bucket=GCS_BUCKET_NAME, + project_id=GCP_PROJECT_ID, + object_conditions=object_conditions, + transfer_options={"deleteObjectsFromSourceAfterTransfer": delete_source}, + wait=True, + ) + op._transfer_job = { + NAME: "transferJobs/789", + PROJECT_ID: GCP_PROJECT_ID, + TRANSFER_SPEC: { + AWS_S3_DATA_SOURCE: {BUCKET_NAME: AWS_BUCKET_NAME}, + GCS_DATA_SINK: {BUCKET_NAME: GCS_BUCKET_NAME}, + "objectConditions": object_conditions, + "transferOptions": {"deleteObjectsFromSourceAfterTransfer": delete_source}, + }, + } + + result = op.get_openlineage_facets_on_complete(task_instance=mock.Mock()) + job_facet = result.job_facets["cloudStorageTransferJob"] + assert job_facet.projectId == GCP_PROJECT_ID + assert job_facet.objectConditions == object_conditions + assert job_facet.transferOptions == {"deleteObjectsFromSourceAfterTransfer": delete_source} + assert isinstance(job_facet.objectConditions, (dict, type(None))) + assert isinstance(job_facet.transferOptions, (dict, type(None))) + + def test_get_openlineage_facets_on_complete_job_facet_without_object_conditions_or_transfer_options(self): + op = CloudDataTransferServiceS3ToGCSOperator( + task_id=TASK_ID, + s3_bucket=AWS_BUCKET_NAME, + gcs_bucket=GCS_BUCKET_NAME, + wait=True, + ) + op._transfer_job = { + NAME: "transferJobs/222", + PROJECT_ID: GCP_PROJECT_ID, + "transferSpec": { + AWS_S3_DATA_SOURCE: {BUCKET_NAME: AWS_BUCKET_NAME}, + GCS_DATA_SINK: {BUCKET_NAME: GCS_BUCKET_NAME}, + }, + } + + result = op.get_openlineage_facets_on_complete(task_instance=mock.Mock()) + job_facet = result.job_facets["cloudStorageTransferJob"] + assert job_facet.objectConditions is None + assert job_facet.transferOptions is None + assert job_facet.schedule is None + + def test_get_openlineage_facets_on_complete_delete_job_after_completion_still_produces_facets(self): + op = CloudDataTransferServiceS3ToGCSOperator( + task_id=TASK_ID, + s3_bucket=AWS_BUCKET_NAME, + gcs_bucket=GCS_BUCKET_NAME, + project_id=GCP_PROJECT_ID, + delete_job_after_completion=True, + wait=True, + ) + op._transfer_job = {NAME: "transferJobs/333", PROJECT_ID: GCP_PROJECT_ID} + + result = op.get_openlineage_facets_on_complete(task_instance=mock.Mock()) + + assert "cloudStorageTransferJob" in result.job_facets + assert "cloudStorageTransferRun" in result.run_facets + run_facet = result.run_facets["cloudStorageTransferRun"] + assert run_facet.deleteJobAfterCompletion is True + + def test_get_openlineage_facets_on_complete_inputs_outputs_when_paths_missing(self): + op = CloudDataTransferServiceS3ToGCSOperator( + task_id=TASK_ID, + s3_bucket=AWS_BUCKET_NAME, + gcs_bucket=GCS_BUCKET_NAME, + ) + op._transfer_job = { + NAME: "transferJobs/444", + PROJECT_ID: GCP_PROJECT_ID, + "transferSpec": { + AWS_S3_DATA_SOURCE: {BUCKET_NAME: AWS_BUCKET_NAME}, + GCS_DATA_SINK: {BUCKET_NAME: GCS_BUCKET_NAME}, + }, + } + + result = op.get_openlineage_facets_on_complete(task_instance=mock.Mock()) + assert result.inputs[0].namespace == f"s3://{AWS_BUCKET_NAME}" + assert result.inputs[0].name == "" + assert result.outputs[0].namespace == f"gs://{GCS_BUCKET_NAME}" + assert result.outputs[0].name == "" + class TestGoogleCloudStorageToGoogleCloudStorageTransferOperator: def test_constructor(self): From 3762f306d207e469a23f62461f59621201dbab4a Mon Sep 17 00:00:00 2001 From: pawelgrochowicz Date: Fri, 3 Oct 2025 15:36:43 +0200 Subject: [PATCH 191/338] feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator (#56348) * feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator * feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator * feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator * feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator * feature: Add OpenLineage support for SpannerQueryDatabaseInstanceOperator --- .../providers/google/cloud/hooks/spanner.py | 45 +++++ .../google/cloud/operators/spanner.py | 28 +++- .../google/cloud/operators/test_spanner.py | 157 ++++++++++++++++++ 3 files changed, 224 insertions(+), 6 deletions(-) diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py b/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py index d364dd17673a0..7b3fefaa18980 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py @@ -31,6 +31,7 @@ from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.hooks.base_google import GoogleBaseHook, get_field +from airflow.providers.openlineage.sqlparser import DatabaseInfo if TYPE_CHECKING: from google.cloud.spanner_v1.database import Database @@ -38,6 +39,8 @@ from google.cloud.spanner_v1.transaction import Transaction from google.longrunning.operations_grpc_pb2 import Operation + from airflow.models.connection import Connection + class SpannerConnectionParams(NamedTuple): """Information about Google Spanner connection parameters.""" @@ -427,3 +430,45 @@ def _execute_sql_in_transaction(transaction: Transaction, queries: list[str]) -> rc = transaction.execute_update(sql) counts[sql] = rc return counts + + def _get_openlineage_authority_part(self, connection: Connection) -> str | None: + """Build Spanner-specific authority part for OpenLineage. Returns {project}/{instance}.""" + extras = connection.extra_dejson + project_id = extras.get("project_id") + instance_id = extras.get("instance_id") + + if not project_id or not instance_id: + return None + + return f"{project_id}/{instance_id}" + + def get_openlineage_database_dialect(self, connection: Connection) -> str: + """Return database dialect for OpenLineage.""" + return "spanner" + + def get_openlineage_database_info(self, connection: Connection) -> DatabaseInfo: + """Return Spanner specific information for OpenLineage.""" + extras = connection.extra_dejson + database_id = extras.get("database_id") + + return DatabaseInfo( + scheme=self.get_openlineage_database_dialect(connection), + authority=self._get_openlineage_authority_part(connection), + database=database_id, + information_schema_columns=[ + "table_schema", + "table_name", + "column_name", + "ordinal_position", + "spanner_type", + ], + ) + + def get_openlineage_default_schema(self) -> str | None: + """ + Spanner expose 'public' or '' schema depending on dialect(Postgres vs GoogleSQL). + + SQLAlchemy dialect for Spanner does not expose default schema, so we return None + to follow the same approach. + """ + return None diff --git a/providers/google/src/airflow/providers/google/cloud/operators/spanner.py b/providers/google/src/airflow/providers/google/cloud/operators/spanner.py index 732b2e19b7c1b..97d9fe797982d 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/spanner.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/spanner.py @@ -20,6 +20,7 @@ from __future__ import annotations from collections.abc import Sequence +from functools import cached_property from typing import TYPE_CHECKING from airflow.exceptions import AirflowException @@ -29,6 +30,7 @@ from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID if TYPE_CHECKING: + from airflow.providers.openlineage.extractors import OperatorLineage from airflow.utils.context import Context @@ -254,6 +256,13 @@ def __init__( self.impersonation_chain = impersonation_chain super().__init__(**kwargs) + @cached_property + def hook(self) -> SpannerHook: + return SpannerHook( + gcp_conn_id=self.gcp_conn_id, + impersonation_chain=self.impersonation_chain, + ) + def _validate_inputs(self) -> None: if self.project_id == "": raise AirflowException("The required parameter 'project_id' is empty") @@ -265,10 +274,6 @@ def _validate_inputs(self) -> None: raise AirflowException("The required parameter 'query' is empty") def execute(self, context: Context): - hook = SpannerHook( - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) if isinstance(self.query, str): queries = [x.strip() for x in self.query.split(";")] self.sanitize_queries(queries) @@ -281,7 +286,7 @@ def execute(self, context: Context): self.database_id, ) self.log.info("Executing queries: %s", queries) - result_rows_count_per_query = hook.execute_dml( + result_rows_count_per_query = self.hook.execute_dml( project_id=self.project_id, instance_id=self.instance_id, database_id=self.database_id, @@ -291,7 +296,7 @@ def execute(self, context: Context): context=context, instance_id=self.instance_id, database_id=self.database_id, - project_id=self.project_id or hook.project_id, + project_id=self.project_id or self.hook.project_id, ) return result_rows_count_per_query @@ -305,6 +310,17 @@ def sanitize_queries(queries: list[str]) -> None: if queries and queries[-1] == "": queries.pop() + def get_openlineage_facets_on_complete(self, task_instance) -> OperatorLineage | None: + """Build a generic OpenLineage facet, aligned with SQL-based operators.""" + from airflow.providers.common.compat.openlineage.utils.sql import get_openlineage_facets_with_sql + + return get_openlineage_facets_with_sql( + hook=self.hook, + sql=self.query, + conn_id=self.gcp_conn_id, + database=self.database_id, + ) + class SpannerDeployDatabaseInstanceOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/tests/unit/google/cloud/operators/test_spanner.py b/providers/google/tests/unit/google/cloud/operators/test_spanner.py index 1784a0499aab0..7a5b33429b9e7 100644 --- a/providers/google/tests/unit/google/cloud/operators/test_spanner.py +++ b/providers/google/tests/unit/google/cloud/operators/test_spanner.py @@ -18,10 +18,13 @@ from __future__ import annotations from unittest import mock +from unittest.mock import MagicMock import pytest from airflow.exceptions import AirflowException +from airflow.models import Connection +from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.google.cloud.operators.spanner import ( SpannerDeleteDatabaseInstanceOperator, SpannerDeleteInstanceOperator, @@ -30,6 +33,7 @@ SpannerQueryDatabaseInstanceOperator, SpannerUpdateDatabaseInstanceOperator, ) +from airflow.providers.openlineage.sqlparser import DatabaseInfo PROJECT_ID = "project-id" INSTANCE_ID = "instance-id" @@ -42,6 +46,32 @@ CREATE_QUERY = "CREATE TABLE my_table1 (id INT64, name STRING(100))" CREATE_QUERY_2 = "CREATE TABLE my_table2 (id INT64, name STRING(100))" DDL_STATEMENTS = [CREATE_QUERY, CREATE_QUERY_2] +TASK_ID = "task-id" + +SCHEMA_ROWS = { + "public.orders": [ + ("public", "orders", "id", 1, "INT64"), + ("public", "orders", "amount", 2, "FLOAT64"), + ], + "public.staging": [ + ("public", "staging", "id", 1, "INT64"), + ("public", "staging", "amount", 2, "FLOAT64"), + ], + "public.customers": [ + ("public", "customers", "id", 1, "INT64"), + ("public", "customers", "name", 2, "STRING(100)"), + ("public", "customers", "customer_id", 3, "INT64"), + ], + "public.logs": [ + ("public", "logs", "id", 1, "INT64"), + ("public", "logs", "message", 2, "STRING(100)"), + ], + "public.t1": [("public", "t1", "col1", 1, "STRING(100)")], + "public.t2": [("public", "t2", "col1", 1, "STRING(100)")], + "public.t3": [("public", "t3", "id", 1, "INT64")], + # example of explicit non-default schema + "myschema.orders": [("myschema", "orders", "id", 1, "INT64")], +} class TestCloudSpanner: @@ -353,6 +383,133 @@ def test_instance_query_dml_list(self, mock_hook): queries=[INSERT_QUERY, INSERT_QUERY_2], ) + @pytest.mark.parametrize( + "sql, expected_inputs, expected_outputs, expected_lineage", + [ + ("SELECT id, amount FROM public.orders", ["db1.public.orders"], [], {}), + ( + "INSERT INTO public.orders (id, amount) SELECT id, amount FROM public.staging", + ["db1.public.staging", "db1.public.orders"], + [], + {}, + ), + ("DELETE FROM public.logs WHERE id=1", [], ["db1.public.logs"], {}), + ( + "SELECT o.id, c.name FROM public.orders o JOIN public.customers c ON o.customer_id = c.id", + ["db1.public.orders", "db1.public.customers"], + [], + {}, + ), + ( + "UPDATE public.customers SET name='x' WHERE id IN (SELECT id FROM public.staging)", + ["db1.public.customers", "db1.public.staging"], + [], + {}, + ), + ( + ["INSERT INTO public.t1 SELECT * FROM public.t2;", "DELETE FROM public.t3 WHERE id=1;"], + ["db1.public.t1", "db1.public.t2", "db1.public.t3"], + [], + {}, + ), + ("SELECT id, amount FROM myschema.orders", ["db1.myschema.orders"], [], {}), + ], + ) + def test_spannerquerydatabaseinstanceoperator_get_openlineage_facets( + self, sql, expected_inputs, expected_outputs, expected_lineage + ): + # Arrange + class SpannerHookForTests(DbApiHook): + conn_name_attr = "gcp_conn_id" + get_conn = MagicMock(name="conn") + get_connection = MagicMock() + database = DB_ID + + def get_openlineage_database_info(self, connection): + return DatabaseInfo( + scheme="spanner", + authority=f"{PROJECT_ID}/{INSTANCE_ID}", + database=DB_ID, + information_schema_columns=[ + "table_schema", + "table_name", + "column_name", + "ordinal_position", + "spanner_type", + ], + information_schema_table_name="information_schema.columns", + use_flat_cross_db_query=False, + is_information_schema_cross_db=False, + is_uppercase_names=False, + ) + + dbapi_hook = SpannerHookForTests() + + class SpannerOperatorForTest(SpannerQueryDatabaseInstanceOperator): + @property + def hook(self): + return dbapi_hook + + op = SpannerOperatorForTest( + task_id=TASK_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + gcp_conn_id="spanner_conn", + query=sql, + ) + + dbapi_hook.get_connection.return_value = Connection( + conn_id="spanner_conn", conn_type="spanner", host="spanner-host" + ) + + combined_rows = [] + for ds in expected_inputs + expected_outputs: + tbl = ds.split(".", 1)[1] + combined_rows.extend(SCHEMA_ROWS.get(tbl, [])) + + dbapi_hook.get_conn.return_value.cursor.return_value.fetchall.side_effect = [combined_rows, []] + + # Act + lineage = op.get_openlineage_facets_on_complete(task_instance=None) + assert lineage is not None + + # Assert inputs + input_names = {ds.name for ds in lineage.inputs} + assert input_names == set(expected_inputs) + for ds in lineage.inputs: + assert ds.namespace == f"spanner://{PROJECT_ID}/{INSTANCE_ID}" + + # Assert outputs + output_names = {ds.name for ds in lineage.outputs} + assert output_names == set(expected_outputs) + for ds in lineage.outputs: + assert ds.namespace == f"spanner://{PROJECT_ID}/{INSTANCE_ID}" + + # Assert SQLJobFacet + sql_job = lineage.job_facets["sql"] + if isinstance(sql, list): + for q in sql: + assert q.replace(";", "").strip() in sql_job.query.replace(";", "") + else: + assert sql_job.query == sql + + # Assert column lineage + found_lineage = { + getattr(field, "field", None) or getattr(field, "name", None): [ + f"{inp.dataset.name}.{getattr(inp, 'field', getattr(inp, 'name', None))}" + for inp in getattr(field, "inputFields", []) + ] + for ds in lineage.outputs + lineage.inputs + for cl_facet in [ds.facets.get("columnLineage")] + if cl_facet + for field in cl_facet.fields + } + + for col, sources in expected_lineage.items(): + assert col in found_lineage + for src in sources: + assert any(src in s for s in found_lineage[col]) + @mock.patch("airflow.providers.google.cloud.operators.spanner.SpannerHook") def test_database_create(self, mock_hook): mock_hook.return_value.get_database.return_value = None From c086f770407ba0e90bcc45343f01871805935e68 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Fri, 3 Oct 2025 19:44:49 +0530 Subject: [PATCH 192/338] Add a section in contributors guide about working with dags (#56362) --- .../03a_contributors_quick_start_beginners.rst | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/contributing-docs/03a_contributors_quick_start_beginners.rst b/contributing-docs/03a_contributors_quick_start_beginners.rst index 3fa429ddb4ad7..9622b44453293 100644 --- a/contributing-docs/03a_contributors_quick_start_beginners.rst +++ b/contributing-docs/03a_contributors_quick_start_beginners.rst @@ -87,6 +87,18 @@ and launches all Airflow necessary components in those terminals. To know more a check out this cheat sheet: https://tmuxcheatsheet.com/. To exit breeze, type ``start-airflow`` in any of the tmux panes and hit Enter. +**Working with DAGs in Breeze:** + +- **Adding your own DAGs**: Place your DAG files in the ``/files/dags/`` directory in your local Airflow repository. This directory is automatically mounted into the Breeze container and your DAGs will be visible in the Airflow UI. + +- **Loading example DAGs**: Use the ``--load-example-dags`` flag to load all example DAGs from the repository: + +.. code-block:: bash + + breeze start-airflow --load-example-dags + +This flag enables configuration to load example DAGs when starting Airflow, which is useful for exploring Airflow's capabilities and testing. + 4. **Make a tiny change** – e.g. fix a typo in docs 5. **Run local checks** From d05620a817c43314528e2ba0b535dc2085c06970 Mon Sep 17 00:00:00 2001 From: Marc Lherbette Date: Fri, 3 Oct 2025 16:47:13 +0200 Subject: [PATCH 193/338] Update _helpers.yaml to support custom apiServerConfigConfigMapName (#52634) * Update _helpers.yaml to support custom apiServerConfigConfigMapName * Add unit tests for ApiServer ConfigMaps * fix: run ruff format --------- Co-authored-by: rom sharon <33751805+romsharon98@users.noreply.github.com> --- .../helm_tests/apiserver/test_apiserver.py | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/helm-tests/tests/helm_tests/apiserver/test_apiserver.py b/helm-tests/tests/helm_tests/apiserver/test_apiserver.py index 2db007234fed1..e7ad1314f18e1 100644 --- a/helm-tests/tests/helm_tests/apiserver/test_apiserver.py +++ b/helm-tests/tests/helm_tests/apiserver/test_apiserver.py @@ -106,3 +106,49 @@ def test_should_add_annotations_to_api_secret_key_secret(self): assert "annotations" in jmespath.search("metadata", docs) assert jmespath.search("metadata.annotations", docs)["test_annotation"] == "test_annotation_value" + + +class TestApiserverConfigmap: + """Tests apiserver configmap.""" + + def test_no_apiserver_config_configmap_by_default(self): + docs = render_chart(show_only=["templates/configmaps/api-server-configmap.yaml"]) + assert len(docs) == 0 + + def test_no_apiserver_config_configmap_with_configmap_name(self): + docs = render_chart( + values={ + "apiServer": { + "apiServerConfig": "CSRF_ENABLED = True # {{ .Release.Name }}", + "apiServerConfigConfigMapName": "my-configmap", + } + }, + show_only=["templates/configmaps/api-server-configmap.yaml"], + ) + assert len(docs) == 0 + + def test_apiserver_with_custom_configmap_name(self): + docs = render_chart( + values={ + "apiServer": { + "apiServerConfigConfigMapName": "my-custom-configmap", + } + }, + show_only=["templates/api-server/api-server-deployment.yaml"], + ) + assert ( + jmespath.search("spec.template.spec.volumes[1].configMap.name", docs[0]) == "my-custom-configmap" + ) + + def test_apiserver_config_configmap(self): + docs = render_chart( + values={"apiServer": {"apiServerConfig": "CSRF_ENABLED = True # {{ .Release.Name }}"}}, + show_only=["templates/configmaps/api-server-configmap.yaml"], + ) + + assert docs[0]["kind"] == "ConfigMap" + assert jmespath.search("metadata.name", docs[0]) == "release-name-api-server-config" + assert ( + jmespath.search('data."webserver_config.py"', docs[0]).strip() + == "CSRF_ENABLED = True # release-name" + ) From baafd38f156d9131b110d0d91238f2aafa01b7b9 Mon Sep 17 00:00:00 2001 From: Javier Herrer Torres Date: Fri, 3 Oct 2025 17:07:51 +0200 Subject: [PATCH 194/338] fix(livy): replace connection type (#55984) Co-authored-by: Javier Herrer Torres --- providers/apache/livy/docs/connections.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/providers/apache/livy/docs/connections.rst b/providers/apache/livy/docs/connections.rst index d14f319fc690d..86752ebbf8abe 100644 --- a/providers/apache/livy/docs/connections.rst +++ b/providers/apache/livy/docs/connections.rst @@ -56,4 +56,4 @@ For example: .. code-block:: bash - export AIRFLOW_CONN_LIVY_DEFAULT='http://username:password@livy-server.com:80?headers=header' + export AIRFLOW_CONN_LIVY_DEFAULT='livy://username:password@livy-server.com:80/http?headers=header' From afa8c32a1dd5d32c0d75ac6ff5425ccdc828f309 Mon Sep 17 00:00:00 2001 From: karunpoudel <62040859+karunpoudel@users.noreply.github.com> Date: Fri, 3 Oct 2025 11:34:27 -0400 Subject: [PATCH 195/338] Reapply "update AzureBaseHook to return credentials that supports get_token method" (#56228) * all changes and fixed * more fix --------- Co-authored-by: Karun Poudel <64540927+karunpoudel-chr@users.noreply.github.com> --- docs/spelling_wordlist.txt | 3 + .../azure/docs/connections/azure.rst | 2 + .../microsoft/azure/hooks/base_azure.py | 108 +++++++++++++++--- .../microsoft/azure/hooks/test_base_azure.py | 90 ++++++++++++++- 4 files changed, 186 insertions(+), 17 deletions(-) diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 11c3497b3c239..35cddc8aa55ee 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -272,6 +272,7 @@ classpaths cleartext cli clientId +ClientSecretCredential cloudant CloudantV cloudbuild @@ -476,6 +477,7 @@ deduplicated deduplication deepcopy deepcopying +DefaultAzureCredential deferrable deidentify DeidentifyContentResponse @@ -1612,6 +1614,7 @@ serializer serializers serverless ServiceAccount +ServicePrincipalCredentials ServiceResource SES sessionmaker diff --git a/providers/microsoft/azure/docs/connections/azure.rst b/providers/microsoft/azure/docs/connections/azure.rst index f8d111fd34a10..abe425067eb84 100644 --- a/providers/microsoft/azure/docs/connections/azure.rst +++ b/providers/microsoft/azure/docs/connections/azure.rst @@ -74,6 +74,8 @@ Extra (optional) It specifies the json that contains the authentication information. * ``managed_identity_client_id``: The client ID of a user-assigned managed identity. If provided with ``workload_identity_tenant_id``, they'll pass to DefaultAzureCredential_. * ``workload_identity_tenant_id``: ID of the application's Microsoft Entra tenant. Also called its "directory" ID. If provided with ``managed_identity_client_id``, they'll pass to DefaultAzureCredential_. + * ``use_azure_identity_object``: If set to true, it will use credential of newer type: ClientSecretCredential or DefaultAzureCredential instead of ServicePrincipalCredentials or AzureIdentityCredentialAdapter. + These newer credentials support get_token method which can be used to generate OAuth token with custom scope. The entire extra column can be left out to fall back on DefaultAzureCredential_. diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py index 2a59234e0a4c7..b04b90bce76ca 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py @@ -16,18 +16,25 @@ # under the License. from __future__ import annotations -from typing import Any +from typing import TYPE_CHECKING, Any from azure.common.client_factory import get_client_from_auth_file, get_client_from_json_dict from azure.common.credentials import ServicePrincipalCredentials +from azure.identity import ClientSecretCredential, DefaultAzureCredential from airflow.exceptions import AirflowException from airflow.providers.microsoft.azure.utils import ( AzureIdentityCredentialAdapter, add_managed_identity_connection_widgets, + get_sync_default_azure_credential, ) from airflow.providers.microsoft.azure.version_compat import BaseHook +if TYPE_CHECKING: + from azure.core.credentials import AccessToken + + from airflow.sdk import Connection + class AzureBaseHook(BaseHook): """ @@ -85,7 +92,7 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]: }, } - def __init__(self, sdk_client: Any, conn_id: str = "azure_default"): + def __init__(self, sdk_client: Any = None, conn_id: str = "azure_default"): self.sdk_client = sdk_client self.conn_id = conn_id super().__init__() @@ -96,8 +103,9 @@ def get_conn(self) -> Any: :return: the authenticated client. """ + if not self.sdk_client: + raise ValueError("`sdk_client` must be provided to AzureBaseHook to use `get_conn` method.") conn = self.get_connection(self.conn_id) - tenant = conn.extra_dejson.get("tenantId") subscription_id = conn.extra_dejson.get("subscriptionId") key_path = conn.extra_dejson.get("key_path") if key_path: @@ -111,22 +119,90 @@ def get_conn(self) -> Any: self.log.info("Getting connection using a JSON config.") return get_client_from_json_dict(client_class=self.sdk_client, config_dict=key_json) - credentials: ServicePrincipalCredentials | AzureIdentityCredentialAdapter + credentials = self.get_credential(conn=conn) + + return self.sdk_client( + credentials=credentials, + subscription_id=subscription_id, + ) + + def get_credential( + self, *, conn: Connection | None = None + ) -> ( + ServicePrincipalCredentials + | AzureIdentityCredentialAdapter + | ClientSecretCredential + | DefaultAzureCredential + ): + """ + Get Azure credential object for the connection. + + Azure Identity based credential object (``ClientSecretCredential``, ``DefaultAzureCredential``) can be used to get OAuth token using ``get_token`` method. + Older Credential objects (``ServicePrincipalCredentials``, ``AzureIdentityCredentialAdapter``) are supported for backward compatibility. + + :return: The Azure credential object + """ + if not conn: + conn = self.get_connection(self.conn_id) + tenant = conn.extra_dejson.get("tenantId") + credential: ( + ServicePrincipalCredentials + | AzureIdentityCredentialAdapter + | ClientSecretCredential + | DefaultAzureCredential + ) if all([conn.login, conn.password, tenant]): - self.log.info("Getting connection using specific credentials and subscription_id.") - credentials = ServicePrincipalCredentials( - client_id=conn.login, secret=conn.password, tenant=tenant - ) + credential = self._get_client_secret_credential(conn) else: - self.log.info("Using DefaultAzureCredential as credential") - managed_identity_client_id = conn.extra_dejson.get("managed_identity_client_id") - workload_identity_tenant_id = conn.extra_dejson.get("workload_identity_tenant_id") - credentials = AzureIdentityCredentialAdapter( + credential = self._get_default_azure_credential(conn) + return credential + + def _get_client_secret_credential( + self, conn: Connection + ) -> ServicePrincipalCredentials | ClientSecretCredential: + self.log.info("Getting credentials using specific credentials and subscription_id.") + extra_dejson = conn.extra_dejson + tenant = extra_dejson.get("tenantId") + use_azure_identity_object = extra_dejson.get("use_azure_identity_object", False) + if use_azure_identity_object: + return ClientSecretCredential( + client_id=conn.login, # type: ignore[arg-type] + client_secret=conn.password, # type: ignore[arg-type] + tenant_id=tenant, # type: ignore[arg-type] + ) + return ServicePrincipalCredentials(client_id=conn.login, secret=conn.password, tenant=tenant) + + def _get_default_azure_credential( + self, conn: Connection + ) -> DefaultAzureCredential | AzureIdentityCredentialAdapter: + self.log.info("Using DefaultAzureCredential as credential") + extra_dejson = conn.extra_dejson + managed_identity_client_id = extra_dejson.get("managed_identity_client_id") + workload_identity_tenant_id = extra_dejson.get("workload_identity_tenant_id") + use_azure_identity_object = extra_dejson.get("use_azure_identity_object", False) + if use_azure_identity_object: + return get_sync_default_azure_credential( managed_identity_client_id=managed_identity_client_id, workload_identity_tenant_id=workload_identity_tenant_id, ) - - return self.sdk_client( - credentials=credentials, - subscription_id=subscription_id, + return AzureIdentityCredentialAdapter( + managed_identity_client_id=managed_identity_client_id, + workload_identity_tenant_id=workload_identity_tenant_id, ) + + def get_token(self, *scopes, **kwargs) -> AccessToken: + """ + Request an access token for `scopes`. + + To use this method, set `use_azure_identity_object: True` in the connection extra field. + ServicePrincipalCredentials and AzureIdentityCredentialAdapter don't support `get_token` method. + """ + credential = self.get_credential() + if isinstance(credential, ServicePrincipalCredentials) or isinstance( + credential, AzureIdentityCredentialAdapter + ): + raise AttributeError( + "ServicePrincipalCredentials and AzureIdentityCredentialAdapter don't support get_token method. " + "Please set `use_azure_identity_object: True` in the connection extra field to use credential that support get_token method." + ) + return credential.get_token(*scopes, **kwargs) diff --git a/providers/microsoft/azure/tests/unit/microsoft/azure/hooks/test_base_azure.py b/providers/microsoft/azure/tests/unit/microsoft/azure/hooks/test_base_azure.py index 89881eae16513..a0fc50857d984 100644 --- a/providers/microsoft/azure/tests/unit/microsoft/azure/hooks/test_base_azure.py +++ b/providers/microsoft/azure/tests/unit/microsoft/azure/hooks/test_base_azure.py @@ -31,6 +31,7 @@ Connection = MagicMock() # type: ignore[misc] MODULE = "airflow.providers.microsoft.azure.hooks.base_azure" +UTILS = "airflow.providers.microsoft.azure.utils" class TestBaseAzureHook: @@ -111,7 +112,7 @@ def test_get_conn_with_credentials(self, mock_spc, mocked_connection): indirect=True, ) @patch("azure.common.credentials.ServicePrincipalCredentials") - @patch("airflow.providers.microsoft.azure.hooks.base_azure.AzureIdentityCredentialAdapter") + @patch(f"{MODULE}.AzureIdentityCredentialAdapter") def test_get_conn_fallback_to_azure_identity_credential_adapter( self, mock_credential_adapter, @@ -133,3 +134,90 @@ def test_get_conn_fallback_to_azure_identity_credential_adapter( credentials=mock_credential, subscription_id="subscription_id", ) + + @patch(f"{MODULE}.ClientSecretCredential") + @pytest.mark.parametrize( + "mocked_connection", + [ + Connection( + conn_id="azure_default", + login="my_login", + password="my_password", + extra={"tenantId": "my_tenant", "use_azure_identity_object": True}, + ), + ], + indirect=True, + ) + def test_get_credential_with_client_secret(self, mock_spc, mocked_connection): + mock_spc.return_value = "foo-bar" + cred = AzureBaseHook().get_credential() + + mock_spc.assert_called_once_with( + client_id=mocked_connection.login, + client_secret=mocked_connection.password, + tenant_id=mocked_connection.extra_dejson["tenantId"], + ) + assert cred == "foo-bar" + + @patch(f"{UTILS}.DefaultAzureCredential") + @pytest.mark.parametrize( + "mocked_connection", + [ + Connection( + conn_id="azure_default", + extra={"use_azure_identity_object": True}, + ), + ], + indirect=True, + ) + def test_get_credential_with_azure_default_credential(self, mock_spc, mocked_connection): + mock_spc.return_value = "foo-bar" + cred = AzureBaseHook().get_credential() + + mock_spc.assert_called_once_with() + assert cred == "foo-bar" + + @patch(f"{UTILS}.DefaultAzureCredential") + @pytest.mark.parametrize( + "mocked_connection", + [ + Connection( + conn_id="azure_default", + extra={ + "managed_identity_client_id": "test_client_id", + "workload_identity_tenant_id": "test_tenant_id", + "use_azure_identity_object": True, + }, + ), + ], + indirect=True, + ) + def test_get_credential_with_azure_default_credential_with_extra(self, mock_spc, mocked_connection): + mock_spc.return_value = "foo-bar" + cred = AzureBaseHook().get_credential() + + mock_spc.assert_called_once_with( + managed_identity_client_id=mocked_connection.extra_dejson.get("managed_identity_client_id"), + workload_identity_tenant_id=mocked_connection.extra_dejson.get("workload_identity_tenant_id"), + additionally_allowed_tenants=[mocked_connection.extra_dejson.get("workload_identity_tenant_id")], + ) + assert cred == "foo-bar" + + @patch(f"{UTILS}.DefaultAzureCredential") + @pytest.mark.parametrize( + "mocked_connection", + [ + Connection( + conn_id="azure_default", + extra={"use_azure_identity_object": True}, + ), + ], + indirect=True, + ) + def test_get_token_with_azure_default_credential(self, mock_spc, mocked_connection): + mock_spc.return_value.get_token.return_value = "new-token" + scope = "custom_scope" + token = AzureBaseHook().get_token(scope) + + mock_spc.assert_called_once_with() + assert token == "new-token" From f384b1af6c94deb41aabeb9026dea4f19b49ced9 Mon Sep 17 00:00:00 2001 From: GPK Date: Fri, 3 Oct 2025 19:56:15 +0100 Subject: [PATCH 196/338] Restrict universal-pathlib 0.3.0 (#56370) --- airflow-core/pyproject.toml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/airflow-core/pyproject.toml b/airflow-core/pyproject.toml index 0dfe6ed78b97d..71c1a7be52175 100644 --- a/airflow-core/pyproject.toml +++ b/airflow-core/pyproject.toml @@ -134,9 +134,8 @@ dependencies = [ "tenacity>=8.3.0", "termcolor>=3.0.0", "typing-extensions>=4.14.1", - # Universal Pathlib 0.2.4 adds extra validation for Paths and our integration with local file paths - # Does not work with it Tracked in https://github.com/fsspec/universal_pathlib/issues/276 - "universal-pathlib>=0.2.2,!=0.2.4", + # https://github.com/apache/airflow/issues/56369 , rework universal-pathlib usage + "universal-pathlib>=0.2.6,<0.3.0", "uuid6>=2024.7.10", "apache-airflow-task-sdk<1.3.0,>=1.1.0", # pre-installed providers From 402e177a60efb2bcd09011d04ead1d5a61d71aa9 Mon Sep 17 00:00:00 2001 From: Bas Harenslak Date: Sat, 4 Oct 2025 06:12:54 +0200 Subject: [PATCH 197/338] Prefer BigQueryInsertJobOperator's project_id over hook's project_id for openlineage (#55948) * Prefer operator's project_id over hook's project_id * Remove unused import * Formatting * Remove unnecessary arguments * Make mypy happy --- .../google/cloud/openlineage/mixins.py | 4 ++- .../google/cloud/openlineage/test_mixins.py | 34 +++++++++++++++++++ 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/providers/google/src/airflow/providers/google/cloud/openlineage/mixins.py b/providers/google/src/airflow/providers/google/cloud/openlineage/mixins.py index 5eba61b188649..539c24f65332f 100644 --- a/providers/google/src/airflow/providers/google/cloud/openlineage/mixins.py +++ b/providers/google/src/airflow/providers/google/cloud/openlineage/mixins.py @@ -97,7 +97,9 @@ def get_openlineage_facets_on_complete(self, _): run_facets: dict[str, RunFacet] = { "externalQuery": ExternalQueryRunFacet(externalQueryId=self.job_id, source="bigquery") } - self._client = self.hook.get_client(project_id=self.hook.project_id, location=self.location) + self._client = self.hook.get_client( + project_id=self.project_id or self.hook.project_id, location=self.location + ) try: job_properties = self._client.get_job(job_id=self.job_id)._properties diff --git a/providers/google/tests/unit/google/cloud/openlineage/test_mixins.py b/providers/google/tests/unit/google/cloud/openlineage/test_mixins.py index 7006d2f9130c4..e7204368fd500 100644 --- a/providers/google/tests/unit/google/cloud/openlineage/test_mixins.py +++ b/providers/google/tests/unit/google/cloud/openlineage/test_mixins.py @@ -1013,3 +1013,37 @@ def test_generate_column_lineage_facet(self): ), } ) + + def test_project_id_selection(self): + """ + Check if project_id set via an argument to the operator takes prevalence over project_id set in a + connection. + """ + from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + + class TestOperator(GoogleCloudBaseOperator, _BigQueryInsertJobOperatorOpenLineageMixin): + def __init__(self, project_id: str | None = None, **_): + self.project_id = project_id + self.job_id = "foobar" + self.location = "foobar" + self.sql = "foobar" + + # First test task where project_id is set explicitly + test = TestOperator(project_id="project_a") + test.hook = MagicMock() + test.hook.project_id = "project_b" + test._client = MagicMock() + + test.get_openlineage_facets_on_complete(None) + _, kwargs = test.hook.get_client.call_args + assert kwargs["project_id"] == "project_a" + + # Then test task where project_id is inherited from the hook + test = TestOperator() + test.hook = MagicMock() + test.hook.project_id = "project_b" + test._client = MagicMock() + + test.get_openlineage_facets_on_complete(None) + _, kwargs = test.hook.get_client.call_args + assert kwargs["project_id"] == "project_b" From 1f8324ddf764e72a589571514f107c650427f0a5 Mon Sep 17 00:00:00 2001 From: "LI,JHE-CHEN" <103923510+RoyLee1224@users.noreply.github.com> Date: Sat, 4 Oct 2025 03:04:35 -0400 Subject: [PATCH 198/338] fix: modify min width for task names (#56378) --- airflow-core/src/airflow/ui/src/layouts/Details/Grid/Grid.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/Grid.tsx b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/Grid.tsx index 5817d753eb908..0943f12f938f0 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/Grid.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/Grid.tsx @@ -105,7 +105,7 @@ export const Grid = ({ dagRunState, limit, runType, showGantt, triggeringUser }: tabIndex={0} width={showGantt ? "1/2" : "full"} > - + setMode("task")} /> From ca037ea675dbb8f18f840922de3c78fbeb4f55b3 Mon Sep 17 00:00:00 2001 From: Elad Kalif <45845474+eladkal@users.noreply.github.com> Date: Sat, 4 Oct 2025 13:34:19 +0300 Subject: [PATCH 199/338] Prepare fab provider rc2 to release (October 2025) (#56384) --- providers/fab/docs/changelog.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/providers/fab/docs/changelog.rst b/providers/fab/docs/changelog.rst index 07b205e4cc303..41f6d23d3a453 100644 --- a/providers/fab/docs/changelog.rst +++ b/providers/fab/docs/changelog.rst @@ -41,11 +41,13 @@ Bug Fixes ~~~~~~~~~ * ``Add 'if_not_exists=True' to FAB migration (#56100)`` +* ``Add if_not_exists to index creation in migrations (#56328)`` .. Below changes are excluded from the changelog. Move them to appropriate section above if needed. Do not delete the lines(!): * ``Fix static check error resulting from not rebased change in FAB5 (#56178)`` * ``Remove placeholder Release Date in changelog and index files (#56056)`` + * ``Prepare fab and amazon providers to release (September 2025) (#56241)`` 2.4.4 ..... From 793c3cef07769bbfed9ad14dba6405d5c7908be6 Mon Sep 17 00:00:00 2001 From: Elad Kalif <45845474+eladkal@users.noreply.github.com> Date: Sat, 4 Oct 2025 15:42:15 +0300 Subject: [PATCH 200/338] Fix providers release manager docs (#56385) --- dev/README_RELEASE_PROVIDERS.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dev/README_RELEASE_PROVIDERS.md b/dev/README_RELEASE_PROVIDERS.md index 273ea9910fa89..a6d584230e7d5 100644 --- a/dev/README_RELEASE_PROVIDERS.md +++ b/dev/README_RELEASE_PROVIDERS.md @@ -1148,7 +1148,7 @@ Email subject: ``` cat < +${RELEASE_MANAGER_NAME} EOF ``` @@ -1339,12 +1339,14 @@ The command does the following: 3. Triggers S3 to GitHub Sync ```shell script + unset GITHUB_TOKEN breeze workflow-run publish-docs --ref --site-env all-providers ``` Or if you just want to publish a few selected providers, you can run: ```shell script + unset GITHUB_TOKEN breeze workflow-run publish-docs --ref --site-env PACKAGE1 PACKAGE2 .. ``` From 9eb8b1ab127db94c74c1b857c5400bb158a58e01 Mon Sep 17 00:00:00 2001 From: andwct <62198411+andwct@users.noreply.github.com> Date: Sat, 4 Oct 2025 20:45:52 +0800 Subject: [PATCH 201/338] update breeze Option B quick setup (#56333) --- ...03a_contributors_quick_start_beginners.rst | 46 ++++++++++++++----- 1 file changed, 34 insertions(+), 12 deletions(-) diff --git a/contributing-docs/03a_contributors_quick_start_beginners.rst b/contributing-docs/03a_contributors_quick_start_beginners.rst index 9622b44453293..9214ba8fc62e5 100644 --- a/contributing-docs/03a_contributors_quick_start_beginners.rst +++ b/contributing-docs/03a_contributors_quick_start_beginners.rst @@ -123,24 +123,46 @@ This flag enables configuration to load example DAGs when starting Airflow, whic git fetch upstream && git rebase upstream/main && git push --force-with-lease -Option B – One-Click GitHub Codespaces -------------------------------------- +## Option B – One-Click GitHub Codespaces + 1. On **your fork**, click *Code → Codespaces → New codespace*. 2. Wait for the VS Code web IDE to appear. A terminal opens automatically. -3. Install Breeze and start the development container +3. Install Docker Buildx and Docker Compose (required for Breeze) .. code-block:: bash - curl -LsSf https://astral.sh/uv/install.sh | sh - uv tool install prek - prek install -f - uv tool install -e ./dev/breeze - uv run setup_vscode.py - breeze start-airflow +``` +mkdir -p ~/.docker/cli-plugins + +# Install Docker Buildx +BUILDX_VERSION=v0.16.2 +curl -SL "https://github.com/docker/buildx/releases/download/${BUILDX_VERSION}/buildx-${BUILDX_VERSION}.linux-amd64" -o ~/.docker/cli-plugins/docker-buildx +chmod +x ~/.docker/cli-plugins/docker-buildx +docker buildx version + +# Install Docker Compose v2 +curl -SL "https://github.com/docker/compose/releases/latest/download/docker-compose-$(uname -s | tr '[:upper:]' '[:lower:]')-$(uname -m)" -o ~/.docker/cli-plugins/docker-compose +chmod +x ~/.docker/cli-plugins/docker-compose +docker compose version +``` + +4. Install Breeze and start the development container + +.. code-block:: bash + +``` +curl -LsSf https://astral.sh/uv/install.sh | sh +uv tool install prek +prek install -f +uv tool install -e ./dev/breeze +uv run setup_vscode.py +breeze start-airflow +``` + +5. Edit a file in the editor, save, and commit via the Source Control +sidebar. Push when prompted. +6. Press **Create pull request** when GitHub offers. -4. Edit a file in the editor, save, and commit via the Source Control - sidebar. Push when prompted. -5. Press **Create pull request** when GitHub offers. Review & Merge -------------- From 46728197a611ddfa5e9ec67b53f50a22b375fada Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Sat, 4 Oct 2025 19:22:02 +0200 Subject: [PATCH 202/338] Add msgraphfs provider (#55454) This enables integration with ObjectStoragePath for onedrive, sharepoint, and teams files. It extends schema support for these as well, so in addition to msgd:// it also accepts sharepoint://, onedrive:// . --- .../azure/docs/filesystems/index.rst | 26 +++ .../azure/docs/filesystems/msgraph.rst | 187 ++++++++++++++++++ providers/microsoft/azure/docs/index.rst | 1 + .../azure/docs/operators/msgraph.rst | 4 + providers/microsoft/azure/provider.yaml | 1 + providers/microsoft/azure/pyproject.toml | 1 + .../providers/microsoft/azure/fs/msgraph.py | 111 +++++++++++ .../microsoft/azure/get_provider_info.py | 5 +- .../microsoft/azure/hooks/msgraph.py | 1 + .../unit/microsoft/azure/fs/test_msgraph.py | 155 +++++++++++++++ 10 files changed, 491 insertions(+), 1 deletion(-) create mode 100644 providers/microsoft/azure/docs/filesystems/index.rst create mode 100644 providers/microsoft/azure/docs/filesystems/msgraph.rst create mode 100644 providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/msgraph.py create mode 100644 providers/microsoft/azure/tests/unit/microsoft/azure/fs/test_msgraph.py diff --git a/providers/microsoft/azure/docs/filesystems/index.rst b/providers/microsoft/azure/docs/filesystems/index.rst new file mode 100644 index 0000000000000..eb0036177a3fa --- /dev/null +++ b/providers/microsoft/azure/docs/filesystems/index.rst @@ -0,0 +1,26 @@ + .. 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. + +Filesystems +=========== + +.. toctree:: + :maxdepth: 1 + :caption: Filesystem Providers + :glob: + + * diff --git a/providers/microsoft/azure/docs/filesystems/msgraph.rst b/providers/microsoft/azure/docs/filesystems/msgraph.rst new file mode 100644 index 0000000000000..7ebea783b7e80 --- /dev/null +++ b/providers/microsoft/azure/docs/filesystems/msgraph.rst @@ -0,0 +1,187 @@ + .. 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. + +Microsoft Graph Filesystem +=========================== + +The Microsoft Graph filesystem provides access to OneDrive, SharePoint, and Teams document libraries through Airflow's ObjectStoragePath interface. + +Supported URL formats: + +* ``msgraph://connection_id/drive_id/path/to/file`` +* ``sharepoint://connection_id/drive_id/path/to/file`` +* ``onedrive://connection_id/drive_id/path/to/file`` +* ``msgd://connection_id/drive_id/path/to/file`` + +Connection Configuration +------------------------ + +Create a Microsoft Graph connection in Airflow with the following parameters: + +* **Connection Type**: msgraph +* **Host**: Tenant ID +* **Login**: Client ID +* **Password**: Client Secret + +The connection form provides additional configuration fields: + +* **Tenant ID**: Azure AD tenant identifier +* **Drive ID**: Specific drive to access (optional - leave empty for general access) +* **Scopes**: OAuth2 scopes (default: https://graph.microsoft.com/.default) + +Additional OAuth2 parameters supported via connection extras: + +* **scope**: OAuth2 access scope +* **token_endpoint**: Custom token endpoint URL +* **redirect_uri**: OAuth2 redirect URI for authorization code flow +* **token_endpoint_auth_method**: Client authentication method (default: client_secret_basic) +* **code_challenge_method**: PKCE code challenge method (e.g., 'S256') +* **username**: Username for password grant flow +* **password**: Password for password grant flow + +Connection extra field configuration example: + +.. code-block:: json + + { + "drive_id": "b!abc123...", + "scope": "https://graph.microsoft.com/.default", + "token_endpoint": "https://login.microsoftonline.com/your-tenant/oauth2/v2.0/token", + "redirect_uri": "http://localhost:8080/callback", + "token_endpoint_auth_method": "client_secret_post" + } + +Usage Examples +-------------- + +Reading Files +^^^^^^^^^^^^^ + +.. code-block:: python + + from airflow.sdk.io.path import ObjectStoragePath + + # Access a file in OneDrive + path = ObjectStoragePath("onedrive://my_conn/drive123/Documents/data.csv") + + # Read file content + with path.open("r") as f: + content = f.read() + +Directory Operations +^^^^^^^^^^^^^^^^^^^^ + +.. code-block:: python + + # List directory contents in SharePoint + sharepoint_path = ObjectStoragePath("sharepoint://sp_conn/site_drive/Shared Documents/") + + for item in sharepoint_path.iterdir(): + print(f"Found: {item.name}") + if item.is_file(): + print(f" Size: {item.stat().st_size} bytes") + +File Operations +^^^^^^^^^^^^^^^ + +.. code-block:: python + + # Copy file between drives + source = ObjectStoragePath("msgraph://conn1/drive1/source.txt") + target = ObjectStoragePath("msgraph://conn2/drive2/backup/source.txt") + source.copy(target) + + # Move file + old_path = ObjectStoragePath("onedrive://conn/drive/temp/file.txt") + new_path = ObjectStoragePath("onedrive://conn/drive/archive/file.txt") + old_path.move(new_path) + + # Delete file + file_to_delete = ObjectStoragePath("msgraph://conn/drive/old_data.csv") + file_to_delete.unlink() + +Writing Files +^^^^^^^^^^^^^ + +.. code-block:: python + + # Write new file + output_path = ObjectStoragePath("sharepoint://sp_conn/docs/reports/report.txt") + + with output_path.open("w") as f: + f.write("Generated report data\n") + f.write(f"Created at: {datetime.now()}\n") + +Drive Discovery +^^^^^^^^^^^^^^^ + +When you need to find the correct drive ID for your URLs, you can use the Microsoft Graph API operators: + +.. code-block:: python + + from airflow.providers.microsoft.azure.operators.msgraph import MSGraphAsyncOperator + + # List all drives for a user + list_drives = MSGraphAsyncOperator( + task_id="list_drives", + conn_id="msgraph_conn", + url="me/drives", + result_processor=lambda response: [ + {"id": drive["id"], "name": drive["name"]} for drive in response["value"] + ], + ) + +URL Scheme Mapping +------------------ + +The different URL schemes map to specific Microsoft Graph endpoints: + +* ``msgraph://`` - General Microsoft Graph access +* ``onedrive://`` - OneDrive personal and business drives +* ``sharepoint://`` - SharePoint document libraries +* ``msgd://`` - Shortened form of msgraph:// + +All schemes use the same underlying Microsoft Graph API and authentication. + +Requirements +------------ + +The Microsoft Graph filesystem requires: + +* ``msgraphfs`` Python package +* Valid Azure AD application registration with appropriate permissions +* Microsoft Graph API access for your tenant + +Required Microsoft Graph permissions: + +* ``Files.Read`` - To read files +* ``Files.ReadWrite`` - To read and write files +* ``Sites.Read.All`` - To access SharePoint sites (if using ``sharepoint://`` URLs) + +Cross-References +---------------- + +* :doc:`Microsoft Graph API Operators ` - For API operations and drive discovery + +Reference +--------- + +For further information, look at: + +* `Microsoft Graph Files API `__ +* `msgraphfs Python package `__ +* `Use the Microsoft Graph API `__ diff --git a/providers/microsoft/azure/docs/index.rst b/providers/microsoft/azure/docs/index.rst index 8ac612221d387..1d9a5dfa686a6 100644 --- a/providers/microsoft/azure/docs/index.rst +++ b/providers/microsoft/azure/docs/index.rst @@ -37,6 +37,7 @@ Connection types Operators Transfers + Filesystems Secrets backends Logging for Tasks Sensors diff --git a/providers/microsoft/azure/docs/operators/msgraph.rst b/providers/microsoft/azure/docs/operators/msgraph.rst index 52f7ff11f52ae..99815a959adeb 100644 --- a/providers/microsoft/azure/docs/operators/msgraph.rst +++ b/providers/microsoft/azure/docs/operators/msgraph.rst @@ -80,6 +80,10 @@ Below is an example of using this operator to create an item schedule in Fabric. :start-after: [START howto_operator_ms_fabric_create_item_schedule] :end-before: [END howto_operator_ms_fabric_create_item_schedule] +Cross-References +---------------- + +* :doc:`Microsoft Graph Filesystem ` - For file operations using ObjectStoragePath Reference --------- diff --git a/providers/microsoft/azure/provider.yaml b/providers/microsoft/azure/provider.yaml index d53f634001ff7..e9a6434ca52e7 100644 --- a/providers/microsoft/azure/provider.yaml +++ b/providers/microsoft/azure/provider.yaml @@ -226,6 +226,7 @@ sensors: filesystems: - airflow.providers.microsoft.azure.fs.adls + - airflow.providers.microsoft.azure.fs.msgraphfs hooks: - integration-name: Microsoft Azure Container Instances diff --git a/providers/microsoft/azure/pyproject.toml b/providers/microsoft/azure/pyproject.toml index 48b06c5e52b33..fcfd11ea1475b 100644 --- a/providers/microsoft/azure/pyproject.toml +++ b/providers/microsoft/azure/pyproject.toml @@ -81,6 +81,7 @@ dependencies = [ "azure-mgmt-containerregistry>=8.0.0", "azure-mgmt-containerinstance>=10.1.0", "msgraph-core>=1.3.3", + "msgraphfs>=0.3.0", "microsoft-kiota-http>=1.9.4,<2.0.0", "microsoft-kiota-serialization-json>=1.9.4", "microsoft-kiota-serialization-text>=1.9.4", diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/msgraph.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/msgraph.py new file mode 100644 index 0000000000000..990b74cf0d9a8 --- /dev/null +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/msgraph.py @@ -0,0 +1,111 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from typing import TYPE_CHECKING, Any + +from airflow.providers.microsoft.azure.utils import get_field +from airflow.providers.microsoft.azure.version_compat import BaseHook + +if TYPE_CHECKING: + from fsspec import AbstractFileSystem + +schemes = ["msgraph", "sharepoint", "onedrive", "msgd"] + + +def get_fs(conn_id: str | None, storage_options: dict[str, Any] | None = None) -> AbstractFileSystem: + from msgraphfs import MSGDriveFS + + if conn_id is None: + return MSGDriveFS({}) + + conn = BaseHook.get_connection(conn_id) + extras = conn.extra_dejson + conn_type = conn.conn_type or "msgraph" + + options: dict[str, Any] = {} + + # Get authentication parameters with fallback handling + client_id = conn.login or get_field( + conn_id=conn_id, conn_type=conn_type, extras=extras, field_name="client_id" + ) + client_secret = conn.password or get_field( + conn_id=conn_id, conn_type=conn_type, extras=extras, field_name="client_secret" + ) + tenant_id = conn.host or get_field( + conn_id=conn_id, conn_type=conn_type, extras=extras, field_name="tenant_id" + ) + + if client_id: + options["client_id"] = client_id + if client_secret: + options["client_secret"] = client_secret + if tenant_id: + options["tenant_id"] = tenant_id + + # Process additional fields from extras + fields = [ + "drive_id", + "scope", + "token_endpoint", + "redirect_uri", + "token_endpoint_auth_method", + "code_challenge_method", + "update_token", + "username", + "password", + ] + for field in fields: + value = get_field(conn_id=conn_id, conn_type=conn_type, extras=extras, field_name=field) + if value is not None: + if value == "": + options.pop(field, "") + else: + options[field] = value + + # Update with storage options + options.update(storage_options or {}) + + # Create oauth2 client parameters if authentication is provided + oauth2_client_params = {} + if options.get("client_id") and options.get("client_secret") and options.get("tenant_id"): + oauth2_client_params = { + "client_id": options["client_id"], + "client_secret": options["client_secret"], + "tenant_id": options["tenant_id"], + } + + # Add additional oauth2 parameters supported by authlib + oauth2_params = [ + "scope", + "token_endpoint", + "redirect_uri", + "token_endpoint_auth_method", + "code_challenge_method", + "update_token", + "username", + "password", + ] + for param in oauth2_params: + if param in options: + oauth2_client_params[param] = options[param] + + # Determine which filesystem to return based on drive_id + drive_id = options.get("drive_id") + + return MSGDriveFS(drive_id=drive_id, oauth2_client_params=oauth2_client_params) diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/get_provider_info.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/get_provider_info.py index 58337ecdf8975..395622f8323f5 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/get_provider_info.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/get_provider_info.py @@ -191,7 +191,10 @@ def get_provider_info(): "python-modules": ["airflow.providers.microsoft.azure.sensors.msgraph"], }, ], - "filesystems": ["airflow.providers.microsoft.azure.fs.adls"], + "filesystems": [ + "airflow.providers.microsoft.azure.fs.adls", + "airflow.providers.microsoft.azure.fs.msgraphfs", + ], "hooks": [ { "integration-name": "Microsoft Azure Container Instances", diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py index fabe2d4365397..bb4c47b023a6d 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py @@ -152,6 +152,7 @@ def get_connection_form_widgets(cls) -> dict[str, Any]: return { "tenant_id": StringField(lazy_gettext("Tenant ID"), widget=BS3TextFieldWidget()), + "drive_id": StringField(lazy_gettext("Drive ID"), widget=BS3TextFieldWidget()), "api_version": StringField( lazy_gettext("API Version"), widget=BS3TextFieldWidget(), default=APIVersion.v1.value ), diff --git a/providers/microsoft/azure/tests/unit/microsoft/azure/fs/test_msgraph.py b/providers/microsoft/azure/tests/unit/microsoft/azure/fs/test_msgraph.py new file mode 100644 index 0000000000000..235219b6dcb30 --- /dev/null +++ b/providers/microsoft/azure/tests/unit/microsoft/azure/fs/test_msgraph.py @@ -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. + +from __future__ import annotations + +from unittest.mock import MagicMock, patch + +import pytest + +from airflow.models.connection import Connection +from airflow.providers.microsoft.azure.fs.msgraph import get_fs + + +@pytest.fixture +def mock_connection(): + return Connection( + conn_id="msgraph_default", + conn_type="msgraph", + login="test_client_id", + password="test_client_secret", + host="test_tenant_id", + extra={"drive_id": "test_drive_id"}, + ) + + +@pytest.fixture +def mock_connection_minimal(): + return Connection( + conn_id="msgraph_minimal", + conn_type="msgraph", + login="test_client_id", + password="test_client_secret", + host="test_tenant_id", + ) + + +class TestMSGraphFS: + @patch("airflow.providers.microsoft.azure.fs.msgraph.BaseHook.get_connection") + @patch("msgraphfs.MSGDriveFS") + def test_get_fs_with_drive_id(self, mock_msgdrivefs, mock_get_connection, mock_connection): + mock_get_connection.return_value = mock_connection + mock_fs_instance = MagicMock() + mock_msgdrivefs.return_value = mock_fs_instance + + result = get_fs("msgraph_default") + + mock_msgdrivefs.assert_called_once_with( + drive_id="test_drive_id", + oauth2_client_params={ + "client_id": "test_client_id", + "client_secret": "test_client_secret", + "tenant_id": "test_tenant_id", + }, + ) + assert result == mock_fs_instance + + @patch("msgraphfs.MSGDriveFS") + def test_get_fs_no_connection(self, mock_msgdrivefs): + mock_fs_instance = MagicMock() + mock_msgdrivefs.return_value = mock_fs_instance + + result = get_fs(None) + + mock_msgdrivefs.assert_called_once_with({}) + assert result == mock_fs_instance + + @patch("airflow.providers.microsoft.azure.fs.msgraph.BaseHook.get_connection") + @patch("msgraphfs.MSGDriveFS") + def test_get_fs_with_extra_oauth_params(self, mock_msgdrivefs, mock_get_connection): + connection = Connection( + conn_id="msgraph_extra", + conn_type="msgraph", + login="test_client_id", + password="test_client_secret", + host="test_tenant_id", + extra={ + "drive_id": "test_drive_id", + "scope": "https://graph.microsoft.com/.default", + "token_endpoint": "https://login.microsoftonline.com/test/oauth2/v2.0/token", + "redirect_uri": "http://localhost:8080/callback", + }, + ) + mock_get_connection.return_value = connection + mock_fs_instance = MagicMock() + mock_msgdrivefs.return_value = mock_fs_instance + + result = get_fs("msgraph_extra") + + expected_oauth2_params = { + "client_id": "test_client_id", + "client_secret": "test_client_secret", + "tenant_id": "test_tenant_id", + "scope": "https://graph.microsoft.com/.default", + "token_endpoint": "https://login.microsoftonline.com/test/oauth2/v2.0/token", + "redirect_uri": "http://localhost:8080/callback", + } + mock_msgdrivefs.assert_called_once_with( + drive_id="test_drive_id", oauth2_client_params=expected_oauth2_params + ) + assert result == mock_fs_instance + + @patch("airflow.providers.microsoft.azure.fs.msgraph.BaseHook.get_connection") + @patch("msgraphfs.MSGDriveFS") + def test_get_fs_with_storage_options(self, mock_msgdrivefs, mock_get_connection, mock_connection_minimal): + mock_get_connection.return_value = mock_connection_minimal + mock_fs_instance = MagicMock() + mock_msgdrivefs.return_value = mock_fs_instance + + storage_options = {"drive_id": "storage_drive_id", "scope": "custom.scope"} + result = get_fs("msgraph_minimal", storage_options=storage_options) + + expected_oauth2_params = { + "client_id": "test_client_id", + "client_secret": "test_client_secret", + "tenant_id": "test_tenant_id", + "scope": "custom.scope", + } + mock_msgdrivefs.assert_called_once_with( + drive_id="storage_drive_id", oauth2_client_params=expected_oauth2_params + ) + assert result == mock_fs_instance + + @patch("airflow.providers.microsoft.azure.fs.msgraph.BaseHook.get_connection") + @patch("msgraphfs.MSGDriveFS") + def test_get_fs_incomplete_credentials(self, mock_msgdrivefs, mock_get_connection): + # Connection with missing client_secret + connection = Connection( + conn_id="msgraph_incomplete", + conn_type="msgraph", + login="test_client_id", + host="test_tenant_id", + ) + mock_get_connection.return_value = connection + mock_fs_instance = MagicMock() + mock_msgdrivefs.return_value = mock_fs_instance + + result = get_fs("msgraph_incomplete") + + # Should return default filesystem when credentials are incomplete + mock_msgdrivefs.assert_called_once_with(drive_id=None, oauth2_client_params={}) + assert result == mock_fs_instance From b107cdf731c297683aa6549b5a308207d67d21dc Mon Sep 17 00:00:00 2001 From: Kalyan R Date: Sun, 5 Oct 2025 02:05:17 +0530 Subject: [PATCH 203/338] include task instance id in log printed by supervisor (#56383) * include task instance id in log printed by supervisor * fix test --- task-sdk/src/airflow/sdk/execution_time/supervisor.py | 8 +++++++- task-sdk/tests/task_sdk/execution_time/test_supervisor.py | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/task-sdk/src/airflow/sdk/execution_time/supervisor.py b/task-sdk/src/airflow/sdk/execution_time/supervisor.py index 01e3094cb5929..55110da616705 100644 --- a/task-sdk/src/airflow/sdk/execution_time/supervisor.py +++ b/task-sdk/src/airflow/sdk/execution_time/supervisor.py @@ -1896,7 +1896,13 @@ def supervise( exit_code = process.wait() end = time.monotonic() - log.info("Task finished", exit_code=exit_code, duration=end - start, final_state=process.final_state) + log.info( + "Task finished", + task_instance_id=str(ti.id), + exit_code=exit_code, + duration=end - start, + final_state=process.final_state, + ) if log_path and log_file_descriptor: log_file_descriptor.close() return exit_code 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 8a5e06e0e8445..107cf46e49e97 100644 --- a/task-sdk/tests/task_sdk/execution_time/test_supervisor.py +++ b/task-sdk/tests/task_sdk/execution_time/test_supervisor.py @@ -667,6 +667,7 @@ def test_supervise_handles_deferred_task( "level": "info", "logger": "supervisor", "loc": mocker.ANY, + "task_instance_id": str(ti.id), } in captured_logs def test_supervisor_handles_already_running_task(self): From 6ee540f1771abdb096a5d1ed05ffcb9265b047f7 Mon Sep 17 00:00:00 2001 From: Anuj Arora <43067216+anujarora0502@users.noreply.github.com> Date: Sun, 5 Oct 2025 12:07:48 +0530 Subject: [PATCH 204/338] replace start-airflow with stop_airflow (#56390) --- contributing-docs/03a_contributors_quick_start_beginners.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contributing-docs/03a_contributors_quick_start_beginners.rst b/contributing-docs/03a_contributors_quick_start_beginners.rst index 9214ba8fc62e5..11a2080238d26 100644 --- a/contributing-docs/03a_contributors_quick_start_beginners.rst +++ b/contributing-docs/03a_contributors_quick_start_beginners.rst @@ -84,7 +84,7 @@ Option A – Breeze on Your Laptop The command starts a shell and launches multiple terminals using tmux and launches all Airflow necessary components in those terminals. To know more about tmux commands, -check out this cheat sheet: https://tmuxcheatsheet.com/. To exit breeze, type ``start-airflow`` in any +check out this cheat sheet: https://tmuxcheatsheet.com/. To exit breeze, type ``stop_airflow`` in any of the tmux panes and hit Enter. **Working with DAGs in Breeze:** From f4a1ae84794982c8b00c724a9833e82c74b52ba0 Mon Sep 17 00:00:00 2001 From: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> Date: Sun, 5 Oct 2025 10:07:21 +0200 Subject: [PATCH 205/338] Make Dag Run ID visible in Dag Header Card (#56392) --- .../src/airflow/ui/src/layouts/Details/DagBreadcrumb.tsx | 3 +-- airflow-core/src/airflow/ui/src/pages/Run/Header.tsx | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/airflow-core/src/airflow/ui/src/layouts/Details/DagBreadcrumb.tsx b/airflow-core/src/airflow/ui/src/layouts/Details/DagBreadcrumb.tsx index c93475ecbfdf9..d877b4daa1b9b 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Details/DagBreadcrumb.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Details/DagBreadcrumb.tsx @@ -28,7 +28,6 @@ import { } from "openapi/queries"; import { BreadcrumbStats } from "src/components/BreadcrumbStats"; import { StateBadge } from "src/components/StateBadge"; -import Time from "src/components/Time"; import { TogglePause } from "src/components/TogglePause"; import { isStatePending, useAutoRefresh } from "src/utils"; @@ -82,7 +81,7 @@ export const DagBreadcrumb = () => { // Add dag run breadcrumb if (runId !== undefined) { links.push({ - label: dagRun === undefined ? runId : ); From 413e82b9c64d6170996386b96521c7151e7af429 Mon Sep 17 00:00:00 2001 From: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> Date: Sun, 5 Oct 2025 10:28:52 +0200 Subject: [PATCH 206/338] Use Task Display Name in Grid if existing (#56393) * Use Task Display Name in Grid if existing * Consider short task ID w/o prefix if no display name * Add some tests with labels on tasks * Apply ruff --- .../api_fastapi/core_api/routes/ui/grid.py | 8 ++++---- .../core_api/services/ui/task_group.py | 9 ++++++--- .../ui/src/pages/TaskInstance/Header.tsx | 1 - .../core_api/routes/ui/test_grid.py | 20 ++++++++++--------- 4 files changed, 21 insertions(+), 17 deletions(-) diff --git a/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/grid.py b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/grid.py index 9372fc798d484..de6eac73a9b65 100644 --- a/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/grid.py +++ b/airflow-core/src/airflow/api_fastapi/core_api/routes/ui/grid.py @@ -196,18 +196,18 @@ def _collect_ids(nodes: list[dict[str, Any]]) -> set[str]: return ids existing_ids = _collect_ids(merged_nodes) - historical_task_ids = session.scalars( - select(TaskInstance.task_id) + historical_tasks = session.execute( + select(TaskInstance.task_id, TaskInstance.task_display_name) .join(TaskInstance.dag_run) .where(TaskInstance.dag_id == dag_id, DagRun.id.in_(run_ids)) .distinct() ) - for task_id in historical_task_ids: + for task_id, task_display_name in historical_tasks: if task_id not in existing_ids: merged_nodes.append( { "id": task_id, - "label": task_id, + "label": task_display_name, "is_mapped": None, "children": None, } diff --git a/airflow-core/src/airflow/api_fastapi/core_api/services/ui/task_group.py b/airflow-core/src/airflow/api_fastapi/core_api/services/ui/task_group.py index ed9a96718e9c6..3b71ea1cff0a9 100644 --- a/airflow-core/src/airflow/api_fastapi/core_api/services/ui/task_group.py +++ b/airflow-core/src/airflow/api_fastapi/core_api/services/ui/task_group.py @@ -25,6 +25,7 @@ from airflow.configuration import conf from airflow.models.mappedoperator import MappedOperator, is_mapped +from airflow.sdk import TaskGroup from airflow.serialization.serialized_objects import SerializedBaseOperator @@ -90,15 +91,17 @@ def task_group_to_dict_grid(task_item_or_group, parent_group_is_mapped=False): setup_teardown_type = "setup" elif task.is_teardown is True: setup_teardown_type = "teardown" + # we explicitly want the short task ID here, not the full doted notation if in a group + task_display_name = task.task_display_name if task.task_display_name != task.task_id else task.label return { "id": task.task_id, - "label": task.label, + "label": task_display_name, "is_mapped": mapped, "children": None, "setup_teardown_type": setup_teardown_type, } - task_group = task_item_or_group + task_group: TaskGroup = task_item_or_group task_group_sort = get_task_group_children_getter() mapped = is_mapped(task_group) children = [ @@ -108,7 +111,7 @@ def task_group_to_dict_grid(task_item_or_group, parent_group_is_mapped=False): return { "id": task_group.group_id, - "label": task_group.label, + "label": task_group.group_display_name or task_group.label, "is_mapped": mapped or None, "children": children or None, } diff --git a/airflow-core/src/airflow/ui/src/pages/TaskInstance/Header.tsx b/airflow-core/src/airflow/ui/src/pages/TaskInstance/Header.tsx index 37349d8f030e5..0698baeaddc4c 100644 --- a/airflow-core/src/airflow/ui/src/pages/TaskInstance/Header.tsx +++ b/airflow-core/src/airflow/ui/src/pages/TaskInstance/Header.tsx @@ -125,7 +125,6 @@ export const Header = ({ isRefreshing={isRefreshing} state={taskInstance.state} stats={stats} - subTitle={ diff --git a/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_grid.py b/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_grid.py index 90ee06e4e0aae..86d8596f1cafe 100644 --- a/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_grid.py +++ b/airflow-core/tests/unit/api_fastapi/core_api/routes/ui/test_grid.py @@ -84,7 +84,7 @@ "is_mapped": True, "label": "mapped_task_group", }, - {"id": "task", "label": "task"}, + {"id": "task", "label": "A Beautiful Task Name 🚀"}, { "children": [ { @@ -92,11 +92,11 @@ { "id": "task_group.inner_task_group.inner_task_group_sub_task", "is_mapped": True, - "label": "inner_task_group_sub_task", + "label": "Inner Task Group Sub Task Label", } ], "id": "task_group.inner_task_group", - "label": "inner_task_group", + "label": "My Inner Task Group", }, {"id": "task_group.mapped_task", "is_mapped": True, "label": "mapped_task"}, ], @@ -121,7 +121,7 @@ def setup(dag_maker, session=None): # DAG 1 with dag_maker(dag_id=DAG_ID, serialized=True, session=session) as dag: - task = EmptyOperator(task_id=TASK_ID) + task = EmptyOperator(task_id=TASK_ID, task_display_name="A Beautiful Task Name 🚀") @task_group def mapped_task_group(arg1): @@ -131,8 +131,10 @@ def mapped_task_group(arg1): with TaskGroup(group_id=TASK_GROUP_ID): MockOperator.partial(task_id=MAPPED_TASK_ID).expand(arg1=["a", "b", "c", "d"]) - with TaskGroup(group_id=INNER_TASK_GROUP): - MockOperator.partial(task_id=INNER_TASK_GROUP_SUB_TASK).expand(arg1=["a", "b"]) + with TaskGroup(group_id=INNER_TASK_GROUP, group_display_name="My Inner Task Group"): + MockOperator.partial( + task_id=INNER_TASK_GROUP_SUB_TASK, task_display_name="Inner Task Group Sub Task Label" + ).expand(arg1=["a", "b"]) # Mapped but never expanded. API should not crash, but count this as one no-status ti. MockOperator.partial(task_id=MAPPED_TASK_ID_2).expand(arg1=task.output) @@ -480,7 +482,7 @@ def test_get_dag_structure(self, session, test_client): "is_mapped": True, "label": "mapped_task_group", }, - {"id": "task", "label": "task"}, + {"id": "task", "label": "A Beautiful Task Name 🚀"}, { "children": [ { @@ -488,11 +490,11 @@ def test_get_dag_structure(self, session, test_client): { "id": "task_group.inner_task_group.inner_task_group_sub_task", "is_mapped": True, - "label": "inner_task_group_sub_task", + "label": "Inner Task Group Sub Task Label", } ], "id": "task_group.inner_task_group", - "label": "inner_task_group", + "label": "My Inner Task Group", }, {"id": "task_group.mapped_task", "is_mapped": True, "label": "mapped_task"}, ], From de3bdd4d4e833070d634bc91fa7079db69017dca Mon Sep 17 00:00:00 2001 From: "LI,JHE-CHEN" <103923510+RoyLee1224@users.noreply.github.com> Date: Sun, 5 Oct 2025 14:10:00 -0400 Subject: [PATCH 207/338] fix: show appropriate time units in grid view (#56403) * fix: show appropriate time units in grid view * i18n: remove second translation keys --- .../ui/public/i18n/locales/ar/common.json | 1 - .../ui/public/i18n/locales/ca/common.json | 1 - .../ui/public/i18n/locales/de/common.json | 1 - .../ui/public/i18n/locales/en/common.json | 1 - .../ui/public/i18n/locales/es/common.json | 1 - .../ui/public/i18n/locales/fr/common.json | 1 - .../ui/public/i18n/locales/he/common.json | 1 - .../ui/public/i18n/locales/hi/common.json | 1 - .../ui/public/i18n/locales/hu/common.json | 1 - .../ui/public/i18n/locales/it/common.json | 1 - .../ui/public/i18n/locales/ko/common.json | 1 - .../ui/public/i18n/locales/nl/common.json | 1 - .../ui/public/i18n/locales/pl/common.json | 1 - .../ui/public/i18n/locales/pt/common.json | 1 - .../ui/public/i18n/locales/tr/common.json | 1 - .../ui/public/i18n/locales/zh-CN/common.json | 1 - .../ui/public/i18n/locales/zh-TW/common.json | 1 - .../src/layouts/Details/Grid/DurationTick.tsx | 17 +++++++---------- 18 files changed, 7 insertions(+), 27 deletions(-) diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/ar/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/ar/common.json index 847ee8fb906d2..db99947efc813 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/ar/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/ar/common.json @@ -189,7 +189,6 @@ }, "tooltip": "اضغط {{hotkey}} للتمرير إلى {{direction}}" }, - "seconds": "ثواني", "security": { "actions": "إجراءات", "permissions": "صلاحيات", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/ca/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/ca/common.json index b1d3bd4d0f141..7edcd6ae1b610 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/ca/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/ca/common.json @@ -161,7 +161,6 @@ }, "tooltip": "Prem {{hotkey}} per desplaçar-te cap a {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Accions", "permissions": "Permisos", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json index 805e94a81afb3..fe14e8e0d6713 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json @@ -161,7 +161,6 @@ }, "tooltip": "Tastenkombination {{hotkey}} zum scrollen nach {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Aktionen", "permissions": "Berechtigungen", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/en/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/en/common.json index 8622fb6a0fc7e..ac440354e9f9f 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/en/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/en/common.json @@ -163,7 +163,6 @@ }, "tooltip": "Press {{hotkey}} to scroll to {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Actions", "permissions": "Permissions", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/es/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/es/common.json index 161a43f365dd8..9424d92f62d39 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/es/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/es/common.json @@ -168,7 +168,6 @@ }, "tooltip": "Presiona {{hotkey}} para desplazarte a {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Acciones", "permissions": "Permisos", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/fr/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/fr/common.json index e8020f8d9c2ae..fe2002b5c0f68 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/fr/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/fr/common.json @@ -168,7 +168,6 @@ }, "tooltip": "Appuyez sur {{hotkey}} pour faire défiler vers le {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Actions", "permissions": "Permissions", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/he/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/he/common.json index 50c60812b33e8..4df731bdc5dfe 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/he/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/he/common.json @@ -168,7 +168,6 @@ }, "tooltip": "לחץ {{hotkey}} לגלילה ל{{direction}}" }, - "seconds": "{{count}} שניות", "security": { "actions": "פעולות", "permissions": "הרשאות", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/hi/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/hi/common.json index bac042e2a173a..a506b9198b85c 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/hi/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/hi/common.json @@ -154,7 +154,6 @@ }, "tooltip": "{{direction}} तक स्क्रॉल करने के लिए {{hotkey}} दबाएं" }, - "seconds": "{{count}}सेकंड", "security": { "actions": "क्रियाएं", "permissions": "अनुमतियां", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/hu/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/hu/common.json index 779a5b914b9c6..f26a1540f08ac 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/hu/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/hu/common.json @@ -154,7 +154,6 @@ }, "tooltip": "Nyomd meg a(z) {{hotkey}} gombot a görgetéshez ide: {{direction}}" }, - "seconds": "{{count}} mp", "security": { "actions": "Műveletek", "permissions": "Jogosultságok", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/it/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/it/common.json index 94f2635c549c6..306359ac1068f 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/it/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/it/common.json @@ -175,7 +175,6 @@ }, "tooltip": "Premi {{hotkey}} per scorrere a {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Azioni", "permissions": "Permessi", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/ko/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/ko/common.json index f33c7b8ae45e9..a909e2495ef4f 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/ko/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/ko/common.json @@ -161,7 +161,6 @@ }, "tooltip": "{{hotkey}}를 눌러 {{direction}}로 스크롤" }, - "seconds": "{{count}}초", "security": { "actions": "작업", "permissions": "권한", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/nl/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/nl/common.json index 3600c32deae78..04136a9510fbf 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/nl/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/nl/common.json @@ -161,7 +161,6 @@ }, "tooltip": "Druk op {{hotkey}} om te scrollen naar {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Acties", "permissions": "Permissies", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json index 629aa2dd3ef98..eaafe9c4b5a6b 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/pl/common.json @@ -177,7 +177,6 @@ }, "tooltip": "Naciśnij {{hotkey}}, aby przewinąć do {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Akcje", "permissions": "Uprawnienia", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/pt/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/pt/common.json index 5422349a406f7..8ab097b993ce4 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/pt/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/pt/common.json @@ -175,7 +175,6 @@ }, "tooltip": "Pressione {{hotkey}} para rolar para {{direction}}" }, - "seconds": "{{count}}s", "security": { "actions": "Ações", "permissions": "Permissões", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/tr/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/tr/common.json index a750582177b42..5d96412a92ed9 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/tr/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/tr/common.json @@ -161,7 +161,6 @@ }, "tooltip": "{{direction}} kaydırmak için {{hotkey}} tuşuna basın" }, - "seconds": "{{count}}sn", "security": { "actions": "Eylemler", "permissions": "İzinler", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/zh-CN/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/zh-CN/common.json index 2f5d89306914e..eef029cc5aeaf 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/zh-CN/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/zh-CN/common.json @@ -161,7 +161,6 @@ }, "tooltip": "按 {{hotkey}} 滚动到{{direction}}" }, - "seconds": "{{count}} 秒", "security": { "actions": "操作", "permissions": "权限", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/zh-TW/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/zh-TW/common.json index b47c0d8761299..3a69da77f9743 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/zh-TW/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/zh-TW/common.json @@ -161,7 +161,6 @@ }, "tooltip": "按 {{hotkey}} 捲動到{{direction}}" }, - "seconds": "{{count}} 秒", "security": { "actions": "操作", "permissions": "權限", diff --git a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/DurationTick.tsx b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/DurationTick.tsx index 199dc185d0791..7b766264a60e9 100644 --- a/airflow-core/src/airflow/ui/src/layouts/Details/Grid/DurationTick.tsx +++ b/airflow-core/src/airflow/ui/src/layouts/Details/Grid/DurationTick.tsx @@ -17,18 +17,15 @@ * under the License. */ import { Text, type TextProps } from "@chakra-ui/react"; -import { useTranslation } from "react-i18next"; + +import { renderDuration } from "src/utils"; type Props = { readonly duration: number; } & TextProps; -export const DurationTick = ({ duration, ...rest }: Props) => { - const { t: translate } = useTranslation(); - - return ( - - {translate("seconds", { count: Math.floor(duration) })} - - ); -}; +export const DurationTick = ({ duration, ...rest }: Props) => ( + + {renderDuration(duration)} + +); From a47cb1d90dfc18fc03368c8830a13513693d7e20 Mon Sep 17 00:00:00 2001 From: Satish Chinthanippu Date: Sun, 5 Oct 2025 11:38:06 -0700 Subject: [PATCH 208/338] Build correct SQLAlchemy URI in TeradataHook (#56305) --- .../providers/teradata/hooks/teradata.py | 37 +++++++++++---- .../unit/teradata/hooks/test_teradata.py | 46 +++++++++++-------- 2 files changed, 55 insertions(+), 28 deletions(-) diff --git a/providers/teradata/src/airflow/providers/teradata/hooks/teradata.py b/providers/teradata/src/airflow/providers/teradata/hooks/teradata.py index 4f0d90e4a298e..866a51db18f8c 100644 --- a/providers/teradata/src/airflow/providers/teradata/hooks/teradata.py +++ b/providers/teradata/src/airflow/providers/teradata/hooks/teradata.py @@ -22,8 +22,8 @@ import re from typing import TYPE_CHECKING, Any -import sqlalchemy import teradatasql +from sqlalchemy.engine import URL from teradatasql import TeradataConnection from airflow.providers.common.sql.hooks.sql import DbApiHook @@ -34,6 +34,7 @@ except ImportError: from airflow.models.connection import Connection # type: ignore[assignment] +DEFAULT_DB_PORT = 1025 PARAM_TYPES = {bool, float, int, str} @@ -166,7 +167,7 @@ def _get_conn_config_teradatasql(self) -> dict[str, Any]: conn: Connection = self.get_connection(self.get_conn_id()) conn_config = { "host": conn.host or "localhost", - "dbs_port": conn.port or "1025", + "dbs_port": conn.port or DEFAULT_DB_PORT, "database": conn.schema or "", "user": conn.login or "dbc", "password": conn.password or "dbc", @@ -195,12 +196,32 @@ def _get_conn_config_teradatasql(self) -> dict[str, Any]: return conn_config - def get_sqlalchemy_engine(self, engine_kwargs=None): - """Return a connection object using sqlalchemy.""" - conn: Connection = self.get_connection(self.get_conn_id()) - link = f"teradatasql://{conn.login}:{conn.password}@{conn.host}" - connection = sqlalchemy.create_engine(link) - return connection + @property + def sqlalchemy_url(self) -> URL: + """ + Override to return a Sqlalchemy.engine.URL object from the Teradata connection. + + :return: the extracted sqlalchemy.engine.URL object. + """ + connection = self.get_connection(self.get_conn_id()) + # Adding only teradatasqlalchemy supported connection parameters. + # https://pypi.org/project/teradatasqlalchemy/#ConnectionParameters + url_kwargs = { + "drivername": "teradatasql", + "username": connection.login, + "password": connection.password, + "host": connection.host, + "port": connection.port, + } + + if connection.schema: # Only include database if it's not None or empty + url_kwargs["database"] = connection.schema + + return URL.create(**url_kwargs) + + def get_uri(self) -> str: + """Override DbApiHook get_uri method for get_sqlalchemy_engine().""" + return self.sqlalchemy_url.render_as_string() @staticmethod def get_ui_field_behaviour() -> dict: diff --git a/providers/teradata/tests/unit/teradata/hooks/test_teradata.py b/providers/teradata/tests/unit/teradata/hooks/test_teradata.py index f9a38e0d607f0..f10c1e629d211 100644 --- a/providers/teradata/tests/unit/teradata/hooks/test_teradata.py +++ b/providers/teradata/tests/unit/teradata/hooks/test_teradata.py @@ -40,6 +40,11 @@ def setup_method(self): self.db_hook.get_connection.return_value = self.connection self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() + self.conn.login = "mock_login" + self.conn.password = "mock_password" + self.conn.host = "mock_host" + self.conn.schema = "mock_schema" + self.conn.port = 1025 self.conn.cursor.return_value = self.cur self.conn.extra_dejson = {} conn = self.conn @@ -53,6 +58,7 @@ def get_connection(cls, conn_id: str) -> Connection: return conn self.test_db_hook = UnitTestTeradataHook(teradata_conn_id="teradata_conn_id") + self.test_db_hook.get_uri = mock.Mock(return_value="sqlite://") @mock.patch("teradatasql.connect") def test_get_conn(self, mock_connect): @@ -62,7 +68,7 @@ def test_get_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" @@ -76,7 +82,7 @@ def test_get_tmode_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["tmode"] == "tera" @@ -91,7 +97,7 @@ def test_get_sslmode_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslmode"] == "require" @@ -106,7 +112,7 @@ def test_get_sslverifyca_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslmode"] == "verify-ca" @@ -122,7 +128,7 @@ def test_get_sslverifyfull_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslmode"] == "verify-full" @@ -138,7 +144,7 @@ def test_get_sslcrc_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslcrc"] == "sslcrc" @@ -153,7 +159,7 @@ def test_get_sslprotocol_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslprotocol"] == "protocol" @@ -168,25 +174,25 @@ def test_get_sslcipher_conn(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert kwargs["sslcipher"] == "cipher" - @mock.patch("sqlalchemy.create_engine") - def test_get_sqlalchemy_conn(self, mock_connect): - self.db_hook.get_sqlalchemy_engine() - assert mock_connect.call_count == 1 - args = mock_connect.call_args.args - assert len(args) == 1 - expected_link = ( - f"teradatasql://{self.connection.login}:{self.connection.password}@{self.connection.host}" - ) - assert expected_link == args[0] + def test_get_uri_without_schema(self): + self.connection.schema = "" # simulate missing schema + self.db_hook.get_connection.return_value = self.connection + uri = self.db_hook.get_uri() + expected_uri = f"teradatasql://{self.connection.login}:***@{self.connection.host}" + assert uri == expected_uri def test_get_uri(self): ret_uri = self.db_hook.get_uri() - expected_uri = f"teradata://{self.connection.login}:{self.connection.password}@{self.connection.host}/{self.connection.schema}" + expected_uri = ( + f"teradatasql://{self.connection.login}:***@{self.connection.host}/{self.connection.schema}" + if self.connection.schema + else f"teradatasql://{self.connection.login}:***@{self.connection.host}" + ) assert expected_uri == ret_uri def test_get_records(self): @@ -260,7 +266,7 @@ def test_query_band_not_in_conn_config(self, mock_connect): assert args == () assert kwargs["host"] == "host" assert kwargs["database"] == "schema" - assert kwargs["dbs_port"] == "1025" + assert kwargs["dbs_port"] == 1025 assert kwargs["user"] == "login" assert kwargs["password"] == "password" assert "query_band" not in kwargs From 10f1eccb0bfb6636e788cca257483d042716627c Mon Sep 17 00:00:00 2001 From: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> Date: Sun, 5 Oct 2025 21:54:33 +0200 Subject: [PATCH 209/338] Close German translation gaps for full UI translation 2025-10-05 (#56415) --- airflow-core/src/airflow/ui/public/i18n/locales/de/common.json | 2 ++ airflow-core/src/airflow/ui/public/i18n/locales/de/dag.json | 2 ++ 2 files changed, 4 insertions(+) diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json b/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json index fe14e8e0d6713..e3a4ddda50284 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/de/common.json @@ -101,6 +101,8 @@ }, "filter": "Filter", "filters": { + "durationFrom": "Laufzeit Von", + "durationTo": "Laufzeit Bis", "logicalDateFrom": "Logisches Datum Von", "logicalDateTo": "Logisches Datum Bis", "runAfterFrom": "Gelaufen-nach von filtern", diff --git a/airflow-core/src/airflow/ui/public/i18n/locales/de/dag.json b/airflow-core/src/airflow/ui/public/i18n/locales/de/dag.json index 7f0b2c603bb99..1803e2ea630aa 100644 --- a/airflow-core/src/airflow/ui/public/i18n/locales/de/dag.json +++ b/airflow-core/src/airflow/ui/public/i18n/locales/de/dag.json @@ -10,6 +10,7 @@ "hourly": "Stündlich", "legend": { "less": "Weniger", + "mixed": "Mittel", "more": "Mehr" }, "navigation": { @@ -19,6 +20,7 @@ "previousYear": "Vorheriges Jahr" }, "noData": "Keine Daten verfügbar", + "noFailedRuns": "Keine fehlgeschlagenen Läufe", "noRuns": "Keine Läufe", "totalRuns": "Gesamtzahl der Läufe", "week": "Kalenderwoche {{weekNumber}}", From e0286c64257b862ace546b714bd163f862641619 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Mon, 6 Oct 2025 11:44:30 +0530 Subject: [PATCH 210/338] Limit astroid to < 4 to fix broken docs builds (#56419) --- devel-common/pyproject.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/devel-common/pyproject.toml b/devel-common/pyproject.toml index 8b7b21c9e8270..2808f3c58f1e6 100644 --- a/devel-common/pyproject.toml +++ b/devel-common/pyproject.toml @@ -67,7 +67,8 @@ dependencies = [ "twine>=4.0.2", ] "docs" = [ - "astroid>=3", + # Astroid 4 released 5 Oct 2025 breaks autoapi https://github.com/apache/airflow/issues/56420 + "astroid>=3,<4", "checksumdir>=1.2.0", "rich-click>=1.7.1", "click>=8.1.8", From 106bc00960b93091b68574295d561429977a4682 Mon Sep 17 00:00:00 2001 From: uti Date: Mon, 6 Oct 2025 08:25:16 +0100 Subject: [PATCH 211/338] Werkzeug Version Check (#56398) * Added feature detection in werkzeug fab providers to allow detecting hashing algorithms available to different versions of werkzeug * Using Version check as done by the upstream FAB provider * Removing used var * Adding missing importlib import --- .../fab/auth_manager/security_manager/override.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/providers/fab/src/airflow/providers/fab/auth_manager/security_manager/override.py b/providers/fab/src/airflow/providers/fab/auth_manager/security_manager/override.py index b9cfa969b7d73..19085c835acb5 100644 --- a/providers/fab/src/airflow/providers/fab/auth_manager/security_manager/override.py +++ b/providers/fab/src/airflow/providers/fab/auth_manager/security_manager/override.py @@ -19,6 +19,7 @@ import copy import datetime +import importlib import itertools import logging import uuid @@ -59,6 +60,7 @@ from flask_login import LoginManager from itsdangerous import want_bytes from markupsafe import Markup, escape +from packaging.version import Version from sqlalchemy import delete, func, inspect, or_, select from sqlalchemy.exc import MultipleResultsFound from sqlalchemy.orm import joinedload @@ -790,10 +792,7 @@ def _init_config(self): current_app.config.setdefault("AUTH_ROLES_SYNC_AT_LOGIN", False) current_app.config.setdefault("AUTH_API_LOGIN_ALLOW_MULTIPLE_PROVIDERS", False) - from packaging.version import Version - from werkzeug import __version__ as werkzeug_version - - parsed_werkzeug_version = Version(werkzeug_version) + parsed_werkzeug_version = Version(importlib.metadata.version("werkzeug")) if parsed_werkzeug_version < Version("3.0.0"): current_app.config.setdefault("FAB_PASSWORD_HASH_METHOD", "pbkdf2:sha256") current_app.config.setdefault( From 4eb560475b70b3c3766e66c5b91aa000ddc4aa77 Mon Sep 17 00:00:00 2001 From: Deji Ibrahim <31637316+dejii@users.noreply.github.com> Date: Mon, 6 Oct 2025 10:43:35 +0100 Subject: [PATCH 212/338] AIP-82: implement Google Pub/Sub message queue provider (#54494) * AIP-82: implement Google Pub/Sub message queue provider * fix: mypy checks * fix: spell checks * fix: tests * switch to scheme based configuration * add system tests, update docs --- dev/breeze/tests/test_selective_checks.py | 8 +- providers/google/docs/index.rst | 1 + .../google/docs/message-queues/index.rst | 70 ++++++++++++++ providers/google/provider.yaml | 3 + providers/google/pyproject.toml | 4 + .../providers/google/cloud/queues/__init__.py | 16 ++++ .../providers/google/cloud/queues/pubsub.py | 68 ++++++++++++++ .../providers/google/get_provider_info.py | 1 + .../example_pubsub_message_queue_trigger.py | 93 +++++++++++++++++++ .../unit/google/cloud/queues/__init__.py | 16 ++++ .../unit/google/cloud/queues/test_pubsub.py | 45 +++++++++ 11 files changed, 321 insertions(+), 4 deletions(-) create mode 100644 providers/google/docs/message-queues/index.rst create mode 100644 providers/google/src/airflow/providers/google/cloud/queues/__init__.py create mode 100644 providers/google/src/airflow/providers/google/cloud/queues/pubsub.py create mode 100644 providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py create mode 100644 providers/google/tests/unit/google/cloud/queues/__init__.py create mode 100644 providers/google/tests/unit/google/cloud/queues/test_pubsub.py diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 5c00a026921b3..f02532312799a 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -1847,7 +1847,7 @@ def test_expected_output_push( ), { "selected-providers-list-as-string": "amazon apache.beam apache.cassandra apache.kafka " - "cncf.kubernetes common.compat common.sql " + "cncf.kubernetes common.compat common.messaging common.sql " "facebook google hashicorp http microsoft.azure microsoft.mssql mysql " "openlineage oracle postgres presto salesforce samba sftp ssh trino", "all-python-versions": f"['{DEFAULT_PYTHON_MAJOR_MINOR_VERSION}']", @@ -1859,7 +1859,7 @@ def test_expected_output_push( "skip-providers-tests": "false", "docs-build": "true", "docs-list-as-string": "apache-airflow helm-chart amazon apache.beam apache.cassandra " - "apache.kafka cncf.kubernetes common.compat common.sql facebook google hashicorp http microsoft.azure " + "apache.kafka cncf.kubernetes common.compat common.messaging common.sql facebook google hashicorp http microsoft.azure " "microsoft.mssql mysql openlineage oracle postgres " "presto salesforce samba sftp ssh trino", "skip-prek-hooks": ALL_SKIPPED_COMMITS_IF_NO_UI, @@ -1873,7 +1873,7 @@ def test_expected_output_push( { "description": "amazon...google", "test_types": "Providers[amazon] Providers[apache.beam,apache.cassandra," - "apache.kafka,cncf.kubernetes,common.compat,common.sql,facebook," + "apache.kafka,cncf.kubernetes,common.compat,common.messaging,common.sql,facebook," "hashicorp,http,microsoft.azure,microsoft.mssql,mysql," "openlineage,oracle,postgres,presto,salesforce,samba,sftp,ssh,trino] " "Providers[google]", @@ -2117,7 +2117,7 @@ def test_upgrade_to_newer_dependencies( ("providers/google/docs/some_file.rst",), { "docs-list-as-string": "amazon apache.beam apache.cassandra apache.kafka " - "cncf.kubernetes common.compat common.sql facebook google hashicorp http " + "cncf.kubernetes common.compat common.messaging common.sql facebook google hashicorp http " "microsoft.azure microsoft.mssql mysql openlineage oracle " "postgres presto salesforce samba sftp ssh trino", }, diff --git a/providers/google/docs/index.rst b/providers/google/docs/index.rst index 3ca89dba9530c..24db6cefe811f 100644 --- a/providers/google/docs/index.rst +++ b/providers/google/docs/index.rst @@ -36,6 +36,7 @@ Connection types Logging handlers + Message queues Secrets backends API Authentication backend Operators diff --git a/providers/google/docs/message-queues/index.rst b/providers/google/docs/message-queues/index.rst new file mode 100644 index 0000000000000..dcc30c5571322 --- /dev/null +++ b/providers/google/docs/message-queues/index.rst @@ -0,0 +1,70 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +Google Cloud Messaging Queues +============================== + +.. contents:: + :local: + :depth: 2 + +Google Cloud Pub/Sub Queue Provider +------------------------------------ + +Implemented by :class:`~airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider` + +The Google Cloud Pub/Sub Queue Provider is a message queue provider that uses Google Cloud Pub/Sub as the underlying message queue system. + +It allows you to send and receive messages using Cloud Pub/Sub in your Airflow workflows +with :class:`~airflow.providers.common.messaging.triggers.msg_queue.MessageQueueTrigger` common message queue interface. + +.. include:: /../src/airflow/providers/google/cloud/queues/pubsub.py + :start-after: [START pubsub_message_queue_provider_description] + :end-before: [END pubsub_message_queue_provider_description] + +Pub/Sub Message Queue Trigger +----------------------------- + +Implemented by :class:`~airflow.providers.google.cloud.triggers.pubsub.PubsubPullTrigger` + +Inherited from :class:`~airflow.providers.common.messaging.triggers.msg_queue.MessageQueueTrigger` + + +Wait for a message in a queue +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Below is an example of how you can configure an Airflow DAG to be triggered by a message in Pub/Sub. + +.. exampleinclude:: /../tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py + :language: python + :start-after: [START howto_trigger_pubsub_message_queue] + :end-before: [END howto_trigger_pubsub_message_queue] + +How it works +------------ + +1. **Pub/Sub Message Queue Trigger**: The ``PubsubPullTrigger`` listens for messages from a Google Cloud Pub/Sub subscription. + +2. **Asset and Watcher**: The ``Asset`` abstracts the external entity, the Pub/Sub subscription in this example. + The ``AssetWatcher`` associate a trigger with a name. This name helps you identify which trigger is associated to which + asset. + +3. **Event-Driven DAG**: Instead of running on a fixed schedule, the DAG executes when the asset receives an update + (e.g., a new message in the queue). + +For how to use the trigger, refer to the documentation of the +:ref:`Messaging Trigger ` diff --git a/providers/google/provider.yaml b/providers/google/provider.yaml index fc04622e4a4ca..6005f302c17e5 100644 --- a/providers/google/provider.yaml +++ b/providers/google/provider.yaml @@ -1257,3 +1257,6 @@ auth-backends: logging: - airflow.providers.google.cloud.log.gcs_task_handler.GCSTaskHandler - airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler + +queues: + - airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider diff --git a/providers/google/pyproject.toml b/providers/google/pyproject.toml index 0bb82701d12fc..e5f0e86acef82 100644 --- a/providers/google/pyproject.toml +++ b/providers/google/pyproject.toml @@ -203,6 +203,9 @@ dependencies = [ "http" = [ "apache-airflow-providers-http" ] +"common.messaging" = [ + "apache-airflow-providers-common-messaging" +] [dependency-groups] dev = [ @@ -214,6 +217,7 @@ dev = [ "apache-airflow-providers-apache-cassandra", "apache-airflow-providers-cncf-kubernetes", "apache-airflow-providers-common-compat", + "apache-airflow-providers-common-messaging", "apache-airflow-providers-common-sql", "apache-airflow-providers-facebook", "apache-airflow-providers-http", diff --git a/providers/google/src/airflow/providers/google/cloud/queues/__init__.py b/providers/google/src/airflow/providers/google/cloud/queues/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/providers/google/src/airflow/providers/google/cloud/queues/__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/google/src/airflow/providers/google/cloud/queues/pubsub.py b/providers/google/src/airflow/providers/google/cloud/queues/pubsub.py new file mode 100644 index 0000000000000..884cc4d21c1c3 --- /dev/null +++ b/providers/google/src/airflow/providers/google/cloud/queues/pubsub.py @@ -0,0 +1,68 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +from typing import TYPE_CHECKING + +from airflow.exceptions import AirflowOptionalProviderFeatureException +from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger + +try: + from airflow.providers.common.messaging.providers.base_provider import BaseMessageQueueProvider +except ImportError: + raise AirflowOptionalProviderFeatureException( + "This feature requires the 'common.messaging' provider to be installed in version >= 1.0.1." + ) + +if TYPE_CHECKING: + from airflow.triggers.base import BaseEventTrigger + + +class PubsubMessageQueueProvider(BaseMessageQueueProvider): + """ + Configuration for PubSub integration with common-messaging. + + [START pubsub_message_queue_provider_description] + * It uses ``google+pubsub`` as the scheme for identifying the provider. + * For parameter definitions, take a look at :class:`~airflow.providers.google.cloud.triggers.pubsub.PubsubPullTrigger`. + + .. code-block:: python + + from airflow.providers.common.messaging.triggers.msg_queue import MessageQueueTrigger + from airflow.sdk import Asset, AssetWatcher + + trigger = MessageQueueTrigger( + scheme="google+pubsub", + # Additional PubsubPullTrigger parameters as needed + project_id="my_project", + subscription="my_subscription", + ack_messages=True, + max_messages=1, + gcp_conn_id="google_cloud_default", + poke_interval=60.0, + ) + + asset = Asset("pubsub_queue_asset", watchers=[AssetWatcher(name="pubsub_watcher", trigger=trigger)]) + + [END pubsub_message_queue_provider_description] + + """ + + scheme = "google+pubsub" + + def trigger_class(self) -> type[BaseEventTrigger]: + return PubsubPullTrigger # type: ignore[return-value] diff --git a/providers/google/src/airflow/providers/google/get_provider_info.py b/providers/google/src/airflow/providers/google/get_provider_info.py index f79dcd19d70b6..72f747c36277d 100644 --- a/providers/google/src/airflow/providers/google/get_provider_info.py +++ b/providers/google/src/airflow/providers/google/get_provider_info.py @@ -1518,4 +1518,5 @@ def get_provider_info(): "airflow.providers.google.cloud.log.gcs_task_handler.GCSTaskHandler", "airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler", ], + "queues": ["airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider"], } diff --git a/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py b/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py new file mode 100644 index 0000000000000..e4d92db83ad82 --- /dev/null +++ b/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py @@ -0,0 +1,93 @@ +# 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. + +""" +Example Airflow DAG that demonstrates using Google Cloud Pub/Sub with MessageQueueTrigger +and Asset Watchers for event-driven workflows. + +This example shows how to create a DAG that triggers when messages arrive in a +Google Cloud Pub/Sub subscription using Asset Watchers. + +Prerequisites +------------- + +Before running this example, ensure you have: + +1. A GCP project with Pub/Sub API enabled +2. The following Pub/Sub resources created in your project: + + - Topic: ``test-topic`` + - Subscription: ``test-subscription`` + +You can create these resources using: + +.. code-block:: bash + + # Create topic + gcloud pubsub topics create test-topic --project={PROJECT_ID} + + # Create subscription + gcloud pubsub subscriptions create test-subscription \\ + --topic=test-topic --project={PROJECT_ID} + +How to test +----------- + +1. Ensure the Pub/Sub resources exist (see Prerequisites above) +2. Publish a message to trigger the DAG: + + .. code-block:: bash + + gcloud pubsub topics publish test-topic \\ + --message="Test message" --project={PROJECT_ID} + +3. The DAG will be triggered automatically when the message arrives +""" + +from __future__ import annotations + +# [START howto_trigger_pubsub_message_queue] +from airflow.providers.common.messaging.triggers.msg_queue import MessageQueueTrigger +from airflow.providers.standard.operators.empty import EmptyOperator +from airflow.sdk import DAG, Asset, AssetWatcher + +# Define a trigger that listens to a Google Cloud Pub/Sub subscription +trigger = MessageQueueTrigger( + scheme="google+pubsub", + project_id="my-project", + subscription="test-subscription", + ack_messages=True, + max_messages=1, + gcp_conn_id="google_cloud_default", + poke_interval=60.0, +) + +# Define an asset that watches for messages on the Pub/Sub subscription +asset = Asset("pubsub_queue_asset_1", watchers=[AssetWatcher(name="pubsub_watcher_1", trigger=trigger)]) + +with DAG( + dag_id="example_pubsub_message_queue_trigger", + schedule=[asset], +) as dag: + process_message_task = EmptyOperator(task_id="process_pubsub_message") +# [END howto_trigger_pubsub_message_queue] + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/google/tests/unit/google/cloud/queues/__init__.py b/providers/google/tests/unit/google/cloud/queues/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/providers/google/tests/unit/google/cloud/queues/__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/google/tests/unit/google/cloud/queues/test_pubsub.py b/providers/google/tests/unit/google/cloud/queues/test_pubsub.py new file mode 100644 index 0000000000000..4b80bae531b94 --- /dev/null +++ b/providers/google/tests/unit/google/cloud/queues/test_pubsub.py @@ -0,0 +1,45 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import pytest + +from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger + +pytest.importorskip("airflow.providers.common.messaging.providers.base_provider") + + +def test_message_pubsub_queue_create(): + from airflow.providers.common.messaging.providers.base_provider import BaseMessageQueueProvider + from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider + + provider = PubsubMessageQueueProvider() + assert isinstance(provider, BaseMessageQueueProvider) + + +def test_message_pubsub_queue_trigger_class(): + from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider + + provider = PubsubMessageQueueProvider() + assert provider.trigger_class() == PubsubPullTrigger + + +def test_scheme_matches(): + from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider + + provider = PubsubMessageQueueProvider() + assert provider.scheme_matches("google+pubsub") From af5350d4a20f33fc9a7db2648eb5d10aa71aaf3b Mon Sep 17 00:00:00 2001 From: LIU ZHE YOU <68415893+jason810496@users.noreply.github.com> Date: Mon, 6 Oct 2025 18:53:09 +0800 Subject: [PATCH 213/338] Revert "AIP-82: implement Google Pub/Sub message queue provider (#54494)" (#56423) --- dev/breeze/tests/test_selective_checks.py | 8 +- providers/google/docs/index.rst | 1 - .../google/docs/message-queues/index.rst | 70 -------------- providers/google/provider.yaml | 3 - providers/google/pyproject.toml | 4 - .../providers/google/cloud/queues/__init__.py | 16 ---- .../providers/google/cloud/queues/pubsub.py | 68 -------------- .../providers/google/get_provider_info.py | 1 - .../example_pubsub_message_queue_trigger.py | 93 ------------------- .../unit/google/cloud/queues/__init__.py | 16 ---- .../unit/google/cloud/queues/test_pubsub.py | 45 --------- 11 files changed, 4 insertions(+), 321 deletions(-) delete mode 100644 providers/google/docs/message-queues/index.rst delete mode 100644 providers/google/src/airflow/providers/google/cloud/queues/__init__.py delete mode 100644 providers/google/src/airflow/providers/google/cloud/queues/pubsub.py delete mode 100644 providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py delete mode 100644 providers/google/tests/unit/google/cloud/queues/__init__.py delete mode 100644 providers/google/tests/unit/google/cloud/queues/test_pubsub.py diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index f02532312799a..5c00a026921b3 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -1847,7 +1847,7 @@ def test_expected_output_push( ), { "selected-providers-list-as-string": "amazon apache.beam apache.cassandra apache.kafka " - "cncf.kubernetes common.compat common.messaging common.sql " + "cncf.kubernetes common.compat common.sql " "facebook google hashicorp http microsoft.azure microsoft.mssql mysql " "openlineage oracle postgres presto salesforce samba sftp ssh trino", "all-python-versions": f"['{DEFAULT_PYTHON_MAJOR_MINOR_VERSION}']", @@ -1859,7 +1859,7 @@ def test_expected_output_push( "skip-providers-tests": "false", "docs-build": "true", "docs-list-as-string": "apache-airflow helm-chart amazon apache.beam apache.cassandra " - "apache.kafka cncf.kubernetes common.compat common.messaging common.sql facebook google hashicorp http microsoft.azure " + "apache.kafka cncf.kubernetes common.compat common.sql facebook google hashicorp http microsoft.azure " "microsoft.mssql mysql openlineage oracle postgres " "presto salesforce samba sftp ssh trino", "skip-prek-hooks": ALL_SKIPPED_COMMITS_IF_NO_UI, @@ -1873,7 +1873,7 @@ def test_expected_output_push( { "description": "amazon...google", "test_types": "Providers[amazon] Providers[apache.beam,apache.cassandra," - "apache.kafka,cncf.kubernetes,common.compat,common.messaging,common.sql,facebook," + "apache.kafka,cncf.kubernetes,common.compat,common.sql,facebook," "hashicorp,http,microsoft.azure,microsoft.mssql,mysql," "openlineage,oracle,postgres,presto,salesforce,samba,sftp,ssh,trino] " "Providers[google]", @@ -2117,7 +2117,7 @@ def test_upgrade_to_newer_dependencies( ("providers/google/docs/some_file.rst",), { "docs-list-as-string": "amazon apache.beam apache.cassandra apache.kafka " - "cncf.kubernetes common.compat common.messaging common.sql facebook google hashicorp http " + "cncf.kubernetes common.compat common.sql facebook google hashicorp http " "microsoft.azure microsoft.mssql mysql openlineage oracle " "postgres presto salesforce samba sftp ssh trino", }, diff --git a/providers/google/docs/index.rst b/providers/google/docs/index.rst index 24db6cefe811f..3ca89dba9530c 100644 --- a/providers/google/docs/index.rst +++ b/providers/google/docs/index.rst @@ -36,7 +36,6 @@ Connection types Logging handlers - Message queues Secrets backends API Authentication backend Operators diff --git a/providers/google/docs/message-queues/index.rst b/providers/google/docs/message-queues/index.rst deleted file mode 100644 index dcc30c5571322..0000000000000 --- a/providers/google/docs/message-queues/index.rst +++ /dev/null @@ -1,70 +0,0 @@ - .. Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - .. http://www.apache.org/licenses/LICENSE-2.0 - - .. Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. - -Google Cloud Messaging Queues -============================== - -.. contents:: - :local: - :depth: 2 - -Google Cloud Pub/Sub Queue Provider ------------------------------------- - -Implemented by :class:`~airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider` - -The Google Cloud Pub/Sub Queue Provider is a message queue provider that uses Google Cloud Pub/Sub as the underlying message queue system. - -It allows you to send and receive messages using Cloud Pub/Sub in your Airflow workflows -with :class:`~airflow.providers.common.messaging.triggers.msg_queue.MessageQueueTrigger` common message queue interface. - -.. include:: /../src/airflow/providers/google/cloud/queues/pubsub.py - :start-after: [START pubsub_message_queue_provider_description] - :end-before: [END pubsub_message_queue_provider_description] - -Pub/Sub Message Queue Trigger ------------------------------ - -Implemented by :class:`~airflow.providers.google.cloud.triggers.pubsub.PubsubPullTrigger` - -Inherited from :class:`~airflow.providers.common.messaging.triggers.msg_queue.MessageQueueTrigger` - - -Wait for a message in a queue -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Below is an example of how you can configure an Airflow DAG to be triggered by a message in Pub/Sub. - -.. exampleinclude:: /../tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py - :language: python - :start-after: [START howto_trigger_pubsub_message_queue] - :end-before: [END howto_trigger_pubsub_message_queue] - -How it works ------------- - -1. **Pub/Sub Message Queue Trigger**: The ``PubsubPullTrigger`` listens for messages from a Google Cloud Pub/Sub subscription. - -2. **Asset and Watcher**: The ``Asset`` abstracts the external entity, the Pub/Sub subscription in this example. - The ``AssetWatcher`` associate a trigger with a name. This name helps you identify which trigger is associated to which - asset. - -3. **Event-Driven DAG**: Instead of running on a fixed schedule, the DAG executes when the asset receives an update - (e.g., a new message in the queue). - -For how to use the trigger, refer to the documentation of the -:ref:`Messaging Trigger ` diff --git a/providers/google/provider.yaml b/providers/google/provider.yaml index 6005f302c17e5..fc04622e4a4ca 100644 --- a/providers/google/provider.yaml +++ b/providers/google/provider.yaml @@ -1257,6 +1257,3 @@ auth-backends: logging: - airflow.providers.google.cloud.log.gcs_task_handler.GCSTaskHandler - airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler - -queues: - - airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider diff --git a/providers/google/pyproject.toml b/providers/google/pyproject.toml index e5f0e86acef82..0bb82701d12fc 100644 --- a/providers/google/pyproject.toml +++ b/providers/google/pyproject.toml @@ -203,9 +203,6 @@ dependencies = [ "http" = [ "apache-airflow-providers-http" ] -"common.messaging" = [ - "apache-airflow-providers-common-messaging" -] [dependency-groups] dev = [ @@ -217,7 +214,6 @@ dev = [ "apache-airflow-providers-apache-cassandra", "apache-airflow-providers-cncf-kubernetes", "apache-airflow-providers-common-compat", - "apache-airflow-providers-common-messaging", "apache-airflow-providers-common-sql", "apache-airflow-providers-facebook", "apache-airflow-providers-http", diff --git a/providers/google/src/airflow/providers/google/cloud/queues/__init__.py b/providers/google/src/airflow/providers/google/cloud/queues/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/providers/google/src/airflow/providers/google/cloud/queues/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/providers/google/src/airflow/providers/google/cloud/queues/pubsub.py b/providers/google/src/airflow/providers/google/cloud/queues/pubsub.py deleted file mode 100644 index 884cc4d21c1c3..0000000000000 --- a/providers/google/src/airflow/providers/google/cloud/queues/pubsub.py +++ /dev/null @@ -1,68 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from typing import TYPE_CHECKING - -from airflow.exceptions import AirflowOptionalProviderFeatureException -from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger - -try: - from airflow.providers.common.messaging.providers.base_provider import BaseMessageQueueProvider -except ImportError: - raise AirflowOptionalProviderFeatureException( - "This feature requires the 'common.messaging' provider to be installed in version >= 1.0.1." - ) - -if TYPE_CHECKING: - from airflow.triggers.base import BaseEventTrigger - - -class PubsubMessageQueueProvider(BaseMessageQueueProvider): - """ - Configuration for PubSub integration with common-messaging. - - [START pubsub_message_queue_provider_description] - * It uses ``google+pubsub`` as the scheme for identifying the provider. - * For parameter definitions, take a look at :class:`~airflow.providers.google.cloud.triggers.pubsub.PubsubPullTrigger`. - - .. code-block:: python - - from airflow.providers.common.messaging.triggers.msg_queue import MessageQueueTrigger - from airflow.sdk import Asset, AssetWatcher - - trigger = MessageQueueTrigger( - scheme="google+pubsub", - # Additional PubsubPullTrigger parameters as needed - project_id="my_project", - subscription="my_subscription", - ack_messages=True, - max_messages=1, - gcp_conn_id="google_cloud_default", - poke_interval=60.0, - ) - - asset = Asset("pubsub_queue_asset", watchers=[AssetWatcher(name="pubsub_watcher", trigger=trigger)]) - - [END pubsub_message_queue_provider_description] - - """ - - scheme = "google+pubsub" - - def trigger_class(self) -> type[BaseEventTrigger]: - return PubsubPullTrigger # type: ignore[return-value] diff --git a/providers/google/src/airflow/providers/google/get_provider_info.py b/providers/google/src/airflow/providers/google/get_provider_info.py index 72f747c36277d..f79dcd19d70b6 100644 --- a/providers/google/src/airflow/providers/google/get_provider_info.py +++ b/providers/google/src/airflow/providers/google/get_provider_info.py @@ -1518,5 +1518,4 @@ def get_provider_info(): "airflow.providers.google.cloud.log.gcs_task_handler.GCSTaskHandler", "airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler", ], - "queues": ["airflow.providers.google.cloud.queues.pubsub.PubsubMessageQueueProvider"], } diff --git a/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py b/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py deleted file mode 100644 index e4d92db83ad82..0000000000000 --- a/providers/google/tests/system/google/cloud/pubsub/example_pubsub_message_queue_trigger.py +++ /dev/null @@ -1,93 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -""" -Example Airflow DAG that demonstrates using Google Cloud Pub/Sub with MessageQueueTrigger -and Asset Watchers for event-driven workflows. - -This example shows how to create a DAG that triggers when messages arrive in a -Google Cloud Pub/Sub subscription using Asset Watchers. - -Prerequisites -------------- - -Before running this example, ensure you have: - -1. A GCP project with Pub/Sub API enabled -2. The following Pub/Sub resources created in your project: - - - Topic: ``test-topic`` - - Subscription: ``test-subscription`` - -You can create these resources using: - -.. code-block:: bash - - # Create topic - gcloud pubsub topics create test-topic --project={PROJECT_ID} - - # Create subscription - gcloud pubsub subscriptions create test-subscription \\ - --topic=test-topic --project={PROJECT_ID} - -How to test ------------ - -1. Ensure the Pub/Sub resources exist (see Prerequisites above) -2. Publish a message to trigger the DAG: - - .. code-block:: bash - - gcloud pubsub topics publish test-topic \\ - --message="Test message" --project={PROJECT_ID} - -3. The DAG will be triggered automatically when the message arrives -""" - -from __future__ import annotations - -# [START howto_trigger_pubsub_message_queue] -from airflow.providers.common.messaging.triggers.msg_queue import MessageQueueTrigger -from airflow.providers.standard.operators.empty import EmptyOperator -from airflow.sdk import DAG, Asset, AssetWatcher - -# Define a trigger that listens to a Google Cloud Pub/Sub subscription -trigger = MessageQueueTrigger( - scheme="google+pubsub", - project_id="my-project", - subscription="test-subscription", - ack_messages=True, - max_messages=1, - gcp_conn_id="google_cloud_default", - poke_interval=60.0, -) - -# Define an asset that watches for messages on the Pub/Sub subscription -asset = Asset("pubsub_queue_asset_1", watchers=[AssetWatcher(name="pubsub_watcher_1", trigger=trigger)]) - -with DAG( - dag_id="example_pubsub_message_queue_trigger", - schedule=[asset], -) as dag: - process_message_task = EmptyOperator(task_id="process_pubsub_message") -# [END howto_trigger_pubsub_message_queue] - - -from tests_common.test_utils.system_tests import get_test_run # noqa: E402 - -# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) -test_run = get_test_run(dag) diff --git a/providers/google/tests/unit/google/cloud/queues/__init__.py b/providers/google/tests/unit/google/cloud/queues/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/providers/google/tests/unit/google/cloud/queues/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/providers/google/tests/unit/google/cloud/queues/test_pubsub.py b/providers/google/tests/unit/google/cloud/queues/test_pubsub.py deleted file mode 100644 index 4b80bae531b94..0000000000000 --- a/providers/google/tests/unit/google/cloud/queues/test_pubsub.py +++ /dev/null @@ -1,45 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger - -pytest.importorskip("airflow.providers.common.messaging.providers.base_provider") - - -def test_message_pubsub_queue_create(): - from airflow.providers.common.messaging.providers.base_provider import BaseMessageQueueProvider - from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider - - provider = PubsubMessageQueueProvider() - assert isinstance(provider, BaseMessageQueueProvider) - - -def test_message_pubsub_queue_trigger_class(): - from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider - - provider = PubsubMessageQueueProvider() - assert provider.trigger_class() == PubsubPullTrigger - - -def test_scheme_matches(): - from airflow.providers.google.cloud.queues.pubsub import PubsubMessageQueueProvider - - provider = PubsubMessageQueueProvider() - assert provider.scheme_matches("google+pubsub") From 579046962a087fec3613b3fb569623e0f83632bb Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 6 Oct 2025 14:19:19 +0100 Subject: [PATCH 214/338] Fix scheduler crash with email notifications (#56429) The ``EmailNotificationRequest`` class name (25 characters) exceeded the database constraint for ``DbCallbackRequest.callback_type column`` (20 characters), causing scheduler crashes when email notifications were triggered for task failures or retries. This fix renames the class to ``EmailRequest`` (12 characters) to fit within the constraint. A backwards compatibility alias ensures existing DB entries with `'EmailNotificationRequest'` can still be deserialized via getattr lookup. The 20-character limit is arbitrary and does not affect performance. In a follow-up PR for 3.2, we should increase this to 50+ characters to accommodate descriptive class names without requiring abbreviations. Fixes #56426 --- .../airflow/callbacks/callback_requests.py | 9 +++++--- .../src/airflow/dag_processing/processor.py | 8 +++---- .../src/airflow/jobs/scheduler_job_runner.py | 4 ++-- .../unit/callbacks/test_callback_requests.py | 22 +++++++++---------- .../unit/dag_processing/test_processor.py | 10 ++++----- 5 files changed, 27 insertions(+), 26 deletions(-) diff --git a/airflow-core/src/airflow/callbacks/callback_requests.py b/airflow-core/src/airflow/callbacks/callback_requests.py index e0666b397c235..d2bdf0968bc20 100644 --- a/airflow-core/src/airflow/callbacks/callback_requests.py +++ b/airflow-core/src/airflow/callbacks/callback_requests.py @@ -77,7 +77,7 @@ def is_failure_callback(self) -> bool: } -class EmailNotificationRequest(BaseCallbackRequest): +class EmailRequest(BaseCallbackRequest): """Email notification request for task failures/retries.""" ti: ti_datamodel.TaskInstance @@ -86,7 +86,7 @@ class EmailNotificationRequest(BaseCallbackRequest): """Whether this is for a failure or retry email""" context_from_server: ti_datamodel.TIRunContext """Task execution context from the Server""" - type: Literal["EmailNotificationRequest"] = "EmailNotificationRequest" + type: Literal["EmailRequest"] = "EmailRequest" class DagRunContext(BaseModel): @@ -108,6 +108,9 @@ class DagCallbackRequest(BaseCallbackRequest): CallbackRequest = Annotated[ - DagCallbackRequest | TaskCallbackRequest | EmailNotificationRequest, + DagCallbackRequest | TaskCallbackRequest | EmailRequest, Field(discriminator="type"), ] + +# Backwards compatibility alias +EmailNotificationRequest = EmailRequest diff --git a/airflow-core/src/airflow/dag_processing/processor.py b/airflow-core/src/airflow/dag_processing/processor.py index cd6f265179ae9..43e47d49dce2c 100644 --- a/airflow-core/src/airflow/dag_processing/processor.py +++ b/airflow-core/src/airflow/dag_processing/processor.py @@ -31,7 +31,7 @@ from airflow.callbacks.callback_requests import ( CallbackRequest, DagCallbackRequest, - EmailNotificationRequest, + EmailRequest, TaskCallbackRequest, ) from airflow.configuration import conf @@ -243,7 +243,7 @@ def _execute_callbacks( _execute_task_callbacks(dagbag, request, log) elif isinstance(request, DagCallbackRequest): _execute_dag_callbacks(dagbag, request, log) - elif isinstance(request, EmailNotificationRequest): + elif isinstance(request, EmailRequest): _execute_email_callbacks(dagbag, request, log) @@ -354,9 +354,7 @@ def get_callback_representation(callback): log.exception("Error in callback at index %d: %s", idx, callback_repr) -def _execute_email_callbacks( - dagbag: DagBag, request: EmailNotificationRequest, log: FilteringBoundLogger -) -> None: +def _execute_email_callbacks(dagbag: DagBag, request: EmailRequest, log: FilteringBoundLogger) -> None: """Execute email notification for task failure/retry.""" dag = dagbag.dags[request.ti.dag_id] task = dag.get_task(request.ti.task_id) diff --git a/airflow-core/src/airflow/jobs/scheduler_job_runner.py b/airflow-core/src/airflow/jobs/scheduler_job_runner.py index f20be4b5e0f14..36610142622c0 100644 --- a/airflow-core/src/airflow/jobs/scheduler_job_runner.py +++ b/airflow-core/src/airflow/jobs/scheduler_job_runner.py @@ -43,7 +43,7 @@ from airflow.callbacks.callback_requests import ( DagCallbackRequest, DagRunContext, - EmailNotificationRequest, + EmailRequest, TaskCallbackRequest, ) from airflow.configuration import conf @@ -959,7 +959,7 @@ def process_executor_events( "Sending email request for task %s to DAG Processor", ti, ) - email_request = EmailNotificationRequest( + email_request = EmailRequest( filepath=ti.dag_model.relative_fileloc, bundle_name=ti.dag_version.bundle_name, bundle_version=ti.dag_version.bundle_version, diff --git a/airflow-core/tests/unit/callbacks/test_callback_requests.py b/airflow-core/tests/unit/callbacks/test_callback_requests.py index 428ee34d166e0..8f1d959e27ec3 100644 --- a/airflow-core/tests/unit/callbacks/test_callback_requests.py +++ b/airflow-core/tests/unit/callbacks/test_callback_requests.py @@ -32,7 +32,7 @@ CallbackRequest, DagCallbackRequest, DagRunContext, - EmailNotificationRequest, + EmailRequest, TaskCallbackRequest, ) from airflow.models.dag import DAG @@ -314,9 +314,9 @@ def test_dag_callback_request_serialization_with_context(self): assert result.context_from_server.last_ti.task_id == "test_task" -class TestEmailNotificationRequest: +class TestEmailRequest: def test_email_notification_request_serialization(self): - """Test EmailNotificationRequest can be serialized and used in CallbackRequest union.""" + """Test EmailRequest can be serialized and used in CallbackRequest union.""" ti_data = TIDataModel( id=str(uuid.uuid4()), task_id="test_task", @@ -331,8 +331,8 @@ def test_email_notification_request_serialization(self): current_time = timezone.utcnow() - # Create EmailNotificationRequest - email_request = EmailNotificationRequest( + # Create EmailRequest + email_request = EmailRequest( filepath="/path/to/dag.py", bundle_name="test_bundle", bundle_version="1.0.0", @@ -359,17 +359,17 @@ def test_email_notification_request_serialization(self): # Test serialization json_str = email_request.to_json() - assert "EmailNotificationRequest" in json_str + assert "EmailRequest" in json_str assert "failure" in json_str # Test deserialization - result = EmailNotificationRequest.from_json(json_str) + result = EmailRequest.from_json(json_str) assert result == email_request assert result.email_type == "failure" assert result.ti.task_id == "test_task" def test_callback_request_union_with_email_notification(self): - """Test EmailNotificationRequest works in CallbackRequest union type.""" + """Test EmailRequest works in CallbackRequest union type.""" ti_data = TIDataModel( id=str(uuid.uuid4()), task_id="test_task", @@ -402,7 +402,7 @@ def test_callback_request_union_with_email_notification(self): ) email_data = { - "type": "EmailNotificationRequest", + "type": "EmailRequest", "filepath": "/path/to/dag.py", "bundle_name": "test_bundle", "bundle_version": "1.0.0", @@ -416,7 +416,7 @@ def test_callback_request_union_with_email_notification(self): adapter = TypeAdapter(CallbackRequest) callback_request = adapter.validate_python(email_data) - # Verify it's correctly identified as EmailNotificationRequest - assert isinstance(callback_request, EmailNotificationRequest) + # Verify it's correctly identified as EmailRequest + assert isinstance(callback_request, EmailRequest) assert callback_request.email_type == "retry" assert callback_request.ti.task_id == "test_task" diff --git a/airflow-core/tests/unit/dag_processing/test_processor.py b/airflow-core/tests/unit/dag_processing/test_processor.py index a90d9a0e30c69..24143e27241e8 100644 --- a/airflow-core/tests/unit/dag_processing/test_processor.py +++ b/airflow-core/tests/unit/dag_processing/test_processor.py @@ -45,7 +45,7 @@ CallbackRequest, DagCallbackRequest, DagRunContext, - EmailNotificationRequest, + EmailRequest, TaskCallbackRequest, ) from airflow.dag_processing.dagbag import DagBag @@ -1303,7 +1303,7 @@ def test_execute_email_callbacks_failure(self, mock_send_email): ) current_time = timezone.utcnow() - request = EmailNotificationRequest( + request = EmailRequest( filepath="/path/to/dag.py", bundle_name="test_bundle", bundle_version="1.0.0", @@ -1363,7 +1363,7 @@ def test_execute_email_callbacks_retry(self, mock_send_email): current_time = timezone.utcnow() - request = EmailNotificationRequest( + request = EmailRequest( filepath="/path/to/dag.py", bundle_name="test_bundle", bundle_version="1.0.0", @@ -1422,7 +1422,7 @@ def test_execute_email_callbacks_no_email_configured(self, mock_send_email): ) current_time = timezone.utcnow() - request = EmailNotificationRequest( + request = EmailRequest( filepath="/path/to/dag.py", bundle_name="test_bundle", bundle_version="1.0.0", @@ -1480,7 +1480,7 @@ def test_execute_email_callbacks_email_disabled_for_type(self, mock_send_email): current_time = timezone.utcnow() # Create request for failure (but email_on_failure is False) - request = EmailNotificationRequest( + request = EmailRequest( filepath="/path/to/dag.py", bundle_name="test_bundle", bundle_version="1.0.0", From 6d67f71224f08294053dc627ebf1dc96ff3b64c9 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 6 Oct 2025 19:54:12 +0100 Subject: [PATCH 215/338] Add uvicorn to spelling ignore list (#56441) --- docs/spelling_wordlist.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 35cddc8aa55ee..9b109a740a237 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -1974,6 +1974,7 @@ Utils utils uuid uv +uvicorn V1JobList validator validators From 88e3c7f9eebeba09a78c87b2af91adcb3511aa04 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Tue, 7 Oct 2025 15:54:39 -0700 Subject: [PATCH 216/338] Bump uv version to 0.8.24 and prek to 0.2.4 (#56464) --- .github/actions/install-prek/action.yml | 4 +- .github/workflows/basic-tests.yml | 2 +- .github/workflows/release_dockerhub_image.yml | 2 +- Dockerfile | 2 +- Dockerfile.ci | 4 +- dev/breeze/README.md | 2 +- dev/breeze/doc/ci/02_images.md | 4 +- dev/breeze/pyproject.toml | 2 +- .../commands/release_management_commands.py | 4 +- .../src/airflow_breeze/global_constants.py | 4 +- dev/breeze/uv.lock | 46 +++++++++---------- pyproject.toml | 2 +- scripts/ci/install_breeze.sh | 2 +- scripts/tools/setup_breeze | 2 +- 14 files changed, 41 insertions(+), 41 deletions(-) diff --git a/.github/actions/install-prek/action.yml b/.github/actions/install-prek/action.yml index ecf2ac7a55cae..e9b45c50ff0b4 100644 --- a/.github/actions/install-prek/action.yml +++ b/.github/actions/install-prek/action.yml @@ -24,10 +24,10 @@ inputs: default: "3.10" uv-version: description: 'uv version to use' - default: "0.8.22" # Keep this comment to allow automatic replacement of uv version + default: "0.8.24" # Keep this comment to allow automatic replacement of uv version prek-version: description: 'prek version to use' - default: "0.2.3" # Keep this comment to allow automatic replacement of prek version + default: "0.2.4" # Keep this comment to allow automatic replacement of prek version skip-prek-hooks: description: "Skip some prek hooks from installation" default: "" diff --git a/.github/workflows/basic-tests.yml b/.github/workflows/basic-tests.yml index f7a4c5bda4c9b..bceb8fd8fe83c 100644 --- a/.github/workflows/basic-tests.yml +++ b/.github/workflows/basic-tests.yml @@ -66,7 +66,7 @@ on: # yamllint disable-line rule:truthy type: string uv-version: description: 'uv version to use' - default: "0.8.22" # Keep this comment to allow automatic replacement of uv version + default: "0.8.24" # Keep this comment to allow automatic replacement of uv version type: string platform: description: 'Platform for the build - linux/amd64 or linux/arm64' diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml index a6b5b45fe010c..c69bac51a2fc6 100644 --- a/.github/workflows/release_dockerhub_image.yml +++ b/.github/workflows/release_dockerhub_image.yml @@ -58,7 +58,7 @@ jobs: AIRFLOW_VERSION: ${{ github.event.inputs.airflowVersion }} AMD_ONLY: ${{ github.event.inputs.amdOnly }} LIMIT_PYTHON_VERSIONS: ${{ github.event.inputs.limitPythonVersions }} - UV_VERSION: "0.8.22" # Keep this comment to allow automatic replacement of uv version + UV_VERSION: "0.8.24" # Keep this comment to allow automatic replacement of uv version if: contains(fromJSON('[ "ashb", "eladkal", diff --git a/Dockerfile b/Dockerfile index 6215c8120efbe..55577306b07d2 100644 --- a/Dockerfile +++ b/Dockerfile @@ -56,7 +56,7 @@ ARG AIRFLOW_PYTHON_VERSION="3.12.11" # Also use `force pip` label on your PR to swap all places we use `uv` to `pip` ARG AIRFLOW_PIP_VERSION=25.2 # ARG AIRFLOW_PIP_VERSION="git+https://github.com/pypa/pip.git@main" -ARG AIRFLOW_UV_VERSION=0.8.22 +ARG AIRFLOW_UV_VERSION=0.8.24 ARG AIRFLOW_USE_UV="false" ARG UV_HTTP_TIMEOUT="300" ARG AIRFLOW_IMAGE_REPOSITORY="https://github.com/apache/airflow" diff --git a/Dockerfile.ci b/Dockerfile.ci index 3ddf9297ebec9..3e8349eabf4cf 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1677,8 +1677,8 @@ COPY --from=scripts common.sh install_packaging_tools.sh install_additional_depe # Also use `force pip` label on your PR to swap all places we use `uv` to `pip` ARG AIRFLOW_PIP_VERSION=25.2 # ARG AIRFLOW_PIP_VERSION="git+https://github.com/pypa/pip.git@main" -ARG AIRFLOW_UV_VERSION=0.8.22 -ARG AIRFLOW_PREK_VERSION="0.2.3" +ARG AIRFLOW_UV_VERSION=0.8.24 +ARG AIRFLOW_PREK_VERSION="0.2.4" # UV_LINK_MODE=copy is needed since we are using cache mounted from the host ENV AIRFLOW_PIP_VERSION=${AIRFLOW_PIP_VERSION} \ diff --git a/dev/breeze/README.md b/dev/breeze/README.md index c9699263cd9a9..2c0ebeb2697f4 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -135,6 +135,6 @@ PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PREK. --------------------------------------------------------------------------------------------------------- -Package config hash: 71b4a2063359e3487de35d5f6b6acc11096799ccf64bf9e4497a3c7d258cba5fca384e8e199b5933bb1c46c262bdf3ff636167c8863b48905cb31cd10b3312c5 +Package config hash: 6687b6346fb381d91da41c55af54c2ba6c4363c0eca4a063db0bb9babb41820a70a89b34c9e88b71cfbbd7cfd272f3ee90b5b1d04e2c78252c28f39307443ca0 --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/doc/ci/02_images.md b/dev/breeze/doc/ci/02_images.md index 4cd1ceaa77399..cec903cc324ba 100644 --- a/dev/breeze/doc/ci/02_images.md +++ b/dev/breeze/doc/ci/02_images.md @@ -443,8 +443,8 @@ can be used for CI images: | `ADDITIONAL_DEV_APT_DEPS` | | Additional apt dev dependencies installed in the first part of the image | | `ADDITIONAL_DEV_APT_ENV` | | Additional env variables defined when installing dev deps | | `AIRFLOW_PIP_VERSION` | `25.2` | `pip` version used. | -| `AIRFLOW_UV_VERSION` | `0.8.22` | `uv` version used. | -| `AIRFLOW_PREK_VERSION` | `0.2.3` | `prek` version used. | +| `AIRFLOW_UV_VERSION` | `0.8.24` | `uv` version used. | +| `AIRFLOW_PREK_VERSION` | `0.2.4` | `prek` version used. | | `AIRFLOW_USE_UV` | `true` | Whether to use UV for installation. | | `PIP_PROGRESS_BAR` | `on` | Progress bar for PIP installation | diff --git a/dev/breeze/pyproject.toml b/dev/breeze/pyproject.toml index 97d37c583666e..44dc04f56b8c0 100644 --- a/dev/breeze/pyproject.toml +++ b/dev/breeze/pyproject.toml @@ -60,7 +60,7 @@ dependencies = [ "jinja2>=3.1.5", "jsonschema>=4.19.1", "packaging>=25.0", - "prek>=0.2.3", + "prek>=0.2.4", "psutil>=5.9.6", "pygithub>=2.1.1", "pytest-xdist>=3.3.1", diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py index 142787b5cec19..d149d7db50e6e 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py @@ -260,11 +260,11 @@ class VersionedFile(NamedTuple): AIRFLOW_PIP_VERSION = "25.2" -AIRFLOW_UV_VERSION = "0.8.22" +AIRFLOW_UV_VERSION = "0.8.24" AIRFLOW_USE_UV = False GITPYTHON_VERSION = "3.1.45" RICH_VERSION = "14.1.0" -PREK_VERSION = "0.2.3" +PREK_VERSION = "0.2.4" HATCH_VERSION = "1.14.2" PYYAML_VERSION = "6.0.3" diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index ae201ea2a1b85..118bd09c7c484 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -203,7 +203,7 @@ ALLOWED_INSTALL_MYSQL_CLIENT_TYPES = ["mariadb", "mysql"] PIP_VERSION = "25.2" -UV_VERSION = "0.8.22" +UV_VERSION = "0.8.24" DEFAULT_UV_HTTP_TIMEOUT = 300 DEFAULT_WSL2_HTTP_TIMEOUT = 900 @@ -806,7 +806,7 @@ def generate_provider_dependencies_if_needed(): "3.10": "3.10.18", "3.11": "3.11.13", "3.12": "3.12.11", - "3.13": "3.13.7", + "3.13": "3.13.8", } # Number of slices for low dep tests diff --git a/dev/breeze/uv.lock b/dev/breeze/uv.lock index 83dfe2038ac1e..9ba485fb01543 100644 --- a/dev/breeze/uv.lock +++ b/dev/breeze/uv.lock @@ -75,7 +75,7 @@ requires-dist = [ { name = "jinja2", specifier = ">=3.1.5" }, { name = "jsonschema", specifier = ">=4.19.1" }, { name = "packaging", specifier = ">=25.0" }, - { name = "prek", specifier = ">=0.2.3" }, + { name = "prek", specifier = ">=0.2.4" }, { name = "psutil", specifier = ">=5.9.6" }, { name = "pygithub", specifier = ">=2.1.1" }, { name = "pytest", specifier = ">=8.3.3" }, @@ -1023,28 +1023,28 @@ wheels = [ [[package]] name = "prek" -version = "0.2.3" -source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a4/8e/ff52d55d27d3756e63f2b9e5b4c4435f7c7f485044df9bd874be01d4bac9/prek-0.2.3.tar.gz", hash = "sha256:a0df9d89618ea8060e766ec21f67bf6c0fac4f320fcbf3073919630b17494996", size = 3007716, upload-time = "2025-09-29T08:59:02.637Z" } -wheels = [ - { url = "https://files.pythonhosted.org/packages/08/2b/bd188d222d55bd8c63c0bbf736f361b79559457b51553fe7d90ff9950839/prek-0.2.3-py3-none-linux_armv6l.whl", hash = "sha256:216c06989e421f79bf5a9eb3df1c470878438fac1bc0a636e03fc4d614bf219e", size = 4364783, upload-time = "2025-09-29T08:58:36.087Z" }, - { url = "https://files.pythonhosted.org/packages/c5/a4/24c2dea15242254e3b187f27f419039a88efb56e2d86658b7d2207dba637/prek-0.2.3-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:37467f2752ce0d4ca6451970d6863211db0ce3f390c74e91db790aad51357eb8", size = 4465162, upload-time = "2025-09-29T08:58:38.138Z" }, - { url = "https://files.pythonhosted.org/packages/f8/2f/1734a1ae08405ba303a31961b9f80b37d0d0ed6e61bb0df9a2ef4ef5f728/prek-0.2.3-py3-none-macosx_11_0_arm64.whl", hash = "sha256:2d24575deb40486a1f08799b1e5e17f3be685cbfe4b5071eb93911a9c2728841", size = 4161043, upload-time = "2025-09-29T08:58:39.273Z" }, - { url = "https://files.pythonhosted.org/packages/d3/c3/b26d307449d805bf8d27d3659f7257395f321399dfea9971a2db27f4e5f8/prek-0.2.3-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:3431ebc218d7a7fc0f708c9eec4027949a081870f4f9fba3a7d65c706db87b72", size = 4343619, upload-time = "2025-09-29T08:58:40.43Z" }, - { url = "https://files.pythonhosted.org/packages/6a/a2/d1850fb04ae63108e896c3f6e822ca70b76616b9aae4187e7dfb33c83588/prek-0.2.3-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:4520c1e0827775af9112787cf3b005a3deee6baeb5d6ebd71004a20b9a7ea73b", size = 4297871, upload-time = "2025-09-29T08:58:41.782Z" }, - { url = "https://files.pythonhosted.org/packages/86/a3/028aae3149f69441932ab0bb154068481bd77283502309880326b979480c/prek-0.2.3-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c8eab4a8d71b978f35a73b7a0e074bbe88c8982e2446a05bfc09fcc046b9a2c2", size = 4581855, upload-time = "2025-09-29T08:58:43.367Z" }, - { url = "https://files.pythonhosted.org/packages/5b/8e/aa4bd8ab2c4f365d02b963df2c4c1c1f6842b623f97c4411da25b2e6c880/prek-0.2.3-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:e5ba2b767f1ab011a41592dbc2b41ab9c4641e39b763ea0a0519e1d7b83f79ee", size = 5010892, upload-time = "2025-09-29T08:58:45.061Z" }, - { url = "https://files.pythonhosted.org/packages/81/eb/7f63d8b30fdecbf2722165406a92d56a819d824c0a2c51e308dcbecbbfc2/prek-0.2.3-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e4afc3876dc812a55dfe7d48c0878bdfadc035697292dbfd6a6bd819d845721b", size = 4938755, upload-time = "2025-09-29T08:58:46.682Z" }, - { url = "https://files.pythonhosted.org/packages/26/9d/0cc72bb823a078638b50dcca674e8eaa7bbd59dca5ee8f8d5311ec4295a5/prek-0.2.3-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2de997b640350a4653c267e6e6e681a0902ab436a8a7e659ab1bced213249f79", size = 5061621, upload-time = "2025-09-29T08:58:48.266Z" }, - { url = "https://files.pythonhosted.org/packages/e5/01/19c1bf227879cee63f8d0644d72728bce43ed96ad3ea355b74cdad77a2d1/prek-0.2.3-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:030a41c816d2558326f3ad9adab35e412280fc3183a81a23f450b86b9525aa1d", size = 4646050, upload-time = "2025-09-29T08:58:49.451Z" }, - { url = "https://files.pythonhosted.org/packages/a4/5c/aaa792519e01c7246a41c6a8983b2aa5d52d9b8c7d3d2a54bd0b528ce204/prek-0.2.3-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:7f567b7f2aab8b7dc09e23bab377df69f172b7ccc630bd98fadd103f04878a0e", size = 4356984, upload-time = "2025-09-29T08:58:51.069Z" }, - { url = "https://files.pythonhosted.org/packages/d4/36/6fe6aecc7302cc870e77d3c8084c62d3db8d3208d0c43aeb6e04de65fa78/prek-0.2.3-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:1193882d0e1fb21b757ccc53a34435d229a2f38ba441edd2c66289532b23d96a", size = 4454158, upload-time = "2025-09-29T08:58:52.628Z" }, - { url = "https://files.pythonhosted.org/packages/0c/d0/2c1a89a10f8fb08f7eb37bed360b25fef9db4b86a99e8181e77bffab0885/prek-0.2.3-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:b7a08dacd23791392e9807f4f42631aa1ac53b5907256e4c94af416833b23d00", size = 4276973, upload-time = "2025-09-29T08:58:54.472Z" }, - { url = "https://files.pythonhosted.org/packages/bf/a8/87e088e97badd0a5c9b79c6837a442a17aff9914d70425ef44ee0819c436/prek-0.2.3-py3-none-musllinux_1_1_i686.whl", hash = "sha256:466eb9ff44575c95b7442751ea86c2f0e9c8c188e7cb79a83134c3a768631c20", size = 4474593, upload-time = "2025-09-29T08:58:55.93Z" }, - { url = "https://files.pythonhosted.org/packages/62/9c/5a844812f37f7fec3087072fa1dac3da867ba29f71d2cab8df8de396a32e/prek-0.2.3-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:4f6f90c5adda349110a9ff6ca50cda8200d9fb10a6891ca9c89f179cb789c957", size = 4751816, upload-time = "2025-09-29T08:58:57.447Z" }, - { url = "https://files.pythonhosted.org/packages/1a/23/354fc3934cf09bc0e5d8fa3c52d40c5b686bf2b3faa314b9b6e8dd72d7f7/prek-0.2.3-py3-none-win32.whl", hash = "sha256:c90e15f8617a956a9d2b0c612783eec585a355da685b8f44d338af62ba667c55", size = 4183840, upload-time = "2025-09-29T08:58:58.636Z" }, - { url = "https://files.pythonhosted.org/packages/4a/07/8a285a062d9d1cf16bbafd1d3782e273e4c2863d521ad84013a1af7d746d/prek-0.2.3-py3-none-win_amd64.whl", hash = "sha256:21cb38ae352772477474cb4c3cd9e9056a43ba7779e634bc23826b6dd01941f5", size = 4748429, upload-time = "2025-09-29T08:58:59.924Z" }, - { url = "https://files.pythonhosted.org/packages/da/bd/916ccaee27bb3a9b018ad845da25d79594922085df77bcdef842379ad99a/prek-0.2.3-py3-none-win_arm64.whl", hash = "sha256:5c8bbdc6f4313d989327407a516b59e27624ff16c75f939c497c9cacf6e4fbba", size = 4436318, upload-time = "2025-09-29T08:59:01.418Z" }, +version = "0.2.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/e1/18/1b3eac3549e760e1286a05a17889037787f353ca2caaca2420225681ee07/prek-0.2.4.tar.gz", hash = "sha256:d38e6a0b16915bcd04438873a9fcf8214844868a26bb11941dc65e10ec8c4bfd", size = 3009030, upload-time = "2025-10-07T09:14:55.931Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c1/05/9c5e247493753dcf7f3059f80897cf989aca42fc4c9f2a26bb6bea991ea9/prek-0.2.4-py3-none-linux_armv6l.whl", hash = "sha256:0dc14939a7ecc3e418665a917a7949f83c34f08bfd9e221bbdc61ea013e633c6", size = 4366340, upload-time = "2025-10-07T09:14:21.758Z" }, + { url = "https://files.pythonhosted.org/packages/b1/62/5a274190f9892e47d083d51f199eac6173ad30a338468237953b3d61d12e/prek-0.2.4-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:552659515d2dcddeac635a5c6b49d0f4f6290d0ef7bf68d05251b1280afcd9dd", size = 4476182, upload-time = "2025-10-07T09:14:23.75Z" }, + { url = "https://files.pythonhosted.org/packages/a1/cd/6375bff5b858b31e7a811d9990eb5b8d96f74959ef5d41268b4886ace166/prek-0.2.4-py3-none-macosx_11_0_arm64.whl", hash = "sha256:b44a809ace8b68ff0970026439547542fdaca5413c24e0af4b9db55dcefb69d4", size = 4170975, upload-time = "2025-10-07T09:14:25.523Z" }, + { url = "https://files.pythonhosted.org/packages/e6/8f/4ed39b27121bf442348a087fae8b7d52595424f7c55d9c3e7f083e3ad0bf/prek-0.2.4-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:e113ec75d8ceff0a4df0a11d27aa58c0ef46723f5b0835d15ced4e4fb314a87d", size = 4356395, upload-time = "2025-10-07T09:14:27.444Z" }, + { url = "https://files.pythonhosted.org/packages/92/80/0a7f41a1a142efc263feae037a8093bd4406b2708c083ac0ded515056898/prek-0.2.4-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:324bb5b2b367b9dc853ade4f5669d72ef560924f1d5e26886400a262e98c8989", size = 4306930, upload-time = "2025-10-07T09:14:28.833Z" }, + { url = "https://files.pythonhosted.org/packages/f7/90/0d7096d207e6e01d2d1ad9d320b050f207b40badc2e3c74bc23fcd7c68c1/prek-0.2.4-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:213087cee04d017ea1f3efbfc5803c76c755d1d64e66270a122de57afc2eb68e", size = 4588571, upload-time = "2025-10-07T09:14:30.546Z" }, + { url = "https://files.pythonhosted.org/packages/26/bf/86496647a0ecf4e3ff5182ebb56901c345cec32eae0dc5b56ef47c63eb55/prek-0.2.4-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:4a606f5078525c2b6609b88885ee6aa8315ccc87093ff24829d77bd9bcac4f78", size = 5023154, upload-time = "2025-10-07T09:14:32.944Z" }, + { url = "https://files.pythonhosted.org/packages/b3/22/88633677c9e61f675e95dfac6b958127d6f3b4275c095b58520f0ff1fe32/prek-0.2.4-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:da20bc847bf0f6acad34d1295abe954228516c76bac379403488a6251186d114", size = 4947888, upload-time = "2025-10-07T09:14:34.883Z" }, + { url = "https://files.pythonhosted.org/packages/67/a6/999a80fd63f52385def9ec3339a64ad484b1c858974fdbed52514b22b45d/prek-0.2.4-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f7165c4ff581b43a545304151248e4db9fadcef8ff24d586870680973e978c1e", size = 5068873, upload-time = "2025-10-07T09:14:36.959Z" }, + { url = "https://files.pythonhosted.org/packages/fb/5f/50a9dfe30c0897d4695fdff958ff9a970b6a9c26be62c2362ced274d85a5/prek-0.2.4-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:39456219bbdfc025e19cadedfbb5799316405cb8380d4245a7b83e43eafa9bd9", size = 4657247, upload-time = "2025-10-07T09:14:38.48Z" }, + { url = "https://files.pythonhosted.org/packages/93/b1/622c3bdb711a8fe3855e30fb7b2a0726acc7f59ceea782b547f22eec1afe/prek-0.2.4-py3-none-manylinux_2_28_aarch64.whl", hash = "sha256:2edbf9d24dcb0681709f5df9c7d9ff15ceac3dd7f5c2cdb8bfec7163e4449278", size = 4368550, upload-time = "2025-10-07T09:14:40.399Z" }, + { url = "https://files.pythonhosted.org/packages/5b/73/d5e78a65065085efaac2a88e4cab0b4414c3ab0936cec2d0917aeafab9bb/prek-0.2.4-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:719b8073bb9df96520008f6eb5e81c248ea7891c10117f01b8344d710ca884fe", size = 4463354, upload-time = "2025-10-07T09:14:42.245Z" }, + { url = "https://files.pythonhosted.org/packages/ff/62/0a527d29880be80febf258f9d1ff13f6fecfd96b257b33c8927809dfcc7a/prek-0.2.4-py3-none-musllinux_1_1_armv7l.whl", hash = "sha256:900319e7ff34c66a235560e4f612f2b3c897a7c6e4b6b29412ca20b0fa8d6e63", size = 4283556, upload-time = "2025-10-07T09:14:44.135Z" }, + { url = "https://files.pythonhosted.org/packages/49/03/9ace2e58c6c6d566110b76bfca3e9bcfa00c339cdbe08f88fef9df101264/prek-0.2.4-py3-none-musllinux_1_1_i686.whl", hash = "sha256:a18dba791b08817e8a8a82b97ba793fe46b5788323cb92d76d3f24a29a5400c0", size = 4482945, upload-time = "2025-10-07T09:14:45.638Z" }, + { url = "https://files.pythonhosted.org/packages/d6/7e/2a1aebb3ae182e2a6a52082eef2e06c378a4a7b2cc33cf3cf483826a403e/prek-0.2.4-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:3b17545a043d4979a7fc4b92ff4957c8a11b8ee3f2c9b66443a9374a858f9dc5", size = 4759310, upload-time = "2025-10-07T09:14:47.104Z" }, + { url = "https://files.pythonhosted.org/packages/74/aa/4e291fbfefefedeaa3cac1a87498d44de1b84e9212919039fe70c2abbc68/prek-0.2.4-py3-none-win32.whl", hash = "sha256:64b0c913a7ccfffecab93eb1faa451431a80f3a21c8337b04c93f3cee883cca9", size = 4189111, upload-time = "2025-10-07T09:14:48.92Z" }, + { url = "https://files.pythonhosted.org/packages/5d/31/1a4358000eb8f5f57215ee0842f47de88cdc577c15cd74e20e7fc11cb36b/prek-0.2.4-py3-none-win_amd64.whl", hash = "sha256:e1817d4f0ed7a05e81eafa67da8272fc1e8e3b61451e217339872f064c677839", size = 4753648, upload-time = "2025-10-07T09:14:50.888Z" }, + { url = "https://files.pythonhosted.org/packages/50/b8/93069590f1f515eaa90bf72502367124b5579afce1586f144fa4425e18e1/prek-0.2.4-py3-none-win_arm64.whl", hash = "sha256:05645d5ccfa7a54a272591567ad9787590bc1f5c13320364d9cfb4043151c442", size = 4438539, upload-time = "2025-10-07T09:14:52.637Z" }, ] [[package]] diff --git a/pyproject.toml b/pyproject.toml index 83ecea1f56e45..e62439bcf24b6 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -535,7 +535,7 @@ packages = [] "apache-airflow-providers-amazon[s3fs]", ] "uv" = [ - "uv>=0.8.22", + "uv>=0.8.24", ] diff --git a/scripts/ci/install_breeze.sh b/scripts/ci/install_breeze.sh index a033e027a3968..56401bd37c5ac 100755 --- a/scripts/ci/install_breeze.sh +++ b/scripts/ci/install_breeze.sh @@ -22,7 +22,7 @@ cd "$( dirname "${BASH_SOURCE[0]}" )/../../" PYTHON_ARG="" PIP_VERSION="25.2" -UV_VERSION="0.8.22" +UV_VERSION="0.8.24" if [[ ${PYTHON_VERSION=} != "" ]]; then PYTHON_ARG="--python=$(which python"${PYTHON_VERSION}") " fi diff --git a/scripts/tools/setup_breeze b/scripts/tools/setup_breeze index 6821d8f3d49a5..85782932b01e2 100755 --- a/scripts/tools/setup_breeze +++ b/scripts/tools/setup_breeze @@ -27,7 +27,7 @@ COLOR_YELLOW=$'\e[33m' COLOR_BLUE=$'\e[34m' COLOR_RESET=$'\e[0m' -UV_VERSION="0.8.22" +UV_VERSION="0.8.24" function manual_instructions() { echo From cac1fc837da4f629e17e0498445baae052d09300 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Wed, 8 Oct 2025 05:08:39 -0700 Subject: [PATCH 217/338] Bump uv version to 0.9.0 and ruff to 0.14.0 (#56467) --- .github/actions/install-prek/action.yml | 2 +- .github/workflows/basic-tests.yml | 2 +- .github/workflows/release_dockerhub_image.yml | 2 +- .pre-commit-config.yaml | 2 +- Dockerfile | 2 +- Dockerfile.ci | 4 ++-- airflow-core/docs/best-practices.rst | 2 +- dev/breeze/doc/ci/02_images.md | 2 +- .../airflow_breeze/commands/release_management_commands.py | 2 +- dev/breeze/src/airflow_breeze/global_constants.py | 2 +- devel-common/pyproject.toml | 2 +- pyproject.toml | 2 +- scripts/ci/install_breeze.sh | 2 +- scripts/ci/prek/check_imports_in_providers.py | 2 +- scripts/ci/prek/ruff_format.py | 2 +- scripts/tools/setup_breeze | 2 +- 16 files changed, 17 insertions(+), 17 deletions(-) diff --git a/.github/actions/install-prek/action.yml b/.github/actions/install-prek/action.yml index e9b45c50ff0b4..5a3685791c180 100644 --- a/.github/actions/install-prek/action.yml +++ b/.github/actions/install-prek/action.yml @@ -24,7 +24,7 @@ inputs: default: "3.10" uv-version: description: 'uv version to use' - default: "0.8.24" # Keep this comment to allow automatic replacement of uv version + default: "0.9.0" # Keep this comment to allow automatic replacement of uv version prek-version: description: 'prek version to use' default: "0.2.4" # Keep this comment to allow automatic replacement of prek version diff --git a/.github/workflows/basic-tests.yml b/.github/workflows/basic-tests.yml index bceb8fd8fe83c..3e71db9389129 100644 --- a/.github/workflows/basic-tests.yml +++ b/.github/workflows/basic-tests.yml @@ -66,7 +66,7 @@ on: # yamllint disable-line rule:truthy type: string uv-version: description: 'uv version to use' - default: "0.8.24" # Keep this comment to allow automatic replacement of uv version + default: "0.9.0" # Keep this comment to allow automatic replacement of uv version type: string platform: description: 'Platform for the build - linux/amd64 or linux/arm64' diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml index c69bac51a2fc6..7b947a917f326 100644 --- a/.github/workflows/release_dockerhub_image.yml +++ b/.github/workflows/release_dockerhub_image.yml @@ -58,7 +58,7 @@ jobs: AIRFLOW_VERSION: ${{ github.event.inputs.airflowVersion }} AMD_ONLY: ${{ github.event.inputs.amdOnly }} LIMIT_PYTHON_VERSIONS: ${{ github.event.inputs.limitPythonVersions }} - UV_VERSION: "0.8.24" # Keep this comment to allow automatic replacement of uv version + UV_VERSION: "0.9.0" # Keep this comment to allow automatic replacement of uv version if: contains(fromJSON('[ "ashb", "eladkal", diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index dd1ff6006daa7..12501e6e40723 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -412,7 +412,7 @@ repos: types_or: [python, pyi] args: [--fix] require_serial: true - additional_dependencies: ['ruff==0.13.3'] + additional_dependencies: ['ruff==0.14.0'] exclude: ^airflow-core/tests/unit/dags/test_imports\.py$|^performance/tests/test_.*\.py$ - id: ruff-format name: Run 'ruff format' diff --git a/Dockerfile b/Dockerfile index 55577306b07d2..6838ac920896f 100644 --- a/Dockerfile +++ b/Dockerfile @@ -56,7 +56,7 @@ ARG AIRFLOW_PYTHON_VERSION="3.12.11" # Also use `force pip` label on your PR to swap all places we use `uv` to `pip` ARG AIRFLOW_PIP_VERSION=25.2 # ARG AIRFLOW_PIP_VERSION="git+https://github.com/pypa/pip.git@main" -ARG AIRFLOW_UV_VERSION=0.8.24 +ARG AIRFLOW_UV_VERSION=0.9.0 ARG AIRFLOW_USE_UV="false" ARG UV_HTTP_TIMEOUT="300" ARG AIRFLOW_IMAGE_REPOSITORY="https://github.com/apache/airflow" diff --git a/Dockerfile.ci b/Dockerfile.ci index 3e8349eabf4cf..7daf551c1d99b 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1542,7 +1542,7 @@ ENV DEV_APT_COMMAND=${DEV_APT_COMMAND} \ ARG AIRFLOW_PYTHON_VERSION="3.12.11" ENV AIRFLOW_PYTHON_VERSION=${AIRFLOW_PYTHON_VERSION} -ENV GOLANG_MAJOR_MINOR_VERSION="1.25.1" +ENV GOLANG_MAJOR_MINOR_VERSION="1.25.2" COPY --from=scripts install_os_dependencies.sh /scripts/docker/ @@ -1677,7 +1677,7 @@ COPY --from=scripts common.sh install_packaging_tools.sh install_additional_depe # Also use `force pip` label on your PR to swap all places we use `uv` to `pip` ARG AIRFLOW_PIP_VERSION=25.2 # ARG AIRFLOW_PIP_VERSION="git+https://github.com/pypa/pip.git@main" -ARG AIRFLOW_UV_VERSION=0.8.24 +ARG AIRFLOW_UV_VERSION=0.9.0 ARG AIRFLOW_PREK_VERSION="0.2.4" # UV_LINK_MODE=copy is needed since we are using cache mounted from the host diff --git a/airflow-core/docs/best-practices.rst b/airflow-core/docs/best-practices.rst index 297884a918573..be3e959946fc5 100644 --- a/airflow-core/docs/best-practices.rst +++ b/airflow-core/docs/best-practices.rst @@ -310,7 +310,7 @@ Installing and Using ruff .. code-block:: bash - pip install "ruff>=0.13.3" + pip install "ruff>=0.14.0" 2. **Running ruff**: Execute ``ruff`` to check your Dags for potential issues: diff --git a/dev/breeze/doc/ci/02_images.md b/dev/breeze/doc/ci/02_images.md index cec903cc324ba..affb647e998ac 100644 --- a/dev/breeze/doc/ci/02_images.md +++ b/dev/breeze/doc/ci/02_images.md @@ -443,7 +443,7 @@ can be used for CI images: | `ADDITIONAL_DEV_APT_DEPS` | | Additional apt dev dependencies installed in the first part of the image | | `ADDITIONAL_DEV_APT_ENV` | | Additional env variables defined when installing dev deps | | `AIRFLOW_PIP_VERSION` | `25.2` | `pip` version used. | -| `AIRFLOW_UV_VERSION` | `0.8.24` | `uv` version used. | +| `AIRFLOW_UV_VERSION` | `0.9.0` | `uv` version used. | | `AIRFLOW_PREK_VERSION` | `0.2.4` | `prek` version used. | | `AIRFLOW_USE_UV` | `true` | Whether to use UV for installation. | | `PIP_PROGRESS_BAR` | `on` | Progress bar for PIP installation | diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py index d149d7db50e6e..fcef12137231d 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py @@ -260,7 +260,7 @@ class VersionedFile(NamedTuple): AIRFLOW_PIP_VERSION = "25.2" -AIRFLOW_UV_VERSION = "0.8.24" +AIRFLOW_UV_VERSION = "0.9.0" AIRFLOW_USE_UV = False GITPYTHON_VERSION = "3.1.45" RICH_VERSION = "14.1.0" diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 118bd09c7c484..27976fdd1d0ed 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -203,7 +203,7 @@ ALLOWED_INSTALL_MYSQL_CLIENT_TYPES = ["mariadb", "mysql"] PIP_VERSION = "25.2" -UV_VERSION = "0.8.24" +UV_VERSION = "0.9.0" DEFAULT_UV_HTTP_TIMEOUT = 300 DEFAULT_WSL2_HTTP_TIMEOUT = 900 diff --git a/devel-common/pyproject.toml b/devel-common/pyproject.toml index 2808f3c58f1e6..e956296a2475d 100644 --- a/devel-common/pyproject.toml +++ b/devel-common/pyproject.toml @@ -35,7 +35,7 @@ dependencies = [ "kgb>=7.2.0", "requests_mock>=1.11.0", "rich>=13.6.0", - "ruff==0.13.3", + "ruff==0.14.0", "semver>=3.0.2", "time-machine>=2.15.0", "wheel>=0.42.0", diff --git a/pyproject.toml b/pyproject.toml index e62439bcf24b6..ccb25e2d374b5 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -535,7 +535,7 @@ packages = [] "apache-airflow-providers-amazon[s3fs]", ] "uv" = [ - "uv>=0.8.24", + "uv>=0.9.0", ] diff --git a/scripts/ci/install_breeze.sh b/scripts/ci/install_breeze.sh index 56401bd37c5ac..8a40e4bc315d9 100755 --- a/scripts/ci/install_breeze.sh +++ b/scripts/ci/install_breeze.sh @@ -22,7 +22,7 @@ cd "$( dirname "${BASH_SOURCE[0]}" )/../../" PYTHON_ARG="" PIP_VERSION="25.2" -UV_VERSION="0.8.24" +UV_VERSION="0.9.0" if [[ ${PYTHON_VERSION=} != "" ]]; then PYTHON_ARG="--python=$(which python"${PYTHON_VERSION}") " fi diff --git a/scripts/ci/prek/check_imports_in_providers.py b/scripts/ci/prek/check_imports_in_providers.py index 08dbc92ed602a..326fc265e8174 100755 --- a/scripts/ci/prek/check_imports_in_providers.py +++ b/scripts/ci/prek/check_imports_in_providers.py @@ -19,7 +19,7 @@ # requires-python = ">=3.10" # dependencies = [ # "rich>=13.6.0", -# "ruff==0.13.3", +# "ruff==0.14.0", # ] # /// from __future__ import annotations diff --git a/scripts/ci/prek/ruff_format.py b/scripts/ci/prek/ruff_format.py index 1b38de4f3751b..26f6925d0448a 100755 --- a/scripts/ci/prek/ruff_format.py +++ b/scripts/ci/prek/ruff_format.py @@ -18,7 +18,7 @@ # /// script # requires-python = ">=3.10" # dependencies = [ -# "ruff==0.13.3", +# "ruff==0.14.0", # ] # /// diff --git a/scripts/tools/setup_breeze b/scripts/tools/setup_breeze index 85782932b01e2..961d5c4c30fd8 100755 --- a/scripts/tools/setup_breeze +++ b/scripts/tools/setup_breeze @@ -27,7 +27,7 @@ COLOR_YELLOW=$'\e[33m' COLOR_BLUE=$'\e[34m' COLOR_RESET=$'\e[0m' -UV_VERSION="0.8.24" +UV_VERSION="0.9.0" function manual_instructions() { echo From e69578f4c44353ca0cb44aeb48f04bbcb1c553c0 Mon Sep 17 00:00:00 2001 From: Dheeraj Turaga Date: Wed, 8 Oct 2025 11:04:31 -0500 Subject: [PATCH 218/338] Fix Edge3 provider navigation with webserver base_url configuration (#56189) * Fix Edge3 provider navigation with webserver base_url configuration The Edge3 provider's BrowserRouter was not respecting the Airflow webserver.base_url configuration, causing navigation links to generate absolute paths from root instead of properly prefixed paths. This resulted in 404 errors when Airflow is deployed with a base URL prefix (e.g., my_company.com/airflow). **Problem:** - Edge3 provider pages were navigating to /plugin/edge_worker instead of /airflow/plugin/edge_worker - This broke navigation in deployments where nginx redirects traffic from my_company.com/airflow to localhost:8080 - The issue was in JobsPage.tsx:94 where RouterLink generated links without the base URL prefix * Fix: Added _get_base_url_path() helper function and updated all URLs to use it * Fix Edge3 provider frontend API calls with webserver base_url configuration The Edge3 provider's frontend was not respecting Airflow's webserver base_url configuration, causing API calls to fail when the webserver was configured with a subpath (e.g., localhost:8080/airflow). The OpenAPI-generated client was using an empty BASE configuration, resulting in API calls to /edge_worker/ui/jobs instead of /airflow/edge_worker/ui/jobs when base_url was set. This fix initializes OpenAPI.BASE from the HTML tag, following the same pattern used in Airflow core, ensuring all API calls are correctly prefixed with the configured base URL path * Update providers/edge3/src/airflow/providers/edge3/plugins/edge_executor_plugin.py Co-authored-by: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> --------- Co-authored-by: Jens Scheffler <95105677+jscheffl@users.noreply.github.com> --- .../edge3/plugins/edge_executor_plugin.py | 36 +++++++++++++------ .../edge3/plugins/www/dist/main.umd.cjs | 36 +++++++++---------- .../plugins/www/src/layouts/EdgeLayout.tsx | 6 +++- .../providers/edge3/plugins/www/src/main.tsx | 6 ++++ providers/edge3/www-hash.txt | 2 +- 5 files changed, 56 insertions(+), 30 deletions(-) diff --git a/providers/edge3/src/airflow/providers/edge3/plugins/edge_executor_plugin.py b/providers/edge3/src/airflow/providers/edge3/plugins/edge_executor_plugin.py index 6c8d5ebeda0a3..cd24e3e68dde4 100644 --- a/providers/edge3/src/airflow/providers/edge3/plugins/edge_executor_plugin.py +++ b/providers/edge3/src/airflow/providers/edge3/plugins/edge_executor_plugin.py @@ -227,6 +227,22 @@ def change_maintenance_comment(self, worker_name: str): RUNNING_ON_APISERVER = "gunicorn" in sys.argv[0] and "airflow-webserver" in sys.argv +def _get_base_url_path(path: str) -> str: + """Construct URL path with webserver base_url prefix.""" + base_url = conf.get("api", "base_url", fallback="/") + # Extract pathname from base_url (handles both full URLs and path-only) + if base_url.startswith(("http://", "https://")): + from urllib.parse import urlparse + + base_path = urlparse(base_url).path + else: + base_path = base_url + + # Normalize paths: remove trailing slash from base, ensure leading slash on path + base_path = base_path.rstrip("/") + return base_path + path + + class EdgeExecutorPlugin(AirflowPlugin): """EdgeExecutor Plugin - provides API endpoints for Edge Workers in Webserver.""" @@ -237,30 +253,30 @@ class EdgeExecutorPlugin(AirflowPlugin): react_apps = [ { "name": "Edge Worker", - "bundle_url": "/edge_worker/static/main.umd.cjs", + "bundle_url": _get_base_url_path("/edge_worker/static/main.umd.cjs"), "destination": "nav", "url_route": "edge_worker", "category": "admin", - "icon": "/edge_worker/res/cloud-computer.svg", - "icon_dark_mode": "/edge_worker/res/cloud-computer-dark.svg", + "icon": _get_base_url_path("/edge_worker/res/cloud-computer.svg"), + "icon_dark_mode": _get_base_url_path("/edge_worker/res/cloud-computer-dark.svg"), }, { "name": "Edge Worker Jobs", - "bundle_url": "/edge_worker/static/main.umd.cjs", + "bundle_url": _get_base_url_path("/edge_worker/static/main.umd.cjs"), "url_route": "edge_jobs", "category": "admin", - "icon": "/edge_worker/res/cloud-computer.svg", - "icon_dark_mode": "/edge_worker/res/cloud-computer-dark.svg", + "icon": _get_base_url_path("/edge_worker/res/cloud-computer.svg"), + "icon_dark_mode": _get_base_url_path("/edge_worker/res/cloud-computer-dark.svg"), }, ] external_views = [ { "name": "Edge Worker API docs", - "href": "/edge_worker/docs", + "href": _get_base_url_path("/edge_worker/docs"), "destination": "nav", "category": "docs", - "icon": "/edge_worker/res/cloud-computer.svg", - "icon_dark_mode": "/edge_worker/res/cloud-computer-dark.svg", + "icon": _get_base_url_path("/edge_worker/res/cloud-computer.svg"), + "icon_dark_mode": _get_base_url_path("/edge_worker/res/cloud-computer-dark.svg"), "url_route": "edge_worker_api_docs", } ] @@ -271,7 +287,7 @@ class EdgeExecutorPlugin(AirflowPlugin): appbuilder_menu_items = [ { "name": "Edge Worker API docs", - "href": "/edge_worker/v1/ui", + "href": _get_base_url_path("/edge_worker/v1/ui"), "category": "Docs", } ] diff --git a/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs b/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs index 8677681e23078..be28996e0ec5c 100644 --- a/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs +++ b/providers/edge3/src/airflow/providers/edge3/plugins/www/dist/main.umd.cjs @@ -1,4 +1,4 @@ -(function(P,Ft){typeof exports=="object"&&typeof module<"u"?module.exports=Ft(require("react"),require("react-dom")):typeof define=="function"&&define.amd?define(["react","react-dom"],Ft):(P=typeof globalThis<"u"?globalThis:P||self,P.AirflowPlugin=Ft(P.React,P.ReactDOM))})(this,(function(P,Ft){"use strict";function Sp(e){const t=Object.create(null,{[Symbol.toStringTag]:{value:"Module"}});if(e){for(const n in e)if(n!=="default"){const r=Object.getOwnPropertyDescriptor(e,n);Object.defineProperty(t,n,r.get?r:{enumerable:!0,get:()=>e[n]})}}return t.default=e,Object.freeze(t)}const C=Sp(P);function Ep(e){return e&&e.__esModule&&Object.prototype.hasOwnProperty.call(e,"default")?e.default:e}var qi={exports:{}},hr={};/** +(function(P,Lt){typeof exports=="object"&&typeof module<"u"?module.exports=Lt(require("react"),require("react-dom")):typeof define=="function"&&define.amd?define(["react","react-dom"],Lt):(P=typeof globalThis<"u"?globalThis:P||self,P.AirflowPlugin=Lt(P.React,P.ReactDOM))})(this,(function(P,Lt){"use strict";function Ep(e){const t=Object.create(null,{[Symbol.toStringTag]:{value:"Module"}});if(e){for(const n in e)if(n!=="default"){const r=Object.getOwnPropertyDescriptor(e,n);Object.defineProperty(t,n,r.get?r:{enumerable:!0,get:()=>e[n]})}}return t.default=e,Object.freeze(t)}const C=Ep(P);function Op(e){return e&&e.__esModule&&Object.prototype.hasOwnProperty.call(e,"default")?e.default:e}var qi={exports:{}},hr={};/** * @license React * react-jsx-runtime.production.js * @@ -6,28 +6,28 @@ * * This source code is licensed under the MIT license found in the * LICENSE file in the root directory of this source tree. - */var ml;function Op(){if(ml)return hr;ml=1;var e=Symbol.for("react.transitional.element"),t=Symbol.for("react.fragment");function n(r,o,i){var s=null;if(i!==void 0&&(s=""+i),o.key!==void 0&&(s=""+o.key),"key"in o){i={};for(var a in o)a!=="key"&&(i[a]=o[a])}else i=o;return o=i.ref,{$$typeof:e,type:r,key:s,ref:o!==void 0?o:null,props:i}}return hr.Fragment=t,hr.jsx=n,hr.jsxs=n,hr}var vl;function Pp(){return vl||(vl=1,qi.exports=Op()),qi.exports}var g=Pp();function bl(e){var t=Object.create(null);return function(n){return t[n]===void 0&&(t[n]=e(n)),t[n]}}var Rp=/^((children|dangerouslySetInnerHTML|key|ref|autoFocus|defaultValue|defaultChecked|innerHTML|suppressContentEditableWarning|suppressHydrationWarning|valueLink|abbr|accept|acceptCharset|accessKey|action|allow|allowUserMedia|allowPaymentRequest|allowFullScreen|allowTransparency|alt|async|autoComplete|autoPlay|capture|cellPadding|cellSpacing|challenge|charSet|checked|cite|classID|className|cols|colSpan|content|contentEditable|contextMenu|controls|controlsList|coords|crossOrigin|data|dateTime|decoding|default|defer|dir|disabled|disablePictureInPicture|disableRemotePlayback|download|draggable|encType|enterKeyHint|fetchpriority|fetchPriority|form|formAction|formEncType|formMethod|formNoValidate|formTarget|frameBorder|headers|height|hidden|high|href|hrefLang|htmlFor|httpEquiv|id|inputMode|integrity|is|keyParams|keyType|kind|label|lang|list|loading|loop|low|marginHeight|marginWidth|max|maxLength|media|mediaGroup|method|min|minLength|multiple|muted|name|nonce|noValidate|open|optimum|pattern|placeholder|playsInline|popover|popoverTarget|popoverTargetAction|poster|preload|profile|radioGroup|readOnly|referrerPolicy|rel|required|reversed|role|rows|rowSpan|sandbox|scope|scoped|scrolling|seamless|selected|shape|size|sizes|slot|span|spellCheck|src|srcDoc|srcLang|srcSet|start|step|style|summary|tabIndex|target|title|translate|type|useMap|value|width|wmode|wrap|about|datatype|inlist|prefix|property|resource|typeof|vocab|autoCapitalize|autoCorrect|autoSave|color|incremental|fallback|inert|itemProp|itemScope|itemType|itemID|itemRef|on|option|results|security|unselectable|accentHeight|accumulate|additive|alignmentBaseline|allowReorder|alphabetic|amplitude|arabicForm|ascent|attributeName|attributeType|autoReverse|azimuth|baseFrequency|baselineShift|baseProfile|bbox|begin|bias|by|calcMode|capHeight|clip|clipPathUnits|clipPath|clipRule|colorInterpolation|colorInterpolationFilters|colorProfile|colorRendering|contentScriptType|contentStyleType|cursor|cx|cy|d|decelerate|descent|diffuseConstant|direction|display|divisor|dominantBaseline|dur|dx|dy|edgeMode|elevation|enableBackground|end|exponent|externalResourcesRequired|fill|fillOpacity|fillRule|filter|filterRes|filterUnits|floodColor|floodOpacity|focusable|fontFamily|fontSize|fontSizeAdjust|fontStretch|fontStyle|fontVariant|fontWeight|format|from|fr|fx|fy|g1|g2|glyphName|glyphOrientationHorizontal|glyphOrientationVertical|glyphRef|gradientTransform|gradientUnits|hanging|horizAdvX|horizOriginX|ideographic|imageRendering|in|in2|intercept|k|k1|k2|k3|k4|kernelMatrix|kernelUnitLength|kerning|keyPoints|keySplines|keyTimes|lengthAdjust|letterSpacing|lightingColor|limitingConeAngle|local|markerEnd|markerMid|markerStart|markerHeight|markerUnits|markerWidth|mask|maskContentUnits|maskUnits|mathematical|mode|numOctaves|offset|opacity|operator|order|orient|orientation|origin|overflow|overlinePosition|overlineThickness|panose1|paintOrder|pathLength|patternContentUnits|patternTransform|patternUnits|pointerEvents|points|pointsAtX|pointsAtY|pointsAtZ|preserveAlpha|preserveAspectRatio|primitiveUnits|r|radius|refX|refY|renderingIntent|repeatCount|repeatDur|requiredExtensions|requiredFeatures|restart|result|rotate|rx|ry|scale|seed|shapeRendering|slope|spacing|specularConstant|specularExponent|speed|spreadMethod|startOffset|stdDeviation|stemh|stemv|stitchTiles|stopColor|stopOpacity|strikethroughPosition|strikethroughThickness|string|stroke|strokeDasharray|strokeDashoffset|strokeLinecap|strokeLinejoin|strokeMiterlimit|strokeOpacity|strokeWidth|surfaceScale|systemLanguage|tableValues|targetX|targetY|textAnchor|textDecoration|textRendering|textLength|to|transform|u1|u2|underlinePosition|underlineThickness|unicode|unicodeBidi|unicodeRange|unitsPerEm|vAlphabetic|vHanging|vIdeographic|vMathematical|values|vectorEffect|version|vertAdvY|vertOriginX|vertOriginY|viewBox|viewTarget|visibility|widths|wordSpacing|writingMode|x|xHeight|x1|x2|xChannelSelector|xlinkActuate|xlinkArcrole|xlinkHref|xlinkRole|xlinkShow|xlinkTitle|xlinkType|xmlBase|xmlns|xmlnsXlink|xmlLang|xmlSpace|y|y1|y2|yChannelSelector|z|zoomAndPan|for|class|autofocus)|(([Dd][Aa][Tt][Aa]|[Aa][Rr][Ii][Aa]|x)-.*))$/,Ip=bl(function(e){return Rp.test(e)||e.charCodeAt(0)===111&&e.charCodeAt(1)===110&&e.charCodeAt(2)<91});function Tp(e){if(e.sheet)return e.sheet;for(var t=0;t0?we(_n,--Ae):0,An--,ue===10&&(An=1,so--),ue}function je(){return ue=Ae2||mr(ue)>3?"":" "}function Wp(e,t){for(;--t&&je()&&!(ue<48||ue>102||ue>57&&ue<65||ue>70&&ue<97););return pr(e,lo()+(t<6&&st()==32&&je()==32))}function Ji(e){for(;je();)switch(ue){case e:return Ae;case 34:case 39:e!==34&&e!==39&&Ji(ue);break;case 40:e===41&&Ji(e);break;case 92:je();break}return Ae}function Hp(e,t){for(;je()&&e+ue!==57;)if(e+ue===84&&st()===47)break;return"/*"+pr(t,Ae-1)+"*"+oo(e===47?e:je())}function Up(e){for(;!mr(st());)je();return pr(e,Ae)}function Gp(e){return Sl(uo("",null,null,null,[""],e=wl(e),0,[0],e))}function uo(e,t,n,r,o,i,s,a,l){for(var c=0,u=0,h=s,d=0,m=0,f=0,p=1,v=1,b=1,y=0,x="",w=o,k=i,O=r,R=x;v;)switch(f=y,y=je()){case 40:if(f!=108&&we(R,h-1)==58){Xi(R+=X(co(y),"&","&\f"),"&\f")!=-1&&(b=-1);break}case 34:case 39:case 91:R+=co(y);break;case 9:case 10:case 13:case 32:R+=Bp(f);break;case 92:R+=Wp(lo()-1,7);continue;case 47:switch(st()){case 42:case 47:io(qp(Hp(je(),lo()),t,n),l);break;default:R+="/"}break;case 123*p:a[c++]=it(R)*b;case 125*p:case 59:case 0:switch(y){case 0:case 125:v=0;case 59+u:b==-1&&(R=X(R,/\f/g,"")),m>0&&it(R)-h&&io(m>32?Ol(R+";",r,n,h-1):Ol(X(R," ","")+";",r,n,h-2),l);break;case 59:R+=";";default:if(io(O=El(R,t,n,c,u,o,a,x,w=[],k=[],h),i),y===123)if(u===0)uo(R,t,O,O,w,i,h,a,k);else switch(d===99&&we(R,3)===110?100:d){case 100:case 108:case 109:case 115:uo(e,O,O,r&&io(El(e,O,O,0,0,o,a,x,o,w=[],h),k),o,k,h,a,r?w:k);break;default:uo(R,O,O,O,[""],k,0,a,k)}}c=u=m=0,p=b=1,x=R="",h=s;break;case 58:h=1+it(R),m=f;default:if(p<1){if(y==123)--p;else if(y==125&&p++==0&&$p()==125)continue}switch(R+=oo(y),y*p){case 38:b=u>0?1:(R+="\f",-1);break;case 44:a[c++]=(it(R)-1)*b,b=1;break;case 64:st()===45&&(R+=co(je())),d=st(),u=h=it(x=R+=Up(lo())),y++;break;case 45:f===45&&it(R)==2&&(p=0)}}return i}function El(e,t,n,r,o,i,s,a,l,c,u){for(var h=o-1,d=o===0?i:[""],m=Qi(d),f=0,p=0,v=0;f0?d[b]+" "+y:X(y,/&\f/g,d[b])))&&(l[v++]=x);return ao(e,t,n,o===0?Ki:a,l,c,u)}function qp(e,t,n){return ao(e,t,n,yl,oo(jp()),fr(e,2,-2),0)}function Ol(e,t,n,r){return ao(e,t,n,Yi,fr(e,0,r),fr(e,r+1,-1),r)}function Vn(e,t){for(var n="",r=Qi(e),o=0;o6)switch(we(e,t+1)){case 109:if(we(e,t+4)!==45)break;case 102:return X(e,/(.+:)(.+)-([^]+)/,"$1"+Y+"$2-$3$1"+ro+(we(e,t+3)==108?"$3":"$2-$3"))+e;case 115:return~Xi(e,"stretch")?Rl(X(e,"stretch","fill-available"),t)+e:e}break;case 4949:if(we(e,t+1)!==115)break;case 6444:switch(we(e,it(e)-3-(~Xi(e,"!important")&&10))){case 107:return X(e,":",":"+Y)+e;case 101:return X(e,/(.+:)([^;!]+)(;|!.+)?/,"$1"+Y+(we(e,14)===45?"inline-":"")+"box$3$1"+Y+"$2$3$1"+Pe+"$2box$3")+e}break;case 5936:switch(we(e,t+11)){case 114:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb")+e;case 108:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb-rl")+e;case 45:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"lr")+e}return Y+e+Pe+e+e}return e}var nm=function(t,n,r,o){if(t.length>-1&&!t.return)switch(t.type){case Yi:t.return=Rl(t.value,t.length);break;case xl:return Vn([gr(t,{value:X(t.value,"@","@"+Y)})],o);case Ki:if(t.length)return Mp(t.props,function(i){switch(zp(i,/(::plac\w+|:read-\w+)/)){case":read-only":case":read-write":return Vn([gr(t,{props:[X(i,/:(read-\w+)/,":"+ro+"$1")]})],o);case"::placeholder":return Vn([gr(t,{props:[X(i,/:(plac\w+)/,":"+Y+"input-$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,":"+ro+"$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,Pe+"input-$1")]})],o)}return""})}},rm=[nm],om=function(t){var n=t.key;if(n==="css"){var r=document.querySelectorAll("style[data-emotion]:not([data-s])");Array.prototype.forEach.call(r,function(p){var v=p.getAttribute("data-emotion");v.indexOf(" ")!==-1&&(document.head.appendChild(p),p.setAttribute("data-s",""))})}var o=t.stylisPlugins||rm,i={},s,a=[];s=t.container||document.head,Array.prototype.forEach.call(document.querySelectorAll('style[data-emotion^="'+n+' "]'),function(p){for(var v=p.getAttribute("data-emotion").split(" "),b=1;b0?we(_n,--Ae):0,An--,ue===10&&(An=1,so--),ue}function je(){return ue=Ae2||mr(ue)>3?"":" "}function Hp(e,t){for(;--t&&je()&&!(ue<48||ue>102||ue>57&&ue<65||ue>70&&ue<97););return pr(e,lo()+(t<6&&st()==32&&je()==32))}function Ji(e){for(;je();)switch(ue){case e:return Ae;case 34:case 39:e!==34&&e!==39&&Ji(ue);break;case 40:e===41&&Ji(e);break;case 92:je();break}return Ae}function Up(e,t){for(;je()&&e+ue!==57;)if(e+ue===84&&st()===47)break;return"/*"+pr(t,Ae-1)+"*"+oo(e===47?e:je())}function Gp(e){for(;!mr(st());)je();return pr(e,Ae)}function qp(e){return Sl(uo("",null,null,null,[""],e=wl(e),0,[0],e))}function uo(e,t,n,r,o,i,s,a,l){for(var c=0,u=0,h=s,d=0,m=0,f=0,p=1,v=1,b=1,y=0,x="",w=o,k=i,O=r,R=x;v;)switch(f=y,y=je()){case 40:if(f!=108&&we(R,h-1)==58){Xi(R+=X(co(y),"&","&\f"),"&\f")!=-1&&(b=-1);break}case 34:case 39:case 91:R+=co(y);break;case 9:case 10:case 13:case 32:R+=Wp(f);break;case 92:R+=Hp(lo()-1,7);continue;case 47:switch(st()){case 42:case 47:io(Kp(Up(je(),lo()),t,n),l);break;default:R+="/"}break;case 123*p:a[c++]=it(R)*b;case 125*p:case 59:case 0:switch(y){case 0:case 125:v=0;case 59+u:b==-1&&(R=X(R,/\f/g,"")),m>0&&it(R)-h&&io(m>32?Ol(R+";",r,n,h-1):Ol(X(R," ","")+";",r,n,h-2),l);break;case 59:R+=";";default:if(io(O=El(R,t,n,c,u,o,a,x,w=[],k=[],h),i),y===123)if(u===0)uo(R,t,O,O,w,i,h,a,k);else switch(d===99&&we(R,3)===110?100:d){case 100:case 108:case 109:case 115:uo(e,O,O,r&&io(El(e,O,O,0,0,o,a,x,o,w=[],h),k),o,k,h,a,r?w:k);break;default:uo(R,O,O,O,[""],k,0,a,k)}}c=u=m=0,p=b=1,x=R="",h=s;break;case 58:h=1+it(R),m=f;default:if(p<1){if(y==123)--p;else if(y==125&&p++==0&&Bp()==125)continue}switch(R+=oo(y),y*p){case 38:b=u>0?1:(R+="\f",-1);break;case 44:a[c++]=(it(R)-1)*b,b=1;break;case 64:st()===45&&(R+=co(je())),d=st(),u=h=it(x=R+=Gp(lo())),y++;break;case 45:f===45&&it(R)==2&&(p=0)}}return i}function El(e,t,n,r,o,i,s,a,l,c,u){for(var h=o-1,d=o===0?i:[""],m=Qi(d),f=0,p=0,v=0;f0?d[b]+" "+y:X(y,/&\f/g,d[b])))&&(l[v++]=x);return ao(e,t,n,o===0?Ki:a,l,c,u)}function Kp(e,t,n){return ao(e,t,n,yl,oo($p()),fr(e,2,-2),0)}function Ol(e,t,n,r){return ao(e,t,n,Yi,fr(e,0,r),fr(e,r+1,-1),r)}function Vn(e,t){for(var n="",r=Qi(e),o=0;o6)switch(we(e,t+1)){case 109:if(we(e,t+4)!==45)break;case 102:return X(e,/(.+:)(.+)-([^]+)/,"$1"+Y+"$2-$3$1"+ro+(we(e,t+3)==108?"$3":"$2-$3"))+e;case 115:return~Xi(e,"stretch")?Rl(X(e,"stretch","fill-available"),t)+e:e}break;case 4949:if(we(e,t+1)!==115)break;case 6444:switch(we(e,it(e)-3-(~Xi(e,"!important")&&10))){case 107:return X(e,":",":"+Y)+e;case 101:return X(e,/(.+:)([^;!]+)(;|!.+)?/,"$1"+Y+(we(e,14)===45?"inline-":"")+"box$3$1"+Y+"$2$3$1"+Pe+"$2box$3")+e}break;case 5936:switch(we(e,t+11)){case 114:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb")+e;case 108:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"tb-rl")+e;case 45:return Y+e+Pe+X(e,/[svh]\w+-[tblr]{2}/,"lr")+e}return Y+e+Pe+e+e}return e}var rm=function(t,n,r,o){if(t.length>-1&&!t.return)switch(t.type){case Yi:t.return=Rl(t.value,t.length);break;case xl:return Vn([gr(t,{value:X(t.value,"@","@"+Y)})],o);case Ki:if(t.length)return jp(t.props,function(i){switch(Mp(i,/(::plac\w+|:read-\w+)/)){case":read-only":case":read-write":return Vn([gr(t,{props:[X(i,/:(read-\w+)/,":"+ro+"$1")]})],o);case"::placeholder":return Vn([gr(t,{props:[X(i,/:(plac\w+)/,":"+Y+"input-$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,":"+ro+"$1")]}),gr(t,{props:[X(i,/:(plac\w+)/,Pe+"input-$1")]})],o)}return""})}},om=[rm],im=function(t){var n=t.key;if(n==="css"){var r=document.querySelectorAll("style[data-emotion]:not([data-s])");Array.prototype.forEach.call(r,function(p){var v=p.getAttribute("data-emotion");v.indexOf(" ")!==-1&&(document.head.appendChild(p),p.setAttribute("data-s",""))})}var o=t.stylisPlugins||om,i={},s,a=[];s=t.container||document.head,Array.prototype.forEach.call(document.querySelectorAll('style[data-emotion^="'+n+' "]'),function(p){for(var v=p.getAttribute("data-emotion").split(" "),b=1;b=4;++r,o-=4)n=e.charCodeAt(r)&255|(e.charCodeAt(++r)&255)<<8|(e.charCodeAt(++r)&255)<<16|(e.charCodeAt(++r)&255)<<24,n=(n&65535)*1540483477+((n>>>16)*59797<<16),n^=n>>>24,t=(n&65535)*1540483477+((n>>>16)*59797<<16)^(t&65535)*1540483477+((t>>>16)*59797<<16);switch(o){case 3:t^=(e.charCodeAt(r+2)&255)<<16;case 2:t^=(e.charCodeAt(r+1)&255)<<8;case 1:t^=e.charCodeAt(r)&255,t=(t&65535)*1540483477+((t>>>16)*59797<<16)}return t^=t>>>13,t=(t&65535)*1540483477+((t>>>16)*59797<<16),((t^t>>>15)>>>0).toString(36)}var um={animationIterationCount:1,aspectRatio:1,borderImageOutset:1,borderImageSlice:1,borderImageWidth:1,boxFlex:1,boxFlexGroup:1,boxOrdinalGroup:1,columnCount:1,columns:1,flex:1,flexGrow:1,flexPositive:1,flexShrink:1,flexNegative:1,flexOrder:1,gridRow:1,gridRowEnd:1,gridRowSpan:1,gridRowStart:1,gridColumn:1,gridColumnEnd:1,gridColumnSpan:1,gridColumnStart:1,msGridRow:1,msGridRowSpan:1,msGridColumn:1,msGridColumnSpan:1,fontWeight:1,lineHeight:1,opacity:1,order:1,orphans:1,scale:1,tabSize:1,widows:1,zIndex:1,zoom:1,WebkitLineClamp:1,fillOpacity:1,floodOpacity:1,stopOpacity:1,strokeDasharray:1,strokeDashoffset:1,strokeMiterlimit:1,strokeOpacity:1,strokeWidth:1},dm=/[A-Z]|^ms/g,hm=/_EMO_([^_]+?)_([^]*?)_EMO_/g,_l=function(t){return t.charCodeAt(1)===45},Vl=function(t){return t!=null&&typeof t!="boolean"},rs=bl(function(e){return _l(e)?e:e.replace(dm,"-$&").toLowerCase()}),Fl=function(t,n){switch(t){case"animation":case"animationName":if(typeof n=="string")return n.replace(hm,function(r,o,i){return at={name:o,styles:i,next:at},o})}return um[t]!==1&&!_l(t)&&typeof n=="number"&&n!==0?n+"px":n};function vr(e,t,n){if(n==null)return"";var r=n;if(r.__emotion_styles!==void 0)return r;switch(typeof n){case"boolean":return"";case"object":{var o=n;if(o.anim===1)return at={name:o.name,styles:o.styles,next:at},o.name;var i=n;if(i.styles!==void 0){var s=i.next;if(s!==void 0)for(;s!==void 0;)at={name:s.name,styles:s.styles,next:at},s=s.next;var a=i.styles+";";return a}return fm(e,t,n)}case"function":{if(e!==void 0){var l=at,c=n(e);return at=l,vr(e,t,c)}break}}var u=n;if(t==null)return u;var h=t[u];return h!==void 0?h:u}function fm(e,t,n){var r="";if(Array.isArray(n))for(var o=0;or?.(...n))}}const xm=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),km=/^on[A-Z]/;function br(...e){let t={};for(let n of e){for(let r in t){if(km.test(r)&&typeof t[r]=="function"&&typeof n[r]=="function"){t[r]=ym(t[r],n[r]);continue}if(r==="className"||r==="class"){t[r]=xm(t[r],n[r]);continue}if(r==="style"){t[r]=Object.assign({},t[r]??{},n[r]??{});continue}t[r]=n[r]!==void 0?n[r]:t[r]}for(let r in n)t[r]===void 0&&(t[r]=n[r])}return t}function Cm(e,t){if(e!=null){if(typeof e=="function"){e(t);return}try{e.current=t}catch{throw new Error(`Cannot assign value '${t}' to ref '${e}'`)}}}function wm(...e){return t=>{e.forEach(n=>{Cm(n,t)})}}function yr(e){const t=Object.assign({},e);for(let n in t)t[n]===void 0&&delete t[n];return t}const Ge=(...e)=>e.filter(Boolean).map(t=>t.trim()).join(" ");function Sm(e){return e.default||e}const _e=e=>e!=null&&typeof e=="object"&&!Array.isArray(e),lt=e=>typeof e=="string",cs=e=>typeof e=="function";function Em(e){const t=C.version;return!lt(t)||t.startsWith("18.")?e?.ref:e?.props?.ref}const Wl=(...e)=>{const t=e.reduce((n,r)=>(r?.forEach(o=>n.add(o)),n),new Set([]));return Array.from(t)};function Om(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Fn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??Om(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[Pm,ho]=Fn({name:"ChakraContext",strict:!0,providerName:""});function Rm(e){const{value:t,children:n}=e;return g.jsxs(Pm,{value:t,children:[!t._config.disableLayers&&g.jsx(Bl,{styles:t.layers.atRule}),g.jsx(Bl,{styles:t._global}),n]})}const Im=(e,t)=>{const n={},r={},o=Object.keys(e);for(const i of o)t(i)?r[i]=e[i]:n[i]=e[i];return[r,n]},Ln=(e,t)=>{const n=cs(t)?t:r=>t.includes(r);return Im(e,n)},Tm=new Set(["htmlWidth","htmlHeight","htmlSize","htmlTranslate"]);function Nm(e){return typeof e=="string"&&Tm.has(e)}function Am(e,t,n){const{css:r,isValidProperty:o}=ho(),{children:i,...s}=e,a=P.useMemo(()=>{const[d,m]=Ln(s,y=>n(y,t.variantKeys)),[f,p]=Ln(m,t.variantKeys),[v,b]=Ln(p,o);return{forwardedProps:d,variantProps:f,styleProps:v,elementProps:b}},[t.variantKeys,n,s,o]),{css:l,...c}=a.styleProps,u=P.useMemo(()=>{const d={...a.variantProps};return t.variantKeys.includes("colorPalette")||(d.colorPalette=s.colorPalette),t.variantKeys.includes("orientation")||(d.orientation=s.orientation),t(d)},[t,a.variantProps,s.colorPalette,s.orientation]);return{styles:P.useMemo(()=>r(u,..._m(l),c),[r,u,l,c]),props:{...a.forwardedProps,...a.elementProps,children:i}}}const _m=e=>(Array.isArray(e)?e:[e]).filter(Boolean).flat(),Vm=Sm(Ip),Fm=e=>e!=="theme",Lm=(e,t,n)=>{let r;if(t){const o=t.shouldForwardProp;r=e.__emotion_forwardProp&&o?i=>e.__emotion_forwardProp(i)&&o(i):o}return typeof r!="function"&&n&&(r=e.__emotion_forwardProp),r};let Dm=typeof document<"u";const Hl=({cache:e,serialized:t,isStringTag:n})=>{ts(e,t,n);const r=zl(()=>ns(e,t,n));if(!Dm&&r!==void 0){let o=t.name,i=t.next;for(;i!==void 0;)o=Ge(o,i.name),i=i.next;return g.jsx("style",{"data-emotion":Ge(e.key,o),dangerouslySetInnerHTML:{__html:r},nonce:e.sheet.nonce})}return null},Ul={path:["d"],text:["x","y"],circle:["cx","cy","r"],rect:["width","height","x","y","rx","ry"],ellipse:["cx","cy","rx","ry"],g:["transform"],stop:["offset","stopOpacity"]},zm=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),us=((e,t={},n={})=>{if(zm(Ul,e)){n.forwardProps||(n.forwardProps=[]);const c=Ul[e];n.forwardProps=Wl([...n.forwardProps,...c])}const r=e.__emotion_real===e,o=r&&e.__emotion_base||e;let i,s;n!==void 0&&(i=n.label,s=n.target);let a=[];const l=is((c,u,h)=>{const{cva:d,isValidProperty:m}=ho(),f=t.__cva__?t:d(t),p=Mm(e.__emotion_cva,f),v=z=>(W,J)=>z.includes(W)?!0:!J?.includes(W)&&!m(W);!n.shouldForwardProp&&n.forwardProps&&(n.shouldForwardProp=v(n.forwardProps));const b=(z,W)=>{const J=typeof e=="string"&&e.charCodeAt(0)>96?Vm:Fm,ee=!W?.includes(z)&&!m(z);return J(z)&&ee},y=Lm(e,n,r)||b,x=C.useMemo(()=>Object.assign({},n.defaultProps,yr(c)),[c]),{props:w,styles:k}=Am(x,p,y);let O="",R=[k],I=w;if(w.theme==null){I={};for(let z in w)I[z]=w[z];I.theme=C.useContext(ss)}typeof w.className=="string"?O=Al(u.registered,R,w.className):w.className!=null&&(O=Ge(O,w.className));const N=os(a.concat(R),u.registered,I);N.styles&&(O=Ge(O,`${u.key}-${N.name}`)),s!==void 0&&(O=Ge(O,s));const A=!y("as");let T=A&&w.as||o,S={};for(let z in w)if(!(A&&z==="as")){if(Nm(z)){const W=z.replace("html","").toLowerCase();S[W]=w[z];continue}y(z)&&(S[z]=w[z])}let _=O.trim();_?S.className=_:Reflect.deleteProperty(S,"className"),S.ref=h;const D=n.forwardAsChild||n.forwardProps?.includes("asChild");if(w.asChild&&!D){const z=C.isValidElement(w.children)?C.Children.only(w.children):C.Children.toArray(w.children).find(C.isValidElement);if(!z)throw new Error("[chakra-ui > factory] No valid child found");T=z.type,S.children=null,Reflect.deleteProperty(S,"asChild"),S=br(S,z.props),S.ref=wm(h,Em(z))}return S.as&&D?(S.as=void 0,g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{asChild:!0,...S,children:g.jsx(w.as,{children:S.children})})]})):g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{...S})]})});return l.displayName=i!==void 0?i:`chakra(${typeof o=="string"?o:o.displayName||o.name||"Component"})`,l.__emotion_real=l,l.__emotion_base=o,l.__emotion_forwardProp=n.shouldForwardProp,l.__emotion_cva=t,Object.defineProperty(l,"toString",{value(){return`.${s}`}}),l}).bind(),ds=new Map,ve=new Proxy(us,{apply(e,t,n){return us(...n)},get(e,t){return ds.has(t)||ds.set(t,us(t)),ds.get(t)}}),Mm=(e,t)=>e&&!t?e:!e&&t?t:e.merge(t),xr=ve("div");xr.displayName="Box";const jm=Object.freeze({}),$m=Object.freeze({});function Bm(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getRecipe(t):{});return r.cva(structuredClone(o))},[t,n,r])}const Wm=e=>e.charAt(0).toUpperCase()+e.slice(1);function Lt(e){const{key:t,recipe:n}=e,r=Wm(t||n.className||"Component"),[o,i]=Fn({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`});function s(c){const{unstyled:u,...h}=c,d=Bm({key:t,recipe:h.recipe||n}),[m,f]=P.useMemo(()=>d.splitVariantProps(h),[d,h]);return{styles:u?jm:d(m),className:d.className,props:f}}const a=(c,u)=>{const h=ve(c,{},u),d=P.forwardRef((m,f)=>{const p=i(),v=P.useMemo(()=>br(p,m),[m,p]),{styles:b,className:y,props:x}=s(v);return g.jsx(h,{...x,ref:f,css:[b,v.css],className:Ge(y,v.className)})});return d.displayName=c.displayName||c.name,d};function l(){return o}return{withContext:a,PropsProvider:o,withPropsProvider:l,usePropsContext:i,useRecipeResult:s}}function fo(e){return e==null?[]:Array.isArray(e)?e:[e]}var kr=e=>e[0],hs=e=>e[e.length-1],Hm=(e,t)=>e.indexOf(t)!==-1,Kt=(e,...t)=>e.concat(t),Yt=(e,...t)=>e.filter(n=>!t.includes(n)),Dn=e=>Array.from(new Set(e)),fs=(e,t)=>{const n=new Set(t);return e.filter(r=>!n.has(r))},zn=(e,t)=>Hm(e,t)?Yt(e,t):Kt(e,t);function Gl(e,t,n={}){const{step:r=1,loop:o=!0}=n,i=t+r,s=e.length,a=s-1;return t===-1?r>0?0:a:i<0?o?a:0:i>=s?o?0:t>s?s:t:i}function Um(e,t,n={}){return e[Gl(e,t,n)]}function Gm(e,t,n={}){const{step:r=1,loop:o=!0}=n;return Gl(e,t,{step:-r,loop:o})}function qm(e,t,n={}){return e[Gm(e,t,n)]}function ql(e,t){return e.reduce(([n,r],o)=>(t(o)?n.push(o):r.push(o),[n,r]),[[],[]])}var Kl=e=>e?.constructor.name==="Array",Km=(e,t)=>{if(e.length!==t.length)return!1;for(let n=0;n{if(Object.is(e,t))return!0;if(e==null&&t!=null||e!=null&&t==null)return!1;if(typeof e?.isEqual=="function"&&typeof t?.isEqual=="function")return e.isEqual(t);if(typeof e=="function"&&typeof t=="function")return e.toString()===t.toString();if(Kl(e)&&Kl(t))return Km(Array.from(e),Array.from(t));if(typeof e!="object"||typeof t!="object")return!1;const n=Object.keys(t??Object.create(null)),r=n.length;for(let o=0;oArray.isArray(e),Ym=e=>e===!0||e===!1,Yl=e=>e!=null&&typeof e=="object",Xt=e=>Yl(e)&&!Cr(e),go=e=>typeof e=="string",Qt=e=>typeof e=="function",Xm=e=>e==null,Dt=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),Qm=e=>Object.prototype.toString.call(e),Xl=Function.prototype.toString,Jm=Xl.call(Object),Zm=e=>{if(!Yl(e)||Qm(e)!="[object Object]"||n0(e))return!1;const t=Object.getPrototypeOf(e);if(t===null)return!0;const n=Dt(t,"constructor")&&t.constructor;return typeof n=="function"&&n instanceof n&&Xl.call(n)==Jm},e0=e=>typeof e=="object"&&e!==null&&"$$typeof"in e&&"props"in e,t0=e=>typeof e=="object"&&e!==null&&"__v_isVNode"in e,n0=e=>e0(e)||t0(e),po=(e,...t)=>(typeof e=="function"?e(...t):e)??void 0,r0=e=>e(),o0=()=>{},mo=(...e)=>(...t)=>{e.forEach(function(n){n?.(...t)})},i0=(()=>{let e=0;return()=>(e++,e.toString(36))})();function xt(e,t,...n){if(e in t){const o=t[e];return Qt(o)?o(...n):o}const r=new Error(`No matching key: ${JSON.stringify(e)} in ${JSON.stringify(Object.keys(t))}`);throw Error.captureStackTrace?.(r,xt),r}var Ql=(e,t)=>{try{return e()}catch(n){return n instanceof Error&&Error.captureStackTrace?.(n,Ql),t?.()}},{floor:Jl,abs:Zl,round:vo,min:s0,max:a0,pow:l0,sign:c0}=Math,gs=e=>Number.isNaN(e),zt=e=>gs(e)?0:e,ec=(e,t)=>(e%t+t)%t,u0=(e,t)=>(e%t+t)%t,d0=(e,t)=>zt(e)>=t,h0=(e,t)=>zt(e)<=t,f0=(e,t,n)=>{const r=zt(e),o=t==null||r>=t,i=n==null||r<=n;return o&&i},g0=(e,t,n)=>vo((zt(e)-t)/n)*n+t,Ve=(e,t,n)=>s0(a0(zt(e),t),n),p0=(e,t,n)=>(zt(e)-t)/(n-t),m0=(e,t,n,r)=>Ve(g0(e*(n-t)+t,t,r),t,n),tc=(e,t)=>{let n=e,r=t.toString(),o=r.indexOf("."),i=o>=0?r.length-o:0;if(i>0){let s=l0(10,i);n=vo(n*s)/s}return n},ps=(e,t)=>typeof t=="number"?Jl(e*t+.5)/t:vo(e),nc=(e,t,n,r)=>{const o=t!=null?Number(t):0,i=Number(n),s=(e-o)%r;let a=Zl(s)*2>=r?e+c0(s)*(r-Zl(s)):e-s;if(a=tc(a,r),!gs(o)&&ai){const l=Jl((i-o)/r),c=o+l*r;a=l<=0||c{const r=Math.pow(n,t);return vo(e*r)/r},rc=e=>{if(!Number.isFinite(e))return 0;let t=1,n=0;for(;Math.round(e*t)/t!==e;)t*=10,n+=1;return n},oc=(e,t,n)=>{let r=t==="+"?e+n:e-n;if(e%1!==0||n%1!==0){const o=10**Math.max(rc(e),rc(n));e=Math.round(e*o),n=Math.round(n*o),r=t==="+"?e+n:e-n,r/=o}return r},v0=(e,t)=>oc(zt(e),"+",t),b0=(e,t)=>oc(zt(e),"-",t);function bo(e){if(!Zm(e)||e===void 0)return e;const t=Reflect.ownKeys(e).filter(r=>typeof r=="string"),n={};for(const r of t){const o=e[r];o!==void 0&&(n[r]=bo(o))}return n}function y0(e,t=Object.is){let n={...e};const r=new Set,o=u=>(r.add(u),()=>r.delete(u)),i=()=>{r.forEach(u=>u())};return{subscribe:o,get:u=>n[u],set:(u,h)=>{t(n[u],h)||(n[u]=h,i())},update:u=>{let h=!1;for(const d in u){const m=u[d];m!==void 0&&!t(n[d],m)&&(n[d]=m,h=!0)}h&&i()},snapshot:()=>({...n})}}function wr(...e){e.length===1?e[0]:e[1],e.length===2&&e[0]}function ic(e,t){if(e==null)throw new Error(t())}function x0(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Mn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??x0(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[CT,sc]=Mn({name:"EnvironmentContext",hookName:"useEnvironmentContext",providerName:"",strict:!1,defaultValue:{getRootNode:()=>document,getDocument:()=>document,getWindow:()=>window}});function k0(e){if(!e)return;const t=e.selectionStart??0,n=e.selectionEnd??0;Math.abs(n-t)===0&&t===0&&e.setSelectionRange(e.value.length,e.value.length)}var ac=e=>Math.max(0,Math.min(1,e)),C0=(e,t)=>e.map((n,r)=>e[(Math.max(t,0)+r)%e.length]),lc=()=>{},yo=e=>typeof e=="object"&&e!==null,w0=2147483647,S0=1,E0=9,O0=11,Te=e=>yo(e)&&e.nodeType===S0&&typeof e.nodeName=="string",ms=e=>yo(e)&&e.nodeType===E0,P0=e=>yo(e)&&e===e.window,cc=e=>Te(e)?e.localName||"":"#document";function R0(e){return["html","body","#document"].includes(cc(e))}var I0=e=>yo(e)&&e.nodeType!==void 0,jn=e=>I0(e)&&e.nodeType===O0&&"host"in e,T0=e=>Te(e)&&e.localName==="input",N0=e=>!!e?.matches("a[href]"),A0=e=>Te(e)?e.offsetWidth>0||e.offsetHeight>0||e.getClientRects().length>0:!1;function _0(e){if(!e)return!1;const t=e.getRootNode();return Sr(t)===e}var V0=/(textarea|select)/;function uc(e){if(e==null||!Te(e))return!1;try{return T0(e)&&e.selectionStart!=null||V0.test(e.localName)||e.isContentEditable||e.getAttribute("contenteditable")==="true"||e.getAttribute("contenteditable")===""}catch{return!1}}function Jt(e,t){if(!e||!t||!Te(e)||!Te(t))return!1;const n=t.getRootNode?.();if(e===t||e.contains(t))return!0;if(n&&jn(n)){let r=t;for(;r;){if(e===r)return!0;r=r.parentNode||r.host}}return!1}function Fe(e){return ms(e)?e:P0(e)?e.document:e?.ownerDocument??document}function F0(e){return Fe(e).documentElement}function be(e){return jn(e)?be(e.host):ms(e)?e.defaultView??window:Te(e)?e.ownerDocument?.defaultView??window:window}function Sr(e){let t=e.activeElement;for(;t?.shadowRoot;){const n=t.shadowRoot.activeElement;if(!n||n===t)break;t=n}return t}function L0(e){if(cc(e)==="html")return e;const t=e.assignedSlot||e.parentNode||jn(e)&&e.host||F0(e);return jn(t)?t.host:t}var vs=new WeakMap;function xo(e){return vs.has(e)||vs.set(e,be(e).getComputedStyle(e)),vs.get(e)}var ko=()=>typeof document<"u";function D0(){return navigator.userAgentData?.platform??navigator.platform}function z0(){const e=navigator.userAgentData;return e&&Array.isArray(e.brands)?e.brands.map(({brand:t,version:n})=>`${t}/${n}`).join(" "):navigator.userAgent}var bs=e=>ko()&&e.test(D0()),dc=e=>ko()&&e.test(z0()),M0=e=>ko()&&e.test(navigator.vendor),hc=()=>ko()&&!!navigator.maxTouchPoints,j0=()=>bs(/^iPhone/i),$0=()=>bs(/^iPad/i)||wo()&&navigator.maxTouchPoints>1,Co=()=>j0()||$0(),B0=()=>wo()||Co(),wo=()=>bs(/^Mac/i),fc=()=>B0()&&M0(/apple/i),W0=()=>dc(/Firefox/i),H0=()=>dc(/Android/i);function U0(e){return e.composedPath?.()??e.nativeEvent?.composedPath?.()}function $e(e){return U0(e)?.[0]??e.target}function G0(e){return X0(e).isComposing||e.keyCode===229}function q0(e){return e.pointerType===""&&e.isTrusted?!0:H0()&&e.pointerType?e.type==="click"&&e.buttons===1:e.detail===0&&!e.pointerType}var K0=e=>e.button===2||wo()&&e.ctrlKey&&e.button===0,Y0=e=>"touches"in e&&e.touches.length>0;function X0(e){return e.nativeEvent??e}function gc(e,t="client"){const n=Y0(e)?e.touches[0]||e.changedTouches[0]:e;return{x:n[`${t}X`],y:n[`${t}Y`]}}var se=(e,t,n,r)=>{const o=typeof e=="function"?e():e;return o?.addEventListener(t,n,r),()=>{o?.removeEventListener(t,n,r)}};function Q0(e,t){const{type:n="HTMLInputElement",property:r="value"}=t,o=be(e)[n].prototype;return Object.getOwnPropertyDescriptor(o,r)??{}}function J0(e){if(e.localName==="input")return"HTMLInputElement";if(e.localName==="textarea")return"HTMLTextAreaElement";if(e.localName==="select")return"HTMLSelectElement"}function So(e,t,n="value"){if(!e)return;const r=J0(e);r&&Q0(e,{type:r,property:n}).set?.call(e,t),e.setAttribute(n,t)}function Z0(e,t){const{value:n,bubbles:r=!0}=t;if(!e)return;const o=be(e);e instanceof o.HTMLInputElement&&(So(e,`${n}`),e.dispatchEvent(new o.Event("input",{bubbles:r})))}function ev(e){return tv(e)?e.form:e.closest("form")}function tv(e){return e.matches("textarea, input, select, button")}function nv(e,t){if(!e)return;const n=ev(e),r=o=>{o.defaultPrevented||t()};return n?.addEventListener("reset",r,{passive:!0}),()=>n?.removeEventListener("reset",r)}function rv(e,t){const n=e?.closest("fieldset");if(!n)return;t(n.disabled);const r=be(n),o=new r.MutationObserver(()=>t(n.disabled));return o.observe(n,{attributes:!0,attributeFilter:["disabled"]}),()=>o.disconnect()}function ys(e,t){if(!e)return;const{onFieldsetDisabledChange:n,onFormReset:r}=t,o=[nv(e,r),rv(e,n)];return()=>o.forEach(i=>i?.())}var pc=e=>Te(e)&&e.tagName==="IFRAME",ov=e=>!Number.isNaN(parseInt(e.getAttribute("tabindex")||"0",10)),iv=e=>parseInt(e.getAttribute("tabindex")||"0",10)<0,xs="input:not([type='hidden']):not([disabled]), select:not([disabled]), textarea:not([disabled]), a[href], button:not([disabled]), [tabindex], iframe, object, embed, area[href], audio[controls], video[controls], [contenteditable]:not([contenteditable='false']), details > summary:first-of-type",mc=(e,t=!1)=>{if(!e)return[];const n=Array.from(e.querySelectorAll(xs));(t==!0||t=="if-empty"&&n.length===0)&&Te(e)&&Mt(e)&&n.unshift(e);const o=n.filter(Mt);return o.forEach((i,s)=>{if(pc(i)&&i.contentDocument){const a=i.contentDocument.body;o.splice(s,1,...mc(a))}}),o};function Mt(e){return!e||e.closest("[inert]")?!1:e.matches(xs)&&A0(e)}function ks(e,t){if(!e)return[];const r=Array.from(e.querySelectorAll(xs)).filter(Zt);return r.forEach((o,i)=>{if(pc(o)&&o.contentDocument){const s=o.contentDocument.body,a=ks(s);r.splice(i,1,...a)}}),r.length,r}function Zt(e){return e!=null&&e.tabIndex>0?!0:Mt(e)&&!iv(e)}function Er(e){return e.tabIndex<0&&(/^(audio|video|details)$/.test(e.localName)||uc(e))&&!ov(e)?0:e.tabIndex}function Cs(e){const{root:t,getInitialEl:n,filter:r,enabled:o=!0}=e;if(!o)return;let i=null;if(i||(i=typeof n=="function"?n():n),i||(i=t?.querySelector("[data-autofocus],[autofocus]")),!i){const s=ks(t);i=r?s.filter(r)[0]:s[0]}return i||t||void 0}function ws(e){const t=new Set;function n(r){const o=globalThis.requestAnimationFrame(r);t.add(()=>globalThis.cancelAnimationFrame(o))}return n(()=>n(e)),function(){t.forEach(o=>o())}}function G(e){let t;const n=globalThis.requestAnimationFrame(()=>{t=e()});return()=>{globalThis.cancelAnimationFrame(n),t?.()}}function sv(e,t,n){const r=G(()=>{e.removeEventListener(t,o,!0),n()}),o=()=>{r(),n()};return e.addEventListener(t,o,{once:!0,capture:!0}),r}function av(e,t){if(!e)return;const{attributes:n,callback:r}=t,o=e.ownerDocument.defaultView||window,i=new o.MutationObserver(s=>{for(const a of s)a.type==="attributes"&&a.attributeName&&n.includes(a.attributeName)&&r(a)});return i.observe(e,{attributes:!0,attributeFilter:n}),()=>i.disconnect()}function Eo(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(av(i,t))})),()=>{o.forEach(i=>i?.())}}function vc(e){const t=()=>{const n=be(e);e.dispatchEvent(new n.MouseEvent("click"))};W0()?sv(e,"keyup",t):queueMicrotask(t)}function Oo(e){const t=L0(e);return R0(t)?Fe(t).body:Te(t)&&Ss(t)?t:Oo(t)}function bc(e,t=[]){const n=Oo(e),r=n===e.ownerDocument.body,o=be(n);return r?t.concat(o,o.visualViewport||[],Ss(n)?n:[]):t.concat(n,bc(n,[]))}var lv=/auto|scroll|overlay|hidden|clip/,cv=new Set(["inline","contents"]);function Ss(e){const t=be(e),{overflow:n,overflowX:r,overflowY:o,display:i}=t.getComputedStyle(e);return lv.test(n+o+r)&&!cv.has(i)}function uv(e){return e.scrollHeight>e.clientHeight||e.scrollWidth>e.clientWidth}function Po(e,t){const{rootEl:n,...r}=t||{};!e||!n||!Ss(n)||!uv(n)||e.scrollIntoView(r)}function yc(e,t){const{left:n,top:r,width:o,height:i}=t.getBoundingClientRect(),s={x:e.x-n,y:e.y-r},a={x:ac(s.x/o),y:ac(s.y/i)};function l(c={}){const{dir:u="ltr",orientation:h="horizontal",inverted:d}=c,m=typeof d=="object"?d.x:d,f=typeof d=="object"?d.y:d;return h==="horizontal"?u==="rtl"||m?1-a.x:a.x:f?1-a.y:a.y}return{offset:s,percent:a,getPercentValue:l}}function dv(e,t){const n=e.body,r="pointerLockElement"in e||"mozPointerLockElement"in e,o=()=>!!e.pointerLockElement;function i(){}function s(l){o(),console.error("PointerLock error occurred:",l),e.exitPointerLock()}if(!r)return;try{n.requestPointerLock()}catch{}const a=[se(e,"pointerlockchange",i,!1),se(e,"pointerlockerror",s,!1)];return()=>{a.forEach(l=>l()),e.exitPointerLock()}}var $n="default",Es="",Ro=new WeakMap;function hv(e={}){const{target:t,doc:n}=e,r=n??document,o=r.documentElement;return Co()?($n==="default"&&(Es=o.style.webkitUserSelect,o.style.webkitUserSelect="none"),$n="disabled"):t&&(Ro.set(t,t.style.userSelect),t.style.userSelect="none"),()=>fv({target:t,doc:r})}function fv(e={}){const{target:t,doc:n}=e,o=(n??document).documentElement;if(Co()){if($n!=="disabled")return;$n="restoring",setTimeout(()=>{ws(()=>{$n==="restoring"&&(o.style.webkitUserSelect==="none"&&(o.style.webkitUserSelect=Es||""),Es="",$n="default")})},300)}else if(t&&Ro.has(t)){const i=Ro.get(t);t.style.userSelect==="none"&&(t.style.userSelect=i??""),t.getAttribute("style")===""&&t.removeAttribute("style"),Ro.delete(t)}}function xc(e={}){const{defer:t,target:n,...r}=e,o=t?G:s=>s(),i=[];return i.push(o(()=>{const s=typeof n=="function"?n():n;i.push(hv({...r,target:s}))})),()=>{i.forEach(s=>s?.())}}function gv(e,t){const{onPointerMove:n,onPointerUp:r}=t,o=a=>{const l=gc(a),c=Math.sqrt(l.x**2+l.y**2),u=a.pointerType==="touch"?10:5;if(!(c{const l=gc(a);r({point:l,event:a})},s=[se(e,"pointermove",o,!1),se(e,"pointerup",i,!1),se(e,"pointercancel",i,!1),se(e,"contextmenu",i,!1),xc({doc:e})];return()=>{s.forEach(a=>a())}}function Io(e,t){return Array.from(e?.querySelectorAll(t)??[])}function pv(e,t){return e?.querySelector(t)??null}var Os=e=>e.id;function mv(e,t,n=Os){return e.find(r=>n(r)===t)}function Ps(e,t,n=Os){const r=mv(e,t,n);return r?e.indexOf(r):-1}function vv(e,t,n=!0){let r=Ps(e,t);return r=n?(r+1)%e.length:Math.min(r+1,e.length-1),e[r]}function bv(e,t,n=!0){let r=Ps(e,t);return r===-1?n?e[e.length-1]:null:(r=n?(r-1+e.length)%e.length:Math.max(0,r-1),e[r])}var yv=e=>e.split("").map(t=>{const n=t.charCodeAt(0);return n>0&&n<128?t:n>=128&&n<=255?`/x${n.toString(16)}`.replace("/","\\"):""}).join("").trim(),xv=e=>yv(e.dataset?.valuetext??e.textContent??""),kv=(e,t)=>e.trim().toLowerCase().startsWith(t.toLowerCase());function Cv(e,t,n,r=Os){const o=n?Ps(e,n,r):-1;let i=n?C0(e,o):e;return t.length===1&&(i=i.filter(a=>r(a)!==n)),i.find(a=>kv(xv(a),t))}function To(e,t){if(!e)return lc;const n=Object.keys(t).reduce((r,o)=>(r[o]=e.style.getPropertyValue(o),r),{});return Object.assign(e.style,t),()=>{Object.assign(e.style,n),e.style.length===0&&e.removeAttribute("style")}}function wv(e,t,n){if(!e)return lc;const r=e.style.getPropertyValue(t);return e.style.setProperty(t,n),()=>{e.style.setProperty(t,r),e.style.length===0&&e.removeAttribute("style")}}function Sv(e,t){const{state:n,activeId:r,key:o,timeout:i=350,itemToId:s}=t,a=n.keysSoFar+o,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a;let u=e.slice();const h=Cv(u,c,r,s);function d(){clearTimeout(n.timer),n.timer=-1}function m(f){n.keysSoFar=f,d(),f!==""&&(n.timer=+setTimeout(()=>{m(""),d()},i))}return m(a),h}var Or=Object.assign(Sv,{defaultOptions:{keysSoFar:"",timer:-1},isValidEvent:Ev});function Ev(e){return e.key.length===1&&!e.ctrlKey&&!e.metaKey}function Ov(e,t,n){const{signal:r}=t;return[new Promise((s,a)=>{const l=setTimeout(()=>{a(new Error(`Timeout of ${n}ms exceeded`))},n);r.addEventListener("abort",()=>{clearTimeout(l),a(new Error("Promise aborted"))}),e.then(c=>{r.aborted||(clearTimeout(l),s(c))}).catch(c=>{r.aborted||(clearTimeout(l),a(c))})}),()=>t.abort()]}function Pv(e,t){const{timeout:n,rootNode:r}=t,o=be(r),i=Fe(r),s=new o.AbortController;return Ov(new Promise(a=>{const l=e();if(l){a(l);return}const c=new o.MutationObserver(()=>{const u=e();u&&u.isConnected&&(c.disconnect(),a(u))});c.observe(i.body,{childList:!0,subtree:!0})}),s,n)}var Rv=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),Iv=/((?:--)?(?:\w+-?)+)\s*:\s*([^;]*)/g,kc=e=>{const t={};let n;for(;n=Iv.exec(e);)t[n[1]]=n[2];return t},Tv=(e,t)=>{if(go(e)){if(go(t))return`${e};${t}`;e=kc(e)}else go(t)&&(t=kc(t));return Object.assign({},e??{},t??{})};function qe(...e){let t={};for(let n of e){if(!n)continue;for(let o in t){if(o.startsWith("on")&&typeof t[o]=="function"&&typeof n[o]=="function"){t[o]=mo(n[o],t[o]);continue}if(o==="className"||o==="class"){t[o]=Rv(t[o],n[o]);continue}if(o==="style"){t[o]=Tv(t[o],n[o]);continue}t[o]=n[o]!==void 0?n[o]:t[o]}for(let o in n)t[o]===void 0&&(t[o]=n[o]);const r=Object.getOwnPropertySymbols(n);for(let o of r)t[o]=n[o]}return t}function Cc(e,t,n){let r=[],o;return i=>{const s=e(i);return(s.length!==r.length||s.some((l,c)=>!Ze(r[c],l)))&&(r=s,o=t(s,i)),o}}function jt(){return{and:(...e)=>function(n){return e.every(r=>n.guard(r))},or:(...e)=>function(n){return e.some(r=>n.guard(r))},not:e=>function(n){return!n.guard(e)}}}function wT(e){return e}function wc(){return{guards:jt(),createMachine:e=>e,choose:e=>function({choose:n}){return n(e)?.actions}}}var Bn=(e=>(e.NotStarted="Not Started",e.Started="Started",e.Stopped="Stopped",e))(Bn||{}),Rs="__init__";function Nv(e){const t=()=>e.getRootNode?.()??document,n=()=>Fe(t());return{...e,getRootNode:t,getDoc:n,getWin:()=>n().defaultView??window,getActiveElement:()=>Sr(t()),isActiveElement:_0,getById:s=>t().getElementById(s)}}function No(...e){return t=>{const n=[];for(const r of e)if(typeof r=="function"){const o=r(t);typeof o=="function"&&n.push(o)}else r&&(r.current=t);if(n.length)return()=>{for(const r of n)r()}}}function Av(e){let t=Object.getOwnPropertyDescriptor(e.props,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning;return n?e.ref:(t=Object.getOwnPropertyDescriptor(e,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning,n?e.props.ref:e.props.ref||e.ref)}const Is=e=>{const t=P.memo(P.forwardRef((n,r)=>{const{asChild:o,children:i,...s}=n;if(!o)return P.createElement(e,{...s,ref:r},i);if(!P.isValidElement(i))return null;const a=P.Children.only(i),l=Av(a);return P.cloneElement(a,{...qe(s,a.props),ref:r?No(r,l):l})}));return t.displayName=e.displayName||e.name,t},kt=(()=>{const e=new Map;return new Proxy(Is,{apply(t,n,r){return Is(r[0])},get(t,n){const r=n;return e.has(r)||e.set(r,Is(r)),e.get(r)}})})(),[ET,_v]=Mn({name:"LocaleContext",hookName:"useLocaleContext",providerName:"",strict:!1,defaultValue:{dir:"ltr",locale:"en-US"}}),Sc=()=>(e,t)=>t.reduce((n,r)=>{const[o,i]=n,s=r;return i[s]!==void 0&&(o[s]=i[s]),delete i[s],[o,i]},[{},{...e}]),Ec=e=>Sc()(e,["immediate","lazyMount","onExitComplete","present","skipAnimationOnMount","unmountOnExit"]);function Vv(e){return new Proxy({},{get(t,n){return n==="style"?r=>e({style:r}).style:e}})}var M=()=>e=>Array.from(new Set(e));function Fv(e,t){const{state:n,send:r,context:o}=e,i=n.matches("mounted","unmountSuspended");return{skip:!o.get("initial"),present:i,setNode(s){s&&r({type:"NODE.SET",node:s})},unmount(){r({type:"UNMOUNT"})}}}var Lv={props({props:e}){return{...e,present:!!e.present}},initialState({prop:e}){return e("present")?"mounted":"unmounted"},refs(){return{node:null,styles:null}},context({bindable:e}){return{unmountAnimationName:e(()=>({defaultValue:null})),prevAnimationName:e(()=>({defaultValue:null})),present:e(()=>({defaultValue:!1})),initial:e(()=>({sync:!0,defaultValue:!1}))}},exit:["clearInitial","cleanupNode"],watch({track:e,prop:t,send:n}){e([()=>t("present")],()=>{n({type:"PRESENCE.CHANGED"})})},on:{"NODE.SET":{actions:["setupNode"]},"PRESENCE.CHANGED":{actions:["setInitial","syncPresence"]}},states:{mounted:{on:{UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]},"UNMOUNT.SUSPEND":{target:"unmountSuspended"}}},unmountSuspended:{effects:["trackAnimationEvents"],on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]},UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]}}},unmounted:{on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]}}}},implementations:{actions:{setInitial:({context:e})=>{e.get("initial")||queueMicrotask(()=>{e.set("initial",!0)})},clearInitial:({context:e})=>{e.set("initial",!1)},invokeOnExitComplete:({prop:e})=>{e("onExitComplete")?.()},setupNode:({refs:e,event:t})=>{e.get("node")!==t.node&&(e.set("node",t.node),e.set("styles",xo(t.node)))},cleanupNode:({refs:e})=>{e.set("node",null),e.set("styles",null)},syncPresence:({context:e,refs:t,send:n,prop:r})=>{const o=r("present");if(o)return n({type:"MOUNT",src:"presence.changed"});const i=t.get("node");if(!o&&i?.ownerDocument.visibilityState==="hidden")return n({type:"UNMOUNT",src:"visibilitychange"});G(()=>{const s=Ao(t.get("styles"));e.set("unmountAnimationName",s),s==="none"||s===e.get("prevAnimationName")||t.get("styles")?.display==="none"||t.get("styles")?.animationDuration==="0s"?n({type:"UNMOUNT",src:"presence.changed"}):n({type:"UNMOUNT.SUSPEND"})})},setPrevAnimationName:({context:e,refs:t})=>{G(()=>{e.set("prevAnimationName",Ao(t.get("styles")))})},clearPrevAnimationName:({context:e})=>{e.set("prevAnimationName",null)}},effects:{trackAnimationEvents:({context:e,refs:t,send:n})=>{const r=t.get("node");if(!r)return;const o=a=>{(a.composedPath?.()?.[0]??a.target)===r&&e.set("prevAnimationName",Ao(t.get("styles")))},i=a=>{const l=Ao(t.get("styles"));$e(a)===r&&l===e.get("unmountAnimationName")&&n({type:"UNMOUNT",src:"animationend"})};r.addEventListener("animationstart",o),r.addEventListener("animationcancel",i),r.addEventListener("animationend",i);const s=To(r,{animationFillMode:"forwards"});return()=>{r.removeEventListener("animationstart",o),r.removeEventListener("animationcancel",i),r.removeEventListener("animationend",i),ws(()=>s())}}}}};function Ao(e){return e?.animationName||"none"}M()(["onExitComplete","present","immediate"]);var Oc=typeof globalThis.document<"u"?P.useLayoutEffect:P.useEffect;function _o(e){const t=e().value??e().defaultValue,n=e().isEqual??Object.is,[r]=P.useState(t),[o,i]=P.useState(r),s=e().value!==void 0,a=P.useRef(o);a.current=s?e().value:o;const l=P.useRef(a.current);Oc(()=>{l.current=a.current},[o,e().value]);const c=h=>{const d=l.current,m=Qt(h)?h(d):h;e().debug&&console.log(`[bindable > ${e().debug}] setValue`,{next:m,prev:d}),s||i(m),n(m,d)||e().onChange?.(m,d)};function u(){return s?e().value:o}return{initial:r,ref:a,get:u,set(h){(e().sync?Ft.flushSync:r0)(()=>c(h))},invoke(h,d){e().onChange?.(h,d)},hash(h){return e().hash?.(h)??String(h)}}}_o.cleanup=e=>{P.useEffect(()=>e,[])},_o.ref=e=>{const t=P.useRef(e);return{get:()=>t.current,set:n=>{t.current=n}}};function Dv(e){const t=P.useRef(e);return{get(n){return t.current[n]},set(n,r){t.current[n]=r}}}var zv=(e,t)=>{const n=P.useRef(!1),r=P.useRef(!1);P.useEffect(()=>{if(n.current&&r.current)return t();r.current=!0},[...(e??[]).map(o=>typeof o=="function"?o():o)]),P.useEffect(()=>(n.current=!0,()=>{n.current=!1}),[])};function Pc(e,t={}){const n=P.useMemo(()=>{const{id:T,ids:S,getRootNode:_}=t;return Nv({id:T,ids:S,getRootNode:_})},[t]),r=(...T)=>{e.debug&&console.log(...T)},o=e.props?.({props:bo(t),scope:n})??t,i=Mv(o),s=e.context?.({prop:i,bindable:_o,scope:n,flush:Ic,getContext(){return l},getComputed(){return k},getRefs(){return p},getEvent(){return m()}}),a=Rc(s),l={get(T){return a.current?.[T].ref.current},set(T,S){a.current?.[T].set(S)},initial(T){return a.current?.[T].initial},hash(T){const S=a.current?.[T].get();return a.current?.[T].hash(S)}},c=P.useRef(new Map),u=P.useRef(null),h=P.useRef(null),d=P.useRef({type:""}),m=()=>({...d.current,current(){return d.current},previous(){return h.current}}),f=()=>({...O,matches(...T){return T.includes(O.ref.current)},hasTag(T){return!!e.states[O.ref.current]?.tags?.includes(T)}}),p=Dv(e.refs?.({prop:i,context:l})??{}),v=()=>({state:f(),context:l,event:m(),prop:i,send:A,action:b,guard:y,track:zv,refs:p,computed:k,flush:Ic,scope:n,choose:w}),b=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(D=>{const z=e.implementations?.actions?.[D];return z||wr(`[zag-js] No implementation found for action "${JSON.stringify(D)}"`),z});for(const D of _)D?.(v())},y=T=>Qt(T)?T(v()):e.implementations?.guards?.[T](v()),x=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(z=>{const W=e.implementations?.effects?.[z];return W||wr(`[zag-js] No implementation found for effect "${JSON.stringify(z)}"`),W}),D=[];for(const z of _){const W=z?.(v());W&&D.push(W)}return()=>D.forEach(z=>z?.())},w=T=>fo(T).find(S=>{let _=!S.guard;return go(S.guard)?_=!!y(S.guard):Qt(S.guard)&&(_=S.guard(v())),_}),k=T=>{ic(e.computed,()=>"[zag-js] No computed object found on machine");const S=e.computed[T];return S({context:l,event:m(),prop:i,refs:p,scope:n,computed:k})},O=_o(()=>({defaultValue:e.initialState({prop:i}),onChange(T,S){S&&(c.current.get(S)?.(),c.current.delete(S)),S&&b(e.states[S]?.exit),b(u.current?.actions);const _=x(e.states[T]?.effects);if(_&&c.current.set(T,_),S===Rs){b(e.entry);const D=x(e.effects);D&&c.current.set(Rs,D)}b(e.states[T]?.entry)}})),R=P.useRef(void 0),I=P.useRef(Bn.NotStarted);Oc(()=>{queueMicrotask(()=>{const _=I.current===Bn.Started;I.current=Bn.Started,r(_?"rehydrating...":"initializing...");const D=R.current??O.initial;O.invoke(D,_?O.get():Rs)});const T=c.current,S=O.ref.current;return()=>{r("unmounting..."),R.current=S,I.current=Bn.Stopped,T.forEach(_=>_?.()),c.current=new Map,u.current=null,queueMicrotask(()=>{b(e.exit)})}},[]);const N=()=>"ref"in O?O.ref.current:O.get(),A=T=>{queueMicrotask(()=>{if(I.current!==Bn.Started)return;h.current=d.current,d.current=T;let S=N();const _=e.states[S].on?.[T.type]??e.on?.[T.type],D=w(_);if(!D)return;u.current=D;const z=D.target??S;r("transition",T.type,D.target||S,`(${D.actions})`);const W=z!==S;W?Ft.flushSync(()=>O.set(z)):D.reenter&&!W?O.invoke(S,S):b(D.actions??[])})};return e.watch?.(v()),{state:f(),send:A,context:l,prop:i,scope:n,refs:p,computed:k,event:m(),getStatus:()=>I.current}}function Rc(e){const t=P.useRef(e);return t.current=e,t}function Mv(e){const t=Rc(e);return function(r){return t.current[r]}}function Ic(e){queueMicrotask(()=>{Ft.flushSync(()=>e())})}var jv=Vv(e=>e);function $v(e,t={}){const{sync:n=!1}=t,r=Bv(e);return P.useCallback((...o)=>n?queueMicrotask(()=>r.current?.(...o)):r.current?.(...o),[n,r])}function Bv(e){const t=P.useRef(e);return t.current=e,t}const Ts=(e={})=>{const{lazyMount:t,unmountOnExit:n,present:r,skipAnimationOnMount:o=!1,...i}=e,s=P.useRef(!1),a={...i,present:r,onExitComplete:$v(e.onExitComplete)},l=Pc(Lv,a),c=Fv(l);c.present&&(s.current=!0);const u=!c.present&&!s.current&&t||n&&!c.present&&s.current,h=()=>({"data-state":c.skip&&o?void 0:r?"open":"closed",hidden:!c.present});return{ref:c.setNode,getPresenceProps:h,present:c.present,unmounted:u}},[Tc,Ns]=Mn({name:"PresenceContext",hookName:"usePresenceContext",providerName:""}),Vo=ve("span"),{withContext:Wv}=Lt({key:"text"}),Le=Wv("p");function Nc(e,t=[]){const n=P.useRef(()=>{throw new Error("Cannot call an event handler while rendering.")});return P.useInsertionEffect(()=>{n.current=e}),P.useCallback((...r)=>n.current?.(...r),t)}function en(e={}){const t=Nc(e.onOpen),n=Nc(e.onClose),[r,o]=P.useState(e.defaultOpen||!1),i=e.open!==void 0?e.open:r,s=e.open!==void 0,a=P.useCallback(()=>{s||o(!1),n?.()},[s,n]),l=P.useCallback(()=>{s||o(!0),t?.()},[s,t]),c=P.useCallback(()=>{i?a():l()},[i,l,a]);return{open:i,onOpen:l,onClose:a,onToggle:c,setOpen:o}}var j=(e,t=[])=>({parts:(...n)=>{if(Hv(t))return j(e,n);throw new Error("createAnatomy().parts(...) should only be called once. Did you mean to use .extendWith(...) ?")},extendWith:(...n)=>j(e,[...t,...n]),omit:(...n)=>j(e,t.filter(r=>!n.includes(r))),rename:n=>j(n,t),keys:()=>t,build:()=>[...new Set(t)].reduce((n,r)=>Object.assign(n,{[r]:{selector:[`&[data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`,`& [data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`].join(", "),attrs:{"data-scope":Wn(e),"data-part":Wn(r)}}}),{})}),Wn=e=>e.replace(/([A-Z])([A-Z])/g,"$1-$2").replace(/([a-z])([A-Z])/g,"$1-$2").replace(/[\s_]+/g,"-").toLowerCase(),Hv=e=>e.length===0,Ac=j("collapsible").parts("root","trigger","content","indicator");Ac.build(),M()(["dir","disabled","getRootNode","id","ids","onExitComplete","onOpenChange","defaultOpen","open"]);var Uv=Object.defineProperty,Gv=(e,t,n)=>t in e?Uv(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,As=(e,t,n)=>Gv(e,t+"",n),qv=(e,t)=>{if(Object.keys(e).length!==Object.keys(t).length)return!1;for(let n in e)if(e[n]!==t[n])return!1;return!0},_s=class{toHexInt(){return this.toFormat("rgba").toHexInt()}getChannelValue(e){if(e in this)return this[e];throw new Error("Unsupported color channel: "+e)}getChannelValuePercent(e,t){const n=t??this.getChannelValue(e),{minValue:r,maxValue:o}=this.getChannelRange(e);return p0(n,r,o)}getChannelPercentValue(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=m0(t,n,r,o);return nc(i,n,r,o)}withChannelValue(e,t){const{minValue:n,maxValue:r}=this.getChannelRange(e);if(e in this){let o=this.clone();return o[e]=Ve(t,n,r),o}throw new Error("Unsupported color channel: "+e)}getColorAxes(e){let{xChannel:t,yChannel:n}=e,r=t||this.getChannels().find(s=>s!==n),o=n||this.getChannels().find(s=>s!==r),i=this.getChannels().find(s=>s!==r&&s!==o);return{xChannel:r,yChannel:o,zChannel:i}}incrementChannel(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=nc(Ve(this.getChannelValue(e)+t,n,r),n,r,o);return this.withChannelValue(e,i)}decrementChannel(e,t){return this.incrementChannel(e,-t)}isEqual(e){return qv(this.toJSON(),e.toJSON())&&this.getChannelValue("alpha")===e.getChannelValue("alpha")}},Kv=/^#[\da-f]+$/i,Yv=/^rgba?\((.*)\)$/,Xv=/[^#]/gi,_c=class Hi extends _s{constructor(t,n,r,o){super(),this.red=t,this.green=n,this.blue=r,this.alpha=o}static parse(t){let n=[];if(Kv.test(t)&&[4,5,7,9].includes(t.length)){const o=(t.length<6?t.replace(Xv,"$&$&"):t).slice(1).split("");for(;o.length>0;)n.push(parseInt(o.splice(0,2).join(""),16));n[3]=n[3]!==void 0?n[3]/255:void 0}const r=t.match(Yv);return r?.[1]&&(n=r[1].split(",").map(o=>Number(o.trim())).map((o,i)=>Ve(o,0,i<3?255:1))),n.length<3?void 0:new Hi(n[0],n[1],n[2],n[3]??1)}toString(t){switch(t){case"hex":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")).toUpperCase();case"hexa":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")+Math.round(this.alpha*255).toString(16).padStart(2,"0")).toUpperCase();case"rgb":return`rgb(${this.red}, ${this.green}, ${this.blue})`;case"css":case"rgba":return`rgba(${this.red}, ${this.green}, ${this.blue}, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"hsb":return this.toHSB().toString("hsb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"rgba":return this;case"hsba":return this.toHSB();case"hsla":return this.toHSL();default:throw new Error("Unsupported color conversion: rgb -> "+t)}}toHexInt(){return this.red<<16|this.green<<8|this.blue}toHSB(){const t=this.red/255,n=this.green/255,r=this.blue/255,o=Math.min(t,n,r),i=Math.max(t,n,r),s=i-o,a=i===0?0:s/i;let l=0;if(s!==0){switch(i){case t:l=(n-r)/s+(nNumber(a.trim().replace("%","")));return new Ui(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsl":return`hsl(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%)`;case"css":case"hsla":return`hsla(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%, ${this.alpha})`;case"hsb":return this.toHSB().toString("hsb");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsla":return this;case"hsba":return this.toHSB();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsl -> "+t)}}toHSB(){let t=this.saturation/100,n=this.lightness/100,r=n+t*Math.min(n,1-n);return t=r===0?0:2*(1-n/r),new Ls(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.lightness/100,o=n*Math.min(r,1-r),i=(s,a=(s+t/30)%12)=>r-o*Math.max(Math.min(a-3,9-a,1),-1);return new Vs(Math.round(i(0)*255),Math.round(i(8)*255),Math.round(i(4)*255),oe(this.alpha,2))}clone(){return new Ui(this.hue,this.saturation,this.lightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"lightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="lightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"lightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,l:this.lightness,a:this.alpha}}getFormat(){return"hsla"}getChannels(){return Ui.colorChannels}};As(Vc,"colorChannels",["hue","saturation","lightness"]);var Fs=Vc,Jv=/hsb\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%)\)|hsba\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d(.\d+)?)\)/,Fc=class Gi extends _s{constructor(t,n,r,o){super(),this.hue=t,this.saturation=n,this.brightness=r,this.alpha=o}static parse(t){let n;if(n=t.match(Jv)){const[r,o,i,s]=(n[1]??n[2]).split(",").map(a=>Number(a.trim().replace("%","")));return new Gi(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"css":return this.toHSL().toString("css");case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsb":return`hsb(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%)`;case"hsba":return`hsba(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsba":return this;case"hsla":return this.toHSL();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsb -> "+t)}}toHSL(){let t=this.saturation/100,n=this.brightness/100,r=n*(1-t/2);return t=r===0||r===1?0:(n-r)/Math.min(r,1-r),new Fs(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.brightness/100,o=(i,s=(i+t/60)%6)=>r-n*r*Math.max(Math.min(s,4-s,1),0);return new Vs(Math.round(o(5)*255),Math.round(o(3)*255),Math.round(o(1)*255),oe(this.alpha,2))}clone(){return new Gi(this.hue,this.saturation,this.brightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"brightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="brightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"brightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,b:this.brightness,a:this.alpha}}getFormat(){return"hsba"}getChannels(){return Gi.colorChannels}};As(Fc,"colorChannels",["hue","saturation","brightness"]);var Ls=Fc,Zv="aliceblue:f0f8ff,antiquewhite:faebd7,aqua:00ffff,aquamarine:7fffd4,azure:f0ffff,beige:f5f5dc,bisque:ffe4c4,black:000000,blanchedalmond:ffebcd,blue:0000ff,blueviolet:8a2be2,brown:a52a2a,burlywood:deb887,cadetblue:5f9ea0,chartreuse:7fff00,chocolate:d2691e,coral:ff7f50,cornflowerblue:6495ed,cornsilk:fff8dc,crimson:dc143c,cyan:00ffff,darkblue:00008b,darkcyan:008b8b,darkgoldenrod:b8860b,darkgray:a9a9a9,darkgreen:006400,darkkhaki:bdb76b,darkmagenta:8b008b,darkolivegreen:556b2f,darkorange:ff8c00,darkorchid:9932cc,darkred:8b0000,darksalmon:e9967a,darkseagreen:8fbc8f,darkslateblue:483d8b,darkslategray:2f4f4f,darkturquoise:00ced1,darkviolet:9400d3,deeppink:ff1493,deepskyblue:00bfff,dimgray:696969,dodgerblue:1e90ff,firebrick:b22222,floralwhite:fffaf0,forestgreen:228b22,fuchsia:ff00ff,gainsboro:dcdcdc,ghostwhite:f8f8ff,gold:ffd700,goldenrod:daa520,gray:808080,green:008000,greenyellow:adff2f,honeydew:f0fff0,hotpink:ff69b4,indianred:cd5c5c,indigo:4b0082,ivory:fffff0,khaki:f0e68c,lavender:e6e6fa,lavenderblush:fff0f5,lawngreen:7cfc00,lemonchiffon:fffacd,lightblue:add8e6,lightcoral:f08080,lightcyan:e0ffff,lightgoldenrodyellow:fafad2,lightgrey:d3d3d3,lightgreen:90ee90,lightpink:ffb6c1,lightsalmon:ffa07a,lightseagreen:20b2aa,lightskyblue:87cefa,lightslategray:778899,lightsteelblue:b0c4de,lightyellow:ffffe0,lime:00ff00,limegreen:32cd32,linen:faf0e6,magenta:ff00ff,maroon:800000,mediumaquamarine:66cdaa,mediumblue:0000cd,mediumorchid:ba55d3,mediumpurple:9370d8,mediumseagreen:3cb371,mediumslateblue:7b68ee,mediumspringgreen:00fa9a,mediumturquoise:48d1cc,mediumvioletred:c71585,midnightblue:191970,mintcream:f5fffa,mistyrose:ffe4e1,moccasin:ffe4b5,navajowhite:ffdead,navy:000080,oldlace:fdf5e6,olive:808000,olivedrab:6b8e23,orange:ffa500,orangered:ff4500,orchid:da70d6,palegoldenrod:eee8aa,palegreen:98fb98,paleturquoise:afeeee,palevioletred:d87093,papayawhip:ffefd5,peachpuff:ffdab9,peru:cd853f,pink:ffc0cb,plum:dda0dd,powderblue:b0e0e6,purple:800080,rebeccapurple:663399,red:ff0000,rosybrown:bc8f8f,royalblue:4169e1,saddlebrown:8b4513,salmon:fa8072,sandybrown:f4a460,seagreen:2e8b57,seashell:fff5ee,sienna:a0522d,silver:c0c0c0,skyblue:87ceeb,slateblue:6a5acd,slategray:708090,snow:fffafa,springgreen:00ff7f,steelblue:4682b4,tan:d2b48c,teal:008080,thistle:d8bfd8,tomato:ff6347,turquoise:40e0d0,violet:ee82ee,wheat:f5deb3,white:ffffff,whitesmoke:f5f5f5,yellow:ffff00,yellowgreen:9acd32",eb=e=>{const t=new Map,n=e.split(",");for(let r=0;r{if(Lc.has(e))return Fo(Lc.get(e));const t=Vs.parse(e)||Ls.parse(e)||Fs.parse(e);if(!t){const n=new Error("Invalid color value: "+e);throw Error.captureStackTrace?.(n,Fo),n}return t};const tb=["top","right","bottom","left"],$t=Math.min,Be=Math.max,Lo=Math.round,Do=Math.floor,ct=e=>({x:e,y:e}),nb={left:"right",right:"left",bottom:"top",top:"bottom"},rb={start:"end",end:"start"};function Ds(e,t,n){return Be(e,$t(t,n))}function Ct(e,t){return typeof e=="function"?e(t):e}function wt(e){return e.split("-")[0]}function Hn(e){return e.split("-")[1]}function zs(e){return e==="x"?"y":"x"}function Ms(e){return e==="y"?"height":"width"}const ob=new Set(["top","bottom"]);function ut(e){return ob.has(wt(e))?"y":"x"}function js(e){return zs(ut(e))}function ib(e,t,n){n===void 0&&(n=!1);const r=Hn(e),o=js(e),i=Ms(o);let s=o==="x"?r===(n?"end":"start")?"right":"left":r==="start"?"bottom":"top";return t.reference[i]>t.floating[i]&&(s=zo(s)),[s,zo(s)]}function sb(e){const t=zo(e);return[$s(e),t,$s(t)]}function $s(e){return e.replace(/start|end/g,t=>rb[t])}const Dc=["left","right"],zc=["right","left"],ab=["top","bottom"],lb=["bottom","top"];function cb(e,t,n){switch(e){case"top":case"bottom":return n?t?zc:Dc:t?Dc:zc;case"left":case"right":return t?ab:lb;default:return[]}}function ub(e,t,n,r){const o=Hn(e);let i=cb(wt(e),n==="start",r);return o&&(i=i.map(s=>s+"-"+o),t&&(i=i.concat(i.map($s)))),i}function zo(e){return e.replace(/left|right|bottom|top/g,t=>nb[t])}function db(e){return{top:0,right:0,bottom:0,left:0,...e}}function Mc(e){return typeof e!="number"?db(e):{top:e,right:e,bottom:e,left:e}}function Mo(e){const{x:t,y:n,width:r,height:o}=e;return{width:r,height:o,top:n,left:t,right:t+r,bottom:n+o,x:t,y:n}}function jc(e,t,n){let{reference:r,floating:o}=e;const i=ut(t),s=js(t),a=Ms(s),l=wt(t),c=i==="y",u=r.x+r.width/2-o.width/2,h=r.y+r.height/2-o.height/2,d=r[a]/2-o[a]/2;let m;switch(l){case"top":m={x:u,y:r.y-o.height};break;case"bottom":m={x:u,y:r.y+r.height};break;case"right":m={x:r.x+r.width,y:h};break;case"left":m={x:r.x-o.width,y:h};break;default:m={x:r.x,y:r.y}}switch(Hn(t)){case"start":m[s]-=d*(n&&c?-1:1);break;case"end":m[s]+=d*(n&&c?-1:1);break}return m}const hb=async(e,t,n)=>{const{placement:r="bottom",strategy:o="absolute",middleware:i=[],platform:s}=n,a=i.filter(Boolean),l=await(s.isRTL==null?void 0:s.isRTL(t));let c=await s.getElementRects({reference:e,floating:t,strategy:o}),{x:u,y:h}=jc(c,r,l),d=r,m={},f=0;for(let p=0;p({name:"arrow",options:e,async fn(t){const{x:n,y:r,placement:o,rects:i,platform:s,elements:a,middlewareData:l}=t,{element:c,padding:u=0}=Ct(e,t)||{};if(c==null)return{};const h=Mc(u),d={x:n,y:r},m=js(o),f=Ms(m),p=await s.getDimensions(c),v=m==="y",b=v?"top":"left",y=v?"bottom":"right",x=v?"clientHeight":"clientWidth",w=i.reference[f]+i.reference[m]-d[m]-i.floating[f],k=d[m]-i.reference[m],O=await(s.getOffsetParent==null?void 0:s.getOffsetParent(c));let R=O?O[x]:0;(!R||!await(s.isElement==null?void 0:s.isElement(O)))&&(R=a.floating[x]||i.floating[f]);const I=w/2-k/2,N=R/2-p[f]/2-1,A=$t(h[b],N),T=$t(h[y],N),S=A,_=R-p[f]-T,D=R/2-p[f]/2+I,z=Ds(S,D,_),W=!l.arrow&&Hn(o)!=null&&D!==z&&i.reference[f]/2-(DD<=0)){var T,S;const D=(((T=i.flip)==null?void 0:T.index)||0)+1,z=R[D];if(z&&(!(h==="alignment"?y!==ut(z):!1)||A.every(ee=>ut(ee.placement)===y?ee.overflows[0]>0:!0)))return{data:{index:D,overflows:A},reset:{placement:z}};let W=(S=A.filter(J=>J.overflows[0]<=0).sort((J,ee)=>J.overflows[1]-ee.overflows[1])[0])==null?void 0:S.placement;if(!W)switch(m){case"bestFit":{var _;const J=(_=A.filter(ee=>{if(O){const $=ut(ee.placement);return $===y||$==="y"}return!0}).map(ee=>[ee.placement,ee.overflows.filter($=>$>0).reduce(($,q)=>$+q,0)]).sort((ee,$)=>ee[1]-$[1])[0])==null?void 0:_[0];J&&(W=J);break}case"initialPlacement":W=a;break}if(o!==W)return{reset:{placement:W}}}return{}}}};function $c(e,t){return{top:e.top-t.height,right:e.right-t.width,bottom:e.bottom-t.height,left:e.left-t.width}}function Bc(e){return tb.some(t=>e[t]>=0)}const pb=function(e){return e===void 0&&(e={}),{name:"hide",options:e,async fn(t){const{rects:n}=t,{strategy:r="referenceHidden",...o}=Ct(e,t);switch(r){case"referenceHidden":{const i=await Pr(t,{...o,elementContext:"reference"}),s=$c(i,n.reference);return{data:{referenceHiddenOffsets:s,referenceHidden:Bc(s)}}}case"escaped":{const i=await Pr(t,{...o,altBoundary:!0}),s=$c(i,n.floating);return{data:{escapedOffsets:s,escaped:Bc(s)}}}default:return{}}}}},Wc=new Set(["left","top"]);async function mb(e,t){const{placement:n,platform:r,elements:o}=e,i=await(r.isRTL==null?void 0:r.isRTL(o.floating)),s=wt(n),a=Hn(n),l=ut(n)==="y",c=Wc.has(s)?-1:1,u=i&&l?-1:1,h=Ct(t,e);let{mainAxis:d,crossAxis:m,alignmentAxis:f}=typeof h=="number"?{mainAxis:h,crossAxis:0,alignmentAxis:null}:{mainAxis:h.mainAxis||0,crossAxis:h.crossAxis||0,alignmentAxis:h.alignmentAxis};return a&&typeof f=="number"&&(m=a==="end"?f*-1:f),l?{x:m*u,y:d*c}:{x:d*c,y:m*u}}const vb=function(e){return e===void 0&&(e=0),{name:"offset",options:e,async fn(t){var n,r;const{x:o,y:i,placement:s,middlewareData:a}=t,l=await mb(t,e);return s===((n=a.offset)==null?void 0:n.placement)&&(r=a.arrow)!=null&&r.alignmentOffset?{}:{x:o+l.x,y:i+l.y,data:{...l,placement:s}}}}},bb=function(e){return e===void 0&&(e={}),{name:"shift",options:e,async fn(t){const{x:n,y:r,placement:o}=t,{mainAxis:i=!0,crossAxis:s=!1,limiter:a={fn:v=>{let{x:b,y}=v;return{x:b,y}}},...l}=Ct(e,t),c={x:n,y:r},u=await Pr(t,l),h=ut(wt(o)),d=zs(h);let m=c[d],f=c[h];if(i){const v=d==="y"?"top":"left",b=d==="y"?"bottom":"right",y=m+u[v],x=m-u[b];m=Ds(y,m,x)}if(s){const v=h==="y"?"top":"left",b=h==="y"?"bottom":"right",y=f+u[v],x=f-u[b];f=Ds(y,f,x)}const p=a.fn({...t,[d]:m,[h]:f});return{...p,data:{x:p.x-n,y:p.y-r,enabled:{[d]:i,[h]:s}}}}}},yb=function(e){return e===void 0&&(e={}),{options:e,fn(t){const{x:n,y:r,placement:o,rects:i,middlewareData:s}=t,{offset:a=0,mainAxis:l=!0,crossAxis:c=!0}=Ct(e,t),u={x:n,y:r},h=ut(o),d=zs(h);let m=u[d],f=u[h];const p=Ct(a,t),v=typeof p=="number"?{mainAxis:p,crossAxis:0}:{mainAxis:0,crossAxis:0,...p};if(l){const x=d==="y"?"height":"width",w=i.reference[d]-i.floating[x]+v.mainAxis,k=i.reference[d]+i.reference[x]-v.mainAxis;mk&&(m=k)}if(c){var b,y;const x=d==="y"?"width":"height",w=Wc.has(wt(o)),k=i.reference[h]-i.floating[x]+(w&&((b=s.offset)==null?void 0:b[h])||0)+(w?0:v.crossAxis),O=i.reference[h]+i.reference[x]+(w?0:((y=s.offset)==null?void 0:y[h])||0)-(w?v.crossAxis:0);fO&&(f=O)}return{[d]:m,[h]:f}}}},xb=function(e){return e===void 0&&(e={}),{name:"size",options:e,async fn(t){var n,r;const{placement:o,rects:i,platform:s,elements:a}=t,{apply:l=()=>{},...c}=Ct(e,t),u=await Pr(t,c),h=wt(o),d=Hn(o),m=ut(o)==="y",{width:f,height:p}=i.floating;let v,b;h==="top"||h==="bottom"?(v=h,b=d===(await(s.isRTL==null?void 0:s.isRTL(a.floating))?"start":"end")?"left":"right"):(b=h,v=d==="end"?"top":"bottom");const y=p-u.top-u.bottom,x=f-u.left-u.right,w=$t(p-u[v],y),k=$t(f-u[b],x),O=!t.middlewareData.shift;let R=w,I=k;if((n=t.middlewareData.shift)!=null&&n.enabled.x&&(I=x),(r=t.middlewareData.shift)!=null&&r.enabled.y&&(R=y),O&&!d){const A=Be(u.left,0),T=Be(u.right,0),S=Be(u.top,0),_=Be(u.bottom,0);m?I=f-2*(A!==0||T!==0?A+T:Be(u.left,u.right)):R=p-2*(S!==0||_!==0?S+_:Be(u.top,u.bottom))}await l({...t,availableWidth:I,availableHeight:R});const N=await s.getDimensions(a.floating);return f!==N.width||p!==N.height?{reset:{rects:!0}}:{}}}};function jo(){return typeof window<"u"}function Un(e){return Hc(e)?(e.nodeName||"").toLowerCase():"#document"}function We(e){var t;return(e==null||(t=e.ownerDocument)==null?void 0:t.defaultView)||window}function dt(e){var t;return(t=(Hc(e)?e.ownerDocument:e.document)||window.document)==null?void 0:t.documentElement}function Hc(e){return jo()?e instanceof Node||e instanceof We(e).Node:!1}function et(e){return jo()?e instanceof Element||e instanceof We(e).Element:!1}function ht(e){return jo()?e instanceof HTMLElement||e instanceof We(e).HTMLElement:!1}function Uc(e){return!jo()||typeof ShadowRoot>"u"?!1:e instanceof ShadowRoot||e instanceof We(e).ShadowRoot}const kb=new Set(["inline","contents"]);function Rr(e){const{overflow:t,overflowX:n,overflowY:r,display:o}=tt(e);return/auto|scroll|overlay|hidden|clip/.test(t+r+n)&&!kb.has(o)}const Cb=new Set(["table","td","th"]);function wb(e){return Cb.has(Un(e))}const Sb=[":popover-open",":modal"];function $o(e){return Sb.some(t=>{try{return e.matches(t)}catch{return!1}})}const Eb=["transform","translate","scale","rotate","perspective"],Ob=["transform","translate","scale","rotate","perspective","filter"],Pb=["paint","layout","strict","content"];function Bs(e){const t=Ws(),n=et(e)?tt(e):e;return Eb.some(r=>n[r]?n[r]!=="none":!1)||(n.containerType?n.containerType!=="normal":!1)||!t&&(n.backdropFilter?n.backdropFilter!=="none":!1)||!t&&(n.filter?n.filter!=="none":!1)||Ob.some(r=>(n.willChange||"").includes(r))||Pb.some(r=>(n.contain||"").includes(r))}function Rb(e){let t=Bt(e);for(;ht(t)&&!Gn(t);){if(Bs(t))return t;if($o(t))return null;t=Bt(t)}return null}function Ws(){return typeof CSS>"u"||!CSS.supports?!1:CSS.supports("-webkit-backdrop-filter","none")}const Ib=new Set(["html","body","#document"]);function Gn(e){return Ib.has(Un(e))}function tt(e){return We(e).getComputedStyle(e)}function Bo(e){return et(e)?{scrollLeft:e.scrollLeft,scrollTop:e.scrollTop}:{scrollLeft:e.scrollX,scrollTop:e.scrollY}}function Bt(e){if(Un(e)==="html")return e;const t=e.assignedSlot||e.parentNode||Uc(e)&&e.host||dt(e);return Uc(t)?t.host:t}function Gc(e){const t=Bt(e);return Gn(t)?e.ownerDocument?e.ownerDocument.body:e.body:ht(t)&&Rr(t)?t:Gc(t)}function Ir(e,t,n){var r;t===void 0&&(t=[]),n===void 0&&(n=!0);const o=Gc(e),i=o===((r=e.ownerDocument)==null?void 0:r.body),s=We(o);if(i){const a=Hs(s);return t.concat(s,s.visualViewport||[],Rr(o)?o:[],a&&n?Ir(a):[])}return t.concat(o,Ir(o,[],n))}function Hs(e){return e.parent&&Object.getPrototypeOf(e.parent)?e.frameElement:null}function qc(e){const t=tt(e);let n=parseFloat(t.width)||0,r=parseFloat(t.height)||0;const o=ht(e),i=o?e.offsetWidth:n,s=o?e.offsetHeight:r,a=Lo(n)!==i||Lo(r)!==s;return a&&(n=i,r=s),{width:n,height:r,$:a}}function Us(e){return et(e)?e:e.contextElement}function qn(e){const t=Us(e);if(!ht(t))return ct(1);const n=t.getBoundingClientRect(),{width:r,height:o,$:i}=qc(t);let s=(i?Lo(n.width):n.width)/r,a=(i?Lo(n.height):n.height)/o;return(!s||!Number.isFinite(s))&&(s=1),(!a||!Number.isFinite(a))&&(a=1),{x:s,y:a}}const Tb=ct(0);function Kc(e){const t=We(e);return!Ws()||!t.visualViewport?Tb:{x:t.visualViewport.offsetLeft,y:t.visualViewport.offsetTop}}function Nb(e,t,n){return t===void 0&&(t=!1),!n||t&&n!==We(e)?!1:t}function tn(e,t,n,r){t===void 0&&(t=!1),n===void 0&&(n=!1);const o=e.getBoundingClientRect(),i=Us(e);let s=ct(1);t&&(r?et(r)&&(s=qn(r)):s=qn(e));const a=Nb(i,n,r)?Kc(i):ct(0);let l=(o.left+a.x)/s.x,c=(o.top+a.y)/s.y,u=o.width/s.x,h=o.height/s.y;if(i){const d=We(i),m=r&&et(r)?We(r):r;let f=d,p=Hs(f);for(;p&&r&&m!==f;){const v=qn(p),b=p.getBoundingClientRect(),y=tt(p),x=b.left+(p.clientLeft+parseFloat(y.paddingLeft))*v.x,w=b.top+(p.clientTop+parseFloat(y.paddingTop))*v.y;l*=v.x,c*=v.y,u*=v.x,h*=v.y,l+=x,c+=w,f=We(p),p=Hs(f)}}return Mo({width:u,height:h,x:l,y:c})}function Wo(e,t){const n=Bo(e).scrollLeft;return t?t.left+n:tn(dt(e)).left+n}function Yc(e,t){const n=e.getBoundingClientRect(),r=n.left+t.scrollLeft-Wo(e,n),o=n.top+t.scrollTop;return{x:r,y:o}}function Ab(e){let{elements:t,rect:n,offsetParent:r,strategy:o}=e;const i=o==="fixed",s=dt(r),a=t?$o(t.floating):!1;if(r===s||a&&i)return n;let l={scrollLeft:0,scrollTop:0},c=ct(1);const u=ct(0),h=ht(r);if((h||!h&&!i)&&((Un(r)!=="body"||Rr(s))&&(l=Bo(r)),ht(r))){const m=tn(r);c=qn(r),u.x=m.x+r.clientLeft,u.y=m.y+r.clientTop}const d=s&&!h&&!i?Yc(s,l):ct(0);return{width:n.width*c.x,height:n.height*c.y,x:n.x*c.x-l.scrollLeft*c.x+u.x+d.x,y:n.y*c.y-l.scrollTop*c.y+u.y+d.y}}function _b(e){return Array.from(e.getClientRects())}function Vb(e){const t=dt(e),n=Bo(e),r=e.ownerDocument.body,o=Be(t.scrollWidth,t.clientWidth,r.scrollWidth,r.clientWidth),i=Be(t.scrollHeight,t.clientHeight,r.scrollHeight,r.clientHeight);let s=-n.scrollLeft+Wo(e);const a=-n.scrollTop;return tt(r).direction==="rtl"&&(s+=Be(t.clientWidth,r.clientWidth)-o),{width:o,height:i,x:s,y:a}}const Xc=25;function Fb(e,t){const n=We(e),r=dt(e),o=n.visualViewport;let i=r.clientWidth,s=r.clientHeight,a=0,l=0;if(o){i=o.width,s=o.height;const u=Ws();(!u||u&&t==="fixed")&&(a=o.offsetLeft,l=o.offsetTop)}const c=Wo(r);if(c<=0){const u=r.ownerDocument,h=u.body,d=getComputedStyle(h),m=u.compatMode==="CSS1Compat"&&parseFloat(d.marginLeft)+parseFloat(d.marginRight)||0,f=Math.abs(r.clientWidth-h.clientWidth-m);f<=Xc&&(i-=f)}else c<=Xc&&(i+=c);return{width:i,height:s,x:a,y:l}}const Lb=new Set(["absolute","fixed"]);function Db(e,t){const n=tn(e,!0,t==="fixed"),r=n.top+e.clientTop,o=n.left+e.clientLeft,i=ht(e)?qn(e):ct(1),s=e.clientWidth*i.x,a=e.clientHeight*i.y,l=o*i.x,c=r*i.y;return{width:s,height:a,x:l,y:c}}function Qc(e,t,n){let r;if(t==="viewport")r=Fb(e,n);else if(t==="document")r=Vb(dt(e));else if(et(t))r=Db(t,n);else{const o=Kc(e);r={x:t.x-o.x,y:t.y-o.y,width:t.width,height:t.height}}return Mo(r)}function Jc(e,t){const n=Bt(e);return n===t||!et(n)||Gn(n)?!1:tt(n).position==="fixed"||Jc(n,t)}function zb(e,t){const n=t.get(e);if(n)return n;let r=Ir(e,[],!1).filter(a=>et(a)&&Un(a)!=="body"),o=null;const i=tt(e).position==="fixed";let s=i?Bt(e):e;for(;et(s)&&!Gn(s);){const a=tt(s),l=Bs(s);!l&&a.position==="fixed"&&(o=null),(i?!l&&!o:!l&&a.position==="static"&&!!o&&Lb.has(o.position)||Rr(s)&&!l&&Jc(e,s))?r=r.filter(u=>u!==s):o=a,s=Bt(s)}return t.set(e,r),r}function Mb(e){let{element:t,boundary:n,rootBoundary:r,strategy:o}=e;const s=[...n==="clippingAncestors"?$o(t)?[]:zb(t,this._c):[].concat(n),r],a=s[0],l=s.reduce((c,u)=>{const h=Qc(t,u,o);return c.top=Be(h.top,c.top),c.right=$t(h.right,c.right),c.bottom=$t(h.bottom,c.bottom),c.left=Be(h.left,c.left),c},Qc(t,a,o));return{width:l.right-l.left,height:l.bottom-l.top,x:l.left,y:l.top}}function jb(e){const{width:t,height:n}=qc(e);return{width:t,height:n}}function $b(e,t,n){const r=ht(t),o=dt(t),i=n==="fixed",s=tn(e,!0,i,t);let a={scrollLeft:0,scrollTop:0};const l=ct(0);function c(){l.x=Wo(o)}if(r||!r&&!i)if((Un(t)!=="body"||Rr(o))&&(a=Bo(t)),r){const m=tn(t,!0,i,t);l.x=m.x+t.clientLeft,l.y=m.y+t.clientTop}else o&&c();i&&!r&&o&&c();const u=o&&!r&&!i?Yc(o,a):ct(0),h=s.left+a.scrollLeft-l.x-u.x,d=s.top+a.scrollTop-l.y-u.y;return{x:h,y:d,width:s.width,height:s.height}}function Gs(e){return tt(e).position==="static"}function Zc(e,t){if(!ht(e)||tt(e).position==="fixed")return null;if(t)return t(e);let n=e.offsetParent;return dt(e)===n&&(n=n.ownerDocument.body),n}function eu(e,t){const n=We(e);if($o(e))return n;if(!ht(e)){let o=Bt(e);for(;o&&!Gn(o);){if(et(o)&&!Gs(o))return o;o=Bt(o)}return n}let r=Zc(e,t);for(;r&&wb(r)&&Gs(r);)r=Zc(r,t);return r&&Gn(r)&&Gs(r)&&!Bs(r)?n:r||Rb(e)||n}const Bb=async function(e){const t=this.getOffsetParent||eu,n=this.getDimensions,r=await n(e.floating);return{reference:$b(e.reference,await t(e.floating),e.strategy),floating:{x:0,y:0,width:r.width,height:r.height}}};function Wb(e){return tt(e).direction==="rtl"}const Hb={convertOffsetParentRelativeRectToViewportRelativeRect:Ab,getDocumentElement:dt,getClippingRect:Mb,getOffsetParent:eu,getElementRects:Bb,getClientRects:_b,getDimensions:jb,getScale:qn,isElement:et,isRTL:Wb};function tu(e,t){return e.x===t.x&&e.y===t.y&&e.width===t.width&&e.height===t.height}function Ub(e,t){let n=null,r;const o=dt(e);function i(){var a;clearTimeout(r),(a=n)==null||a.disconnect(),n=null}function s(a,l){a===void 0&&(a=!1),l===void 0&&(l=1),i();const c=e.getBoundingClientRect(),{left:u,top:h,width:d,height:m}=c;if(a||t(),!d||!m)return;const f=Do(h),p=Do(o.clientWidth-(u+d)),v=Do(o.clientHeight-(h+m)),b=Do(u),x={rootMargin:-f+"px "+-p+"px "+-v+"px "+-b+"px",threshold:Be(0,$t(1,l))||1};let w=!0;function k(O){const R=O[0].intersectionRatio;if(R!==l){if(!w)return s();R?s(!1,R):r=setTimeout(()=>{s(!1,1e-7)},1e3)}R===1&&!tu(c,e.getBoundingClientRect())&&s(),w=!1}try{n=new IntersectionObserver(k,{...x,root:o.ownerDocument})}catch{n=new IntersectionObserver(k,x)}n.observe(e)}return s(!0),i}function Gb(e,t,n,r){r===void 0&&(r={});const{ancestorScroll:o=!0,ancestorResize:i=!0,elementResize:s=typeof ResizeObserver=="function",layoutShift:a=typeof IntersectionObserver=="function",animationFrame:l=!1}=r,c=Us(e),u=o||i?[...c?Ir(c):[],...Ir(t)]:[];u.forEach(b=>{o&&b.addEventListener("scroll",n,{passive:!0}),i&&b.addEventListener("resize",n)});const h=c&&a?Ub(c,n):null;let d=-1,m=null;s&&(m=new ResizeObserver(b=>{let[y]=b;y&&y.target===c&&m&&(m.unobserve(t),cancelAnimationFrame(d),d=requestAnimationFrame(()=>{var x;(x=m)==null||x.observe(t)})),n()}),c&&!l&&m.observe(c),m.observe(t));let f,p=l?tn(e):null;l&&v();function v(){const b=tn(e);p&&!tu(p,b)&&n(),p=b,f=requestAnimationFrame(v)}return n(),()=>{var b;u.forEach(y=>{o&&y.removeEventListener("scroll",n),i&&y.removeEventListener("resize",n)}),h?.(),(b=m)==null||b.disconnect(),m=null,l&&cancelAnimationFrame(f)}}const qb=vb,Kb=bb,Yb=gb,Xb=xb,Qb=pb,Jb=fb,Zb=yb,ey=(e,t,n)=>{const r=new Map,o={platform:Hb,...n},i={...o.platform,_c:r};return hb(e,t,{...o,platform:i})};function nu(e=0,t=0,n=0,r=0){if(typeof DOMRect=="function")return new DOMRect(e,t,n,r);const o={x:e,y:t,width:n,height:r,top:t,right:e+n,bottom:t+r,left:e};return{...o,toJSON:()=>o}}function ty(e){if(!e)return nu();const{x:t,y:n,width:r,height:o}=e;return nu(t,n,r,o)}function ny(e,t){return{contextElement:Te(e)?e:void 0,getBoundingClientRect:()=>{const n=e,r=t?.(n);return r||!n?ty(r):n.getBoundingClientRect()}}}var ru=e=>({variable:e,reference:`var(${e})`}),ou={transformOrigin:ru("--transform-origin"),arrowOffset:ru("--arrow-offset")},ry=e=>e==="top"||e==="bottom"?"y":"x";function oy(e,t){return{name:"transformOrigin",fn(n){const{elements:r,middlewareData:o,placement:i,rects:s,y:a}=n,l=i.split("-")[0],c=ry(l),u=o.arrow?.x||0,h=o.arrow?.y||0,d=t?.clientWidth||0,m=t?.clientHeight||0,f=u+d/2,p=h+m/2,v=Math.abs(o.shift?.y||0),b=s.reference.height/2,y=m/2,x=e.offset?.mainAxis??e.gutter,w=typeof x=="number"?x+y:x??y,k=v>w,O={top:`${f}px calc(100% + ${w}px)`,bottom:`${f}px ${-w}px`,left:`calc(100% + ${w}px) ${p}px`,right:`${-w}px ${p}px`}[l],R=`${f}px ${s.reference.y+b-a}px`,I=!!e.overlap&&c==="y"&&k;return r.floating.style.setProperty(ou.transformOrigin.variable,I?R:O),{data:{transformOrigin:I?R:O}}}}}var iy={name:"rects",fn({rects:e}){return{data:e}}},sy=e=>{if(e)return{name:"shiftArrow",fn({placement:t,middlewareData:n}){if(!n.arrow)return{};const{x:r,y:o}=n.arrow,i=t.split("-")[0];return Object.assign(e.style,{left:r!=null?`${r}px`:"",top:o!=null?`${o}px`:"",[i]:`calc(100% + ${ou.arrowOffset.reference})`}),{}}}};function ay(e){const[t,n]=e.split("-");return{side:t,align:n,hasAlign:n!=null}}function ly(e){return e.split("-")[0]}var cy={strategy:"absolute",placement:"bottom",listeners:!0,gutter:8,flip:!0,slide:!0,overlap:!1,sameWidth:!1,fitViewport:!1,overflowPadding:8,arrowPadding:4};function iu(e,t){const n=e.devicePixelRatio||1;return Math.round(t*n)/n}function qs(e){return typeof e=="function"?e():e==="clipping-ancestors"?"clippingAncestors":e}function uy(e,t,n){const r=e||t.createElement("div");return Jb({element:r,padding:n.arrowPadding})}function dy(e,t){if(!Xm(t.offset??t.gutter))return qb(({placement:n})=>{const r=(e?.clientHeight||0)/2,o=t.offset?.mainAxis??t.gutter,i=typeof o=="number"?o+r:o??r,{hasAlign:s}=ay(n),a=s?void 0:t.shift,l=t.offset?.crossAxis??a;return bo({crossAxis:l,mainAxis:i,alignmentAxis:t.shift})})}function hy(e){if(!e.flip)return;const t=qs(e.boundary);return Yb({...t?{boundary:t}:void 0,padding:e.overflowPadding,fallbackPlacements:e.flip===!0?void 0:e.flip})}function fy(e){if(!e.slide&&!e.overlap)return;const t=qs(e.boundary);return Kb({...t?{boundary:t}:void 0,mainAxis:e.slide,crossAxis:e.overlap,padding:e.overflowPadding,limiter:Zb()})}function gy(e){return Xb({padding:e.overflowPadding,apply({elements:t,rects:n,availableHeight:r,availableWidth:o}){const i=t.floating,s=Math.round(n.reference.width),a=Math.round(n.reference.height);o=Math.floor(o),r=Math.floor(r),i.style.setProperty("--reference-width",`${s}px`),i.style.setProperty("--reference-height",`${a}px`),i.style.setProperty("--available-width",`${o}px`),i.style.setProperty("--available-height",`${r}px`)}})}function py(e){if(e.hideWhenDetached)return Qb({strategy:"referenceHidden",boundary:qs(e.boundary)??"clippingAncestors"})}function my(e){return e?e===!0?{ancestorResize:!0,ancestorScroll:!0,elementResize:!0,layoutShift:!0}:e:{}}function vy(e,t,n={}){const r=ny(e,n.getAnchorRect);if(!t||!r)return;const o=Object.assign({},cy,n),i=t.querySelector("[data-part=arrow]"),s=[dy(i,o),hy(o),fy(o),uy(i,t.ownerDocument,o),sy(i),oy({gutter:o.gutter,offset:o.offset,overlap:o.overlap},i),gy(o),py(o),iy],{placement:a,strategy:l,onComplete:c,onPositioned:u}=o,h=async()=>{if(!r||!t)return;const p=await ey(r,t,{placement:a,middleware:s,strategy:l});c?.(p),u?.({placed:!0});const v=be(t),b=iu(v,p.x),y=iu(v,p.y);t.style.setProperty("--x",`${b}px`),t.style.setProperty("--y",`${y}px`),o.hideWhenDetached&&(p.middlewareData.hide?.referenceHidden?(t.style.setProperty("visibility","hidden"),t.style.setProperty("pointer-events","none")):(t.style.removeProperty("visibility"),t.style.removeProperty("pointer-events")));const x=t.firstElementChild;if(x){const w=xo(x);t.style.setProperty("--z-index",w.zIndex)}},d=async()=>{n.updatePosition?(await n.updatePosition({updatePosition:h,floatingElement:t}),u?.({placed:!0})):await h()},m=my(o.listeners),f=o.listeners?Gb(r,t,d,m):o0;return d(),()=>{f?.(),u?.({placed:!1})}}function nt(e,t,n={}){const{defer:r,...o}=n,i=r?G:a=>a(),s=[];return s.push(i(()=>{const a=typeof e=="function"?e():e,l=typeof t=="function"?t():t;s.push(vy(a,l,o))})),()=>{s.forEach(a=>a?.())}}function by(e){const t={each(n){for(let r=0;r{try{i.document.addEventListener(n,r,o)}catch{}}),()=>{try{t.removeEventListener(n,r,o)}catch{}}},removeEventListener(n,r,o){t.each(i=>{try{i.document.removeEventListener(n,r,o)}catch{}})}};return t}function yy(e){const t=e.frameElement!=null?e.parent:null;return{addEventListener:(n,r,o)=>{try{t?.addEventListener(n,r,o)}catch{}return()=>{try{t?.removeEventListener(n,r,o)}catch{}}},removeEventListener:(n,r,o)=>{try{t?.removeEventListener(n,r,o)}catch{}}}}var su="pointerdown.outside",au="focus.outside";function xy(e){for(const t of e)if(Te(t)&&Mt(t))return!0;return!1}var lu=e=>"clientY"in e;function ky(e,t){if(!lu(t)||!e)return!1;const n=e.getBoundingClientRect();return n.width===0||n.height===0?!1:n.top<=t.clientY&&t.clientY<=n.top+n.height&&n.left<=t.clientX&&t.clientX<=n.left+n.width}function Cy(e,t){return e.y<=t.y&&t.y<=e.y+e.height&&e.x<=t.x&&t.x<=e.x+e.width}function cu(e,t){if(!t||!lu(e))return!1;const n=t.scrollHeight>t.clientHeight,r=n&&e.clientX>t.offsetLeft+t.clientWidth,o=t.scrollWidth>t.clientWidth,i=o&&e.clientY>t.offsetTop+t.clientHeight,s={x:t.offsetLeft,y:t.offsetTop,width:t.clientWidth+(n?16:0),height:t.clientHeight+(o?16:0)},a={x:e.clientX,y:e.clientY};return Cy(s,a)?r||i:!1}function wy(e,t){const{exclude:n,onFocusOutside:r,onPointerDownOutside:o,onInteractOutside:i,defer:s}=t;if(!e)return;const a=Fe(e),l=be(e),c=by(l),u=yy(l);function h(y,x){if(!Te(x)||!x.isConnected||Jt(e,x)||ky(e,y))return!1;const w=a.querySelector(`[aria-controls="${e.id}"]`);if(w){const O=Oo(w);if(cu(y,O))return!1}const k=Oo(e);return cu(y,k)?!1:!n?.(x)}const d=new Set,m=jn(e?.getRootNode());function f(y){function x(w){const k=s&&!hc()?G:I=>I(),O=w??y,R=O?.composedPath?.()??[O?.target];k(()=>{const I=m?R[0]:$e(y);if(!(!e||!h(y,I))){if(o||i){const N=mo(o,i);e.addEventListener(su,N,{once:!0})}uu(e,su,{bubbles:!1,cancelable:!0,detail:{originalEvent:O,contextmenu:K0(O),focusable:xy(R),target:I}})}})}y.pointerType==="touch"?(d.forEach(w=>w()),d.add(se(a,"click",x,{once:!0})),d.add(u.addEventListener("click",x,{once:!0})),d.add(c.addEventListener("click",x,{once:!0}))):x()}const p=new Set,v=setTimeout(()=>{p.add(se(a,"pointerdown",f,!0)),p.add(u.addEventListener("pointerdown",f,!0)),p.add(c.addEventListener("pointerdown",f,!0))},0);function b(y){(s?G:w=>w())(()=>{const w=$e(y);if(!(!e||!h(y,w))){if(r||i){const k=mo(r,i);e.addEventListener(au,k,{once:!0})}uu(e,au,{bubbles:!1,cancelable:!0,detail:{originalEvent:y,contextmenu:!1,focusable:Mt(w),target:w}})}})}return hc()||(p.add(se(a,"focusin",b,!0)),p.add(u.addEventListener("focusin",b,!0)),p.add(c.addEventListener("focusin",b,!0))),()=>{clearTimeout(v),d.forEach(y=>y()),p.forEach(y=>y())}}function Sy(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(wy(i,t))})),()=>{o.forEach(i=>i?.())}}function uu(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,n);return e.dispatchEvent(o)}function Ey(e,t){const n=r=>{r.key==="Escape"&&(r.isComposing||t?.(r))};return se(Fe(e),"keydown",n,{capture:!0})}var du="layer:request-dismiss",Ke={layers:[],branches:[],count(){return this.layers.length},pointerBlockingLayers(){return this.layers.filter(e=>e.pointerBlocking)},topMostPointerBlockingLayer(){return[...this.pointerBlockingLayers()].slice(-1)[0]},hasPointerBlockingLayer(){return this.pointerBlockingLayers().length>0},isBelowPointerBlockingLayer(e){const t=this.indexOf(e),n=this.topMostPointerBlockingLayer()?this.indexOf(this.topMostPointerBlockingLayer()?.node):-1;return tt.type===e)},getNestedLayersByType(e,t){const n=this.indexOf(e);return n===-1?[]:this.layers.slice(n+1).filter(r=>r.type===t)},getParentLayerOfType(e,t){const n=this.indexOf(e);if(!(n<=0))return this.layers.slice(0,n).reverse().find(r=>r.type===t)},countNestedLayersOfType(e,t){return this.getNestedLayersByType(e,t).length},isInNestedLayer(e,t){return this.getNestedLayers(e).some(n=>Jt(n.node,t))},isInBranch(e){return Array.from(this.branches).some(t=>Jt(t,e))},add(e){this.layers.push(e),this.syncLayers()},addBranch(e){this.branches.push(e)},remove(e){const t=this.indexOf(e);t<0||(tKe.dismiss(r.node,e)),this.layers.splice(t,1),this.syncLayers())},removeBranch(e){const t=this.branches.indexOf(e);t>=0&&this.branches.splice(t,1)},syncLayers(){this.layers.forEach((e,t)=>{e.node.style.setProperty("--layer-index",`${t}`),e.node.removeAttribute("data-nested"),e.node.removeAttribute("data-has-nested"),this.getParentLayerOfType(e.node,e.type)&&e.node.setAttribute("data-nested",e.type);const r=this.countNestedLayersOfType(e.node,e.type);r>0&&e.node.setAttribute("data-has-nested",e.type),e.node.style.setProperty("--nested-layer-count",`${r}`)})},indexOf(e){return this.layers.findIndex(t=>t.node===e)},dismiss(e,t){const n=this.indexOf(e);if(n===-1)return;const r=this.layers[n];Py(e,du,o=>{r.requestDismiss?.(o),o.defaultPrevented||r?.dismiss()}),Oy(e,du,{originalLayer:e,targetLayer:t,originalIndex:n,targetIndex:t?this.indexOf(t):-1}),this.syncLayers()},clear(){this.remove(this.layers[0].node)}};function Oy(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,{cancelable:!0,bubbles:!0,detail:n});return e.dispatchEvent(o)}function Py(e,t,n){e.addEventListener(t,n,{once:!0})}var hu;function fu(){Ke.layers.forEach(({node:e})=>{e.style.pointerEvents=Ke.isBelowPointerBlockingLayer(e)?"none":"auto"})}function Ry(e){e.style.pointerEvents=""}function Iy(e,t){const n=Fe(e),r=[];return Ke.hasPointerBlockingLayer()&&!n.body.hasAttribute("data-inert")&&(hu=document.body.style.pointerEvents,queueMicrotask(()=>{n.body.style.pointerEvents="none",n.body.setAttribute("data-inert","")})),t?.forEach(o=>{const[i,s]=Pv(()=>{const a=o();return Te(a)?a:null},{timeout:1e3});i.then(a=>r.push(To(a,{pointerEvents:"auto"}))),r.push(s)}),()=>{Ke.hasPointerBlockingLayer()||(queueMicrotask(()=>{n.body.style.pointerEvents=hu,n.body.removeAttribute("data-inert"),n.body.style.length===0&&n.body.removeAttribute("style")}),r.forEach(o=>o()))}}function Ty(e,t){const{warnOnMissingNode:n=!0}=t;if(n&&!e){wr("[@zag-js/dismissable] node is `null` or `undefined`");return}if(!e)return;const{onDismiss:r,onRequestDismiss:o,pointerBlocking:i,exclude:s,debug:a,type:l="dialog"}=t,c={dismiss:r,node:e,type:l,pointerBlocking:i,requestDismiss:o};Ke.add(c),fu();function u(p){const v=$e(p.detail.originalEvent);Ke.isBelowPointerBlockingLayer(e)||Ke.isInBranch(v)||(t.onPointerDownOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onPointerDownOutside:",p.detail.originalEvent),r?.()))}function h(p){const v=$e(p.detail.originalEvent);Ke.isInBranch(v)||(t.onFocusOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onFocusOutside:",p.detail.originalEvent),r?.()))}function d(p){Ke.isTopMost(e)&&(t.onEscapeKeyDown?.(p),!p.defaultPrevented&&r&&(p.preventDefault(),r()))}function m(p){if(!e)return!1;const v=typeof s=="function"?s():s,b=Array.isArray(v)?v:[v],y=t.persistentElements?.map(x=>x()).filter(Te);return y&&b.push(...y),b.some(x=>Jt(x,p))||Ke.isInNestedLayer(e,p)}const f=[i?Iy(e,t.persistentElements):void 0,Ey(e,d),Sy(e,{exclude:m,onFocusOutside:h,onPointerDownOutside:u,defer:t.defer})];return()=>{Ke.remove(e),fu(),Ry(e),f.forEach(p=>p?.())}}function Kn(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=Qt(e)?e():e;o.push(Ty(i,t))})),()=>{o.forEach(i=>i?.())}}var gu=j("color-picker",["root","label","control","trigger","positioner","content","area","areaThumb","valueText","areaBackground","channelSlider","channelSliderLabel","channelSliderTrack","channelSliderThumb","channelSliderValueText","channelInput","transparencyGrid","swatchGroup","swatchTrigger","swatchIndicator","swatch","eyeDropperTrigger","formatTrigger","formatSelect"]);gu.build();var Ny=e=>e.ids?.hiddenInput??`color-picker:${e.id}:hidden-input`,Ay=e=>e.ids?.control??`color-picker:${e.id}:control`,_y=e=>e.ids?.trigger??`color-picker:${e.id}:trigger`,Vy=e=>e.ids?.content??`color-picker:${e.id}:content`,Fy=e=>e.ids?.positioner??`color-picker:${e.id}:positioner`,Ly=e=>e.ids?.formatSelect??`color-picker:${e.id}:format-select`,Dy=e=>e.ids?.area??`color-picker:${e.id}:area`,zy=e=>e.ids?.areaThumb??`color-picker:${e.id}:area-thumb`,My=(e,t)=>e.ids?.channelSliderTrack?.(t)??`color-picker:${e.id}:slider-track:${t}`,jy=(e,t)=>e.ids?.channelSliderThumb?.(t)??`color-picker:${e.id}:slider-thumb:${t}`,Ho=e=>e.getById(Vy(e)),$y=e=>e.getById(zy(e)),By=(e,t)=>e.getById(jy(e,t)),Wy=e=>e.getById(Ly(e)),pu=e=>e.getById(Ny(e)),Hy=e=>e.getById(Dy(e)),Uy=(e,t,n)=>{const r=Hy(e);if(!r)return;const{getPercentValue:o}=yc(t,r);return{x:o({dir:n,orientation:"horizontal"}),y:o({orientation:"vertical"})}},Gy=e=>e.getById(Ay(e)),Ks=e=>e.getById(_y(e)),qy=e=>e.getById(Fy(e)),Ky=(e,t)=>e.getById(My(e,t)),Yy=(e,t,n,r)=>{const o=Ky(e,n);if(!o)return;const{getPercentValue:i}=yc(t,o);return{x:i({dir:r,orientation:"horizontal"}),y:i({orientation:"vertical"})}},Xy=e=>[...Io(Ho(e),"input[data-channel]"),...Io(Gy(e),"input[data-channel]")];function Qy(e,t){if(t==null)return"";if(t==="hex")return e.toString("hex");if(t==="css")return e.toString("css");if(t in e)return e.getChannelValue(t).toString();const n=e.getFormat()==="hsla";switch(t){case"hue":return n?e.toFormat("hsla").getChannelValue("hue").toString():e.toFormat("hsba").getChannelValue("hue").toString();case"saturation":return n?e.toFormat("hsla").getChannelValue("saturation").toString():e.toFormat("hsba").getChannelValue("saturation").toString();case"lightness":return e.toFormat("hsla").getChannelValue("lightness").toString();case"brightness":return e.toFormat("hsba").getChannelValue("brightness").toString();case"red":case"green":case"blue":return e.toFormat("rgba").getChannelValue(t).toString();default:return e.getChannelValue(t).toString()}}var mu=e=>Fo(e),Jy=/^[0-9a-fA-F]{3,8}$/;function Zy(e){return Jy.test(e)}function ex(e){return e.startsWith("#")?e:Zy(e)?`#${e}`:e}var{and:tx}=jt();tx("isOpenControlled","closeOnSelect");function vu(e,t,n){const r=Xy(e);G(()=>{r.forEach(o=>{const i=o.dataset.channel;So(o,Qy(n||t,i))})})}function nx(e,t){const n=Wy(e);n&&G(()=>So(n,t))}M()(["closeOnSelect","dir","disabled","format","defaultFormat","getRootNode","id","ids","initialFocusEl","inline","name","positioning","onFocusOutside","onFormatChange","onInteractOutside","onOpenChange","onPointerDownOutside","onValueChange","onValueChangeEnd","defaultOpen","open","positioning","required","readOnly","value","defaultValue","invalid","openAutoFocus"]),M()(["xChannel","yChannel"]),M()(["channel","orientation"]),M()(["value","disabled"]),M()(["value","respectAlpha"]),M()(["size"]);const[bu,rx]=Mn({name:"RenderStrategyContext",hookName:"useRenderStrategyContext",providerName:""}),yu=e=>Sc()(e,["lazyMount","unmountOnExit"]);var xu=j("accordion").parts("root","item","itemTrigger","itemContent","itemIndicator");xu.build();var ku=e=>e.ids?.root??`accordion:${e.id}`,Cu=(e,t)=>e.ids?.itemTrigger?.(t)??`accordion:${e.id}:trigger:${t}`,ox=e=>e.getById(ku(e)),Uo=e=>{const n=`[aria-controls][data-ownedby='${CSS.escape(ku(e))}']:not([disabled])`;return Io(ox(e),n)},ix=e=>kr(Uo(e)),sx=e=>hs(Uo(e)),ax=(e,t)=>vv(Uo(e),Cu(e,t)),lx=(e,t)=>bv(Uo(e),Cu(e,t)),{and:cx,not:ux}=jt();cx("isExpanded","canToggle"),ux("isExpanded"),M()(["collapsible","dir","disabled","getRootNode","id","ids","multiple","onFocusChange","onValueChange","orientation","value","defaultValue"]),M()(["value","disabled"]);var Tr=(e,t)=>({x:e,y:t});function dx(e){const{x:t,y:n,width:r,height:o}=e,i=t+r/2,s=n+o/2;return{x:t,y:n,width:r,height:o,minX:t,minY:n,maxX:t+r,maxY:n+o,midX:i,midY:s,center:Tr(i,s)}}function hx(e){const t=Tr(e.minX,e.minY),n=Tr(e.maxX,e.minY),r=Tr(e.maxX,e.maxY),o=Tr(e.minX,e.maxY);return{top:t,right:n,bottom:r,left:o}}function fx(e,t){const n=dx(e),{top:r,right:o,left:i,bottom:s}=hx(n),[a]=t.split("-");return{top:[i,r,o,s],right:[r,o,s,i],bottom:[r,i,s,o],left:[o,r,i,s]}[a]}function gx(e,t){const{x:n,y:r}=t;let o=!1;for(let i=0,s=e.length-1;ir!=u>r&&n<(c-a)*(r-l)/(u-l)+a&&(o=!o)}return o}var wu=j("avatar").parts("root","image","fallback");wu.build(),M()(["dir","id","ids","onStatusChange","getRootNode"]);function px(e){return!(e.metaKey||!wo()&&e.altKey||e.ctrlKey||e.key==="Control"||e.key==="Shift"||e.key==="Meta")}var mx=new Set(["checkbox","radio","range","color","file","image","button","submit","reset"]);function vx(e,t,n){const r=n?$e(n):null,o=be(r);return e=e||r instanceof o.HTMLInputElement&&!mx.has(r?.type)||r instanceof o.HTMLTextAreaElement||r instanceof o.HTMLElement&&r.isContentEditable,!(e&&t==="keyboard"&&n instanceof o.KeyboardEvent&&!Reflect.has(bx,n.key))}var nn=null,Ys=new Set,Nr=new Map,rn=!1,Xs=!1,bx={Tab:!0,Escape:!0};function Go(e,t){for(let n of Ys)n(e,t)}function qo(e){rn=!0,px(e)&&(nn="keyboard",Go("keyboard",e))}function Ye(e){nn="pointer",(e.type==="mousedown"||e.type==="pointerdown")&&(rn=!0,Go("pointer",e))}function Su(e){q0(e)&&(rn=!0,nn="virtual")}function Eu(e){const t=$e(e);t===be(t)||t===Fe(t)||(!rn&&!Xs&&(nn="virtual",Go("virtual",e)),rn=!1,Xs=!1)}function Ou(){rn=!1,Xs=!0}function yx(e){if(typeof window>"u"||Nr.get(be(e)))return;const t=be(e),n=Fe(e);let r=t.HTMLElement.prototype.focus;function o(){nn="virtual",Go("virtual",null),rn=!0,r.apply(this,arguments)}Object.defineProperty(t.HTMLElement.prototype,"focus",{configurable:!0,value:o}),n.addEventListener("keydown",qo,!0),n.addEventListener("keyup",qo,!0),n.addEventListener("click",Su,!0),t.addEventListener("focus",Eu,!0),t.addEventListener("blur",Ou,!1),typeof t.PointerEvent<"u"?(n.addEventListener("pointerdown",Ye,!0),n.addEventListener("pointermove",Ye,!0),n.addEventListener("pointerup",Ye,!0)):(n.addEventListener("mousedown",Ye,!0),n.addEventListener("mousemove",Ye,!0),n.addEventListener("mouseup",Ye,!0)),t.addEventListener("beforeunload",()=>{xx(e)},{once:!0}),Nr.set(t,{focus:r})}var xx=(e,t)=>{const n=be(e),r=Fe(e);Nr.has(n)&&(n.HTMLElement.prototype.focus=Nr.get(n).focus,r.removeEventListener("keydown",qo,!0),r.removeEventListener("keyup",qo,!0),r.removeEventListener("click",Su,!0),n.removeEventListener("focus",Eu,!0),n.removeEventListener("blur",Ou,!1),typeof n.PointerEvent<"u"?(r.removeEventListener("pointerdown",Ye,!0),r.removeEventListener("pointermove",Ye,!0),r.removeEventListener("pointerup",Ye,!0)):(r.removeEventListener("mousedown",Ye,!0),r.removeEventListener("mousemove",Ye,!0),r.removeEventListener("mouseup",Ye,!0)),Nr.delete(n))};function Pu(){return nn==="keyboard"}function kx(e={}){const{isTextInput:t,autoFocus:n,onChange:r,root:o}=e;yx(o),r?.({isFocusVisible:n||Pu(),modality:nn});const i=(s,a)=>{vx(!!t,s,a)&&r?.({isFocusVisible:Pu(),modality:s})};return Ys.add(i),()=>{Ys.delete(i)}}var Ru=j("checkbox").parts("root","label","control","indicator");Ru.build(),M()(["defaultChecked","checked","dir","disabled","form","getRootNode","id","ids","invalid","name","onCheckedChange","readOnly","required","value"]);const Cx=Ru.extendWith("group"),[OT,Iu]=Mn({name:"FieldContext",hookName:"useFieldContext",providerName:"",strict:!1});var Tu=j("clipboard").parts("root","control","trigger","indicator","input","label");Tu.build(),M()(["getRootNode","id","ids","value","defaultValue","timeout","onStatusChange","onValueChange"]),M()(["copied"]);const wx=gu.extendWith("view");var Sx=Object.defineProperty,Ex=(e,t,n)=>t in e?Sx(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,V=(e,t,n)=>Ex(e,typeof t!="symbol"?t+"":t,n),Ko={itemToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},itemToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Ko.itemToValue(e)},isItemDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1}},Yo=class kp{constructor(t){this.options=t,V(this,"items"),V(this,"indexMap",null),V(this,"copy",n=>new kp({...this.options,items:n??[...this.items]})),V(this,"isEqual",n=>Ze(this.items,n.items)),V(this,"setItems",n=>this.copy(n)),V(this,"getValues",(n=this.items)=>{const r=[];for(const o of n){const i=this.getItemValue(o);i!=null&&r.push(i)}return r}),V(this,"find",n=>{if(n==null)return null;const r=this.indexOf(n);return r!==-1?this.at(r):null}),V(this,"findMany",n=>{const r=[];for(const o of n){const i=this.find(o);i!=null&&r.push(i)}return r}),V(this,"at",n=>{if(!this.options.groupBy&&!this.options.groupSort)return this.items[n]??null;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){if(r===n)return s;r++}return null}),V(this,"sortFn",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return(o??0)-(i??0)}),V(this,"sort",n=>[...n].sort(this.sortFn.bind(this))),V(this,"getItemValue",n=>n==null?null:this.options.itemToValue?.(n)??Ko.itemToValue(n)),V(this,"getItemDisabled",n=>n==null?!1:this.options.isItemDisabled?.(n)??Ko.isItemDisabled(n)),V(this,"stringifyItem",n=>n==null?null:this.options.itemToString?.(n)??Ko.itemToString(n)),V(this,"stringify",n=>n==null?null:this.stringifyItem(this.find(n))),V(this,"stringifyItems",(n,r=", ")=>{const o=[];for(const i of n){const s=this.stringifyItem(i);s!=null&&o.push(s)}return o.join(r)}),V(this,"stringifyMany",(n,r)=>this.stringifyItems(this.findMany(n),r)),V(this,"has",n=>this.indexOf(n)!==-1),V(this,"hasItem",n=>n==null?!1:this.has(this.getItemValue(n))),V(this,"group",()=>{const{groupBy:n,groupSort:r}=this.options;if(!n)return[["",[...this.items]]];const o=new Map;this.items.forEach((s,a)=>{const l=n(s,a);o.has(l)||o.set(l,[]),o.get(l).push(s)});let i=Array.from(o.entries());return r&&i.sort(([s],[a])=>{if(typeof r=="function")return r(s,a);if(Array.isArray(r)){const l=r.indexOf(s),c=r.indexOf(a);return l===-1?1:c===-1?-1:l-c}return r==="asc"?s.localeCompare(a):r==="desc"?a.localeCompare(s):0}),i}),V(this,"getNextValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.min(i+r,this.size-1):i+r;i<=this.size&&this.getItemDisabled(this.at(i));)i++;return this.getItemValue(this.at(i))}),V(this,"getPreviousValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.max(i-r,0):i-r;i>=0&&this.getItemDisabled(this.at(i));)i--;return this.getItemValue(this.at(i))}),V(this,"indexOf",n=>{if(n==null)return-1;if(!this.options.groupBy&&!this.options.groupSort)return this.items.findIndex(r=>this.getItemValue(r)===n);if(!this.indexMap){this.indexMap=new Map;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){const a=this.getItemValue(s);a!=null&&this.indexMap.set(a,r),r++}}return this.indexMap.get(n)??-1}),V(this,"getByText",(n,r)=>{const o=r!=null?this.indexOf(r):-1,i=n.length===1;for(let s=0;s{const{state:o,currentValue:i,timeout:s=350}=r,a=o.keysSoFar+n,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a,u=this.getByText(c,i),h=this.getItemValue(u);function d(){clearTimeout(o.timer),o.timer=-1}function m(f){o.keysSoFar=f,d(),f!==""&&(o.timer=+setTimeout(()=>{m(""),d()},s))}return m(a),h}),V(this,"update",(n,r)=>{let o=this.indexOf(n);return o===-1?this:this.copy([...this.items.slice(0,o),r,...this.items.slice(o+1)])}),V(this,"upsert",(n,r,o="append")=>{let i=this.indexOf(n);return i===-1?(o==="append"?this.append:this.prepend)(r):this.copy([...this.items.slice(0,i),r,...this.items.slice(i+1)])}),V(this,"insert",(n,...r)=>this.copy(Ar(this.items,n,...r))),V(this,"insertBefore",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Ar(this.items,o,...r))}),V(this,"insertAfter",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Ar(this.items,o+1,...r))}),V(this,"prepend",(...n)=>this.copy(Ar(this.items,0,...n))),V(this,"append",(...n)=>this.copy(Ar(this.items,this.items.length,...n))),V(this,"filter",n=>{const r=this.items.filter((o,i)=>n(this.stringifyItem(o),i,o));return this.copy(r)}),V(this,"remove",(...n)=>{const r=n.map(o=>typeof o=="string"?o:this.getItemValue(o));return this.copy(this.items.filter(o=>{const i=this.getItemValue(o);return i==null?!1:!r.includes(i)}))}),V(this,"move",(n,r)=>{const o=this.indexOf(n);return o===-1?this:this.copy(Xo(this.items,[o],r))}),V(this,"moveBefore",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Xo(this.items,i,o))}),V(this,"moveAfter",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Xo(this.items,i,o+1))}),V(this,"reorder",(n,r)=>this.copy(Xo(this.items,[n],r))),V(this,"compareValue",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return oi?1:0}),V(this,"range",(n,r)=>{let o=[],i=n;for(;i!=null;){if(this.find(i)&&o.push(i),i===r)return o;i=this.getNextValue(i)}return[]}),V(this,"getValueRange",(n,r)=>n&&r?this.compareValue(n,r)<=0?this.range(n,r):this.range(r,n):[]),V(this,"toString",()=>{let n="";for(const r of this.items){const o=this.getItemValue(r),i=this.stringifyItem(r),s=this.getItemDisabled(r),a=[o,i,s].filter(Boolean).join(":");n+=a+","}return n}),V(this,"toJSON",()=>({size:this.size,first:this.firstValue,last:this.lastValue})),this.items=[...t.items]}get size(){return this.items.length}get firstValue(){let t=0;for(;this.getItemDisabled(this.at(t));)t++;return this.getItemValue(this.at(t))}get lastValue(){let t=this.size-1;for(;this.getItemDisabled(this.at(t));)t--;return this.getItemValue(this.at(t))}*[Symbol.iterator](){yield*this.items}},Ox=(e,t)=>!!e?.toLowerCase().startsWith(t.toLowerCase());function Ar(e,t,...n){return[...e.slice(0,t),...n,...e.slice(t)]}function Xo(e,t,n){t=[...t].sort((o,i)=>o-i);const r=t.map(o=>e[o]);for(let o=t.length-1;o>=0;o--)e=[...e.slice(0,t[o]),...e.slice(t[o]+1)];return n=Math.max(0,n-t.filter(o=>ot[n])return 1}return e.length-t.length}function Rx(e){return e.sort(Au)}function Ix(e,t){let n;return He(e,{...t,onEnter:(r,o)=>{if(t.predicate(r,o))return n=r,"stop"}}),n}function Tx(e,t){const n=[];return He(e,{onEnter:(r,o)=>{t.predicate(r,o)&&n.push(r)},getChildren:t.getChildren}),n}function _u(e,t){let n;return He(e,{onEnter:(r,o)=>{if(t.predicate(r,o))return n=[...o],"stop"},getChildren:t.getChildren}),n}function Nx(e,t){let n=t.initialResult;return He(e,{...t,onEnter:(r,o)=>{n=t.nextResult(n,r,o)}}),n}function Ax(e,t){return Nx(e,{...t,initialResult:[],nextResult:(n,r,o)=>(n.push(...t.transform(r,o)),n)})}function _x(e,t){const{predicate:n,create:r,getChildren:o}=t,i=(s,a)=>{const l=o(s,a),c=[];l.forEach((m,f)=>{const p=[...a,f],v=i(m,p);v&&c.push(v)});const u=a.length===0,h=n(s,a),d=c.length>0;return u||h||d?r(s,c,a):null};return i(e,[])||r(e,[],[])}function Vx(e,t){const n=[];let r=0;const o=new Map,i=new Map;return He(e,{getChildren:t.getChildren,onEnter:(s,a)=>{o.has(s)||o.set(s,r++);const l=t.getChildren(s,a);l.forEach(m=>{i.has(m)||i.set(m,s),o.has(m)||o.set(m,r++)});const c=l.length>0?l.map(m=>o.get(m)):void 0,u=i.get(s),h=u?o.get(u):void 0,d=o.get(s);n.push({...s,_children:c,_parent:h,_index:d})}}),n}function Fx(e,t){return{type:"insert",index:e,nodes:t}}function Lx(e){return{type:"remove",indexes:e}}function Qs(){return{type:"replace"}}function Vu(e){return[e.slice(0,-1),e[e.length-1]]}function Fu(e,t,n=new Map){const[r,o]=Vu(e);for(let s=r.length-1;s>=0;s--){const a=r.slice(0,s).join();switch(n.get(a)?.type){case"remove":continue}n.set(a,Qs())}const i=n.get(r.join());switch(i?.type){case"remove":n.set(r.join(),{type:"removeThenInsert",removeIndexes:i.indexes,insertIndex:o,insertNodes:t});break;default:n.set(r.join(),Fx(o,t))}return n}function Lu(e){const t=new Map,n=new Map;for(const r of e){const o=r.slice(0,-1).join(),i=n.get(o)??[];i.push(r[r.length-1]),n.set(o,i.sort((s,a)=>s-a))}for(const r of e)for(let o=r.length-2;o>=0;o--){const i=r.slice(0,o).join();t.has(i)||t.set(i,Qs())}for(const[r,o]of n)t.set(r,Lx(o));return t}function Dx(e,t){const n=new Map,[r,o]=Vu(e);for(let i=r.length-1;i>=0;i--){const s=r.slice(0,i).join();n.set(s,Qs())}return n.set(r.join(),{type:"removeThenInsert",removeIndexes:[o],insertIndex:o,insertNodes:[t]}),n}function Qo(e,t,n){return zx(e,{...n,getChildren:(r,o)=>{const i=o.join();switch(t.get(i)?.type){case"replace":case"remove":case"removeThenInsert":case"insert":return n.getChildren(r,o);default:return[]}},transform:(r,o,i)=>{const s=i.join(),a=t.get(s);switch(a?.type){case"remove":return n.create(r,o.filter((u,h)=>!a.indexes.includes(h)),i);case"removeThenInsert":const l=o.filter((u,h)=>!a.removeIndexes.includes(h)),c=a.removeIndexes.reduce((u,h)=>h{const i=[0,...o],s=i.join(),a=t.transform(r,n[s]??[],o),l=i.slice(0,-1).join(),c=n[l]??[];c.push(a),n[l]=c}}),n[""][0]}function Mx(e,t){const{nodes:n,at:r}=t;if(r.length===0)throw new Error("Can't insert nodes at the root");const o=Fu(r,n);return Qo(e,o,t)}function jx(e,t){if(t.at.length===0)return t.node;const n=Dx(t.at,t.node);return Qo(e,n,t)}function $x(e,t){if(t.indexPaths.length===0)return e;for(const r of t.indexPaths)if(r.length===0)throw new Error("Can't remove the root node");const n=Lu(t.indexPaths);return Qo(e,n,t)}function Bx(e,t){if(t.indexPaths.length===0)return e;for(const i of t.indexPaths)if(i.length===0)throw new Error("Can't move the root node");if(t.to.length===0)throw new Error("Can't move nodes to the root");const n=Px(t.indexPaths),r=n.map(i=>Nu(e,i,t)),o=Fu(t.to,r,Lu(n));return Qo(e,o,t)}function He(e,t){const{onEnter:n,onLeave:r,getChildren:o}=t;let i=[],s=[{node:e}];const a=t.reuseIndexPath?()=>i:()=>i.slice();for(;s.length>0;){let l=s[s.length-1];if(l.state===void 0){const u=n?.(l.node,a());if(u==="stop")return;l.state=u==="skip"?-1:0}const c=l.children||o(l.node,a());if(l.children||(l.children=c),l.state!==-1){if(l.stateZe(this.rootNode,n.rootNode)),V(this,"getNodeChildren",n=>this.options.nodeToChildren?.(n)??Yn.nodeToChildren(n)??[]),V(this,"resolveIndexPath",n=>typeof n=="string"?this.getIndexPath(n):n),V(this,"resolveNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(r):void 0}),V(this,"getNodeChildrenCount",n=>this.options.nodeToChildrenCount?.(n)??Yn.nodeToChildrenCount(n)),V(this,"getNodeValue",n=>this.options.nodeToValue?.(n)??Yn.nodeToValue(n)),V(this,"getNodeDisabled",n=>this.options.isNodeDisabled?.(n)??Yn.isNodeDisabled(n)),V(this,"stringify",n=>{const r=this.findNode(n);return r?this.stringifyNode(r):null}),V(this,"stringifyNode",n=>this.options.nodeToString?.(n)??Yn.nodeToString(n)),V(this,"getFirstNode",(n=this.rootNode)=>{let r;return He(n,{getChildren:this.getNodeChildren,onEnter:(o,i)=>{if(!r&&i.length>0&&!this.getNodeDisabled(o))return r=o,"stop"}}),r}),V(this,"getLastNode",(n=this.rootNode,r={})=>{let o;return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isSameNode(i,n)){if(r.skip?.({value:this.getNodeValue(i),node:i,indexPath:s}))return"skip";s.length>0&&!this.getNodeDisabled(i)&&(o=i)}}}),o}),V(this,"at",n=>Nu(this.rootNode,n,{getChildren:this.getNodeChildren})),V(this,"findNode",(n,r=this.rootNode)=>Ix(r,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})),V(this,"findNodes",(n,r=this.rootNode)=>{const o=new Set(n.filter(i=>i!=null));return Tx(r,{getChildren:this.getNodeChildren,predicate:i=>o.has(this.getNodeValue(i))})}),V(this,"sort",n=>n.reduce((r,o)=>{const i=this.getIndexPath(o);return i&&r.push({value:o,indexPath:i}),r},[]).sort((r,o)=>Au(r.indexPath,o.indexPath)).map(({value:r})=>r)),V(this,"getIndexPath",n=>_u(this.rootNode,{getChildren:this.getNodeChildren,predicate:r=>this.getNodeValue(r)===n})),V(this,"getValue",n=>{const r=this.at(n);return r?this.getNodeValue(r):void 0}),V(this,"getValuePath",n=>{if(!n)return[];const r=[];let o=[...n];for(;o.length>0;){const i=this.at(o);i&&r.unshift(this.getNodeValue(i)),o.pop()}return r}),V(this,"getDepth",n=>_u(this.rootNode,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})?.length??0),V(this,"isSameNode",(n,r)=>this.getNodeValue(n)===this.getNodeValue(r)),V(this,"isRootNode",n=>this.isSameNode(n,this.rootNode)),V(this,"contains",(n,r)=>!n||!r?!1:r.slice(0,n.length).every((o,i)=>n[i]===r[i])),V(this,"getNextNode",(n,r={})=>{let o=!1,i;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return l===n&&(o=!0),"skip";if(o&&!this.getNodeDisabled(s))return i=s,"stop";l===n&&(o=!0)}}),i}),V(this,"getPreviousNode",(n,r={})=>{let o,i=!1;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return"skip";if(l===n)return i=!0,"stop";this.getNodeDisabled(s)||(o=s)}}),i?o:void 0}),V(this,"getParentNodes",n=>{const r=this.resolveIndexPath(n)?.slice();if(!r)return[];const o=[];for(;r.length>0;){r.pop();const i=this.at(r);i&&!this.isRootNode(i)&&o.unshift(i)}return o}),V(this,"getDescendantNodes",(n,r)=>{const o=this.resolveNode(n);if(!o)return[];const i=[];return He(o,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{a.length!==0&&(!r?.withBranch&&this.isBranchNode(s)||i.push(s))}}),i}),V(this,"getDescendantValues",(n,r)=>this.getDescendantNodes(n,r).map(i=>this.getNodeValue(i))),V(this,"getParentIndexPath",n=>n.slice(0,-1)),V(this,"getParentNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(this.getParentIndexPath(r)):void 0}),V(this,"visit",n=>{const{skip:r,...o}=n;He(this.rootNode,{...o,getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isRootNode(i))return r?.({value:this.getNodeValue(i),node:i,indexPath:s})?"skip":o.onEnter?.(i,s)}})}),V(this,"getPreviousSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;--i>=0;){const s=o[i];if(!this.getNodeDisabled(s))return s}}),V(this,"getNextSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;++i{const r=this.getParentNode(n);return r?this.getNodeChildren(r):[]}),V(this,"getValues",(n=this.rootNode)=>Ax(n,{getChildren:this.getNodeChildren,transform:o=>[this.getNodeValue(o)]}).slice(1)),V(this,"isValidDepth",(n,r)=>r==null?!0:typeof r=="function"?r(n.length):n.length===r),V(this,"isBranchNode",n=>this.getNodeChildren(n).length>0||this.getNodeChildrenCount(n)!=null),V(this,"getBranchValues",(n=this.rootNode,r={})=>{let o=[];return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(s.length===0)return;const a=this.getNodeValue(i);if(r.skip?.({value:a,node:i,indexPath:s}))return"skip";this.isBranchNode(i)&&this.isValidDepth(s,r.depth)&&o.push(this.getNodeValue(i))}}),o}),V(this,"flatten",(n=this.rootNode)=>Vx(n,{getChildren:this.getNodeChildren})),V(this,"_create",(n,r)=>this.getNodeChildren(n).length>0||r.length>0?{...n,children:r}:{...n}),V(this,"_insert",(n,r,o)=>this.copy(Mx(n,{at:r,nodes:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"copy",n=>new Cp({...this.options,rootNode:n})),V(this,"_replace",(n,r,o)=>this.copy(jx(n,{at:r,node:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_move",(n,r,o)=>this.copy(Bx(n,{indexPaths:r,to:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_remove",(n,r)=>this.copy($x(n,{indexPaths:r,getChildren:this.getNodeChildren,create:this._create}))),V(this,"replace",(n,r)=>this._replace(this.rootNode,n,r)),V(this,"remove",n=>this._remove(this.rootNode,n)),V(this,"insertBefore",(n,r)=>this.getParentNode(n)?this._insert(this.rootNode,n,r):void 0),V(this,"insertAfter",(n,r)=>{if(!this.getParentNode(n))return;const i=[...n.slice(0,-1),n[n.length-1]+1];return this._insert(this.rootNode,i,r)}),V(this,"move",(n,r)=>this._move(this.rootNode,n,r)),V(this,"filter",n=>{const r=_x(this.rootNode,{predicate:n,getChildren:this.getNodeChildren,create:this._create});return this.copy(r)}),V(this,"toJSON",()=>this.getValues(this.rootNode)),this.rootNode=t.rootNode}},Yn={nodeToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},nodeToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Yn.nodeToValue(e)},isNodeDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1},nodeToChildren(e){return e.children},nodeToChildrenCount(e){if(Xt(e)&&Dt(e,"childrenCount"))return e.childrenCount}},Xn=new WeakMap,Jo=new WeakMap,Zo={},Js=0,Mu=e=>e&&(e.host||Mu(e.parentNode)),Wx=(e,t)=>t.map(n=>{if(e.contains(n))return n;const r=Mu(n);return r&&e.contains(r)?r:(console.error("[zag-js > ariaHidden] target",n,"in not contained inside",e,". Doing nothing"),null)}).filter(n=>!!n),Hx=new Set(["script","output","status","next-route-announcer"]),Ux=e=>Hx.has(e.localName)||e.role==="status"||e.hasAttribute("aria-live")?!0:e.matches("[data-live-announcer]"),Gx=(e,t)=>{const{parentNode:n,markerName:r,controlAttribute:o}=t,i=Wx(n,Array.isArray(e)?e:[e]);Zo[r]||(Zo[r]=new WeakMap);const s=Zo[r],a=[],l=new Set,c=new Set(i),u=d=>{!d||l.has(d)||(l.add(d),u(d.parentNode))};i.forEach(u);const h=d=>{!d||c.has(d)||Array.prototype.forEach.call(d.children,m=>{if(l.has(m))h(m);else try{if(Ux(m))return;const p=m.getAttribute(o)==="true",v=(Xn.get(m)||0)+1,b=(s.get(m)||0)+1;Xn.set(m,v),s.set(m,b),a.push(m),v===1&&p&&Jo.set(m,!0),b===1&&m.setAttribute(r,""),p||m.setAttribute(o,"true")}catch(f){console.error("[zag-js > ariaHidden] cannot operate on ",m,f)}})};return h(n),l.clear(),Js++,()=>{a.forEach(d=>{const m=Xn.get(d)-1,f=s.get(d)-1;Xn.set(d,m),s.set(d,f),m||(Jo.has(d)||d.removeAttribute(o),Jo.delete(d)),f||d.removeAttribute(r)}),Js--,Js||(Xn=new WeakMap,Xn=new WeakMap,Jo=new WeakMap,Zo={})}},qx=e=>(Array.isArray(e)?e[0]:e).ownerDocument.body,Kx=(e,t=qx(e),n="data-aria-hidden")=>{if(t)return Gx(e,{parentNode:t,markerName:n,controlAttribute:"aria-hidden"})},Yx=e=>{const t=requestAnimationFrame(()=>e());return()=>cancelAnimationFrame(t)};function ju(e,t={}){const{defer:n=!0}=t,r=n?Yx:i=>i(),o=[];return o.push(r(()=>{const s=(typeof e=="function"?e():e).filter(Boolean);s.length!==0&&o.push(Kx(s))})),()=>{o.forEach(i=>i?.())}}var $u=j("combobox").parts("root","clearTrigger","content","control","input","item","itemGroup","itemGroupLabel","itemIndicator","itemText","label","list","positioner","trigger");$u.build();var Bu=e=>new Yo(e);Bu.empty=()=>new Yo({items:[]});var Xx=e=>e.ids?.control??`combobox:${e.id}:control`,Qx=e=>e.ids?.input??`combobox:${e.id}:input`,Jx=e=>e.ids?.content??`combobox:${e.id}:content`,Zx=e=>e.ids?.positioner??`combobox:${e.id}:popper`,e1=e=>e.ids?.trigger??`combobox:${e.id}:toggle-btn`,t1=e=>e.ids?.clearTrigger??`combobox:${e.id}:clear-btn`,Wt=e=>e.getById(Jx(e)),Qn=e=>e.getById(Qx(e)),Wu=e=>e.getById(Zx(e)),Hu=e=>e.getById(Xx(e)),_r=e=>e.getById(e1(e)),Uu=e=>e.getById(t1(e)),Vr=(e,t)=>{if(t==null)return null;const n=`[role=option][data-value="${CSS.escape(t)}"]`;return pv(Wt(e),n)},Gu=e=>{const t=Qn(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},n1=e=>{const t=_r(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},{guards:r1,createMachine:o1,choose:i1}=wc(),{and:ye,not:Ue}=r1;o1({props({props:e}){return{loopFocus:!0,openOnClick:!1,defaultValue:[],closeOnSelect:!e.multiple,allowCustomValue:!1,alwaysSubmitOnEnter:!1,inputBehavior:"none",selectionBehavior:e.multiple?"clear":"replace",openOnKeyPress:!0,openOnChange:!0,composite:!0,navigate({node:t}){vc(t)},collection:Bu.empty(),...e,positioning:{placement:"bottom",sameWidth:!0,...e.positioning},translations:{triggerLabel:"Toggle suggestions",clearTriggerLabel:"Clear value",...e.translations}}},initialState({prop:e}){return e("open")||e("defaultOpen")?"suggesting":"idle"},context({prop:e,bindable:t,getContext:n,getEvent:r}){return{currentPlacement:t(()=>({defaultValue:void 0})),value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),isEqual:Ze,hash(o){return o.join(",")},onChange(o){const i=n(),s=i.get("selectedItems"),a=e("collection"),l=o.map(c=>s.find(h=>a.getItemValue(h)===c)||a.find(c));i.set("selectedItems",l),e("onValueChange")?.({value:o,items:l})}})),highlightedValue:t(()=>({defaultValue:e("defaultHighlightedValue")||null,value:e("highlightedValue"),onChange(o){const i=e("collection").find(o);e("onHighlightChange")?.({highlightedValue:o,highlightedItem:i})}})),inputValue:t(()=>{let o=e("inputValue")||e("defaultInputValue")||"";const i=e("defaultValue")||e("value")||[];if(!o.trim()&&!e("multiple")){const s=e("collection").stringifyMany(i);o=xt(e("selectionBehavior"),{preserve:o||s,replace:s,clear:""})}return{defaultValue:o,value:e("inputValue"),onChange(s){const a=r(),l=(a.previousEvent||a).src;e("onInputValueChange")?.({inputValue:s,reason:l})}}}),highlightedItem:t(()=>{const o=e("highlightedValue");return{defaultValue:e("collection").find(o)}}),selectedItems:t(()=>{const o=e("value")||e("defaultValue")||[];return{defaultValue:e("collection").findMany(o)}})}},computed:{isInputValueEmpty:({context:e})=>e.get("inputValue").length===0,isInteractive:({prop:e})=>!(e("readOnly")||e("disabled")),autoComplete:({prop:e})=>e("inputBehavior")==="autocomplete",autoHighlight:({prop:e})=>e("inputBehavior")==="autohighlight",hasSelectedItems:({context:e})=>e.get("value").length>0,valueAsString:({context:e,prop:t})=>t("collection").stringifyItems(e.get("selectedItems")),isCustomValue:({context:e,computed:t})=>e.get("inputValue")!==t("valueAsString")},watch({context:e,prop:t,track:n,action:r,send:o}){n([()=>e.hash("value")],()=>{r(["syncSelectedItems"])}),n([()=>e.get("inputValue")],()=>{r(["syncInputValue"])}),n([()=>e.get("highlightedValue")],()=>{r(["syncHighlightedItem","autofillInputValue"])}),n([()=>t("open")],()=>{r(["toggleVisibility"])}),n([()=>t("collection").toString()],()=>{o({type:"CHILDREN_CHANGE"})})},on:{"SELECTED_ITEMS.SYNC":{actions:["syncSelectedItems"]},"HIGHLIGHTED_VALUE.SET":{actions:["setHighlightedValue"]},"HIGHLIGHTED_VALUE.CLEAR":{actions:["clearHighlightedValue"]},"ITEM.SELECT":{actions:["selectItem"]},"ITEM.CLEAR":{actions:["clearItem"]},"VALUE.SET":{actions:["setValue"]},"INPUT_VALUE.SET":{actions:["setInputValue"]},"POSITIONING.SET":{actions:["reposition"]}},entry:i1([{guard:"autoFocus",actions:["setInitialFocus"]}]),states:{idle:{tags:["idle","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":{target:"interacting"},"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.FOCUS":{target:"focused"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{target:"focused",actions:["clearInputValue","clearSelectedItems","setInitialFocus"]}}},focused:{tags:["focused","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":[{guard:"isChangeEvent",target:"suggesting"},{target:"interacting"}],"INPUT.CHANGE":[{guard:ye("isOpenControlled","openOnChange"),actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{guard:"openOnChange",target:"suggesting",actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{actions:["setInputValue"]}],"LAYER.INTERACT_OUTSIDE":{target:"idle"},"INPUT.ESCAPE":{guard:ye("isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue"]},"INPUT.BLUR":{target:"idle"},"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_DOWN":[{guard:ye("isOpenControlled","autoComplete"),actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"isOpenControlled",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_UP":[{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]}],OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{actions:["clearInputValue","clearSelectedItems"]}}},interacting:{tags:["open","focused"],entry:["setInitialFocus"],effects:["scrollToHighlightedItem","trackDismissableLayer","trackPlacement","hideOtherElements"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]},{actions:["scrollToHighlightedItem"]}],"INPUT.HOME":{actions:["highlightFirstItem"]},"INPUT.END":{actions:["highlightLastItem"]},"INPUT.ARROW_DOWN":[{guard:ye("autoComplete","isLastItemHighlighted"),actions:["clearHighlightedValue","scrollContentToTop"]},{actions:["highlightNextItem"]}],"INPUT.ARROW_UP":[{guard:ye("autoComplete","isFirstItemHighlighted"),actions:["clearHighlightedValue"]},{actions:["highlightPrevItem"]}],"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":[{guard:"autoComplete",target:"suggesting",actions:["setInputValue"]},{target:"suggesting",actions:["clearHighlightedValue","setInputValue"]}],"ITEM.POINTER_MOVE":{actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],"LAYER.ESCAPE":[{guard:ye("isOpenControlled","autoComplete"),actions:["syncInputValue","invokeOnClose"]},{guard:"autoComplete",target:"focused",actions:["syncInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}},suggesting:{tags:["open","focused"],effects:["trackDismissableLayer","scrollToHighlightedItem","trackPlacement","hideOtherElements"],entry:["setInitialFocus"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"autoHighlight",actions:["highlightFirstItem"]},{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]}],"INPUT.ARROW_DOWN":{target:"interacting",actions:["highlightNextItem"]},"INPUT.ARROW_UP":{target:"interacting",actions:["highlightPrevItem"]},"INPUT.HOME":{target:"interacting",actions:["highlightFirstItem"]},"INPUT.END":{target:"interacting",actions:["highlightLastItem"]},"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":{actions:["setInputValue"]},"LAYER.ESCAPE":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.POINTER_MOVE":{target:"interacting",actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}}},implementations:{guards:{isInputValueEmpty:({computed:e})=>e("isInputValueEmpty"),autoComplete:({computed:e,prop:t})=>e("autoComplete")&&!t("multiple"),autoHighlight:({computed:e})=>e("autoHighlight"),isFirstItemHighlighted:({prop:e,context:t})=>e("collection").firstValue===t.get("highlightedValue"),isLastItemHighlighted:({prop:e,context:t})=>e("collection").lastValue===t.get("highlightedValue"),isCustomValue:({computed:e})=>e("isCustomValue"),allowCustomValue:({prop:e})=>!!e("allowCustomValue"),hasHighlightedItem:({context:e})=>e.get("highlightedValue")!=null,closeOnSelect:({prop:e})=>!!e("closeOnSelect"),isOpenControlled:({prop:e})=>e("open")!=null,openOnChange:({prop:e,context:t})=>{const n=e("openOnChange");return Ym(n)?n:!!n?.({inputValue:t.get("inputValue")})},restoreFocus:({event:e})=>e.restoreFocus==null?!0:!!e.restoreFocus,isChangeEvent:({event:e})=>e.previousEvent?.type==="INPUT.CHANGE",autoFocus:({prop:e})=>!!e("autoFocus"),isHighlightedItemRemoved:({prop:e,context:t})=>!e("collection").has(t.get("highlightedValue"))},effects:{trackDismissableLayer({send:e,prop:t,scope:n}){return t("disableLayer")?void 0:Kn(()=>Wt(n),{type:"listbox",defer:!0,exclude:()=>[Qn(n),_r(n),Uu(n)],onFocusOutside:t("onFocusOutside"),onPointerDownOutside:t("onPointerDownOutside"),onInteractOutside:t("onInteractOutside"),onEscapeKeyDown(o){o.preventDefault(),o.stopPropagation(),e({type:"LAYER.ESCAPE",src:"escape-key"})},onDismiss(){e({type:"LAYER.INTERACT_OUTSIDE",src:"interact-outside",restoreFocus:!1})}})},hideOtherElements({scope:e}){return ju([Qn(e),Wt(e),_r(e),Uu(e)])},trackPlacement({context:e,prop:t,scope:n}){const r=()=>Hu(n)||_r(n),o=()=>Wu(n);return e.set("currentPlacement",t("positioning").placement),nt(r,o,{...t("positioning"),defer:!0,onComplete(i){e.set("currentPlacement",i.placement)}})},scrollToHighlightedItem({context:e,prop:t,scope:n,event:r}){const o=Qn(n);let i=[];const s=c=>{const u=r.current().type.includes("POINTER"),h=e.get("highlightedValue");if(u||!h)return;const d=Wt(n),m=t("scrollToIndexFn");if(m){const v=t("collection").indexOf(h);m({index:v,immediate:c,getElement:()=>Vr(n,h)});return}const f=Vr(n,h),p=G(()=>{Po(f,{rootEl:d,block:"nearest"})});i.push(p)},a=G(()=>s(!0));i.push(a);const l=Eo(o,{attributes:["aria-activedescendant"],callback:()=>s(!1)});return i.push(l),()=>{i.forEach(c=>c())}}},actions:{reposition({context:e,prop:t,scope:n,event:r}){nt(()=>Hu(n),()=>Wu(n),{...t("positioning"),...r.options,defer:!0,listeners:!1,onComplete(s){e.set("currentPlacement",s.placement)}})},setHighlightedValue({context:e,event:t}){t.value!=null&&e.set("highlightedValue",t.value)},clearHighlightedValue({context:e}){e.set("highlightedValue",null)},selectHighlightedItem(e){const{context:t,prop:n}=e,r=n("collection"),o=t.get("highlightedValue");if(!o||!r.has(o))return;const i=n("multiple")?zn(t.get("value"),o):[o];n("onSelect")?.({value:i,itemValue:o}),t.set("value",i);const s=xt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(i),clear:""});t.set("inputValue",s)},scrollToHighlightedItem({context:e,prop:t,scope:n}){ws(()=>{const r=e.get("highlightedValue");if(r==null)return;const o=Vr(n,r),i=Wt(n),s=t("scrollToIndexFn");if(s){const a=t("collection").indexOf(r);s({index:a,immediate:!0,getElement:()=>Vr(n,r)});return}Po(o,{rootEl:i,block:"nearest"})})},selectItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=o("multiple")?zn(t.get("value"),n.value):[n.value];o("onSelect")?.({value:i,itemValue:n.value}),t.set("value",i);const s=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},clearItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=Yt(t.get("value"),n.value);t.set("value",i);const s=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},setInitialFocus({scope:e}){G(()=>{Gu(e)})},setFinalFocus({scope:e}){G(()=>{_r(e)?.dataset.focusable==null?Gu(e):n1(e)})},syncInputValue({context:e,scope:t,event:n}){const r=Qn(t);r&&(r.value=e.get("inputValue"),queueMicrotask(()=>{n.current().type!=="INPUT.CHANGE"&&k0(r)}))},setInputValue({context:e,event:t}){e.set("inputValue",t.value)},clearInputValue({context:e}){e.set("inputValue","")},revertInputValue({context:e,prop:t,computed:n}){const r=t("selectionBehavior"),o=xt(r,{replace:n("hasSelectedItems")?n("valueAsString"):"",preserve:e.get("inputValue"),clear:""});e.set("inputValue",o)},setValue(e){const{context:t,flush:n,event:r,prop:o}=e;n(()=>{t.set("value",r.value);const i=xt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(r.value),clear:""});t.set("inputValue",i)})},clearSelectedItems(e){const{context:t,flush:n,prop:r}=e;n(()=>{t.set("value",[]);const o=xt(r("selectionBehavior"),{preserve:t.get("inputValue"),replace:r("collection").stringifyMany([]),clear:""});t.set("inputValue",o)})},scrollContentToTop({prop:e,scope:t}){const n=e("scrollToIndexFn");if(n){const r=e("collection").firstValue;n({index:0,immediate:!0,getElement:()=>Vr(t,r)})}else{const r=Wt(t);if(!r)return;r.scrollTop=0}},invokeOnOpen({prop:e,event:t}){const n=qu(t);e("onOpenChange")?.({open:!0,reason:n})},invokeOnClose({prop:e,event:t}){const n=qu(t);e("onOpenChange")?.({open:!1,reason:n})},highlightFirstItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").firstValue;o&&e.set("highlightedValue",o)})},highlightFirstItemIfNeeded({computed:e,action:t}){e("autoHighlight")&&t(["highlightFirstItem"])},highlightLastItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").lastValue;o&&e.set("highlightedValue",o)})},highlightNextItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getNextValue(r),!n&&t("loopFocus")&&(n=o.firstValue)):n=o.firstValue,n&&e.set("highlightedValue",n)},highlightPrevItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getPreviousValue(r),!n&&t("loopFocus")&&(n=o.lastValue)):n=o.lastValue,n&&e.set("highlightedValue",n)},highlightFirstSelectedItem({context:e,prop:t}){G(()=>{const[n]=t("collection").sort(e.get("value"));n&&e.set("highlightedValue",n)})},highlightFirstOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{let r=null;n("hasSelectedItems")?r=t("collection").sort(e.get("value"))[0]:r=t("collection").firstValue,r&&e.set("highlightedValue",r)})},highlightLastOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{const r=t("collection");let o=null;n("hasSelectedItems")?o=r.sort(e.get("value"))[0]:o=r.lastValue,o&&e.set("highlightedValue",o)})},autofillInputValue({context:e,computed:t,prop:n,event:r,scope:o}){const i=Qn(o),s=n("collection");if(!t("autoComplete")||!i||!r.keypress)return;const a=s.stringify(e.get("highlightedValue"));G(()=>{i.value=a||e.get("inputValue")})},syncSelectedItems(e){queueMicrotask(()=>{const{context:t,prop:n}=e,r=n("collection"),o=t.get("value"),i=o.map(a=>t.get("selectedItems").find(c=>r.getItemValue(c)===a)||r.find(a));t.set("selectedItems",i);const s=xt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(o),clear:""});t.set("inputValue",s)})},syncHighlightedItem({context:e,prop:t}){const n=t("collection").find(e.get("highlightedValue"));e.set("highlightedItem",n)},toggleVisibility({event:e,send:t,prop:n}){t({type:n("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:e})}}}});function qu(e){return(e.previousEvent||e).src}M()(["allowCustomValue","autoFocus","closeOnSelect","collection","composite","defaultHighlightedValue","defaultInputValue","defaultOpen","defaultValue","dir","disabled","disableLayer","form","getRootNode","highlightedValue","id","ids","inputBehavior","inputValue","invalid","loopFocus","multiple","name","navigate","onFocusOutside","onHighlightChange","onInputValueChange","onInteractOutside","onOpenChange","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","openOnChange","openOnClick","openOnKeyPress","placeholder","positioning","readOnly","required","scrollToIndexFn","selectionBehavior","translations","value","alwaysSubmitOnEnter"]),M()(["htmlFor"]),M()(["id"]),M()(["item","persistFocus"]);const s1=$u.extendWith("empty"),[Ku,Ht]=Mn({name:"DialogContext",hookName:"useDialogContext",providerName:""}),Yu=P.forwardRef((e,t)=>{const n=Ht(),r=rx(),o=Ts({...r,present:n.open}),i=qe(n.getBackdropProps(),o.getPresenceProps(),e);return o.unmounted?null:g.jsx(kt.div,{...i,ref:No(o.ref,t)})});Yu.displayName="DialogBackdrop";const Xu=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getCloseTriggerProps(),e);return g.jsx(kt.button,{...r,ref:t})});Xu.displayName="DialogCloseTrigger";const Qu=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe(n.getContentProps(),r.getPresenceProps(),e);return r.unmounted?null:g.jsx(kt.div,{...o,ref:No(r.ref,t)})});Qu.displayName="DialogContent";const Ju=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getDescriptionProps(),e);return g.jsx(kt.div,{...r,ref:t})});Ju.displayName="DialogDescription";const Zu=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getPositionerProps(),e);return Ns().unmounted?null:g.jsx(kt.div,{...r,ref:t})});Zu.displayName="DialogPositioner";var a1=Object.defineProperty,l1=(e,t,n)=>t in e?a1(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,ae=(e,t,n)=>l1(e,typeof t!="symbol"?t+"":t,n),ed={activateTrap(e,t){if(e.length>0){const r=e[e.length-1];r!==t&&r.pause()}const n=e.indexOf(t);n===-1||e.splice(n,1),e.push(t)},deactivateTrap(e,t){const n=e.indexOf(t);n!==-1&&e.splice(n,1),e.length>0&&e[e.length-1].unpause()}},c1=[],u1=class{constructor(e,t){ae(this,"trapStack"),ae(this,"config"),ae(this,"doc"),ae(this,"state",{containers:[],containerGroups:[],tabbableGroups:[],nodeFocusedBeforeActivation:null,mostRecentlyFocusedNode:null,active:!1,paused:!1,delayInitialFocusTimer:void 0,recentNavEvent:void 0}),ae(this,"listenerCleanups",[]),ae(this,"handleFocus",r=>{const o=$e(r),i=this.findContainerIndex(o,r)>=0;if(i||ms(o))i&&(this.state.mostRecentlyFocusedNode=o);else{r.stopImmediatePropagation();let s,a=!0;if(this.state.mostRecentlyFocusedNode)if(Er(this.state.mostRecentlyFocusedNode)>0){const l=this.findContainerIndex(this.state.mostRecentlyFocusedNode),{tabbableNodes:c}=this.state.containerGroups[l];if(c.length>0){const u=c.findIndex(h=>h===this.state.mostRecentlyFocusedNode);u>=0&&(this.config.isKeyForward(this.state.recentNavEvent)?u+1=0&&(s=c[u-1],a=!1))}}else this.state.containerGroups.some(l=>l.tabbableNodes.some(c=>Er(c)>0))||(a=!1);else a=!1;a&&(s=this.findNextNavNode({target:this.state.mostRecentlyFocusedNode,isBackward:this.config.isKeyBackward(this.state.recentNavEvent)})),s?this.tryFocus(s):this.tryFocus(this.state.mostRecentlyFocusedNode||this.getInitialFocusNode())}this.state.recentNavEvent=void 0}),ae(this,"handlePointerDown",r=>{const o=$e(r);if(!(this.findContainerIndex(o,r)>=0)){if(Lr(this.config.clickOutsideDeactivates,r)){this.deactivate({returnFocus:this.config.returnFocusOnDeactivate});return}Lr(this.config.allowOutsideClick,r)||r.preventDefault()}}),ae(this,"handleClick",r=>{const o=$e(r);this.findContainerIndex(o,r)>=0||Lr(this.config.clickOutsideDeactivates,r)||Lr(this.config.allowOutsideClick,r)||(r.preventDefault(),r.stopImmediatePropagation())}),ae(this,"handleTabKey",r=>{if(this.config.isKeyForward(r)||this.config.isKeyBackward(r)){this.state.recentNavEvent=r;const o=this.config.isKeyBackward(r),i=this.findNextNavNode({event:r,isBackward:o});if(!i)return;Fr(r)&&r.preventDefault(),this.tryFocus(i)}}),ae(this,"handleEscapeKey",r=>{d1(r)&&Lr(this.config.escapeDeactivates,r)!==!1&&(r.preventDefault(),this.deactivate())}),ae(this,"_mutationObserver"),ae(this,"setupMutationObserver",()=>{const r=this.doc.defaultView||window;this._mutationObserver=new r.MutationObserver(o=>{o.some(s=>Array.from(s.removedNodes).some(l=>l===this.state.mostRecentlyFocusedNode))&&this.tryFocus(this.getInitialFocusNode())})}),ae(this,"updateObservedNodes",()=>{this._mutationObserver?.disconnect(),this.state.active&&!this.state.paused&&this.state.containers.map(r=>{this._mutationObserver?.observe(r,{subtree:!0,childList:!0})})}),ae(this,"getInitialFocusNode",()=>{let r=this.getNodeForOption("initialFocus",{hasFallback:!0});if(r===!1)return!1;if(r===void 0||r&&!Mt(r)){const o=Sr(this.doc);if(o&&this.findContainerIndex(o)>=0)r=o;else{const i=this.state.tabbableGroups[0];r=i&&i.firstTabbableNode||this.getNodeForOption("fallbackFocus")}}else r===null&&(r=this.getNodeForOption("fallbackFocus"));if(!r)throw new Error("Your focus-trap needs to have at least one focusable element");return r.isConnected||(r=this.getNodeForOption("fallbackFocus")),r}),ae(this,"tryFocus",r=>{if(r!==!1&&r!==Sr(this.doc)){if(!r||!r.focus){this.tryFocus(this.getInitialFocusNode());return}r.focus({preventScroll:!!this.config.preventScroll}),this.state.mostRecentlyFocusedNode=r,h1(r)&&r.select()}}),ae(this,"deactivate",r=>{if(!this.state.active)return this;const o={onDeactivate:this.config.onDeactivate,onPostDeactivate:this.config.onPostDeactivate,checkCanReturnFocus:this.config.checkCanReturnFocus,...r};clearTimeout(this.state.delayInitialFocusTimer),this.state.delayInitialFocusTimer=void 0,this.removeListeners(),this.state.active=!1,this.state.paused=!1,this.updateObservedNodes(),ed.deactivateTrap(this.trapStack,this);const i=this.getOption(o,"onDeactivate"),s=this.getOption(o,"onPostDeactivate"),a=this.getOption(o,"checkCanReturnFocus"),l=this.getOption(o,"returnFocus","returnFocusOnDeactivate");i?.();const c=()=>{td(()=>{if(l){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);this.tryFocus(u)}s?.()})};if(l&&a){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);return a(u).then(c,c),this}return c(),this}),ae(this,"pause",r=>{if(this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onPause"),i=this.getOption(r,"onPostPause");return this.state.paused=!0,o?.(),this.removeListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"unpause",r=>{if(!this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onUnpause"),i=this.getOption(r,"onPostUnpause");return this.state.paused=!1,o?.(),this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"updateContainerElements",r=>(this.state.containers=Array.isArray(r)?r.filter(Boolean):[r].filter(Boolean),this.state.active&&this.updateTabbableNodes(),this.updateObservedNodes(),this)),ae(this,"getReturnFocusNode",r=>{const o=this.getNodeForOption("setReturnFocus",{params:[r]});return o||(o===!1?!1:r)}),ae(this,"getOption",(r,o,i)=>r&&r[o]!==void 0?r[o]:this.config[i||o]),ae(this,"getNodeForOption",(r,{hasFallback:o=!1,params:i=[]}={})=>{let s=this.config[r];if(typeof s=="function"&&(s=s(...i)),s===!0&&(s=void 0),!s){if(s===void 0||s===!1)return s;throw new Error(`\`${r}\` was specified but was not a node, or did not return a node`)}let a=s;if(typeof s=="string"){try{a=this.doc.querySelector(s)}catch(l){throw new Error(`\`${r}\` appears to be an invalid selector; error="${l.message}"`)}if(!a&&!o)throw new Error(`\`${r}\` as selector refers to no known node`)}return a}),ae(this,"findNextNavNode",r=>{const{event:o,isBackward:i=!1}=r,s=r.target||$e(o);this.updateTabbableNodes();let a=null;if(this.state.tabbableGroups.length>0){const l=this.findContainerIndex(s,o),c=l>=0?this.state.containerGroups[l]:void 0;if(l<0)i?a=this.state.tabbableGroups[this.state.tabbableGroups.length-1].lastTabbableNode:a=this.state.tabbableGroups[0].firstTabbableNode;else if(i){let u=this.state.tabbableGroups.findIndex(({firstTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s,!1))&&(u=l),u>=0){const h=u===0?this.state.tabbableGroups.length-1:u-1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.lastTabbableNode:d.lastDomTabbableNode}else Fr(o)||(a=c?.nextTabbableNode(s,!1))}else{let u=this.state.tabbableGroups.findIndex(({lastTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s))&&(u=l),u>=0){const h=u===this.state.tabbableGroups.length-1?0:u+1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.firstTabbableNode:d.firstDomTabbableNode}else Fr(o)||(a=c?.nextTabbableNode(s))}}else a=this.getNodeForOption("fallbackFocus");return a}),this.trapStack=t.trapStack||c1;const n={returnFocusOnDeactivate:!0,escapeDeactivates:!0,delayInitialFocus:!0,isKeyForward(r){return Fr(r)&&!r.shiftKey},isKeyBackward(r){return Fr(r)&&r.shiftKey},...t};this.doc=n.document||Fe(Array.isArray(e)?e[0]:e),this.config=n,this.updateContainerElements(e),this.setupMutationObserver()}get active(){return this.state.active}get paused(){return this.state.paused}findContainerIndex(e,t){const n=typeof t?.composedPath=="function"?t.composedPath():void 0;return this.state.containerGroups.findIndex(({container:r,tabbableNodes:o})=>r.contains(e)||n?.includes(r)||o.find(i=>i===e))}updateTabbableNodes(){if(this.state.containerGroups=this.state.containers.map(e=>{const t=ks(e),n=mc(e),r=t.length>0?t[0]:void 0,o=t.length>0?t[t.length-1]:void 0,i=n.find(c=>Zt(c)),s=n.slice().reverse().find(c=>Zt(c)),a=!!t.find(c=>Er(c)>0);function l(c,u=!0){const h=t.indexOf(c);return h<0?u?n.slice(n.indexOf(c)+1).find(d=>Zt(d)):n.slice(0,n.indexOf(c)).reverse().find(d=>Zt(d)):t[h+(u?1:-1)]}return{container:e,tabbableNodes:t,focusableNodes:n,posTabIndexesFound:a,firstTabbableNode:r,lastTabbableNode:o,firstDomTabbableNode:i,lastDomTabbableNode:s,nextTabbableNode:l}}),this.state.tabbableGroups=this.state.containerGroups.filter(e=>e.tabbableNodes.length>0),this.state.tabbableGroups.length<=0&&!this.getNodeForOption("fallbackFocus"))throw new Error("Your focus-trap must have at least one container with at least one tabbable node in it at all times");if(this.state.containerGroups.find(e=>e.posTabIndexesFound)&&this.state.containerGroups.length>1)throw new Error("At least one node with a positive tabindex was found in one of your focus-trap's multiple containers. Positive tabindexes are only supported in single-container focus-traps.")}addListeners(){if(this.state.active)return ed.activateTrap(this.trapStack,this),this.state.delayInitialFocusTimer=this.config.delayInitialFocus?td(()=>{this.tryFocus(this.getInitialFocusNode())}):this.tryFocus(this.getInitialFocusNode()),this.listenerCleanups.push(se(this.doc,"focusin",this.handleFocus,!0),se(this.doc,"mousedown",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"touchstart",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"click",this.handleClick,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleTabKey,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleEscapeKey)),this}removeListeners(){if(this.state.active)return this.listenerCleanups.forEach(e=>e()),this.listenerCleanups=[],this}activate(e){if(this.state.active)return this;const t=this.getOption(e,"onActivate"),n=this.getOption(e,"onPostActivate"),r=this.getOption(e,"checkCanFocusTrap");r||this.updateTabbableNodes(),this.state.active=!0,this.state.paused=!1,this.state.nodeFocusedBeforeActivation=Sr(this.doc),t?.();const o=()=>{r&&this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),n?.()};return r?(r(this.state.containers.concat()).then(o,o),this):(o(),this)}},Fr=e=>e.key==="Tab",Lr=(e,...t)=>typeof e=="function"?e(...t):e,d1=e=>!e.isComposing&&e.key==="Escape",td=e=>setTimeout(e,0),h1=e=>e.localName==="input"&&"select"in e&&typeof e.select=="function";function f1(e,t={}){let n;const r=G(()=>{const o=typeof e=="function"?e():e;if(o){n=new u1(o,{escapeDeactivates:!1,allowOutsideClick:!0,preventScroll:!0,returnFocusOnDeactivate:!0,delayInitialFocus:!1,fallbackFocus:o,...t,document:Fe(o)});try{n.activate()}catch{}}});return function(){n?.deactivate(),r()}}var Zs="data-scroll-lock";function g1(e){const t=e.getBoundingClientRect().left;return Math.round(t)+e.scrollLeft?"paddingLeft":"paddingRight"}function p1(e){const t=e??document,n=t.defaultView??window,{documentElement:r,body:o}=t;if(o.hasAttribute(Zs))return;const s=n.innerWidth-r.clientWidth;o.setAttribute(Zs,"");const a=()=>wv(r,"--scrollbar-width",`${s}px`),l=g1(r),c=()=>To(o,{overflow:"hidden",[l]:`${s}px`}),u=()=>{const{scrollX:d,scrollY:m,visualViewport:f}=n,p=f?.offsetLeft??0,v=f?.offsetTop??0,b=To(o,{position:"fixed",overflow:"hidden",top:`${-(m-Math.floor(v))}px`,left:`${-(d-Math.floor(p))}px`,right:"0",[l]:`${s}px`});return()=>{b?.(),n.scrollTo({left:d,top:m,behavior:"instant"})}},h=[a(),Co()?u():c()];return()=>{h.forEach(d=>d?.()),o.removeAttribute(Zs)}}var ea=j("dialog").parts("trigger","backdrop","positioner","content","title","description","closeTrigger"),on=ea.build(),nd=e=>e.ids?.positioner??`dialog:${e.id}:positioner`,rd=e=>e.ids?.backdrop??`dialog:${e.id}:backdrop`,ta=e=>e.ids?.content??`dialog:${e.id}:content`,od=e=>e.ids?.trigger??`dialog:${e.id}:trigger`,na=e=>e.ids?.title??`dialog:${e.id}:title`,ra=e=>e.ids?.description??`dialog:${e.id}:description`,id=e=>e.ids?.closeTrigger??`dialog:${e.id}:close`,ei=e=>e.getById(ta(e)),m1=e=>e.getById(nd(e)),v1=e=>e.getById(rd(e)),b1=e=>e.getById(od(e)),y1=e=>e.getById(na(e)),x1=e=>e.getById(ra(e)),k1=e=>e.getById(id(e));function C1(e,t){const{state:n,send:r,context:o,prop:i,scope:s}=e,a=i("aria-label"),l=n.matches("open");return{open:l,setOpen(c){n.matches("open")!==c&&r({type:c?"OPEN":"CLOSE"})},getTriggerProps(){return t.button({...on.trigger.attrs,dir:i("dir"),id:od(s),"aria-haspopup":"dialog",type:"button","aria-expanded":l,"data-state":l?"open":"closed","aria-controls":ta(s),onClick(c){c.defaultPrevented||r({type:"TOGGLE"})}})},getBackdropProps(){return t.element({...on.backdrop.attrs,dir:i("dir"),hidden:!l,id:rd(s),"data-state":l?"open":"closed"})},getPositionerProps(){return t.element({...on.positioner.attrs,dir:i("dir"),id:nd(s),style:{pointerEvents:l?void 0:"none"}})},getContentProps(){const c=o.get("rendered");return t.element({...on.content.attrs,dir:i("dir"),role:i("role"),hidden:!l,id:ta(s),tabIndex:-1,"data-state":l?"open":"closed","aria-modal":!0,"aria-label":a||void 0,"aria-labelledby":a||!c.title?void 0:na(s),"aria-describedby":c.description?ra(s):void 0})},getTitleProps(){return t.element({...on.title.attrs,dir:i("dir"),id:na(s)})},getDescriptionProps(){return t.element({...on.description.attrs,dir:i("dir"),id:ra(s)})},getCloseTriggerProps(){return t.button({...on.closeTrigger.attrs,dir:i("dir"),id:id(s),type:"button",onClick(c){c.defaultPrevented||(c.stopPropagation(),r({type:"CLOSE"}))}})}}}var w1={props({props:e,scope:t}){const n=e.role==="alertdialog",r=n?()=>k1(t):void 0,o=typeof e.modal=="boolean"?e.modal:!0;return{role:"dialog",modal:o,trapFocus:o,preventScroll:o,closeOnInteractOutside:!n,closeOnEscape:!0,restoreFocus:!0,initialFocusEl:r,...e}},initialState({prop:e}){return e("open")||e("defaultOpen")?"open":"closed"},context({bindable:e}){return{rendered:e(()=>({defaultValue:{title:!0,description:!0}}))}},watch({track:e,action:t,prop:n}){e([()=>n("open")],()=>{t(["toggleVisibility"])})},states:{open:{entry:["checkRenderedElements","syncZIndex"],effects:["trackDismissableElement","trapFocus","preventScroll","hideContentBelow"],on:{"CONTROLLED.CLOSE":{target:"closed"},CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}]}},closed:{on:{"CONTROLLED.OPEN":{target:"open"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}]}}},implementations:{guards:{isOpenControlled:({prop:e})=>e("open")!=null},effects:{trackDismissableElement({scope:e,send:t,prop:n}){return Kn(()=>ei(e),{type:"dialog",defer:!0,pointerBlocking:n("modal"),exclude:[b1(e)],onInteractOutside(o){n("onInteractOutside")?.(o),n("closeOnInteractOutside")||o.preventDefault()},persistentElements:n("persistentElements"),onFocusOutside:n("onFocusOutside"),onPointerDownOutside:n("onPointerDownOutside"),onRequestDismiss:n("onRequestDismiss"),onEscapeKeyDown(o){n("onEscapeKeyDown")?.(o),n("closeOnEscape")||o.preventDefault()},onDismiss(){t({type:"CLOSE",src:"interact-outside"})}})},preventScroll({scope:e,prop:t}){if(t("preventScroll"))return p1(e.getDoc())},trapFocus({scope:e,prop:t}){return t("trapFocus")?f1(()=>ei(e),{preventScroll:!0,returnFocusOnDeactivate:!!t("restoreFocus"),initialFocus:t("initialFocusEl"),setReturnFocus:r=>t("finalFocusEl")?.()??r}):void 0},hideContentBelow({scope:e,prop:t}){return t("modal")?ju(()=>[ei(e)],{defer:!0}):void 0}},actions:{checkRenderedElements({context:e,scope:t}){G(()=>{e.set("rendered",{title:!!y1(t),description:!!x1(t)})})},syncZIndex({scope:e}){G(()=>{const t=ei(e);if(!t)return;const n=xo(t);[m1(e),v1(e)].forEach(o=>{o?.style.setProperty("--z-index",n.zIndex),o?.style.setProperty("--layer-index",n.getPropertyValue("--layer-index"))})})},invokeOnClose({prop:e}){e("onOpenChange")?.({open:!1})},invokeOnOpen({prop:e}){e("onOpenChange")?.({open:!0})},toggleVisibility({prop:e,send:t,event:n}){t({type:e("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:n})}}}};M()(["aria-label","closeOnEscape","closeOnInteractOutside","dir","finalFocusEl","getRootNode","getRootNode","id","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","preventScroll","restoreFocus","role","trapFocus"]);const S1=e=>{const t=P.useId(),{getRootNode:n}=sc(),{dir:r}=_v(),o={id:t,getRootNode:n,dir:r,...e},i=Pc(w1,o);return C1(i,jv)},E1=e=>{const[t,{children:n,...r}]=Ec(e),[o]=yu(t),i=S1(r),s=Ts(qe({present:i.open},t));return g.jsx(Ku,{value:i,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:s,children:n})})})},O1=e=>{const[t,{value:n,children:r}]=Ec(e),[o]=yu(t),i=Ts(qe({present:n.open},t));return g.jsx(Ku,{value:n,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:i,children:r})})})},sd=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getTitleProps(),e);return g.jsx(kt.h2,{...r,ref:t})});sd.displayName="DialogTitle";const ad=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe({...n.getTriggerProps(),"aria-controls":r.unmounted?void 0:n.getTriggerProps()["aria-controls"]},e);return g.jsx(kt.button,{...o,ref:t})});ad.displayName="DialogTrigger";var ld=j("editable").parts("root","area","label","preview","input","editTrigger","submitTrigger","cancelTrigger","control");ld.build(),M()(["activationMode","autoResize","dir","disabled","finalFocusEl","form","getRootNode","id","ids","invalid","maxLength","name","onEditChange","onFocusOutside","onInteractOutside","onPointerDownOutside","onValueChange","onValueCommit","onValueRevert","placeholder","readOnly","required","selectOnFocus","edit","defaultEdit","submitMode","translations","defaultValue","value"]);const cd=P.forwardRef((e,t)=>{const n=Iu(),r=qe(n?.getInputProps(),e);return g.jsx(kt.input,{...r,ref:t})});cd.displayName="FieldInput";const ud=j("field").parts("root","errorText","helperText","input","label","select","textarea","requiredIndicator");ud.build();var P1=e=>{if(!e)return;const t=xo(e),n=be(e),r=Fe(e),o=()=>{requestAnimationFrame(()=>{e.style.height="auto";let c;t.boxSizing==="content-box"?c=e.scrollHeight-(parseFloat(t.paddingTop)+parseFloat(t.paddingBottom)):c=e.scrollHeight+parseFloat(t.borderTopWidth)+parseFloat(t.borderBottomWidth),t.maxHeight!=="none"&&c>parseFloat(t.maxHeight)?(t.overflowY==="hidden"&&(e.style.overflowY="scroll"),c=parseFloat(t.maxHeight)):t.overflowY!=="hidden"&&(e.style.overflowY="hidden"),e.style.height=`${c}px`})};e.addEventListener("input",o),e.form?.addEventListener("reset",o);const i=Object.getPrototypeOf(e),s=Object.getOwnPropertyDescriptor(i,"value");Object.defineProperty(e,"value",{...s,set(){s?.set?.apply(this,arguments),o()}});const a=new n.ResizeObserver(()=>{requestAnimationFrame(()=>o())});a.observe(e);const l=new n.MutationObserver(()=>o());return l.observe(e,{attributes:!0,attributeFilter:["rows","placeholder"]}),r.fonts?.addEventListener("loadingdone",o),()=>{e.removeEventListener("input",o),e.form?.removeEventListener("reset",o),r.fonts?.removeEventListener("loadingdone",o),a.disconnect(),l.disconnect()}};const dd=P.forwardRef((e,t)=>{const{autoresize:n,...r}=e,o=P.useRef(null),i=Iu(),s=qe(i?.getTextareaProps(),{style:{resize:n?"none":void 0}},r);return P.useEffect(()=>{if(n)return P1(o.current)},[n]),g.jsx(kt.textarea,{...s,ref:No(t,o)})});dd.displayName="FieldTextarea";const hd=j("fieldset").parts("root","errorText","helperText","legend");hd.build();var fd=j("file-upload").parts("root","dropzone","item","itemDeleteTrigger","itemGroup","itemName","itemPreview","itemPreviewImage","itemSizeText","label","trigger","clearTrigger");fd.build(),M()(["accept","acceptedFiles","allowDrop","capture","defaultAcceptedFiles","dir","directory","disabled","getRootNode","id","ids","invalid","locale","maxFiles","maxFileSize","minFileSize","name","onFileAccept","onFileChange","onFileReject","preventDocumentDrop","required","transformFiles","translations","validate"]),M()(["file","type"]);var gd=j("hoverCard").parts("arrow","arrowTip","trigger","positioner","content");gd.build();var R1=e=>e.ids?.trigger??`hover-card:${e.id}:trigger`,I1=e=>e.ids?.content??`hover-card:${e.id}:content`,T1=e=>e.ids?.positioner??`hover-card:${e.id}:popper`,oa=e=>e.getById(R1(e)),N1=e=>e.getById(I1(e)),pd=e=>e.getById(T1(e)),{not:ti,and:md}=jt();md("isOpenControlled",ti("isPointer")),ti("isPointer"),md("isOpenControlled",ti("isPointer")),ti("isPointer"),M()(["closeDelay","dir","getRootNode","id","ids","disabled","onOpenChange","defaultOpen","open","openDelay","positioning","onInteractOutside","onPointerDownOutside","onFocusOutside"]);var vd=j("tree-view").parts("branch","branchContent","branchControl","branchIndentGuide","branchIndicator","branchText","branchTrigger","item","itemIndicator","itemText","label","nodeCheckbox","root","tree");vd.build();var bd=e=>new zu(e);bd.empty=()=>new zu({rootNode:{children:[]}});var A1=(e,t)=>e.ids?.node?.(t)??`tree:${e.id}:node:${t}`,sn=(e,t)=>{t!=null&&e.getById(A1(e,t))?.focus()};function _1(e,t,n){const r=e.getDescendantValues(t),o=r.every(i=>n.includes(i));return Dn(o?Yt(n,...r):Kt(n,...r))}function ni(e,t){const{context:n,prop:r,refs:o}=e;if(!r("loadChildren")){n.set("expandedValue",p=>Dn(Kt(p,...t)));return}const i=n.get("loadingStatus"),[s,a]=ql(t,p=>i[p]==="loaded");if(s.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...s))),a.length===0)return;const l=r("collection"),[c,u]=ql(a,p=>{const v=l.findNode(p);return l.getNodeChildren(v).length>0});if(c.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...c))),u.length===0)return;n.set("loadingStatus",p=>({...p,...u.reduce((v,b)=>({...v,[b]:"loading"}),{})}));const h=u.map(p=>{const v=l.getIndexPath(p),b=l.getValuePath(v),y=l.findNode(p);return{id:p,indexPath:v,valuePath:b,node:y}}),d=o.get("pendingAborts"),m=r("loadChildren");ic(m,()=>"[zag-js/tree-view] `loadChildren` is required for async expansion");const f=h.map(({id:p,indexPath:v,valuePath:b,node:y})=>{const x=d.get(p);x&&(x.abort(),d.delete(p));const w=new AbortController;return d.set(p,w),m({valuePath:b,indexPath:v,node:y,signal:w.signal})});Promise.allSettled(f).then(p=>{const v=[],b=[],y=n.get("loadingStatus");let x=r("collection");p.forEach((w,k)=>{const{id:O,indexPath:R,node:I,valuePath:N}=h[k];w.status==="fulfilled"?(y[O]="loaded",v.push(O),x=x.replace(R,{...I,children:w.value})):(d.delete(O),Reflect.deleteProperty(y,O),b.push({node:I,error:w.reason,indexPath:R,valuePath:N}))}),n.set("loadingStatus",y),v.length&&(n.set("expandedValue",w=>Dn(Kt(w,...v))),r("onLoadChildrenComplete")?.({collection:x})),b.length&&r("onLoadChildrenError")?.({nodes:b})})}function Ut(e){const{prop:t,context:n}=e;return function({indexPath:o}){return t("collection").getValuePath(o).slice(0,-1).some(s=>!n.get("expandedValue").includes(s))}}var{and:ft}=jt();ft("isMultipleSelection","moveFocus"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isBranchFocused","isBranchExpanded"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),M()(["ids","collection","dir","expandedValue","expandOnClick","defaultFocusedValue","focusedValue","getRootNode","id","onExpandedChange","onFocusChange","onSelectionChange","checkedValue","selectedValue","selectionMode","typeahead","defaultExpandedValue","defaultSelectedValue","defaultCheckedValue","onCheckedChange","onLoadChildrenComplete","onLoadChildrenError","loadChildren"]),M()(["node","indexPath"]);var yd=j("listbox").parts("label","input","item","itemText","itemIndicator","itemGroup","itemGroupLabel","content","root","valueText");yd.build(),M()(["collection","defaultHighlightedValue","defaultValue","dir","disabled","deselectable","disallowSelectAll","getRootNode","highlightedValue","id","ids","loopFocus","onHighlightChange","onSelect","onValueChange","orientation","scrollToIndexFn","selectionMode","selectOnHighlight","typeahead","value"]),M()(["item","highlightOnHover"]),M()(["id"]),M()(["htmlFor"]);const V1=yd.extendWith("empty");var xd=j("menu").parts("arrow","arrowTip","content","contextTrigger","indicator","item","itemGroup","itemGroupLabel","itemIndicator","itemText","positioner","separator","trigger","triggerItem");xd.build();var kd=e=>e.ids?.trigger??`menu:${e.id}:trigger`,F1=e=>e.ids?.contextTrigger??`menu:${e.id}:ctx-trigger`,Cd=e=>e.ids?.content??`menu:${e.id}:content`,L1=e=>e.ids?.positioner??`menu:${e.id}:popper`,ia=(e,t)=>`${e.id}/${t}`,an=e=>e?.dataset.value??null,Gt=e=>e.getById(Cd(e)),wd=e=>e.getById(L1(e)),ri=e=>e.getById(kd(e)),D1=(e,t)=>t?e.getById(ia(e,t)):null,sa=e=>e.getById(F1(e)),Dr=e=>{const n=`[role^="menuitem"][data-ownedby=${CSS.escape(Cd(e))}]:not([data-disabled])`;return Io(Gt(e),n)},z1=e=>kr(Dr(e)),M1=e=>hs(Dr(e)),aa=(e,t)=>t?e.id===t||e.dataset.value===t:!1,j1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return Um(n,r,{loop:t.loop??t.loopFocus})},$1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return qm(n,r,{loop:t.loop??t.loopFocus})},B1=(e,t)=>{const n=Dr(e),r=n.find(o=>aa(o,t.value));return Or(n,{state:t.typeaheadState,key:t.key,activeId:r?.id??null})},W1=e=>!!e?.getAttribute("role")?.startsWith("menuitem")&&!!e?.hasAttribute("aria-controls"),H1="menu:select";function U1(e,t){if(!e)return;const n=be(e),r=new n.CustomEvent(H1,{detail:{value:t}});e.dispatchEvent(r)}var{not:Xe,and:Jn,or:G1}=jt();Xe("isSubmenu"),G1("isOpenAutoFocusEvent","isArrowDownEvent"),Jn(Xe("isTriggerItem"),"isOpenControlled"),Xe("isTriggerItem"),Jn("isSubmenu","isOpenControlled"),Xe("isPointerSuspended"),Jn(Xe("isPointerSuspended"),Xe("isTriggerItem")),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect","isOpenControlled"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"));function Sd(e){let t=e.parent;for(;t&&t.context.get("isSubmenu");)t=t.refs.get("parent");t?.send({type:"CLOSE"})}function q1(e,t){return e?gx(e,t):!1}function K1(e,t,n){const r=Object.keys(e).length>0;if(!t)return null;if(!r)return ia(n,t);for(const o in e){const i=e[o],s=kd(i.scope);if(s===t)return s}return ia(n,t)}M()(["anchorPoint","aria-label","closeOnSelect","composite","defaultHighlightedValue","defaultOpen","dir","getRootNode","highlightedValue","id","ids","loopFocus","navigate","onEscapeKeyDown","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","onSelect","open","positioning","typeahead"]),M()(["closeOnSelect","disabled","value","valueText"]),M()(["htmlFor"]),M()(["id"]),M()(["checked","closeOnSelect","disabled","onCheckedChange","type","value","valueText"]);let la=new Map,ca=!1;try{ca=new Intl.NumberFormat("de-DE",{signDisplay:"exceptZero"}).resolvedOptions().signDisplay==="exceptZero"}catch{}let oi=!1;try{oi=new Intl.NumberFormat("de-DE",{style:"unit",unit:"degree"}).resolvedOptions().style==="unit"}catch{}const Ed={degree:{narrow:{default:"°","ja-JP":" 度","zh-TW":"度","sl-SI":" °"}}};class Y1{format(t){let n="";if(!ca&&this.options.signDisplay!=null?n=Q1(this.numberFormatter,this.options.signDisplay,t):n=this.numberFormatter.format(t),this.options.style==="unit"&&!oi){var r;let{unit:o,unitDisplay:i="short",locale:s}=this.resolvedOptions();if(!o)return n;let a=(r=Ed[o])===null||r===void 0?void 0:r[i];n+=a[s]||a.default}return n}formatToParts(t){return this.numberFormatter.formatToParts(t)}formatRange(t,n){if(typeof this.numberFormatter.formatRange=="function")return this.numberFormatter.formatRange(t,n);if(n= start date");return`${this.format(t)} – ${this.format(n)}`}formatRangeToParts(t,n){if(typeof this.numberFormatter.formatRangeToParts=="function")return this.numberFormatter.formatRangeToParts(t,n);if(n= start date");let r=this.numberFormatter.formatToParts(t),o=this.numberFormatter.formatToParts(n);return[...r.map(i=>({...i,source:"startRange"})),{type:"literal",value:" – ",source:"shared"},...o.map(i=>({...i,source:"endRange"}))]}resolvedOptions(){let t=this.numberFormatter.resolvedOptions();return!ca&&this.options.signDisplay!=null&&(t={...t,signDisplay:this.options.signDisplay}),!oi&&this.options.style==="unit"&&(t={...t,style:"unit",unit:this.options.unit,unitDisplay:this.options.unitDisplay}),t}constructor(t,n={}){this.numberFormatter=X1(t,n),this.options=n}}function X1(e,t={}){let{numberingSystem:n}=t;if(n&&e.includes("-nu-")&&(e.includes("-u-")||(e+="-u-"),e+=`-nu-${n}`),t.style==="unit"&&!oi){var r;let{unit:s,unitDisplay:a="short"}=t;if(!s)throw new Error('unit option must be provided with style: "unit"');if(!(!((r=Ed[s])===null||r===void 0)&&r[a]))throw new Error(`Unsupported unit ${s} with unitDisplay = ${a}`);t={...t,style:"decimal"}}let o=e+(t?Object.entries(t).sort((s,a)=>s[0]0||Object.is(n,0):t==="exceptZero"&&(Object.is(n,-0)||Object.is(n,0)?n=Math.abs(n):r=n>0),r){let o=e.format(-n),i=e.format(n),s=o.replace(i,"").replace(/\u200e|\u061C/,"");return[...s].length!==1&&console.warn("@react-aria/i18n polyfill for NumberFormat signDisplay: Unsupported case"),o.replace(i,"!!!").replace(s,"+").replace("!!!",i)}else return e.format(n)}}const J1=new RegExp("^.*\\(.*\\).*$"),Z1=["latn","arab","hanidec","deva","beng","fullwide"];class Od{parse(t){return ua(this.locale,this.options,t).parse(t)}isValidPartialNumber(t,n,r){return ua(this.locale,this.options,t).isValidPartialNumber(t,n,r)}getNumberingSystem(t){return ua(this.locale,this.options,t).options.numberingSystem}constructor(t,n={}){this.locale=t,this.options=n}}const Pd=new Map;function ua(e,t,n){let r=Rd(e,t);if(!e.includes("-nu-")&&!r.isValidPartialNumber(n)){for(let o of Z1)if(o!==r.options.numberingSystem){let i=Rd(e+(e.includes("-u-")?"-nu-":"-u-nu-")+o,t);if(i.isValidPartialNumber(n))return i}}return r}function Rd(e,t){let n=e+(t?Object.entries(t).sort((o,i)=>o[0]-1&&(n=`-${n}`)}let r=n?+n:NaN;if(isNaN(r))return NaN;if(this.options.style==="percent"){var o,i;let s={...this.options,style:"decimal",minimumFractionDigits:Math.min(((o=this.options.minimumFractionDigits)!==null&&o!==void 0?o:0)+2,20),maximumFractionDigits:Math.min(((i=this.options.maximumFractionDigits)!==null&&i!==void 0?i:0)+2,20)};return new Od(this.locale,s).parse(new Y1(this.locale,s).format(r))}return this.options.currencySign==="accounting"&&J1.test(t)&&(r=-1*r),r}sanitize(t){return t=t.replace(this.symbols.literals,""),this.symbols.minusSign&&(t=t.replace("-",this.symbols.minusSign)),this.options.numberingSystem==="arab"&&(this.symbols.decimal&&(t=t.replace(",",this.symbols.decimal),t=t.replace("،",this.symbols.decimal)),this.symbols.group&&(t=Zn(t,".",this.symbols.group))),this.symbols.group==="’"&&t.includes("'")&&(t=Zn(t,"'",this.symbols.group)),this.options.locale==="fr-FR"&&this.symbols.group&&(t=Zn(t," ",this.symbols.group),t=Zn(t,/\u00A0/g,this.symbols.group)),t}isValidPartialNumber(t,n=-1/0,r=1/0){return t=this.sanitize(t),this.symbols.minusSign&&t.startsWith(this.symbols.minusSign)&&n<0?t=t.slice(this.symbols.minusSign.length):this.symbols.plusSign&&t.startsWith(this.symbols.plusSign)&&r>0&&(t=t.slice(this.symbols.plusSign.length)),this.symbols.group&&t.startsWith(this.symbols.group)||this.symbols.decimal&&t.indexOf(this.symbols.decimal)>-1&&this.options.maximumFractionDigits===0?!1:(this.symbols.group&&(t=Zn(t,this.symbols.group,"")),t=t.replace(this.symbols.numeral,""),this.symbols.decimal&&(t=t.replace(this.symbols.decimal,"")),t.length===0)}constructor(t,n={}){this.locale=t,n.roundingIncrement!==1&&n.roundingIncrement!=null&&(n.maximumFractionDigits==null&&n.minimumFractionDigits==null?(n.maximumFractionDigits=0,n.minimumFractionDigits=0):n.maximumFractionDigits==null?n.maximumFractionDigits=n.minimumFractionDigits:n.minimumFractionDigits==null&&(n.minimumFractionDigits=n.maximumFractionDigits)),this.formatter=new Intl.NumberFormat(t,n),this.options=this.formatter.resolvedOptions(),this.symbols=nk(t,this.formatter,this.options,n);var r,o;this.options.style==="percent"&&(((r=this.options.minimumFractionDigits)!==null&&r!==void 0?r:0)>18||((o=this.options.maximumFractionDigits)!==null&&o!==void 0?o:0)>18)&&console.warn("NumberParser cannot handle percentages with greater than 18 decimal places, please reduce the number in your options.")}}const Id=new Set(["decimal","fraction","integer","minusSign","plusSign","group"]),tk=[0,4,2,1,11,20,3,7,100,21,.1,1.1];function nk(e,t,n,r){var o,i,s,a;let l=new Intl.NumberFormat(e,{...n,minimumSignificantDigits:1,maximumSignificantDigits:21,roundingIncrement:1,roundingPriority:"auto",roundingMode:"halfExpand"}),c=l.formatToParts(-10000.111),u=l.formatToParts(10000.111),h=tk.map(A=>l.formatToParts(A));var d;let m=(d=(o=c.find(A=>A.type==="minusSign"))===null||o===void 0?void 0:o.value)!==null&&d!==void 0?d:"-",f=(i=u.find(A=>A.type==="plusSign"))===null||i===void 0?void 0:i.value;!f&&(r?.signDisplay==="exceptZero"||r?.signDisplay==="always")&&(f="+");let v=(s=new Intl.NumberFormat(e,{...n,minimumFractionDigits:2,maximumFractionDigits:2}).formatToParts(.001).find(A=>A.type==="decimal"))===null||s===void 0?void 0:s.value,b=(a=c.find(A=>A.type==="group"))===null||a===void 0?void 0:a.value,y=c.filter(A=>!Id.has(A.type)).map(A=>Td(A.value)),x=h.flatMap(A=>A.filter(T=>!Id.has(T.type)).map(T=>Td(T.value))),w=[...new Set([...y,...x])].sort((A,T)=>T.length-A.length),k=w.length===0?new RegExp("[\\p{White_Space}]","gu"):new RegExp(`${w.join("|")}|[\\p{White_Space}]`,"gu"),O=[...new Intl.NumberFormat(n.locale,{useGrouping:!1}).format(9876543210)].reverse(),R=new Map(O.map((A,T)=>[A,T])),I=new RegExp(`[${O.join("")}]`,"g");return{minusSign:m,plusSign:f,decimal:v,group:b,literals:k,numeral:I,index:A=>String(R.get(A))}}function Zn(e,t,n){return e.replaceAll?e.replaceAll(t,n):e.split(t).join(n)}function Td(e){return e.replace(/[.*+?^${}()|[\]\\]/g,"\\$&")}var Nd=j("numberInput").parts("root","label","input","control","valueText","incrementTrigger","decrementTrigger","scrubber");Nd.build();var rk=e=>e.ids?.input??`number-input:${e.id}:input`,ok=e=>e.ids?.incrementTrigger??`number-input:${e.id}:inc`,ik=e=>e.ids?.decrementTrigger??`number-input:${e.id}:dec`,Ad=e=>`number-input:${e.id}:cursor`,ii=e=>e.getById(rk(e)),sk=e=>e.getById(ok(e)),ak=e=>e.getById(ik(e)),_d=e=>e.getDoc().getElementById(Ad(e)),lk=(e,t)=>{let n=null;return t==="increment"&&(n=sk(e)),t==="decrement"&&(n=ak(e)),n},ck=(e,t)=>{if(!fc())return hk(e,t),()=>{_d(e)?.remove()}},uk=e=>{const t=e.getDoc(),n=t.documentElement,r=t.body;return r.style.pointerEvents="none",n.style.userSelect="none",n.style.cursor="ew-resize",()=>{r.style.pointerEvents="",n.style.userSelect="",n.style.cursor="",n.style.length||n.removeAttribute("style"),r.style.length||r.removeAttribute("style")}},dk=(e,t)=>{const{point:n,isRtl:r,event:o}=t,i=e.getWin(),s=ps(o.movementX,i.devicePixelRatio),a=ps(o.movementY,i.devicePixelRatio);let l=s>0?"increment":s<0?"decrement":null;r&&l==="increment"&&(l="decrement"),r&&l==="decrement"&&(l="increment");const c={x:n.x+s,y:n.y+a},u=i.innerWidth,h=ps(7.5,i.devicePixelRatio);return c.x=u0(c.x+h,u)-h,{hint:l,point:c}},hk=(e,t)=>{const n=e.getDoc(),r=n.createElement("div");r.className="scrubber--cursor",r.id=Ad(e),Object.assign(r.style,{width:"15px",height:"15px",position:"fixed",pointerEvents:"none",left:"0px",top:"0px",zIndex:w0,transform:t?`translate3d(${t.x}px, ${t.y}px, 0px)`:void 0,willChange:"transform"}),r.innerHTML=` + */var Il;function sm(){if(Il)return Q;Il=1;var e=typeof Symbol=="function"&&Symbol.for,t=e?Symbol.for("react.element"):60103,n=e?Symbol.for("react.portal"):60106,r=e?Symbol.for("react.fragment"):60107,o=e?Symbol.for("react.strict_mode"):60108,i=e?Symbol.for("react.profiler"):60114,s=e?Symbol.for("react.provider"):60109,a=e?Symbol.for("react.context"):60110,l=e?Symbol.for("react.async_mode"):60111,c=e?Symbol.for("react.concurrent_mode"):60111,u=e?Symbol.for("react.forward_ref"):60112,h=e?Symbol.for("react.suspense"):60113,d=e?Symbol.for("react.suspense_list"):60120,m=e?Symbol.for("react.memo"):60115,f=e?Symbol.for("react.lazy"):60116,p=e?Symbol.for("react.block"):60121,v=e?Symbol.for("react.fundamental"):60117,b=e?Symbol.for("react.responder"):60118,y=e?Symbol.for("react.scope"):60119;function x(k){if(typeof k=="object"&&k!==null){var O=k.$$typeof;switch(O){case t:switch(k=k.type,k){case l:case c:case r:case i:case o:case h:return k;default:switch(k=k&&k.$$typeof,k){case a:case u:case f:case m:case s:return k;default:return O}}case n:return O}}}function w(k){return x(k)===c}return Q.AsyncMode=l,Q.ConcurrentMode=c,Q.ContextConsumer=a,Q.ContextProvider=s,Q.Element=t,Q.ForwardRef=u,Q.Fragment=r,Q.Lazy=f,Q.Memo=m,Q.Portal=n,Q.Profiler=i,Q.StrictMode=o,Q.Suspense=h,Q.isAsyncMode=function(k){return w(k)||x(k)===l},Q.isConcurrentMode=w,Q.isContextConsumer=function(k){return x(k)===a},Q.isContextProvider=function(k){return x(k)===s},Q.isElement=function(k){return typeof k=="object"&&k!==null&&k.$$typeof===t},Q.isForwardRef=function(k){return x(k)===u},Q.isFragment=function(k){return x(k)===r},Q.isLazy=function(k){return x(k)===f},Q.isMemo=function(k){return x(k)===m},Q.isPortal=function(k){return x(k)===n},Q.isProfiler=function(k){return x(k)===i},Q.isStrictMode=function(k){return x(k)===o},Q.isSuspense=function(k){return x(k)===h},Q.isValidElementType=function(k){return typeof k=="string"||typeof k=="function"||k===r||k===c||k===i||k===o||k===h||k===d||typeof k=="object"&&k!==null&&(k.$$typeof===f||k.$$typeof===m||k.$$typeof===s||k.$$typeof===a||k.$$typeof===u||k.$$typeof===v||k.$$typeof===b||k.$$typeof===y||k.$$typeof===p)},Q.typeOf=x,Q}var Tl;function am(){return Tl||(Tl=1,Zi.exports=sm()),Zi.exports}var es,Nl;function lm(){if(Nl)return es;Nl=1;var e=am(),t={childContextTypes:!0,contextType:!0,contextTypes:!0,defaultProps:!0,displayName:!0,getDefaultProps:!0,getDerivedStateFromError:!0,getDerivedStateFromProps:!0,mixins:!0,propTypes:!0,type:!0},n={name:!0,length:!0,prototype:!0,caller:!0,callee:!0,arguments:!0,arity:!0},r={$$typeof:!0,render:!0,defaultProps:!0,displayName:!0,propTypes:!0},o={$$typeof:!0,compare:!0,defaultProps:!0,displayName:!0,propTypes:!0,type:!0},i={};i[e.ForwardRef]=r,i[e.Memo]=o;function s(f){return e.isMemo(f)?o:i[f.$$typeof]||t}var a=Object.defineProperty,l=Object.getOwnPropertyNames,c=Object.getOwnPropertySymbols,u=Object.getOwnPropertyDescriptor,h=Object.getPrototypeOf,d=Object.prototype;function m(f,p,v){if(typeof p!="string"){if(d){var b=h(p);b&&b!==d&&m(f,b,v)}var y=l(p);c&&(y=y.concat(c(p)));for(var x=s(f),w=s(p),k=0;k=4;++r,o-=4)n=e.charCodeAt(r)&255|(e.charCodeAt(++r)&255)<<8|(e.charCodeAt(++r)&255)<<16|(e.charCodeAt(++r)&255)<<24,n=(n&65535)*1540483477+((n>>>16)*59797<<16),n^=n>>>24,t=(n&65535)*1540483477+((n>>>16)*59797<<16)^(t&65535)*1540483477+((t>>>16)*59797<<16);switch(o){case 3:t^=(e.charCodeAt(r+2)&255)<<16;case 2:t^=(e.charCodeAt(r+1)&255)<<8;case 1:t^=e.charCodeAt(r)&255,t=(t&65535)*1540483477+((t>>>16)*59797<<16)}return t^=t>>>13,t=(t&65535)*1540483477+((t>>>16)*59797<<16),((t^t>>>15)>>>0).toString(36)}var dm={animationIterationCount:1,aspectRatio:1,borderImageOutset:1,borderImageSlice:1,borderImageWidth:1,boxFlex:1,boxFlexGroup:1,boxOrdinalGroup:1,columnCount:1,columns:1,flex:1,flexGrow:1,flexPositive:1,flexShrink:1,flexNegative:1,flexOrder:1,gridRow:1,gridRowEnd:1,gridRowSpan:1,gridRowStart:1,gridColumn:1,gridColumnEnd:1,gridColumnSpan:1,gridColumnStart:1,msGridRow:1,msGridRowSpan:1,msGridColumn:1,msGridColumnSpan:1,fontWeight:1,lineHeight:1,opacity:1,order:1,orphans:1,scale:1,tabSize:1,widows:1,zIndex:1,zoom:1,WebkitLineClamp:1,fillOpacity:1,floodOpacity:1,stopOpacity:1,strokeDasharray:1,strokeDashoffset:1,strokeMiterlimit:1,strokeOpacity:1,strokeWidth:1},hm=/[A-Z]|^ms/g,fm=/_EMO_([^_]+?)_([^]*?)_EMO_/g,_l=function(t){return t.charCodeAt(1)===45},Vl=function(t){return t!=null&&typeof t!="boolean"},rs=bl(function(e){return _l(e)?e:e.replace(hm,"-$&").toLowerCase()}),Ll=function(t,n){switch(t){case"animation":case"animationName":if(typeof n=="string")return n.replace(fm,function(r,o,i){return at={name:o,styles:i,next:at},o})}return dm[t]!==1&&!_l(t)&&typeof n=="number"&&n!==0?n+"px":n};function vr(e,t,n){if(n==null)return"";var r=n;if(r.__emotion_styles!==void 0)return r;switch(typeof n){case"boolean":return"";case"object":{var o=n;if(o.anim===1)return at={name:o.name,styles:o.styles,next:at},o.name;var i=n;if(i.styles!==void 0){var s=i.next;if(s!==void 0)for(;s!==void 0;)at={name:s.name,styles:s.styles,next:at},s=s.next;var a=i.styles+";";return a}return gm(e,t,n)}case"function":{if(e!==void 0){var l=at,c=n(e);return at=l,vr(e,t,c)}break}}var u=n;if(t==null)return u;var h=t[u];return h!==void 0?h:u}function gm(e,t,n){var r="";if(Array.isArray(n))for(var o=0;or?.(...n))}}const km=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),Cm=/^on[A-Z]/;function br(...e){let t={};for(let n of e){for(let r in t){if(Cm.test(r)&&typeof t[r]=="function"&&typeof n[r]=="function"){t[r]=xm(t[r],n[r]);continue}if(r==="className"||r==="class"){t[r]=km(t[r],n[r]);continue}if(r==="style"){t[r]=Object.assign({},t[r]??{},n[r]??{});continue}t[r]=n[r]!==void 0?n[r]:t[r]}for(let r in n)t[r]===void 0&&(t[r]=n[r])}return t}function wm(e,t){if(e!=null){if(typeof e=="function"){e(t);return}try{e.current=t}catch{throw new Error(`Cannot assign value '${t}' to ref '${e}'`)}}}function Sm(...e){return t=>{e.forEach(n=>{wm(n,t)})}}function yr(e){const t=Object.assign({},e);for(let n in t)t[n]===void 0&&delete t[n];return t}const Ge=(...e)=>e.filter(Boolean).map(t=>t.trim()).join(" ");function Em(e){return e.default||e}const _e=e=>e!=null&&typeof e=="object"&&!Array.isArray(e),lt=e=>typeof e=="string",cs=e=>typeof e=="function";function Om(e){const t=C.version;return!lt(t)||t.startsWith("18.")?e?.ref:e?.props?.ref}const Wl=(...e)=>{const t=e.reduce((n,r)=>(r?.forEach(o=>n.add(o)),n),new Set([]));return Array.from(t)};function Pm(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Ln(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??Pm(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[Rm,ho]=Ln({name:"ChakraContext",strict:!0,providerName:""});function Im(e){const{value:t,children:n}=e;return g.jsxs(Rm,{value:t,children:[!t._config.disableLayers&&g.jsx(Bl,{styles:t.layers.atRule}),g.jsx(Bl,{styles:t._global}),n]})}const Tm=(e,t)=>{const n={},r={},o=Object.keys(e);for(const i of o)t(i)?r[i]=e[i]:n[i]=e[i];return[r,n]},Fn=(e,t)=>{const n=cs(t)?t:r=>t.includes(r);return Tm(e,n)},Nm=new Set(["htmlWidth","htmlHeight","htmlSize","htmlTranslate"]);function Am(e){return typeof e=="string"&&Nm.has(e)}function _m(e,t,n){const{css:r,isValidProperty:o}=ho(),{children:i,...s}=e,a=P.useMemo(()=>{const[d,m]=Fn(s,y=>n(y,t.variantKeys)),[f,p]=Fn(m,t.variantKeys),[v,b]=Fn(p,o);return{forwardedProps:d,variantProps:f,styleProps:v,elementProps:b}},[t.variantKeys,n,s,o]),{css:l,...c}=a.styleProps,u=P.useMemo(()=>{const d={...a.variantProps};return t.variantKeys.includes("colorPalette")||(d.colorPalette=s.colorPalette),t.variantKeys.includes("orientation")||(d.orientation=s.orientation),t(d)},[t,a.variantProps,s.colorPalette,s.orientation]);return{styles:P.useMemo(()=>r(u,...Vm(l),c),[r,u,l,c]),props:{...a.forwardedProps,...a.elementProps,children:i}}}const Vm=e=>(Array.isArray(e)?e:[e]).filter(Boolean).flat(),Lm=Em(Tp),Fm=e=>e!=="theme",Dm=(e,t,n)=>{let r;if(t){const o=t.shouldForwardProp;r=e.__emotion_forwardProp&&o?i=>e.__emotion_forwardProp(i)&&o(i):o}return typeof r!="function"&&n&&(r=e.__emotion_forwardProp),r};let zm=typeof document<"u";const Hl=({cache:e,serialized:t,isStringTag:n})=>{ts(e,t,n);const r=zl(()=>ns(e,t,n));if(!zm&&r!==void 0){let o=t.name,i=t.next;for(;i!==void 0;)o=Ge(o,i.name),i=i.next;return g.jsx("style",{"data-emotion":Ge(e.key,o),dangerouslySetInnerHTML:{__html:r},nonce:e.sheet.nonce})}return null},Ul={path:["d"],text:["x","y"],circle:["cx","cy","r"],rect:["width","height","x","y","rx","ry"],ellipse:["cx","cy","rx","ry"],g:["transform"],stop:["offset","stopOpacity"]},Mm=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),us=((e,t={},n={})=>{if(Mm(Ul,e)){n.forwardProps||(n.forwardProps=[]);const c=Ul[e];n.forwardProps=Wl([...n.forwardProps,...c])}const r=e.__emotion_real===e,o=r&&e.__emotion_base||e;let i,s;n!==void 0&&(i=n.label,s=n.target);let a=[];const l=is((c,u,h)=>{const{cva:d,isValidProperty:m}=ho(),f=t.__cva__?t:d(t),p=jm(e.__emotion_cva,f),v=z=>(W,J)=>z.includes(W)?!0:!J?.includes(W)&&!m(W);!n.shouldForwardProp&&n.forwardProps&&(n.shouldForwardProp=v(n.forwardProps));const b=(z,W)=>{const J=typeof e=="string"&&e.charCodeAt(0)>96?Lm:Fm,ee=!W?.includes(z)&&!m(z);return J(z)&&ee},y=Dm(e,n,r)||b,x=C.useMemo(()=>Object.assign({},n.defaultProps,yr(c)),[c]),{props:w,styles:k}=_m(x,p,y);let O="",R=[k],I=w;if(w.theme==null){I={};for(let z in w)I[z]=w[z];I.theme=C.useContext(ss)}typeof w.className=="string"?O=Al(u.registered,R,w.className):w.className!=null&&(O=Ge(O,w.className));const N=os(a.concat(R),u.registered,I);N.styles&&(O=Ge(O,`${u.key}-${N.name}`)),s!==void 0&&(O=Ge(O,s));const A=!y("as");let T=A&&w.as||o,S={};for(let z in w)if(!(A&&z==="as")){if(Am(z)){const W=z.replace("html","").toLowerCase();S[W]=w[z];continue}y(z)&&(S[z]=w[z])}let _=O.trim();_?S.className=_:Reflect.deleteProperty(S,"className"),S.ref=h;const D=n.forwardAsChild||n.forwardProps?.includes("asChild");if(w.asChild&&!D){const z=C.isValidElement(w.children)?C.Children.only(w.children):C.Children.toArray(w.children).find(C.isValidElement);if(!z)throw new Error("[chakra-ui > factory] No valid child found");T=z.type,S.children=null,Reflect.deleteProperty(S,"asChild"),S=br(S,z.props),S.ref=Sm(h,Om(z))}return S.as&&D?(S.as=void 0,g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{asChild:!0,...S,children:g.jsx(w.as,{children:S.children})})]})):g.jsxs(C.Fragment,{children:[g.jsx(Hl,{cache:u,serialized:N,isStringTag:typeof T=="string"}),g.jsx(T,{...S})]})});return l.displayName=i!==void 0?i:`chakra(${typeof o=="string"?o:o.displayName||o.name||"Component"})`,l.__emotion_real=l,l.__emotion_base=o,l.__emotion_forwardProp=n.shouldForwardProp,l.__emotion_cva=t,Object.defineProperty(l,"toString",{value(){return`.${s}`}}),l}).bind(),ds=new Map,ve=new Proxy(us,{apply(e,t,n){return us(...n)},get(e,t){return ds.has(t)||ds.set(t,us(t)),ds.get(t)}}),jm=(e,t)=>e&&!t?e:!e&&t?t:e.merge(t),xr=ve("div");xr.displayName="Box";const $m=Object.freeze({}),Bm=Object.freeze({});function Wm(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getRecipe(t):{});return r.cva(structuredClone(o))},[t,n,r])}const Hm=e=>e.charAt(0).toUpperCase()+e.slice(1);function Ft(e){const{key:t,recipe:n}=e,r=Hm(t||n.className||"Component"),[o,i]=Ln({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`});function s(c){const{unstyled:u,...h}=c,d=Wm({key:t,recipe:h.recipe||n}),[m,f]=P.useMemo(()=>d.splitVariantProps(h),[d,h]);return{styles:u?$m:d(m),className:d.className,props:f}}const a=(c,u)=>{const h=ve(c,{},u),d=P.forwardRef((m,f)=>{const p=i(),v=P.useMemo(()=>br(p,m),[m,p]),{styles:b,className:y,props:x}=s(v);return g.jsx(h,{...x,ref:f,css:[b,v.css],className:Ge(y,v.className)})});return d.displayName=c.displayName||c.name,d};function l(){return o}return{withContext:a,PropsProvider:o,withPropsProvider:l,usePropsContext:i,useRecipeResult:s}}function fo(e){return e==null?[]:Array.isArray(e)?e:[e]}var kr=e=>e[0],hs=e=>e[e.length-1],Um=(e,t)=>e.indexOf(t)!==-1,Kt=(e,...t)=>e.concat(t),Yt=(e,...t)=>e.filter(n=>!t.includes(n)),Dn=e=>Array.from(new Set(e)),fs=(e,t)=>{const n=new Set(t);return e.filter(r=>!n.has(r))},zn=(e,t)=>Um(e,t)?Yt(e,t):Kt(e,t);function Gl(e,t,n={}){const{step:r=1,loop:o=!0}=n,i=t+r,s=e.length,a=s-1;return t===-1?r>0?0:a:i<0?o?a:0:i>=s?o?0:t>s?s:t:i}function Gm(e,t,n={}){return e[Gl(e,t,n)]}function qm(e,t,n={}){const{step:r=1,loop:o=!0}=n;return Gl(e,t,{step:-r,loop:o})}function Km(e,t,n={}){return e[qm(e,t,n)]}function ql(e,t){return e.reduce(([n,r],o)=>(t(o)?n.push(o):r.push(o),[n,r]),[[],[]])}var Kl=e=>e?.constructor.name==="Array",Ym=(e,t)=>{if(e.length!==t.length)return!1;for(let n=0;n{if(Object.is(e,t))return!0;if(e==null&&t!=null||e!=null&&t==null)return!1;if(typeof e?.isEqual=="function"&&typeof t?.isEqual=="function")return e.isEqual(t);if(typeof e=="function"&&typeof t=="function")return e.toString()===t.toString();if(Kl(e)&&Kl(t))return Ym(Array.from(e),Array.from(t));if(typeof e!="object"||typeof t!="object")return!1;const n=Object.keys(t??Object.create(null)),r=n.length;for(let o=0;oArray.isArray(e),Xm=e=>e===!0||e===!1,Yl=e=>e!=null&&typeof e=="object",Xt=e=>Yl(e)&&!Cr(e),go=e=>typeof e=="string",Qt=e=>typeof e=="function",Qm=e=>e==null,Dt=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),Jm=e=>Object.prototype.toString.call(e),Xl=Function.prototype.toString,Zm=Xl.call(Object),e0=e=>{if(!Yl(e)||Jm(e)!="[object Object]"||r0(e))return!1;const t=Object.getPrototypeOf(e);if(t===null)return!0;const n=Dt(t,"constructor")&&t.constructor;return typeof n=="function"&&n instanceof n&&Xl.call(n)==Zm},t0=e=>typeof e=="object"&&e!==null&&"$$typeof"in e&&"props"in e,n0=e=>typeof e=="object"&&e!==null&&"__v_isVNode"in e,r0=e=>t0(e)||n0(e),po=(e,...t)=>(typeof e=="function"?e(...t):e)??void 0,o0=e=>e(),i0=()=>{},mo=(...e)=>(...t)=>{e.forEach(function(n){n?.(...t)})},s0=(()=>{let e=0;return()=>(e++,e.toString(36))})();function kt(e,t,...n){if(e in t){const o=t[e];return Qt(o)?o(...n):o}const r=new Error(`No matching key: ${JSON.stringify(e)} in ${JSON.stringify(Object.keys(t))}`);throw Error.captureStackTrace?.(r,kt),r}var Ql=(e,t)=>{try{return e()}catch(n){return n instanceof Error&&Error.captureStackTrace?.(n,Ql),t?.()}},{floor:Jl,abs:Zl,round:vo,min:a0,max:l0,pow:c0,sign:u0}=Math,gs=e=>Number.isNaN(e),zt=e=>gs(e)?0:e,ec=(e,t)=>(e%t+t)%t,d0=(e,t)=>(e%t+t)%t,h0=(e,t)=>zt(e)>=t,f0=(e,t)=>zt(e)<=t,g0=(e,t,n)=>{const r=zt(e),o=t==null||r>=t,i=n==null||r<=n;return o&&i},p0=(e,t,n)=>vo((zt(e)-t)/n)*n+t,Ve=(e,t,n)=>a0(l0(zt(e),t),n),m0=(e,t,n)=>(zt(e)-t)/(n-t),v0=(e,t,n,r)=>Ve(p0(e*(n-t)+t,t,r),t,n),tc=(e,t)=>{let n=e,r=t.toString(),o=r.indexOf("."),i=o>=0?r.length-o:0;if(i>0){let s=c0(10,i);n=vo(n*s)/s}return n},ps=(e,t)=>typeof t=="number"?Jl(e*t+.5)/t:vo(e),nc=(e,t,n,r)=>{const o=t!=null?Number(t):0,i=Number(n),s=(e-o)%r;let a=Zl(s)*2>=r?e+u0(s)*(r-Zl(s)):e-s;if(a=tc(a,r),!gs(o)&&ai){const l=Jl((i-o)/r),c=o+l*r;a=l<=0||c{const r=Math.pow(n,t);return vo(e*r)/r},rc=e=>{if(!Number.isFinite(e))return 0;let t=1,n=0;for(;Math.round(e*t)/t!==e;)t*=10,n+=1;return n},oc=(e,t,n)=>{let r=t==="+"?e+n:e-n;if(e%1!==0||n%1!==0){const o=10**Math.max(rc(e),rc(n));e=Math.round(e*o),n=Math.round(n*o),r=t==="+"?e+n:e-n,r/=o}return r},b0=(e,t)=>oc(zt(e),"+",t),y0=(e,t)=>oc(zt(e),"-",t);function bo(e){if(!e0(e)||e===void 0)return e;const t=Reflect.ownKeys(e).filter(r=>typeof r=="string"),n={};for(const r of t){const o=e[r];o!==void 0&&(n[r]=bo(o))}return n}function x0(e,t=Object.is){let n={...e};const r=new Set,o=u=>(r.add(u),()=>r.delete(u)),i=()=>{r.forEach(u=>u())};return{subscribe:o,get:u=>n[u],set:(u,h)=>{t(n[u],h)||(n[u]=h,i())},update:u=>{let h=!1;for(const d in u){const m=u[d];m!==void 0&&!t(n[d],m)&&(n[d]=m,h=!0)}h&&i()},snapshot:()=>({...n})}}function wr(...e){e.length===1?e[0]:e[1],e.length===2&&e[0]}function ic(e,t){if(e==null)throw new Error(t())}function k0(e,t){return`${e} returned \`undefined\`. Seems you forgot to wrap component within ${t}`}function Mn(e={}){const{name:t,strict:n=!0,hookName:r="useContext",providerName:o="Provider",errorMessage:i,defaultValue:s}=e,a=P.createContext(s);a.displayName=t;function l(){const c=P.useContext(a);if(!c&&n){const u=new Error(i??k0(r,o));throw u.name="ContextError",Error.captureStackTrace?.(u,l),u}return c}return[a.Provider,l,a]}const[CT,sc]=Mn({name:"EnvironmentContext",hookName:"useEnvironmentContext",providerName:"",strict:!1,defaultValue:{getRootNode:()=>document,getDocument:()=>document,getWindow:()=>window}});function C0(e){if(!e)return;const t=e.selectionStart??0,n=e.selectionEnd??0;Math.abs(n-t)===0&&t===0&&e.setSelectionRange(e.value.length,e.value.length)}var ac=e=>Math.max(0,Math.min(1,e)),w0=(e,t)=>e.map((n,r)=>e[(Math.max(t,0)+r)%e.length]),lc=()=>{},yo=e=>typeof e=="object"&&e!==null,S0=2147483647,E0=1,O0=9,P0=11,Te=e=>yo(e)&&e.nodeType===E0&&typeof e.nodeName=="string",ms=e=>yo(e)&&e.nodeType===O0,R0=e=>yo(e)&&e===e.window,cc=e=>Te(e)?e.localName||"":"#document";function I0(e){return["html","body","#document"].includes(cc(e))}var T0=e=>yo(e)&&e.nodeType!==void 0,jn=e=>T0(e)&&e.nodeType===P0&&"host"in e,N0=e=>Te(e)&&e.localName==="input",A0=e=>!!e?.matches("a[href]"),_0=e=>Te(e)?e.offsetWidth>0||e.offsetHeight>0||e.getClientRects().length>0:!1;function V0(e){if(!e)return!1;const t=e.getRootNode();return Sr(t)===e}var L0=/(textarea|select)/;function uc(e){if(e==null||!Te(e))return!1;try{return N0(e)&&e.selectionStart!=null||L0.test(e.localName)||e.isContentEditable||e.getAttribute("contenteditable")==="true"||e.getAttribute("contenteditable")===""}catch{return!1}}function Jt(e,t){if(!e||!t||!Te(e)||!Te(t))return!1;const n=t.getRootNode?.();if(e===t||e.contains(t))return!0;if(n&&jn(n)){let r=t;for(;r;){if(e===r)return!0;r=r.parentNode||r.host}}return!1}function Le(e){return ms(e)?e:R0(e)?e.document:e?.ownerDocument??document}function F0(e){return Le(e).documentElement}function be(e){return jn(e)?be(e.host):ms(e)?e.defaultView??window:Te(e)?e.ownerDocument?.defaultView??window:window}function Sr(e){let t=e.activeElement;for(;t?.shadowRoot;){const n=t.shadowRoot.activeElement;if(!n||n===t)break;t=n}return t}function D0(e){if(cc(e)==="html")return e;const t=e.assignedSlot||e.parentNode||jn(e)&&e.host||F0(e);return jn(t)?t.host:t}var vs=new WeakMap;function xo(e){return vs.has(e)||vs.set(e,be(e).getComputedStyle(e)),vs.get(e)}var ko=()=>typeof document<"u";function z0(){return navigator.userAgentData?.platform??navigator.platform}function M0(){const e=navigator.userAgentData;return e&&Array.isArray(e.brands)?e.brands.map(({brand:t,version:n})=>`${t}/${n}`).join(" "):navigator.userAgent}var bs=e=>ko()&&e.test(z0()),dc=e=>ko()&&e.test(M0()),j0=e=>ko()&&e.test(navigator.vendor),hc=()=>ko()&&!!navigator.maxTouchPoints,$0=()=>bs(/^iPhone/i),B0=()=>bs(/^iPad/i)||wo()&&navigator.maxTouchPoints>1,Co=()=>$0()||B0(),W0=()=>wo()||Co(),wo=()=>bs(/^Mac/i),fc=()=>W0()&&j0(/apple/i),H0=()=>dc(/Firefox/i),U0=()=>dc(/Android/i);function G0(e){return e.composedPath?.()??e.nativeEvent?.composedPath?.()}function $e(e){return G0(e)?.[0]??e.target}function q0(e){return Q0(e).isComposing||e.keyCode===229}function K0(e){return e.pointerType===""&&e.isTrusted?!0:U0()&&e.pointerType?e.type==="click"&&e.buttons===1:e.detail===0&&!e.pointerType}var Y0=e=>e.button===2||wo()&&e.ctrlKey&&e.button===0,X0=e=>"touches"in e&&e.touches.length>0;function Q0(e){return e.nativeEvent??e}function gc(e,t="client"){const n=X0(e)?e.touches[0]||e.changedTouches[0]:e;return{x:n[`${t}X`],y:n[`${t}Y`]}}var se=(e,t,n,r)=>{const o=typeof e=="function"?e():e;return o?.addEventListener(t,n,r),()=>{o?.removeEventListener(t,n,r)}};function J0(e,t){const{type:n="HTMLInputElement",property:r="value"}=t,o=be(e)[n].prototype;return Object.getOwnPropertyDescriptor(o,r)??{}}function Z0(e){if(e.localName==="input")return"HTMLInputElement";if(e.localName==="textarea")return"HTMLTextAreaElement";if(e.localName==="select")return"HTMLSelectElement"}function So(e,t,n="value"){if(!e)return;const r=Z0(e);r&&J0(e,{type:r,property:n}).set?.call(e,t),e.setAttribute(n,t)}function ev(e,t){const{value:n,bubbles:r=!0}=t;if(!e)return;const o=be(e);e instanceof o.HTMLInputElement&&(So(e,`${n}`),e.dispatchEvent(new o.Event("input",{bubbles:r})))}function tv(e){return nv(e)?e.form:e.closest("form")}function nv(e){return e.matches("textarea, input, select, button")}function rv(e,t){if(!e)return;const n=tv(e),r=o=>{o.defaultPrevented||t()};return n?.addEventListener("reset",r,{passive:!0}),()=>n?.removeEventListener("reset",r)}function ov(e,t){const n=e?.closest("fieldset");if(!n)return;t(n.disabled);const r=be(n),o=new r.MutationObserver(()=>t(n.disabled));return o.observe(n,{attributes:!0,attributeFilter:["disabled"]}),()=>o.disconnect()}function ys(e,t){if(!e)return;const{onFieldsetDisabledChange:n,onFormReset:r}=t,o=[rv(e,r),ov(e,n)];return()=>o.forEach(i=>i?.())}var pc=e=>Te(e)&&e.tagName==="IFRAME",iv=e=>!Number.isNaN(parseInt(e.getAttribute("tabindex")||"0",10)),sv=e=>parseInt(e.getAttribute("tabindex")||"0",10)<0,xs="input:not([type='hidden']):not([disabled]), select:not([disabled]), textarea:not([disabled]), a[href], button:not([disabled]), [tabindex], iframe, object, embed, area[href], audio[controls], video[controls], [contenteditable]:not([contenteditable='false']), details > summary:first-of-type",mc=(e,t=!1)=>{if(!e)return[];const n=Array.from(e.querySelectorAll(xs));(t==!0||t=="if-empty"&&n.length===0)&&Te(e)&&Mt(e)&&n.unshift(e);const o=n.filter(Mt);return o.forEach((i,s)=>{if(pc(i)&&i.contentDocument){const a=i.contentDocument.body;o.splice(s,1,...mc(a))}}),o};function Mt(e){return!e||e.closest("[inert]")?!1:e.matches(xs)&&_0(e)}function ks(e,t){if(!e)return[];const r=Array.from(e.querySelectorAll(xs)).filter(Zt);return r.forEach((o,i)=>{if(pc(o)&&o.contentDocument){const s=o.contentDocument.body,a=ks(s);r.splice(i,1,...a)}}),r.length,r}function Zt(e){return e!=null&&e.tabIndex>0?!0:Mt(e)&&!sv(e)}function Er(e){return e.tabIndex<0&&(/^(audio|video|details)$/.test(e.localName)||uc(e))&&!iv(e)?0:e.tabIndex}function Cs(e){const{root:t,getInitialEl:n,filter:r,enabled:o=!0}=e;if(!o)return;let i=null;if(i||(i=typeof n=="function"?n():n),i||(i=t?.querySelector("[data-autofocus],[autofocus]")),!i){const s=ks(t);i=r?s.filter(r)[0]:s[0]}return i||t||void 0}function ws(e){const t=new Set;function n(r){const o=globalThis.requestAnimationFrame(r);t.add(()=>globalThis.cancelAnimationFrame(o))}return n(()=>n(e)),function(){t.forEach(o=>o())}}function G(e){let t;const n=globalThis.requestAnimationFrame(()=>{t=e()});return()=>{globalThis.cancelAnimationFrame(n),t?.()}}function av(e,t,n){const r=G(()=>{e.removeEventListener(t,o,!0),n()}),o=()=>{r(),n()};return e.addEventListener(t,o,{once:!0,capture:!0}),r}function lv(e,t){if(!e)return;const{attributes:n,callback:r}=t,o=e.ownerDocument.defaultView||window,i=new o.MutationObserver(s=>{for(const a of s)a.type==="attributes"&&a.attributeName&&n.includes(a.attributeName)&&r(a)});return i.observe(e,{attributes:!0,attributeFilter:n}),()=>i.disconnect()}function Eo(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(lv(i,t))})),()=>{o.forEach(i=>i?.())}}function vc(e){const t=()=>{const n=be(e);e.dispatchEvent(new n.MouseEvent("click"))};H0()?av(e,"keyup",t):queueMicrotask(t)}function Oo(e){const t=D0(e);return I0(t)?Le(t).body:Te(t)&&Ss(t)?t:Oo(t)}function bc(e,t=[]){const n=Oo(e),r=n===e.ownerDocument.body,o=be(n);return r?t.concat(o,o.visualViewport||[],Ss(n)?n:[]):t.concat(n,bc(n,[]))}var cv=/auto|scroll|overlay|hidden|clip/,uv=new Set(["inline","contents"]);function Ss(e){const t=be(e),{overflow:n,overflowX:r,overflowY:o,display:i}=t.getComputedStyle(e);return cv.test(n+o+r)&&!uv.has(i)}function dv(e){return e.scrollHeight>e.clientHeight||e.scrollWidth>e.clientWidth}function Po(e,t){const{rootEl:n,...r}=t||{};!e||!n||!Ss(n)||!dv(n)||e.scrollIntoView(r)}function yc(e,t){const{left:n,top:r,width:o,height:i}=t.getBoundingClientRect(),s={x:e.x-n,y:e.y-r},a={x:ac(s.x/o),y:ac(s.y/i)};function l(c={}){const{dir:u="ltr",orientation:h="horizontal",inverted:d}=c,m=typeof d=="object"?d.x:d,f=typeof d=="object"?d.y:d;return h==="horizontal"?u==="rtl"||m?1-a.x:a.x:f?1-a.y:a.y}return{offset:s,percent:a,getPercentValue:l}}function hv(e,t){const n=e.body,r="pointerLockElement"in e||"mozPointerLockElement"in e,o=()=>!!e.pointerLockElement;function i(){}function s(l){o(),console.error("PointerLock error occurred:",l),e.exitPointerLock()}if(!r)return;try{n.requestPointerLock()}catch{}const a=[se(e,"pointerlockchange",i,!1),se(e,"pointerlockerror",s,!1)];return()=>{a.forEach(l=>l()),e.exitPointerLock()}}var $n="default",Es="",Ro=new WeakMap;function fv(e={}){const{target:t,doc:n}=e,r=n??document,o=r.documentElement;return Co()?($n==="default"&&(Es=o.style.webkitUserSelect,o.style.webkitUserSelect="none"),$n="disabled"):t&&(Ro.set(t,t.style.userSelect),t.style.userSelect="none"),()=>gv({target:t,doc:r})}function gv(e={}){const{target:t,doc:n}=e,o=(n??document).documentElement;if(Co()){if($n!=="disabled")return;$n="restoring",setTimeout(()=>{ws(()=>{$n==="restoring"&&(o.style.webkitUserSelect==="none"&&(o.style.webkitUserSelect=Es||""),Es="",$n="default")})},300)}else if(t&&Ro.has(t)){const i=Ro.get(t);t.style.userSelect==="none"&&(t.style.userSelect=i??""),t.getAttribute("style")===""&&t.removeAttribute("style"),Ro.delete(t)}}function xc(e={}){const{defer:t,target:n,...r}=e,o=t?G:s=>s(),i=[];return i.push(o(()=>{const s=typeof n=="function"?n():n;i.push(fv({...r,target:s}))})),()=>{i.forEach(s=>s?.())}}function pv(e,t){const{onPointerMove:n,onPointerUp:r}=t,o=a=>{const l=gc(a),c=Math.sqrt(l.x**2+l.y**2),u=a.pointerType==="touch"?10:5;if(!(c{const l=gc(a);r({point:l,event:a})},s=[se(e,"pointermove",o,!1),se(e,"pointerup",i,!1),se(e,"pointercancel",i,!1),se(e,"contextmenu",i,!1),xc({doc:e})];return()=>{s.forEach(a=>a())}}function Io(e,t){return Array.from(e?.querySelectorAll(t)??[])}function mv(e,t){return e?.querySelector(t)??null}var Os=e=>e.id;function vv(e,t,n=Os){return e.find(r=>n(r)===t)}function Ps(e,t,n=Os){const r=vv(e,t,n);return r?e.indexOf(r):-1}function bv(e,t,n=!0){let r=Ps(e,t);return r=n?(r+1)%e.length:Math.min(r+1,e.length-1),e[r]}function yv(e,t,n=!0){let r=Ps(e,t);return r===-1?n?e[e.length-1]:null:(r=n?(r-1+e.length)%e.length:Math.max(0,r-1),e[r])}var xv=e=>e.split("").map(t=>{const n=t.charCodeAt(0);return n>0&&n<128?t:n>=128&&n<=255?`/x${n.toString(16)}`.replace("/","\\"):""}).join("").trim(),kv=e=>xv(e.dataset?.valuetext??e.textContent??""),Cv=(e,t)=>e.trim().toLowerCase().startsWith(t.toLowerCase());function wv(e,t,n,r=Os){const o=n?Ps(e,n,r):-1;let i=n?w0(e,o):e;return t.length===1&&(i=i.filter(a=>r(a)!==n)),i.find(a=>Cv(kv(a),t))}function To(e,t){if(!e)return lc;const n=Object.keys(t).reduce((r,o)=>(r[o]=e.style.getPropertyValue(o),r),{});return Object.assign(e.style,t),()=>{Object.assign(e.style,n),e.style.length===0&&e.removeAttribute("style")}}function Sv(e,t,n){if(!e)return lc;const r=e.style.getPropertyValue(t);return e.style.setProperty(t,n),()=>{e.style.setProperty(t,r),e.style.length===0&&e.removeAttribute("style")}}function Ev(e,t){const{state:n,activeId:r,key:o,timeout:i=350,itemToId:s}=t,a=n.keysSoFar+o,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a;let u=e.slice();const h=wv(u,c,r,s);function d(){clearTimeout(n.timer),n.timer=-1}function m(f){n.keysSoFar=f,d(),f!==""&&(n.timer=+setTimeout(()=>{m(""),d()},i))}return m(a),h}var Or=Object.assign(Ev,{defaultOptions:{keysSoFar:"",timer:-1},isValidEvent:Ov});function Ov(e){return e.key.length===1&&!e.ctrlKey&&!e.metaKey}function Pv(e,t,n){const{signal:r}=t;return[new Promise((s,a)=>{const l=setTimeout(()=>{a(new Error(`Timeout of ${n}ms exceeded`))},n);r.addEventListener("abort",()=>{clearTimeout(l),a(new Error("Promise aborted"))}),e.then(c=>{r.aborted||(clearTimeout(l),s(c))}).catch(c=>{r.aborted||(clearTimeout(l),a(c))})}),()=>t.abort()]}function Rv(e,t){const{timeout:n,rootNode:r}=t,o=be(r),i=Le(r),s=new o.AbortController;return Pv(new Promise(a=>{const l=e();if(l){a(l);return}const c=new o.MutationObserver(()=>{const u=e();u&&u.isConnected&&(c.disconnect(),a(u))});c.observe(i.body,{childList:!0,subtree:!0})}),s,n)}var Iv=(...e)=>e.map(t=>t?.trim?.()).filter(Boolean).join(" "),Tv=/((?:--)?(?:\w+-?)+)\s*:\s*([^;]*)/g,kc=e=>{const t={};let n;for(;n=Tv.exec(e);)t[n[1]]=n[2];return t},Nv=(e,t)=>{if(go(e)){if(go(t))return`${e};${t}`;e=kc(e)}else go(t)&&(t=kc(t));return Object.assign({},e??{},t??{})};function qe(...e){let t={};for(let n of e){if(!n)continue;for(let o in t){if(o.startsWith("on")&&typeof t[o]=="function"&&typeof n[o]=="function"){t[o]=mo(n[o],t[o]);continue}if(o==="className"||o==="class"){t[o]=Iv(t[o],n[o]);continue}if(o==="style"){t[o]=Nv(t[o],n[o]);continue}t[o]=n[o]!==void 0?n[o]:t[o]}for(let o in n)t[o]===void 0&&(t[o]=n[o]);const r=Object.getOwnPropertySymbols(n);for(let o of r)t[o]=n[o]}return t}function Cc(e,t,n){let r=[],o;return i=>{const s=e(i);return(s.length!==r.length||s.some((l,c)=>!Ze(r[c],l)))&&(r=s,o=t(s,i)),o}}function jt(){return{and:(...e)=>function(n){return e.every(r=>n.guard(r))},or:(...e)=>function(n){return e.some(r=>n.guard(r))},not:e=>function(n){return!n.guard(e)}}}function wT(e){return e}function wc(){return{guards:jt(),createMachine:e=>e,choose:e=>function({choose:n}){return n(e)?.actions}}}var Bn=(e=>(e.NotStarted="Not Started",e.Started="Started",e.Stopped="Stopped",e))(Bn||{}),Rs="__init__";function Av(e){const t=()=>e.getRootNode?.()??document,n=()=>Le(t());return{...e,getRootNode:t,getDoc:n,getWin:()=>n().defaultView??window,getActiveElement:()=>Sr(t()),isActiveElement:V0,getById:s=>t().getElementById(s)}}function No(...e){return t=>{const n=[];for(const r of e)if(typeof r=="function"){const o=r(t);typeof o=="function"&&n.push(o)}else r&&(r.current=t);if(n.length)return()=>{for(const r of n)r()}}}function _v(e){let t=Object.getOwnPropertyDescriptor(e.props,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning;return n?e.ref:(t=Object.getOwnPropertyDescriptor(e,"ref")?.get,n=t&&"isReactWarning"in t&&t.isReactWarning,n?e.props.ref:e.props.ref||e.ref)}const Is=e=>{const t=P.memo(P.forwardRef((n,r)=>{const{asChild:o,children:i,...s}=n;if(!o)return P.createElement(e,{...s,ref:r},i);if(!P.isValidElement(i))return null;const a=P.Children.only(i),l=_v(a);return P.cloneElement(a,{...qe(s,a.props),ref:r?No(r,l):l})}));return t.displayName=e.displayName||e.name,t},Ct=(()=>{const e=new Map;return new Proxy(Is,{apply(t,n,r){return Is(r[0])},get(t,n){const r=n;return e.has(r)||e.set(r,Is(r)),e.get(r)}})})(),[ET,Vv]=Mn({name:"LocaleContext",hookName:"useLocaleContext",providerName:"",strict:!1,defaultValue:{dir:"ltr",locale:"en-US"}}),Sc=()=>(e,t)=>t.reduce((n,r)=>{const[o,i]=n,s=r;return i[s]!==void 0&&(o[s]=i[s]),delete i[s],[o,i]},[{},{...e}]),Ec=e=>Sc()(e,["immediate","lazyMount","onExitComplete","present","skipAnimationOnMount","unmountOnExit"]);function Lv(e){return new Proxy({},{get(t,n){return n==="style"?r=>e({style:r}).style:e}})}var M=()=>e=>Array.from(new Set(e));function Fv(e,t){const{state:n,send:r,context:o}=e,i=n.matches("mounted","unmountSuspended");return{skip:!o.get("initial"),present:i,setNode(s){s&&r({type:"NODE.SET",node:s})},unmount(){r({type:"UNMOUNT"})}}}var Dv={props({props:e}){return{...e,present:!!e.present}},initialState({prop:e}){return e("present")?"mounted":"unmounted"},refs(){return{node:null,styles:null}},context({bindable:e}){return{unmountAnimationName:e(()=>({defaultValue:null})),prevAnimationName:e(()=>({defaultValue:null})),present:e(()=>({defaultValue:!1})),initial:e(()=>({sync:!0,defaultValue:!1}))}},exit:["clearInitial","cleanupNode"],watch({track:e,prop:t,send:n}){e([()=>t("present")],()=>{n({type:"PRESENCE.CHANGED"})})},on:{"NODE.SET":{actions:["setupNode"]},"PRESENCE.CHANGED":{actions:["setInitial","syncPresence"]}},states:{mounted:{on:{UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]},"UNMOUNT.SUSPEND":{target:"unmountSuspended"}}},unmountSuspended:{effects:["trackAnimationEvents"],on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]},UNMOUNT:{target:"unmounted",actions:["clearPrevAnimationName","invokeOnExitComplete"]}}},unmounted:{on:{MOUNT:{target:"mounted",actions:["setPrevAnimationName"]}}}},implementations:{actions:{setInitial:({context:e})=>{e.get("initial")||queueMicrotask(()=>{e.set("initial",!0)})},clearInitial:({context:e})=>{e.set("initial",!1)},invokeOnExitComplete:({prop:e})=>{e("onExitComplete")?.()},setupNode:({refs:e,event:t})=>{e.get("node")!==t.node&&(e.set("node",t.node),e.set("styles",xo(t.node)))},cleanupNode:({refs:e})=>{e.set("node",null),e.set("styles",null)},syncPresence:({context:e,refs:t,send:n,prop:r})=>{const o=r("present");if(o)return n({type:"MOUNT",src:"presence.changed"});const i=t.get("node");if(!o&&i?.ownerDocument.visibilityState==="hidden")return n({type:"UNMOUNT",src:"visibilitychange"});G(()=>{const s=Ao(t.get("styles"));e.set("unmountAnimationName",s),s==="none"||s===e.get("prevAnimationName")||t.get("styles")?.display==="none"||t.get("styles")?.animationDuration==="0s"?n({type:"UNMOUNT",src:"presence.changed"}):n({type:"UNMOUNT.SUSPEND"})})},setPrevAnimationName:({context:e,refs:t})=>{G(()=>{e.set("prevAnimationName",Ao(t.get("styles")))})},clearPrevAnimationName:({context:e})=>{e.set("prevAnimationName",null)}},effects:{trackAnimationEvents:({context:e,refs:t,send:n})=>{const r=t.get("node");if(!r)return;const o=a=>{(a.composedPath?.()?.[0]??a.target)===r&&e.set("prevAnimationName",Ao(t.get("styles")))},i=a=>{const l=Ao(t.get("styles"));$e(a)===r&&l===e.get("unmountAnimationName")&&n({type:"UNMOUNT",src:"animationend"})};r.addEventListener("animationstart",o),r.addEventListener("animationcancel",i),r.addEventListener("animationend",i);const s=To(r,{animationFillMode:"forwards"});return()=>{r.removeEventListener("animationstart",o),r.removeEventListener("animationcancel",i),r.removeEventListener("animationend",i),ws(()=>s())}}}}};function Ao(e){return e?.animationName||"none"}M()(["onExitComplete","present","immediate"]);var Oc=typeof globalThis.document<"u"?P.useLayoutEffect:P.useEffect;function _o(e){const t=e().value??e().defaultValue,n=e().isEqual??Object.is,[r]=P.useState(t),[o,i]=P.useState(r),s=e().value!==void 0,a=P.useRef(o);a.current=s?e().value:o;const l=P.useRef(a.current);Oc(()=>{l.current=a.current},[o,e().value]);const c=h=>{const d=l.current,m=Qt(h)?h(d):h;e().debug&&console.log(`[bindable > ${e().debug}] setValue`,{next:m,prev:d}),s||i(m),n(m,d)||e().onChange?.(m,d)};function u(){return s?e().value:o}return{initial:r,ref:a,get:u,set(h){(e().sync?Lt.flushSync:o0)(()=>c(h))},invoke(h,d){e().onChange?.(h,d)},hash(h){return e().hash?.(h)??String(h)}}}_o.cleanup=e=>{P.useEffect(()=>e,[])},_o.ref=e=>{const t=P.useRef(e);return{get:()=>t.current,set:n=>{t.current=n}}};function zv(e){const t=P.useRef(e);return{get(n){return t.current[n]},set(n,r){t.current[n]=r}}}var Mv=(e,t)=>{const n=P.useRef(!1),r=P.useRef(!1);P.useEffect(()=>{if(n.current&&r.current)return t();r.current=!0},[...(e??[]).map(o=>typeof o=="function"?o():o)]),P.useEffect(()=>(n.current=!0,()=>{n.current=!1}),[])};function Pc(e,t={}){const n=P.useMemo(()=>{const{id:T,ids:S,getRootNode:_}=t;return Av({id:T,ids:S,getRootNode:_})},[t]),r=(...T)=>{e.debug&&console.log(...T)},o=e.props?.({props:bo(t),scope:n})??t,i=jv(o),s=e.context?.({prop:i,bindable:_o,scope:n,flush:Ic,getContext(){return l},getComputed(){return k},getRefs(){return p},getEvent(){return m()}}),a=Rc(s),l={get(T){return a.current?.[T].ref.current},set(T,S){a.current?.[T].set(S)},initial(T){return a.current?.[T].initial},hash(T){const S=a.current?.[T].get();return a.current?.[T].hash(S)}},c=P.useRef(new Map),u=P.useRef(null),h=P.useRef(null),d=P.useRef({type:""}),m=()=>({...d.current,current(){return d.current},previous(){return h.current}}),f=()=>({...O,matches(...T){return T.includes(O.ref.current)},hasTag(T){return!!e.states[O.ref.current]?.tags?.includes(T)}}),p=zv(e.refs?.({prop:i,context:l})??{}),v=()=>({state:f(),context:l,event:m(),prop:i,send:A,action:b,guard:y,track:Mv,refs:p,computed:k,flush:Ic,scope:n,choose:w}),b=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(D=>{const z=e.implementations?.actions?.[D];return z||wr(`[zag-js] No implementation found for action "${JSON.stringify(D)}"`),z});for(const D of _)D?.(v())},y=T=>Qt(T)?T(v()):e.implementations?.guards?.[T](v()),x=T=>{const S=Qt(T)?T(v()):T;if(!S)return;const _=S.map(z=>{const W=e.implementations?.effects?.[z];return W||wr(`[zag-js] No implementation found for effect "${JSON.stringify(z)}"`),W}),D=[];for(const z of _){const W=z?.(v());W&&D.push(W)}return()=>D.forEach(z=>z?.())},w=T=>fo(T).find(S=>{let _=!S.guard;return go(S.guard)?_=!!y(S.guard):Qt(S.guard)&&(_=S.guard(v())),_}),k=T=>{ic(e.computed,()=>"[zag-js] No computed object found on machine");const S=e.computed[T];return S({context:l,event:m(),prop:i,refs:p,scope:n,computed:k})},O=_o(()=>({defaultValue:e.initialState({prop:i}),onChange(T,S){S&&(c.current.get(S)?.(),c.current.delete(S)),S&&b(e.states[S]?.exit),b(u.current?.actions);const _=x(e.states[T]?.effects);if(_&&c.current.set(T,_),S===Rs){b(e.entry);const D=x(e.effects);D&&c.current.set(Rs,D)}b(e.states[T]?.entry)}})),R=P.useRef(void 0),I=P.useRef(Bn.NotStarted);Oc(()=>{queueMicrotask(()=>{const _=I.current===Bn.Started;I.current=Bn.Started,r(_?"rehydrating...":"initializing...");const D=R.current??O.initial;O.invoke(D,_?O.get():Rs)});const T=c.current,S=O.ref.current;return()=>{r("unmounting..."),R.current=S,I.current=Bn.Stopped,T.forEach(_=>_?.()),c.current=new Map,u.current=null,queueMicrotask(()=>{b(e.exit)})}},[]);const N=()=>"ref"in O?O.ref.current:O.get(),A=T=>{queueMicrotask(()=>{if(I.current!==Bn.Started)return;h.current=d.current,d.current=T;let S=N();const _=e.states[S].on?.[T.type]??e.on?.[T.type],D=w(_);if(!D)return;u.current=D;const z=D.target??S;r("transition",T.type,D.target||S,`(${D.actions})`);const W=z!==S;W?Lt.flushSync(()=>O.set(z)):D.reenter&&!W?O.invoke(S,S):b(D.actions??[])})};return e.watch?.(v()),{state:f(),send:A,context:l,prop:i,scope:n,refs:p,computed:k,event:m(),getStatus:()=>I.current}}function Rc(e){const t=P.useRef(e);return t.current=e,t}function jv(e){const t=Rc(e);return function(r){return t.current[r]}}function Ic(e){queueMicrotask(()=>{Lt.flushSync(()=>e())})}var $v=Lv(e=>e);function Bv(e,t={}){const{sync:n=!1}=t,r=Wv(e);return P.useCallback((...o)=>n?queueMicrotask(()=>r.current?.(...o)):r.current?.(...o),[n,r])}function Wv(e){const t=P.useRef(e);return t.current=e,t}const Ts=(e={})=>{const{lazyMount:t,unmountOnExit:n,present:r,skipAnimationOnMount:o=!1,...i}=e,s=P.useRef(!1),a={...i,present:r,onExitComplete:Bv(e.onExitComplete)},l=Pc(Dv,a),c=Fv(l);c.present&&(s.current=!0);const u=!c.present&&!s.current&&t||n&&!c.present&&s.current,h=()=>({"data-state":c.skip&&o?void 0:r?"open":"closed",hidden:!c.present});return{ref:c.setNode,getPresenceProps:h,present:c.present,unmounted:u}},[Tc,Ns]=Mn({name:"PresenceContext",hookName:"usePresenceContext",providerName:""}),Vo=ve("span"),{withContext:Hv}=Ft({key:"text"}),Fe=Hv("p");function Nc(e,t=[]){const n=P.useRef(()=>{throw new Error("Cannot call an event handler while rendering.")});return P.useInsertionEffect(()=>{n.current=e}),P.useCallback((...r)=>n.current?.(...r),t)}function en(e={}){const t=Nc(e.onOpen),n=Nc(e.onClose),[r,o]=P.useState(e.defaultOpen||!1),i=e.open!==void 0?e.open:r,s=e.open!==void 0,a=P.useCallback(()=>{s||o(!1),n?.()},[s,n]),l=P.useCallback(()=>{s||o(!0),t?.()},[s,t]),c=P.useCallback(()=>{i?a():l()},[i,l,a]);return{open:i,onOpen:l,onClose:a,onToggle:c,setOpen:o}}var j=(e,t=[])=>({parts:(...n)=>{if(Uv(t))return j(e,n);throw new Error("createAnatomy().parts(...) should only be called once. Did you mean to use .extendWith(...) ?")},extendWith:(...n)=>j(e,[...t,...n]),omit:(...n)=>j(e,t.filter(r=>!n.includes(r))),rename:n=>j(n,t),keys:()=>t,build:()=>[...new Set(t)].reduce((n,r)=>Object.assign(n,{[r]:{selector:[`&[data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`,`& [data-scope="${Wn(e)}"][data-part="${Wn(r)}"]`].join(", "),attrs:{"data-scope":Wn(e),"data-part":Wn(r)}}}),{})}),Wn=e=>e.replace(/([A-Z])([A-Z])/g,"$1-$2").replace(/([a-z])([A-Z])/g,"$1-$2").replace(/[\s_]+/g,"-").toLowerCase(),Uv=e=>e.length===0,Ac=j("collapsible").parts("root","trigger","content","indicator");Ac.build(),M()(["dir","disabled","getRootNode","id","ids","onExitComplete","onOpenChange","defaultOpen","open"]);var Gv=Object.defineProperty,qv=(e,t,n)=>t in e?Gv(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,As=(e,t,n)=>qv(e,t+"",n),Kv=(e,t)=>{if(Object.keys(e).length!==Object.keys(t).length)return!1;for(let n in e)if(e[n]!==t[n])return!1;return!0},_s=class{toHexInt(){return this.toFormat("rgba").toHexInt()}getChannelValue(e){if(e in this)return this[e];throw new Error("Unsupported color channel: "+e)}getChannelValuePercent(e,t){const n=t??this.getChannelValue(e),{minValue:r,maxValue:o}=this.getChannelRange(e);return m0(n,r,o)}getChannelPercentValue(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=v0(t,n,r,o);return nc(i,n,r,o)}withChannelValue(e,t){const{minValue:n,maxValue:r}=this.getChannelRange(e);if(e in this){let o=this.clone();return o[e]=Ve(t,n,r),o}throw new Error("Unsupported color channel: "+e)}getColorAxes(e){let{xChannel:t,yChannel:n}=e,r=t||this.getChannels().find(s=>s!==n),o=n||this.getChannels().find(s=>s!==r),i=this.getChannels().find(s=>s!==r&&s!==o);return{xChannel:r,yChannel:o,zChannel:i}}incrementChannel(e,t){const{minValue:n,maxValue:r,step:o}=this.getChannelRange(e),i=nc(Ve(this.getChannelValue(e)+t,n,r),n,r,o);return this.withChannelValue(e,i)}decrementChannel(e,t){return this.incrementChannel(e,-t)}isEqual(e){return Kv(this.toJSON(),e.toJSON())&&this.getChannelValue("alpha")===e.getChannelValue("alpha")}},Yv=/^#[\da-f]+$/i,Xv=/^rgba?\((.*)\)$/,Qv=/[^#]/gi,_c=class Hi extends _s{constructor(t,n,r,o){super(),this.red=t,this.green=n,this.blue=r,this.alpha=o}static parse(t){let n=[];if(Yv.test(t)&&[4,5,7,9].includes(t.length)){const o=(t.length<6?t.replace(Qv,"$&$&"):t).slice(1).split("");for(;o.length>0;)n.push(parseInt(o.splice(0,2).join(""),16));n[3]=n[3]!==void 0?n[3]/255:void 0}const r=t.match(Xv);return r?.[1]&&(n=r[1].split(",").map(o=>Number(o.trim())).map((o,i)=>Ve(o,0,i<3?255:1))),n.length<3?void 0:new Hi(n[0],n[1],n[2],n[3]??1)}toString(t){switch(t){case"hex":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")).toUpperCase();case"hexa":return"#"+(this.red.toString(16).padStart(2,"0")+this.green.toString(16).padStart(2,"0")+this.blue.toString(16).padStart(2,"0")+Math.round(this.alpha*255).toString(16).padStart(2,"0")).toUpperCase();case"rgb":return`rgb(${this.red}, ${this.green}, ${this.blue})`;case"css":case"rgba":return`rgba(${this.red}, ${this.green}, ${this.blue}, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"hsb":return this.toHSB().toString("hsb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"rgba":return this;case"hsba":return this.toHSB();case"hsla":return this.toHSL();default:throw new Error("Unsupported color conversion: rgb -> "+t)}}toHexInt(){return this.red<<16|this.green<<8|this.blue}toHSB(){const t=this.red/255,n=this.green/255,r=this.blue/255,o=Math.min(t,n,r),i=Math.max(t,n,r),s=i-o,a=i===0?0:s/i;let l=0;if(s!==0){switch(i){case t:l=(n-r)/s+(nNumber(a.trim().replace("%","")));return new Ui(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsl":return`hsl(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%)`;case"css":case"hsla":return`hsla(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.lightness,2)}%, ${this.alpha})`;case"hsb":return this.toHSB().toString("hsb");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsla":return this;case"hsba":return this.toHSB();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsl -> "+t)}}toHSB(){let t=this.saturation/100,n=this.lightness/100,r=n+t*Math.min(n,1-n);return t=r===0?0:2*(1-n/r),new Fs(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.lightness/100,o=n*Math.min(r,1-r),i=(s,a=(s+t/30)%12)=>r-o*Math.max(Math.min(a-3,9-a,1),-1);return new Vs(Math.round(i(0)*255),Math.round(i(8)*255),Math.round(i(4)*255),oe(this.alpha,2))}clone(){return new Ui(this.hue,this.saturation,this.lightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"lightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="lightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"lightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,l:this.lightness,a:this.alpha}}getFormat(){return"hsla"}getChannels(){return Ui.colorChannels}};As(Vc,"colorChannels",["hue","saturation","lightness"]);var Ls=Vc,Zv=/hsb\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%)\)|hsba\(([-+]?\d+(?:.\d+)?\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d+(?:.\d+)?%\s*,\s*[-+]?\d(.\d+)?)\)/,Lc=class Gi extends _s{constructor(t,n,r,o){super(),this.hue=t,this.saturation=n,this.brightness=r,this.alpha=o}static parse(t){let n;if(n=t.match(Zv)){const[r,o,i,s]=(n[1]??n[2]).split(",").map(a=>Number(a.trim().replace("%","")));return new Gi(ec(r,360),Ve(o,0,100),Ve(i,0,100),Ve(s??1,0,1))}}toString(t){switch(t){case"css":return this.toHSL().toString("css");case"hex":return this.toRGB().toString("hex");case"hexa":return this.toRGB().toString("hexa");case"hsb":return`hsb(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%)`;case"hsba":return`hsba(${this.hue}, ${oe(this.saturation,2)}%, ${oe(this.brightness,2)}%, ${this.alpha})`;case"hsl":return this.toHSL().toString("hsl");case"rgb":return this.toRGB().toString("rgb");default:return this.toFormat(t).toString(t)}}toFormat(t){switch(t){case"hsba":return this;case"hsla":return this.toHSL();case"rgba":return this.toRGB();default:throw new Error("Unsupported color conversion: hsb -> "+t)}}toHSL(){let t=this.saturation/100,n=this.brightness/100,r=n*(1-t/2);return t=r===0||r===1?0:(n-r)/Math.min(r,1-r),new Ls(oe(this.hue,2),oe(t*100,2),oe(r*100,2),oe(this.alpha,2))}toRGB(){let t=this.hue,n=this.saturation/100,r=this.brightness/100,o=(i,s=(i+t/60)%6)=>r-n*r*Math.max(Math.min(s,4-s,1),0);return new Vs(Math.round(o(5)*255),Math.round(o(3)*255),Math.round(o(1)*255),oe(this.alpha,2))}clone(){return new Gi(this.hue,this.saturation,this.brightness,this.alpha)}getChannelFormatOptions(t){switch(t){case"hue":return{style:"unit",unit:"degree",unitDisplay:"narrow"};case"saturation":case"brightness":case"alpha":return{style:"percent"};default:throw new Error("Unknown color channel: "+t)}}formatChannelValue(t,n){let r=this.getChannelFormatOptions(t),o=this.getChannelValue(t);return(t==="saturation"||t==="brightness")&&(o/=100),new Intl.NumberFormat(n,r).format(o)}getChannelRange(t){switch(t){case"hue":return{minValue:0,maxValue:360,step:1,pageSize:15};case"saturation":case"brightness":return{minValue:0,maxValue:100,step:1,pageSize:10};case"alpha":return{minValue:0,maxValue:1,step:.01,pageSize:.1};default:throw new Error("Unknown color channel: "+t)}}toJSON(){return{h:this.hue,s:this.saturation,b:this.brightness,a:this.alpha}}getFormat(){return"hsba"}getChannels(){return Gi.colorChannels}};As(Lc,"colorChannels",["hue","saturation","brightness"]);var Fs=Lc,eb="aliceblue:f0f8ff,antiquewhite:faebd7,aqua:00ffff,aquamarine:7fffd4,azure:f0ffff,beige:f5f5dc,bisque:ffe4c4,black:000000,blanchedalmond:ffebcd,blue:0000ff,blueviolet:8a2be2,brown:a52a2a,burlywood:deb887,cadetblue:5f9ea0,chartreuse:7fff00,chocolate:d2691e,coral:ff7f50,cornflowerblue:6495ed,cornsilk:fff8dc,crimson:dc143c,cyan:00ffff,darkblue:00008b,darkcyan:008b8b,darkgoldenrod:b8860b,darkgray:a9a9a9,darkgreen:006400,darkkhaki:bdb76b,darkmagenta:8b008b,darkolivegreen:556b2f,darkorange:ff8c00,darkorchid:9932cc,darkred:8b0000,darksalmon:e9967a,darkseagreen:8fbc8f,darkslateblue:483d8b,darkslategray:2f4f4f,darkturquoise:00ced1,darkviolet:9400d3,deeppink:ff1493,deepskyblue:00bfff,dimgray:696969,dodgerblue:1e90ff,firebrick:b22222,floralwhite:fffaf0,forestgreen:228b22,fuchsia:ff00ff,gainsboro:dcdcdc,ghostwhite:f8f8ff,gold:ffd700,goldenrod:daa520,gray:808080,green:008000,greenyellow:adff2f,honeydew:f0fff0,hotpink:ff69b4,indianred:cd5c5c,indigo:4b0082,ivory:fffff0,khaki:f0e68c,lavender:e6e6fa,lavenderblush:fff0f5,lawngreen:7cfc00,lemonchiffon:fffacd,lightblue:add8e6,lightcoral:f08080,lightcyan:e0ffff,lightgoldenrodyellow:fafad2,lightgrey:d3d3d3,lightgreen:90ee90,lightpink:ffb6c1,lightsalmon:ffa07a,lightseagreen:20b2aa,lightskyblue:87cefa,lightslategray:778899,lightsteelblue:b0c4de,lightyellow:ffffe0,lime:00ff00,limegreen:32cd32,linen:faf0e6,magenta:ff00ff,maroon:800000,mediumaquamarine:66cdaa,mediumblue:0000cd,mediumorchid:ba55d3,mediumpurple:9370d8,mediumseagreen:3cb371,mediumslateblue:7b68ee,mediumspringgreen:00fa9a,mediumturquoise:48d1cc,mediumvioletred:c71585,midnightblue:191970,mintcream:f5fffa,mistyrose:ffe4e1,moccasin:ffe4b5,navajowhite:ffdead,navy:000080,oldlace:fdf5e6,olive:808000,olivedrab:6b8e23,orange:ffa500,orangered:ff4500,orchid:da70d6,palegoldenrod:eee8aa,palegreen:98fb98,paleturquoise:afeeee,palevioletred:d87093,papayawhip:ffefd5,peachpuff:ffdab9,peru:cd853f,pink:ffc0cb,plum:dda0dd,powderblue:b0e0e6,purple:800080,rebeccapurple:663399,red:ff0000,rosybrown:bc8f8f,royalblue:4169e1,saddlebrown:8b4513,salmon:fa8072,sandybrown:f4a460,seagreen:2e8b57,seashell:fff5ee,sienna:a0522d,silver:c0c0c0,skyblue:87ceeb,slateblue:6a5acd,slategray:708090,snow:fffafa,springgreen:00ff7f,steelblue:4682b4,tan:d2b48c,teal:008080,thistle:d8bfd8,tomato:ff6347,turquoise:40e0d0,violet:ee82ee,wheat:f5deb3,white:ffffff,whitesmoke:f5f5f5,yellow:ffff00,yellowgreen:9acd32",tb=e=>{const t=new Map,n=e.split(",");for(let r=0;r{if(Fc.has(e))return Lo(Fc.get(e));const t=Vs.parse(e)||Fs.parse(e)||Ls.parse(e);if(!t){const n=new Error("Invalid color value: "+e);throw Error.captureStackTrace?.(n,Lo),n}return t};const nb=["top","right","bottom","left"],$t=Math.min,Be=Math.max,Fo=Math.round,Do=Math.floor,ct=e=>({x:e,y:e}),rb={left:"right",right:"left",bottom:"top",top:"bottom"},ob={start:"end",end:"start"};function Ds(e,t,n){return Be(e,$t(t,n))}function wt(e,t){return typeof e=="function"?e(t):e}function St(e){return e.split("-")[0]}function Hn(e){return e.split("-")[1]}function zs(e){return e==="x"?"y":"x"}function Ms(e){return e==="y"?"height":"width"}const ib=new Set(["top","bottom"]);function ut(e){return ib.has(St(e))?"y":"x"}function js(e){return zs(ut(e))}function sb(e,t,n){n===void 0&&(n=!1);const r=Hn(e),o=js(e),i=Ms(o);let s=o==="x"?r===(n?"end":"start")?"right":"left":r==="start"?"bottom":"top";return t.reference[i]>t.floating[i]&&(s=zo(s)),[s,zo(s)]}function ab(e){const t=zo(e);return[$s(e),t,$s(t)]}function $s(e){return e.replace(/start|end/g,t=>ob[t])}const Dc=["left","right"],zc=["right","left"],lb=["top","bottom"],cb=["bottom","top"];function ub(e,t,n){switch(e){case"top":case"bottom":return n?t?zc:Dc:t?Dc:zc;case"left":case"right":return t?lb:cb;default:return[]}}function db(e,t,n,r){const o=Hn(e);let i=ub(St(e),n==="start",r);return o&&(i=i.map(s=>s+"-"+o),t&&(i=i.concat(i.map($s)))),i}function zo(e){return e.replace(/left|right|bottom|top/g,t=>rb[t])}function hb(e){return{top:0,right:0,bottom:0,left:0,...e}}function Mc(e){return typeof e!="number"?hb(e):{top:e,right:e,bottom:e,left:e}}function Mo(e){const{x:t,y:n,width:r,height:o}=e;return{width:r,height:o,top:n,left:t,right:t+r,bottom:n+o,x:t,y:n}}function jc(e,t,n){let{reference:r,floating:o}=e;const i=ut(t),s=js(t),a=Ms(s),l=St(t),c=i==="y",u=r.x+r.width/2-o.width/2,h=r.y+r.height/2-o.height/2,d=r[a]/2-o[a]/2;let m;switch(l){case"top":m={x:u,y:r.y-o.height};break;case"bottom":m={x:u,y:r.y+r.height};break;case"right":m={x:r.x+r.width,y:h};break;case"left":m={x:r.x-o.width,y:h};break;default:m={x:r.x,y:r.y}}switch(Hn(t)){case"start":m[s]-=d*(n&&c?-1:1);break;case"end":m[s]+=d*(n&&c?-1:1);break}return m}const fb=async(e,t,n)=>{const{placement:r="bottom",strategy:o="absolute",middleware:i=[],platform:s}=n,a=i.filter(Boolean),l=await(s.isRTL==null?void 0:s.isRTL(t));let c=await s.getElementRects({reference:e,floating:t,strategy:o}),{x:u,y:h}=jc(c,r,l),d=r,m={},f=0;for(let p=0;p({name:"arrow",options:e,async fn(t){const{x:n,y:r,placement:o,rects:i,platform:s,elements:a,middlewareData:l}=t,{element:c,padding:u=0}=wt(e,t)||{};if(c==null)return{};const h=Mc(u),d={x:n,y:r},m=js(o),f=Ms(m),p=await s.getDimensions(c),v=m==="y",b=v?"top":"left",y=v?"bottom":"right",x=v?"clientHeight":"clientWidth",w=i.reference[f]+i.reference[m]-d[m]-i.floating[f],k=d[m]-i.reference[m],O=await(s.getOffsetParent==null?void 0:s.getOffsetParent(c));let R=O?O[x]:0;(!R||!await(s.isElement==null?void 0:s.isElement(O)))&&(R=a.floating[x]||i.floating[f]);const I=w/2-k/2,N=R/2-p[f]/2-1,A=$t(h[b],N),T=$t(h[y],N),S=A,_=R-p[f]-T,D=R/2-p[f]/2+I,z=Ds(S,D,_),W=!l.arrow&&Hn(o)!=null&&D!==z&&i.reference[f]/2-(DD<=0)){var T,S;const D=(((T=i.flip)==null?void 0:T.index)||0)+1,z=R[D];if(z&&(!(h==="alignment"?y!==ut(z):!1)||A.every(ee=>ut(ee.placement)===y?ee.overflows[0]>0:!0)))return{data:{index:D,overflows:A},reset:{placement:z}};let W=(S=A.filter(J=>J.overflows[0]<=0).sort((J,ee)=>J.overflows[1]-ee.overflows[1])[0])==null?void 0:S.placement;if(!W)switch(m){case"bestFit":{var _;const J=(_=A.filter(ee=>{if(O){const $=ut(ee.placement);return $===y||$==="y"}return!0}).map(ee=>[ee.placement,ee.overflows.filter($=>$>0).reduce(($,q)=>$+q,0)]).sort((ee,$)=>ee[1]-$[1])[0])==null?void 0:_[0];J&&(W=J);break}case"initialPlacement":W=a;break}if(o!==W)return{reset:{placement:W}}}return{}}}};function $c(e,t){return{top:e.top-t.height,right:e.right-t.width,bottom:e.bottom-t.height,left:e.left-t.width}}function Bc(e){return nb.some(t=>e[t]>=0)}const mb=function(e){return e===void 0&&(e={}),{name:"hide",options:e,async fn(t){const{rects:n}=t,{strategy:r="referenceHidden",...o}=wt(e,t);switch(r){case"referenceHidden":{const i=await Pr(t,{...o,elementContext:"reference"}),s=$c(i,n.reference);return{data:{referenceHiddenOffsets:s,referenceHidden:Bc(s)}}}case"escaped":{const i=await Pr(t,{...o,altBoundary:!0}),s=$c(i,n.floating);return{data:{escapedOffsets:s,escaped:Bc(s)}}}default:return{}}}}},Wc=new Set(["left","top"]);async function vb(e,t){const{placement:n,platform:r,elements:o}=e,i=await(r.isRTL==null?void 0:r.isRTL(o.floating)),s=St(n),a=Hn(n),l=ut(n)==="y",c=Wc.has(s)?-1:1,u=i&&l?-1:1,h=wt(t,e);let{mainAxis:d,crossAxis:m,alignmentAxis:f}=typeof h=="number"?{mainAxis:h,crossAxis:0,alignmentAxis:null}:{mainAxis:h.mainAxis||0,crossAxis:h.crossAxis||0,alignmentAxis:h.alignmentAxis};return a&&typeof f=="number"&&(m=a==="end"?f*-1:f),l?{x:m*u,y:d*c}:{x:d*c,y:m*u}}const bb=function(e){return e===void 0&&(e=0),{name:"offset",options:e,async fn(t){var n,r;const{x:o,y:i,placement:s,middlewareData:a}=t,l=await vb(t,e);return s===((n=a.offset)==null?void 0:n.placement)&&(r=a.arrow)!=null&&r.alignmentOffset?{}:{x:o+l.x,y:i+l.y,data:{...l,placement:s}}}}},yb=function(e){return e===void 0&&(e={}),{name:"shift",options:e,async fn(t){const{x:n,y:r,placement:o}=t,{mainAxis:i=!0,crossAxis:s=!1,limiter:a={fn:v=>{let{x:b,y}=v;return{x:b,y}}},...l}=wt(e,t),c={x:n,y:r},u=await Pr(t,l),h=ut(St(o)),d=zs(h);let m=c[d],f=c[h];if(i){const v=d==="y"?"top":"left",b=d==="y"?"bottom":"right",y=m+u[v],x=m-u[b];m=Ds(y,m,x)}if(s){const v=h==="y"?"top":"left",b=h==="y"?"bottom":"right",y=f+u[v],x=f-u[b];f=Ds(y,f,x)}const p=a.fn({...t,[d]:m,[h]:f});return{...p,data:{x:p.x-n,y:p.y-r,enabled:{[d]:i,[h]:s}}}}}},xb=function(e){return e===void 0&&(e={}),{options:e,fn(t){const{x:n,y:r,placement:o,rects:i,middlewareData:s}=t,{offset:a=0,mainAxis:l=!0,crossAxis:c=!0}=wt(e,t),u={x:n,y:r},h=ut(o),d=zs(h);let m=u[d],f=u[h];const p=wt(a,t),v=typeof p=="number"?{mainAxis:p,crossAxis:0}:{mainAxis:0,crossAxis:0,...p};if(l){const x=d==="y"?"height":"width",w=i.reference[d]-i.floating[x]+v.mainAxis,k=i.reference[d]+i.reference[x]-v.mainAxis;mk&&(m=k)}if(c){var b,y;const x=d==="y"?"width":"height",w=Wc.has(St(o)),k=i.reference[h]-i.floating[x]+(w&&((b=s.offset)==null?void 0:b[h])||0)+(w?0:v.crossAxis),O=i.reference[h]+i.reference[x]+(w?0:((y=s.offset)==null?void 0:y[h])||0)-(w?v.crossAxis:0);fO&&(f=O)}return{[d]:m,[h]:f}}}},kb=function(e){return e===void 0&&(e={}),{name:"size",options:e,async fn(t){var n,r;const{placement:o,rects:i,platform:s,elements:a}=t,{apply:l=()=>{},...c}=wt(e,t),u=await Pr(t,c),h=St(o),d=Hn(o),m=ut(o)==="y",{width:f,height:p}=i.floating;let v,b;h==="top"||h==="bottom"?(v=h,b=d===(await(s.isRTL==null?void 0:s.isRTL(a.floating))?"start":"end")?"left":"right"):(b=h,v=d==="end"?"top":"bottom");const y=p-u.top-u.bottom,x=f-u.left-u.right,w=$t(p-u[v],y),k=$t(f-u[b],x),O=!t.middlewareData.shift;let R=w,I=k;if((n=t.middlewareData.shift)!=null&&n.enabled.x&&(I=x),(r=t.middlewareData.shift)!=null&&r.enabled.y&&(R=y),O&&!d){const A=Be(u.left,0),T=Be(u.right,0),S=Be(u.top,0),_=Be(u.bottom,0);m?I=f-2*(A!==0||T!==0?A+T:Be(u.left,u.right)):R=p-2*(S!==0||_!==0?S+_:Be(u.top,u.bottom))}await l({...t,availableWidth:I,availableHeight:R});const N=await s.getDimensions(a.floating);return f!==N.width||p!==N.height?{reset:{rects:!0}}:{}}}};function jo(){return typeof window<"u"}function Un(e){return Hc(e)?(e.nodeName||"").toLowerCase():"#document"}function We(e){var t;return(e==null||(t=e.ownerDocument)==null?void 0:t.defaultView)||window}function dt(e){var t;return(t=(Hc(e)?e.ownerDocument:e.document)||window.document)==null?void 0:t.documentElement}function Hc(e){return jo()?e instanceof Node||e instanceof We(e).Node:!1}function et(e){return jo()?e instanceof Element||e instanceof We(e).Element:!1}function ht(e){return jo()?e instanceof HTMLElement||e instanceof We(e).HTMLElement:!1}function Uc(e){return!jo()||typeof ShadowRoot>"u"?!1:e instanceof ShadowRoot||e instanceof We(e).ShadowRoot}const Cb=new Set(["inline","contents"]);function Rr(e){const{overflow:t,overflowX:n,overflowY:r,display:o}=tt(e);return/auto|scroll|overlay|hidden|clip/.test(t+r+n)&&!Cb.has(o)}const wb=new Set(["table","td","th"]);function Sb(e){return wb.has(Un(e))}const Eb=[":popover-open",":modal"];function $o(e){return Eb.some(t=>{try{return e.matches(t)}catch{return!1}})}const Ob=["transform","translate","scale","rotate","perspective"],Pb=["transform","translate","scale","rotate","perspective","filter"],Rb=["paint","layout","strict","content"];function Bs(e){const t=Ws(),n=et(e)?tt(e):e;return Ob.some(r=>n[r]?n[r]!=="none":!1)||(n.containerType?n.containerType!=="normal":!1)||!t&&(n.backdropFilter?n.backdropFilter!=="none":!1)||!t&&(n.filter?n.filter!=="none":!1)||Pb.some(r=>(n.willChange||"").includes(r))||Rb.some(r=>(n.contain||"").includes(r))}function Ib(e){let t=Bt(e);for(;ht(t)&&!Gn(t);){if(Bs(t))return t;if($o(t))return null;t=Bt(t)}return null}function Ws(){return typeof CSS>"u"||!CSS.supports?!1:CSS.supports("-webkit-backdrop-filter","none")}const Tb=new Set(["html","body","#document"]);function Gn(e){return Tb.has(Un(e))}function tt(e){return We(e).getComputedStyle(e)}function Bo(e){return et(e)?{scrollLeft:e.scrollLeft,scrollTop:e.scrollTop}:{scrollLeft:e.scrollX,scrollTop:e.scrollY}}function Bt(e){if(Un(e)==="html")return e;const t=e.assignedSlot||e.parentNode||Uc(e)&&e.host||dt(e);return Uc(t)?t.host:t}function Gc(e){const t=Bt(e);return Gn(t)?e.ownerDocument?e.ownerDocument.body:e.body:ht(t)&&Rr(t)?t:Gc(t)}function Ir(e,t,n){var r;t===void 0&&(t=[]),n===void 0&&(n=!0);const o=Gc(e),i=o===((r=e.ownerDocument)==null?void 0:r.body),s=We(o);if(i){const a=Hs(s);return t.concat(s,s.visualViewport||[],Rr(o)?o:[],a&&n?Ir(a):[])}return t.concat(o,Ir(o,[],n))}function Hs(e){return e.parent&&Object.getPrototypeOf(e.parent)?e.frameElement:null}function qc(e){const t=tt(e);let n=parseFloat(t.width)||0,r=parseFloat(t.height)||0;const o=ht(e),i=o?e.offsetWidth:n,s=o?e.offsetHeight:r,a=Fo(n)!==i||Fo(r)!==s;return a&&(n=i,r=s),{width:n,height:r,$:a}}function Us(e){return et(e)?e:e.contextElement}function qn(e){const t=Us(e);if(!ht(t))return ct(1);const n=t.getBoundingClientRect(),{width:r,height:o,$:i}=qc(t);let s=(i?Fo(n.width):n.width)/r,a=(i?Fo(n.height):n.height)/o;return(!s||!Number.isFinite(s))&&(s=1),(!a||!Number.isFinite(a))&&(a=1),{x:s,y:a}}const Nb=ct(0);function Kc(e){const t=We(e);return!Ws()||!t.visualViewport?Nb:{x:t.visualViewport.offsetLeft,y:t.visualViewport.offsetTop}}function Ab(e,t,n){return t===void 0&&(t=!1),!n||t&&n!==We(e)?!1:t}function tn(e,t,n,r){t===void 0&&(t=!1),n===void 0&&(n=!1);const o=e.getBoundingClientRect(),i=Us(e);let s=ct(1);t&&(r?et(r)&&(s=qn(r)):s=qn(e));const a=Ab(i,n,r)?Kc(i):ct(0);let l=(o.left+a.x)/s.x,c=(o.top+a.y)/s.y,u=o.width/s.x,h=o.height/s.y;if(i){const d=We(i),m=r&&et(r)?We(r):r;let f=d,p=Hs(f);for(;p&&r&&m!==f;){const v=qn(p),b=p.getBoundingClientRect(),y=tt(p),x=b.left+(p.clientLeft+parseFloat(y.paddingLeft))*v.x,w=b.top+(p.clientTop+parseFloat(y.paddingTop))*v.y;l*=v.x,c*=v.y,u*=v.x,h*=v.y,l+=x,c+=w,f=We(p),p=Hs(f)}}return Mo({width:u,height:h,x:l,y:c})}function Wo(e,t){const n=Bo(e).scrollLeft;return t?t.left+n:tn(dt(e)).left+n}function Yc(e,t){const n=e.getBoundingClientRect(),r=n.left+t.scrollLeft-Wo(e,n),o=n.top+t.scrollTop;return{x:r,y:o}}function _b(e){let{elements:t,rect:n,offsetParent:r,strategy:o}=e;const i=o==="fixed",s=dt(r),a=t?$o(t.floating):!1;if(r===s||a&&i)return n;let l={scrollLeft:0,scrollTop:0},c=ct(1);const u=ct(0),h=ht(r);if((h||!h&&!i)&&((Un(r)!=="body"||Rr(s))&&(l=Bo(r)),ht(r))){const m=tn(r);c=qn(r),u.x=m.x+r.clientLeft,u.y=m.y+r.clientTop}const d=s&&!h&&!i?Yc(s,l):ct(0);return{width:n.width*c.x,height:n.height*c.y,x:n.x*c.x-l.scrollLeft*c.x+u.x+d.x,y:n.y*c.y-l.scrollTop*c.y+u.y+d.y}}function Vb(e){return Array.from(e.getClientRects())}function Lb(e){const t=dt(e),n=Bo(e),r=e.ownerDocument.body,o=Be(t.scrollWidth,t.clientWidth,r.scrollWidth,r.clientWidth),i=Be(t.scrollHeight,t.clientHeight,r.scrollHeight,r.clientHeight);let s=-n.scrollLeft+Wo(e);const a=-n.scrollTop;return tt(r).direction==="rtl"&&(s+=Be(t.clientWidth,r.clientWidth)-o),{width:o,height:i,x:s,y:a}}const Xc=25;function Fb(e,t){const n=We(e),r=dt(e),o=n.visualViewport;let i=r.clientWidth,s=r.clientHeight,a=0,l=0;if(o){i=o.width,s=o.height;const u=Ws();(!u||u&&t==="fixed")&&(a=o.offsetLeft,l=o.offsetTop)}const c=Wo(r);if(c<=0){const u=r.ownerDocument,h=u.body,d=getComputedStyle(h),m=u.compatMode==="CSS1Compat"&&parseFloat(d.marginLeft)+parseFloat(d.marginRight)||0,f=Math.abs(r.clientWidth-h.clientWidth-m);f<=Xc&&(i-=f)}else c<=Xc&&(i+=c);return{width:i,height:s,x:a,y:l}}const Db=new Set(["absolute","fixed"]);function zb(e,t){const n=tn(e,!0,t==="fixed"),r=n.top+e.clientTop,o=n.left+e.clientLeft,i=ht(e)?qn(e):ct(1),s=e.clientWidth*i.x,a=e.clientHeight*i.y,l=o*i.x,c=r*i.y;return{width:s,height:a,x:l,y:c}}function Qc(e,t,n){let r;if(t==="viewport")r=Fb(e,n);else if(t==="document")r=Lb(dt(e));else if(et(t))r=zb(t,n);else{const o=Kc(e);r={x:t.x-o.x,y:t.y-o.y,width:t.width,height:t.height}}return Mo(r)}function Jc(e,t){const n=Bt(e);return n===t||!et(n)||Gn(n)?!1:tt(n).position==="fixed"||Jc(n,t)}function Mb(e,t){const n=t.get(e);if(n)return n;let r=Ir(e,[],!1).filter(a=>et(a)&&Un(a)!=="body"),o=null;const i=tt(e).position==="fixed";let s=i?Bt(e):e;for(;et(s)&&!Gn(s);){const a=tt(s),l=Bs(s);!l&&a.position==="fixed"&&(o=null),(i?!l&&!o:!l&&a.position==="static"&&!!o&&Db.has(o.position)||Rr(s)&&!l&&Jc(e,s))?r=r.filter(u=>u!==s):o=a,s=Bt(s)}return t.set(e,r),r}function jb(e){let{element:t,boundary:n,rootBoundary:r,strategy:o}=e;const s=[...n==="clippingAncestors"?$o(t)?[]:Mb(t,this._c):[].concat(n),r],a=s[0],l=s.reduce((c,u)=>{const h=Qc(t,u,o);return c.top=Be(h.top,c.top),c.right=$t(h.right,c.right),c.bottom=$t(h.bottom,c.bottom),c.left=Be(h.left,c.left),c},Qc(t,a,o));return{width:l.right-l.left,height:l.bottom-l.top,x:l.left,y:l.top}}function $b(e){const{width:t,height:n}=qc(e);return{width:t,height:n}}function Bb(e,t,n){const r=ht(t),o=dt(t),i=n==="fixed",s=tn(e,!0,i,t);let a={scrollLeft:0,scrollTop:0};const l=ct(0);function c(){l.x=Wo(o)}if(r||!r&&!i)if((Un(t)!=="body"||Rr(o))&&(a=Bo(t)),r){const m=tn(t,!0,i,t);l.x=m.x+t.clientLeft,l.y=m.y+t.clientTop}else o&&c();i&&!r&&o&&c();const u=o&&!r&&!i?Yc(o,a):ct(0),h=s.left+a.scrollLeft-l.x-u.x,d=s.top+a.scrollTop-l.y-u.y;return{x:h,y:d,width:s.width,height:s.height}}function Gs(e){return tt(e).position==="static"}function Zc(e,t){if(!ht(e)||tt(e).position==="fixed")return null;if(t)return t(e);let n=e.offsetParent;return dt(e)===n&&(n=n.ownerDocument.body),n}function eu(e,t){const n=We(e);if($o(e))return n;if(!ht(e)){let o=Bt(e);for(;o&&!Gn(o);){if(et(o)&&!Gs(o))return o;o=Bt(o)}return n}let r=Zc(e,t);for(;r&&Sb(r)&&Gs(r);)r=Zc(r,t);return r&&Gn(r)&&Gs(r)&&!Bs(r)?n:r||Ib(e)||n}const Wb=async function(e){const t=this.getOffsetParent||eu,n=this.getDimensions,r=await n(e.floating);return{reference:Bb(e.reference,await t(e.floating),e.strategy),floating:{x:0,y:0,width:r.width,height:r.height}}};function Hb(e){return tt(e).direction==="rtl"}const Ub={convertOffsetParentRelativeRectToViewportRelativeRect:_b,getDocumentElement:dt,getClippingRect:jb,getOffsetParent:eu,getElementRects:Wb,getClientRects:Vb,getDimensions:$b,getScale:qn,isElement:et,isRTL:Hb};function tu(e,t){return e.x===t.x&&e.y===t.y&&e.width===t.width&&e.height===t.height}function Gb(e,t){let n=null,r;const o=dt(e);function i(){var a;clearTimeout(r),(a=n)==null||a.disconnect(),n=null}function s(a,l){a===void 0&&(a=!1),l===void 0&&(l=1),i();const c=e.getBoundingClientRect(),{left:u,top:h,width:d,height:m}=c;if(a||t(),!d||!m)return;const f=Do(h),p=Do(o.clientWidth-(u+d)),v=Do(o.clientHeight-(h+m)),b=Do(u),x={rootMargin:-f+"px "+-p+"px "+-v+"px "+-b+"px",threshold:Be(0,$t(1,l))||1};let w=!0;function k(O){const R=O[0].intersectionRatio;if(R!==l){if(!w)return s();R?s(!1,R):r=setTimeout(()=>{s(!1,1e-7)},1e3)}R===1&&!tu(c,e.getBoundingClientRect())&&s(),w=!1}try{n=new IntersectionObserver(k,{...x,root:o.ownerDocument})}catch{n=new IntersectionObserver(k,x)}n.observe(e)}return s(!0),i}function qb(e,t,n,r){r===void 0&&(r={});const{ancestorScroll:o=!0,ancestorResize:i=!0,elementResize:s=typeof ResizeObserver=="function",layoutShift:a=typeof IntersectionObserver=="function",animationFrame:l=!1}=r,c=Us(e),u=o||i?[...c?Ir(c):[],...Ir(t)]:[];u.forEach(b=>{o&&b.addEventListener("scroll",n,{passive:!0}),i&&b.addEventListener("resize",n)});const h=c&&a?Gb(c,n):null;let d=-1,m=null;s&&(m=new ResizeObserver(b=>{let[y]=b;y&&y.target===c&&m&&(m.unobserve(t),cancelAnimationFrame(d),d=requestAnimationFrame(()=>{var x;(x=m)==null||x.observe(t)})),n()}),c&&!l&&m.observe(c),m.observe(t));let f,p=l?tn(e):null;l&&v();function v(){const b=tn(e);p&&!tu(p,b)&&n(),p=b,f=requestAnimationFrame(v)}return n(),()=>{var b;u.forEach(y=>{o&&y.removeEventListener("scroll",n),i&&y.removeEventListener("resize",n)}),h?.(),(b=m)==null||b.disconnect(),m=null,l&&cancelAnimationFrame(f)}}const Kb=bb,Yb=yb,Xb=pb,Qb=kb,Jb=mb,Zb=gb,ey=xb,ty=(e,t,n)=>{const r=new Map,o={platform:Ub,...n},i={...o.platform,_c:r};return fb(e,t,{...o,platform:i})};function nu(e=0,t=0,n=0,r=0){if(typeof DOMRect=="function")return new DOMRect(e,t,n,r);const o={x:e,y:t,width:n,height:r,top:t,right:e+n,bottom:t+r,left:e};return{...o,toJSON:()=>o}}function ny(e){if(!e)return nu();const{x:t,y:n,width:r,height:o}=e;return nu(t,n,r,o)}function ry(e,t){return{contextElement:Te(e)?e:void 0,getBoundingClientRect:()=>{const n=e,r=t?.(n);return r||!n?ny(r):n.getBoundingClientRect()}}}var ru=e=>({variable:e,reference:`var(${e})`}),ou={transformOrigin:ru("--transform-origin"),arrowOffset:ru("--arrow-offset")},oy=e=>e==="top"||e==="bottom"?"y":"x";function iy(e,t){return{name:"transformOrigin",fn(n){const{elements:r,middlewareData:o,placement:i,rects:s,y:a}=n,l=i.split("-")[0],c=oy(l),u=o.arrow?.x||0,h=o.arrow?.y||0,d=t?.clientWidth||0,m=t?.clientHeight||0,f=u+d/2,p=h+m/2,v=Math.abs(o.shift?.y||0),b=s.reference.height/2,y=m/2,x=e.offset?.mainAxis??e.gutter,w=typeof x=="number"?x+y:x??y,k=v>w,O={top:`${f}px calc(100% + ${w}px)`,bottom:`${f}px ${-w}px`,left:`calc(100% + ${w}px) ${p}px`,right:`${-w}px ${p}px`}[l],R=`${f}px ${s.reference.y+b-a}px`,I=!!e.overlap&&c==="y"&&k;return r.floating.style.setProperty(ou.transformOrigin.variable,I?R:O),{data:{transformOrigin:I?R:O}}}}}var sy={name:"rects",fn({rects:e}){return{data:e}}},ay=e=>{if(e)return{name:"shiftArrow",fn({placement:t,middlewareData:n}){if(!n.arrow)return{};const{x:r,y:o}=n.arrow,i=t.split("-")[0];return Object.assign(e.style,{left:r!=null?`${r}px`:"",top:o!=null?`${o}px`:"",[i]:`calc(100% + ${ou.arrowOffset.reference})`}),{}}}};function ly(e){const[t,n]=e.split("-");return{side:t,align:n,hasAlign:n!=null}}function cy(e){return e.split("-")[0]}var uy={strategy:"absolute",placement:"bottom",listeners:!0,gutter:8,flip:!0,slide:!0,overlap:!1,sameWidth:!1,fitViewport:!1,overflowPadding:8,arrowPadding:4};function iu(e,t){const n=e.devicePixelRatio||1;return Math.round(t*n)/n}function qs(e){return typeof e=="function"?e():e==="clipping-ancestors"?"clippingAncestors":e}function dy(e,t,n){const r=e||t.createElement("div");return Zb({element:r,padding:n.arrowPadding})}function hy(e,t){if(!Qm(t.offset??t.gutter))return Kb(({placement:n})=>{const r=(e?.clientHeight||0)/2,o=t.offset?.mainAxis??t.gutter,i=typeof o=="number"?o+r:o??r,{hasAlign:s}=ly(n),a=s?void 0:t.shift,l=t.offset?.crossAxis??a;return bo({crossAxis:l,mainAxis:i,alignmentAxis:t.shift})})}function fy(e){if(!e.flip)return;const t=qs(e.boundary);return Xb({...t?{boundary:t}:void 0,padding:e.overflowPadding,fallbackPlacements:e.flip===!0?void 0:e.flip})}function gy(e){if(!e.slide&&!e.overlap)return;const t=qs(e.boundary);return Yb({...t?{boundary:t}:void 0,mainAxis:e.slide,crossAxis:e.overlap,padding:e.overflowPadding,limiter:ey()})}function py(e){return Qb({padding:e.overflowPadding,apply({elements:t,rects:n,availableHeight:r,availableWidth:o}){const i=t.floating,s=Math.round(n.reference.width),a=Math.round(n.reference.height);o=Math.floor(o),r=Math.floor(r),i.style.setProperty("--reference-width",`${s}px`),i.style.setProperty("--reference-height",`${a}px`),i.style.setProperty("--available-width",`${o}px`),i.style.setProperty("--available-height",`${r}px`)}})}function my(e){if(e.hideWhenDetached)return Jb({strategy:"referenceHidden",boundary:qs(e.boundary)??"clippingAncestors"})}function vy(e){return e?e===!0?{ancestorResize:!0,ancestorScroll:!0,elementResize:!0,layoutShift:!0}:e:{}}function by(e,t,n={}){const r=ry(e,n.getAnchorRect);if(!t||!r)return;const o=Object.assign({},uy,n),i=t.querySelector("[data-part=arrow]"),s=[hy(i,o),fy(o),gy(o),dy(i,t.ownerDocument,o),ay(i),iy({gutter:o.gutter,offset:o.offset,overlap:o.overlap},i),py(o),my(o),sy],{placement:a,strategy:l,onComplete:c,onPositioned:u}=o,h=async()=>{if(!r||!t)return;const p=await ty(r,t,{placement:a,middleware:s,strategy:l});c?.(p),u?.({placed:!0});const v=be(t),b=iu(v,p.x),y=iu(v,p.y);t.style.setProperty("--x",`${b}px`),t.style.setProperty("--y",`${y}px`),o.hideWhenDetached&&(p.middlewareData.hide?.referenceHidden?(t.style.setProperty("visibility","hidden"),t.style.setProperty("pointer-events","none")):(t.style.removeProperty("visibility"),t.style.removeProperty("pointer-events")));const x=t.firstElementChild;if(x){const w=xo(x);t.style.setProperty("--z-index",w.zIndex)}},d=async()=>{n.updatePosition?(await n.updatePosition({updatePosition:h,floatingElement:t}),u?.({placed:!0})):await h()},m=vy(o.listeners),f=o.listeners?qb(r,t,d,m):i0;return d(),()=>{f?.(),u?.({placed:!1})}}function nt(e,t,n={}){const{defer:r,...o}=n,i=r?G:a=>a(),s=[];return s.push(i(()=>{const a=typeof e=="function"?e():e,l=typeof t=="function"?t():t;s.push(by(a,l,o))})),()=>{s.forEach(a=>a?.())}}function yy(e){const t={each(n){for(let r=0;r{try{i.document.addEventListener(n,r,o)}catch{}}),()=>{try{t.removeEventListener(n,r,o)}catch{}}},removeEventListener(n,r,o){t.each(i=>{try{i.document.removeEventListener(n,r,o)}catch{}})}};return t}function xy(e){const t=e.frameElement!=null?e.parent:null;return{addEventListener:(n,r,o)=>{try{t?.addEventListener(n,r,o)}catch{}return()=>{try{t?.removeEventListener(n,r,o)}catch{}}},removeEventListener:(n,r,o)=>{try{t?.removeEventListener(n,r,o)}catch{}}}}var su="pointerdown.outside",au="focus.outside";function ky(e){for(const t of e)if(Te(t)&&Mt(t))return!0;return!1}var lu=e=>"clientY"in e;function Cy(e,t){if(!lu(t)||!e)return!1;const n=e.getBoundingClientRect();return n.width===0||n.height===0?!1:n.top<=t.clientY&&t.clientY<=n.top+n.height&&n.left<=t.clientX&&t.clientX<=n.left+n.width}function wy(e,t){return e.y<=t.y&&t.y<=e.y+e.height&&e.x<=t.x&&t.x<=e.x+e.width}function cu(e,t){if(!t||!lu(e))return!1;const n=t.scrollHeight>t.clientHeight,r=n&&e.clientX>t.offsetLeft+t.clientWidth,o=t.scrollWidth>t.clientWidth,i=o&&e.clientY>t.offsetTop+t.clientHeight,s={x:t.offsetLeft,y:t.offsetTop,width:t.clientWidth+(n?16:0),height:t.clientHeight+(o?16:0)},a={x:e.clientX,y:e.clientY};return wy(s,a)?r||i:!1}function Sy(e,t){const{exclude:n,onFocusOutside:r,onPointerDownOutside:o,onInteractOutside:i,defer:s}=t;if(!e)return;const a=Le(e),l=be(e),c=yy(l),u=xy(l);function h(y,x){if(!Te(x)||!x.isConnected||Jt(e,x)||Cy(e,y))return!1;const w=a.querySelector(`[aria-controls="${e.id}"]`);if(w){const O=Oo(w);if(cu(y,O))return!1}const k=Oo(e);return cu(y,k)?!1:!n?.(x)}const d=new Set,m=jn(e?.getRootNode());function f(y){function x(w){const k=s&&!hc()?G:I=>I(),O=w??y,R=O?.composedPath?.()??[O?.target];k(()=>{const I=m?R[0]:$e(y);if(!(!e||!h(y,I))){if(o||i){const N=mo(o,i);e.addEventListener(su,N,{once:!0})}uu(e,su,{bubbles:!1,cancelable:!0,detail:{originalEvent:O,contextmenu:Y0(O),focusable:ky(R),target:I}})}})}y.pointerType==="touch"?(d.forEach(w=>w()),d.add(se(a,"click",x,{once:!0})),d.add(u.addEventListener("click",x,{once:!0})),d.add(c.addEventListener("click",x,{once:!0}))):x()}const p=new Set,v=setTimeout(()=>{p.add(se(a,"pointerdown",f,!0)),p.add(u.addEventListener("pointerdown",f,!0)),p.add(c.addEventListener("pointerdown",f,!0))},0);function b(y){(s?G:w=>w())(()=>{const w=$e(y);if(!(!e||!h(y,w))){if(r||i){const k=mo(r,i);e.addEventListener(au,k,{once:!0})}uu(e,au,{bubbles:!1,cancelable:!0,detail:{originalEvent:y,contextmenu:!1,focusable:Mt(w),target:w}})}})}return hc()||(p.add(se(a,"focusin",b,!0)),p.add(u.addEventListener("focusin",b,!0)),p.add(c.addEventListener("focusin",b,!0))),()=>{clearTimeout(v),d.forEach(y=>y()),p.forEach(y=>y())}}function Ey(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=typeof e=="function"?e():e;o.push(Sy(i,t))})),()=>{o.forEach(i=>i?.())}}function uu(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,n);return e.dispatchEvent(o)}function Oy(e,t){const n=r=>{r.key==="Escape"&&(r.isComposing||t?.(r))};return se(Le(e),"keydown",n,{capture:!0})}var du="layer:request-dismiss",Ke={layers:[],branches:[],count(){return this.layers.length},pointerBlockingLayers(){return this.layers.filter(e=>e.pointerBlocking)},topMostPointerBlockingLayer(){return[...this.pointerBlockingLayers()].slice(-1)[0]},hasPointerBlockingLayer(){return this.pointerBlockingLayers().length>0},isBelowPointerBlockingLayer(e){const t=this.indexOf(e),n=this.topMostPointerBlockingLayer()?this.indexOf(this.topMostPointerBlockingLayer()?.node):-1;return tt.type===e)},getNestedLayersByType(e,t){const n=this.indexOf(e);return n===-1?[]:this.layers.slice(n+1).filter(r=>r.type===t)},getParentLayerOfType(e,t){const n=this.indexOf(e);if(!(n<=0))return this.layers.slice(0,n).reverse().find(r=>r.type===t)},countNestedLayersOfType(e,t){return this.getNestedLayersByType(e,t).length},isInNestedLayer(e,t){return this.getNestedLayers(e).some(n=>Jt(n.node,t))},isInBranch(e){return Array.from(this.branches).some(t=>Jt(t,e))},add(e){this.layers.push(e),this.syncLayers()},addBranch(e){this.branches.push(e)},remove(e){const t=this.indexOf(e);t<0||(tKe.dismiss(r.node,e)),this.layers.splice(t,1),this.syncLayers())},removeBranch(e){const t=this.branches.indexOf(e);t>=0&&this.branches.splice(t,1)},syncLayers(){this.layers.forEach((e,t)=>{e.node.style.setProperty("--layer-index",`${t}`),e.node.removeAttribute("data-nested"),e.node.removeAttribute("data-has-nested"),this.getParentLayerOfType(e.node,e.type)&&e.node.setAttribute("data-nested",e.type);const r=this.countNestedLayersOfType(e.node,e.type);r>0&&e.node.setAttribute("data-has-nested",e.type),e.node.style.setProperty("--nested-layer-count",`${r}`)})},indexOf(e){return this.layers.findIndex(t=>t.node===e)},dismiss(e,t){const n=this.indexOf(e);if(n===-1)return;const r=this.layers[n];Ry(e,du,o=>{r.requestDismiss?.(o),o.defaultPrevented||r?.dismiss()}),Py(e,du,{originalLayer:e,targetLayer:t,originalIndex:n,targetIndex:t?this.indexOf(t):-1}),this.syncLayers()},clear(){this.remove(this.layers[0].node)}};function Py(e,t,n){const r=e.ownerDocument.defaultView||window,o=new r.CustomEvent(t,{cancelable:!0,bubbles:!0,detail:n});return e.dispatchEvent(o)}function Ry(e,t,n){e.addEventListener(t,n,{once:!0})}var hu;function fu(){Ke.layers.forEach(({node:e})=>{e.style.pointerEvents=Ke.isBelowPointerBlockingLayer(e)?"none":"auto"})}function Iy(e){e.style.pointerEvents=""}function Ty(e,t){const n=Le(e),r=[];return Ke.hasPointerBlockingLayer()&&!n.body.hasAttribute("data-inert")&&(hu=document.body.style.pointerEvents,queueMicrotask(()=>{n.body.style.pointerEvents="none",n.body.setAttribute("data-inert","")})),t?.forEach(o=>{const[i,s]=Rv(()=>{const a=o();return Te(a)?a:null},{timeout:1e3});i.then(a=>r.push(To(a,{pointerEvents:"auto"}))),r.push(s)}),()=>{Ke.hasPointerBlockingLayer()||(queueMicrotask(()=>{n.body.style.pointerEvents=hu,n.body.removeAttribute("data-inert"),n.body.style.length===0&&n.body.removeAttribute("style")}),r.forEach(o=>o()))}}function Ny(e,t){const{warnOnMissingNode:n=!0}=t;if(n&&!e){wr("[@zag-js/dismissable] node is `null` or `undefined`");return}if(!e)return;const{onDismiss:r,onRequestDismiss:o,pointerBlocking:i,exclude:s,debug:a,type:l="dialog"}=t,c={dismiss:r,node:e,type:l,pointerBlocking:i,requestDismiss:o};Ke.add(c),fu();function u(p){const v=$e(p.detail.originalEvent);Ke.isBelowPointerBlockingLayer(e)||Ke.isInBranch(v)||(t.onPointerDownOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onPointerDownOutside:",p.detail.originalEvent),r?.()))}function h(p){const v=$e(p.detail.originalEvent);Ke.isInBranch(v)||(t.onFocusOutside?.(p),t.onInteractOutside?.(p),!p.defaultPrevented&&(a&&console.log("onFocusOutside:",p.detail.originalEvent),r?.()))}function d(p){Ke.isTopMost(e)&&(t.onEscapeKeyDown?.(p),!p.defaultPrevented&&r&&(p.preventDefault(),r()))}function m(p){if(!e)return!1;const v=typeof s=="function"?s():s,b=Array.isArray(v)?v:[v],y=t.persistentElements?.map(x=>x()).filter(Te);return y&&b.push(...y),b.some(x=>Jt(x,p))||Ke.isInNestedLayer(e,p)}const f=[i?Ty(e,t.persistentElements):void 0,Oy(e,d),Ey(e,{exclude:m,onFocusOutside:h,onPointerDownOutside:u,defer:t.defer})];return()=>{Ke.remove(e),fu(),Iy(e),f.forEach(p=>p?.())}}function Kn(e,t){const{defer:n}=t,r=n?G:i=>i(),o=[];return o.push(r(()=>{const i=Qt(e)?e():e;o.push(Ny(i,t))})),()=>{o.forEach(i=>i?.())}}var gu=j("color-picker",["root","label","control","trigger","positioner","content","area","areaThumb","valueText","areaBackground","channelSlider","channelSliderLabel","channelSliderTrack","channelSliderThumb","channelSliderValueText","channelInput","transparencyGrid","swatchGroup","swatchTrigger","swatchIndicator","swatch","eyeDropperTrigger","formatTrigger","formatSelect"]);gu.build();var Ay=e=>e.ids?.hiddenInput??`color-picker:${e.id}:hidden-input`,_y=e=>e.ids?.control??`color-picker:${e.id}:control`,Vy=e=>e.ids?.trigger??`color-picker:${e.id}:trigger`,Ly=e=>e.ids?.content??`color-picker:${e.id}:content`,Fy=e=>e.ids?.positioner??`color-picker:${e.id}:positioner`,Dy=e=>e.ids?.formatSelect??`color-picker:${e.id}:format-select`,zy=e=>e.ids?.area??`color-picker:${e.id}:area`,My=e=>e.ids?.areaThumb??`color-picker:${e.id}:area-thumb`,jy=(e,t)=>e.ids?.channelSliderTrack?.(t)??`color-picker:${e.id}:slider-track:${t}`,$y=(e,t)=>e.ids?.channelSliderThumb?.(t)??`color-picker:${e.id}:slider-thumb:${t}`,Ho=e=>e.getById(Ly(e)),By=e=>e.getById(My(e)),Wy=(e,t)=>e.getById($y(e,t)),Hy=e=>e.getById(Dy(e)),pu=e=>e.getById(Ay(e)),Uy=e=>e.getById(zy(e)),Gy=(e,t,n)=>{const r=Uy(e);if(!r)return;const{getPercentValue:o}=yc(t,r);return{x:o({dir:n,orientation:"horizontal"}),y:o({orientation:"vertical"})}},qy=e=>e.getById(_y(e)),Ks=e=>e.getById(Vy(e)),Ky=e=>e.getById(Fy(e)),Yy=(e,t)=>e.getById(jy(e,t)),Xy=(e,t,n,r)=>{const o=Yy(e,n);if(!o)return;const{getPercentValue:i}=yc(t,o);return{x:i({dir:r,orientation:"horizontal"}),y:i({orientation:"vertical"})}},Qy=e=>[...Io(Ho(e),"input[data-channel]"),...Io(qy(e),"input[data-channel]")];function Jy(e,t){if(t==null)return"";if(t==="hex")return e.toString("hex");if(t==="css")return e.toString("css");if(t in e)return e.getChannelValue(t).toString();const n=e.getFormat()==="hsla";switch(t){case"hue":return n?e.toFormat("hsla").getChannelValue("hue").toString():e.toFormat("hsba").getChannelValue("hue").toString();case"saturation":return n?e.toFormat("hsla").getChannelValue("saturation").toString():e.toFormat("hsba").getChannelValue("saturation").toString();case"lightness":return e.toFormat("hsla").getChannelValue("lightness").toString();case"brightness":return e.toFormat("hsba").getChannelValue("brightness").toString();case"red":case"green":case"blue":return e.toFormat("rgba").getChannelValue(t).toString();default:return e.getChannelValue(t).toString()}}var mu=e=>Lo(e),Zy=/^[0-9a-fA-F]{3,8}$/;function ex(e){return Zy.test(e)}function tx(e){return e.startsWith("#")?e:ex(e)?`#${e}`:e}var{and:nx}=jt();nx("isOpenControlled","closeOnSelect");function vu(e,t,n){const r=Qy(e);G(()=>{r.forEach(o=>{const i=o.dataset.channel;So(o,Jy(n||t,i))})})}function rx(e,t){const n=Hy(e);n&&G(()=>So(n,t))}M()(["closeOnSelect","dir","disabled","format","defaultFormat","getRootNode","id","ids","initialFocusEl","inline","name","positioning","onFocusOutside","onFormatChange","onInteractOutside","onOpenChange","onPointerDownOutside","onValueChange","onValueChangeEnd","defaultOpen","open","positioning","required","readOnly","value","defaultValue","invalid","openAutoFocus"]),M()(["xChannel","yChannel"]),M()(["channel","orientation"]),M()(["value","disabled"]),M()(["value","respectAlpha"]),M()(["size"]);const[bu,ox]=Mn({name:"RenderStrategyContext",hookName:"useRenderStrategyContext",providerName:""}),yu=e=>Sc()(e,["lazyMount","unmountOnExit"]);var xu=j("accordion").parts("root","item","itemTrigger","itemContent","itemIndicator");xu.build();var ku=e=>e.ids?.root??`accordion:${e.id}`,Cu=(e,t)=>e.ids?.itemTrigger?.(t)??`accordion:${e.id}:trigger:${t}`,ix=e=>e.getById(ku(e)),Uo=e=>{const n=`[aria-controls][data-ownedby='${CSS.escape(ku(e))}']:not([disabled])`;return Io(ix(e),n)},sx=e=>kr(Uo(e)),ax=e=>hs(Uo(e)),lx=(e,t)=>bv(Uo(e),Cu(e,t)),cx=(e,t)=>yv(Uo(e),Cu(e,t)),{and:ux,not:dx}=jt();ux("isExpanded","canToggle"),dx("isExpanded"),M()(["collapsible","dir","disabled","getRootNode","id","ids","multiple","onFocusChange","onValueChange","orientation","value","defaultValue"]),M()(["value","disabled"]);var Tr=(e,t)=>({x:e,y:t});function hx(e){const{x:t,y:n,width:r,height:o}=e,i=t+r/2,s=n+o/2;return{x:t,y:n,width:r,height:o,minX:t,minY:n,maxX:t+r,maxY:n+o,midX:i,midY:s,center:Tr(i,s)}}function fx(e){const t=Tr(e.minX,e.minY),n=Tr(e.maxX,e.minY),r=Tr(e.maxX,e.maxY),o=Tr(e.minX,e.maxY);return{top:t,right:n,bottom:r,left:o}}function gx(e,t){const n=hx(e),{top:r,right:o,left:i,bottom:s}=fx(n),[a]=t.split("-");return{top:[i,r,o,s],right:[r,o,s,i],bottom:[r,i,s,o],left:[o,r,i,s]}[a]}function px(e,t){const{x:n,y:r}=t;let o=!1;for(let i=0,s=e.length-1;ir!=u>r&&n<(c-a)*(r-l)/(u-l)+a&&(o=!o)}return o}var wu=j("avatar").parts("root","image","fallback");wu.build(),M()(["dir","id","ids","onStatusChange","getRootNode"]);var mx=Object.defineProperty,vx=(e,t,n)=>t in e?mx(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,ae=(e,t,n)=>vx(e,typeof t!="symbol"?t+"":t,n),Su={activateTrap(e,t){if(e.length>0){const r=e[e.length-1];r!==t&&r.pause()}const n=e.indexOf(t);n===-1||e.splice(n,1),e.push(t)},deactivateTrap(e,t){const n=e.indexOf(t);n!==-1&&e.splice(n,1),e.length>0&&e[e.length-1].unpause()}},bx=[],yx=class{constructor(e,t){ae(this,"trapStack"),ae(this,"config"),ae(this,"doc"),ae(this,"state",{containers:[],containerGroups:[],tabbableGroups:[],nodeFocusedBeforeActivation:null,mostRecentlyFocusedNode:null,active:!1,paused:!1,delayInitialFocusTimer:void 0,recentNavEvent:void 0}),ae(this,"listenerCleanups",[]),ae(this,"handleFocus",r=>{const o=$e(r),i=this.findContainerIndex(o,r)>=0;if(i||ms(o))i&&(this.state.mostRecentlyFocusedNode=o);else{r.stopImmediatePropagation();let s,a=!0;if(this.state.mostRecentlyFocusedNode)if(Er(this.state.mostRecentlyFocusedNode)>0){const l=this.findContainerIndex(this.state.mostRecentlyFocusedNode),{tabbableNodes:c}=this.state.containerGroups[l];if(c.length>0){const u=c.findIndex(h=>h===this.state.mostRecentlyFocusedNode);u>=0&&(this.config.isKeyForward(this.state.recentNavEvent)?u+1=0&&(s=c[u-1],a=!1))}}else this.state.containerGroups.some(l=>l.tabbableNodes.some(c=>Er(c)>0))||(a=!1);else a=!1;a&&(s=this.findNextNavNode({target:this.state.mostRecentlyFocusedNode,isBackward:this.config.isKeyBackward(this.state.recentNavEvent)})),s?this.tryFocus(s):this.tryFocus(this.state.mostRecentlyFocusedNode||this.getInitialFocusNode())}this.state.recentNavEvent=void 0}),ae(this,"handlePointerDown",r=>{const o=$e(r);if(!(this.findContainerIndex(o,r)>=0)){if(Ar(this.config.clickOutsideDeactivates,r)){this.deactivate({returnFocus:this.config.returnFocusOnDeactivate});return}Ar(this.config.allowOutsideClick,r)||r.preventDefault()}}),ae(this,"handleClick",r=>{const o=$e(r);this.findContainerIndex(o,r)>=0||Ar(this.config.clickOutsideDeactivates,r)||Ar(this.config.allowOutsideClick,r)||(r.preventDefault(),r.stopImmediatePropagation())}),ae(this,"handleTabKey",r=>{if(this.config.isKeyForward(r)||this.config.isKeyBackward(r)){this.state.recentNavEvent=r;const o=this.config.isKeyBackward(r),i=this.findNextNavNode({event:r,isBackward:o});if(!i)return;Nr(r)&&r.preventDefault(),this.tryFocus(i)}}),ae(this,"handleEscapeKey",r=>{xx(r)&&Ar(this.config.escapeDeactivates,r)!==!1&&(r.preventDefault(),this.deactivate())}),ae(this,"_mutationObserver"),ae(this,"setupMutationObserver",()=>{const r=this.doc.defaultView||window;this._mutationObserver=new r.MutationObserver(o=>{o.some(s=>Array.from(s.removedNodes).some(l=>l===this.state.mostRecentlyFocusedNode))&&this.tryFocus(this.getInitialFocusNode())})}),ae(this,"updateObservedNodes",()=>{this._mutationObserver?.disconnect(),this.state.active&&!this.state.paused&&this.state.containers.map(r=>{this._mutationObserver?.observe(r,{subtree:!0,childList:!0})})}),ae(this,"getInitialFocusNode",()=>{let r=this.getNodeForOption("initialFocus",{hasFallback:!0});if(r===!1)return!1;if(r===void 0||r&&!Mt(r)){const o=Sr(this.doc);if(o&&this.findContainerIndex(o)>=0)r=o;else{const i=this.state.tabbableGroups[0];r=i&&i.firstTabbableNode||this.getNodeForOption("fallbackFocus")}}else r===null&&(r=this.getNodeForOption("fallbackFocus"));if(!r)throw new Error("Your focus-trap needs to have at least one focusable element");return r.isConnected||(r=this.getNodeForOption("fallbackFocus")),r}),ae(this,"tryFocus",r=>{if(r!==!1&&r!==Sr(this.doc)){if(!r||!r.focus){this.tryFocus(this.getInitialFocusNode());return}r.focus({preventScroll:!!this.config.preventScroll}),this.state.mostRecentlyFocusedNode=r,kx(r)&&r.select()}}),ae(this,"deactivate",r=>{if(!this.state.active)return this;const o={onDeactivate:this.config.onDeactivate,onPostDeactivate:this.config.onPostDeactivate,checkCanReturnFocus:this.config.checkCanReturnFocus,...r};clearTimeout(this.state.delayInitialFocusTimer),this.state.delayInitialFocusTimer=void 0,this.removeListeners(),this.state.active=!1,this.state.paused=!1,this.updateObservedNodes(),Su.deactivateTrap(this.trapStack,this);const i=this.getOption(o,"onDeactivate"),s=this.getOption(o,"onPostDeactivate"),a=this.getOption(o,"checkCanReturnFocus"),l=this.getOption(o,"returnFocus","returnFocusOnDeactivate");i?.();const c=()=>{Eu(()=>{if(l){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);this.tryFocus(u)}s?.()})};if(l&&a){const u=this.getReturnFocusNode(this.state.nodeFocusedBeforeActivation);return a(u).then(c,c),this}return c(),this}),ae(this,"pause",r=>{if(this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onPause"),i=this.getOption(r,"onPostPause");return this.state.paused=!0,o?.(),this.removeListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"unpause",r=>{if(!this.state.paused||!this.state.active)return this;const o=this.getOption(r,"onUnpause"),i=this.getOption(r,"onPostUnpause");return this.state.paused=!1,o?.(),this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),i?.(),this}),ae(this,"updateContainerElements",r=>(this.state.containers=Array.isArray(r)?r.filter(Boolean):[r].filter(Boolean),this.state.active&&this.updateTabbableNodes(),this.updateObservedNodes(),this)),ae(this,"getReturnFocusNode",r=>{const o=this.getNodeForOption("setReturnFocus",{params:[r]});return o||(o===!1?!1:r)}),ae(this,"getOption",(r,o,i)=>r&&r[o]!==void 0?r[o]:this.config[i||o]),ae(this,"getNodeForOption",(r,{hasFallback:o=!1,params:i=[]}={})=>{let s=this.config[r];if(typeof s=="function"&&(s=s(...i)),s===!0&&(s=void 0),!s){if(s===void 0||s===!1)return s;throw new Error(`\`${r}\` was specified but was not a node, or did not return a node`)}let a=s;if(typeof s=="string"){try{a=this.doc.querySelector(s)}catch(l){throw new Error(`\`${r}\` appears to be an invalid selector; error="${l.message}"`)}if(!a&&!o)throw new Error(`\`${r}\` as selector refers to no known node`)}return a}),ae(this,"findNextNavNode",r=>{const{event:o,isBackward:i=!1}=r,s=r.target||$e(o);this.updateTabbableNodes();let a=null;if(this.state.tabbableGroups.length>0){const l=this.findContainerIndex(s,o),c=l>=0?this.state.containerGroups[l]:void 0;if(l<0)i?a=this.state.tabbableGroups[this.state.tabbableGroups.length-1].lastTabbableNode:a=this.state.tabbableGroups[0].firstTabbableNode;else if(i){let u=this.state.tabbableGroups.findIndex(({firstTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s,!1))&&(u=l),u>=0){const h=u===0?this.state.tabbableGroups.length-1:u-1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.lastTabbableNode:d.lastDomTabbableNode}else Nr(o)||(a=c?.nextTabbableNode(s,!1))}else{let u=this.state.tabbableGroups.findIndex(({lastTabbableNode:h})=>s===h);if(u<0&&(c?.container===s||Mt(s)&&!Zt(s)&&!c?.nextTabbableNode(s))&&(u=l),u>=0){const h=u===this.state.tabbableGroups.length-1?0:u+1,d=this.state.tabbableGroups[h];a=Er(s)>=0?d.firstTabbableNode:d.firstDomTabbableNode}else Nr(o)||(a=c?.nextTabbableNode(s))}}else a=this.getNodeForOption("fallbackFocus");return a}),this.trapStack=t.trapStack||bx;const n={returnFocusOnDeactivate:!0,escapeDeactivates:!0,delayInitialFocus:!0,isKeyForward(r){return Nr(r)&&!r.shiftKey},isKeyBackward(r){return Nr(r)&&r.shiftKey},...t};this.doc=n.document||Le(Array.isArray(e)?e[0]:e),this.config=n,this.updateContainerElements(e),this.setupMutationObserver()}get active(){return this.state.active}get paused(){return this.state.paused}findContainerIndex(e,t){const n=typeof t?.composedPath=="function"?t.composedPath():void 0;return this.state.containerGroups.findIndex(({container:r,tabbableNodes:o})=>r.contains(e)||n?.includes(r)||o.find(i=>i===e))}updateTabbableNodes(){if(this.state.containerGroups=this.state.containers.map(e=>{const t=ks(e),n=mc(e),r=t.length>0?t[0]:void 0,o=t.length>0?t[t.length-1]:void 0,i=n.find(c=>Zt(c)),s=n.slice().reverse().find(c=>Zt(c)),a=!!t.find(c=>Er(c)>0);function l(c,u=!0){const h=t.indexOf(c);return h<0?u?n.slice(n.indexOf(c)+1).find(d=>Zt(d)):n.slice(0,n.indexOf(c)).reverse().find(d=>Zt(d)):t[h+(u?1:-1)]}return{container:e,tabbableNodes:t,focusableNodes:n,posTabIndexesFound:a,firstTabbableNode:r,lastTabbableNode:o,firstDomTabbableNode:i,lastDomTabbableNode:s,nextTabbableNode:l}}),this.state.tabbableGroups=this.state.containerGroups.filter(e=>e.tabbableNodes.length>0),this.state.tabbableGroups.length<=0&&!this.getNodeForOption("fallbackFocus"))throw new Error("Your focus-trap must have at least one container with at least one tabbable node in it at all times");if(this.state.containerGroups.find(e=>e.posTabIndexesFound)&&this.state.containerGroups.length>1)throw new Error("At least one node with a positive tabindex was found in one of your focus-trap's multiple containers. Positive tabindexes are only supported in single-container focus-traps.")}addListeners(){if(this.state.active)return Su.activateTrap(this.trapStack,this),this.state.delayInitialFocusTimer=this.config.delayInitialFocus?Eu(()=>{this.tryFocus(this.getInitialFocusNode())}):this.tryFocus(this.getInitialFocusNode()),this.listenerCleanups.push(se(this.doc,"focusin",this.handleFocus,!0),se(this.doc,"mousedown",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"touchstart",this.handlePointerDown,{capture:!0,passive:!1}),se(this.doc,"click",this.handleClick,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleTabKey,{capture:!0,passive:!1}),se(this.doc,"keydown",this.handleEscapeKey)),this}removeListeners(){if(this.state.active)return this.listenerCleanups.forEach(e=>e()),this.listenerCleanups=[],this}activate(e){if(this.state.active)return this;const t=this.getOption(e,"onActivate"),n=this.getOption(e,"onPostActivate"),r=this.getOption(e,"checkCanFocusTrap");r||this.updateTabbableNodes(),this.state.active=!0,this.state.paused=!1,this.state.nodeFocusedBeforeActivation=Sr(this.doc),t?.();const o=()=>{r&&this.updateTabbableNodes(),this.addListeners(),this.updateObservedNodes(),n?.()};return r?(r(this.state.containers.concat()).then(o,o),this):(o(),this)}},Nr=e=>e.key==="Tab",Ar=(e,...t)=>typeof e=="function"?e(...t):e,xx=e=>!e.isComposing&&e.key==="Escape",Eu=e=>setTimeout(e,0),kx=e=>e.localName==="input"&&"select"in e&&typeof e.select=="function";function Cx(e,t={}){let n;const r=G(()=>{const o=typeof e=="function"?e():e;if(o){n=new yx(o,{escapeDeactivates:!1,allowOutsideClick:!0,preventScroll:!0,returnFocusOnDeactivate:!0,delayInitialFocus:!1,fallbackFocus:o,...t,document:Le(o)});try{n.activate()}catch{}}});return function(){n?.deactivate(),r()}}var Ys="data-scroll-lock";function wx(e){const t=e.getBoundingClientRect().left;return Math.round(t)+e.scrollLeft?"paddingLeft":"paddingRight"}function Sx(e){const t=e??document,n=t.defaultView??window,{documentElement:r,body:o}=t;if(o.hasAttribute(Ys))return;const s=n.innerWidth-r.clientWidth;o.setAttribute(Ys,"");const a=()=>Sv(r,"--scrollbar-width",`${s}px`),l=wx(r),c=()=>To(o,{overflow:"hidden",[l]:`${s}px`}),u=()=>{const{scrollX:d,scrollY:m,visualViewport:f}=n,p=f?.offsetLeft??0,v=f?.offsetTop??0,b=To(o,{position:"fixed",overflow:"hidden",top:`${-(m-Math.floor(v))}px`,left:`${-(d-Math.floor(p))}px`,right:"0",[l]:`${s}px`});return()=>{b?.(),n.scrollTo({left:d,top:m,behavior:"instant"})}},h=[a(),Co()?u():c()];return()=>{h.forEach(d=>d?.()),o.removeAttribute(Ys)}}var Yn=new WeakMap,Go=new WeakMap,qo={},Xs=0,Ou=e=>e&&(e.host||Ou(e.parentNode)),Ex=(e,t)=>t.map(n=>{if(e.contains(n))return n;const r=Ou(n);return r&&e.contains(r)?r:(console.error("[zag-js > ariaHidden] target",n,"in not contained inside",e,". Doing nothing"),null)}).filter(n=>!!n),Ox=new Set(["script","output","status","next-route-announcer"]),Px=e=>Ox.has(e.localName)||e.role==="status"||e.hasAttribute("aria-live")?!0:e.matches("[data-live-announcer]"),Rx=(e,t)=>{const{parentNode:n,markerName:r,controlAttribute:o}=t,i=Ex(n,Array.isArray(e)?e:[e]);qo[r]||(qo[r]=new WeakMap);const s=qo[r],a=[],l=new Set,c=new Set(i),u=d=>{!d||l.has(d)||(l.add(d),u(d.parentNode))};i.forEach(u);const h=d=>{!d||c.has(d)||Array.prototype.forEach.call(d.children,m=>{if(l.has(m))h(m);else try{if(Px(m))return;const p=m.getAttribute(o)==="true",v=(Yn.get(m)||0)+1,b=(s.get(m)||0)+1;Yn.set(m,v),s.set(m,b),a.push(m),v===1&&p&&Go.set(m,!0),b===1&&m.setAttribute(r,""),p||m.setAttribute(o,"true")}catch(f){console.error("[zag-js > ariaHidden] cannot operate on ",m,f)}})};return h(n),l.clear(),Xs++,()=>{a.forEach(d=>{const m=Yn.get(d)-1,f=s.get(d)-1;Yn.set(d,m),s.set(d,f),m||(Go.has(d)||d.removeAttribute(o),Go.delete(d)),f||d.removeAttribute(r)}),Xs--,Xs||(Yn=new WeakMap,Yn=new WeakMap,Go=new WeakMap,qo={})}},Ix=e=>(Array.isArray(e)?e[0]:e).ownerDocument.body,Tx=(e,t=Ix(e),n="data-aria-hidden")=>{if(t)return Rx(e,{parentNode:t,markerName:n,controlAttribute:"aria-hidden"})},Nx=e=>{const t=requestAnimationFrame(()=>e());return()=>cancelAnimationFrame(t)};function Pu(e,t={}){const{defer:n=!0}=t,r=n?Nx:i=>i(),o=[];return o.push(r(()=>{const s=(typeof e=="function"?e():e).filter(Boolean);s.length!==0&&o.push(Tx(s))})),()=>{o.forEach(i=>i?.())}}function Ax(e){return!(e.metaKey||!wo()&&e.altKey||e.ctrlKey||e.key==="Control"||e.key==="Shift"||e.key==="Meta")}var _x=new Set(["checkbox","radio","range","color","file","image","button","submit","reset"]);function Vx(e,t,n){const r=n?$e(n):null,o=be(r);return e=e||r instanceof o.HTMLInputElement&&!_x.has(r?.type)||r instanceof o.HTMLTextAreaElement||r instanceof o.HTMLElement&&r.isContentEditable,!(e&&t==="keyboard"&&n instanceof o.KeyboardEvent&&!Reflect.has(Lx,n.key))}var nn=null,Qs=new Set,_r=new Map,rn=!1,Js=!1,Lx={Tab:!0,Escape:!0};function Ko(e,t){for(let n of Qs)n(e,t)}function Yo(e){rn=!0,Ax(e)&&(nn="keyboard",Ko("keyboard",e))}function Ye(e){nn="pointer",(e.type==="mousedown"||e.type==="pointerdown")&&(rn=!0,Ko("pointer",e))}function Ru(e){K0(e)&&(rn=!0,nn="virtual")}function Iu(e){const t=$e(e);t===be(t)||t===Le(t)||(!rn&&!Js&&(nn="virtual",Ko("virtual",e)),rn=!1,Js=!1)}function Tu(){rn=!1,Js=!0}function Fx(e){if(typeof window>"u"||_r.get(be(e)))return;const t=be(e),n=Le(e);let r=t.HTMLElement.prototype.focus;function o(){nn="virtual",Ko("virtual",null),rn=!0,r.apply(this,arguments)}Object.defineProperty(t.HTMLElement.prototype,"focus",{configurable:!0,value:o}),n.addEventListener("keydown",Yo,!0),n.addEventListener("keyup",Yo,!0),n.addEventListener("click",Ru,!0),t.addEventListener("focus",Iu,!0),t.addEventListener("blur",Tu,!1),typeof t.PointerEvent<"u"?(n.addEventListener("pointerdown",Ye,!0),n.addEventListener("pointermove",Ye,!0),n.addEventListener("pointerup",Ye,!0)):(n.addEventListener("mousedown",Ye,!0),n.addEventListener("mousemove",Ye,!0),n.addEventListener("mouseup",Ye,!0)),t.addEventListener("beforeunload",()=>{Dx(e)},{once:!0}),_r.set(t,{focus:r})}var Dx=(e,t)=>{const n=be(e),r=Le(e);_r.has(n)&&(n.HTMLElement.prototype.focus=_r.get(n).focus,r.removeEventListener("keydown",Yo,!0),r.removeEventListener("keyup",Yo,!0),r.removeEventListener("click",Ru,!0),n.removeEventListener("focus",Iu,!0),n.removeEventListener("blur",Tu,!1),typeof n.PointerEvent<"u"?(r.removeEventListener("pointerdown",Ye,!0),r.removeEventListener("pointermove",Ye,!0),r.removeEventListener("pointerup",Ye,!0)):(r.removeEventListener("mousedown",Ye,!0),r.removeEventListener("mousemove",Ye,!0),r.removeEventListener("mouseup",Ye,!0)),_r.delete(n))};function Nu(){return nn==="keyboard"}function zx(e={}){const{isTextInput:t,autoFocus:n,onChange:r,root:o}=e;Fx(o),r?.({isFocusVisible:n||Nu(),modality:nn});const i=(s,a)=>{Vx(!!t,s,a)&&r?.({isFocusVisible:Nu(),modality:s})};return Qs.add(i),()=>{Qs.delete(i)}}var Au=j("checkbox").parts("root","label","control","indicator");Au.build(),M()(["defaultChecked","checked","dir","disabled","form","getRootNode","id","ids","invalid","name","onCheckedChange","readOnly","required","value"]);const Mx=Au.extendWith("group"),[OT,_u]=Mn({name:"FieldContext",hookName:"useFieldContext",providerName:"",strict:!1});var Vu=j("clipboard").parts("root","control","trigger","indicator","input","label");Vu.build(),M()(["getRootNode","id","ids","value","defaultValue","timeout","onStatusChange","onValueChange"]),M()(["copied"]);const jx=gu.extendWith("view");var $x=Object.defineProperty,Bx=(e,t,n)=>t in e?$x(e,t,{enumerable:!0,configurable:!0,writable:!0,value:n}):e[t]=n,V=(e,t,n)=>Bx(e,typeof t!="symbol"?t+"":t,n),Xo={itemToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},itemToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Xo.itemToValue(e)},isItemDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1}},Qo=class Cp{constructor(t){this.options=t,V(this,"items"),V(this,"indexMap",null),V(this,"copy",n=>new Cp({...this.options,items:n??[...this.items]})),V(this,"isEqual",n=>Ze(this.items,n.items)),V(this,"setItems",n=>this.copy(n)),V(this,"getValues",(n=this.items)=>{const r=[];for(const o of n){const i=this.getItemValue(o);i!=null&&r.push(i)}return r}),V(this,"find",n=>{if(n==null)return null;const r=this.indexOf(n);return r!==-1?this.at(r):null}),V(this,"findMany",n=>{const r=[];for(const o of n){const i=this.find(o);i!=null&&r.push(i)}return r}),V(this,"at",n=>{if(!this.options.groupBy&&!this.options.groupSort)return this.items[n]??null;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){if(r===n)return s;r++}return null}),V(this,"sortFn",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return(o??0)-(i??0)}),V(this,"sort",n=>[...n].sort(this.sortFn.bind(this))),V(this,"getItemValue",n=>n==null?null:this.options.itemToValue?.(n)??Xo.itemToValue(n)),V(this,"getItemDisabled",n=>n==null?!1:this.options.isItemDisabled?.(n)??Xo.isItemDisabled(n)),V(this,"stringifyItem",n=>n==null?null:this.options.itemToString?.(n)??Xo.itemToString(n)),V(this,"stringify",n=>n==null?null:this.stringifyItem(this.find(n))),V(this,"stringifyItems",(n,r=", ")=>{const o=[];for(const i of n){const s=this.stringifyItem(i);s!=null&&o.push(s)}return o.join(r)}),V(this,"stringifyMany",(n,r)=>this.stringifyItems(this.findMany(n),r)),V(this,"has",n=>this.indexOf(n)!==-1),V(this,"hasItem",n=>n==null?!1:this.has(this.getItemValue(n))),V(this,"group",()=>{const{groupBy:n,groupSort:r}=this.options;if(!n)return[["",[...this.items]]];const o=new Map;this.items.forEach((s,a)=>{const l=n(s,a);o.has(l)||o.set(l,[]),o.get(l).push(s)});let i=Array.from(o.entries());return r&&i.sort(([s],[a])=>{if(typeof r=="function")return r(s,a);if(Array.isArray(r)){const l=r.indexOf(s),c=r.indexOf(a);return l===-1?1:c===-1?-1:l-c}return r==="asc"?s.localeCompare(a):r==="desc"?a.localeCompare(s):0}),i}),V(this,"getNextValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.min(i+r,this.size-1):i+r;i<=this.size&&this.getItemDisabled(this.at(i));)i++;return this.getItemValue(this.at(i))}),V(this,"getPreviousValue",(n,r=1,o=!1)=>{let i=this.indexOf(n);if(i===-1)return null;for(i=o?Math.max(i-r,0):i-r;i>=0&&this.getItemDisabled(this.at(i));)i--;return this.getItemValue(this.at(i))}),V(this,"indexOf",n=>{if(n==null)return-1;if(!this.options.groupBy&&!this.options.groupSort)return this.items.findIndex(r=>this.getItemValue(r)===n);if(!this.indexMap){this.indexMap=new Map;let r=0;const o=this.group();for(const[,i]of o)for(const s of i){const a=this.getItemValue(s);a!=null&&this.indexMap.set(a,r),r++}}return this.indexMap.get(n)??-1}),V(this,"getByText",(n,r)=>{const o=r!=null?this.indexOf(r):-1,i=n.length===1;for(let s=0;s{const{state:o,currentValue:i,timeout:s=350}=r,a=o.keysSoFar+n,c=a.length>1&&Array.from(a).every(f=>f===a[0])?a[0]:a,u=this.getByText(c,i),h=this.getItemValue(u);function d(){clearTimeout(o.timer),o.timer=-1}function m(f){o.keysSoFar=f,d(),f!==""&&(o.timer=+setTimeout(()=>{m(""),d()},s))}return m(a),h}),V(this,"update",(n,r)=>{let o=this.indexOf(n);return o===-1?this:this.copy([...this.items.slice(0,o),r,...this.items.slice(o+1)])}),V(this,"upsert",(n,r,o="append")=>{let i=this.indexOf(n);return i===-1?(o==="append"?this.append:this.prepend)(r):this.copy([...this.items.slice(0,i),r,...this.items.slice(i+1)])}),V(this,"insert",(n,...r)=>this.copy(Vr(this.items,n,...r))),V(this,"insertBefore",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Vr(this.items,o,...r))}),V(this,"insertAfter",(n,...r)=>{let o=this.indexOf(n);if(o===-1)if(this.items.length===0)o=0;else return this;return this.copy(Vr(this.items,o+1,...r))}),V(this,"prepend",(...n)=>this.copy(Vr(this.items,0,...n))),V(this,"append",(...n)=>this.copy(Vr(this.items,this.items.length,...n))),V(this,"filter",n=>{const r=this.items.filter((o,i)=>n(this.stringifyItem(o),i,o));return this.copy(r)}),V(this,"remove",(...n)=>{const r=n.map(o=>typeof o=="string"?o:this.getItemValue(o));return this.copy(this.items.filter(o=>{const i=this.getItemValue(o);return i==null?!1:!r.includes(i)}))}),V(this,"move",(n,r)=>{const o=this.indexOf(n);return o===-1?this:this.copy(Jo(this.items,[o],r))}),V(this,"moveBefore",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Jo(this.items,i,o))}),V(this,"moveAfter",(n,...r)=>{let o=this.items.findIndex(s=>this.getItemValue(s)===n);if(o===-1)return this;let i=r.map(s=>this.items.findIndex(a=>this.getItemValue(a)===s)).sort((s,a)=>s-a);return this.copy(Jo(this.items,i,o+1))}),V(this,"reorder",(n,r)=>this.copy(Jo(this.items,[n],r))),V(this,"compareValue",(n,r)=>{const o=this.indexOf(n),i=this.indexOf(r);return oi?1:0}),V(this,"range",(n,r)=>{let o=[],i=n;for(;i!=null;){if(this.find(i)&&o.push(i),i===r)return o;i=this.getNextValue(i)}return[]}),V(this,"getValueRange",(n,r)=>n&&r?this.compareValue(n,r)<=0?this.range(n,r):this.range(r,n):[]),V(this,"toString",()=>{let n="";for(const r of this.items){const o=this.getItemValue(r),i=this.stringifyItem(r),s=this.getItemDisabled(r),a=[o,i,s].filter(Boolean).join(":");n+=a+","}return n}),V(this,"toJSON",()=>({size:this.size,first:this.firstValue,last:this.lastValue})),this.items=[...t.items]}get size(){return this.items.length}get firstValue(){let t=0;for(;this.getItemDisabled(this.at(t));)t++;return this.getItemValue(this.at(t))}get lastValue(){let t=this.size-1;for(;this.getItemDisabled(this.at(t));)t--;return this.getItemValue(this.at(t))}*[Symbol.iterator](){yield*this.items}},Wx=(e,t)=>!!e?.toLowerCase().startsWith(t.toLowerCase());function Vr(e,t,...n){return[...e.slice(0,t),...n,...e.slice(t)]}function Jo(e,t,n){t=[...t].sort((o,i)=>o-i);const r=t.map(o=>e[o]);for(let o=t.length-1;o>=0;o--)e=[...e.slice(0,t[o]),...e.slice(t[o]+1)];return n=Math.max(0,n-t.filter(o=>ot[n])return 1}return e.length-t.length}function Ux(e){return e.sort(Fu)}function Gx(e,t){let n;return He(e,{...t,onEnter:(r,o)=>{if(t.predicate(r,o))return n=r,"stop"}}),n}function qx(e,t){const n=[];return He(e,{onEnter:(r,o)=>{t.predicate(r,o)&&n.push(r)},getChildren:t.getChildren}),n}function Du(e,t){let n;return He(e,{onEnter:(r,o)=>{if(t.predicate(r,o))return n=[...o],"stop"},getChildren:t.getChildren}),n}function Kx(e,t){let n=t.initialResult;return He(e,{...t,onEnter:(r,o)=>{n=t.nextResult(n,r,o)}}),n}function Yx(e,t){return Kx(e,{...t,initialResult:[],nextResult:(n,r,o)=>(n.push(...t.transform(r,o)),n)})}function Xx(e,t){const{predicate:n,create:r,getChildren:o}=t,i=(s,a)=>{const l=o(s,a),c=[];l.forEach((m,f)=>{const p=[...a,f],v=i(m,p);v&&c.push(v)});const u=a.length===0,h=n(s,a),d=c.length>0;return u||h||d?r(s,c,a):null};return i(e,[])||r(e,[],[])}function Qx(e,t){const n=[];let r=0;const o=new Map,i=new Map;return He(e,{getChildren:t.getChildren,onEnter:(s,a)=>{o.has(s)||o.set(s,r++);const l=t.getChildren(s,a);l.forEach(m=>{i.has(m)||i.set(m,s),o.has(m)||o.set(m,r++)});const c=l.length>0?l.map(m=>o.get(m)):void 0,u=i.get(s),h=u?o.get(u):void 0,d=o.get(s);n.push({...s,_children:c,_parent:h,_index:d})}}),n}function Jx(e,t){return{type:"insert",index:e,nodes:t}}function Zx(e){return{type:"remove",indexes:e}}function Zs(){return{type:"replace"}}function zu(e){return[e.slice(0,-1),e[e.length-1]]}function Mu(e,t,n=new Map){const[r,o]=zu(e);for(let s=r.length-1;s>=0;s--){const a=r.slice(0,s).join();switch(n.get(a)?.type){case"remove":continue}n.set(a,Zs())}const i=n.get(r.join());switch(i?.type){case"remove":n.set(r.join(),{type:"removeThenInsert",removeIndexes:i.indexes,insertIndex:o,insertNodes:t});break;default:n.set(r.join(),Jx(o,t))}return n}function ju(e){const t=new Map,n=new Map;for(const r of e){const o=r.slice(0,-1).join(),i=n.get(o)??[];i.push(r[r.length-1]),n.set(o,i.sort((s,a)=>s-a))}for(const r of e)for(let o=r.length-2;o>=0;o--){const i=r.slice(0,o).join();t.has(i)||t.set(i,Zs())}for(const[r,o]of n)t.set(r,Zx(o));return t}function e1(e,t){const n=new Map,[r,o]=zu(e);for(let i=r.length-1;i>=0;i--){const s=r.slice(0,i).join();n.set(s,Zs())}return n.set(r.join(),{type:"removeThenInsert",removeIndexes:[o],insertIndex:o,insertNodes:[t]}),n}function Zo(e,t,n){return t1(e,{...n,getChildren:(r,o)=>{const i=o.join();switch(t.get(i)?.type){case"replace":case"remove":case"removeThenInsert":case"insert":return n.getChildren(r,o);default:return[]}},transform:(r,o,i)=>{const s=i.join(),a=t.get(s);switch(a?.type){case"remove":return n.create(r,o.filter((u,h)=>!a.indexes.includes(h)),i);case"removeThenInsert":const l=o.filter((u,h)=>!a.removeIndexes.includes(h)),c=a.removeIndexes.reduce((u,h)=>h{const i=[0,...o],s=i.join(),a=t.transform(r,n[s]??[],o),l=i.slice(0,-1).join(),c=n[l]??[];c.push(a),n[l]=c}}),n[""][0]}function n1(e,t){const{nodes:n,at:r}=t;if(r.length===0)throw new Error("Can't insert nodes at the root");const o=Mu(r,n);return Zo(e,o,t)}function r1(e,t){if(t.at.length===0)return t.node;const n=e1(t.at,t.node);return Zo(e,n,t)}function o1(e,t){if(t.indexPaths.length===0)return e;for(const r of t.indexPaths)if(r.length===0)throw new Error("Can't remove the root node");const n=ju(t.indexPaths);return Zo(e,n,t)}function i1(e,t){if(t.indexPaths.length===0)return e;for(const i of t.indexPaths)if(i.length===0)throw new Error("Can't move the root node");if(t.to.length===0)throw new Error("Can't move nodes to the root");const n=Hx(t.indexPaths),r=n.map(i=>Lu(e,i,t)),o=Mu(t.to,r,ju(n));return Zo(e,o,t)}function He(e,t){const{onEnter:n,onLeave:r,getChildren:o}=t;let i=[],s=[{node:e}];const a=t.reuseIndexPath?()=>i:()=>i.slice();for(;s.length>0;){let l=s[s.length-1];if(l.state===void 0){const u=n?.(l.node,a());if(u==="stop")return;l.state=u==="skip"?-1:0}const c=l.children||o(l.node,a());if(l.children||(l.children=c),l.state!==-1){if(l.stateZe(this.rootNode,n.rootNode)),V(this,"getNodeChildren",n=>this.options.nodeToChildren?.(n)??Xn.nodeToChildren(n)??[]),V(this,"resolveIndexPath",n=>typeof n=="string"?this.getIndexPath(n):n),V(this,"resolveNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(r):void 0}),V(this,"getNodeChildrenCount",n=>this.options.nodeToChildrenCount?.(n)??Xn.nodeToChildrenCount(n)),V(this,"getNodeValue",n=>this.options.nodeToValue?.(n)??Xn.nodeToValue(n)),V(this,"getNodeDisabled",n=>this.options.isNodeDisabled?.(n)??Xn.isNodeDisabled(n)),V(this,"stringify",n=>{const r=this.findNode(n);return r?this.stringifyNode(r):null}),V(this,"stringifyNode",n=>this.options.nodeToString?.(n)??Xn.nodeToString(n)),V(this,"getFirstNode",(n=this.rootNode)=>{let r;return He(n,{getChildren:this.getNodeChildren,onEnter:(o,i)=>{if(!r&&i.length>0&&!this.getNodeDisabled(o))return r=o,"stop"}}),r}),V(this,"getLastNode",(n=this.rootNode,r={})=>{let o;return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isSameNode(i,n)){if(r.skip?.({value:this.getNodeValue(i),node:i,indexPath:s}))return"skip";s.length>0&&!this.getNodeDisabled(i)&&(o=i)}}}),o}),V(this,"at",n=>Lu(this.rootNode,n,{getChildren:this.getNodeChildren})),V(this,"findNode",(n,r=this.rootNode)=>Gx(r,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})),V(this,"findNodes",(n,r=this.rootNode)=>{const o=new Set(n.filter(i=>i!=null));return qx(r,{getChildren:this.getNodeChildren,predicate:i=>o.has(this.getNodeValue(i))})}),V(this,"sort",n=>n.reduce((r,o)=>{const i=this.getIndexPath(o);return i&&r.push({value:o,indexPath:i}),r},[]).sort((r,o)=>Fu(r.indexPath,o.indexPath)).map(({value:r})=>r)),V(this,"getIndexPath",n=>Du(this.rootNode,{getChildren:this.getNodeChildren,predicate:r=>this.getNodeValue(r)===n})),V(this,"getValue",n=>{const r=this.at(n);return r?this.getNodeValue(r):void 0}),V(this,"getValuePath",n=>{if(!n)return[];const r=[];let o=[...n];for(;o.length>0;){const i=this.at(o);i&&r.unshift(this.getNodeValue(i)),o.pop()}return r}),V(this,"getDepth",n=>Du(this.rootNode,{getChildren:this.getNodeChildren,predicate:o=>this.getNodeValue(o)===n})?.length??0),V(this,"isSameNode",(n,r)=>this.getNodeValue(n)===this.getNodeValue(r)),V(this,"isRootNode",n=>this.isSameNode(n,this.rootNode)),V(this,"contains",(n,r)=>!n||!r?!1:r.slice(0,n.length).every((o,i)=>n[i]===r[i])),V(this,"getNextNode",(n,r={})=>{let o=!1,i;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return l===n&&(o=!0),"skip";if(o&&!this.getNodeDisabled(s))return i=s,"stop";l===n&&(o=!0)}}),i}),V(this,"getPreviousNode",(n,r={})=>{let o,i=!1;return He(this.rootNode,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{if(this.isRootNode(s))return;const l=this.getNodeValue(s);if(r.skip?.({value:l,node:s,indexPath:a}))return"skip";if(l===n)return i=!0,"stop";this.getNodeDisabled(s)||(o=s)}}),i?o:void 0}),V(this,"getParentNodes",n=>{const r=this.resolveIndexPath(n)?.slice();if(!r)return[];const o=[];for(;r.length>0;){r.pop();const i=this.at(r);i&&!this.isRootNode(i)&&o.unshift(i)}return o}),V(this,"getDescendantNodes",(n,r)=>{const o=this.resolveNode(n);if(!o)return[];const i=[];return He(o,{getChildren:this.getNodeChildren,onEnter:(s,a)=>{a.length!==0&&(!r?.withBranch&&this.isBranchNode(s)||i.push(s))}}),i}),V(this,"getDescendantValues",(n,r)=>this.getDescendantNodes(n,r).map(i=>this.getNodeValue(i))),V(this,"getParentIndexPath",n=>n.slice(0,-1)),V(this,"getParentNode",n=>{const r=this.resolveIndexPath(n);return r?this.at(this.getParentIndexPath(r)):void 0}),V(this,"visit",n=>{const{skip:r,...o}=n;He(this.rootNode,{...o,getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(!this.isRootNode(i))return r?.({value:this.getNodeValue(i),node:i,indexPath:s})?"skip":o.onEnter?.(i,s)}})}),V(this,"getPreviousSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;--i>=0;){const s=o[i];if(!this.getNodeDisabled(s))return s}}),V(this,"getNextSibling",n=>{const r=this.getParentNode(n);if(!r)return;const o=this.getNodeChildren(r);let i=n[n.length-1];for(;++i{const r=this.getParentNode(n);return r?this.getNodeChildren(r):[]}),V(this,"getValues",(n=this.rootNode)=>Yx(n,{getChildren:this.getNodeChildren,transform:o=>[this.getNodeValue(o)]}).slice(1)),V(this,"isValidDepth",(n,r)=>r==null?!0:typeof r=="function"?r(n.length):n.length===r),V(this,"isBranchNode",n=>this.getNodeChildren(n).length>0||this.getNodeChildrenCount(n)!=null),V(this,"getBranchValues",(n=this.rootNode,r={})=>{let o=[];return He(n,{getChildren:this.getNodeChildren,onEnter:(i,s)=>{if(s.length===0)return;const a=this.getNodeValue(i);if(r.skip?.({value:a,node:i,indexPath:s}))return"skip";this.isBranchNode(i)&&this.isValidDepth(s,r.depth)&&o.push(this.getNodeValue(i))}}),o}),V(this,"flatten",(n=this.rootNode)=>Qx(n,{getChildren:this.getNodeChildren})),V(this,"_create",(n,r)=>this.getNodeChildren(n).length>0||r.length>0?{...n,children:r}:{...n}),V(this,"_insert",(n,r,o)=>this.copy(n1(n,{at:r,nodes:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"copy",n=>new wp({...this.options,rootNode:n})),V(this,"_replace",(n,r,o)=>this.copy(r1(n,{at:r,node:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_move",(n,r,o)=>this.copy(i1(n,{indexPaths:r,to:o,getChildren:this.getNodeChildren,create:this._create}))),V(this,"_remove",(n,r)=>this.copy(o1(n,{indexPaths:r,getChildren:this.getNodeChildren,create:this._create}))),V(this,"replace",(n,r)=>this._replace(this.rootNode,n,r)),V(this,"remove",n=>this._remove(this.rootNode,n)),V(this,"insertBefore",(n,r)=>this.getParentNode(n)?this._insert(this.rootNode,n,r):void 0),V(this,"insertAfter",(n,r)=>{if(!this.getParentNode(n))return;const i=[...n.slice(0,-1),n[n.length-1]+1];return this._insert(this.rootNode,i,r)}),V(this,"move",(n,r)=>this._move(this.rootNode,n,r)),V(this,"filter",n=>{const r=Xx(this.rootNode,{predicate:n,getChildren:this.getNodeChildren,create:this._create});return this.copy(r)}),V(this,"toJSON",()=>this.getValues(this.rootNode)),this.rootNode=t.rootNode}},Xn={nodeToValue(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"value")?e.value:""},nodeToString(e){return typeof e=="string"?e:Xt(e)&&Dt(e,"label")?e.label:Xn.nodeToValue(e)},isNodeDisabled(e){return Xt(e)&&Dt(e,"disabled")?!!e.disabled:!1},nodeToChildren(e){return e.children},nodeToChildrenCount(e){if(Xt(e)&&Dt(e,"childrenCount"))return e.childrenCount}},Wu=j("combobox").parts("root","clearTrigger","content","control","input","item","itemGroup","itemGroupLabel","itemIndicator","itemText","label","list","positioner","trigger");Wu.build();var Hu=e=>new Qo(e);Hu.empty=()=>new Qo({items:[]});var s1=e=>e.ids?.control??`combobox:${e.id}:control`,a1=e=>e.ids?.input??`combobox:${e.id}:input`,l1=e=>e.ids?.content??`combobox:${e.id}:content`,c1=e=>e.ids?.positioner??`combobox:${e.id}:popper`,u1=e=>e.ids?.trigger??`combobox:${e.id}:toggle-btn`,d1=e=>e.ids?.clearTrigger??`combobox:${e.id}:clear-btn`,Wt=e=>e.getById(l1(e)),Qn=e=>e.getById(a1(e)),Uu=e=>e.getById(c1(e)),Gu=e=>e.getById(s1(e)),Lr=e=>e.getById(u1(e)),qu=e=>e.getById(d1(e)),Fr=(e,t)=>{if(t==null)return null;const n=`[role=option][data-value="${CSS.escape(t)}"]`;return mv(Wt(e),n)},Ku=e=>{const t=Qn(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},h1=e=>{const t=Lr(e);e.isActiveElement(t)||t?.focus({preventScroll:!0})},{guards:f1,createMachine:g1,choose:p1}=wc(),{and:ye,not:Ue}=f1;g1({props({props:e}){return{loopFocus:!0,openOnClick:!1,defaultValue:[],closeOnSelect:!e.multiple,allowCustomValue:!1,alwaysSubmitOnEnter:!1,inputBehavior:"none",selectionBehavior:e.multiple?"clear":"replace",openOnKeyPress:!0,openOnChange:!0,composite:!0,navigate({node:t}){vc(t)},collection:Hu.empty(),...e,positioning:{placement:"bottom",sameWidth:!0,...e.positioning},translations:{triggerLabel:"Toggle suggestions",clearTriggerLabel:"Clear value",...e.translations}}},initialState({prop:e}){return e("open")||e("defaultOpen")?"suggesting":"idle"},context({prop:e,bindable:t,getContext:n,getEvent:r}){return{currentPlacement:t(()=>({defaultValue:void 0})),value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),isEqual:Ze,hash(o){return o.join(",")},onChange(o){const i=n(),s=i.get("selectedItems"),a=e("collection"),l=o.map(c=>s.find(h=>a.getItemValue(h)===c)||a.find(c));i.set("selectedItems",l),e("onValueChange")?.({value:o,items:l})}})),highlightedValue:t(()=>({defaultValue:e("defaultHighlightedValue")||null,value:e("highlightedValue"),onChange(o){const i=e("collection").find(o);e("onHighlightChange")?.({highlightedValue:o,highlightedItem:i})}})),inputValue:t(()=>{let o=e("inputValue")||e("defaultInputValue")||"";const i=e("defaultValue")||e("value")||[];if(!o.trim()&&!e("multiple")){const s=e("collection").stringifyMany(i);o=kt(e("selectionBehavior"),{preserve:o||s,replace:s,clear:""})}return{defaultValue:o,value:e("inputValue"),onChange(s){const a=r(),l=(a.previousEvent||a).src;e("onInputValueChange")?.({inputValue:s,reason:l})}}}),highlightedItem:t(()=>{const o=e("highlightedValue");return{defaultValue:e("collection").find(o)}}),selectedItems:t(()=>{const o=e("value")||e("defaultValue")||[];return{defaultValue:e("collection").findMany(o)}})}},computed:{isInputValueEmpty:({context:e})=>e.get("inputValue").length===0,isInteractive:({prop:e})=>!(e("readOnly")||e("disabled")),autoComplete:({prop:e})=>e("inputBehavior")==="autocomplete",autoHighlight:({prop:e})=>e("inputBehavior")==="autohighlight",hasSelectedItems:({context:e})=>e.get("value").length>0,valueAsString:({context:e,prop:t})=>t("collection").stringifyItems(e.get("selectedItems")),isCustomValue:({context:e,computed:t})=>e.get("inputValue")!==t("valueAsString")},watch({context:e,prop:t,track:n,action:r,send:o}){n([()=>e.hash("value")],()=>{r(["syncSelectedItems"])}),n([()=>e.get("inputValue")],()=>{r(["syncInputValue"])}),n([()=>e.get("highlightedValue")],()=>{r(["syncHighlightedItem","autofillInputValue"])}),n([()=>t("open")],()=>{r(["toggleVisibility"])}),n([()=>t("collection").toString()],()=>{o({type:"CHILDREN_CHANGE"})})},on:{"SELECTED_ITEMS.SYNC":{actions:["syncSelectedItems"]},"HIGHLIGHTED_VALUE.SET":{actions:["setHighlightedValue"]},"HIGHLIGHTED_VALUE.CLEAR":{actions:["clearHighlightedValue"]},"ITEM.SELECT":{actions:["selectItem"]},"ITEM.CLEAR":{actions:["clearItem"]},"VALUE.SET":{actions:["setValue"]},"INPUT_VALUE.SET":{actions:["setInputValue"]},"POSITIONING.SET":{actions:["reposition"]}},entry:p1([{guard:"autoFocus",actions:["setInitialFocus"]}]),states:{idle:{tags:["idle","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":{target:"interacting"},"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.FOCUS":{target:"focused"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{target:"focused",actions:["clearInputValue","clearSelectedItems","setInitialFocus"]}}},focused:{tags:["focused","closed"],entry:["scrollContentToTop","clearHighlightedValue"],on:{"CONTROLLED.OPEN":[{guard:"isChangeEvent",target:"suggesting"},{target:"interacting"}],"INPUT.CHANGE":[{guard:ye("isOpenControlled","openOnChange"),actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{guard:"openOnChange",target:"suggesting",actions:["setInputValue","invokeOnOpen","highlightFirstItemIfNeeded"]},{actions:["setInputValue"]}],"LAYER.INTERACT_OUTSIDE":{target:"idle"},"INPUT.ESCAPE":{guard:ye("isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue"]},"INPUT.BLUR":{target:"idle"},"INPUT.CLICK":[{guard:"isOpenControlled",actions:["highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstSelectedItem","invokeOnOpen"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["setInitialFocus","highlightFirstSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_DOWN":[{guard:ye("isOpenControlled","autoComplete"),actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"isOpenControlled",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightFirstOrSelectedItem","invokeOnOpen"]}],"INPUT.ARROW_UP":[{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{guard:"autoComplete",target:"interacting",actions:["invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]},{target:"interacting",actions:["highlightLastOrSelectedItem","invokeOnOpen"]}],OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"interacting",actions:["invokeOnOpen"]}],"VALUE.CLEAR":{actions:["clearInputValue","clearSelectedItems"]}}},interacting:{tags:["open","focused"],entry:["setInitialFocus"],effects:["scrollToHighlightedItem","trackDismissableLayer","trackPlacement","hideOtherElements"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]},{actions:["scrollToHighlightedItem"]}],"INPUT.HOME":{actions:["highlightFirstItem"]},"INPUT.END":{actions:["highlightLastItem"]},"INPUT.ARROW_DOWN":[{guard:ye("autoComplete","isLastItemHighlighted"),actions:["clearHighlightedValue","scrollContentToTop"]},{actions:["highlightNextItem"]}],"INPUT.ARROW_UP":[{guard:ye("autoComplete","isFirstItemHighlighted"),actions:["clearHighlightedValue"]},{actions:["highlightPrevItem"]}],"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":[{guard:"autoComplete",target:"suggesting",actions:["setInputValue"]},{target:"suggesting",actions:["clearHighlightedValue","setInputValue"]}],"ITEM.POINTER_MOVE":{actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],"LAYER.ESCAPE":[{guard:ye("isOpenControlled","autoComplete"),actions:["syncInputValue","invokeOnClose"]},{guard:"autoComplete",target:"focused",actions:["syncInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}},suggesting:{tags:["open","focused"],effects:["trackDismissableLayer","scrollToHighlightedItem","trackPlacement","hideOtherElements"],entry:["setInitialFocus"],on:{"CONTROLLED.CLOSE":[{guard:"restoreFocus",target:"focused",actions:["setFinalFocus"]},{target:"idle"}],CHILDREN_CHANGE:[{guard:"autoHighlight",actions:["highlightFirstItem"]},{guard:"isHighlightedItemRemoved",actions:["clearHighlightedValue"]}],"INPUT.ARROW_DOWN":{target:"interacting",actions:["highlightNextItem"]},"INPUT.ARROW_UP":{target:"interacting",actions:["highlightPrevItem"]},"INPUT.HOME":{target:"interacting",actions:["highlightFirstItem"]},"INPUT.END":{target:"interacting",actions:["highlightLastItem"]},"INPUT.ENTER":[{guard:ye("isOpenControlled","isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("hasHighlightedItem"),Ue("allowCustomValue")),target:"focused",actions:["revertInputValue","invokeOnClose"]},{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectHighlightedItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectHighlightedItem","invokeOnClose","setFinalFocus"]},{actions:["selectHighlightedItem"]}],"INPUT.CHANGE":{actions:["setInputValue"]},"LAYER.ESCAPE":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.POINTER_MOVE":{target:"interacting",actions:["setHighlightedValue"]},"ITEM.POINTER_LEAVE":{actions:["clearHighlightedValue"]},"LAYER.INTERACT_OUTSIDE":[{guard:ye("isOpenControlled","isCustomValue",Ue("allowCustomValue")),actions:["revertInputValue","invokeOnClose"]},{guard:ye("isCustomValue",Ue("allowCustomValue")),target:"idle",actions:["revertInputValue","invokeOnClose"]},{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"idle",actions:["invokeOnClose"]}],"TRIGGER.CLICK":[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose"]}],"ITEM.CLICK":[{guard:ye("isOpenControlled","closeOnSelect"),actions:["selectItem","invokeOnClose"]},{guard:"closeOnSelect",target:"focused",actions:["selectItem","invokeOnClose","setFinalFocus"]},{actions:["selectItem"]}],CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"focused",actions:["invokeOnClose","setFinalFocus"]}],"VALUE.CLEAR":[{guard:"isOpenControlled",actions:["clearInputValue","clearSelectedItems","invokeOnClose"]},{target:"focused",actions:["clearInputValue","clearSelectedItems","invokeOnClose","setFinalFocus"]}]}}},implementations:{guards:{isInputValueEmpty:({computed:e})=>e("isInputValueEmpty"),autoComplete:({computed:e,prop:t})=>e("autoComplete")&&!t("multiple"),autoHighlight:({computed:e})=>e("autoHighlight"),isFirstItemHighlighted:({prop:e,context:t})=>e("collection").firstValue===t.get("highlightedValue"),isLastItemHighlighted:({prop:e,context:t})=>e("collection").lastValue===t.get("highlightedValue"),isCustomValue:({computed:e})=>e("isCustomValue"),allowCustomValue:({prop:e})=>!!e("allowCustomValue"),hasHighlightedItem:({context:e})=>e.get("highlightedValue")!=null,closeOnSelect:({prop:e})=>!!e("closeOnSelect"),isOpenControlled:({prop:e})=>e("open")!=null,openOnChange:({prop:e,context:t})=>{const n=e("openOnChange");return Xm(n)?n:!!n?.({inputValue:t.get("inputValue")})},restoreFocus:({event:e})=>e.restoreFocus==null?!0:!!e.restoreFocus,isChangeEvent:({event:e})=>e.previousEvent?.type==="INPUT.CHANGE",autoFocus:({prop:e})=>!!e("autoFocus"),isHighlightedItemRemoved:({prop:e,context:t})=>!e("collection").has(t.get("highlightedValue"))},effects:{trackDismissableLayer({send:e,prop:t,scope:n}){return t("disableLayer")?void 0:Kn(()=>Wt(n),{type:"listbox",defer:!0,exclude:()=>[Qn(n),Lr(n),qu(n)],onFocusOutside:t("onFocusOutside"),onPointerDownOutside:t("onPointerDownOutside"),onInteractOutside:t("onInteractOutside"),onEscapeKeyDown(o){o.preventDefault(),o.stopPropagation(),e({type:"LAYER.ESCAPE",src:"escape-key"})},onDismiss(){e({type:"LAYER.INTERACT_OUTSIDE",src:"interact-outside",restoreFocus:!1})}})},hideOtherElements({scope:e}){return Pu([Qn(e),Wt(e),Lr(e),qu(e)])},trackPlacement({context:e,prop:t,scope:n}){const r=()=>Gu(n)||Lr(n),o=()=>Uu(n);return e.set("currentPlacement",t("positioning").placement),nt(r,o,{...t("positioning"),defer:!0,onComplete(i){e.set("currentPlacement",i.placement)}})},scrollToHighlightedItem({context:e,prop:t,scope:n,event:r}){const o=Qn(n);let i=[];const s=c=>{const u=r.current().type.includes("POINTER"),h=e.get("highlightedValue");if(u||!h)return;const d=Wt(n),m=t("scrollToIndexFn");if(m){const v=t("collection").indexOf(h);m({index:v,immediate:c,getElement:()=>Fr(n,h)});return}const f=Fr(n,h),p=G(()=>{Po(f,{rootEl:d,block:"nearest"})});i.push(p)},a=G(()=>s(!0));i.push(a);const l=Eo(o,{attributes:["aria-activedescendant"],callback:()=>s(!1)});return i.push(l),()=>{i.forEach(c=>c())}}},actions:{reposition({context:e,prop:t,scope:n,event:r}){nt(()=>Gu(n),()=>Uu(n),{...t("positioning"),...r.options,defer:!0,listeners:!1,onComplete(s){e.set("currentPlacement",s.placement)}})},setHighlightedValue({context:e,event:t}){t.value!=null&&e.set("highlightedValue",t.value)},clearHighlightedValue({context:e}){e.set("highlightedValue",null)},selectHighlightedItem(e){const{context:t,prop:n}=e,r=n("collection"),o=t.get("highlightedValue");if(!o||!r.has(o))return;const i=n("multiple")?zn(t.get("value"),o):[o];n("onSelect")?.({value:i,itemValue:o}),t.set("value",i);const s=kt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(i),clear:""});t.set("inputValue",s)},scrollToHighlightedItem({context:e,prop:t,scope:n}){ws(()=>{const r=e.get("highlightedValue");if(r==null)return;const o=Fr(n,r),i=Wt(n),s=t("scrollToIndexFn");if(s){const a=t("collection").indexOf(r);s({index:a,immediate:!0,getElement:()=>Fr(n,r)});return}Po(o,{rootEl:i,block:"nearest"})})},selectItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=o("multiple")?zn(t.get("value"),n.value):[n.value];o("onSelect")?.({value:i,itemValue:n.value}),t.set("value",i);const s=kt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},clearItem(e){const{context:t,event:n,flush:r,prop:o}=e;n.value!=null&&r(()=>{const i=Yt(t.get("value"),n.value);t.set("value",i);const s=kt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(i),clear:""});t.set("inputValue",s)})},setInitialFocus({scope:e}){G(()=>{Ku(e)})},setFinalFocus({scope:e}){G(()=>{Lr(e)?.dataset.focusable==null?Ku(e):h1(e)})},syncInputValue({context:e,scope:t,event:n}){const r=Qn(t);r&&(r.value=e.get("inputValue"),queueMicrotask(()=>{n.current().type!=="INPUT.CHANGE"&&C0(r)}))},setInputValue({context:e,event:t}){e.set("inputValue",t.value)},clearInputValue({context:e}){e.set("inputValue","")},revertInputValue({context:e,prop:t,computed:n}){const r=t("selectionBehavior"),o=kt(r,{replace:n("hasSelectedItems")?n("valueAsString"):"",preserve:e.get("inputValue"),clear:""});e.set("inputValue",o)},setValue(e){const{context:t,flush:n,event:r,prop:o}=e;n(()=>{t.set("value",r.value);const i=kt(o("selectionBehavior"),{preserve:t.get("inputValue"),replace:o("collection").stringifyMany(r.value),clear:""});t.set("inputValue",i)})},clearSelectedItems(e){const{context:t,flush:n,prop:r}=e;n(()=>{t.set("value",[]);const o=kt(r("selectionBehavior"),{preserve:t.get("inputValue"),replace:r("collection").stringifyMany([]),clear:""});t.set("inputValue",o)})},scrollContentToTop({prop:e,scope:t}){const n=e("scrollToIndexFn");if(n){const r=e("collection").firstValue;n({index:0,immediate:!0,getElement:()=>Fr(t,r)})}else{const r=Wt(t);if(!r)return;r.scrollTop=0}},invokeOnOpen({prop:e,event:t}){const n=Yu(t);e("onOpenChange")?.({open:!0,reason:n})},invokeOnClose({prop:e,event:t}){const n=Yu(t);e("onOpenChange")?.({open:!1,reason:n})},highlightFirstItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").firstValue;o&&e.set("highlightedValue",o)})},highlightFirstItemIfNeeded({computed:e,action:t}){e("autoHighlight")&&t(["highlightFirstItem"])},highlightLastItem({context:e,prop:t,scope:n}){(Wt(n)?queueMicrotask:G)(()=>{const o=t("collection").lastValue;o&&e.set("highlightedValue",o)})},highlightNextItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getNextValue(r),!n&&t("loopFocus")&&(n=o.firstValue)):n=o.firstValue,n&&e.set("highlightedValue",n)},highlightPrevItem({context:e,prop:t}){let n=null;const r=e.get("highlightedValue"),o=t("collection");r?(n=o.getPreviousValue(r),!n&&t("loopFocus")&&(n=o.lastValue)):n=o.lastValue,n&&e.set("highlightedValue",n)},highlightFirstSelectedItem({context:e,prop:t}){G(()=>{const[n]=t("collection").sort(e.get("value"));n&&e.set("highlightedValue",n)})},highlightFirstOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{let r=null;n("hasSelectedItems")?r=t("collection").sort(e.get("value"))[0]:r=t("collection").firstValue,r&&e.set("highlightedValue",r)})},highlightLastOrSelectedItem({context:e,prop:t,computed:n}){G(()=>{const r=t("collection");let o=null;n("hasSelectedItems")?o=r.sort(e.get("value"))[0]:o=r.lastValue,o&&e.set("highlightedValue",o)})},autofillInputValue({context:e,computed:t,prop:n,event:r,scope:o}){const i=Qn(o),s=n("collection");if(!t("autoComplete")||!i||!r.keypress)return;const a=s.stringify(e.get("highlightedValue"));G(()=>{i.value=a||e.get("inputValue")})},syncSelectedItems(e){queueMicrotask(()=>{const{context:t,prop:n}=e,r=n("collection"),o=t.get("value"),i=o.map(a=>t.get("selectedItems").find(c=>r.getItemValue(c)===a)||r.find(a));t.set("selectedItems",i);const s=kt(n("selectionBehavior"),{preserve:t.get("inputValue"),replace:r.stringifyMany(o),clear:""});t.set("inputValue",s)})},syncHighlightedItem({context:e,prop:t}){const n=t("collection").find(e.get("highlightedValue"));e.set("highlightedItem",n)},toggleVisibility({event:e,send:t,prop:n}){t({type:n("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:e})}}}});function Yu(e){return(e.previousEvent||e).src}M()(["allowCustomValue","autoFocus","closeOnSelect","collection","composite","defaultHighlightedValue","defaultInputValue","defaultOpen","defaultValue","dir","disabled","disableLayer","form","getRootNode","highlightedValue","id","ids","inputBehavior","inputValue","invalid","loopFocus","multiple","name","navigate","onFocusOutside","onHighlightChange","onInputValueChange","onInteractOutside","onOpenChange","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","openOnChange","openOnClick","openOnKeyPress","placeholder","positioning","readOnly","required","scrollToIndexFn","selectionBehavior","translations","value","alwaysSubmitOnEnter"]),M()(["htmlFor"]),M()(["id"]),M()(["item","persistFocus"]);const m1=Wu.extendWith("empty"),[Xu,Ht]=Mn({name:"DialogContext",hookName:"useDialogContext",providerName:""}),Qu=P.forwardRef((e,t)=>{const n=Ht(),r=ox(),o=Ts({...r,present:n.open}),i=qe(n.getBackdropProps(),o.getPresenceProps(),e);return o.unmounted?null:g.jsx(Ct.div,{...i,ref:No(o.ref,t)})});Qu.displayName="DialogBackdrop";const Ju=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getCloseTriggerProps(),e);return g.jsx(Ct.button,{...r,ref:t})});Ju.displayName="DialogCloseTrigger";const Zu=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe(n.getContentProps(),r.getPresenceProps(),e);return r.unmounted?null:g.jsx(Ct.div,{...o,ref:No(r.ref,t)})});Zu.displayName="DialogContent";const ed=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getDescriptionProps(),e);return g.jsx(Ct.div,{...r,ref:t})});ed.displayName="DialogDescription";const td=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getPositionerProps(),e);return Ns().unmounted?null:g.jsx(Ct.div,{...r,ref:t})});td.displayName="DialogPositioner";var ea=j("dialog").parts("trigger","backdrop","positioner","content","title","description","closeTrigger"),on=ea.build(),nd=e=>e.ids?.positioner??`dialog:${e.id}:positioner`,rd=e=>e.ids?.backdrop??`dialog:${e.id}:backdrop`,ta=e=>e.ids?.content??`dialog:${e.id}:content`,od=e=>e.ids?.trigger??`dialog:${e.id}:trigger`,na=e=>e.ids?.title??`dialog:${e.id}:title`,ra=e=>e.ids?.description??`dialog:${e.id}:description`,id=e=>e.ids?.closeTrigger??`dialog:${e.id}:close`,ei=e=>e.getById(ta(e)),v1=e=>e.getById(nd(e)),b1=e=>e.getById(rd(e)),y1=e=>e.getById(od(e)),x1=e=>e.getById(na(e)),k1=e=>e.getById(ra(e)),C1=e=>e.getById(id(e));function w1(e,t){const{state:n,send:r,context:o,prop:i,scope:s}=e,a=i("aria-label"),l=n.matches("open");return{open:l,setOpen(c){n.matches("open")!==c&&r({type:c?"OPEN":"CLOSE"})},getTriggerProps(){return t.button({...on.trigger.attrs,dir:i("dir"),id:od(s),"aria-haspopup":"dialog",type:"button","aria-expanded":l,"data-state":l?"open":"closed","aria-controls":ta(s),onClick(c){c.defaultPrevented||r({type:"TOGGLE"})}})},getBackdropProps(){return t.element({...on.backdrop.attrs,dir:i("dir"),hidden:!l,id:rd(s),"data-state":l?"open":"closed"})},getPositionerProps(){return t.element({...on.positioner.attrs,dir:i("dir"),id:nd(s),style:{pointerEvents:l?void 0:"none"}})},getContentProps(){const c=o.get("rendered");return t.element({...on.content.attrs,dir:i("dir"),role:i("role"),hidden:!l,id:ta(s),tabIndex:-1,"data-state":l?"open":"closed","aria-modal":!0,"aria-label":a||void 0,"aria-labelledby":a||!c.title?void 0:na(s),"aria-describedby":c.description?ra(s):void 0})},getTitleProps(){return t.element({...on.title.attrs,dir:i("dir"),id:na(s)})},getDescriptionProps(){return t.element({...on.description.attrs,dir:i("dir"),id:ra(s)})},getCloseTriggerProps(){return t.button({...on.closeTrigger.attrs,dir:i("dir"),id:id(s),type:"button",onClick(c){c.defaultPrevented||(c.stopPropagation(),r({type:"CLOSE"}))}})}}}var S1={props({props:e,scope:t}){const n=e.role==="alertdialog",r=n?()=>C1(t):void 0,o=typeof e.modal=="boolean"?e.modal:!0;return{role:"dialog",modal:o,trapFocus:o,preventScroll:o,closeOnInteractOutside:!n,closeOnEscape:!0,restoreFocus:!0,initialFocusEl:r,...e}},initialState({prop:e}){return e("open")||e("defaultOpen")?"open":"closed"},context({bindable:e}){return{rendered:e(()=>({defaultValue:{title:!0,description:!0}}))}},watch({track:e,action:t,prop:n}){e([()=>n("open")],()=>{t(["toggleVisibility"])})},states:{open:{entry:["checkRenderedElements","syncZIndex"],effects:["trackDismissableElement","trapFocus","preventScroll","hideContentBelow"],on:{"CONTROLLED.CLOSE":{target:"closed"},CLOSE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnClose"]},{target:"closed",actions:["invokeOnClose"]}]}},closed:{on:{"CONTROLLED.OPEN":{target:"open"},OPEN:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}],TOGGLE:[{guard:"isOpenControlled",actions:["invokeOnOpen"]},{target:"open",actions:["invokeOnOpen"]}]}}},implementations:{guards:{isOpenControlled:({prop:e})=>e("open")!=null},effects:{trackDismissableElement({scope:e,send:t,prop:n}){return Kn(()=>ei(e),{type:"dialog",defer:!0,pointerBlocking:n("modal"),exclude:[y1(e)],onInteractOutside(o){n("onInteractOutside")?.(o),n("closeOnInteractOutside")||o.preventDefault()},persistentElements:n("persistentElements"),onFocusOutside:n("onFocusOutside"),onPointerDownOutside:n("onPointerDownOutside"),onRequestDismiss:n("onRequestDismiss"),onEscapeKeyDown(o){n("onEscapeKeyDown")?.(o),n("closeOnEscape")||o.preventDefault()},onDismiss(){t({type:"CLOSE",src:"interact-outside"})}})},preventScroll({scope:e,prop:t}){if(t("preventScroll"))return Sx(e.getDoc())},trapFocus({scope:e,prop:t}){return t("trapFocus")?Cx(()=>ei(e),{preventScroll:!0,returnFocusOnDeactivate:!!t("restoreFocus"),initialFocus:t("initialFocusEl"),setReturnFocus:r=>t("finalFocusEl")?.()??r}):void 0},hideContentBelow({scope:e,prop:t}){return t("modal")?Pu(()=>[ei(e)],{defer:!0}):void 0}},actions:{checkRenderedElements({context:e,scope:t}){G(()=>{e.set("rendered",{title:!!x1(t),description:!!k1(t)})})},syncZIndex({scope:e}){G(()=>{const t=ei(e);if(!t)return;const n=xo(t);[v1(e),b1(e)].forEach(o=>{o?.style.setProperty("--z-index",n.zIndex),o?.style.setProperty("--layer-index",n.getPropertyValue("--layer-index"))})})},invokeOnClose({prop:e}){e("onOpenChange")?.({open:!1})},invokeOnOpen({prop:e}){e("onOpenChange")?.({open:!0})},toggleVisibility({prop:e,send:t,event:n}){t({type:e("open")?"CONTROLLED.OPEN":"CONTROLLED.CLOSE",previousEvent:n})}}}};M()(["aria-label","closeOnEscape","closeOnInteractOutside","dir","finalFocusEl","getRootNode","getRootNode","id","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","preventScroll","restoreFocus","role","trapFocus"]);const E1=e=>{const t=P.useId(),{getRootNode:n}=sc(),{dir:r}=Vv(),o={id:t,getRootNode:n,dir:r,...e},i=Pc(S1,o);return w1(i,$v)},O1=e=>{const[t,{children:n,...r}]=Ec(e),[o]=yu(t),i=E1(r),s=Ts(qe({present:i.open},t));return g.jsx(Xu,{value:i,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:s,children:n})})})},P1=e=>{const[t,{value:n,children:r}]=Ec(e),[o]=yu(t),i=Ts(qe({present:n.open},t));return g.jsx(Xu,{value:n,children:g.jsx(bu,{value:o,children:g.jsx(Tc,{value:i,children:r})})})},sd=P.forwardRef((e,t)=>{const n=Ht(),r=qe(n.getTitleProps(),e);return g.jsx(Ct.h2,{...r,ref:t})});sd.displayName="DialogTitle";const ad=P.forwardRef((e,t)=>{const n=Ht(),r=Ns(),o=qe({...n.getTriggerProps(),"aria-controls":r.unmounted?void 0:n.getTriggerProps()["aria-controls"]},e);return g.jsx(Ct.button,{...o,ref:t})});ad.displayName="DialogTrigger";var ld=j("editable").parts("root","area","label","preview","input","editTrigger","submitTrigger","cancelTrigger","control");ld.build(),M()(["activationMode","autoResize","dir","disabled","finalFocusEl","form","getRootNode","id","ids","invalid","maxLength","name","onEditChange","onFocusOutside","onInteractOutside","onPointerDownOutside","onValueChange","onValueCommit","onValueRevert","placeholder","readOnly","required","selectOnFocus","edit","defaultEdit","submitMode","translations","defaultValue","value"]);const cd=P.forwardRef((e,t)=>{const n=_u(),r=qe(n?.getInputProps(),e);return g.jsx(Ct.input,{...r,ref:t})});cd.displayName="FieldInput";const ud=j("field").parts("root","errorText","helperText","input","label","select","textarea","requiredIndicator");ud.build();var R1=e=>{if(!e)return;const t=xo(e),n=be(e),r=Le(e),o=()=>{requestAnimationFrame(()=>{e.style.height="auto";let c;t.boxSizing==="content-box"?c=e.scrollHeight-(parseFloat(t.paddingTop)+parseFloat(t.paddingBottom)):c=e.scrollHeight+parseFloat(t.borderTopWidth)+parseFloat(t.borderBottomWidth),t.maxHeight!=="none"&&c>parseFloat(t.maxHeight)?(t.overflowY==="hidden"&&(e.style.overflowY="scroll"),c=parseFloat(t.maxHeight)):t.overflowY!=="hidden"&&(e.style.overflowY="hidden"),e.style.height=`${c}px`})};e.addEventListener("input",o),e.form?.addEventListener("reset",o);const i=Object.getPrototypeOf(e),s=Object.getOwnPropertyDescriptor(i,"value");Object.defineProperty(e,"value",{...s,set(){s?.set?.apply(this,arguments),o()}});const a=new n.ResizeObserver(()=>{requestAnimationFrame(()=>o())});a.observe(e);const l=new n.MutationObserver(()=>o());return l.observe(e,{attributes:!0,attributeFilter:["rows","placeholder"]}),r.fonts?.addEventListener("loadingdone",o),()=>{e.removeEventListener("input",o),e.form?.removeEventListener("reset",o),r.fonts?.removeEventListener("loadingdone",o),a.disconnect(),l.disconnect()}};const dd=P.forwardRef((e,t)=>{const{autoresize:n,...r}=e,o=P.useRef(null),i=_u(),s=qe(i?.getTextareaProps(),{style:{resize:n?"none":void 0}},r);return P.useEffect(()=>{if(n)return R1(o.current)},[n]),g.jsx(Ct.textarea,{...s,ref:No(t,o)})});dd.displayName="FieldTextarea";const hd=j("fieldset").parts("root","errorText","helperText","legend");hd.build();var fd=j("file-upload").parts("root","dropzone","item","itemDeleteTrigger","itemGroup","itemName","itemPreview","itemPreviewImage","itemSizeText","label","trigger","clearTrigger");fd.build(),M()(["accept","acceptedFiles","allowDrop","capture","defaultAcceptedFiles","dir","directory","disabled","getRootNode","id","ids","invalid","locale","maxFiles","maxFileSize","minFileSize","name","onFileAccept","onFileChange","onFileReject","preventDocumentDrop","required","transformFiles","translations","validate"]),M()(["file","type"]);var gd=j("hoverCard").parts("arrow","arrowTip","trigger","positioner","content");gd.build();var I1=e=>e.ids?.trigger??`hover-card:${e.id}:trigger`,T1=e=>e.ids?.content??`hover-card:${e.id}:content`,N1=e=>e.ids?.positioner??`hover-card:${e.id}:popper`,oa=e=>e.getById(I1(e)),A1=e=>e.getById(T1(e)),pd=e=>e.getById(N1(e)),{not:ti,and:md}=jt();md("isOpenControlled",ti("isPointer")),ti("isPointer"),md("isOpenControlled",ti("isPointer")),ti("isPointer"),M()(["closeDelay","dir","getRootNode","id","ids","disabled","onOpenChange","defaultOpen","open","openDelay","positioning","onInteractOutside","onPointerDownOutside","onFocusOutside"]);var vd=j("tree-view").parts("branch","branchContent","branchControl","branchIndentGuide","branchIndicator","branchText","branchTrigger","item","itemIndicator","itemText","label","nodeCheckbox","root","tree");vd.build();var bd=e=>new Bu(e);bd.empty=()=>new Bu({rootNode:{children:[]}});var _1=(e,t)=>e.ids?.node?.(t)??`tree:${e.id}:node:${t}`,sn=(e,t)=>{t!=null&&e.getById(_1(e,t))?.focus()};function V1(e,t,n){const r=e.getDescendantValues(t),o=r.every(i=>n.includes(i));return Dn(o?Yt(n,...r):Kt(n,...r))}function ni(e,t){const{context:n,prop:r,refs:o}=e;if(!r("loadChildren")){n.set("expandedValue",p=>Dn(Kt(p,...t)));return}const i=n.get("loadingStatus"),[s,a]=ql(t,p=>i[p]==="loaded");if(s.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...s))),a.length===0)return;const l=r("collection"),[c,u]=ql(a,p=>{const v=l.findNode(p);return l.getNodeChildren(v).length>0});if(c.length>0&&n.set("expandedValue",p=>Dn(Kt(p,...c))),u.length===0)return;n.set("loadingStatus",p=>({...p,...u.reduce((v,b)=>({...v,[b]:"loading"}),{})}));const h=u.map(p=>{const v=l.getIndexPath(p),b=l.getValuePath(v),y=l.findNode(p);return{id:p,indexPath:v,valuePath:b,node:y}}),d=o.get("pendingAborts"),m=r("loadChildren");ic(m,()=>"[zag-js/tree-view] `loadChildren` is required for async expansion");const f=h.map(({id:p,indexPath:v,valuePath:b,node:y})=>{const x=d.get(p);x&&(x.abort(),d.delete(p));const w=new AbortController;return d.set(p,w),m({valuePath:b,indexPath:v,node:y,signal:w.signal})});Promise.allSettled(f).then(p=>{const v=[],b=[],y=n.get("loadingStatus");let x=r("collection");p.forEach((w,k)=>{const{id:O,indexPath:R,node:I,valuePath:N}=h[k];w.status==="fulfilled"?(y[O]="loaded",v.push(O),x=x.replace(R,{...I,children:w.value})):(d.delete(O),Reflect.deleteProperty(y,O),b.push({node:I,error:w.reason,indexPath:R,valuePath:N}))}),n.set("loadingStatus",y),v.length&&(n.set("expandedValue",w=>Dn(Kt(w,...v))),r("onLoadChildrenComplete")?.({collection:x})),b.length&&r("onLoadChildrenError")?.({nodes:b})})}function Ut(e){const{prop:t,context:n}=e;return function({indexPath:o}){return t("collection").getValuePath(o).slice(0,-1).some(s=>!n.get("expandedValue").includes(s))}}var{and:ft}=jt();ft("isMultipleSelection","moveFocus"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isBranchFocused","isBranchExpanded"),ft("isShiftKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),ft("isCtrlKey","isMultipleSelection"),ft("isShiftKey","isMultipleSelection"),M()(["ids","collection","dir","expandedValue","expandOnClick","defaultFocusedValue","focusedValue","getRootNode","id","onExpandedChange","onFocusChange","onSelectionChange","checkedValue","selectedValue","selectionMode","typeahead","defaultExpandedValue","defaultSelectedValue","defaultCheckedValue","onCheckedChange","onLoadChildrenComplete","onLoadChildrenError","loadChildren"]),M()(["node","indexPath"]);var yd=j("listbox").parts("label","input","item","itemText","itemIndicator","itemGroup","itemGroupLabel","content","root","valueText");yd.build(),M()(["collection","defaultHighlightedValue","defaultValue","dir","disabled","deselectable","disallowSelectAll","getRootNode","highlightedValue","id","ids","loopFocus","onHighlightChange","onSelect","onValueChange","orientation","scrollToIndexFn","selectionMode","selectOnHighlight","typeahead","value"]),M()(["item","highlightOnHover"]),M()(["id"]),M()(["htmlFor"]);const L1=yd.extendWith("empty");var xd=j("menu").parts("arrow","arrowTip","content","contextTrigger","indicator","item","itemGroup","itemGroupLabel","itemIndicator","itemText","positioner","separator","trigger","triggerItem");xd.build();var kd=e=>e.ids?.trigger??`menu:${e.id}:trigger`,F1=e=>e.ids?.contextTrigger??`menu:${e.id}:ctx-trigger`,Cd=e=>e.ids?.content??`menu:${e.id}:content`,D1=e=>e.ids?.positioner??`menu:${e.id}:popper`,ia=(e,t)=>`${e.id}/${t}`,an=e=>e?.dataset.value??null,Gt=e=>e.getById(Cd(e)),wd=e=>e.getById(D1(e)),ri=e=>e.getById(kd(e)),z1=(e,t)=>t?e.getById(ia(e,t)):null,sa=e=>e.getById(F1(e)),Dr=e=>{const n=`[role^="menuitem"][data-ownedby=${CSS.escape(Cd(e))}]:not([data-disabled])`;return Io(Gt(e),n)},M1=e=>kr(Dr(e)),j1=e=>hs(Dr(e)),aa=(e,t)=>t?e.id===t||e.dataset.value===t:!1,$1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return Gm(n,r,{loop:t.loop??t.loopFocus})},B1=(e,t)=>{const n=Dr(e),r=n.findIndex(o=>aa(o,t.value));return Km(n,r,{loop:t.loop??t.loopFocus})},W1=(e,t)=>{const n=Dr(e),r=n.find(o=>aa(o,t.value));return Or(n,{state:t.typeaheadState,key:t.key,activeId:r?.id??null})},H1=e=>!!e?.getAttribute("role")?.startsWith("menuitem")&&!!e?.hasAttribute("aria-controls"),U1="menu:select";function G1(e,t){if(!e)return;const n=be(e),r=new n.CustomEvent(U1,{detail:{value:t}});e.dispatchEvent(r)}var{not:Xe,and:Jn,or:q1}=jt();Xe("isSubmenu"),q1("isOpenAutoFocusEvent","isArrowDownEvent"),Jn(Xe("isTriggerItem"),"isOpenControlled"),Xe("isTriggerItem"),Jn("isSubmenu","isOpenControlled"),Xe("isPointerSuspended"),Jn(Xe("isPointerSuspended"),Xe("isTriggerItem")),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect","isOpenControlled"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"),"closeOnSelect"),Jn(Xe("isTriggerItemHighlighted"),Xe("isHighlightedItemEditable"));function Sd(e){let t=e.parent;for(;t&&t.context.get("isSubmenu");)t=t.refs.get("parent");t?.send({type:"CLOSE"})}function K1(e,t){return e?px(e,t):!1}function Y1(e,t,n){const r=Object.keys(e).length>0;if(!t)return null;if(!r)return ia(n,t);for(const o in e){const i=e[o],s=kd(i.scope);if(s===t)return s}return ia(n,t)}M()(["anchorPoint","aria-label","closeOnSelect","composite","defaultHighlightedValue","defaultOpen","dir","getRootNode","highlightedValue","id","ids","loopFocus","navigate","onEscapeKeyDown","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","onSelect","open","positioning","typeahead"]),M()(["closeOnSelect","disabled","value","valueText"]),M()(["htmlFor"]),M()(["id"]),M()(["checked","closeOnSelect","disabled","onCheckedChange","type","value","valueText"]);let la=new Map,ca=!1;try{ca=new Intl.NumberFormat("de-DE",{signDisplay:"exceptZero"}).resolvedOptions().signDisplay==="exceptZero"}catch{}let oi=!1;try{oi=new Intl.NumberFormat("de-DE",{style:"unit",unit:"degree"}).resolvedOptions().style==="unit"}catch{}const Ed={degree:{narrow:{default:"°","ja-JP":" 度","zh-TW":"度","sl-SI":" °"}}};class X1{format(t){let n="";if(!ca&&this.options.signDisplay!=null?n=J1(this.numberFormatter,this.options.signDisplay,t):n=this.numberFormatter.format(t),this.options.style==="unit"&&!oi){var r;let{unit:o,unitDisplay:i="short",locale:s}=this.resolvedOptions();if(!o)return n;let a=(r=Ed[o])===null||r===void 0?void 0:r[i];n+=a[s]||a.default}return n}formatToParts(t){return this.numberFormatter.formatToParts(t)}formatRange(t,n){if(typeof this.numberFormatter.formatRange=="function")return this.numberFormatter.formatRange(t,n);if(n= start date");return`${this.format(t)} – ${this.format(n)}`}formatRangeToParts(t,n){if(typeof this.numberFormatter.formatRangeToParts=="function")return this.numberFormatter.formatRangeToParts(t,n);if(n= start date");let r=this.numberFormatter.formatToParts(t),o=this.numberFormatter.formatToParts(n);return[...r.map(i=>({...i,source:"startRange"})),{type:"literal",value:" – ",source:"shared"},...o.map(i=>({...i,source:"endRange"}))]}resolvedOptions(){let t=this.numberFormatter.resolvedOptions();return!ca&&this.options.signDisplay!=null&&(t={...t,signDisplay:this.options.signDisplay}),!oi&&this.options.style==="unit"&&(t={...t,style:"unit",unit:this.options.unit,unitDisplay:this.options.unitDisplay}),t}constructor(t,n={}){this.numberFormatter=Q1(t,n),this.options=n}}function Q1(e,t={}){let{numberingSystem:n}=t;if(n&&e.includes("-nu-")&&(e.includes("-u-")||(e+="-u-"),e+=`-nu-${n}`),t.style==="unit"&&!oi){var r;let{unit:s,unitDisplay:a="short"}=t;if(!s)throw new Error('unit option must be provided with style: "unit"');if(!(!((r=Ed[s])===null||r===void 0)&&r[a]))throw new Error(`Unsupported unit ${s} with unitDisplay = ${a}`);t={...t,style:"decimal"}}let o=e+(t?Object.entries(t).sort((s,a)=>s[0]0||Object.is(n,0):t==="exceptZero"&&(Object.is(n,-0)||Object.is(n,0)?n=Math.abs(n):r=n>0),r){let o=e.format(-n),i=e.format(n),s=o.replace(i,"").replace(/\u200e|\u061C/,"");return[...s].length!==1&&console.warn("@react-aria/i18n polyfill for NumberFormat signDisplay: Unsupported case"),o.replace(i,"!!!").replace(s,"+").replace("!!!",i)}else return e.format(n)}}const Z1=new RegExp("^.*\\(.*\\).*$"),ek=["latn","arab","hanidec","deva","beng","fullwide"];class Od{parse(t){return ua(this.locale,this.options,t).parse(t)}isValidPartialNumber(t,n,r){return ua(this.locale,this.options,t).isValidPartialNumber(t,n,r)}getNumberingSystem(t){return ua(this.locale,this.options,t).options.numberingSystem}constructor(t,n={}){this.locale=t,this.options=n}}const Pd=new Map;function ua(e,t,n){let r=Rd(e,t);if(!e.includes("-nu-")&&!r.isValidPartialNumber(n)){for(let o of ek)if(o!==r.options.numberingSystem){let i=Rd(e+(e.includes("-u-")?"-nu-":"-u-nu-")+o,t);if(i.isValidPartialNumber(n))return i}}return r}function Rd(e,t){let n=e+(t?Object.entries(t).sort((o,i)=>o[0]-1&&(n=`-${n}`)}let r=n?+n:NaN;if(isNaN(r))return NaN;if(this.options.style==="percent"){var o,i;let s={...this.options,style:"decimal",minimumFractionDigits:Math.min(((o=this.options.minimumFractionDigits)!==null&&o!==void 0?o:0)+2,20),maximumFractionDigits:Math.min(((i=this.options.maximumFractionDigits)!==null&&i!==void 0?i:0)+2,20)};return new Od(this.locale,s).parse(new X1(this.locale,s).format(r))}return this.options.currencySign==="accounting"&&Z1.test(t)&&(r=-1*r),r}sanitize(t){return t=t.replace(this.symbols.literals,""),this.symbols.minusSign&&(t=t.replace("-",this.symbols.minusSign)),this.options.numberingSystem==="arab"&&(this.symbols.decimal&&(t=t.replace(",",this.symbols.decimal),t=t.replace("،",this.symbols.decimal)),this.symbols.group&&(t=Zn(t,".",this.symbols.group))),this.symbols.group==="’"&&t.includes("'")&&(t=Zn(t,"'",this.symbols.group)),this.options.locale==="fr-FR"&&this.symbols.group&&(t=Zn(t," ",this.symbols.group),t=Zn(t,/\u00A0/g,this.symbols.group)),t}isValidPartialNumber(t,n=-1/0,r=1/0){return t=this.sanitize(t),this.symbols.minusSign&&t.startsWith(this.symbols.minusSign)&&n<0?t=t.slice(this.symbols.minusSign.length):this.symbols.plusSign&&t.startsWith(this.symbols.plusSign)&&r>0&&(t=t.slice(this.symbols.plusSign.length)),this.symbols.group&&t.startsWith(this.symbols.group)||this.symbols.decimal&&t.indexOf(this.symbols.decimal)>-1&&this.options.maximumFractionDigits===0?!1:(this.symbols.group&&(t=Zn(t,this.symbols.group,"")),t=t.replace(this.symbols.numeral,""),this.symbols.decimal&&(t=t.replace(this.symbols.decimal,"")),t.length===0)}constructor(t,n={}){this.locale=t,n.roundingIncrement!==1&&n.roundingIncrement!=null&&(n.maximumFractionDigits==null&&n.minimumFractionDigits==null?(n.maximumFractionDigits=0,n.minimumFractionDigits=0):n.maximumFractionDigits==null?n.maximumFractionDigits=n.minimumFractionDigits:n.minimumFractionDigits==null&&(n.minimumFractionDigits=n.maximumFractionDigits)),this.formatter=new Intl.NumberFormat(t,n),this.options=this.formatter.resolvedOptions(),this.symbols=rk(t,this.formatter,this.options,n);var r,o;this.options.style==="percent"&&(((r=this.options.minimumFractionDigits)!==null&&r!==void 0?r:0)>18||((o=this.options.maximumFractionDigits)!==null&&o!==void 0?o:0)>18)&&console.warn("NumberParser cannot handle percentages with greater than 18 decimal places, please reduce the number in your options.")}}const Id=new Set(["decimal","fraction","integer","minusSign","plusSign","group"]),nk=[0,4,2,1,11,20,3,7,100,21,.1,1.1];function rk(e,t,n,r){var o,i,s,a;let l=new Intl.NumberFormat(e,{...n,minimumSignificantDigits:1,maximumSignificantDigits:21,roundingIncrement:1,roundingPriority:"auto",roundingMode:"halfExpand"}),c=l.formatToParts(-10000.111),u=l.formatToParts(10000.111),h=nk.map(A=>l.formatToParts(A));var d;let m=(d=(o=c.find(A=>A.type==="minusSign"))===null||o===void 0?void 0:o.value)!==null&&d!==void 0?d:"-",f=(i=u.find(A=>A.type==="plusSign"))===null||i===void 0?void 0:i.value;!f&&(r?.signDisplay==="exceptZero"||r?.signDisplay==="always")&&(f="+");let v=(s=new Intl.NumberFormat(e,{...n,minimumFractionDigits:2,maximumFractionDigits:2}).formatToParts(.001).find(A=>A.type==="decimal"))===null||s===void 0?void 0:s.value,b=(a=c.find(A=>A.type==="group"))===null||a===void 0?void 0:a.value,y=c.filter(A=>!Id.has(A.type)).map(A=>Td(A.value)),x=h.flatMap(A=>A.filter(T=>!Id.has(T.type)).map(T=>Td(T.value))),w=[...new Set([...y,...x])].sort((A,T)=>T.length-A.length),k=w.length===0?new RegExp("[\\p{White_Space}]","gu"):new RegExp(`${w.join("|")}|[\\p{White_Space}]`,"gu"),O=[...new Intl.NumberFormat(n.locale,{useGrouping:!1}).format(9876543210)].reverse(),R=new Map(O.map((A,T)=>[A,T])),I=new RegExp(`[${O.join("")}]`,"g");return{minusSign:m,plusSign:f,decimal:v,group:b,literals:k,numeral:I,index:A=>String(R.get(A))}}function Zn(e,t,n){return e.replaceAll?e.replaceAll(t,n):e.split(t).join(n)}function Td(e){return e.replace(/[.*+?^${}()|[\]\\]/g,"\\$&")}var Nd=j("numberInput").parts("root","label","input","control","valueText","incrementTrigger","decrementTrigger","scrubber");Nd.build();var ok=e=>e.ids?.input??`number-input:${e.id}:input`,ik=e=>e.ids?.incrementTrigger??`number-input:${e.id}:inc`,sk=e=>e.ids?.decrementTrigger??`number-input:${e.id}:dec`,Ad=e=>`number-input:${e.id}:cursor`,ii=e=>e.getById(ok(e)),ak=e=>e.getById(ik(e)),lk=e=>e.getById(sk(e)),_d=e=>e.getDoc().getElementById(Ad(e)),ck=(e,t)=>{let n=null;return t==="increment"&&(n=ak(e)),t==="decrement"&&(n=lk(e)),n},uk=(e,t)=>{if(!fc())return fk(e,t),()=>{_d(e)?.remove()}},dk=e=>{const t=e.getDoc(),n=t.documentElement,r=t.body;return r.style.pointerEvents="none",n.style.userSelect="none",n.style.cursor="ew-resize",()=>{r.style.pointerEvents="",n.style.userSelect="",n.style.cursor="",n.style.length||n.removeAttribute("style"),r.style.length||r.removeAttribute("style")}},hk=(e,t)=>{const{point:n,isRtl:r,event:o}=t,i=e.getWin(),s=ps(o.movementX,i.devicePixelRatio),a=ps(o.movementY,i.devicePixelRatio);let l=s>0?"increment":s<0?"decrement":null;r&&l==="increment"&&(l="decrement"),r&&l==="decrement"&&(l="increment");const c={x:n.x+s,y:n.y+a},u=i.innerWidth,h=ps(7.5,i.devicePixelRatio);return c.x=d0(c.x+h,u)-h,{hint:l,point:c}},fk=(e,t)=>{const n=e.getDoc(),r=n.createElement("div");r.className="scrubber--cursor",r.id=Ad(e),Object.assign(r.style,{width:"15px",height:"15px",position:"fixed",pointerEvents:"none",left:"0px",top:"0px",zIndex:S0,transform:t?`translate3d(${t.x}px, ${t.y}px, 0px)`:void 0,willChange:"transform"}),r.innerHTML=` - `,n.body.appendChild(r)};function fk(e,t){if(!(!e||!t.isActiveElement(e)))try{const{selectionStart:n,selectionEnd:r,value:o}=e,i=o.substring(0,n),s=o.substring(r);return{start:n,end:r,value:o,beforeTxt:i,afterTxt:s}}catch{}}function gk(e,t,n){if(!(!e||!n.isActiveElement(e))){if(!t){e.setSelectionRange(e.value.length,e.value.length);return}try{const{value:r}=e,{beforeTxt:o="",afterTxt:i="",start:s}=t;let a=r.length;if(r.endsWith(i))a=r.length-i.length;else if(r.startsWith(o))a=o.length;else if(s!=null){const l=o[s-1],c=r.indexOf(l,s-1);c!==-1&&(a=c+1)}e.setSelectionRange(a,a)}catch{}}}var pk=(e,t={})=>new Intl.NumberFormat(e,t),mk=(e,t={})=>new Od(e,t),da=(e,t)=>{const{prop:n,computed:r}=t;return n("formatOptions")?e===""?Number.NaN:r("parser").parse(e):parseFloat(e)},ln=(e,t)=>{const{prop:n,computed:r}=t;return Number.isNaN(e)?"":n("formatOptions")?r("formatter").format(e):e.toString()},vk=(e,t)=>{let n=e!==void 0&&!Number.isNaN(e)?e:1;return t?.style==="percent"&&(e===void 0||Number.isNaN(e))&&(n=.01),n},{choose:bk,guards:yk,createMachine:xk}=wc(),{not:Vd,and:Fd}=yk;xk({props({props:e}){const t=vk(e.step,e.formatOptions);return{dir:"ltr",locale:"en-US",focusInputOnChange:!0,clampValueOnBlur:!e.allowOverflow,allowOverflow:!1,inputMode:"decimal",pattern:"-?[0-9]*(.[0-9]+)?",defaultValue:"",step:t,min:Number.MIN_SAFE_INTEGER,max:Number.MAX_SAFE_INTEGER,spinOnPress:!0,...e,translations:{incrementLabel:"increment value",decrementLabel:"decrease value",...e.translations}}},initialState(){return"idle"},context({prop:e,bindable:t,getComputed:n}){return{value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),onChange(r){const o=n(),i=da(r,{computed:o,prop:e});e("onValueChange")?.({value:r,valueAsNumber:i})}})),hint:t(()=>({defaultValue:null})),scrubberCursorPoint:t(()=>({defaultValue:null,hash(r){return r?`x:${r.x}, y:${r.y}`:""}})),fieldsetDisabled:t(()=>({defaultValue:!1}))}},computed:{isRtl:({prop:e})=>e("dir")==="rtl",valueAsNumber:({context:e,computed:t,prop:n})=>da(e.get("value"),{computed:t,prop:n}),formattedValue:({computed:e,prop:t})=>ln(e("valueAsNumber"),{computed:e,prop:t}),isAtMin:({computed:e,prop:t})=>h0(e("valueAsNumber"),t("min")),isAtMax:({computed:e,prop:t})=>d0(e("valueAsNumber"),t("max")),isOutOfRange:({computed:e,prop:t})=>!f0(e("valueAsNumber"),t("min"),t("max")),isValueEmpty:({context:e})=>e.get("value")==="",isDisabled:({prop:e,context:t})=>!!e("disabled")||t.get("fieldsetDisabled"),canIncrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMax"),canDecrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMin"),valueText:({prop:e,context:t})=>e("translations").valueText?.(t.get("value")),formatter:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>pk(e,t)),parser:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>mk(e,t))},watch({track:e,action:t,context:n,computed:r,prop:o}){e([()=>n.get("value"),()=>o("locale")],()=>{t(["syncInputElement"])}),e([()=>r("isOutOfRange")],()=>{t(["invokeOnInvalid"])}),e([()=>n.hash("scrubberCursorPoint")],()=>{t(["setVirtualCursorPosition"])})},effects:["trackFormControl"],on:{"VALUE.SET":{actions:["setRawValue"]},"VALUE.CLEAR":{actions:["clearValue"]},"VALUE.INCREMENT":{actions:["increment"]},"VALUE.DECREMENT":{actions:["decrement"]}},states:{idle:{on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","invokeOnFocus","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","invokeOnFocus","setHint","setCursorPoint"]},"INPUT.FOCUS":{target:"focused",actions:["focusInput","invokeOnFocus"]}}},focused:{tags:["focus"],effects:["attachWheelListener"],on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","setHint","setCursorPoint"]},"INPUT.ARROW_UP":{actions:["increment"]},"INPUT.ARROW_DOWN":{actions:["decrement"]},"INPUT.HOME":{actions:["decrementToMin"]},"INPUT.END":{actions:["incrementToMax"]},"INPUT.CHANGE":{actions:["setValue","setHint"]},"INPUT.BLUR":[{guard:Fd("clampValueOnBlur",Vd("isInRange")),target:"idle",actions:["setClampedValue","clearHint","invokeOnBlur"]},{guard:Vd("isInRange"),target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur","invokeOnInvalid"]},{target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur"]}],"INPUT.ENTER":{actions:["setFormattedValue","clearHint","invokeOnBlur"]}}},"before:spin":{tags:["focus"],effects:["trackButtonDisabled","waitForChangeDelay"],entry:bk([{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}]),on:{CHANGE_DELAY:{target:"spinning",guard:Fd("isInRange","spinOnPress")},"TRIGGER.PRESS_UP":[{guard:"isTouchPointer",target:"focused",actions:["clearHint"]},{target:"focused",actions:["focusInput","clearHint"]}]}},spinning:{tags:["focus"],effects:["trackButtonDisabled","spinValue"],on:{SPIN:[{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}],"TRIGGER.PRESS_UP":{target:"focused",actions:["focusInput","clearHint"]}}},scrubbing:{tags:["focus"],effects:["activatePointerLock","trackMousemove","setupVirtualCursor","preventTextSelection"],on:{"SCRUBBER.POINTER_UP":{target:"focused",actions:["focusInput","clearCursorPoint"]},"SCRUBBER.POINTER_MOVE":[{guard:"isIncrementHint",actions:["increment","setCursorPoint"]},{guard:"isDecrementHint",actions:["decrement","setCursorPoint"]}]}}},implementations:{guards:{clampValueOnBlur:({prop:e})=>e("clampValueOnBlur"),spinOnPress:({prop:e})=>!!e("spinOnPress"),isInRange:({computed:e})=>!e("isOutOfRange"),isDecrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="decrement",isIncrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="increment",isTouchPointer:({event:e})=>e.pointerType==="touch"},effects:{waitForChangeDelay({send:e}){const t=setTimeout(()=>{e({type:"CHANGE_DELAY"})},300);return()=>clearTimeout(t)},spinValue({send:e}){const t=setInterval(()=>{e({type:"SPIN"})},50);return()=>clearInterval(t)},trackFormControl({context:e,scope:t}){const n=ii(t);return ys(n,{onFieldsetDisabledChange(r){e.set("fieldsetDisabled",r)},onFormReset(){e.set("value",e.initial("value"))}})},setupVirtualCursor({context:e,scope:t}){const n=e.get("scrubberCursorPoint");return ck(t,n)},preventTextSelection({scope:e}){return uk(e)},trackButtonDisabled({context:e,scope:t,send:n}){const r=e.get("hint"),o=lk(t,r);return Eo(o,{attributes:["disabled"],callback(){n({type:"TRIGGER.PRESS_UP",src:"attr"})}})},attachWheelListener({scope:e,send:t,prop:n}){const r=ii(e);if(!r||!e.isActiveElement(r)||!n("allowMouseWheel"))return;function o(i){i.preventDefault();const s=Math.sign(i.deltaY)*-1;s===1?t({type:"VALUE.INCREMENT"}):s===-1&&t({type:"VALUE.DECREMENT"})}return se(r,"wheel",o,{passive:!1})},activatePointerLock({scope:e}){if(!fc())return dv(e.getDoc())},trackMousemove({scope:e,send:t,context:n,computed:r}){const o=e.getDoc();function i(a){const l=n.get("scrubberCursorPoint"),c=r("isRtl"),u=dk(e,{point:l,isRtl:c,event:a});u.hint&&t({type:"SCRUBBER.POINTER_MOVE",hint:u.hint,point:u.point})}function s(){t({type:"SCRUBBER.POINTER_UP"})}return mo(se(o,"mousemove",i,!1),se(o,"mouseup",s,!1))}},actions:{focusInput({scope:e,prop:t}){if(!t("focusInputOnChange"))return;const n=ii(e);e.isActiveElement(n)||G(()=>n?.focus({preventScroll:!0}))},increment({context:e,event:t,prop:n,computed:r}){let o=v0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},decrement({context:e,event:t,prop:n,computed:r}){let o=b0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setClampedValue({context:e,prop:t,computed:n}){const r=Ve(n("valueAsNumber"),t("min"),t("max"));e.set("value",ln(r,{computed:n,prop:t}))},setRawValue({context:e,event:t,prop:n,computed:r}){let o=da(t.value,{computed:r,prop:n});n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setValue({context:e,event:t}){const n=t.target?.value??t.value;e.set("value",n)},clearValue({context:e}){e.set("value","")},incrementToMax({context:e,prop:t,computed:n}){const r=ln(t("max"),{computed:n,prop:t});e.set("value",r)},decrementToMin({context:e,prop:t,computed:n}){const r=ln(t("min"),{computed:n,prop:t});e.set("value",r)},setHint({context:e,event:t}){e.set("hint",t.hint)},clearHint({context:e}){e.set("hint",null)},invokeOnFocus({computed:e,prop:t}){t("onFocusChange")?.({focused:!0,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnBlur({computed:e,prop:t}){t("onFocusChange")?.({focused:!1,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnInvalid({computed:e,prop:t,event:n}){if(n.type==="INPUT.CHANGE")return;const r=e("valueAsNumber")>t("max")?"rangeOverflow":"rangeUnderflow";t("onValueInvalid")?.({reason:r,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},syncInputElement({context:e,event:t,computed:n,scope:r}){const o=t.type.endsWith("CHANGE")?e.get("value"):n("formattedValue"),i=ii(r),s=fk(i,r);G(()=>{So(i,o),gk(i,s,r)})},setFormattedValue({context:e,computed:t}){e.set("value",t("formattedValue"))},setCursorPoint({context:e,event:t}){e.set("scrubberCursorPoint",t.point)},clearCursorPoint({context:e}){e.set("scrubberCursorPoint",null)},setVirtualCursorPosition({context:e,scope:t}){const n=_d(t),r=e.get("scrubberCursorPoint");!n||!r||(n.style.transform=`translate3d(${r.x}px, ${r.y}px, 0px)`)}}}}),M()(["allowMouseWheel","allowOverflow","clampValueOnBlur","dir","disabled","focusInputOnChange","form","formatOptions","getRootNode","id","ids","inputMode","invalid","locale","max","min","name","onFocusChange","onValueChange","onValueInvalid","pattern","required","readOnly","spinOnPress","step","translations","value","defaultValue"]);var Ld=j("pinInput").parts("root","label","input","control");Ld.build(),M()(["autoFocus","blurOnComplete","count","defaultValue","dir","disabled","form","getRootNode","id","ids","invalid","mask","name","onValueChange","onValueComplete","onValueInvalid","otp","pattern","placeholder","readOnly","required","selectOnFocus","translations","type","value"]);var Dd=j("popover").parts("arrow","arrowTip","anchor","trigger","indicator","positioner","content","title","description","closeTrigger");Dd.build(),M()(["autoFocus","closeOnEscape","closeOnInteractOutside","dir","getRootNode","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","portalled","positioning"]);const cn=e=>{const{children:t,disabled:n}=e,[r,o]=P.useState(e.container?.current),i=P.useSyncExternalStore(Ck,()=>!1,()=>!0),{getRootNode:s}=sc();if(P.useEffect(()=>{o(()=>e.container?.current)},[e.container]),i||n)return g.jsx(g.Fragment,{children:t});const a=r??kk(s);return g.jsx(g.Fragment,{children:P.Children.map(t,l=>Ft.createPortal(l,a))})},kk=e=>{const t=e?.(),n=t.getRootNode();return jn(n)?n:Fe(t).body},Ck=()=>()=>{};var ha=j("progress").parts("root","label","track","range","valueText","view","circle","circleTrack","circleRange");ha.build(),M()(["dir","getRootNode","id","ids","max","min","orientation","translations","value","onValueChange","defaultValue","formatOptions","locale"]);var zd=j("qr-code").parts("root","frame","pattern","overlay","downloadTrigger");zd.build(),M()(["ids","defaultValue","value","id","encoding","dir","getRootNode","onValueChange","pixelSize"]);var fa=j("radio-group").parts("root","label","item","itemText","itemControl","indicator");fa.build(),M()(["dir","disabled","form","getRootNode","id","ids","name","onValueChange","orientation","readOnly","value","defaultValue"]),M()(["value","disabled","invalid"]);var Md=j("rating-group").parts("root","label","item","control");Md.build(),M()(["allowHalf","autoFocus","count","dir","disabled","form","getRootNode","id","ids","name","onHoverChange","onValueChange","required","readOnly","translations","value","defaultValue"]),M()(["index"]);var jd=j("scroll-area").parts("root","viewport","content","scrollbar","thumb","corner");jd.build(),M()(["dir","getRootNode","ids","id"]);const $d=fa.rename("segment-group");$d.build();var Bd=j("select").parts("label","positioner","trigger","indicator","clearTrigger","item","itemText","itemIndicator","itemGroup","itemGroupLabel","list","content","root","control","valueText");Bd.build();var Wd=e=>new Yo(e);Wd.empty=()=>new Yo({items:[]});var wk=e=>e.ids?.content??`select:${e.id}:content`,Sk=e=>e.ids?.trigger??`select:${e.id}:trigger`,Ek=e=>e.ids?.clearTrigger??`select:${e.id}:clear-trigger`,Ok=(e,t)=>e.ids?.item?.(t)??`select:${e.id}:option:${t}`,Pk=e=>e.ids?.hiddenSelect??`select:${e.id}:select`,Rk=e=>e.ids?.positioner??`select:${e.id}:positioner`,ga=e=>e.getById(Pk(e)),zr=e=>e.getById(wk(e)),si=e=>e.getById(Sk(e)),Ik=e=>e.getById(Ek(e)),Hd=e=>e.getById(Rk(e)),pa=(e,t)=>t==null?null:e.getById(Ok(e,t)),{and:Mr,not:un,or:Tk}=jt();Tk("isTriggerArrowDownEvent","isTriggerEnterEvent"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),un("multiple"),un("multiple"),un("multiple"),Mr("closeOnSelect","isOpenControlled"),Mr("hasHighlightedItem","loop","isLastItemHighlighted"),Mr("hasHighlightedItem","loop","isFirstItemHighlighted");function Ud(e){const t=e.restoreFocus??e.previousEvent?.restoreFocus;return t==null||!!t}M()(["closeOnSelect","collection","composite","defaultHighlightedValue","defaultOpen","defaultValue","deselectable","dir","disabled","form","getRootNode","highlightedValue","id","ids","invalid","loopFocus","multiple","name","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","positioning","readOnly","required","scrollToIndexFn","value"]),M()(["item","persistFocus"]),M()(["id"]),M()(["htmlFor"]);var Gd=j("slider").parts("root","label","thumb","valueText","track","range","control","markerGroup","marker","draggingIndicator");Gd.build(),M()(["aria-label","aria-labelledby","dir","disabled","form","getAriaValueText","getRootNode","id","ids","invalid","max","min","minStepsBetweenThumbs","name","onFocusChange","onValueChange","onValueChangeEnd","orientation","origin","readOnly","step","thumbAlignment","thumbAlignment","thumbSize","value","defaultValue"]),M()(["index","name"]);var qd=j("switch").parts("root","label","control","thumb");qd.build(),M()(["checked","defaultChecked","dir","disabled","form","getRootNode","id","ids","invalid","label","name","onCheckedChange","readOnly","required","value"]);var Nk=j("toast").parts("group","root","title","description","actionTrigger","closeTrigger");Nk.build();var Ak=(e,t)=>({...t,...bo(e)});function _k(e={}){const t=Ak(e,{placement:"bottom",overlap:!1,max:24,gap:16,offsets:"1rem",hotkey:["altKey","KeyT"],removeDelay:200,pauseOnPageIdle:!0});let n=[],r=[],o=new Set,i=[];const s=S=>(n.push(S),()=>{const _=n.indexOf(S);n.splice(_,1)}),a=S=>(n.forEach(_=>_(S)),S),l=S=>{if(r.length>=t.max){i.push(S);return}a(S),r.unshift(S)},c=()=>{for(;i.length>0&&r.length{const _=S.id??`toast:${i0()}`,D=r.find(z=>z.id===_);return o.has(_)&&o.delete(_),D?r=r.map(z=>z.id===_?a({...z,...S,id:_}):z):l({id:_,duration:t.duration,removeDelay:t.removeDelay,type:"info",...S,stacked:!t.overlap,gap:t.gap}),_},h=S=>(o.add(S),S?(n.forEach(_=>_({id:S,dismiss:!0})),r=r.filter(_=>_.id!==S),c()):(r.forEach(_=>{n.forEach(D=>D({id:_.id,dismiss:!0}))}),r=[],i=[]),S);return{attrs:t,subscribe:s,create:u,update:(S,_)=>u({id:S,..._}),remove:h,dismiss:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"DISMISS"}):_):r=r.map(_=>a({..._,message:"DISMISS"}))},error:S=>u({...S,type:"error"}),success:S=>u({...S,type:"success"}),info:S=>u({...S,type:"info"}),warning:S=>u({...S,type:"warning"}),loading:S=>u({...S,type:"loading"}),getVisibleToasts:()=>r.filter(S=>!o.has(S.id)),getCount:()=>r.length,promise:(S,_,D={})=>{if(!_||!_.loading){wr("[zag-js > toast] toaster.promise() requires at least a 'loading' option to be specified");return}const z=u({...D,..._.loading,promise:S,type:"loading"});let W=!0,J;const ee=po(S).then(async q=>{if(J=["resolve",q],Vk(q)&&!q.ok){W=!1;const Z=po(_.error,`HTTP Error! status: ${q.status}`);u({...D,...Z,id:z,type:"error"})}else if(_.success!==void 0){W=!1;const Z=po(_.success,q);u({...D,...Z,id:z,type:"success"})}}).catch(async q=>{if(J=["reject",q],_.error!==void 0){W=!1;const Z=po(_.error,q);u({...D,...Z,id:z,type:"error"})}}).finally(()=>{W&&h(z),_.finally?.()});return{id:z,unwrap:()=>new Promise((q,Z)=>ee.then(()=>J[0]==="reject"?Z(J[1]):q(J[1])).catch(Z))}},pause:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"PAUSE"}):_):r=r.map(_=>a({..._,message:"PAUSE"}))},resume:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"RESUME"}):_):r=r.map(_=>a({..._,message:"RESUME"}))},isVisible:S=>!o.has(S)&&!!r.find(_=>_.id===S),isDismissed:S=>o.has(S),expand:()=>{r=r.map(S=>a({...S,stacked:!0}))},collapse:()=>{r=r.map(S=>a({...S,stacked:!1}))}}}var Vk=e=>e&&typeof e=="object"&&"ok"in e&&typeof e.ok=="boolean"&&"status"in e&&typeof e.status=="number";const Fk=e=>_k(e);var Kd=j("tooltip").parts("trigger","arrow","arrowTip","positioner","content");Kd.build();var Lk=e=>e.ids?.trigger??`tooltip:${e.id}:trigger`,Dk=e=>e.ids?.positioner??`tooltip:${e.id}:popper`,ma=e=>e.getById(Lk(e)),Yd=e=>e.getById(Dk(e)),dn=y0({id:null}),{and:zk,not:Xd}=jt();zk("noVisibleTooltip",Xd("hasPointerMoveOpened")),Xd("hasPointerMoveOpened"),M()(["aria-label","closeDelay","closeOnEscape","closeOnPointerDown","closeOnScroll","closeOnClick","dir","disabled","getRootNode","id","ids","interactive","onOpenChange","defaultOpen","open","openDelay","positioning"]);function Qd(e,t=[]){const n=Object.assign({},e);for(const r of t)r in n&&delete n[r];return n}const Mk=(e,t)=>{if(!e||typeof e!="string")return{invalid:!0,value:e};const[n,r]=e.split("/");if(!n||!r||n==="currentBg")return{invalid:!0,value:n};const o=t(`colors.${n}`),i=t.raw(`opacity.${r}`)?.value;if(!i&&isNaN(Number(r)))return{invalid:!0,value:n};const s=i?Number(i)*100+"%":`${r}%`,a=o??n;return{invalid:!1,color:a,value:`color-mix(in srgb, ${a} ${s}, transparent)`}},te=e=>(t,n)=>{const r=n.utils.colorMix(t);if(r.invalid)return{[e]:t};const o="--mix-"+e;return{[o]:r.value,[e]:`var(${o}, ${r.color})`}};function va(e){if(e===null||typeof e!="object")return e;if(Array.isArray(e))return e.map(n=>va(n));const t=Object.create(Object.getPrototypeOf(e));for(const n of Object.keys(e))t[n]=va(e[n]);return t}function ba(e,t){if(t==null)return e;for(const n of Object.keys(t))if(!(t[n]===void 0||n==="__proto__"))if(!_e(e[n])&&_e(t[n]))Object.assign(e,{[n]:t[n]});else if(e[n]&&_e(t[n]))ba(e[n],t[n]);else if(Array.isArray(t[n])&&Array.isArray(e[n])){let r=0;for(;re!=null;function gt(e,t,n={}){const{stop:r,getKey:o}=n;function i(s,a=[]){if(_e(s)||Array.isArray(s)){const l={};for(const[c,u]of Object.entries(s)){const h=o?.(c,u)??c,d=[...a,h];if(r?.(s,d))return t(s,a);const m=i(u,d);ya(m)&&(l[h]=m)}return l}return t(s,a)}return i(e)}function Jd(e,t){return Array.isArray(e)?e.map(n=>ya(n)?t(n):n):_e(e)?gt(e,n=>t(n)):ya(e)?t(e):e}const ai=["value","type","description"],jk=e=>e&&typeof e=="object"&&!Array.isArray(e),Zd=(...e)=>{const t=er({},...e.map(va));return t.theme?.tokens&>(t.theme.tokens,n=>{const i=Object.keys(n).filter(a=>!ai.includes(a)).length>0,s=ai.some(a=>n[a]!=null);return i&&s&&(n.DEFAULT||(n.DEFAULT={}),ai.forEach(a=>{var l;n[a]!=null&&((l=n.DEFAULT)[a]||(l[a]=n[a]),delete n[a])})),n},{stop(n){return jk(n)&&Object.keys(n).some(r=>ai.includes(r)||r!==r.toLowerCase()&&r!==r.toUpperCase())}}),t},$k=e=>e,fe=e=>e,B=e=>e,Bk=e=>e,Wk=e=>e,tr=e=>e,Hk=e=>e,Uk=e=>e,Gk=e=>e;function eh(){const e=t=>t;return new Proxy(e,{get(){return e}})}const de=eh(),xa=eh(),ka=e=>e,qk=/[^a-zA-Z0-9_\u0081-\uffff-]/g;function Kk(e){return`${e}`.replace(qk,t=>`\\${t}`)}const Yk=/[A-Z]/g;function Xk(e){return e.replace(Yk,t=>`-${t.toLowerCase()}`)}function th(e,t={}){const{fallback:n="",prefix:r=""}=t,o=Xk(["-",r,Kk(e)].filter(Boolean).join("-"));return{var:o,ref:`var(${o}${n?`, ${n}`:""})`}}const Qk=e=>/^var\(--.+\)$/.test(e),xe=(e,t)=>t!=null?`${e}(${t})`:t,hn=e=>{if(Qk(e)||e==null)return e;const t=typeof e=="string"&&!e.endsWith("deg");return typeof e=="number"||t?`${e}deg`:e},nh=e=>({values:["outside","inside","mixed","none"],transform(t,{token:n}){const r=n("colors.colorPalette.focusRing");return{inside:{"--focus-ring-color":r,[e]:{outlineOffset:"0px",outlineWidth:"var(--focus-ring-width, 1px)",outlineColor:"var(--focus-ring-color)",outlineStyle:"var(--focus-ring-style, solid)",borderColor:"var(--focus-ring-color)"}},outside:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 2px)",outlineOffset:"var(--focus-ring-offset, 2px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"var(--focus-ring-color)"}},mixed:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 3px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"color-mix(in srgb, var(--focus-ring-color), transparent 60%)",borderColor:"var(--focus-ring-color)"}},none:{"--focus-ring-color":r,[e]:{outline:"none"}}}[t]??{}}}),Jk=te("borderColor"),St=e=>({transition:e,transitionTimingFunction:"cubic-bezier(0.4, 0, 0.2, 1)",transitionDuration:"150ms"}),Zk=$k({hover:["@media (hover: hover)","&:is(:hover, [data-hover]):not(:disabled, [data-disabled])"],active:"&:is(:active, [data-active]):not(:disabled, [data-disabled], [data-state=open])",focus:"&:is(:focus, [data-focus])",focusWithin:"&:is(:focus-within, [data-focus-within])",focusVisible:"&:is(:focus-visible, [data-focus-visible])",disabled:"&:is(:disabled, [disabled], [data-disabled], [aria-disabled=true])",visited:"&:visited",target:"&:target",readOnly:"&:is([data-readonly], [aria-readonly=true], [readonly])",readWrite:"&:read-write",empty:"&:is(:empty, [data-empty])",checked:"&:is(:checked, [data-checked], [aria-checked=true], [data-state=checked])",enabled:"&:enabled",expanded:"&:is([aria-expanded=true], [data-expanded], [data-state=expanded])",highlighted:"&[data-highlighted]",complete:"&[data-complete]",incomplete:"&[data-incomplete]",dragging:"&[data-dragging]",before:"&::before",after:"&::after",firstLetter:"&::first-letter",firstLine:"&::first-line",marker:"&::marker",selection:"&::selection",file:"&::file-selector-button",backdrop:"&::backdrop",first:"&:first-of-type",last:"&:last-of-type",notFirst:"&:not(:first-of-type)",notLast:"&:not(:last-of-type)",only:"&:only-child",even:"&:nth-of-type(even)",odd:"&:nth-of-type(odd)",peerFocus:".peer:is(:focus, [data-focus]) ~ &",peerHover:".peer:is(:hover, [data-hover]):not(:disabled, [data-disabled]) ~ &",peerActive:".peer:is(:active, [data-active]):not(:disabled, [data-disabled]) ~ &",peerFocusWithin:".peer:focus-within ~ &",peerFocusVisible:".peer:is(:focus-visible, [data-focus-visible]) ~ &",peerDisabled:".peer:is(:disabled, [disabled], [data-disabled]) ~ &",peerChecked:".peer:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) ~ &",peerInvalid:".peer:is(:invalid, [data-invalid], [aria-invalid=true]) ~ &",peerExpanded:".peer:is([aria-expanded=true], [data-expanded], [data-state=expanded]) ~ &",peerPlaceholderShown:".peer:placeholder-shown ~ &",groupFocus:".group:is(:focus, [data-focus]) &",groupHover:".group:is(:hover, [data-hover]):not(:disabled, [data-disabled]) &",groupActive:".group:is(:active, [data-active]):not(:disabled, [data-disabled]) &",groupFocusWithin:".group:focus-within &",groupFocusVisible:".group:is(:focus-visible, [data-focus-visible]) &",groupDisabled:".group:is(:disabled, [disabled], [data-disabled]) &",groupChecked:".group:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) &",groupExpanded:".group:is([aria-expanded=true], [data-expanded], [data-state=expanded]) &",groupInvalid:".group:invalid &",indeterminate:"&:is(:indeterminate, [data-indeterminate], [aria-checked=mixed], [data-state=indeterminate])",required:"&:is([data-required], [aria-required=true])",valid:"&:is([data-valid], [data-state=valid])",invalid:"&:is([data-invalid], [aria-invalid=true], [data-state=invalid])",autofill:"&:autofill",inRange:"&:is(:in-range, [data-in-range])",outOfRange:"&:is(:out-of-range, [data-outside-range])",placeholder:"&::placeholder, &[data-placeholder]",placeholderShown:"&:is(:placeholder-shown, [data-placeholder-shown])",pressed:"&:is([aria-pressed=true], [data-pressed])",selected:"&:is([aria-selected=true], [data-selected])",grabbed:"&:is([aria-grabbed=true], [data-grabbed])",underValue:"&[data-state=under-value]",overValue:"&[data-state=over-value]",atValue:"&[data-state=at-value]",default:"&:default",optional:"&:optional",open:"&:is([open], [data-open], [data-state=open])",closed:"&:is([closed], [data-closed], [data-state=closed])",fullscreen:"&:is(:fullscreen, [data-fullscreen])",loading:"&:is([data-loading], [aria-busy=true])",hidden:"&:is([hidden], [data-hidden])",current:"&[data-current]",currentPage:"&[aria-current=page]",currentStep:"&[aria-current=step]",today:"&[data-today]",unavailable:"&[data-unavailable]",rangeStart:"&[data-range-start]",rangeEnd:"&[data-range-end]",now:"&[data-now]",topmost:"&[data-topmost]",motionReduce:"@media (prefers-reduced-motion: reduce)",motionSafe:"@media (prefers-reduced-motion: no-preference)",print:"@media print",landscape:"@media (orientation: landscape)",portrait:"@media (orientation: portrait)",dark:".dark &, .dark .chakra-theme:not(.light) &",light:":root &, .light &",osDark:"@media (prefers-color-scheme: dark)",osLight:"@media (prefers-color-scheme: light)",highContrast:"@media (forced-colors: active)",lessContrast:"@media (prefers-contrast: less)",moreContrast:"@media (prefers-contrast: more)",ltr:"[dir=ltr] &",rtl:"[dir=rtl] &",scrollbar:"&::-webkit-scrollbar",scrollbarThumb:"&::-webkit-scrollbar-thumb",scrollbarTrack:"&::-webkit-scrollbar-track",horizontal:"&[data-orientation=horizontal]",vertical:"&[data-orientation=vertical]",icon:"& :where(svg)",starting:"@starting-style"}),nr=th("bg-currentcolor"),rh=e=>e===nr.ref||e==="currentBg",ne=e=>({...e("colors"),currentBg:nr}),eC=ka({conditions:Zk,utilities:{background:{values:ne,shorthand:["bg"],transform(e,t){if(rh(t.raw))return{background:nr.ref};const n=te("background")(e,t);return{...n,[nr.var]:n?.background}}},backgroundColor:{values:ne,shorthand:["bgColor"],transform(e,t){if(rh(t.raw))return{backgroundColor:nr.ref};const n=te("backgroundColor")(e,t);return{...n,[nr.var]:n?.backgroundColor}}},backgroundSize:{shorthand:["bgSize"]},backgroundPosition:{shorthand:["bgPos"]},backgroundRepeat:{shorthand:["bgRepeat"]},backgroundAttachment:{shorthand:["bgAttachment"]},backgroundClip:{shorthand:["bgClip"],values:["text"],transform(e){return e==="text"?{color:"transparent",backgroundClip:"text"}:{backgroundClip:e}}},backgroundGradient:{shorthand:["bgGradient"],values(e){return{...e("gradients"),"to-t":"linear-gradient(to top, var(--gradient))","to-tr":"linear-gradient(to top right, var(--gradient))","to-r":"linear-gradient(to right, var(--gradient))","to-br":"linear-gradient(to bottom right, var(--gradient))","to-b":"linear-gradient(to bottom, var(--gradient))","to-bl":"linear-gradient(to bottom left, var(--gradient))","to-l":"linear-gradient(to left, var(--gradient))","to-tl":"linear-gradient(to top left, var(--gradient))"}},transform(e){return{"--gradient-stops":"var(--gradient-from), var(--gradient-to)","--gradient":"var(--gradient-via-stops, var(--gradient-stops))",backgroundImage:e}}},gradientFrom:{values:ne,transform:te("--gradient-from")},gradientTo:{values:ne,transform:te("--gradient-to")},gradientVia:{values:ne,transform(e,t){return{...te("--gradient-via")(e,t),"--gradient-via-stops":"var(--gradient-from), var(--gradient-via), var(--gradient-to)"}}},backgroundImage:{values(e){return{...e("gradients"),...e("assets")}},shorthand:["bgImg","bgImage"]},border:{values:"borders"},borderTop:{values:"borders"},borderLeft:{values:"borders"},borderBlockStart:{values:"borders"},borderRight:{values:"borders"},borderBottom:{values:"borders"},borderBlockEnd:{values:"borders"},borderInlineStart:{values:"borders",shorthand:["borderStart"]},borderInlineEnd:{values:"borders",shorthand:["borderEnd"]},borderInline:{values:"borders",shorthand:["borderX"]},borderBlock:{values:"borders",shorthand:["borderY"]},borderColor:{values:ne,transform:te("borderColor")},borderTopColor:{values:ne,transform:te("borderTopColor")},borderBlockStartColor:{values:ne,transform:te("borderBlockStartColor")},borderBottomColor:{values:ne,transform:te("borderBottomColor")},borderBlockEndColor:{values:ne,transform:te("borderBlockEndColor")},borderLeftColor:{values:ne,transform:te("borderLeftColor")},borderInlineStartColor:{values:ne,shorthand:["borderStartColor"],transform:te("borderInlineStartColor")},borderRightColor:{values:ne,transform:te("borderRightColor")},borderInlineEndColor:{values:ne,shorthand:["borderEndColor"],transform:te("borderInlineEndColor")},borderStyle:{values:"borderStyles"},borderTopStyle:{values:"borderStyles"},borderBlockStartStyle:{values:"borderStyles"},borderBottomStyle:{values:"borderStyles"},borderBlockEndStyle:{values:"borderStyles"},borderInlineStartStyle:{values:"borderStyles",shorthand:["borderStartStyle"]},borderInlineEndStyle:{values:"borderStyles",shorthand:["borderEndStyle"]},borderLeftStyle:{values:"borderStyles"},borderRightStyle:{values:"borderStyles"},borderRadius:{values:"radii",shorthand:["rounded"]},borderTopLeftRadius:{values:"radii",shorthand:["roundedTopLeft"]},borderStartStartRadius:{values:"radii",shorthand:["roundedStartStart","borderTopStartRadius"]},borderEndStartRadius:{values:"radii",shorthand:["roundedEndStart","borderBottomStartRadius"]},borderTopRightRadius:{values:"radii",shorthand:["roundedTopRight"]},borderStartEndRadius:{values:"radii",shorthand:["roundedStartEnd","borderTopEndRadius"]},borderEndEndRadius:{values:"radii",shorthand:["roundedEndEnd","borderBottomEndRadius"]},borderBottomLeftRadius:{values:"radii",shorthand:["roundedBottomLeft"]},borderBottomRightRadius:{values:"radii",shorthand:["roundedBottomRight"]},borderInlineStartRadius:{values:"radii",property:"borderRadius",shorthand:["roundedStart","borderStartRadius"],transform:e=>({borderStartStartRadius:e,borderEndStartRadius:e})},borderInlineEndRadius:{values:"radii",property:"borderRadius",shorthand:["roundedEnd","borderEndRadius"],transform:e=>({borderStartEndRadius:e,borderEndEndRadius:e})},borderTopRadius:{values:"radii",property:"borderRadius",shorthand:["roundedTop"],transform:e=>({borderTopLeftRadius:e,borderTopRightRadius:e})},borderBottomRadius:{values:"radii",property:"borderRadius",shorthand:["roundedBottom"],transform:e=>({borderBottomLeftRadius:e,borderBottomRightRadius:e})},borderLeftRadius:{values:"radii",property:"borderRadius",shorthand:["roundedLeft"],transform:e=>({borderTopLeftRadius:e,borderBottomLeftRadius:e})},borderRightRadius:{values:"radii",property:"borderRadius",shorthand:["roundedRight"],transform:e=>({borderTopRightRadius:e,borderBottomRightRadius:e})},borderWidth:{values:"borderWidths"},borderBlockStartWidth:{values:"borderWidths"},borderTopWidth:{values:"borderWidths"},borderBottomWidth:{values:"borderWidths"},borderBlockEndWidth:{values:"borderWidths"},borderRightWidth:{values:"borderWidths"},borderInlineWidth:{values:"borderWidths",shorthand:["borderXWidth"]},borderInlineStartWidth:{values:"borderWidths",shorthand:["borderStartWidth"]},borderInlineEndWidth:{values:"borderWidths",shorthand:["borderEndWidth"]},borderLeftWidth:{values:"borderWidths"},borderBlockWidth:{values:"borderWidths",shorthand:["borderYWidth"]},color:{values:ne,transform:te("color")},fill:{values:ne,transform:te("fill")},stroke:{values:ne,transform:te("stroke")},accentColor:{values:ne,transform:te("accentColor")},divideX:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderInlineStartWidth:e,borderInlineEndWidth:"0px"}}}},divideY:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderTopWidth:e,borderBottomWidth:"0px"}}}},divideColor:{values:ne,transform(e,t){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":Jk(e,t)}}},divideStyle:{property:"borderStyle",transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderStyle:e}}}},boxShadow:{values:"shadows",shorthand:["shadow"]},boxShadowColor:{values:ne,transform:te("--shadow-color"),shorthand:["shadowColor"]},mixBlendMode:{shorthand:["blendMode"]},backgroundBlendMode:{shorthand:["bgBlendMode"]},opacity:{values:"opacity"},filter:{transform(e){return e!=="auto"?{filter:e}:{filter:"var(--blur) var(--brightness) var(--contrast) var(--grayscale) var(--hue-rotate) var(--invert) var(--saturate) var(--sepia) var(--drop-shadow)"}}},blur:{values:"blurs",transform:e=>({"--blur":xe("blur",e)})},brightness:{transform:e=>({"--brightness":xe("brightness",e)})},contrast:{transform:e=>({"--contrast":xe("contrast",e)})},grayscale:{transform:e=>({"--grayscale":xe("grayscale",e)})},hueRotate:{transform:e=>({"--hue-rotate":xe("hue-rotate",hn(e))})},invert:{transform:e=>({"--invert":xe("invert",e)})},saturate:{transform:e=>({"--saturate":xe("saturate",e)})},sepia:{transform:e=>({"--sepia":xe("sepia",e)})},dropShadow:{transform:e=>({"--drop-shadow":xe("drop-shadow",e)})},backdropFilter:{transform(e){return e!=="auto"?{backdropFilter:e}:{backdropFilter:"var(--backdrop-blur) var(--backdrop-brightness) var(--backdrop-contrast) var(--backdrop-grayscale) var(--backdrop-hue-rotate) var(--backdrop-invert) var(--backdrop-opacity) var(--backdrop-saturate) var(--backdrop-sepia)"}}},backdropBlur:{values:"blurs",transform:e=>({"--backdrop-blur":xe("blur",e)})},backdropBrightness:{transform:e=>({"--backdrop-brightness":xe("brightness",e)})},backdropContrast:{transform:e=>({"--backdrop-contrast":xe("contrast",e)})},backdropGrayscale:{transform:e=>({"--backdrop-grayscale":xe("grayscale",e)})},backdropHueRotate:{transform:e=>({"--backdrop-hue-rotate":xe("hue-rotate",hn(e))})},backdropInvert:{transform:e=>({"--backdrop-invert":xe("invert",e)})},backdropOpacity:{transform:e=>({"--backdrop-opacity":xe("opacity",e)})},backdropSaturate:{transform:e=>({"--backdrop-saturate":xe("saturate",e)})},backdropSepia:{transform:e=>({"--backdrop-sepia":xe("sepia",e)})},flexBasis:{values:"sizes"},gap:{values:"spacing"},rowGap:{values:"spacing",shorthand:["gapY"]},columnGap:{values:"spacing",shorthand:["gapX"]},flexDirection:{shorthand:["flexDir"]},gridGap:{values:"spacing"},gridColumnGap:{values:"spacing"},gridRowGap:{values:"spacing"},outlineColor:{values:ne,transform:te("outlineColor")},focusRing:nh("&:is(:focus, [data-focus])"),focusVisibleRing:nh("&:is(:focus-visible, [data-focus-visible])"),focusRingColor:{values:ne,transform:te("--focus-ring-color")},focusRingOffset:{values:"spacing",transform:e=>({"--focus-ring-offset":e})},focusRingWidth:{values:"borderWidths",property:"outlineWidth",transform:e=>({"--focus-ring-width":e})},focusRingStyle:{values:"borderStyles",property:"outlineStyle",transform:e=>({"--focus-ring-style":e})},aspectRatio:{values:"aspectRatios"},width:{values:"sizes",shorthand:["w"]},inlineSize:{values:"sizes"},height:{values:"sizes",shorthand:["h"]},blockSize:{values:"sizes"},boxSize:{values:"sizes",property:"width",transform:e=>({width:e,height:e})},minWidth:{values:"sizes",shorthand:["minW"]},minInlineSize:{values:"sizes"},minHeight:{values:"sizes",shorthand:["minH"]},minBlockSize:{values:"sizes"},maxWidth:{values:"sizes",shorthand:["maxW"]},maxInlineSize:{values:"sizes"},maxHeight:{values:"sizes",shorthand:["maxH"]},maxBlockSize:{values:"sizes"},hideFrom:{values:"breakpoints",transform:(e,{raw:t,token:n})=>({[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}`:`@media screen and (min-width: ${e})`]:{display:"none"}})},hideBelow:{values:"breakpoints",transform(e,{raw:t,token:n}){return{[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}Down`:`@media screen and (max-width: ${e})`]:{display:"none"}}}},overscrollBehavior:{shorthand:["overscroll"]},overscrollBehaviorX:{shorthand:["overscrollX"]},overscrollBehaviorY:{shorthand:["overscrollY"]},scrollbar:{values:["visible","hidden"],transform(e){switch(e){case"visible":return{msOverflowStyle:"auto",scrollbarWidth:"auto","&::-webkit-scrollbar":{display:"block"}};case"hidden":return{msOverflowStyle:"none",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}};default:return{}}}},scrollbarColor:{values:ne,transform:te("scrollbarColor")},scrollbarGutter:{values:"spacing"},scrollbarWidth:{values:"sizes"},scrollMargin:{values:"spacing"},scrollMarginTop:{values:"spacing"},scrollMarginBottom:{values:"spacing"},scrollMarginLeft:{values:"spacing"},scrollMarginRight:{values:"spacing"},scrollMarginX:{values:"spacing",transform:e=>({scrollMarginLeft:e,scrollMarginRight:e})},scrollMarginY:{values:"spacing",transform:e=>({scrollMarginTop:e,scrollMarginBottom:e})},scrollPadding:{values:"spacing"},scrollPaddingTop:{values:"spacing"},scrollPaddingBottom:{values:"spacing"},scrollPaddingLeft:{values:"spacing"},scrollPaddingRight:{values:"spacing"},scrollPaddingInline:{values:"spacing",shorthand:["scrollPaddingX"]},scrollPaddingBlock:{values:"spacing",shorthand:["scrollPaddingY"]},scrollSnapType:{values:{none:"none",x:"x var(--scroll-snap-strictness)",y:"y var(--scroll-snap-strictness)",both:"both var(--scroll-snap-strictness)"}},scrollSnapStrictness:{values:["mandatory","proximity"],transform:e=>({"--scroll-snap-strictness":e})},scrollSnapMargin:{values:"spacing"},scrollSnapMarginTop:{values:"spacing"},scrollSnapMarginBottom:{values:"spacing"},scrollSnapMarginLeft:{values:"spacing"},scrollSnapMarginRight:{values:"spacing"},listStylePosition:{shorthand:["listStylePos"]},listStyleImage:{values:"assets",shorthand:["listStyleImg"]},position:{shorthand:["pos"]},zIndex:{values:"zIndex"},inset:{values:"spacing"},insetInline:{values:"spacing",shorthand:["insetX"]},insetBlock:{values:"spacing",shorthand:["insetY"]},top:{values:"spacing"},insetBlockStart:{values:"spacing"},bottom:{values:"spacing"},insetBlockEnd:{values:"spacing"},left:{values:"spacing"},right:{values:"spacing"},insetInlineStart:{values:"spacing",shorthand:["insetStart"]},insetInlineEnd:{values:"spacing",shorthand:["insetEnd"]},ring:{transform(e){return{"--ring-offset-shadow":"var(--ring-inset) 0 0 0 var(--ring-offset-width) var(--ring-offset-color)","--ring-shadow":"var(--ring-inset) 0 0 0 calc(var(--ring-width) + var(--ring-offset-width)) var(--ring-color)","--ring-width":e,boxShadow:"var(--ring-offset-shadow), var(--ring-shadow), var(--shadow, 0 0 #0000)"}}},ringColor:{values:ne,transform:te("--ring-color")},ringOffset:{transform:e=>({"--ring-offset-width":e})},ringOffsetColor:{values:ne,transform:te("--ring-offset-color")},ringInset:{transform:e=>({"--ring-inset":e})},margin:{values:"spacing",shorthand:["m"]},marginTop:{values:"spacing",shorthand:["mt"]},marginBlockStart:{values:"spacing"},marginRight:{values:"spacing",shorthand:["mr"]},marginBottom:{values:"spacing",shorthand:["mb"]},marginBlockEnd:{values:"spacing"},marginLeft:{values:"spacing",shorthand:["ml"]},marginInlineStart:{values:"spacing",shorthand:["ms","marginStart"]},marginInlineEnd:{values:"spacing",shorthand:["me","marginEnd"]},marginInline:{values:"spacing",shorthand:["mx","marginX"]},marginBlock:{values:"spacing",shorthand:["my","marginY"]},padding:{values:"spacing",shorthand:["p"]},paddingTop:{values:"spacing",shorthand:["pt"]},paddingRight:{values:"spacing",shorthand:["pr"]},paddingBottom:{values:"spacing",shorthand:["pb"]},paddingBlockStart:{values:"spacing"},paddingBlockEnd:{values:"spacing"},paddingLeft:{values:"spacing",shorthand:["pl"]},paddingInlineStart:{values:"spacing",shorthand:["ps","paddingStart"]},paddingInlineEnd:{values:"spacing",shorthand:["pe","paddingEnd"]},paddingInline:{values:"spacing",shorthand:["px","paddingX"]},paddingBlock:{values:"spacing",shorthand:["py","paddingY"]},textDecoration:{shorthand:["textDecor"]},textDecorationColor:{values:ne,transform:te("textDecorationColor")},textShadow:{values:"shadows"},transform:{transform:e=>{let t=e;return e==="auto"&&(t="translateX(var(--translate-x, 0)) translateY(var(--translate-y, 0)) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),e==="auto-gpu"&&(t="translate3d(var(--translate-x, 0), var(--translate-y, 0), 0) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),{transform:t}}},skewX:{transform:e=>({"--skew-x":hn(e)})},skewY:{transform:e=>({"--skew-y":hn(e)})},scaleX:{transform:e=>({"--scale-x":e})},scaleY:{transform:e=>({"--scale-y":e})},scale:{transform(e){return e!=="auto"?{scale:e}:{scale:"var(--scale-x, 1) var(--scale-y, 1)"}}},spaceXReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":e?"1":void 0}}}},spaceX:{property:"marginInlineStart",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":"0",marginInlineStart:`calc(${e} * calc(1 - var(--space-x-reverse)))`,marginInlineEnd:`calc(${e} * var(--space-x-reverse))`}})},spaceYReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":e?"1":void 0}}}},spaceY:{property:"marginTop",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":"0",marginTop:`calc(${e} * calc(1 - var(--space-y-reverse)))`,marginBottom:`calc(${e} * var(--space-y-reverse))`}})},rotate:{transform(e){return e!=="auto"?{rotate:hn(e)}:{rotate:"var(--rotate-x, 0) var(--rotate-y, 0) var(--rotate-z, 0)"}}},rotateX:{transform(e){return{"--rotate-x":hn(e)}}},rotateY:{transform(e){return{"--rotate-y":hn(e)}}},translate:{transform(e){return e!=="auto"?{translate:e}:{translate:"var(--translate-x) var(--translate-y)"}}},translateX:{values:"spacing",transform:e=>({"--translate-x":e})},translateY:{values:"spacing",transform:e=>({"--translate-y":e})},transition:{values:["all","common","colors","opacity","position","backgrounds","size","shadow","transform"],transform(e){switch(e){case"all":return St("all");case"position":return St("left, right, top, bottom, inset-inline, inset-block");case"colors":return St("color, background-color, border-color, text-decoration-color, fill, stroke");case"opacity":return St("opacity");case"shadow":return St("box-shadow");case"transform":return St("transform");case"size":return St("width, height");case"backgrounds":return St("background, background-color, background-image, background-position");case"common":return St("color, background-color, border-color, text-decoration-color, fill, stroke, opacity, box-shadow, transform, filter, backdrop-filter");default:return{transition:e}}}},transitionDuration:{values:"durations"},transitionProperty:{values:{common:"background-color, border-color, color, fill, stroke, opacity, box-shadow, translate, transform",colors:"background-color, border-color, color, fill, stroke",size:"width, height",position:"left, right, top, bottom, inset-inline, inset-block",background:"background, background-color, background-image, background-position"}},transitionTimingFunction:{values:"easings"},animation:{values:"animations"},animationDuration:{values:"durations"},animationDelay:{values:"durations"},animationTimingFunction:{values:"easings"},fontFamily:{values:"fonts"},fontSize:{values:"fontSizes"},fontWeight:{values:"fontWeights"},lineHeight:{values:"lineHeights"},letterSpacing:{values:"letterSpacings"},textIndent:{values:"spacing"},truncate:{values:{type:"boolean"},transform(e){return e===!0?{overflow:"hidden",textOverflow:"ellipsis",whiteSpace:"nowrap"}:{}}},lineClamp:{transform(e){return e==="none"?{WebkitLineClamp:"unset"}:{overflow:"hidden",display:"-webkit-box",WebkitLineClamp:e,WebkitBoxOrient:"vertical",textWrap:"wrap"}}},borderSpacing:{values:e=>({...e("spacing"),auto:"var(--border-spacing-x, 0) var(--border-spacing-y, 0)"})},borderSpacingX:{values:"spacing",transform(e){return{"--border-spacing-x":e}}},borderSpacingY:{values:"spacing",transform(e){return{"--border-spacing-y":e}}},srOnly:{values:{type:"boolean"},transform(e){return tC[e]||{}}},debug:{values:{type:"boolean"},transform(e){return e?{outline:"1px solid blue !important","& > *":{outline:"1px solid red !important"}}:{}}},caretColor:{values:ne,transform:te("caretColor")},cursor:{values:"cursor"}}}),tC={true:{position:"absolute",width:"1px",height:"1px",padding:"0",margin:"-1px",overflow:"hidden",clip:"rect(0, 0, 0, 0)",whiteSpace:"nowrap",borderWidth:"0"},false:{position:"static",width:"auto",height:"auto",padding:"0",margin:"0",overflow:"visible",clip:"auto",whiteSpace:"normal"}};var nC="",rC=nC.split(","),oC="WebkitAppearance,WebkitBorderBefore,WebkitBorderBeforeColor,WebkitBorderBeforeStyle,WebkitBorderBeforeWidth,WebkitBoxReflect,WebkitLineClamp,WebkitMask,WebkitMaskAttachment,WebkitMaskClip,WebkitMaskComposite,WebkitMaskImage,WebkitMaskOrigin,WebkitMaskPosition,WebkitMaskPositionX,WebkitMaskPositionY,WebkitMaskRepeat,WebkitMaskRepeatX,WebkitMaskRepeatY,WebkitMaskSize,WebkitOverflowScrolling,WebkitTapHighlightColor,WebkitTextFillColor,WebkitTextStroke,WebkitTextStrokeColor,WebkitTextStrokeWidth,WebkitTouchCallout,WebkitUserModify,WebkitUserSelect,accentColor,alignContent,alignItems,alignSelf,alignTracks,all,anchorName,anchorScope,animation,animationComposition,animationDelay,animationDirection,animationDuration,animationFillMode,animationIterationCount,animationName,animationPlayState,animationRange,animationRangeEnd,animationRangeStart,animationTimeline,animationTimingFunction,appearance,aspectRatio,backdropFilter,backfaceVisibility,background,backgroundAttachment,backgroundBlendMode,backgroundClip,backgroundColor,backgroundImage,backgroundOrigin,backgroundPosition,backgroundPositionX,backgroundPositionY,backgroundRepeat,backgroundSize,blockSize,border,borderBlock,borderBlockColor,borderBlockEnd,borderBlockEndColor,borderBlockEndStyle,borderBlockEndWidth,borderBlockStart,borderBlockStartColor,borderBlockStartStyle,borderBlockStartWidth,borderBlockStyle,borderBlockWidth,borderBottom,borderBottomColor,borderBottomLeftRadius,borderBottomRightRadius,borderBottomStyle,borderBottomWidth,borderCollapse,borderColor,borderEndEndRadius,borderEndStartRadius,borderImage,borderImageOutset,borderImageRepeat,borderImageSlice,borderImageSource,borderImageWidth,borderInline,borderInlineColor,borderInlineEnd,borderInlineEndColor,borderInlineEndStyle,borderInlineEndWidth,borderInlineStart,borderInlineStartColor,borderInlineStartStyle,borderInlineStartWidth,borderInlineStyle,borderInlineWidth,borderLeft,borderLeftColor,borderLeftStyle,borderLeftWidth,borderRadius,borderRight,borderRightColor,borderRightStyle,borderRightWidth,borderSpacing,borderStartEndRadius,borderStartStartRadius,borderStyle,borderTop,borderTopColor,borderTopLeftRadius,borderTopRightRadius,borderTopStyle,borderTopWidth,borderWidth,bottom,boxAlign,boxDecorationBreak,boxDirection,boxFlex,boxFlexGroup,boxLines,boxOrdinalGroup,boxOrient,boxPack,boxShadow,boxSizing,breakAfter,breakBefore,breakInside,captionSide,caret,caretColor,caretShape,clear,clip,clipPath,clipRule,color,colorInterpolationFilters,colorScheme,columnCount,columnFill,columnGap,columnRule,columnRuleColor,columnRuleStyle,columnRuleWidth,columnSpan,columnWidth,columns,contain,containIntrinsicBlockSize,containIntrinsicHeight,containIntrinsicInlineSize,containIntrinsicSize,containIntrinsicWidth,container,containerName,containerType,content,contentVisibility,counterIncrement,counterReset,counterSet,cursor,cx,cy,d,direction,display,dominantBaseline,emptyCells,fieldSizing,fill,fillOpacity,fillRule,filter,flex,flexBasis,flexDirection,flexFlow,flexGrow,flexShrink,flexWrap,float,floodColor,floodOpacity,font,fontFamily,fontFeatureSettings,fontKerning,fontLanguageOverride,fontOpticalSizing,fontPalette,fontSize,fontSizeAdjust,fontSmooth,fontStretch,fontStyle,fontSynthesis,fontSynthesisPosition,fontSynthesisSmallCaps,fontSynthesisStyle,fontSynthesisWeight,fontVariant,fontVariantAlternates,fontVariantCaps,fontVariantEastAsian,fontVariantEmoji,fontVariantLigatures,fontVariantNumeric,fontVariantPosition,fontVariationSettings,fontWeight,forcedColorAdjust,gap,grid,gridArea,gridAutoColumns,gridAutoFlow,gridAutoRows,gridColumn,gridColumnEnd,gridColumnGap,gridColumnStart,gridGap,gridRow,gridRowEnd,gridRowGap,gridRowStart,gridTemplate,gridTemplateAreas,gridTemplateColumns,gridTemplateRows,hangingPunctuation,height,hyphenateCharacter,hyphenateLimitChars,hyphens,imageOrientation,imageRendering,imageResolution,imeMode,initialLetter,initialLetterAlign,inlineSize,inset,insetBlock,insetBlockEnd,insetBlockStart,insetInline,insetInlineEnd,insetInlineStart,interpolateSize,isolation,justifyContent,justifyItems,justifySelf,justifyTracks,left,letterSpacing,lightingColor,lineBreak,lineClamp,lineHeight,lineHeightStep,listStyle,listStyleImage,listStylePosition,listStyleType,margin,marginBlock,marginBlockEnd,marginBlockStart,marginBottom,marginInline,marginInlineEnd,marginInlineStart,marginLeft,marginRight,marginTop,marginTrim,marker,markerEnd,markerMid,markerStart,mask,maskBorder,maskBorderMode,maskBorderOutset,maskBorderRepeat,maskBorderSlice,maskBorderSource,maskBorderWidth,maskClip,maskComposite,maskImage,maskMode,maskOrigin,maskPosition,maskRepeat,maskSize,maskType,masonryAutoFlow,mathDepth,mathShift,mathStyle,maxBlockSize,maxHeight,maxInlineSize,maxLines,maxWidth,minBlockSize,minHeight,minInlineSize,minWidth,mixBlendMode,objectFit,objectPosition,offset,offsetAnchor,offsetDistance,offsetPath,offsetPosition,offsetRotate,opacity,order,orphans,outline,outlineColor,outlineOffset,outlineStyle,outlineWidth,overflow,overflowAnchor,overflowBlock,overflowClipBox,overflowClipMargin,overflowInline,overflowWrap,overflowX,overflowY,overlay,overscrollBehavior,overscrollBehaviorBlock,overscrollBehaviorInline,overscrollBehaviorX,overscrollBehaviorY,padding,paddingBlock,paddingBlockEnd,paddingBlockStart,paddingBottom,paddingInline,paddingInlineEnd,paddingInlineStart,paddingLeft,paddingRight,paddingTop,page,pageBreakAfter,pageBreakBefore,pageBreakInside,paintOrder,perspective,perspectiveOrigin,placeContent,placeItems,placeSelf,pointerEvents,position,positionAnchor,positionArea,positionTry,positionTryFallbacks,positionTryOrder,positionVisibility,printColorAdjust,quotes,r,resize,right,rotate,rowGap,rubyAlign,rubyMerge,rubyPosition,rx,ry,scale,scrollBehavior,scrollMargin,scrollMarginBlock,scrollMarginBlockEnd,scrollMarginBlockStart,scrollMarginBottom,scrollMarginInline,scrollMarginInlineEnd,scrollMarginInlineStart,scrollMarginLeft,scrollMarginRight,scrollMarginTop,scrollPadding,scrollPaddingBlock,scrollPaddingBlockEnd,scrollPaddingBlockStart,scrollPaddingBottom,scrollPaddingInline,scrollPaddingInlineEnd,scrollPaddingInlineStart,scrollPaddingLeft,scrollPaddingRight,scrollPaddingTop,scrollSnapAlign,scrollSnapCoordinate,scrollSnapDestination,scrollSnapPointsX,scrollSnapPointsY,scrollSnapStop,scrollSnapType,scrollSnapTypeX,scrollSnapTypeY,scrollTimeline,scrollTimelineAxis,scrollTimelineName,scrollbarColor,scrollbarGutter,scrollbarWidth,shapeImageThreshold,shapeMargin,shapeOutside,shapeRendering,stopColor,stopOpacity,stroke,strokeDasharray,strokeDashoffset,strokeLinecap,strokeLinejoin,strokeMiterlimit,strokeOpacity,strokeWidth,tabSize,tableLayout,textAlign,textAlignLast,textAnchor,textBox,textBoxEdge,textBoxTrim,textCombineUpright,textDecoration,textDecorationColor,textDecorationLine,textDecorationSkip,textDecorationSkipInk,textDecorationStyle,textDecorationThickness,textEmphasis,textEmphasisColor,textEmphasisPosition,textEmphasisStyle,textIndent,textJustify,textOrientation,textOverflow,textRendering,textShadow,textSizeAdjust,textSpacingTrim,textTransform,textUnderlineOffset,textUnderlinePosition,textWrap,textWrapMode,textWrapStyle,timelineScope,top,touchAction,transform,transformBox,transformOrigin,transformStyle,transition,transitionBehavior,transitionDelay,transitionDuration,transitionProperty,transitionTimingFunction,translate,unicodeBidi,userSelect,vectorEffect,verticalAlign,viewTimeline,viewTimelineAxis,viewTimelineInset,viewTimelineName,viewTransitionName,visibility,whiteSpace,whiteSpaceCollapse,widows,width,willChange,wordBreak,wordSpacing,wordWrap,writingMode,x,y,zIndex,zoom,alignmentBaseline,baselineShift,colorInterpolation,colorRendering,glyphOrientationVertical",iC=oC.split(",").concat(rC),sC=new Map(iC.map(e=>[e,!0]));function aC(e){const t=Object.create(null);return n=>(t[n]===void 0&&(t[n]=e(n)),t[n])}var lC=/&|@/,cC=aC(e=>sC.has(e)||e.startsWith("--")||lC.test(e));function oh(e,t){const n={};return gt(e,(r,o)=>{r&&(n[o.join(".")]=r.value)},{stop:t}),n}var Ca,ih;function uC(){if(ih)return Ca;ih=1,Ca=i,i.default=i,i.stable=c,i.stableStringify=c;var e="[...]",t="[Circular]",n=[],r=[];function o(){return{depthLimit:Number.MAX_SAFE_INTEGER,edgesLimit:Number.MAX_SAFE_INTEGER}}function i(d,m,f,p){typeof p>"u"&&(p=o()),a(d,"",0,[],void 0,0,p);var v;try{r.length===0?v=JSON.stringify(d,m,f):v=JSON.stringify(d,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var b=n.pop();b.length===4?Object.defineProperty(b[0],b[1],b[3]):b[0][b[1]]=b[2]}}return v}function s(d,m,f,p){var v=Object.getOwnPropertyDescriptor(p,f);v.get!==void 0?v.configurable?(Object.defineProperty(p,f,{value:d}),n.push([p,f,m,v])):r.push([m,f,d]):(p[f]=d,n.push([p,f,m]))}function a(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;xm?1:0}function c(d,m,f,p){typeof p>"u"&&(p=o());var v=u(d,"",0,[],void 0,0,p)||d,b;try{r.length===0?b=JSON.stringify(v,m,f):b=JSON.stringify(v,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var y=n.pop();y.length===4?Object.defineProperty(y[0],y[1],y[3]):y[0][y[1]]=y[2]}}return b}function u(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;x0)for(var p=0;p{const t=Object.create(null);function n(...r){const o=r.map(i=>hC(i)).join("|");return t[o]===void 0&&(t[o]=e(...r)),t[o]}return n},sh=16,li="px",wa="em",jr="rem";function ah(e=""){const t=new RegExp(String.raw`-?\d+(?:\.\d+|\d*)`),n=new RegExp(`${li}|${wa}|${jr}`);return e.match(new RegExp(`${t.source}(${n.source})`))?.[1]}function lh(e=""){if(typeof e=="number")return`${e}px`;const t=ah(e);if(!t||t===li)return e;if(t===wa||t===jr)return`${parseFloat(e)*sh}${li}`}function ch(e=""){const t=ah(e);if(!t||t===jr)return e;if(t===wa)return`${parseFloat(e)}${jr}`;if(t===li)return`${parseFloat(e)/sh}${jr}`}const fC=e=>e.charAt(0).toUpperCase()+e.slice(1);function gC(e){const t=pC(e),n=Object.fromEntries(t);function r(d){return n[d]}function o(d){return rr(r(d))}function i(){const d=Object.keys(n),m=mC(d),f=d.flatMap(p=>{const v=r(p),b=[`${p}Down`,rr({max:ci(v.min)})],y=[p,rr({min:v.min})],x=[`${p}Only`,o(p)];return[y,x,b]}).filter(([,p])=>p!=="").concat(m.map(([p,v])=>{const b=r(p),y=r(v);return[`${p}To${fC(v)}`,rr({min:b.min,max:ci(y.min)})]}));return Object.fromEntries(f)}function s(){const d=i();return Object.fromEntries(Object.entries(d))}const a=s(),l=d=>a[d];function c(){return["base",...Object.keys(n)]}function u(d){return rr({min:r(d).min})}function h(d){return rr({max:ci(r(d).min)})}return{values:Object.values(n),only:o,keys:c,conditions:a,getCondition:l,up:u,down:h}}function ci(e){const t=parseFloat(lh(e)??"")-.04;return ch(`${t}px`)}function pC(e){return Object.entries(e).sort(([,n],[,r])=>parseInt(n,10){let s=null;return o<=i.length-1&&(s=i[o+1]?.[1]),s!=null&&(s=ci(s)),[n,{name:n,min:ch(r),max:s}]})}function mC(e){const t=[];return e.forEach((n,r)=>{let o=r;o++;let i=e[o];for(;i;)t.push([n,i]),o++,i=e[o]}),t}function rr({min:e,max:t}){return e==null&&t==null?"":["@media screen",e&&`(min-width: ${e})`,t&&`(max-width: ${t})`].filter(Boolean).join(" and ")}const vC=(e,t)=>Object.fromEntries(Object.entries(e).map(([n,r])=>t(n,r))),bC=e=>{const{breakpoints:t,conditions:n={}}=e,r=vC(n,(u,h)=>[`_${u}`,h]),o=Object.assign({},r,t.conditions);function i(){return Object.keys(o)}function s(u){return i().includes(u)||/^@|&|&$/.test(u)||u.startsWith("_")}function a(u){return u.filter(h=>h!=="base").sort((h,d)=>{const m=s(h),f=s(d);return m&&!f?1:!m&&f?-1:0})}function l(u){return u.startsWith("@breakpoint")?t.getCondition(u.replace("@breakpoint ","")):u}function c(u){return Reflect.get(o,u)||u}return{keys:i,sort:a,has:s,resolve:c,breakpoints:t.keys(),expandAtRule:l}},uh=e=>({minMax:new RegExp(`(!?\\(\\s*min(-device-)?-${e})(.| + `,n.body.appendChild(r)};function gk(e,t,n){if(!(!e||!n.isActiveElement(e))){if(!t){const r=e.value.length;e.setSelectionRange(r,r);return}try{const r=e.value,{start:o,end:i,value:s}=t;if(r===s){e.setSelectionRange(o,i);return}const a=Vd(s,r,o),l=o===i?a:Vd(s,r,i),c=Math.max(0,Math.min(a,r.length)),u=Math.max(c,Math.min(l,r.length));e.setSelectionRange(c,u)}catch{const r=e.value.length;e.setSelectionRange(r,r)}}}function Vd(e,t,n){const r=e.slice(0,n),o=e.slice(n);let i=0;const s=Math.min(r.length,t.length);for(let c=0;c=r.length)return i;if(a>=o.length)return t.length-a;if(i>0)return i;if(a>0)return t.length-a;if(e.length>0){const c=n/e.length;return Math.round(c*t.length)}return t.length}var pk=(e,t={})=>new Intl.NumberFormat(e,t),mk=(e,t={})=>new Od(e,t),da=(e,t)=>{const{prop:n,computed:r}=t;return n("formatOptions")?e===""?Number.NaN:r("parser").parse(e):parseFloat(e)},ln=(e,t)=>{const{prop:n,computed:r}=t;return Number.isNaN(e)?"":n("formatOptions")?r("formatter").format(e):e.toString()},vk=(e,t)=>{let n=e!==void 0&&!Number.isNaN(e)?e:1;return t?.style==="percent"&&(e===void 0||Number.isNaN(e))&&(n=.01),n},{choose:bk,guards:yk,createMachine:xk}=wc(),{not:Ld,and:Fd}=yk;xk({props({props:e}){const t=vk(e.step,e.formatOptions);return{dir:"ltr",locale:"en-US",focusInputOnChange:!0,clampValueOnBlur:!e.allowOverflow,allowOverflow:!1,inputMode:"decimal",pattern:"-?[0-9]*(.[0-9]+)?",defaultValue:"",step:t,min:Number.MIN_SAFE_INTEGER,max:Number.MAX_SAFE_INTEGER,spinOnPress:!0,...e,translations:{incrementLabel:"increment value",decrementLabel:"decrease value",...e.translations}}},initialState(){return"idle"},context({prop:e,bindable:t,getComputed:n}){return{value:t(()=>({defaultValue:e("defaultValue"),value:e("value"),onChange(r){const o=n(),i=da(r,{computed:o,prop:e});e("onValueChange")?.({value:r,valueAsNumber:i})}})),hint:t(()=>({defaultValue:null})),scrubberCursorPoint:t(()=>({defaultValue:null,hash(r){return r?`x:${r.x}, y:${r.y}`:""}})),fieldsetDisabled:t(()=>({defaultValue:!1}))}},computed:{isRtl:({prop:e})=>e("dir")==="rtl",valueAsNumber:({context:e,computed:t,prop:n})=>da(e.get("value"),{computed:t,prop:n}),formattedValue:({computed:e,prop:t})=>ln(e("valueAsNumber"),{computed:e,prop:t}),isAtMin:({computed:e,prop:t})=>f0(e("valueAsNumber"),t("min")),isAtMax:({computed:e,prop:t})=>h0(e("valueAsNumber"),t("max")),isOutOfRange:({computed:e,prop:t})=>!g0(e("valueAsNumber"),t("min"),t("max")),isValueEmpty:({context:e})=>e.get("value")==="",isDisabled:({prop:e,context:t})=>!!e("disabled")||t.get("fieldsetDisabled"),canIncrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMax"),canDecrement:({prop:e,computed:t})=>e("allowOverflow")||!t("isAtMin"),valueText:({prop:e,context:t})=>e("translations").valueText?.(t.get("value")),formatter:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>pk(e,t)),parser:Cc(({prop:e})=>[e("locale"),e("formatOptions")],([e,t])=>mk(e,t))},watch({track:e,action:t,context:n,computed:r,prop:o}){e([()=>n.get("value"),()=>o("locale")],()=>{t(["syncInputElement"])}),e([()=>r("isOutOfRange")],()=>{t(["invokeOnInvalid"])}),e([()=>n.hash("scrubberCursorPoint")],()=>{t(["setVirtualCursorPosition"])})},effects:["trackFormControl"],on:{"VALUE.SET":{actions:["setRawValue"]},"VALUE.CLEAR":{actions:["clearValue"]},"VALUE.INCREMENT":{actions:["increment"]},"VALUE.DECREMENT":{actions:["decrement"]}},states:{idle:{on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","invokeOnFocus","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","invokeOnFocus","setHint","setCursorPoint"]},"INPUT.FOCUS":{target:"focused",actions:["focusInput","invokeOnFocus"]}}},focused:{tags:["focus"],effects:["attachWheelListener"],on:{"TRIGGER.PRESS_DOWN":[{guard:"isTouchPointer",target:"before:spin",actions:["setHint"]},{target:"before:spin",actions:["focusInput","setHint"]}],"SCRUBBER.PRESS_DOWN":{target:"scrubbing",actions:["focusInput","setHint","setCursorPoint"]},"INPUT.ARROW_UP":{actions:["increment"]},"INPUT.ARROW_DOWN":{actions:["decrement"]},"INPUT.HOME":{actions:["decrementToMin"]},"INPUT.END":{actions:["incrementToMax"]},"INPUT.CHANGE":{actions:["setValue","setHint"]},"INPUT.BLUR":[{guard:Fd("clampValueOnBlur",Ld("isInRange")),target:"idle",actions:["setClampedValue","clearHint","invokeOnBlur"]},{guard:Ld("isInRange"),target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur","invokeOnInvalid"]},{target:"idle",actions:["setFormattedValue","clearHint","invokeOnBlur"]}],"INPUT.ENTER":{actions:["setFormattedValue","clearHint","invokeOnBlur"]}}},"before:spin":{tags:["focus"],effects:["trackButtonDisabled","waitForChangeDelay"],entry:bk([{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}]),on:{CHANGE_DELAY:{target:"spinning",guard:Fd("isInRange","spinOnPress")},"TRIGGER.PRESS_UP":[{guard:"isTouchPointer",target:"focused",actions:["clearHint"]},{target:"focused",actions:["focusInput","clearHint"]}]}},spinning:{tags:["focus"],effects:["trackButtonDisabled","spinValue"],on:{SPIN:[{guard:"isIncrementHint",actions:["increment"]},{guard:"isDecrementHint",actions:["decrement"]}],"TRIGGER.PRESS_UP":{target:"focused",actions:["focusInput","clearHint"]}}},scrubbing:{tags:["focus"],effects:["activatePointerLock","trackMousemove","setupVirtualCursor","preventTextSelection"],on:{"SCRUBBER.POINTER_UP":{target:"focused",actions:["focusInput","clearCursorPoint"]},"SCRUBBER.POINTER_MOVE":[{guard:"isIncrementHint",actions:["increment","setCursorPoint"]},{guard:"isDecrementHint",actions:["decrement","setCursorPoint"]}]}}},implementations:{guards:{clampValueOnBlur:({prop:e})=>e("clampValueOnBlur"),spinOnPress:({prop:e})=>!!e("spinOnPress"),isInRange:({computed:e})=>!e("isOutOfRange"),isDecrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="decrement",isIncrementHint:({context:e,event:t})=>(t.hint??e.get("hint"))==="increment",isTouchPointer:({event:e})=>e.pointerType==="touch"},effects:{waitForChangeDelay({send:e}){const t=setTimeout(()=>{e({type:"CHANGE_DELAY"})},300);return()=>clearTimeout(t)},spinValue({send:e}){const t=setInterval(()=>{e({type:"SPIN"})},50);return()=>clearInterval(t)},trackFormControl({context:e,scope:t}){const n=ii(t);return ys(n,{onFieldsetDisabledChange(r){e.set("fieldsetDisabled",r)},onFormReset(){e.set("value",e.initial("value"))}})},setupVirtualCursor({context:e,scope:t}){const n=e.get("scrubberCursorPoint");return uk(t,n)},preventTextSelection({scope:e}){return dk(e)},trackButtonDisabled({context:e,scope:t,send:n}){const r=e.get("hint"),o=ck(t,r);return Eo(o,{attributes:["disabled"],callback(){n({type:"TRIGGER.PRESS_UP",src:"attr"})}})},attachWheelListener({scope:e,send:t,prop:n}){const r=ii(e);if(!r||!e.isActiveElement(r)||!n("allowMouseWheel"))return;function o(i){i.preventDefault();const s=Math.sign(i.deltaY)*-1;s===1?t({type:"VALUE.INCREMENT"}):s===-1&&t({type:"VALUE.DECREMENT"})}return se(r,"wheel",o,{passive:!1})},activatePointerLock({scope:e}){if(!fc())return hv(e.getDoc())},trackMousemove({scope:e,send:t,context:n,computed:r}){const o=e.getDoc();function i(a){const l=n.get("scrubberCursorPoint"),c=r("isRtl"),u=hk(e,{point:l,isRtl:c,event:a});u.hint&&t({type:"SCRUBBER.POINTER_MOVE",hint:u.hint,point:u.point})}function s(){t({type:"SCRUBBER.POINTER_UP"})}return mo(se(o,"mousemove",i,!1),se(o,"mouseup",s,!1))}},actions:{focusInput({scope:e,prop:t}){if(!t("focusInputOnChange"))return;const n=ii(e);e.isActiveElement(n)||G(()=>n?.focus({preventScroll:!0}))},increment({context:e,event:t,prop:n,computed:r}){let o=b0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},decrement({context:e,event:t,prop:n,computed:r}){let o=y0(r("valueAsNumber"),t.step??n("step"));n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setClampedValue({context:e,prop:t,computed:n}){const r=Ve(n("valueAsNumber"),t("min"),t("max"));e.set("value",ln(r,{computed:n,prop:t}))},setRawValue({context:e,event:t,prop:n,computed:r}){let o=da(t.value,{computed:r,prop:n});n("allowOverflow")||(o=Ve(o,n("min"),n("max"))),e.set("value",ln(o,{computed:r,prop:n}))},setValue({context:e,event:t}){const n=t.target?.value??t.value;e.set("value",n)},clearValue({context:e}){e.set("value","")},incrementToMax({context:e,prop:t,computed:n}){const r=ln(t("max"),{computed:n,prop:t});e.set("value",r)},decrementToMin({context:e,prop:t,computed:n}){const r=ln(t("min"),{computed:n,prop:t});e.set("value",r)},setHint({context:e,event:t}){e.set("hint",t.hint)},clearHint({context:e}){e.set("hint",null)},invokeOnFocus({computed:e,prop:t}){t("onFocusChange")?.({focused:!0,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnBlur({computed:e,prop:t}){t("onFocusChange")?.({focused:!1,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},invokeOnInvalid({computed:e,prop:t,event:n}){if(n.type==="INPUT.CHANGE")return;const r=e("valueAsNumber")>t("max")?"rangeOverflow":"rangeUnderflow";t("onValueInvalid")?.({reason:r,value:e("formattedValue"),valueAsNumber:e("valueAsNumber")})},syncInputElement({context:e,event:t,computed:n,scope:r}){const o=t.type.endsWith("CHANGE")?e.get("value"):n("formattedValue"),i=ii(r),s=t.selection;G(()=>{So(i,o),gk(i,s,r)})},setFormattedValue({context:e,computed:t}){e.set("value",t("formattedValue"))},setCursorPoint({context:e,event:t}){e.set("scrubberCursorPoint",t.point)},clearCursorPoint({context:e}){e.set("scrubberCursorPoint",null)},setVirtualCursorPosition({context:e,scope:t}){const n=_d(t),r=e.get("scrubberCursorPoint");!n||!r||(n.style.transform=`translate3d(${r.x}px, ${r.y}px, 0px)`)}}}}),M()(["allowMouseWheel","allowOverflow","clampValueOnBlur","dir","disabled","focusInputOnChange","form","formatOptions","getRootNode","id","ids","inputMode","invalid","locale","max","min","name","onFocusChange","onValueChange","onValueInvalid","pattern","required","readOnly","spinOnPress","step","translations","value","defaultValue"]);var Dd=j("pinInput").parts("root","label","input","control");Dd.build(),M()(["autoFocus","blurOnComplete","count","defaultValue","dir","disabled","form","getRootNode","id","ids","invalid","mask","name","onValueChange","onValueComplete","onValueInvalid","otp","pattern","placeholder","readOnly","required","selectOnFocus","translations","type","value"]);var zd=j("popover").parts("arrow","arrowTip","anchor","trigger","indicator","positioner","content","title","description","closeTrigger");zd.build(),M()(["autoFocus","closeOnEscape","closeOnInteractOutside","dir","getRootNode","id","ids","initialFocusEl","modal","onEscapeKeyDown","onFocusOutside","onInteractOutside","onOpenChange","onPointerDownOutside","onRequestDismiss","defaultOpen","open","persistentElements","portalled","positioning"]);const cn=e=>{const{children:t,disabled:n}=e,[r,o]=P.useState(e.container?.current),i=P.useSyncExternalStore(Ck,()=>!1,()=>!0),{getRootNode:s}=sc();if(P.useEffect(()=>{o(()=>e.container?.current)},[e.container]),i||n)return g.jsx(g.Fragment,{children:t});const a=r??kk(s);return g.jsx(g.Fragment,{children:P.Children.map(t,l=>Lt.createPortal(l,a))})},kk=e=>{const t=e?.(),n=t.getRootNode();return jn(n)?n:Le(t).body},Ck=()=>()=>{};var ha=j("progress").parts("root","label","track","range","valueText","view","circle","circleTrack","circleRange");ha.build(),M()(["dir","getRootNode","id","ids","max","min","orientation","translations","value","onValueChange","defaultValue","formatOptions","locale"]);var Md=j("qr-code").parts("root","frame","pattern","overlay","downloadTrigger");Md.build(),M()(["ids","defaultValue","value","id","encoding","dir","getRootNode","onValueChange","pixelSize"]);var fa=j("radio-group").parts("root","label","item","itemText","itemControl","indicator");fa.build(),M()(["dir","disabled","form","getRootNode","id","ids","name","onValueChange","orientation","readOnly","value","defaultValue"]),M()(["value","disabled","invalid"]);var jd=j("rating-group").parts("root","label","item","control");jd.build(),M()(["allowHalf","autoFocus","count","dir","disabled","form","getRootNode","id","ids","name","onHoverChange","onValueChange","required","readOnly","translations","value","defaultValue"]),M()(["index"]);var $d=j("scroll-area").parts("root","viewport","content","scrollbar","thumb","corner");$d.build(),M()(["dir","getRootNode","ids","id"]);const Bd=fa.rename("segment-group");Bd.build();var Wd=j("select").parts("label","positioner","trigger","indicator","clearTrigger","item","itemText","itemIndicator","itemGroup","itemGroupLabel","list","content","root","control","valueText");Wd.build();var Hd=e=>new Qo(e);Hd.empty=()=>new Qo({items:[]});var wk=e=>e.ids?.content??`select:${e.id}:content`,Sk=e=>e.ids?.trigger??`select:${e.id}:trigger`,Ek=e=>e.ids?.clearTrigger??`select:${e.id}:clear-trigger`,Ok=(e,t)=>e.ids?.item?.(t)??`select:${e.id}:option:${t}`,Pk=e=>e.ids?.hiddenSelect??`select:${e.id}:select`,Rk=e=>e.ids?.positioner??`select:${e.id}:positioner`,ga=e=>e.getById(Pk(e)),zr=e=>e.getById(wk(e)),si=e=>e.getById(Sk(e)),Ik=e=>e.getById(Ek(e)),Ud=e=>e.getById(Rk(e)),pa=(e,t)=>t==null?null:e.getById(Ok(e,t)),{and:Mr,not:un,or:Tk}=jt();Tk("isTriggerArrowDownEvent","isTriggerEnterEvent"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),Mr(un("multiple"),"hasSelectedItems"),un("multiple"),un("multiple"),un("multiple"),un("multiple"),Mr("closeOnSelect","isOpenControlled"),Mr("hasHighlightedItem","loop","isLastItemHighlighted"),Mr("hasHighlightedItem","loop","isFirstItemHighlighted");function Gd(e){const t=e.restoreFocus??e.previousEvent?.restoreFocus;return t==null||!!t}M()(["closeOnSelect","collection","composite","defaultHighlightedValue","defaultOpen","defaultValue","deselectable","dir","disabled","form","getRootNode","highlightedValue","id","ids","invalid","loopFocus","multiple","name","onFocusOutside","onHighlightChange","onInteractOutside","onOpenChange","onPointerDownOutside","onSelect","onValueChange","open","positioning","readOnly","required","scrollToIndexFn","value"]),M()(["item","persistFocus"]),M()(["id"]),M()(["htmlFor"]);var qd=j("slider").parts("root","label","thumb","valueText","track","range","control","markerGroup","marker","draggingIndicator");qd.build(),M()(["aria-label","aria-labelledby","dir","disabled","form","getAriaValueText","getRootNode","id","ids","invalid","max","min","minStepsBetweenThumbs","name","onFocusChange","onValueChange","onValueChangeEnd","orientation","origin","readOnly","step","thumbAlignment","thumbAlignment","thumbSize","value","defaultValue"]),M()(["index","name"]);var Kd=j("switch").parts("root","label","control","thumb");Kd.build(),M()(["checked","defaultChecked","dir","disabled","form","getRootNode","id","ids","invalid","label","name","onCheckedChange","readOnly","required","value"]);var Nk=j("toast").parts("group","root","title","description","actionTrigger","closeTrigger");Nk.build();var Ak=(e,t)=>({...t,...bo(e)});function _k(e={}){const t=Ak(e,{placement:"bottom",overlap:!1,max:24,gap:16,offsets:"1rem",hotkey:["altKey","KeyT"],removeDelay:200,pauseOnPageIdle:!0});let n=[],r=[],o=new Set,i=[];const s=S=>(n.push(S),()=>{const _=n.indexOf(S);n.splice(_,1)}),a=S=>(n.forEach(_=>_(S)),S),l=S=>{if(r.length>=t.max){i.push(S);return}a(S),r.unshift(S)},c=()=>{for(;i.length>0&&r.length{const _=S.id??`toast:${s0()}`,D=r.find(z=>z.id===_);return o.has(_)&&o.delete(_),D?r=r.map(z=>z.id===_?a({...z,...S,id:_}):z):l({id:_,duration:t.duration,removeDelay:t.removeDelay,type:"info",...S,stacked:!t.overlap,gap:t.gap}),_},h=S=>(o.add(S),S?(n.forEach(_=>_({id:S,dismiss:!0})),r=r.filter(_=>_.id!==S),c()):(r.forEach(_=>{n.forEach(D=>D({id:_.id,dismiss:!0}))}),r=[],i=[]),S);return{attrs:t,subscribe:s,create:u,update:(S,_)=>u({id:S,..._}),remove:h,dismiss:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"DISMISS"}):_):r=r.map(_=>a({..._,message:"DISMISS"}))},error:S=>u({...S,type:"error"}),success:S=>u({...S,type:"success"}),info:S=>u({...S,type:"info"}),warning:S=>u({...S,type:"warning"}),loading:S=>u({...S,type:"loading"}),getVisibleToasts:()=>r.filter(S=>!o.has(S.id)),getCount:()=>r.length,promise:(S,_,D={})=>{if(!_||!_.loading){wr("[zag-js > toast] toaster.promise() requires at least a 'loading' option to be specified");return}const z=u({...D,..._.loading,promise:S,type:"loading"});let W=!0,J;const ee=po(S).then(async q=>{if(J=["resolve",q],Vk(q)&&!q.ok){W=!1;const Z=po(_.error,`HTTP Error! status: ${q.status}`);u({...D,...Z,id:z,type:"error"})}else if(_.success!==void 0){W=!1;const Z=po(_.success,q);u({...D,...Z,id:z,type:"success"})}}).catch(async q=>{if(J=["reject",q],_.error!==void 0){W=!1;const Z=po(_.error,q);u({...D,...Z,id:z,type:"error"})}}).finally(()=>{W&&h(z),_.finally?.()});return{id:z,unwrap:()=>new Promise((q,Z)=>ee.then(()=>J[0]==="reject"?Z(J[1]):q(J[1])).catch(Z))}},pause:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"PAUSE"}):_):r=r.map(_=>a({..._,message:"PAUSE"}))},resume:S=>{S!=null?r=r.map(_=>_.id===S?a({..._,message:"RESUME"}):_):r=r.map(_=>a({..._,message:"RESUME"}))},isVisible:S=>!o.has(S)&&!!r.find(_=>_.id===S),isDismissed:S=>o.has(S),expand:()=>{r=r.map(S=>a({...S,stacked:!0}))},collapse:()=>{r=r.map(S=>a({...S,stacked:!1}))}}}var Vk=e=>e&&typeof e=="object"&&"ok"in e&&typeof e.ok=="boolean"&&"status"in e&&typeof e.status=="number";const Lk=e=>_k(e);var Yd=j("tooltip").parts("trigger","arrow","arrowTip","positioner","content");Yd.build();var Fk=e=>e.ids?.trigger??`tooltip:${e.id}:trigger`,Dk=e=>e.ids?.positioner??`tooltip:${e.id}:popper`,ma=e=>e.getById(Fk(e)),Xd=e=>e.getById(Dk(e)),dn=x0({id:null}),{and:zk,not:Qd}=jt();zk("noVisibleTooltip",Qd("hasPointerMoveOpened")),Qd("hasPointerMoveOpened"),M()(["aria-label","closeDelay","closeOnEscape","closeOnPointerDown","closeOnScroll","closeOnClick","dir","disabled","getRootNode","id","ids","interactive","onOpenChange","defaultOpen","open","openDelay","positioning"]);function Jd(e,t=[]){const n=Object.assign({},e);for(const r of t)r in n&&delete n[r];return n}const Mk=(e,t)=>{if(!e||typeof e!="string")return{invalid:!0,value:e};const[n,r]=e.split("/");if(!n||!r||n==="currentBg")return{invalid:!0,value:n};const o=t(`colors.${n}`),i=t.raw(`opacity.${r}`)?.value;if(!i&&isNaN(Number(r)))return{invalid:!0,value:n};const s=i?Number(i)*100+"%":`${r}%`,a=o??n;return{invalid:!1,color:a,value:`color-mix(in srgb, ${a} ${s}, transparent)`}},te=e=>(t,n)=>{const r=n.utils.colorMix(t);if(r.invalid)return{[e]:t};const o="--mix-"+e;return{[o]:r.value,[e]:`var(${o}, ${r.color})`}};function va(e){if(e===null||typeof e!="object")return e;if(Array.isArray(e))return e.map(n=>va(n));const t=Object.create(Object.getPrototypeOf(e));for(const n of Object.keys(e))t[n]=va(e[n]);return t}function ba(e,t){if(t==null)return e;for(const n of Object.keys(t))if(!(t[n]===void 0||n==="__proto__"))if(!_e(e[n])&&_e(t[n]))Object.assign(e,{[n]:t[n]});else if(e[n]&&_e(t[n]))ba(e[n],t[n]);else if(Array.isArray(t[n])&&Array.isArray(e[n])){let r=0;for(;re!=null;function gt(e,t,n={}){const{stop:r,getKey:o}=n;function i(s,a=[]){if(_e(s)||Array.isArray(s)){const l={};for(const[c,u]of Object.entries(s)){const h=o?.(c,u)??c,d=[...a,h];if(r?.(s,d))return t(s,a);const m=i(u,d);ya(m)&&(l[h]=m)}return l}return t(s,a)}return i(e)}function Zd(e,t){return Array.isArray(e)?e.map(n=>ya(n)?t(n):n):_e(e)?gt(e,n=>t(n)):ya(e)?t(e):e}const ai=["value","type","description"],jk=e=>e&&typeof e=="object"&&!Array.isArray(e),eh=(...e)=>{const t=er({},...e.map(va));return t.theme?.tokens&>(t.theme.tokens,n=>{const i=Object.keys(n).filter(a=>!ai.includes(a)).length>0,s=ai.some(a=>n[a]!=null);return i&&s&&(n.DEFAULT||(n.DEFAULT={}),ai.forEach(a=>{var l;n[a]!=null&&((l=n.DEFAULT)[a]||(l[a]=n[a]),delete n[a])})),n},{stop(n){return jk(n)&&Object.keys(n).some(r=>ai.includes(r)||r!==r.toLowerCase()&&r!==r.toUpperCase())}}),t},$k=e=>e,fe=e=>e,B=e=>e,Bk=e=>e,Wk=e=>e,tr=e=>e,Hk=e=>e,Uk=e=>e,Gk=e=>e;function th(){const e=t=>t;return new Proxy(e,{get(){return e}})}const de=th(),xa=th(),ka=e=>e,qk=/[^a-zA-Z0-9_\u0081-\uffff-]/g;function Kk(e){return`${e}`.replace(qk,t=>`\\${t}`)}const Yk=/[A-Z]/g;function Xk(e){return e.replace(Yk,t=>`-${t.toLowerCase()}`)}function nh(e,t={}){const{fallback:n="",prefix:r=""}=t,o=Xk(["-",r,Kk(e)].filter(Boolean).join("-"));return{var:o,ref:`var(${o}${n?`, ${n}`:""})`}}const Qk=e=>/^var\(--.+\)$/.test(e),xe=(e,t)=>t!=null?`${e}(${t})`:t,hn=e=>{if(Qk(e)||e==null)return e;const t=typeof e=="string"&&!e.endsWith("deg");return typeof e=="number"||t?`${e}deg`:e},rh=e=>({values:["outside","inside","mixed","none"],transform(t,{token:n}){const r=n("colors.colorPalette.focusRing");return{inside:{"--focus-ring-color":r,[e]:{outlineOffset:"0px",outlineWidth:"var(--focus-ring-width, 1px)",outlineColor:"var(--focus-ring-color)",outlineStyle:"var(--focus-ring-style, solid)",borderColor:"var(--focus-ring-color)"}},outside:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 2px)",outlineOffset:"var(--focus-ring-offset, 2px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"var(--focus-ring-color)"}},mixed:{"--focus-ring-color":r,[e]:{outlineWidth:"var(--focus-ring-width, 3px)",outlineStyle:"var(--focus-ring-style, solid)",outlineColor:"color-mix(in srgb, var(--focus-ring-color), transparent 60%)",borderColor:"var(--focus-ring-color)"}},none:{"--focus-ring-color":r,[e]:{outline:"none"}}}[t]??{}}}),Jk=te("borderColor"),Et=e=>({transition:e,transitionTimingFunction:"cubic-bezier(0.4, 0, 0.2, 1)",transitionDuration:"150ms"}),Zk=$k({hover:["@media (hover: hover)","&:is(:hover, [data-hover]):not(:disabled, [data-disabled])"],active:"&:is(:active, [data-active]):not(:disabled, [data-disabled], [data-state=open])",focus:"&:is(:focus, [data-focus])",focusWithin:"&:is(:focus-within, [data-focus-within])",focusVisible:"&:is(:focus-visible, [data-focus-visible])",disabled:"&:is(:disabled, [disabled], [data-disabled], [aria-disabled=true])",visited:"&:visited",target:"&:target",readOnly:"&:is([data-readonly], [aria-readonly=true], [readonly])",readWrite:"&:read-write",empty:"&:is(:empty, [data-empty])",checked:"&:is(:checked, [data-checked], [aria-checked=true], [data-state=checked])",enabled:"&:enabled",expanded:"&:is([aria-expanded=true], [data-expanded], [data-state=expanded])",highlighted:"&[data-highlighted]",complete:"&[data-complete]",incomplete:"&[data-incomplete]",dragging:"&[data-dragging]",before:"&::before",after:"&::after",firstLetter:"&::first-letter",firstLine:"&::first-line",marker:"&::marker",selection:"&::selection",file:"&::file-selector-button",backdrop:"&::backdrop",first:"&:first-of-type",last:"&:last-of-type",notFirst:"&:not(:first-of-type)",notLast:"&:not(:last-of-type)",only:"&:only-child",even:"&:nth-of-type(even)",odd:"&:nth-of-type(odd)",peerFocus:".peer:is(:focus, [data-focus]) ~ &",peerHover:".peer:is(:hover, [data-hover]):not(:disabled, [data-disabled]) ~ &",peerActive:".peer:is(:active, [data-active]):not(:disabled, [data-disabled]) ~ &",peerFocusWithin:".peer:focus-within ~ &",peerFocusVisible:".peer:is(:focus-visible, [data-focus-visible]) ~ &",peerDisabled:".peer:is(:disabled, [disabled], [data-disabled]) ~ &",peerChecked:".peer:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) ~ &",peerInvalid:".peer:is(:invalid, [data-invalid], [aria-invalid=true]) ~ &",peerExpanded:".peer:is([aria-expanded=true], [data-expanded], [data-state=expanded]) ~ &",peerPlaceholderShown:".peer:placeholder-shown ~ &",groupFocus:".group:is(:focus, [data-focus]) &",groupHover:".group:is(:hover, [data-hover]):not(:disabled, [data-disabled]) &",groupActive:".group:is(:active, [data-active]):not(:disabled, [data-disabled]) &",groupFocusWithin:".group:focus-within &",groupFocusVisible:".group:is(:focus-visible, [data-focus-visible]) &",groupDisabled:".group:is(:disabled, [disabled], [data-disabled]) &",groupChecked:".group:is(:checked, [data-checked], [aria-checked=true], [data-state=checked]) &",groupExpanded:".group:is([aria-expanded=true], [data-expanded], [data-state=expanded]) &",groupInvalid:".group:invalid &",indeterminate:"&:is(:indeterminate, [data-indeterminate], [aria-checked=mixed], [data-state=indeterminate])",required:"&:is([data-required], [aria-required=true])",valid:"&:is([data-valid], [data-state=valid])",invalid:"&:is([data-invalid], [aria-invalid=true], [data-state=invalid])",autofill:"&:autofill",inRange:"&:is(:in-range, [data-in-range])",outOfRange:"&:is(:out-of-range, [data-outside-range])",placeholder:"&::placeholder, &[data-placeholder]",placeholderShown:"&:is(:placeholder-shown, [data-placeholder-shown])",pressed:"&:is([aria-pressed=true], [data-pressed])",selected:"&:is([aria-selected=true], [data-selected])",grabbed:"&:is([aria-grabbed=true], [data-grabbed])",underValue:"&[data-state=under-value]",overValue:"&[data-state=over-value]",atValue:"&[data-state=at-value]",default:"&:default",optional:"&:optional",open:"&:is([open], [data-open], [data-state=open])",closed:"&:is([closed], [data-closed], [data-state=closed])",fullscreen:"&:is(:fullscreen, [data-fullscreen])",loading:"&:is([data-loading], [aria-busy=true])",hidden:"&:is([hidden], [data-hidden])",current:"&[data-current]",currentPage:"&[aria-current=page]",currentStep:"&[aria-current=step]",today:"&[data-today]",unavailable:"&[data-unavailable]",rangeStart:"&[data-range-start]",rangeEnd:"&[data-range-end]",now:"&[data-now]",topmost:"&[data-topmost]",motionReduce:"@media (prefers-reduced-motion: reduce)",motionSafe:"@media (prefers-reduced-motion: no-preference)",print:"@media print",landscape:"@media (orientation: landscape)",portrait:"@media (orientation: portrait)",dark:".dark &, .dark .chakra-theme:not(.light) &",light:":root &, .light &",osDark:"@media (prefers-color-scheme: dark)",osLight:"@media (prefers-color-scheme: light)",highContrast:"@media (forced-colors: active)",lessContrast:"@media (prefers-contrast: less)",moreContrast:"@media (prefers-contrast: more)",ltr:"[dir=ltr] &",rtl:"[dir=rtl] &",scrollbar:"&::-webkit-scrollbar",scrollbarThumb:"&::-webkit-scrollbar-thumb",scrollbarTrack:"&::-webkit-scrollbar-track",horizontal:"&[data-orientation=horizontal]",vertical:"&[data-orientation=vertical]",icon:"& :where(svg)",starting:"@starting-style"}),nr=nh("bg-currentcolor"),oh=e=>e===nr.ref||e==="currentBg",ne=e=>({...e("colors"),currentBg:nr}),eC=ka({conditions:Zk,utilities:{background:{values:ne,shorthand:["bg"],transform(e,t){if(oh(t.raw))return{background:nr.ref};const n=te("background")(e,t);return{...n,[nr.var]:n?.background}}},backgroundColor:{values:ne,shorthand:["bgColor"],transform(e,t){if(oh(t.raw))return{backgroundColor:nr.ref};const n=te("backgroundColor")(e,t);return{...n,[nr.var]:n?.backgroundColor}}},backgroundSize:{shorthand:["bgSize"]},backgroundPosition:{shorthand:["bgPos"]},backgroundRepeat:{shorthand:["bgRepeat"]},backgroundAttachment:{shorthand:["bgAttachment"]},backgroundClip:{shorthand:["bgClip"],values:["text"],transform(e){return e==="text"?{color:"transparent",backgroundClip:"text"}:{backgroundClip:e}}},backgroundGradient:{shorthand:["bgGradient"],values(e){return{...e("gradients"),"to-t":"linear-gradient(to top, var(--gradient))","to-tr":"linear-gradient(to top right, var(--gradient))","to-r":"linear-gradient(to right, var(--gradient))","to-br":"linear-gradient(to bottom right, var(--gradient))","to-b":"linear-gradient(to bottom, var(--gradient))","to-bl":"linear-gradient(to bottom left, var(--gradient))","to-l":"linear-gradient(to left, var(--gradient))","to-tl":"linear-gradient(to top left, var(--gradient))"}},transform(e){return{"--gradient-stops":"var(--gradient-from), var(--gradient-to)","--gradient":"var(--gradient-via-stops, var(--gradient-stops))",backgroundImage:e}}},gradientFrom:{values:ne,transform:te("--gradient-from")},gradientTo:{values:ne,transform:te("--gradient-to")},gradientVia:{values:ne,transform(e,t){return{...te("--gradient-via")(e,t),"--gradient-via-stops":"var(--gradient-from), var(--gradient-via), var(--gradient-to)"}}},backgroundImage:{values(e){return{...e("gradients"),...e("assets")}},shorthand:["bgImg","bgImage"]},border:{values:"borders"},borderTop:{values:"borders"},borderLeft:{values:"borders"},borderBlockStart:{values:"borders"},borderRight:{values:"borders"},borderBottom:{values:"borders"},borderBlockEnd:{values:"borders"},borderInlineStart:{values:"borders",shorthand:["borderStart"]},borderInlineEnd:{values:"borders",shorthand:["borderEnd"]},borderInline:{values:"borders",shorthand:["borderX"]},borderBlock:{values:"borders",shorthand:["borderY"]},borderColor:{values:ne,transform:te("borderColor")},borderTopColor:{values:ne,transform:te("borderTopColor")},borderBlockStartColor:{values:ne,transform:te("borderBlockStartColor")},borderBottomColor:{values:ne,transform:te("borderBottomColor")},borderBlockEndColor:{values:ne,transform:te("borderBlockEndColor")},borderLeftColor:{values:ne,transform:te("borderLeftColor")},borderInlineStartColor:{values:ne,shorthand:["borderStartColor"],transform:te("borderInlineStartColor")},borderRightColor:{values:ne,transform:te("borderRightColor")},borderInlineEndColor:{values:ne,shorthand:["borderEndColor"],transform:te("borderInlineEndColor")},borderStyle:{values:"borderStyles"},borderTopStyle:{values:"borderStyles"},borderBlockStartStyle:{values:"borderStyles"},borderBottomStyle:{values:"borderStyles"},borderBlockEndStyle:{values:"borderStyles"},borderInlineStartStyle:{values:"borderStyles",shorthand:["borderStartStyle"]},borderInlineEndStyle:{values:"borderStyles",shorthand:["borderEndStyle"]},borderLeftStyle:{values:"borderStyles"},borderRightStyle:{values:"borderStyles"},borderRadius:{values:"radii",shorthand:["rounded"]},borderTopLeftRadius:{values:"radii",shorthand:["roundedTopLeft"]},borderStartStartRadius:{values:"radii",shorthand:["roundedStartStart","borderTopStartRadius"]},borderEndStartRadius:{values:"radii",shorthand:["roundedEndStart","borderBottomStartRadius"]},borderTopRightRadius:{values:"radii",shorthand:["roundedTopRight"]},borderStartEndRadius:{values:"radii",shorthand:["roundedStartEnd","borderTopEndRadius"]},borderEndEndRadius:{values:"radii",shorthand:["roundedEndEnd","borderBottomEndRadius"]},borderBottomLeftRadius:{values:"radii",shorthand:["roundedBottomLeft"]},borderBottomRightRadius:{values:"radii",shorthand:["roundedBottomRight"]},borderInlineStartRadius:{values:"radii",property:"borderRadius",shorthand:["roundedStart","borderStartRadius"],transform:e=>({borderStartStartRadius:e,borderEndStartRadius:e})},borderInlineEndRadius:{values:"radii",property:"borderRadius",shorthand:["roundedEnd","borderEndRadius"],transform:e=>({borderStartEndRadius:e,borderEndEndRadius:e})},borderTopRadius:{values:"radii",property:"borderRadius",shorthand:["roundedTop"],transform:e=>({borderTopLeftRadius:e,borderTopRightRadius:e})},borderBottomRadius:{values:"radii",property:"borderRadius",shorthand:["roundedBottom"],transform:e=>({borderBottomLeftRadius:e,borderBottomRightRadius:e})},borderLeftRadius:{values:"radii",property:"borderRadius",shorthand:["roundedLeft"],transform:e=>({borderTopLeftRadius:e,borderBottomLeftRadius:e})},borderRightRadius:{values:"radii",property:"borderRadius",shorthand:["roundedRight"],transform:e=>({borderTopRightRadius:e,borderBottomRightRadius:e})},borderWidth:{values:"borderWidths"},borderBlockStartWidth:{values:"borderWidths"},borderTopWidth:{values:"borderWidths"},borderBottomWidth:{values:"borderWidths"},borderBlockEndWidth:{values:"borderWidths"},borderRightWidth:{values:"borderWidths"},borderInlineWidth:{values:"borderWidths",shorthand:["borderXWidth"]},borderInlineStartWidth:{values:"borderWidths",shorthand:["borderStartWidth"]},borderInlineEndWidth:{values:"borderWidths",shorthand:["borderEndWidth"]},borderLeftWidth:{values:"borderWidths"},borderBlockWidth:{values:"borderWidths",shorthand:["borderYWidth"]},color:{values:ne,transform:te("color")},fill:{values:ne,transform:te("fill")},stroke:{values:ne,transform:te("stroke")},accentColor:{values:ne,transform:te("accentColor")},divideX:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderInlineStartWidth:e,borderInlineEndWidth:"0px"}}}},divideY:{values:{type:"string"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderTopWidth:e,borderBottomWidth:"0px"}}}},divideColor:{values:ne,transform(e,t){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":Jk(e,t)}}},divideStyle:{property:"borderStyle",transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{borderStyle:e}}}},boxShadow:{values:"shadows",shorthand:["shadow"]},boxShadowColor:{values:ne,transform:te("--shadow-color"),shorthand:["shadowColor"]},mixBlendMode:{shorthand:["blendMode"]},backgroundBlendMode:{shorthand:["bgBlendMode"]},opacity:{values:"opacity"},filter:{transform(e){return e!=="auto"?{filter:e}:{filter:"var(--blur) var(--brightness) var(--contrast) var(--grayscale) var(--hue-rotate) var(--invert) var(--saturate) var(--sepia) var(--drop-shadow)"}}},blur:{values:"blurs",transform:e=>({"--blur":xe("blur",e)})},brightness:{transform:e=>({"--brightness":xe("brightness",e)})},contrast:{transform:e=>({"--contrast":xe("contrast",e)})},grayscale:{transform:e=>({"--grayscale":xe("grayscale",e)})},hueRotate:{transform:e=>({"--hue-rotate":xe("hue-rotate",hn(e))})},invert:{transform:e=>({"--invert":xe("invert",e)})},saturate:{transform:e=>({"--saturate":xe("saturate",e)})},sepia:{transform:e=>({"--sepia":xe("sepia",e)})},dropShadow:{transform:e=>({"--drop-shadow":xe("drop-shadow",e)})},backdropFilter:{transform(e){return e!=="auto"?{backdropFilter:e}:{backdropFilter:"var(--backdrop-blur) var(--backdrop-brightness) var(--backdrop-contrast) var(--backdrop-grayscale) var(--backdrop-hue-rotate) var(--backdrop-invert) var(--backdrop-opacity) var(--backdrop-saturate) var(--backdrop-sepia)"}}},backdropBlur:{values:"blurs",transform:e=>({"--backdrop-blur":xe("blur",e)})},backdropBrightness:{transform:e=>({"--backdrop-brightness":xe("brightness",e)})},backdropContrast:{transform:e=>({"--backdrop-contrast":xe("contrast",e)})},backdropGrayscale:{transform:e=>({"--backdrop-grayscale":xe("grayscale",e)})},backdropHueRotate:{transform:e=>({"--backdrop-hue-rotate":xe("hue-rotate",hn(e))})},backdropInvert:{transform:e=>({"--backdrop-invert":xe("invert",e)})},backdropOpacity:{transform:e=>({"--backdrop-opacity":xe("opacity",e)})},backdropSaturate:{transform:e=>({"--backdrop-saturate":xe("saturate",e)})},backdropSepia:{transform:e=>({"--backdrop-sepia":xe("sepia",e)})},flexBasis:{values:"sizes"},gap:{values:"spacing"},rowGap:{values:"spacing",shorthand:["gapY"]},columnGap:{values:"spacing",shorthand:["gapX"]},flexDirection:{shorthand:["flexDir"]},gridGap:{values:"spacing"},gridColumnGap:{values:"spacing"},gridRowGap:{values:"spacing"},outlineColor:{values:ne,transform:te("outlineColor")},focusRing:rh("&:is(:focus, [data-focus])"),focusVisibleRing:rh("&:is(:focus-visible, [data-focus-visible])"),focusRingColor:{values:ne,transform:te("--focus-ring-color")},focusRingOffset:{values:"spacing",transform:e=>({"--focus-ring-offset":e})},focusRingWidth:{values:"borderWidths",property:"outlineWidth",transform:e=>({"--focus-ring-width":e})},focusRingStyle:{values:"borderStyles",property:"outlineStyle",transform:e=>({"--focus-ring-style":e})},aspectRatio:{values:"aspectRatios"},width:{values:"sizes",shorthand:["w"]},inlineSize:{values:"sizes"},height:{values:"sizes",shorthand:["h"]},blockSize:{values:"sizes"},boxSize:{values:"sizes",property:"width",transform:e=>({width:e,height:e})},minWidth:{values:"sizes",shorthand:["minW"]},minInlineSize:{values:"sizes"},minHeight:{values:"sizes",shorthand:["minH"]},minBlockSize:{values:"sizes"},maxWidth:{values:"sizes",shorthand:["maxW"]},maxInlineSize:{values:"sizes"},maxHeight:{values:"sizes",shorthand:["maxH"]},maxBlockSize:{values:"sizes"},hideFrom:{values:"breakpoints",transform:(e,{raw:t,token:n})=>({[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}`:`@media screen and (min-width: ${e})`]:{display:"none"}})},hideBelow:{values:"breakpoints",transform(e,{raw:t,token:n}){return{[n.raw(`breakpoints.${t}`)?`@breakpoint ${t}Down`:`@media screen and (max-width: ${e})`]:{display:"none"}}}},overscrollBehavior:{shorthand:["overscroll"]},overscrollBehaviorX:{shorthand:["overscrollX"]},overscrollBehaviorY:{shorthand:["overscrollY"]},scrollbar:{values:["visible","hidden"],transform(e){switch(e){case"visible":return{msOverflowStyle:"auto",scrollbarWidth:"auto","&::-webkit-scrollbar":{display:"block"}};case"hidden":return{msOverflowStyle:"none",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}};default:return{}}}},scrollbarColor:{values:ne,transform:te("scrollbarColor")},scrollbarGutter:{values:"spacing"},scrollbarWidth:{values:"sizes"},scrollMargin:{values:"spacing"},scrollMarginTop:{values:"spacing"},scrollMarginBottom:{values:"spacing"},scrollMarginLeft:{values:"spacing"},scrollMarginRight:{values:"spacing"},scrollMarginX:{values:"spacing",transform:e=>({scrollMarginLeft:e,scrollMarginRight:e})},scrollMarginY:{values:"spacing",transform:e=>({scrollMarginTop:e,scrollMarginBottom:e})},scrollPadding:{values:"spacing"},scrollPaddingTop:{values:"spacing"},scrollPaddingBottom:{values:"spacing"},scrollPaddingLeft:{values:"spacing"},scrollPaddingRight:{values:"spacing"},scrollPaddingInline:{values:"spacing",shorthand:["scrollPaddingX"]},scrollPaddingBlock:{values:"spacing",shorthand:["scrollPaddingY"]},scrollSnapType:{values:{none:"none",x:"x var(--scroll-snap-strictness)",y:"y var(--scroll-snap-strictness)",both:"both var(--scroll-snap-strictness)"}},scrollSnapStrictness:{values:["mandatory","proximity"],transform:e=>({"--scroll-snap-strictness":e})},scrollSnapMargin:{values:"spacing"},scrollSnapMarginTop:{values:"spacing"},scrollSnapMarginBottom:{values:"spacing"},scrollSnapMarginLeft:{values:"spacing"},scrollSnapMarginRight:{values:"spacing"},listStylePosition:{shorthand:["listStylePos"]},listStyleImage:{values:"assets",shorthand:["listStyleImg"]},position:{shorthand:["pos"]},zIndex:{values:"zIndex"},inset:{values:"spacing"},insetInline:{values:"spacing",shorthand:["insetX"]},insetBlock:{values:"spacing",shorthand:["insetY"]},top:{values:"spacing"},insetBlockStart:{values:"spacing"},bottom:{values:"spacing"},insetBlockEnd:{values:"spacing"},left:{values:"spacing"},right:{values:"spacing"},insetInlineStart:{values:"spacing",shorthand:["insetStart"]},insetInlineEnd:{values:"spacing",shorthand:["insetEnd"]},ring:{transform(e){return{"--ring-offset-shadow":"var(--ring-inset) 0 0 0 var(--ring-offset-width) var(--ring-offset-color)","--ring-shadow":"var(--ring-inset) 0 0 0 calc(var(--ring-width) + var(--ring-offset-width)) var(--ring-color)","--ring-width":e,boxShadow:"var(--ring-offset-shadow), var(--ring-shadow), var(--shadow, 0 0 #0000)"}}},ringColor:{values:ne,transform:te("--ring-color")},ringOffset:{transform:e=>({"--ring-offset-width":e})},ringOffsetColor:{values:ne,transform:te("--ring-offset-color")},ringInset:{transform:e=>({"--ring-inset":e})},margin:{values:"spacing",shorthand:["m"]},marginTop:{values:"spacing",shorthand:["mt"]},marginBlockStart:{values:"spacing"},marginRight:{values:"spacing",shorthand:["mr"]},marginBottom:{values:"spacing",shorthand:["mb"]},marginBlockEnd:{values:"spacing"},marginLeft:{values:"spacing",shorthand:["ml"]},marginInlineStart:{values:"spacing",shorthand:["ms","marginStart"]},marginInlineEnd:{values:"spacing",shorthand:["me","marginEnd"]},marginInline:{values:"spacing",shorthand:["mx","marginX"]},marginBlock:{values:"spacing",shorthand:["my","marginY"]},padding:{values:"spacing",shorthand:["p"]},paddingTop:{values:"spacing",shorthand:["pt"]},paddingRight:{values:"spacing",shorthand:["pr"]},paddingBottom:{values:"spacing",shorthand:["pb"]},paddingBlockStart:{values:"spacing"},paddingBlockEnd:{values:"spacing"},paddingLeft:{values:"spacing",shorthand:["pl"]},paddingInlineStart:{values:"spacing",shorthand:["ps","paddingStart"]},paddingInlineEnd:{values:"spacing",shorthand:["pe","paddingEnd"]},paddingInline:{values:"spacing",shorthand:["px","paddingX"]},paddingBlock:{values:"spacing",shorthand:["py","paddingY"]},textDecoration:{shorthand:["textDecor"]},textDecorationColor:{values:ne,transform:te("textDecorationColor")},textShadow:{values:"shadows"},transform:{transform:e=>{let t=e;return e==="auto"&&(t="translateX(var(--translate-x, 0)) translateY(var(--translate-y, 0)) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),e==="auto-gpu"&&(t="translate3d(var(--translate-x, 0), var(--translate-y, 0), 0) rotate(var(--rotate, 0)) scaleX(var(--scale-x, 1)) scaleY(var(--scale-y, 1)) skewX(var(--skew-x, 0)) skewY(var(--skew-y, 0))"),{transform:t}}},skewX:{transform:e=>({"--skew-x":hn(e)})},skewY:{transform:e=>({"--skew-y":hn(e)})},scaleX:{transform:e=>({"--scale-x":e})},scaleY:{transform:e=>({"--scale-y":e})},scale:{transform(e){return e!=="auto"?{scale:e}:{scale:"var(--scale-x, 1) var(--scale-y, 1)"}}},spaceXReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":e?"1":void 0}}}},spaceX:{property:"marginInlineStart",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-x-reverse":"0",marginInlineStart:`calc(${e} * calc(1 - var(--space-x-reverse)))`,marginInlineEnd:`calc(${e} * var(--space-x-reverse))`}})},spaceYReverse:{values:{type:"boolean"},transform(e){return{"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":e?"1":void 0}}}},spaceY:{property:"marginTop",values:"spacing",transform:e=>({"& > :not(style, [hidden]) ~ :not(style, [hidden])":{"--space-y-reverse":"0",marginTop:`calc(${e} * calc(1 - var(--space-y-reverse)))`,marginBottom:`calc(${e} * var(--space-y-reverse))`}})},rotate:{transform(e){return e!=="auto"?{rotate:hn(e)}:{rotate:"var(--rotate-x, 0) var(--rotate-y, 0) var(--rotate-z, 0)"}}},rotateX:{transform(e){return{"--rotate-x":hn(e)}}},rotateY:{transform(e){return{"--rotate-y":hn(e)}}},translate:{transform(e){return e!=="auto"?{translate:e}:{translate:"var(--translate-x) var(--translate-y)"}}},translateX:{values:"spacing",transform:e=>({"--translate-x":e})},translateY:{values:"spacing",transform:e=>({"--translate-y":e})},transition:{values:["all","common","colors","opacity","position","backgrounds","size","shadow","transform"],transform(e){switch(e){case"all":return Et("all");case"position":return Et("left, right, top, bottom, inset-inline, inset-block");case"colors":return Et("color, background-color, border-color, text-decoration-color, fill, stroke");case"opacity":return Et("opacity");case"shadow":return Et("box-shadow");case"transform":return Et("transform");case"size":return Et("width, height");case"backgrounds":return Et("background, background-color, background-image, background-position");case"common":return Et("color, background-color, border-color, text-decoration-color, fill, stroke, opacity, box-shadow, transform, filter, backdrop-filter");default:return{transition:e}}}},transitionDuration:{values:"durations"},transitionProperty:{values:{common:"background-color, border-color, color, fill, stroke, opacity, box-shadow, translate, transform",colors:"background-color, border-color, color, fill, stroke",size:"width, height",position:"left, right, top, bottom, inset-inline, inset-block",background:"background, background-color, background-image, background-position"}},transitionTimingFunction:{values:"easings"},animation:{values:"animations"},animationDuration:{values:"durations"},animationDelay:{values:"durations"},animationTimingFunction:{values:"easings"},fontFamily:{values:"fonts"},fontSize:{values:"fontSizes"},fontWeight:{values:"fontWeights"},lineHeight:{values:"lineHeights"},letterSpacing:{values:"letterSpacings"},textIndent:{values:"spacing"},truncate:{values:{type:"boolean"},transform(e){return e===!0?{overflow:"hidden",textOverflow:"ellipsis",whiteSpace:"nowrap"}:{}}},lineClamp:{transform(e){return e==="none"?{WebkitLineClamp:"unset"}:{overflow:"hidden",display:"-webkit-box",WebkitLineClamp:e,WebkitBoxOrient:"vertical",textWrap:"wrap"}}},borderSpacing:{values:e=>({...e("spacing"),auto:"var(--border-spacing-x, 0) var(--border-spacing-y, 0)"})},borderSpacingX:{values:"spacing",transform(e){return{"--border-spacing-x":e}}},borderSpacingY:{values:"spacing",transform(e){return{"--border-spacing-y":e}}},srOnly:{values:{type:"boolean"},transform(e){return tC[e]||{}}},debug:{values:{type:"boolean"},transform(e){return e?{outline:"1px solid blue !important","& > *":{outline:"1px solid red !important"}}:{}}},caretColor:{values:ne,transform:te("caretColor")},cursor:{values:"cursor"}}}),tC={true:{position:"absolute",width:"1px",height:"1px",padding:"0",margin:"-1px",overflow:"hidden",clip:"rect(0, 0, 0, 0)",whiteSpace:"nowrap",borderWidth:"0"},false:{position:"static",width:"auto",height:"auto",padding:"0",margin:"0",overflow:"visible",clip:"auto",whiteSpace:"normal"}};var nC="",rC=nC.split(","),oC="WebkitAppearance,WebkitBorderBefore,WebkitBorderBeforeColor,WebkitBorderBeforeStyle,WebkitBorderBeforeWidth,WebkitBoxReflect,WebkitLineClamp,WebkitMask,WebkitMaskAttachment,WebkitMaskClip,WebkitMaskComposite,WebkitMaskImage,WebkitMaskOrigin,WebkitMaskPosition,WebkitMaskPositionX,WebkitMaskPositionY,WebkitMaskRepeat,WebkitMaskRepeatX,WebkitMaskRepeatY,WebkitMaskSize,WebkitOverflowScrolling,WebkitTapHighlightColor,WebkitTextFillColor,WebkitTextStroke,WebkitTextStrokeColor,WebkitTextStrokeWidth,WebkitTouchCallout,WebkitUserModify,WebkitUserSelect,accentColor,alignContent,alignItems,alignSelf,alignTracks,all,anchorName,anchorScope,animation,animationComposition,animationDelay,animationDirection,animationDuration,animationFillMode,animationIterationCount,animationName,animationPlayState,animationRange,animationRangeEnd,animationRangeStart,animationTimeline,animationTimingFunction,appearance,aspectRatio,backdropFilter,backfaceVisibility,background,backgroundAttachment,backgroundBlendMode,backgroundClip,backgroundColor,backgroundImage,backgroundOrigin,backgroundPosition,backgroundPositionX,backgroundPositionY,backgroundRepeat,backgroundSize,blockSize,border,borderBlock,borderBlockColor,borderBlockEnd,borderBlockEndColor,borderBlockEndStyle,borderBlockEndWidth,borderBlockStart,borderBlockStartColor,borderBlockStartStyle,borderBlockStartWidth,borderBlockStyle,borderBlockWidth,borderBottom,borderBottomColor,borderBottomLeftRadius,borderBottomRightRadius,borderBottomStyle,borderBottomWidth,borderCollapse,borderColor,borderEndEndRadius,borderEndStartRadius,borderImage,borderImageOutset,borderImageRepeat,borderImageSlice,borderImageSource,borderImageWidth,borderInline,borderInlineColor,borderInlineEnd,borderInlineEndColor,borderInlineEndStyle,borderInlineEndWidth,borderInlineStart,borderInlineStartColor,borderInlineStartStyle,borderInlineStartWidth,borderInlineStyle,borderInlineWidth,borderLeft,borderLeftColor,borderLeftStyle,borderLeftWidth,borderRadius,borderRight,borderRightColor,borderRightStyle,borderRightWidth,borderSpacing,borderStartEndRadius,borderStartStartRadius,borderStyle,borderTop,borderTopColor,borderTopLeftRadius,borderTopRightRadius,borderTopStyle,borderTopWidth,borderWidth,bottom,boxAlign,boxDecorationBreak,boxDirection,boxFlex,boxFlexGroup,boxLines,boxOrdinalGroup,boxOrient,boxPack,boxShadow,boxSizing,breakAfter,breakBefore,breakInside,captionSide,caret,caretColor,caretShape,clear,clip,clipPath,clipRule,color,colorInterpolationFilters,colorScheme,columnCount,columnFill,columnGap,columnRule,columnRuleColor,columnRuleStyle,columnRuleWidth,columnSpan,columnWidth,columns,contain,containIntrinsicBlockSize,containIntrinsicHeight,containIntrinsicInlineSize,containIntrinsicSize,containIntrinsicWidth,container,containerName,containerType,content,contentVisibility,counterIncrement,counterReset,counterSet,cursor,cx,cy,d,direction,display,dominantBaseline,emptyCells,fieldSizing,fill,fillOpacity,fillRule,filter,flex,flexBasis,flexDirection,flexFlow,flexGrow,flexShrink,flexWrap,float,floodColor,floodOpacity,font,fontFamily,fontFeatureSettings,fontKerning,fontLanguageOverride,fontOpticalSizing,fontPalette,fontSize,fontSizeAdjust,fontSmooth,fontStretch,fontStyle,fontSynthesis,fontSynthesisPosition,fontSynthesisSmallCaps,fontSynthesisStyle,fontSynthesisWeight,fontVariant,fontVariantAlternates,fontVariantCaps,fontVariantEastAsian,fontVariantEmoji,fontVariantLigatures,fontVariantNumeric,fontVariantPosition,fontVariationSettings,fontWeight,forcedColorAdjust,gap,grid,gridArea,gridAutoColumns,gridAutoFlow,gridAutoRows,gridColumn,gridColumnEnd,gridColumnGap,gridColumnStart,gridGap,gridRow,gridRowEnd,gridRowGap,gridRowStart,gridTemplate,gridTemplateAreas,gridTemplateColumns,gridTemplateRows,hangingPunctuation,height,hyphenateCharacter,hyphenateLimitChars,hyphens,imageOrientation,imageRendering,imageResolution,imeMode,initialLetter,initialLetterAlign,inlineSize,inset,insetBlock,insetBlockEnd,insetBlockStart,insetInline,insetInlineEnd,insetInlineStart,interpolateSize,isolation,justifyContent,justifyItems,justifySelf,justifyTracks,left,letterSpacing,lightingColor,lineBreak,lineClamp,lineHeight,lineHeightStep,listStyle,listStyleImage,listStylePosition,listStyleType,margin,marginBlock,marginBlockEnd,marginBlockStart,marginBottom,marginInline,marginInlineEnd,marginInlineStart,marginLeft,marginRight,marginTop,marginTrim,marker,markerEnd,markerMid,markerStart,mask,maskBorder,maskBorderMode,maskBorderOutset,maskBorderRepeat,maskBorderSlice,maskBorderSource,maskBorderWidth,maskClip,maskComposite,maskImage,maskMode,maskOrigin,maskPosition,maskRepeat,maskSize,maskType,masonryAutoFlow,mathDepth,mathShift,mathStyle,maxBlockSize,maxHeight,maxInlineSize,maxLines,maxWidth,minBlockSize,minHeight,minInlineSize,minWidth,mixBlendMode,objectFit,objectPosition,offset,offsetAnchor,offsetDistance,offsetPath,offsetPosition,offsetRotate,opacity,order,orphans,outline,outlineColor,outlineOffset,outlineStyle,outlineWidth,overflow,overflowAnchor,overflowBlock,overflowClipBox,overflowClipMargin,overflowInline,overflowWrap,overflowX,overflowY,overlay,overscrollBehavior,overscrollBehaviorBlock,overscrollBehaviorInline,overscrollBehaviorX,overscrollBehaviorY,padding,paddingBlock,paddingBlockEnd,paddingBlockStart,paddingBottom,paddingInline,paddingInlineEnd,paddingInlineStart,paddingLeft,paddingRight,paddingTop,page,pageBreakAfter,pageBreakBefore,pageBreakInside,paintOrder,perspective,perspectiveOrigin,placeContent,placeItems,placeSelf,pointerEvents,position,positionAnchor,positionArea,positionTry,positionTryFallbacks,positionTryOrder,positionVisibility,printColorAdjust,quotes,r,resize,right,rotate,rowGap,rubyAlign,rubyMerge,rubyPosition,rx,ry,scale,scrollBehavior,scrollMargin,scrollMarginBlock,scrollMarginBlockEnd,scrollMarginBlockStart,scrollMarginBottom,scrollMarginInline,scrollMarginInlineEnd,scrollMarginInlineStart,scrollMarginLeft,scrollMarginRight,scrollMarginTop,scrollPadding,scrollPaddingBlock,scrollPaddingBlockEnd,scrollPaddingBlockStart,scrollPaddingBottom,scrollPaddingInline,scrollPaddingInlineEnd,scrollPaddingInlineStart,scrollPaddingLeft,scrollPaddingRight,scrollPaddingTop,scrollSnapAlign,scrollSnapCoordinate,scrollSnapDestination,scrollSnapPointsX,scrollSnapPointsY,scrollSnapStop,scrollSnapType,scrollSnapTypeX,scrollSnapTypeY,scrollTimeline,scrollTimelineAxis,scrollTimelineName,scrollbarColor,scrollbarGutter,scrollbarWidth,shapeImageThreshold,shapeMargin,shapeOutside,shapeRendering,stopColor,stopOpacity,stroke,strokeDasharray,strokeDashoffset,strokeLinecap,strokeLinejoin,strokeMiterlimit,strokeOpacity,strokeWidth,tabSize,tableLayout,textAlign,textAlignLast,textAnchor,textBox,textBoxEdge,textBoxTrim,textCombineUpright,textDecoration,textDecorationColor,textDecorationLine,textDecorationSkip,textDecorationSkipInk,textDecorationStyle,textDecorationThickness,textEmphasis,textEmphasisColor,textEmphasisPosition,textEmphasisStyle,textIndent,textJustify,textOrientation,textOverflow,textRendering,textShadow,textSizeAdjust,textSpacingTrim,textTransform,textUnderlineOffset,textUnderlinePosition,textWrap,textWrapMode,textWrapStyle,timelineScope,top,touchAction,transform,transformBox,transformOrigin,transformStyle,transition,transitionBehavior,transitionDelay,transitionDuration,transitionProperty,transitionTimingFunction,translate,unicodeBidi,userSelect,vectorEffect,verticalAlign,viewTimeline,viewTimelineAxis,viewTimelineInset,viewTimelineName,viewTransitionName,visibility,whiteSpace,whiteSpaceCollapse,widows,width,willChange,wordBreak,wordSpacing,wordWrap,writingMode,x,y,zIndex,zoom,alignmentBaseline,baselineShift,colorInterpolation,colorRendering,glyphOrientationVertical",iC=oC.split(",").concat(rC),sC=new Map(iC.map(e=>[e,!0]));function aC(e){const t=Object.create(null);return n=>(t[n]===void 0&&(t[n]=e(n)),t[n])}var lC=/&|@/,cC=aC(e=>sC.has(e)||e.startsWith("--")||lC.test(e));function ih(e,t){const n={};return gt(e,(r,o)=>{r&&(n[o.join(".")]=r.value)},{stop:t}),n}var Ca,sh;function uC(){if(sh)return Ca;sh=1,Ca=i,i.default=i,i.stable=c,i.stableStringify=c;var e="[...]",t="[Circular]",n=[],r=[];function o(){return{depthLimit:Number.MAX_SAFE_INTEGER,edgesLimit:Number.MAX_SAFE_INTEGER}}function i(d,m,f,p){typeof p>"u"&&(p=o()),a(d,"",0,[],void 0,0,p);var v;try{r.length===0?v=JSON.stringify(d,m,f):v=JSON.stringify(d,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var b=n.pop();b.length===4?Object.defineProperty(b[0],b[1],b[3]):b[0][b[1]]=b[2]}}return v}function s(d,m,f,p){var v=Object.getOwnPropertyDescriptor(p,f);v.get!==void 0?v.configurable?(Object.defineProperty(p,f,{value:d}),n.push([p,f,m,v])):r.push([m,f,d]):(p[f]=d,n.push([p,f,m]))}function a(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;xm?1:0}function c(d,m,f,p){typeof p>"u"&&(p=o());var v=u(d,"",0,[],void 0,0,p)||d,b;try{r.length===0?b=JSON.stringify(v,m,f):b=JSON.stringify(v,h(m),f)}catch{return JSON.stringify("[unable to serialize, circular reference is too complex to analyze]")}finally{for(;n.length!==0;){var y=n.pop();y.length===4?Object.defineProperty(y[0],y[1],y[3]):y[0][y[1]]=y[2]}}return b}function u(d,m,f,p,v,b,y){b+=1;var x;if(typeof d=="object"&&d!==null){for(x=0;xy.depthLimit){s(e,d,m,v);return}if(typeof y.edgesLimit<"u"&&f+1>y.edgesLimit){s(e,d,m,v);return}if(p.push(d),Array.isArray(d))for(x=0;x0)for(var p=0;p{const t=Object.create(null);function n(...r){const o=r.map(i=>hC(i)).join("|");return t[o]===void 0&&(t[o]=e(...r)),t[o]}return n},ah=16,li="px",wa="em",jr="rem";function lh(e=""){const t=new RegExp(String.raw`-?\d+(?:\.\d+|\d*)`),n=new RegExp(`${li}|${wa}|${jr}`);return e.match(new RegExp(`${t.source}(${n.source})`))?.[1]}function ch(e=""){if(typeof e=="number")return`${e}px`;const t=lh(e);if(!t||t===li)return e;if(t===wa||t===jr)return`${parseFloat(e)*ah}${li}`}function uh(e=""){const t=lh(e);if(!t||t===jr)return e;if(t===wa)return`${parseFloat(e)}${jr}`;if(t===li)return`${parseFloat(e)/ah}${jr}`}const fC=e=>e.charAt(0).toUpperCase()+e.slice(1);function gC(e){const t=pC(e),n=Object.fromEntries(t);function r(d){return n[d]}function o(d){return rr(r(d))}function i(){const d=Object.keys(n),m=mC(d),f=d.flatMap(p=>{const v=r(p),b=[`${p}Down`,rr({max:ci(v.min)})],y=[p,rr({min:v.min})],x=[`${p}Only`,o(p)];return[y,x,b]}).filter(([,p])=>p!=="").concat(m.map(([p,v])=>{const b=r(p),y=r(v);return[`${p}To${fC(v)}`,rr({min:b.min,max:ci(y.min)})]}));return Object.fromEntries(f)}function s(){const d=i();return Object.fromEntries(Object.entries(d))}const a=s(),l=d=>a[d];function c(){return["base",...Object.keys(n)]}function u(d){return rr({min:r(d).min})}function h(d){return rr({max:ci(r(d).min)})}return{values:Object.values(n),only:o,keys:c,conditions:a,getCondition:l,up:u,down:h}}function ci(e){const t=parseFloat(ch(e)??"")-.04;return uh(`${t}px`)}function pC(e){return Object.entries(e).sort(([,n],[,r])=>parseInt(n,10){let s=null;return o<=i.length-1&&(s=i[o+1]?.[1]),s!=null&&(s=ci(s)),[n,{name:n,min:uh(r),max:s}]})}function mC(e){const t=[];return e.forEach((n,r)=>{let o=r;o++;let i=e[o];for(;i;)t.push([n,i]),o++,i=e[o]}),t}function rr({min:e,max:t}){return e==null&&t==null?"":["@media screen",e&&`(min-width: ${e})`,t&&`(max-width: ${t})`].filter(Boolean).join(" and ")}const vC=(e,t)=>Object.fromEntries(Object.entries(e).map(([n,r])=>t(n,r))),bC=e=>{const{breakpoints:t,conditions:n={}}=e,r=vC(n,(u,h)=>[`_${u}`,h]),o=Object.assign({},r,t.conditions);function i(){return Object.keys(o)}function s(u){return i().includes(u)||/^@|&|&$/.test(u)||u.startsWith("_")}function a(u){return u.filter(h=>h!=="base").sort((h,d)=>{const m=s(h),f=s(d);return m&&!f?1:!m&&f?-1:0})}function l(u){return u.startsWith("@breakpoint")?t.getCondition(u.replace("@breakpoint ","")):u}function c(u){return Reflect.get(o,u)||u}return{keys:i,sort:a,has:s,resolve:c,breakpoints:t.keys(),expandAtRule:l}},dh=e=>({minMax:new RegExp(`(!?\\(\\s*min(-device-)?-${e})(.| )+\\(\\s*max(-device)?-${e}`,"i"),min:new RegExp(`\\(\\s*min(-device)?-${e}`,"i"),maxMin:new RegExp(`(!?\\(\\s*max(-device)?-${e})(.| -)+\\(\\s*min(-device)?-${e}`,"i"),max:new RegExp(`\\(\\s*max(-device)?-${e}`,"i")}),yC=uh("width"),xC=uh("height"),dh=e=>({isMin:vh(e.minMax,e.maxMin,e.min),isMax:vh(e.maxMin,e.minMax,e.max)}),{isMin:Sa,isMax:hh}=dh(yC),{isMin:Ea,isMax:fh}=dh(xC),gh=/print/i,ph=/^print$/i,kC=/(-?\d*\.?\d+)(ch|em|ex|px|rem)/,CC=/(\d)/,$r=Number.MAX_VALUE,wC={ch:8.8984375,em:16,rem:16,ex:8.296875,px:1};function mh(e){const t=kC.exec(e)||(Sa(e)||Ea(e)?CC.exec(e):null);if(!t)return $r;if(t[0]==="0")return 0;const n=parseFloat(t[1]),r=t[2];return n*(wC[r]||1)}function vh(e,t,n){return r=>e.test(r)||!t.test(r)&&n.test(r)}function SC(e,t){const n=gh.test(e),r=ph.test(e),o=gh.test(t),i=ph.test(t);return n&&o?!r&&i?1:r&&!i?-1:e.localeCompare(t):n?1:o?-1:null}const EC=rt((e,t)=>{const n=SC(e,t);if(n!==null)return n;const r=Sa(e)||Ea(e),o=hh(e)||fh(e),i=Sa(t)||Ea(t),s=hh(t)||fh(t);if(r&&s)return-1;if(o&&i)return 1;const a=mh(e),l=mh(t);return a===$r&&l===$r?e.localeCompare(t):a===$r?1:l===$r?-1:a!==l?a>l?o?-1:1:o?1:-1:e.localeCompare(t)});function bh(e){return e.sort(([t],[n])=>EC(t,n))}function yh(e){const t=[],n=[],r={};for(const[s,a]of Object.entries(e))s.startsWith("@media")?t.push([s,a]):s.startsWith("@container")?n.push([s,a]):_e(a)?r[s]=yh(a):r[s]=a;const o=bh(t),i=bh(n);return{...r,...Object.fromEntries(o),...Object.fromEntries(i)}}const xh=/\s*!(important)?/i,OC=e=>lt(e)?xh.test(e):!1,PC=e=>lt(e)?e.replace(xh,"").trim():e;function kh(e){const{transform:t,conditions:n,normalize:r}=e,o=TC(e);return rt(function(...s){const a=o(...s),l=r(a),c=Object.create(null);return gt(l,(u,h)=>{const d=OC(u);if(u==null)return;const[m,...f]=n.sort(h).map(n.resolve);d&&(u=PC(u));let p=t(m,u)??Object.create(null);p=gt(p,v=>lt(v)&&d?`${v} !important`:v,{getKey:v=>n.expandAtRule(v)}),RC(c,f.flat(),p)}),yh(c)})}function RC(e,t,n){let r=e;for(const o of t)o&&(r[o]||(r[o]=Object.create(null)),r=r[o]);er(r,n)}function IC(...e){return e.filter(t=>_e(t)&&Object.keys(yr(t)).length>0)}function TC(e){function t(n){const r=IC(...n);return r.length===1?r:r.map(o=>e.normalize(o))}return rt(function(...r){return er({},...t(r))})}const Ch=e=>({base:{},variants:{},defaultVariants:{},compoundVariants:[],...e});function NC(e){const{css:t,conditions:n,normalize:r,layers:o}=e;function i(a={}){const{base:l,variants:c,defaultVariants:u,compoundVariants:h}=Ch(a),d=kh({conditions:n,normalize:r,transform(y,x){return c[y]?.[x]}}),m=(y={})=>{const x=r({...u,...yr(y)});let w={...l};er(w,d(x));const k=s(h,x);return o.wrap("recipes",t(w,k))},f=Object.keys(c),p=y=>{const x=Qd(y,["recipe"]),[w,k]=Ln(x,f);return f.includes("colorPalette")||(w.colorPalette=y.colorPalette||u.colorPalette),f.includes("orientation")&&(k.orientation=y.orientation),[w,k]},v=Object.fromEntries(Object.entries(c).map(([y,x])=>[y,Object.keys(x)]));return Object.assign(y=>t(m(y)),{className:a.className,__cva__:!0,variantMap:v,variantKeys:f,raw:m,config:a,splitVariantProps:p,merge(y){return i(AC(e)(this,y))}})}function s(a,l){let c={};return a.forEach(u=>{Object.entries(u).every(([d,m])=>d==="css"?!0:(Array.isArray(m)?m:[m]).some(p=>l[d]===p))&&(c=t(c,u.css))}),c}return i}function AC(e){const{css:t}=e;return function(r,o){const i=Ch(o.config),s=Wl(r.variantKeys,Object.keys(o.variants)),a=t(r.base,i.base),l=Object.fromEntries(s.map(d=>[d,t(r.config.variants[d],i.variants[d])])),c=er(r.config.defaultVariants,i.defaultVariants),u=[...r.compoundVariants,...i.compoundVariants];return{className:Ge(r.className,o.className),base:a,variants:l,defaultVariants:c,compoundVariants:u}}}const _C={reset:"reset",base:"base",tokens:"tokens",recipes:"recipes"},wh={reset:0,base:1,tokens:2,recipes:3};function VC(e){const t=e.layers??_C,r=Object.values(t).sort((o,i)=>wh[o]-wh[i]);return{names:r,atRule:`@layer ${r.join(", ")};`,wrap(o,i){return e.disableLayers?i:{[`@layer ${t[o]}`]:i}}}}function FC(e){const{utility:t,normalize:n}=e,{hasShorthand:r,resolveShorthand:o}=t;return function(i){return gt(i,n,{stop:s=>Array.isArray(s),getKey:r?o:void 0})}}function LC(e){const{preflight:t}=e;if(!t)return{};const{scope:n="",level:r="parent"}=_e(t)?t:{};let o="";n&&r==="parent"?o=`${n} `:n&&r==="element"&&(o=`&${n}`);const i={"*":{margin:"0px",padding:"0px",font:"inherit",wordWrap:"break-word",WebkitTapHighlightColor:"transparent"},"*, *::before, *::after, *::backdrop":{boxSizing:"border-box",borderWidth:"0px",borderStyle:"solid",borderColor:"var(--global-color-border, currentColor)"},hr:{height:"0px",color:"inherit",borderTopWidth:"1px"},body:{minHeight:"100dvh",position:"relative"},img:{borderStyle:"none"},"img, svg, video, canvas, audio, iframe, embed, object":{display:"block",verticalAlign:"middle"},iframe:{border:"none"},"img, video":{maxWidth:"100%",height:"auto"},"p, h1, h2, h3, h4, h5, h6":{overflowWrap:"break-word"},"ol, ul":{listStyle:"none"},"code, kbd, pre, samp":{fontSize:"1em"},"button, [type='button'], [type='reset'], [type='submit']":{WebkitAppearance:"button",backgroundColor:"transparent",backgroundImage:"none"},"button, input, optgroup, select, textarea":{color:"inherit"},"button, select":{textTransform:"none"},table:{textIndent:"0px",borderColor:"inherit",borderCollapse:"collapse"},"*::placeholder":{opacity:"unset",color:"#9ca3af",userSelect:"none"},textarea:{resize:"vertical"},summary:{display:"list-item"},small:{fontSize:"80%"},"sub, sup":{fontSize:"75%",lineHeight:0,position:"relative",verticalAlign:"baseline"},sub:{bottom:"-0.25em"},sup:{top:"-0.5em"},dialog:{padding:"0px"},a:{color:"inherit",textDecoration:"inherit"},"abbr:where([title])":{textDecoration:"underline dotted"},"b, strong":{fontWeight:"bolder"},"code, kbd, samp, pre":{fontSize:"1em","--font-mono-fallback":"ui-monospace, SFMono-Regular, Menlo, Monaco, Consolas, 'Liberation Mono', 'Courier New'",fontFamily:"var(--global-font-mono, var(--font-mono-fallback))"},'input[type="text"], input[type="email"], input[type="search"], input[type="password"]':{WebkitAppearance:"none",MozAppearance:"none"},"input[type='search']":{WebkitAppearance:"textfield",outlineOffset:"-2px"},"::-webkit-search-decoration, ::-webkit-search-cancel-button":{WebkitAppearance:"none"},"::-webkit-file-upload-button":{WebkitAppearance:"button",font:"inherit"},'input[type="number"]::-webkit-inner-spin-button, input[type="number"]::-webkit-outer-spin-button':{height:"auto"},"input[type='number']":{MozAppearance:"textfield"},":-moz-ui-invalid":{boxShadow:"none"},":-moz-focusring":{outline:"auto"},"[hidden]:where(:not([hidden='until-found']))":{display:"none !important"}},s={[n||"html"]:{lineHeight:1.5,"--font-fallback":"ui-sans-serif, system-ui, -apple-system, BlinkMacSystemFont, 'Segoe UI', Roboto, 'Helvetica Neue', Arial, 'Noto Sans', sans-serif, 'Apple Color Emoji', 'Segoe UI Emoji', 'Segoe UI Symbol', 'Noto Color Emoji'",WebkitTextSizeAdjust:"100%",WebkitFontSmoothing:"antialiased",MozOsxFontSmoothing:"grayscale",textRendering:"optimizeLegibility",touchAction:"manipulation",MozTabSize:"4",tabSize:"4",fontFamily:"var(--global-font-body, var(--font-fallback))"}};if(r==="element"){const a=Object.entries(i).reduce((l,[c,u])=>(l[c]={[o]:u},l),{});Object.assign(s,a)}else o?s[o]=i:Object.assign(s,i);return s}function DC(e){const{conditions:t,isValidProperty:n}=e;return function(o){return gt(o,i=>i,{getKey:(i,s)=>_e(s)&&!t.has(i)&&!n(i)?zC(i).map(a=>"&"+a).join(", "):i})}}function zC(e){const t=[];let n=0,r="",o=!1;for(let i=0;i{const t=o=>({base:e.base?.[o]??{},variants:{},defaultVariants:e.defaultVariants??{},compoundVariants:e.compoundVariants?jC(e.compoundVariants,o):[]}),r=(e.slots??[]).map(o=>[o,t(o)]);for(const[o,i]of Object.entries(e.variants??{}))for(const[s,a]of Object.entries(i))r.forEach(([l,c])=>{var u;(u=c.variants)[o]??(u[o]={}),c.variants[o][s]=a[l]??{}});return Object.fromEntries(r)},jC=(e,t)=>e.filter(n=>n.css[t]).map(n=>({...n,css:n.css[t]}));function $C(e){const{cva:t}=e;return function(r={}){const o=Object.entries(MC(r)).map(([h,d])=>[h,t(d)]);function i(h){const d=o.map(([m,f])=>[m,f(h)]);return Object.fromEntries(d)}const s=r.variants??{},a=Object.keys(s);function l(h){const d=Qd(h,["recipe"]),[m,f]=Ln(d,a);return a.includes("colorPalette")||(m.colorPalette=h.colorPalette||r.defaultVariants?.colorPalette),a.includes("orientation")&&(f.orientation=h.orientation),[m,f]}const c=Object.fromEntries(Object.entries(s).map(([h,d])=>[h,Object.keys(d)]));let u={};return r.className&&(u=Object.fromEntries(r.slots.map(h=>[h,`${r.className}__${h}`]))),Object.assign(i,{variantMap:c,variantKeys:a,splitVariantProps:l,classNameMap:u})}}const BC=()=>e=>Array.from(new Set(e)),WC=/([\0-\x1f\x7f]|^-?\d)|^-$|^-|[^\x80-\uFFFF\w-]/g,HC=function(e,t){return t?e==="\0"?"�":e==="-"&&e.length===1?"\\-":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16):"\\"+e},Sh=e=>(e+"").replace(WC,HC),Eh=(e,t)=>{let n="",r=0,o="char",i="",s="";const a=[];for(;r{let t=0;const n=["("];for(;t{n instanceof Map?t[r]=Object.fromEntries(n):t[r]=n}),t}const Ph=/({([^}]*)})/g,GC=/[{}]/g,qC=/\w+\.\w+/,Rh=e=>{if(!lt(e))return[];const t=e.match(Ph);return t?t.map(n=>n.replace(GC,"")).map(n=>n.trim()):[]},KC=e=>Ph.test(e);function Ih(e){if(!e.extensions?.references)return e.extensions?.cssVar?.ref??e.value;const t=e.extensions.references??{};return e.value=Object.keys(t).reduce((n,r)=>{const o=t[r];if(o.extensions.conditions)return n;const i=Ih(o);return n.replace(`{${r}}`,i)},e.value),delete e.extensions.references,e.value}function Th(e){return _e(e)&&e.reference?e.reference:String(e)}const ui=(e,...t)=>t.map(Th).join(` ${e} `).replace(/calc/g,""),Nh=(...e)=>`calc(${ui("+",...e)})`,Ah=(...e)=>`calc(${ui("-",...e)})`,Oa=(...e)=>`calc(${ui("*",...e)})`,_h=(...e)=>`calc(${ui("/",...e)})`,Vh=e=>{const t=Th(e);return t!=null&&!Number.isNaN(parseFloat(t))?String(t).startsWith("-")?String(t).slice(1):`-${t}`:Oa(t,-1)},or=Object.assign(e=>({add:(...t)=>or(Nh(e,...t)),subtract:(...t)=>or(Ah(e,...t)),multiply:(...t)=>or(Oa(e,...t)),divide:(...t)=>or(_h(e,...t)),negate:()=>or(Vh(e)),toString:()=>e.toString()}),{add:Nh,subtract:Ah,multiply:Oa,divide:_h,negate:Vh}),YC={enforce:"pre",transform(e){const{prefix:t,allTokens:n,formatCssVar:r,formatTokenName:o,registerToken:i}=e;n.filter(({extensions:a})=>a.category==="spacing").forEach(a=>{const l=a.path.slice(),c=r(l,t);if(lt(a.value)&&a.value==="0rem")return;const u=structuredClone(a);Object.assign(u.extensions,{negative:!0,prop:`-${a.extensions.prop}`,originalPath:l}),u.value=or.negate(c.ref);const h=u.path[u.path.length-1];h!=null&&(u.path[u.path.length-1]=`-${h}`),u.path&&(u.name=o(u.path)),i(u)})}},XC=new Set(["spacing","sizes","borderWidths","fontSizes","radii"]),QC=[YC,{enforce:"post",transform(e){const{allTokens:t,registerToken:n,formatTokenName:r}=e,o=t.filter(({extensions:a})=>a.category==="colors"),i=new Map,s=new Map;o.forEach(a=>{const{colorPalette:l}=a.extensions;l&&(l.keys.forEach(c=>{i.set(r(c),c)}),l.roots.forEach(c=>{const u=r(c),h=s.get(u)||[];if(h.push(a),s.set(u,h),a.extensions.default&&c.length===1){const d=l.keys[0]?.filter(Boolean);if(!d.length)return;const m=c.concat(d);i.set(r(m),[])}}))}),i.forEach(a=>{const l=["colors","colorPalette",...a].filter(Boolean),c=r(l),u=r(l.slice(1));n({name:c,value:c,originalValue:c,path:l,extensions:{condition:"base",originalPath:l,category:"colors",prop:u,virtual:!0}},"pre")})}},{enforce:"post",transform(e){e.allTokens.filter(n=>XC.has(n.extensions.category)&&!n.extensions.negative).forEach(n=>{Object.assign(n.extensions,{pixelValue:lh(n.value)})})}},{enforce:"post",transform(e){e.allTokens=e.allTokens.filter(t=>t.value!=="")}}],JC=[{type:"extensions",enforce:"pre",name:"tokens/css-var",transform(e,t){const{prefix:n,formatCssVar:r}=t,{negative:o,originalPath:i}=e.extensions,s=o?i:e.path;return{cssVar:r(s.filter(Boolean),n)}}},{enforce:"post",type:"value",name:"tokens/conditionals",transform(e,t){const{prefix:n,formatCssVar:r}=t,o=Rh(e.value);return o.length&&o.forEach(i=>{const s=r(i.split("."),n);e.value=e.value.replace(`{${s.ref}}`,s)}),e.value}},{type:"extensions",enforce:"pre",name:"tokens/colors/colorPalette",match(e){return e.extensions.category==="colors"&&!e.extensions.virtual},transform(e,t){let n=e.path.slice();if(n.pop(),n.shift(),n.length===0){const a=[...e.path];a.shift(),n=a}if(n.length===0)return{};const r=n.reduce((a,l,c,u)=>{const h=u.slice(0,c+1);return a.push(h),a},[]),o=n[0],i=t.formatTokenName(n),s=e.path.slice(e.path.indexOf(o)+1).reduce((a,l,c,u)=>(a.push(u.slice(c)),a),[]);return s.length===0&&s.push([""]),{colorPalette:{value:i,roots:r,keys:s}}}}],Fh=e=>_e(e)&&Object.prototype.hasOwnProperty.call(e,"value");function ZC(e){return e?{breakpoints:Jd(e,t=>({value:t})),sizes:Object.fromEntries(Object.entries(e).map(([t,n])=>[`breakpoint-${t}`,{value:n}]))}:{breakpoints:{},sizes:{}}}function ew(e){const{prefix:t="",tokens:n={},semanticTokens:r={},breakpoints:o={}}=e,i=L=>L.join("."),s=(L,F)=>th(L.join("-"),{prefix:F}),a=[],l=new Map,c=new Map,u=new Map,h=new Map,d=new Map,m=new Map,f=new Map,p=new Map,v=[];function b(L,F){a.push(L),l.set(L.name,L),F&&p.forEach(K=>{K.enforce===F&&q(K,L)})}const y=ZC(o),x=yr({...n,breakpoints:y.breakpoints,sizes:{...n.sizes,...y.sizes}});function w(){gt(x,(L,F)=>{const K=F.includes("DEFAULT");F=Lh(F);const ie=F[0],me=i(F),Oe=lt(L)?{value:L}:L,Vt={value:Oe.value,originalValue:Oe.value,name:me,path:F,extensions:{condition:"base",originalPath:F,category:ie,prop:i(F.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh}),gt(r,(L,F)=>{const K=F.includes("DEFAULT");F=Dh(Lh(F));const ie=F[0],me=i(F),Oe=lt(L.value)?{value:{base:L.value}}:L,Vt={value:Oe.value.base||"",originalValue:Oe.value.base||"",name:me,path:F,extensions:{originalPath:F,category:ie,conditions:Oe.value,condition:"base",prop:i(F.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh})}function k(L){return l.get(L)}function O(L){const{condition:F}=L.extensions;F&&(c.has(F)||c.set(F,new Set),c.get(F).add(L))}function R(L){const{category:F,prop:K}=L.extensions;F&&(f.has(F)||f.set(F,new Map),f.get(F).set(K,L))}function I(L){const{condition:F,negative:K,virtual:ie,cssVar:me}=L.extensions;K||ie||!F||!me||(u.has(F)||u.set(F,new Map),u.get(F).set(me.var,L.value))}function N(L){const{category:F,prop:K,cssVar:ie,negative:me}=L.extensions;if(!F)return;m.has(F)||m.set(F,new Map);const Oe=me?L.extensions.conditions?L.originalValue:L.value:ie.ref;m.get(F).set(K,Oe),d.set([F,K].join("."),Oe)}function A(L){const{colorPalette:F,virtual:K,default:ie}=L.extensions;!F||K||F.roots.forEach(me=>{const Oe=i(me);h.has(Oe)||h.set(Oe,new Map);const Vt=nw([...L.path],[...me]),Wi=i(Vt),gl=k(Wi);if(!gl||!gl.extensions.cssVar)return;const{var:mT}=gl.extensions.cssVar;if(h.get(Oe).set(mT,L.extensions.cssVar.ref),ie&&me.length===1){const vT=i(["colors","colorPalette"]),bp=k(vT);if(!bp)return;const bT=i(L.path),yp=k(bT);if(!yp)return;const xp=F.keys[0]?.filter(Boolean);if(!xp.length)return;const pl=i(me.concat(xp));h.has(pl)||h.set(pl,new Map),h.get(pl).set(bp.extensions.cssVar.var,yp.extensions.cssVar.ref)}})}let T={};function S(){a.forEach(L=>{O(L),R(L),I(L),N(L),A(L)}),T=Oh(m)}const _=(L,F)=>{if(!L||typeof L!="string")return{invalid:!0,value:L};const[K,ie]=L.split("/");if(!K||!ie)return{invalid:!0,value:K};const me=F(K),Oe=k(`opacity.${ie}`)?.value;if(!Oe&&isNaN(Number(ie)))return{invalid:!0,value:K};const Vt=Oe?Number(Oe)*100+"%":`${ie}%`,Wi=me??K;return{invalid:!1,color:Wi,value:`color-mix(in srgb, ${Wi} ${Vt}, transparent)`}},D=rt((L,F)=>d.get(L)??F),z=rt(L=>T[L]||null),W=rt(L=>Eh(L,F=>{if(!F)return;if(F.includes("/")){const ie=_(F,me=>D(me));if(ie.invalid)throw new Error("Invalid color mix at "+F+": "+ie.value);return ie.value}const K=D(F);return K||(qC.test(F)?Sh(F):F)})),J={prefix:t,allTokens:a,tokenMap:l,registerToken:b,getByName:k,formatTokenName:i,formatCssVar:s,flatMap:d,cssVarMap:u,categoryMap:f,colorPaletteMap:h,getVar:D,getCategoryValues:z,expandReferenceInValue:W};function ee(...L){L.forEach(F=>{p.set(F.name,F)})}function $(...L){v.push(...L)}function q(L,F){if(F.extensions.references||cs(L.match)&&!L.match(F))return;const ie=(me=>L.transform(me,J))(F);switch(!0){case L.type==="extensions":Object.assign(F.extensions,ie);break;case L.type==="value":F.value=ie;break;default:F[L.type]=ie;break}}function Z(L){v.forEach(F=>{F.enforce===L&&F.transform(J)})}function pe(L){p.forEach(F=>{F.enforce===L&&a.forEach(K=>{q(F,K)})})}function Je(){a.forEach(L=>{const F=tw(L);!F||F.length===0||F.forEach(K=>{b(K)})})}function yt(L){return Rh(L).map(K=>k(K)).filter(Boolean)}function _t(){a.forEach(L=>{if(!KC(L.value))return;const F=yt(L.value);L.extensions.references=F.reduce((K,ie)=>(K[ie.name]=ie,K),{})})}function fl(){a.forEach(L=>{Ih(L)})}function pT(){Z("pre"),pe("pre"),Je(),_t(),fl(),Z("post"),pe("post"),S()}return w(),ee(...JC),$(...QC),pT(),J}function Lh(e){return e[0]==="DEFAULT"?e:e.filter(t=>t!=="DEFAULT")}function Dh(e){return e.filter(t=>t!=="base")}function tw(e){if(!e.extensions.conditions)return;const{conditions:t}=e.extensions,n=[];return gt(t,(r,o)=>{const i=Dh(o);if(!i.length)return;const s=structuredClone(e);s.value=r,s.extensions.condition=i.join(":"),n.push(s)}),n}function nw(e,t){const n=e.findIndex((r,o)=>t.every((i,s)=>e[o+s]===i));return n===-1||(e.splice(n,t.length),e.splice(n,0,"colorPalette")),e}BC()(["aspectRatios","zIndex","opacity","colors","fonts","fontSizes","fontWeights","lineHeights","letterSpacings","sizes","shadows","spacing","radii","cursor","borders","borderWidths","borderStyles","durations","easings","animations","blurs","gradients","breakpoints","assets"]);function _T(e){return e}function rw(e){return Object.fromEntries(Object.entries(e).map(([t,n])=>[t,n]))}function ow(e){const t=rw(e.config),n=e.tokens,r=new Map,o=new Map;function i(I,N){t[I]=N,s(I,N)}const s=(I,N)=>{const A=p(N);A&&(o.set(I,A),h(I,N))},a=()=>{for(const[I,N]of Object.entries(t))N&&s(I,N)},l=()=>{for(const[I,N]of Object.entries(t)){const{shorthand:A}=N??{};if(!A)continue;(Array.isArray(A)?A:[A]).forEach(S=>r.set(S,I))}},c=()=>{const I=Oh(n.colorPaletteMap);i("colorPalette",{values:Object.keys(I),transform:rt(N=>I[N])})},u=new Map,h=(I,N)=>{if(!N)return;const A=p(N,S=>`type:Tokens["${S}"]`);if(typeof A=="object"&&A.type){u.set(I,new Set([`type:${A.type}`]));return}if(A){const S=new Set(Object.keys(A));u.set(I,S)}const T=u.get(I)??new Set;N.property&&u.set(I,T.add(`CssProperties["${N.property}"]`))},d=()=>{for(const[I,N]of Object.entries(t))N&&h(I,N)},m=(I,N)=>{const A=u.get(I)??new Set;u.set(I,new Set([...A,...N]))},f=()=>{const I=new Map;for(const[N,A]of u.entries()){if(A.size===0){I.set(N,["string"]);continue}const T=Array.from(A).map(S=>S.startsWith("CssProperties")?S:S.startsWith("type:")?S.replace("type:",""):JSON.stringify(S));I.set(N,T)}return I},p=(I,N)=>{const{values:A}=I,T=S=>{const _=N?.(S);return _?{[_]:_}:void 0};return lt(A)?T?.(A)??n.getCategoryValues(A)??{}:Array.isArray(A)?A.reduce((S,_)=>(S[_]=_,S),{}):cs(A)?A(N?T:n.getCategoryValues):A},v=rt((I,N)=>({[I]:I.startsWith("--")?n.getVar(N,N):N})),b=Object.assign(n.getVar,{raw:I=>n.getByName(I)}),y=rt((I,N)=>{const A=k(I);lt(N)&&!N.includes("_EMO_")&&(N=n.expandReferenceInValue(N));const T=t[A];if(!T)return v(A,N);const S=o.get(A)?.[N];if(!T.transform)return v(I,S??N);const _=D=>Mk(D,b);return T.transform(S??N,{raw:N,token:b,utils:{colorMix:_}})});function x(){l(),c(),a(),d()}x();const w=r.size>0,k=rt(I=>r.get(I)??I);return{keys:()=>[...Array.from(r.keys()),...Object.keys(t)],hasShorthand:w,transform:y,shorthands:r,resolveShorthand:k,register:i,getTypes:f,addPropertyType:m}}const De={};function zh(...e){const t=Zd(...e),{theme:n={},utilities:r={},globalCss:o={},cssVarsRoot:i=":where(:root, :host)",cssVarsPrefix:s="chakra",preflight:a}=t,l=VC(t),c=ew({breakpoints:n.breakpoints,tokens:n.tokens,semanticTokens:n.semanticTokens,prefix:s}),u=gC(n.breakpoints??De),h=bC({conditions:t.conditions??De,breakpoints:u}),d=ow({config:r,tokens:c});function m(){const{textStyles:$,layerStyles:q,animationStyles:Z}=n,pe=yr({textStyle:$,layerStyle:q,animationStyle:Z});for(const[Je,yt]of Object.entries(pe)){const _t=oh(yt??De,Mh);d.register(Je,{values:Object.keys(_t),transform(fl){return x(_t[fl])}})}}m(),d.addPropertyType("animationName",Object.keys(n.keyframes??De));const f=new Set(["css",...d.keys(),...h.keys()]),p=rt($=>f.has($)||cC($)),v=$=>Array.isArray($)?$.reduce((q,Z,pe)=>{const Je=h.breakpoints[pe];return Z!=null&&(q[Je]=Z),q},{}):$,b=FC({utility:d,normalize:v}),y=DC({conditions:h,isValidProperty:p}),x=kh({transform:d.transform,conditions:h,normalize:b}),w=NC({css:x,conditions:h,normalize:b,layers:l}),k=$C({cva:w});function O(){const $={};for(const[q,Z]of c.cssVarMap.entries()){const pe=Object.fromEntries(Z);if(Object.keys(pe).length===0)continue;const Je=q==="base"?i:h.resolve(q),yt=Je.startsWith("@"),_t=x(y({[Je]:yt?{[i]:pe}:pe}));er($,_t)}return l.wrap("tokens",$)}function R(){const $=Object.fromEntries(Object.entries(n.keyframes??De).map(([Z,pe])=>[`@keyframes ${Z}`,pe])),q=Object.assign({},$,x(y(o)));return l.wrap("base",q)}function I($){return Ln($,p)}function N(){const $=LC({preflight:a});return l.wrap("reset",$)}const A=iw(c),T=($,q)=>A.get($)?.value||q;T.var=($,q)=>A.get($)?.variable||q;function S($,q){return n.recipes?.[$]??q}function _($,q){return n.slotRecipes?.[$]??q}function D($){return Object.hasOwnProperty.call(n.recipes??De,$)}function z($){return Object.hasOwnProperty.call(n.slotRecipes??De,$)}function W($){return D($)||z($)}const J=[N(),R(),O()],ee={layerStyles:Pa(n.layerStyles??De),textStyles:Pa(n.textStyles??De),animationStyles:Pa(n.animationStyles??De),tokens:jh(c,Object.keys(n.tokens??De),($,q)=>!$.extensions.conditions&&!q.includes("colorPalette")),semanticTokens:jh(c,Object.keys(n.semanticTokens??De),$=>!!$.extensions.conditions),keyframes:$h(n.keyframes??De),breakpoints:$h(n.breakpoints??De)};return{$$chakra:!0,_config:t,_global:J,breakpoints:u,tokens:c,conditions:h,utility:d,token:T,properties:f,layers:l,isValidProperty:p,splitCssProps:I,normalizeValue:v,getTokenCss:O,getGlobalCss:R,getPreflightCss:N,css:x,cva:w,sva:k,getRecipe:S,getSlotRecipe:_,hasRecipe:W,isRecipe:D,isSlotRecipe:z,query:ee}}function iw(e){const t=new Map;return e.allTokens.forEach(n=>{const{cssVar:r,virtual:o,conditions:i}=n.extensions,s=i||o?r.ref:n.value;t.set(n.name,{value:s,variable:r.ref})}),t}const Mh=e=>_e(e)&&"value"in e,Pa=e=>({list(){return Object.keys(oh(e,Mh))},search(t){return this.list().filter(n=>n.includes(t))}}),jh=(e,t,n)=>({categoryKeys:t,list(r){return Array.from(e.categoryMap.get(r)?.entries()??[]).reduce((o,[i,s])=>(n(s,i)&&o.push(i),o),[])},search(r,o){return this.list(r).filter(i=>i.includes(o))}}),$h=e=>({list(){return Object.keys(e)},search(t){return this.list().filter(n=>n.includes(t))}}),sw={sm:"480px",md:"768px",lg:"1024px",xl:"1280px","2xl":"1536px"},Ra="var(--chakra-empty,/*!*/ /*!*/)",aw=Wk({"*":{fontFeatureSettings:'"cv11"',"--ring-inset":Ra,"--ring-offset-width":"0px","--ring-offset-color":"#fff","--ring-color":"rgba(66, 153, 225, 0.6)","--ring-offset-shadow":"0 0 #0000","--ring-shadow":"0 0 #0000",...Object.fromEntries(["brightness","contrast","grayscale","hue-rotate","invert","saturate","sepia","drop-shadow"].map(e=>[`--${e}`,Ra])),...Object.fromEntries(["blur","brightness","contrast","grayscale","hue-rotate","invert","opacity","saturate","sepia"].map(e=>[`--backdrop-${e}`,Ra])),"--global-font-mono":"fonts.mono","--global-font-body":"fonts.body","--global-color-border":"colors.border"},html:{color:"fg",bg:"bg",lineHeight:"1.5",colorPalette:"gray"},"*::placeholder, *[data-placeholder]":{color:"fg.muted/80"},"*::selection":{bg:"colorPalette.emphasized/80"}}),lw=Gk({"fill.muted":{value:{background:"colorPalette.muted",color:"colorPalette.fg"}},"fill.subtle":{value:{background:"colorPalette.subtle",color:"colorPalette.fg"}},"fill.surface":{value:{background:"colorPalette.subtle",color:"colorPalette.fg",boxShadow:"0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.muted"}},"fill.solid":{value:{background:"colorPalette.solid",color:"colorPalette.contrast"}},"outline.subtle":{value:{color:"colorPalette.fg",boxShadow:"inset 0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.subtle"}},"outline.solid":{value:{borderWidth:"1px",borderColor:"colorPalette.solid",color:"colorPalette.fg"}},"indicator.bottom":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",bottom:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.top":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",top:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.start":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineStart:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.end":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineEnd:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},disabled:{value:{opacity:"0.5",cursor:"not-allowed"}},none:{value:{}}}),cw=Uk({"slide-fade-in":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-from-bottom, fade-in"},"&[data-placement^=bottom]":{animationName:"slide-from-top, fade-in"},"&[data-placement^=left]":{animationName:"slide-from-right, fade-in"},"&[data-placement^=right]":{animationName:"slide-from-left, fade-in"}}},"slide-fade-out":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-to-bottom, fade-out"},"&[data-placement^=bottom]":{animationName:"slide-to-top, fade-out"},"&[data-placement^=left]":{animationName:"slide-to-right, fade-out"},"&[data-placement^=right]":{animationName:"slide-to-left, fade-out"}}},"scale-fade-in":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-in, fade-in"}},"scale-fade-out":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-out, fade-out"}}}),Ia=fe({className:"chakra-badge",base:{display:"inline-flex",alignItems:"center",borderRadius:"l2",gap:"1",fontWeight:"medium",fontVariantNumeric:"tabular-nums",whiteSpace:"nowrap",userSelect:"none"},variants:{variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg"},outline:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},plain:{color:"colorPalette.fg"}},size:{xs:{textStyle:"2xs",px:"1",minH:"4"},sm:{textStyle:"xs",px:"1.5",minH:"5"},md:{textStyle:"sm",px:"2",minH:"6"},lg:{textStyle:"sm",px:"2.5",minH:"7"}}},defaultVariants:{variant:"subtle",size:"sm"}}),uw=fe({className:"chakra-button",base:{display:"inline-flex",appearance:"none",alignItems:"center",justifyContent:"center",userSelect:"none",position:"relative",borderRadius:"l2",whiteSpace:"nowrap",verticalAlign:"middle",borderWidth:"1px",borderColor:"transparent",cursor:"button",flexShrink:"0",outline:"0",lineHeight:"1.2",isolation:"isolate",fontWeight:"medium",transitionProperty:"common",transitionDuration:"moderate",focusVisibleRing:"outside",_disabled:{layerStyle:"disabled"},_icon:{flexShrink:"0"}},variants:{size:{"2xs":{h:"6",minW:"6",textStyle:"xs",px:"2",gap:"1",_icon:{width:"3.5",height:"3.5"}},xs:{h:"8",minW:"8",textStyle:"xs",px:"2.5",gap:"1",_icon:{width:"4",height:"4"}},sm:{h:"9",minW:"9",px:"3.5",textStyle:"sm",gap:"2",_icon:{width:"4",height:"4"}},md:{h:"10",minW:"10",textStyle:"sm",px:"4",gap:"2",_icon:{width:"5",height:"5"}},lg:{h:"11",minW:"11",textStyle:"md",px:"5",gap:"3",_icon:{width:"5",height:"5"}},xl:{h:"12",minW:"12",textStyle:"md",px:"5",gap:"2.5",_icon:{width:"5",height:"5"}},"2xl":{h:"16",minW:"16",textStyle:"lg",px:"7",gap:"3",_icon:{width:"6",height:"6"}}},variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"transparent",_hover:{bg:"colorPalette.solid/90"},_expanded:{bg:"colorPalette.solid/90"}},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"transparent",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},outline:{borderWidth:"1px",borderColor:"colorPalette.muted",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},ghost:{bg:"transparent",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},plain:{color:"colorPalette.fg"}}},defaultVariants:{size:"md",variant:"solid"}}),Se=fe({className:"chakra-checkmark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"l1",cursor:"checkbox",focusVisibleRing:"outside",_icon:{boxSize:"full"},_invalid:{colorPalette:"red",borderColor:"border.error"},_disabled:{opacity:"0.5",cursor:"disabled"}},variants:{size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5",p:"0.5"},lg:{boxSize:"6",p:"0.5"}},variant:{solid:{borderColor:"border.emphasized","&:is([data-state=checked], [data-state=indeterminate])":{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},outline:{borderColor:"border","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg",borderColor:"colorPalette.solid"}},subtle:{bg:"colorPalette.muted",borderColor:"colorPalette.muted","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},plain:{"&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},inverted:{borderColor:"border",color:"colorPalette.fg","&:is([data-state=checked], [data-state=indeterminate])":{borderColor:"colorPalette.solid"}}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),{variants:dw,defaultVariants:hw}=Ia,fw=fe({className:"chakra-code",base:{fontFamily:"mono",alignItems:"center",display:"inline-flex",borderRadius:"l2"},variants:dw,defaultVariants:hw}),Bh=fe({className:"color-swatch",base:{boxSize:"var(--swatch-size)",shadow:"inset 0 0 0 1px rgba(0, 0, 0, 0.1)","--checker-size":"8px","--checker-bg":"colors.bg","--checker-fg":"colors.bg.emphasized",background:"linear-gradient(var(--color), var(--color)), repeating-conic-gradient(var(--checker-fg) 0%, var(--checker-fg) 25%, var(--checker-bg) 0%, var(--checker-bg) 50%) 0% 50% / var(--checker-size) var(--checker-size) !important",display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},variants:{size:{"2xs":{"--swatch-size":"sizes.3.5"},xs:{"--swatch-size":"sizes.4"},sm:{"--swatch-size":"sizes.4.5"},md:{"--swatch-size":"sizes.5"},lg:{"--swatch-size":"sizes.6"},xl:{"--swatch-size":"sizes.7"},"2xl":{"--swatch-size":"sizes.8"},inherit:{"--swatch-size":"inherit"},full:{"--swatch-size":"100%"}},shape:{square:{borderRadius:"none"},circle:{borderRadius:"full"},rounded:{borderRadius:"l1"}}},defaultVariants:{size:"md",shape:"rounded"}}),gw=fe({className:"chakra-container",base:{position:"relative",maxWidth:"8xl",w:"100%",mx:"auto",px:{base:"4",md:"6",lg:"8"}},variants:{centerContent:{true:{display:"flex",flexDirection:"column",alignItems:"center"}},fluid:{true:{maxWidth:"full"}}}}),pw=fe({className:"chakra-heading",base:{fontFamily:"heading",fontWeight:"semibold"},variants:{size:{xs:{textStyle:"xs"},sm:{textStyle:"sm"},md:{textStyle:"md"},lg:{textStyle:"lg"},xl:{textStyle:"xl"},"2xl":{textStyle:"2xl"},"3xl":{textStyle:"3xl"},"4xl":{textStyle:"4xl"},"5xl":{textStyle:"5xl"},"6xl":{textStyle:"6xl"},"7xl":{textStyle:"7xl"}}},defaultVariants:{size:"xl"}}),mw=fe({className:"chakra-icon",base:{display:"inline-block",lineHeight:"1em",flexShrink:"0",color:"currentcolor",verticalAlign:"middle"},variants:{size:{inherit:{},xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"},xl:{boxSize:"7"},"2xl":{boxSize:"8"}}},defaultVariants:{size:"inherit"}}),he=fe({className:"chakra-input",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},height:"var(--input-height)",minW:"var(--input-height)","--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{"2xs":{textStyle:"xs",px:"2","--input-height":"sizes.7"},xs:{textStyle:"xs",px:"2","--input-height":"sizes.8"},sm:{textStyle:"sm",px:"2.5","--input-height":"sizes.9"},md:{textStyle:"sm",px:"3","--input-height":"sizes.10"},lg:{textStyle:"md",px:"4","--input-height":"sizes.11"},xl:{textStyle:"md",px:"4.5","--input-height":"sizes.12"},"2xl":{textStyle:"lg",px:"5","--input-height":"sizes.16"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)",_invalid:{borderColor:"var(--error-color)",boxShadow:"0px 1px 0px 0px var(--error-color)"}}}}},defaultVariants:{size:"md",variant:"outline"}}),vw=fe({className:"chakra-input-addon",base:{flex:"0 0 auto",width:"auto",display:"flex",alignItems:"center",whiteSpace:"nowrap",alignSelf:"stretch",borderRadius:"l2"},variants:{size:he.variants.size,variant:{outline:{borderWidth:"1px",borderColor:"border",bg:"bg.muted"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.emphasized"},flushed:{borderBottom:"1px solid",borderColor:"inherit",borderRadius:"0",px:"0",bg:"transparent"}}},defaultVariants:{size:"md",variant:"outline"}}),bw=fe({className:"chakra-kbd",base:{display:"inline-flex",alignItems:"center",fontWeight:"medium",fontFamily:"mono",flexShrink:"0",whiteSpace:"nowrap",wordSpacing:"-0.5em",userSelect:"none",px:"1",borderRadius:"l2"},variants:{variant:{raised:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderWidth:"1px",borderBottomWidth:"2px",borderColor:"colorPalette.muted"},outline:{borderWidth:"1px",color:"colorPalette.fg"},subtle:{bg:"colorPalette.muted",color:"colorPalette.fg"},plain:{color:"colorPalette.fg"}},size:{sm:{textStyle:"xs",height:"4.5"},md:{textStyle:"sm",height:"5"},lg:{textStyle:"md",height:"6"}}},defaultVariants:{size:"md",variant:"raised"}}),yw=fe({className:"chakra-link",base:{display:"inline-flex",alignItems:"center",outline:"none",gap:"1.5",cursor:"pointer",borderRadius:"l1",focusRing:"outside"},variants:{variant:{underline:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"},plain:{color:"colorPalette.fg",_hover:{textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"}}}},defaultVariants:{variant:"plain"}}),xw=fe({className:"chakra-mark",base:{bg:"transparent",color:"inherit",whiteSpace:"nowrap"},variants:{variant:{subtle:{bg:"colorPalette.subtle",color:"inherit"},solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},text:{fontWeight:"medium"},plain:{}}}}),Ee=fe({className:"chakra-radiomark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:0,verticalAlign:"top",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"full",cursor:"radio",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing",outlineOffset:"2px"},_invalid:{colorPalette:"red",borderColor:"red.500"},_disabled:{opacity:"0.5",cursor:"disabled"},"& .dot":{height:"100%",width:"100%",borderRadius:"full",bg:"currentColor",scale:"0.4"}},variants:{variant:{solid:{borderWidth:"1px",borderColor:"border.emphasized",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},subtle:{borderWidth:"1px",bg:"colorPalette.muted",borderColor:"colorPalette.muted",color:"transparent",_checked:{color:"colorPalette.fg"}},outline:{borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.fg",borderColor:"colorPalette.solid"},"& .dot":{scale:"0.6"}},inverted:{bg:"bg",borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.solid",borderColor:"currentcolor"}}},size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),kw=fe({className:"chakra-separator",base:{display:"block",borderColor:"border"},variants:{variant:{solid:{borderStyle:"solid"},dashed:{borderStyle:"dashed"},dotted:{borderStyle:"dotted"}},orientation:{vertical:{borderInlineStartWidth:"var(--separator-thickness)"},horizontal:{borderTopWidth:"var(--separator-thickness)"}},size:{xs:{"--separator-thickness":"0.5px"},sm:{"--separator-thickness":"1px"},md:{"--separator-thickness":"2px"},lg:{"--separator-thickness":"3px"}}},defaultVariants:{size:"sm",variant:"solid",orientation:"horizontal"}}),Cw=fe({className:"chakra-skeleton",base:{},variants:{loading:{true:{borderRadius:"l2",boxShadow:"none",backgroundClip:"padding-box",cursor:"default",color:"transparent",pointerEvents:"none",userSelect:"none",flexShrink:"0","&::before, &::after, *":{visibility:"hidden"}},false:{background:"unset",animation:"fade-in var(--fade-duration, 0.1s) ease-out !important"}},variant:{pulse:{background:"bg.emphasized",animation:"pulse",animationDuration:"var(--duration, 1.2s)"},shine:{"--animate-from":"200%","--animate-to":"-200%","--start-color":"colors.bg.muted","--end-color":"colors.bg.emphasized",backgroundImage:"linear-gradient(270deg,var(--start-color),var(--end-color),var(--end-color),var(--start-color))",backgroundSize:"400% 100%",animation:"bg-position var(--duration, 5s) ease-in-out infinite"},none:{animation:"none"}}},defaultVariants:{variant:"pulse",loading:!0}}),ww=fe({className:"chakra-skip-nav",base:{display:"inline-flex",bg:"bg.panel",padding:"2.5",borderRadius:"l2",fontWeight:"semibold",focusVisibleRing:"outside",textStyle:"sm",userSelect:"none",border:"0",height:"1px",width:"1px",margin:"-1px",outline:"0",overflow:"hidden",position:"absolute",clip:"rect(0 0 0 0)",_focusVisible:{clip:"auto",width:"auto",height:"auto",position:"fixed",top:"6",insetStart:"6"}}}),Sw=fe({className:"chakra-spinner",base:{display:"inline-block",borderColor:"currentColor",borderStyle:"solid",borderWidth:"2px",borderRadius:"full",width:"var(--spinner-size)",height:"var(--spinner-size)",animation:"spin",animationDuration:"slowest","--spinner-track-color":"transparent",borderBottomColor:"var(--spinner-track-color)",borderInlineStartColor:"var(--spinner-track-color)"},variants:{size:{inherit:{"--spinner-size":"1em"},xs:{"--spinner-size":"sizes.3"},sm:{"--spinner-size":"sizes.4"},md:{"--spinner-size":"sizes.5"},lg:{"--spinner-size":"sizes.8"},xl:{"--spinner-size":"sizes.10"}}},defaultVariants:{size:"md"}}),Ew=fe({className:"chakra-textarea",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{xs:{textStyle:"xs",px:"2",py:"1.5",scrollPaddingBottom:"1.5"},sm:{textStyle:"sm",px:"2.5",py:"2",scrollPaddingBottom:"2"},md:{textStyle:"sm",px:"3",py:"2",scrollPaddingBottom:"2"},lg:{textStyle:"md",px:"4",py:"3",scrollPaddingBottom:"3"},xl:{textStyle:"md",px:"4.5",py:"3.5",scrollPaddingBottom:"3.5"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}}}},defaultVariants:{size:"md",variant:"outline"}}),Ow={badge:Ia,button:uw,code:fw,container:gw,heading:pw,input:he,inputAddon:vw,kbd:bw,link:yw,mark:xw,separator:kw,skeleton:Cw,skipNavLink:ww,spinner:Sw,textarea:Ew,icon:mw,checkmark:Se,radiomark:Ee,colorSwatch:Bh},Pw=xa.colors({bg:{DEFAULT:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},emphasized:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},inverted:{value:{_light:"{colors.black}",_dark:"{colors.white}"}},panel:{value:{_light:"{colors.white}",_dark:"{colors.gray.950}"}},error:{value:{_light:"{colors.red.50}",_dark:"{colors.red.950}"}},warning:{value:{_light:"{colors.orange.50}",_dark:"{colors.orange.950}"}},success:{value:{_light:"{colors.green.50}",_dark:"{colors.green.950}"}},info:{value:{_light:"{colors.blue.50}",_dark:"{colors.blue.950}"}}},fg:{DEFAULT:{value:{_light:"{colors.black}",_dark:"{colors.gray.50}"}},muted:{value:{_light:"{colors.gray.600}",_dark:"{colors.gray.400}"}},subtle:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.500}"}},inverted:{value:{_light:"{colors.gray.50}",_dark:"{colors.black}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.300}"}},success:{value:{_light:"{colors.green.600}",_dark:"{colors.green.300}"}},info:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.300}"}}},border:{DEFAULT:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},inverted:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.400}"}},success:{value:{_light:"{colors.green.500}",_dark:"{colors.green.400}"}},info:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.400}"}}},gray:{contrast:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},fg:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},subtle:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},muted:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},solid:{value:{_light:"{colors.gray.900}",_dark:"{colors.white}"}},focusRing:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.400}"}}},red:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.red.700}",_dark:"{colors.red.300}"}},subtle:{value:{_light:"{colors.red.100}",_dark:"{colors.red.900}"}},muted:{value:{_light:"{colors.red.200}",_dark:"{colors.red.800}"}},emphasized:{value:{_light:"{colors.red.300}",_dark:"{colors.red.700}"}},solid:{value:{_light:"{colors.red.600}",_dark:"{colors.red.600}"}},focusRing:{value:{_light:"{colors.red.500}",_dark:"{colors.red.500}"}}},orange:{contrast:{value:{_light:"white",_dark:"black"}},fg:{value:{_light:"{colors.orange.700}",_dark:"{colors.orange.300}"}},subtle:{value:{_light:"{colors.orange.100}",_dark:"{colors.orange.900}"}},muted:{value:{_light:"{colors.orange.200}",_dark:"{colors.orange.800}"}},emphasized:{value:{_light:"{colors.orange.300}",_dark:"{colors.orange.700}"}},solid:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.500}"}},focusRing:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.500}"}}},green:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.green.700}",_dark:"{colors.green.300}"}},subtle:{value:{_light:"{colors.green.100}",_dark:"{colors.green.900}"}},muted:{value:{_light:"{colors.green.200}",_dark:"{colors.green.800}"}},emphasized:{value:{_light:"{colors.green.300}",_dark:"{colors.green.700}"}},solid:{value:{_light:"{colors.green.600}",_dark:"{colors.green.600}"}},focusRing:{value:{_light:"{colors.green.500}",_dark:"{colors.green.500}"}}},blue:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.blue.700}",_dark:"{colors.blue.300}"}},subtle:{value:{_light:"{colors.blue.100}",_dark:"{colors.blue.900}"}},muted:{value:{_light:"{colors.blue.200}",_dark:"{colors.blue.800}"}},emphasized:{value:{_light:"{colors.blue.300}",_dark:"{colors.blue.700}"}},solid:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.600}"}},focusRing:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.500}"}}},yellow:{contrast:{value:{_light:"black",_dark:"black"}},fg:{value:{_light:"{colors.yellow.800}",_dark:"{colors.yellow.300}"}},subtle:{value:{_light:"{colors.yellow.100}",_dark:"{colors.yellow.900}"}},muted:{value:{_light:"{colors.yellow.200}",_dark:"{colors.yellow.800}"}},emphasized:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.700}"}},solid:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.300}"}},focusRing:{value:{_light:"{colors.yellow.500}",_dark:"{colors.yellow.500}"}}},teal:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.teal.700}",_dark:"{colors.teal.300}"}},subtle:{value:{_light:"{colors.teal.100}",_dark:"{colors.teal.900}"}},muted:{value:{_light:"{colors.teal.200}",_dark:"{colors.teal.800}"}},emphasized:{value:{_light:"{colors.teal.300}",_dark:"{colors.teal.700}"}},solid:{value:{_light:"{colors.teal.600}",_dark:"{colors.teal.600}"}},focusRing:{value:{_light:"{colors.teal.500}",_dark:"{colors.teal.500}"}}},purple:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.purple.700}",_dark:"{colors.purple.300}"}},subtle:{value:{_light:"{colors.purple.100}",_dark:"{colors.purple.900}"}},muted:{value:{_light:"{colors.purple.200}",_dark:"{colors.purple.800}"}},emphasized:{value:{_light:"{colors.purple.300}",_dark:"{colors.purple.700}"}},solid:{value:{_light:"{colors.purple.600}",_dark:"{colors.purple.600}"}},focusRing:{value:{_light:"{colors.purple.500}",_dark:"{colors.purple.500}"}}},pink:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.pink.700}",_dark:"{colors.pink.300}"}},subtle:{value:{_light:"{colors.pink.100}",_dark:"{colors.pink.900}"}},muted:{value:{_light:"{colors.pink.200}",_dark:"{colors.pink.800}"}},emphasized:{value:{_light:"{colors.pink.300}",_dark:"{colors.pink.700}"}},solid:{value:{_light:"{colors.pink.600}",_dark:"{colors.pink.600}"}},focusRing:{value:{_light:"{colors.pink.500}",_dark:"{colors.pink.500}"}}},cyan:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.cyan.700}",_dark:"{colors.cyan.300}"}},subtle:{value:{_light:"{colors.cyan.100}",_dark:"{colors.cyan.900}"}},muted:{value:{_light:"{colors.cyan.200}",_dark:"{colors.cyan.800}"}},emphasized:{value:{_light:"{colors.cyan.300}",_dark:"{colors.cyan.700}"}},solid:{value:{_light:"{colors.cyan.600}",_dark:"{colors.cyan.600}"}},focusRing:{value:{_light:"{colors.cyan.500}",_dark:"{colors.cyan.500}"}}}}),Rw=xa.radii({l1:{value:"{radii.xs}"},l2:{value:"{radii.sm}"},l3:{value:"{radii.md}"}}),Iw=xa.shadows({xs:{value:{_light:"0px 1px 2px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/20}",_dark:"0px 1px 1px {black/64}, 0px 0px 1px inset {colors.gray.300/20}"}},sm:{value:{_light:"0px 2px 4px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 2px 4px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},md:{value:{_light:"0px 4px 8px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 4px 8px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},lg:{value:{_light:"0px 8px 16px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 8px 16px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},xl:{value:{_light:"0px 16px 24px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 16px 24px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},"2xl":{value:{_light:"0px 24px 40px {colors.gray.900/16}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 24px 40px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},inner:{value:{_light:"inset 0 2px 4px 0 {black/5}",_dark:"inset 0 2px 4px 0 black"}},inset:{value:{_light:"inset 0 0 0 1px {black/5}",_dark:"inset 0 0 0 1px {colors.gray.300/5}"}}}),Tw=xu.extendWith("itemBody"),Nw=j("action-bar").parts("positioner","content","separator","selectionTrigger","closeTrigger"),Aw=j("alert").parts("title","description","root","indicator","content"),_w=j("breadcrumb").parts("link","currentLink","item","list","root","ellipsis","separator"),Vw=j("blockquote").parts("root","icon","content","caption"),Fw=j("card").parts("root","header","body","footer","title","description"),Lw=j("checkbox-card",["root","control","label","description","addon","indicator","content"]),Dw=j("data-list").parts("root","item","itemLabel","itemValue"),zw=ea.extendWith("header","body","footer","backdrop"),Mw=ea.extendWith("header","body","footer","backdrop"),jw=ld.extendWith("textarea"),$w=j("empty-state",["root","content","indicator","title","description"]),Bw=ud.extendWith("requiredIndicator"),Ww=hd.extendWith("content"),Hw=fd.extendWith("itemContent","dropzoneContent","fileText"),Uw=j("list").parts("root","item","indicator"),Gw=xd.extendWith("itemCommand"),qw=j("select").parts("root","field","indicator"),Kw=Dd.extendWith("header","body","footer"),Wh=fa.extendWith("itemAddon","itemIndicator"),Yw=Wh.extendWith("itemContent","itemDescription"),Xw=Md.extendWith("itemIndicator"),Qw=Bd.extendWith("indicatorGroup"),Jw=s1.extendWith("indicatorGroup","empty"),Zw=Gd.extendWith("markerIndicator"),eS=j("stat").parts("root","label","helpText","valueText","valueUnit","indicator"),tS=j("status").parts("root","indicator"),nS=j("steps",["root","list","item","trigger","indicator","separator","content","title","description","nextTrigger","prevTrigger","progress"]),rS=qd.extendWith("indicator"),oS=j("table").parts("root","header","body","row","columnHeader","cell","footer","caption"),iS=j("toast").parts("root","title","description","indicator","closeTrigger","actionTrigger"),sS=j("tabs").parts("root","trigger","list","content","contentGroup","indicator"),aS=j("tag").parts("root","label","closeTrigger","startElement","endElement"),lS=j("timeline").parts("root","item","content","separator","indicator","connector","title","description"),cS=wx.extendWith("channelText"),uS=j("code-block",["root","content","title","header","footer","control","overlay","code","codeText","copyTrigger","copyIndicator","collapseTrigger","collapseIndicator","collapseText"]);Tu.extendWith("valueText");const dS=V1,hS=B({className:"chakra-accordion",slots:Tw.keys(),base:{root:{width:"full","--accordion-radius":"radii.l2"},item:{overflowAnchor:"none"},itemTrigger:{display:"flex",alignItems:"center",textAlign:"start",width:"full",outline:"0",gap:"3",fontWeight:"medium",borderRadius:"var(--accordion-radius)",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{layerStyle:"disabled"}},itemBody:{pt:"var(--accordion-padding-y)",pb:"calc(var(--accordion-padding-y) * 2)"},itemContent:{overflow:"hidden",borderRadius:"var(--accordion-radius)",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}},itemIndicator:{transition:"rotate 0.2s",transformOrigin:"center",color:"fg.subtle",_open:{rotate:"180deg"},_icon:{width:"1.2em",height:"1.2em"}}},variants:{variant:{outline:{item:{borderBottomWidth:"1px"}},subtle:{itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{borderRadius:"var(--accordion-radius)",_open:{bg:"colorPalette.subtle"}}},enclosed:{root:{borderWidth:"1px",borderRadius:"var(--accordion-radius)",divideY:"1px",overflow:"hidden"},itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{_open:{bg:"bg.subtle"}}},plain:{}},size:{sm:{root:{"--accordion-padding-x":"spacing.3","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"sm",py:"var(--accordion-padding-y)"}},md:{root:{"--accordion-padding-x":"spacing.4","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"md",py:"var(--accordion-padding-y)"}},lg:{root:{"--accordion-padding-x":"spacing.4.5","--accordion-padding-y":"spacing.2.5"},itemTrigger:{textStyle:"lg",py:"var(--accordion-padding-y)"}}}},defaultVariants:{size:"md",variant:"outline"}}),fS=B({className:"chakra-action-bar",slots:Nw.keys(),base:{positioner:{position:"fixed",display:"flex",justifyContent:"center",pointerEvents:"none",insetInline:"0",top:"unset",bottom:"calc(env(safe-area-inset-bottom) + 20px)"},content:{bg:"bg.panel",shadow:"md",display:"flex",alignItems:"center",gap:"3",borderRadius:"l3",py:"2.5",px:"3",pointerEvents:"auto",translate:"calc(-1 * var(--scrollbar-width) / 2) 0px",_open:{animationName:"slide-from-bottom, fade-in",animationDuration:"moderate"},_closed:{animationName:"slide-to-bottom, fade-out",animationDuration:"faster"}},separator:{width:"1px",height:"5",bg:"border"},selectionTrigger:{display:"inline-flex",alignItems:"center",gap:"2",alignSelf:"stretch",textStyle:"sm",px:"4",py:"1",borderRadius:"l2",borderWidth:"1px",borderStyle:"dashed"}}}),gS=B({slots:Aw.keys(),className:"chakra-alert",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",borderRadius:"l3"},title:{fontWeight:"medium"},description:{display:"inline"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",width:"1em",height:"1em",_icon:{boxSize:"full"}},content:{display:"flex",flex:"1",gap:"1"}},variants:{status:{info:{root:{colorPalette:"blue"}},warning:{root:{colorPalette:"orange"}},success:{root:{colorPalette:"green"}},error:{root:{colorPalette:"red"}},neutral:{root:{colorPalette:"gray"}}},inline:{true:{content:{display:"inline-flex",flexDirection:"row",alignItems:"center"}},false:{content:{display:"flex",flexDirection:"column"}}},variant:{subtle:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg"}},surface:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"},indicator:{color:"colorPalette.contrast"}}},size:{sm:{root:{gap:"2",px:"3",py:"3",textStyle:"xs"},indicator:{textStyle:"lg"}},md:{root:{gap:"3",px:"4",py:"4",textStyle:"sm"},indicator:{textStyle:"xl"}},lg:{root:{gap:"3",px:"4",py:"4",textStyle:"md"},indicator:{textStyle:"2xl"}}}},defaultVariants:{status:"info",variant:"subtle",size:"md",inline:!1}}),pS=B({slots:wu.keys(),className:"chakra-avatar",base:{root:{display:"inline-flex",alignItems:"center",justifyContent:"center",fontWeight:"medium",position:"relative",verticalAlign:"top",flexShrink:"0",userSelect:"none",width:"var(--avatar-size)",height:"var(--avatar-size)",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)","&[data-group-item]":{borderWidth:"2px",borderColor:"bg"}},image:{width:"100%",height:"100%",objectFit:"cover",borderRadius:"var(--avatar-radius)"},fallback:{lineHeight:"1",textTransform:"uppercase",fontWeight:"medium",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)"}},variants:{size:{full:{root:{"--avatar-size":"100%","--avatar-font-size":"100%"}},"2xs":{root:{"--avatar-font-size":"fontSizes.2xs","--avatar-size":"sizes.6"}},xs:{root:{"--avatar-font-size":"fontSizes.xs","--avatar-size":"sizes.8"}},sm:{root:{"--avatar-font-size":"fontSizes.sm","--avatar-size":"sizes.9"}},md:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.10"}},lg:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.11"}},xl:{root:{"--avatar-font-size":"fontSizes.lg","--avatar-size":"sizes.12"}},"2xl":{root:{"--avatar-font-size":"fontSizes.xl","--avatar-size":"sizes.16"}}},variant:{solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},subtle:{root:{bg:"colorPalette.muted",color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",borderWidth:"1px",borderColor:"colorPalette.muted"}}},shape:{square:{},rounded:{root:{"--avatar-radius":"radii.l3"}},full:{root:{"--avatar-radius":"radii.full"}}},borderless:{true:{root:{"&[data-group-item]":{borderWidth:"0px"}}}}},defaultVariants:{size:"md",shape:"full",variant:"subtle"}}),mS=B({className:"chakra-blockquote",slots:Vw.keys(),base:{root:{position:"relative",display:"flex",flexDirection:"column",gap:"2"},caption:{textStyle:"sm",color:"fg.muted"},icon:{boxSize:"5"}},variants:{justify:{start:{root:{alignItems:"flex-start",textAlign:"start"}},center:{root:{alignItems:"center",textAlign:"center"}},end:{root:{alignItems:"flex-end",textAlign:"end"}}},variant:{subtle:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.muted"},icon:{color:"colorPalette.fg"}},solid:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.solid"},icon:{color:"colorPalette.solid"}},plain:{root:{paddingX:"5"},icon:{color:"colorPalette.solid"}}}},defaultVariants:{variant:"subtle",justify:"start"}}),vS=B({className:"chakra-breadcrumb",slots:_w.keys(),base:{list:{display:"flex",alignItems:"center",wordBreak:"break-word",color:"fg.muted",listStyle:"none"},link:{outline:"0",textDecoration:"none",borderRadius:"l1",focusRing:"outside",display:"inline-flex",alignItems:"center",gap:"2"},item:{display:"inline-flex",alignItems:"center"},separator:{color:"fg.muted",opacity:"0.8",_icon:{boxSize:"1em"},_rtl:{rotate:"180deg"}},ellipsis:{display:"inline-flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"1em"}}},variants:{variant:{underline:{link:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"0.2em",textDecorationColor:"colorPalette.muted"},currentLink:{color:"colorPalette.fg"}},plain:{link:{color:"fg.muted",_hover:{color:"fg"}},currentLink:{color:"fg"}}},size:{sm:{list:{gap:"1",textStyle:"xs"}},md:{list:{gap:"1.5",textStyle:"sm"}},lg:{list:{gap:"2",textStyle:"md"}}}},defaultVariants:{variant:"plain",size:"md"}}),bS=B({className:"chakra-card",slots:Fw.keys(),base:{root:{display:"flex",flexDirection:"column",position:"relative",minWidth:"0",wordWrap:"break-word",borderRadius:"l3",color:"fg",textAlign:"start"},title:{fontWeight:"semibold"},description:{color:"fg.muted",fontSize:"sm"},header:{paddingInline:"var(--card-padding)",paddingTop:"var(--card-padding)",display:"flex",flexDirection:"column",gap:"1.5"},body:{padding:"var(--card-padding)",flex:"1",display:"flex",flexDirection:"column"},footer:{display:"flex",alignItems:"center",gap:"2",paddingInline:"var(--card-padding)",paddingBottom:"var(--card-padding)"}},variants:{size:{sm:{root:{"--card-padding":"spacing.4"},title:{textStyle:"md"}},md:{root:{"--card-padding":"spacing.6"},title:{textStyle:"lg"}},lg:{root:{"--card-padding":"spacing.7"},title:{textStyle:"xl"}}},variant:{elevated:{root:{bg:"bg.panel",boxShadow:"md"}},outline:{root:{bg:"bg.panel",borderWidth:"1px",borderColor:"border"}},subtle:{root:{bg:"bg.muted"}}}},defaultVariants:{variant:"outline",size:"md"}}),yS=B({slots:Cx.keys(),className:"chakra-checkbox",base:{root:{display:"inline-flex",gap:"2",alignItems:"center",verticalAlign:"top",position:"relative"},control:Se.base,label:{fontWeight:"medium",userSelect:"none",_disabled:{opacity:"0.5"}}},variants:{size:{xs:{root:{gap:"1.5"},label:{textStyle:"xs"},control:Se.variants?.size?.xs},sm:{root:{gap:"2"},label:{textStyle:"sm"},control:Se.variants?.size?.sm},md:{root:{gap:"2.5"},label:{textStyle:"sm"},control:Se.variants?.size?.md},lg:{root:{gap:"3"},label:{textStyle:"md"},control:Se.variants?.size?.lg}},variant:{outline:{control:Se.variants?.variant?.outline},solid:{control:Se.variants?.variant?.solid},subtle:{control:Se.variants?.variant?.subtle}}},defaultVariants:{variant:"solid",size:"md"}}),xS=B({slots:Lw.keys(),className:"chakra-checkbox-card",base:{root:{display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",flex:"1",focusVisibleRing:"outside",_disabled:{opacity:"0.8"},_invalid:{outline:"2px solid",outlineColor:"border.error"}},control:{display:"inline-flex",flex:"1",position:"relative",borderRadius:"inherit",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"},label:{fontWeight:"medium",display:"flex",alignItems:"center",gap:"2",flex:"1",_disabled:{opacity:"0.5"}},description:{opacity:"0.64",textStyle:"sm",_disabled:{opacity:"0.5"}},addon:{_disabled:{opacity:"0.5"}},indicator:Se.base,content:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"}},variants:{size:{sm:{root:{textStyle:"sm"},control:{padding:"3",gap:"1.5"},addon:{px:"3",py:"1.5",borderTopWidth:"1px"},indicator:Se.variants?.size.sm},md:{root:{textStyle:"sm"},control:{padding:"4",gap:"2.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.md},lg:{root:{textStyle:"md"},control:{padding:"4",gap:"3.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.lg}},variant:{surface:{root:{borderWidth:"1px",borderColor:"border",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"},_disabled:{bg:"bg.muted"}},indicator:Se.variants?.variant.solid},subtle:{root:{bg:"bg.muted"},control:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},indicator:Se.variants?.variant.plain},outline:{root:{borderWidth:"1px",borderColor:"border",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.solid},solid:{root:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.inverted}},justify:{start:{root:{"--checkbox-card-justify":"flex-start"}},end:{root:{"--checkbox-card-justify":"flex-end"}},center:{root:{"--checkbox-card-justify":"center"}}},align:{start:{root:{"--checkbox-card-align":"flex-start"},content:{textAlign:"start"}},end:{root:{"--checkbox-card-align":"flex-end"},content:{textAlign:"end"}},center:{root:{"--checkbox-card-align":"center"},content:{textAlign:"center"}}},orientation:{vertical:{control:{flexDirection:"column"}},horizontal:{control:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),kS=B({slots:uS.keys(),className:"code-block",base:{root:{colorPalette:"gray",rounded:"var(--code-block-radius)",overflow:"hidden",bg:"bg",color:"fg",borderWidth:"1px","--code-block-max-height":"320px","--code-block-bg":"colors.bg","--code-block-fg":"colors.fg","--code-block-obscured-opacity":"0.5","--code-block-obscured-blur":"1px","--code-block-line-number-width":"sizes.3","--code-block-line-number-margin":"spacing.4","--code-block-highlight-bg":"{colors.teal.focusRing/20}","--code-block-highlight-border":"colors.teal.focusRing","--code-block-highlight-added-bg":"{colors.green.focusRing/20}","--code-block-highlight-added-border":"colors.green.focusRing","--code-block-highlight-removed-bg":"{colors.red.focusRing/20}","--code-block-highlight-removed-border":"colors.red.focusRing"},header:{display:"flex",alignItems:"center",gap:"2",position:"relative",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)",mb:"calc(var(--code-block-padding) / 2 * -1)"},title:{display:"inline-flex",alignItems:"center",gap:"1.5",flex:"1",color:"fg.muted"},control:{gap:"1.5",display:"inline-flex",alignItems:"center"},footer:{display:"flex",alignItems:"center",justifyContent:"center",gap:"2",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)"},content:{position:"relative",colorScheme:"dark",overflowX:"auto",overflowY:"hidden",borderBottomRadius:"var(--code-block-radius)",maxHeight:"var(--code-block-max-height)","& ::selection":{bg:"blue.500/40"},_expanded:{maxHeight:"unset"}},overlay:{"--bg":"{colors.black/50}",display:"flex",alignItems:"flex-end",justifyContent:"center",padding:"4",bgImage:"linear-gradient(0deg,var(--bg) 25%,transparent 100%)",color:"white",minH:"5rem",pos:"absolute",bottom:"0",insetInline:"0",zIndex:"1",fontWeight:"medium",_expanded:{display:"none"}},code:{fontFamily:"mono",lineHeight:"tall",whiteSpace:"pre",counterReset:"line 0"},codeText:{px:"var(--code-block-padding)",py:"var(--code-block-padding)",position:"relative",display:"block",width:"100%","&[data-has-focused]":{"& [data-line]:not([data-focused])":{transitionProperty:"opacity, filter",transitionDuration:"moderate",transitionTimingFunction:"ease-in-out",opacity:"var(--code-block-obscured-opacity)",filter:"blur(var(--code-block-obscured-blur))"},"&:hover":{"--code-block-obscured-opacity":"1","--code-block-obscured-blur":"0px"}},"&[data-has-line-numbers][data-plaintext]":{paddingInlineStart:"calc(var(--code-block-line-number-width) + var(--code-block-line-number-margin) + var(--code-block-padding))"},"& [data-line]":{position:"relative","--highlight-bg":"var(--code-block-highlight-bg)","--highlight-border":"var(--code-block-highlight-border)","&[data-highlight], &[data-diff]":{display:"inline-block",width:"full","&:after":{content:"''",display:"block",position:"absolute",insetStart:"calc(var(--code-block-padding) * -1)",insetEnd:"0px",width:"calc(100% + var(--code-block-padding) * 2)",height:"100%",bg:"var(--highlight-bg)",borderStartWidth:"2px",borderStartColor:"var(--highlight-border)"}},"&[data-diff='added']":{"--highlight-bg":"var(--code-block-highlight-added-bg)","--highlight-border":"var(--code-block-highlight-added-border)"},"&[data-diff='removed']":{"--highlight-bg":"var(--code-block-highlight-removed-bg)","--highlight-border":"var(--code-block-highlight-removed-border)"}},"&[data-word-wrap]":{"&[data-plaintext], & [data-line]":{whiteSpace:"pre-wrap",wordBreak:"break-all"}},"&[data-has-line-numbers]":{"--content":"counter(line)","& [data-line]:before":{content:"var(--content)",counterIncrement:"line",width:"var(--code-block-line-number-width)",marginRight:"var(--code-block-line-number-margin)",display:"inline-block",textAlign:"end",userSelect:"none",whiteSpace:"nowrap",opacity:.4},"& [data-diff='added']:before":{content:"'+'"},"& [data-diff='removed']:before":{content:"'-'"}}}},variants:{size:{sm:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.md","--code-block-header-height":"sizes.8"},title:{textStyle:"xs"},code:{fontSize:"xs"}},md:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.lg","--code-block-header-height":"sizes.10"},title:{textStyle:"xs"},code:{fontSize:"sm"}},lg:{root:{"--code-block-padding":"spacing.5","--code-block-radius":"radii.xl","--code-block-header-height":"sizes.12"},title:{textStyle:"sm"},code:{fontSize:"sm"}}}},defaultVariants:{size:"md"}}),CS=B({slots:Ac.keys(),className:"chakra-collapsible",base:{content:{overflow:"hidden",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}),wS=B({className:"colorPicker",slots:cS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5"},label:{color:"fg",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},valueText:{textAlign:"start"},control:{display:"flex",alignItems:"center",flexDirection:"row",gap:"2",position:"relative"},swatchTrigger:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{display:"flex",alignItems:"center",justifyContent:"center",flexDirection:"row",flexShrink:"0",gap:"2",textStyle:"sm",minH:"var(--input-height)",minW:"var(--input-height)",px:"1",rounded:"l2",_disabled:{opacity:"0.5"},"--focus-color":"colors.colorPalette.focusRing","&:focus-visible":{borderColor:"var(--focus-color)",outline:"1px solid var(--focus-color)"},"&[data-fit-content]":{"--input-height":"unset",px:"0",border:"0"}},content:{display:"flex",flexDirection:"column",bg:"bg.panel",borderRadius:"l3",boxShadow:"lg",width:"64",p:"4",gap:"3",zIndex:"dropdown",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},area:{height:"180px",borderRadius:"l2",overflow:"hidden"},areaThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",focusVisibleRing:"mixed",focusRingColor:"white"},areaBackground:{height:"full"},channelSlider:{borderRadius:"l2",flex:"1"},channelSliderTrack:{height:"var(--slider-height)",borderRadius:"inherit",boxShadow:"inset 0 0 0 1px rgba(0,0,0,0.1)"},channelText:{textStyle:"xs",color:"fg.muted",fontWeight:"medium",textTransform:"capitalize"},swatchGroup:{display:"flex",flexDirection:"row",flexWrap:"wrap",gap:"2"},swatch:{...Bh.base,borderRadius:"l1"},swatchIndicator:{color:"white",rounded:"full"},channelSliderThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",transform:"translate(-50%, -50%)",focusVisibleRing:"outside",focusRingOffset:"1px"},channelInput:{...he.base,"&::-webkit-inner-spin-button, &::-webkit-outer-spin-button":{WebkitAppearance:"none",margin:0}},formatSelect:{textStyle:"xs",textTransform:"uppercase",borderWidth:"1px",minH:"6",focusRing:"inside",rounded:"l2"},transparencyGrid:{borderRadius:"l2"},view:{display:"flex",flexDirection:"column",gap:"2"}},variants:{size:{"2xs":{channelInput:he.variants?.size?.["2xs"],swatch:{"--swatch-size":"sizes.4.5"},trigger:{"--input-height":"sizes.7"},area:{"--thumb-size":"sizes.3"},channelSlider:{"--slider-height":"sizes.3","--thumb-size":"sizes.3"}},xs:{channelInput:he.variants?.size?.xs,swatch:{"--swatch-size":"sizes.5"},trigger:{"--input-height":"sizes.8"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},sm:{channelInput:he.variants?.size?.sm,swatch:{"--swatch-size":"sizes.6"},trigger:{"--input-height":"sizes.9"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},md:{channelInput:he.variants?.size?.md,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.10"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},lg:{channelInput:he.variants?.size?.lg,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.11"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},xl:{channelInput:he.variants?.size?.xl,swatch:{"--swatch-size":"sizes.8"},trigger:{"--input-height":"sizes.12"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},"2xl":{channelInput:he.variants?.size?.["2xl"],swatch:{"--swatch-size":"sizes.10"},trigger:{"--input-height":"sizes.16"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}}},variant:{outline:{channelInput:he.variants?.variant?.outline,trigger:{borderWidth:"1px"}},subtle:{channelInput:he.variants?.variant?.subtle,trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}}},defaultVariants:{size:"md",variant:"outline"}}),SS=B({className:"chakra-combobox",slots:Jw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},input:{display:"flex",alignItems:"center",justifyContent:"space-between",background:"bg.panel",width:"full",minH:"var(--combobox-input-height)",px:"var(--combobox-input-padding-x)","--input-height":"var(--combobox-input-height)",borderRadius:"l2",outline:0,userSelect:"none",textAlign:"start",_placeholderShown:{color:"fg.muted"},_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},trigger:{display:"inline-flex",alignItems:"center",justifyContent:"center","--input-height":"var(--combobox-input-height)"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"},control:{pos:"relative"},indicatorGroup:{display:"flex",alignItems:"center",justifyContent:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--combobox-input-padding-x)",_icon:{boxSize:"var(--combobox-indicator-size)"},"[data-disabled] &":{opacity:.5}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"0s"},"&[data-empty]:not(:has([data-scope=combobox][data-part=empty]))":{opacity:0}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{boxSize:"var(--combobox-indicator-size)"}},empty:{py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{pb:"var(--combobox-item-padding-y)",_last:{pb:"0"}},itemGroupLabel:{fontWeight:"medium",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"}},variants:{variant:{outline:{input:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"}},subtle:{input:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"}},flushed:{input:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}},indicatorGroup:{px:"0"}}},size:{xs:{root:{"--combobox-input-height":"sizes.8","--combobox-input-padding-x":"spacing.2","--combobox-indicator-size":"sizes.3.5"},input:{textStyle:"xs"},content:{"--combobox-item-padding-x":"spacing.1.5","--combobox-item-padding-y":"spacing.1","--combobox-indicator-size":"sizes.3.5",p:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"}},sm:{root:{"--combobox-input-height":"sizes.9","--combobox-input-padding-x":"spacing.2.5","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"}},md:{root:{"--combobox-input-height":"sizes.10","--combobox-input-padding-x":"spacing.3","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{textStyle:"sm",gap:"2"}},lg:{root:{"--combobox-input-height":"sizes.12","--combobox-input-padding-x":"spacing.4","--combobox-indicator-size":"sizes.5"},input:{textStyle:"md"},content:{"--combobox-item-padding-y":"spacing.2","--combobox-item-padding-x":"spacing.3","--combobox-indicator-size":"sizes.5",p:"1.5",textStyle:"md"},trigger:{textStyle:"md",py:"3",gap:"2"}}}},defaultVariants:{size:"md",variant:"outline"}}),ES=B({slots:Dw.keys(),className:"chakra-data-list",base:{itemLabel:{display:"flex",alignItems:"center",gap:"1"},itemValue:{display:"flex",minWidth:"0",flex:"1"}},variants:{orientation:{horizontal:{root:{display:"flex",flexDirection:"column"},item:{display:"inline-flex",alignItems:"center",gap:"4"},itemLabel:{minWidth:"120px"}},vertical:{root:{display:"flex",flexDirection:"column"},item:{display:"flex",flexDirection:"column",gap:"1"}}},size:{sm:{root:{gap:"3"},item:{textStyle:"xs"}},md:{root:{gap:"4"},item:{textStyle:"sm"}},lg:{root:{gap:"5"},item:{textStyle:"md"}}},variant:{subtle:{itemLabel:{color:"fg.muted"}},bold:{itemLabel:{fontWeight:"medium"},itemValue:{color:"fg.muted"}}}},defaultVariants:{size:"md",orientation:"vertical",variant:"subtle"}}),OS=B({slots:zw.keys(),className:"chakra-dialog",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",left:0,top:0,w:"100dvw",h:"100dvh",zIndex:"var(--z-index)",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100dvw",height:"100dvh",position:"fixed",left:0,top:0,"--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",justifyContent:"center",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,borderRadius:"l3",textStyle:"sm",my:"var(--dialog-margin, var(--dialog-base-margin))","--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"moderate"},_closed:{animationDuration:"faster"}},header:{display:"flex",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{flex:"1",px:"6",pt:"2",pb:"6"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"2",insetEnd:"2"}},variants:{placement:{center:{positioner:{alignItems:"center"},content:{"--dialog-base-margin":"auto",mx:"auto"}},top:{positioner:{alignItems:"flex-start"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}},bottom:{positioner:{alignItems:"flex-end"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}}},scrollBehavior:{inside:{positioner:{overflow:"hidden"},content:{maxH:"calc(100% - 7.5rem)"},body:{overflow:"auto"}},outside:{positioner:{overflow:"auto",pointerEvents:"auto"}}},size:{xs:{content:{maxW:"sm"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},cover:{positioner:{padding:"10"},content:{width:"100%",height:"100%","--dialog-margin":"0"}},full:{content:{maxW:"100dvw",minH:"100dvh","--dialog-margin":"0",borderRadius:"0"}}},motionPreset:{scale:{content:{_open:{animationName:"scale-in, fade-in"},_closed:{animationName:"scale-out, fade-out"}}},"slide-in-bottom":{content:{_open:{animationName:"slide-from-bottom, fade-in"},_closed:{animationName:"slide-to-bottom, fade-out"}}},"slide-in-top":{content:{_open:{animationName:"slide-from-top, fade-in"},_closed:{animationName:"slide-to-top, fade-out"}}},"slide-in-left":{content:{_open:{animationName:"slide-from-left, fade-in"},_closed:{animationName:"slide-to-left, fade-out"}}},"slide-in-right":{content:{_open:{animationName:"slide-from-right, fade-in"},_closed:{animationName:"slide-to-right, fade-out"}}},none:{}}},defaultVariants:{size:"md",scrollBehavior:"outside",placement:"top",motionPreset:"scale"}}),PS=B({slots:Mw.keys(),className:"chakra-drawer",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",insetInlineStart:0,top:0,w:"100vw",h:"100dvh",zIndex:"overlay",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100vw",height:"100dvh",position:"fixed",insetInlineStart:0,top:0,zIndex:"modal",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,zIndex:"modal",textStyle:"sm",maxH:"100dvh",color:"inherit",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"slowest",animationTimingFunction:"ease-in-smooth"},_closed:{animationDuration:"slower",animationTimingFunction:"ease-in-smooth"}},header:{display:"flex",alignItems:"center",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{px:"6",py:"2",flex:"1",overflow:"auto"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{flex:"1",textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"3",insetEnd:"2"}},variants:{size:{xs:{content:{maxW:"xs"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},full:{content:{maxW:"100vw",h:"100dvh"}}},placement:{start:{positioner:{justifyContent:"flex-start",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-left-full, fade-in",_rtl:"slide-from-right-full, fade-in"}},_closed:{animationName:{base:"slide-to-left-full, fade-out",_rtl:"slide-to-right-full, fade-out"}}}},end:{positioner:{justifyContent:"flex-end",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-right-full, fade-in",_rtl:"slide-from-left-full, fade-in"}},_closed:{animationName:{base:"slide-to-right-full, fade-out",_rtl:"slide-to-left-full, fade-out"}}}},top:{positioner:{justifyContent:"stretch",alignItems:"flex-start"},content:{maxW:"100%",_open:{animationName:"slide-from-top-full, fade-in"},_closed:{animationName:"slide-to-top-full, fade-out"}}},bottom:{positioner:{justifyContent:"stretch",alignItems:"flex-end"},content:{maxW:"100%",_open:{animationName:"slide-from-bottom-full, fade-in"},_closed:{animationName:"slide-to-bottom-full, fade-out"}}}},contained:{true:{positioner:{padding:"4"},content:{borderRadius:"l3"}}}},defaultVariants:{size:"xs",placement:"end"}}),Hh=tr({fontSize:"inherit",fontWeight:"inherit",textAlign:"inherit",bg:"transparent",borderRadius:"l2"}),RS=B({slots:jw.keys(),className:"chakra-editable",base:{root:{display:"inline-flex",alignItems:"center",position:"relative",gap:"1.5",width:"full"},preview:{...Hh,py:"1",px:"1",display:"inline-flex",alignItems:"center",transitionProperty:"common",transitionDuration:"moderate",cursor:"text",_hover:{bg:"bg.muted"},_disabled:{userSelect:"none"}},input:{...Hh,outline:"0",py:"1",px:"1",transitionProperty:"common",transitionDuration:"normal",width:"full",focusVisibleRing:"inside",focusRingWidth:"2px",_placeholder:{opacity:.6}},control:{display:"inline-flex",alignItems:"center",gap:"1.5"}},variants:{size:{sm:{root:{textStyle:"sm"},preview:{minH:"8"},input:{minH:"8"}},md:{root:{textStyle:"sm"},preview:{minH:"9"},input:{minH:"9"}},lg:{root:{textStyle:"md"},preview:{minH:"10"},input:{minH:"10"}}}},defaultVariants:{size:"md"}}),IS=B({slots:$w.keys(),className:"chakra-empty-state",base:{root:{width:"full"},content:{display:"flex",flexDirection:"column",alignItems:"center",justifyContent:"center"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:"fg.subtle",_icon:{boxSize:"1em"}},title:{fontWeight:"semibold"},description:{textStyle:"sm",color:"fg.muted"}},variants:{size:{sm:{root:{px:"4",py:"6"},title:{textStyle:"md"},content:{gap:"4"},indicator:{textStyle:"2xl"}},md:{root:{px:"8",py:"12"},title:{textStyle:"lg"},content:{gap:"6"},indicator:{textStyle:"4xl"}},lg:{root:{px:"12",py:"16"},title:{textStyle:"xl"},content:{gap:"8"},indicator:{textStyle:"6xl"}}}},defaultVariants:{size:"md"}}),TS=B({className:"chakra-field",slots:Bw.keys(),base:{requiredIndicator:{color:"fg.error",lineHeight:"1"},root:{display:"flex",width:"100%",position:"relative",gap:"1.5"},label:{display:"flex",alignItems:"center",textAlign:"start",textStyle:"sm",fontWeight:"medium",gap:"1",userSelect:"none",_disabled:{opacity:"0.5"}},errorText:{display:"inline-flex",alignItems:"center",fontWeight:"medium",gap:"1",color:"fg.error",textStyle:"xs"},helperText:{color:"fg.muted",textStyle:"xs"}},variants:{orientation:{vertical:{root:{flexDirection:"column",alignItems:"flex-start"}},horizontal:{root:{flexDirection:"row",alignItems:"center",justifyContent:"space-between"},label:{flex:"0 0 var(--field-label-width, 80px)"}}}},defaultVariants:{orientation:"vertical"}}),NS=B({className:"fieldset",slots:Ww.keys(),base:{root:{display:"flex",flexDirection:"column",width:"full"},content:{display:"flex",flexDirection:"column",width:"full"},legend:{color:"fg",fontWeight:"medium",_disabled:{opacity:"0.5"}},helperText:{color:"fg.muted",textStyle:"sm"},errorText:{display:"inline-flex",alignItems:"center",color:"fg.error",gap:"2",fontWeight:"medium",textStyle:"sm"}},variants:{size:{sm:{root:{spaceY:"2"},content:{gap:"1.5"},legend:{textStyle:"sm"}},md:{root:{spaceY:"4"},content:{gap:"4"},legend:{textStyle:"sm"}},lg:{root:{spaceY:"6"},content:{gap:"4"},legend:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),AS=B({className:"chakra-file-upload",slots:Hw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"4",width:"100%",alignItems:"flex-start"},label:{fontWeight:"medium",textStyle:"sm"},dropzone:{background:"bg",borderRadius:"l3",borderWidth:"2px",borderStyle:"dashed",display:"flex",alignItems:"center",flexDirection:"column",gap:"4",justifyContent:"center",minHeight:"2xs",px:"3",py:"2",transition:"backgrounds",focusVisibleRing:"outside",_hover:{bg:"bg.subtle"},_dragging:{bg:"colorPalette.subtle",borderStyle:"solid",borderColor:"colorPalette.solid"}},dropzoneContent:{display:"flex",flexDirection:"column",alignItems:"center",textAlign:"center",gap:"1",textStyle:"sm"},item:{pos:"relative",textStyle:"sm",animationName:"fade-in",animationDuration:"moderate",background:"bg",borderRadius:"l2",borderWidth:"1px",width:"100%",display:"flex",alignItems:"center",gap:"3",p:"4"},itemGroup:{width:"100%",display:"flex",flexDirection:"column",gap:"3",_empty:{display:"none"}},itemName:{color:"fg",fontWeight:"medium",lineClamp:"1"},itemContent:{display:"flex",flexDirection:"column",gap:"0.5",flex:"1"},itemSizeText:{color:"fg.muted",textStyle:"xs"},itemDeleteTrigger:{display:"flex",alignItems:"center",justifyContent:"center",alignSelf:"flex-start",boxSize:"5",p:"2px",color:"fg.muted",cursor:"button"},itemPreview:{color:"fg.muted",_icon:{boxSize:"4.5"}}},defaultVariants:{}}),_S=B({className:"chakra-hover-card",slots:gd.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--hovercard-bg":"colors.bg.panel",bg:"var(--hovercard-bg)",boxShadow:"lg",maxWidth:"80",borderRadius:"l3",zIndex:"popover",transformOrigin:"var(--transform-origin)",outline:"0",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--hovercard-bg)"},arrowTip:{borderTopWidth:"0.5px",borderInlineStartWidth:"0.5px"}},variants:{size:{xs:{content:{padding:"3"}},sm:{content:{padding:"4"}},md:{content:{padding:"5"}},lg:{content:{padding:"6"}}}},defaultVariants:{size:"md"}}),VS=B({className:"chakra-list",slots:Uw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"var(--list-gap)","& :where(ul, ol)":{marginTop:"var(--list-gap)"}},item:{whiteSpace:"normal",display:"list-item"},indicator:{marginEnd:"2",minHeight:"1lh",flexShrink:0,display:"inline-block",verticalAlign:"middle"}},variants:{variant:{marker:{root:{listStyle:"revert"},item:{_marker:{color:"fg.subtle"}}},plain:{item:{alignItems:"flex-start",display:"inline-flex"}}},align:{center:{item:{alignItems:"center"}},start:{item:{alignItems:"flex-start"}},end:{item:{alignItems:"flex-end"}}}},defaultVariants:{variant:"marker"}}),FS=B({className:"chakra-listbox",slots:dS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},content:{display:"flex",maxH:"96",p:"1",gap:"1",textStyle:"sm",outline:"none",scrollPadding:"1",_horizontal:{flexDirection:"row",overflowX:"auto"},_vertical:{flexDirection:"column",overflowY:"auto"},"--listbox-item-padding-x":"spacing.2","--listbox-item-padding-y":"spacing.1.5"},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"pointer",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)",_highlighted:{outline:"2px solid",outlineColor:"border.emphasized"},_disabled:{pointerEvents:"none",opacity:"0.5"}},empty:{py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{mt:"1.5",_first:{mt:"0"}},itemGroupLabel:{py:"1.5",px:"2",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"4"}}},variants:{variant:{subtle:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_hover:{bg:"bg.emphasized/60"},_selected:{bg:"bg.muted"}}},solid:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_selected:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}},plain:{}}},defaultVariants:{variant:"subtle"}}),LS=B({className:"chakra-menu",slots:Gw.keys(),base:{content:{outline:0,bg:"bg.panel",boxShadow:"lg",color:"fg",maxHeight:"var(--available-height)","--menu-z-index":"zIndex.dropdown",zIndex:"calc(var(--menu-z-index) + var(--layer-index, 0))",borderRadius:"l2",overflow:"hidden",overflowY:"auto",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},item:{textDecoration:"none",color:"fg",userSelect:"none",borderRadius:"l1",width:"100%",display:"flex",cursor:"menuitem",alignItems:"center",textAlign:"start",position:"relative",flex:"0 0 auto",outline:0,_disabled:{layerStyle:"disabled"},"&[data-type]":{ps:"8"}},itemText:{flex:"1"},itemIndicator:{position:"absolute",insetStart:"2",transform:"translateY(-50%)",top:"50%"},itemGroupLabel:{px:"2",py:"1.5",fontWeight:"semibold",textStyle:"sm"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},itemCommand:{opacity:"0.6",textStyle:"xs",ms:"auto",ps:"4",letterSpacing:"widest",fontFamily:"inherit"},separator:{height:"1px",bg:"bg.muted",my:"1",mx:"-1"}},variants:{variant:{subtle:{item:{_highlighted:{bg:"bg.emphasized/60"}}},solid:{item:{_highlighted:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}}},size:{sm:{content:{minW:"8rem",padding:"1",scrollPadding:"1"},item:{gap:"1",textStyle:"xs",py:"1",px:"1.5"}},md:{content:{minW:"8rem",padding:"1.5",scrollPadding:"1.5"},item:{gap:"2",textStyle:"sm",py:"1.5",px:"2"}}}},defaultVariants:{size:"md",variant:"subtle"}}),di=B({className:"chakra-select",slots:Qw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},trigger:{display:"flex",alignItems:"center",justifyContent:"space-between",width:"full",minH:"var(--select-trigger-height)","--input-height":"var(--select-trigger-height)",px:"var(--select-trigger-padding-x)",borderRadius:"l2",userSelect:"none",textAlign:"start",focusVisibleRing:"inside",_placeholderShown:{color:"fg.muted/80"},_disabled:{layerStyle:"disabled"},_invalid:{borderColor:"border.error"}},indicatorGroup:{display:"flex",alignItems:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--select-trigger-padding-x)",pointerEvents:"none"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:{base:"fg.muted",_disabled:"fg.subtle",_invalid:"fg.error"}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"fastest"}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{width:"4",height:"4"}},control:{pos:"relative"},itemText:{flex:"1"},itemGroup:{_first:{mt:"0"}},itemGroupLabel:{py:"1",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"}},variants:{variant:{outline:{trigger:{bg:"transparent",borderWidth:"1px",borderColor:"border",_expanded:{borderColor:"border.emphasized"}}},subtle:{trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}},size:{xs:{root:{"--select-trigger-height":"sizes.8","--select-trigger-padding-x":"spacing.2"},content:{p:"1",gap:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"},item:{py:"1",px:"2"},itemGroupLabel:{py:"1",px:"2"},indicator:{_icon:{width:"3.5",height:"3.5"}}},sm:{root:{"--select-trigger-height":"sizes.9","--select-trigger-padding-x":"spacing.2.5"},content:{p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"},indicator:{_icon:{width:"4",height:"4"}},item:{py:"1",px:"1.5"},itemGroup:{mt:"1"},itemGroupLabel:{py:"1",px:"1.5"}},md:{root:{"--select-trigger-height":"sizes.10","--select-trigger-padding-x":"spacing.3"},content:{p:"1",textStyle:"sm"},itemGroup:{mt:"1.5"},item:{py:"1.5",px:"2"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},itemGroupLabel:{py:"1.5",px:"2"},trigger:{textStyle:"sm",gap:"2"},indicator:{_icon:{width:"4",height:"4"}}},lg:{root:{"--select-trigger-height":"sizes.12","--select-trigger-padding-x":"spacing.4"},content:{p:"1.5",textStyle:"md"},itemGroup:{mt:"2"},item:{py:"2",px:"3"},itemGroupLabel:{py:"2",px:"3"},trigger:{textStyle:"md",py:"3",gap:"2"},indicator:{_icon:{width:"5",height:"5"}}}}},defaultVariants:{size:"md",variant:"outline"}}),DS=B({className:"chakra-native-select",slots:qw.keys(),base:{root:{height:"fit-content",display:"flex",width:"100%",position:"relative"},field:{width:"100%",minWidth:"0",outline:"0",appearance:"none",borderRadius:"l2","--error-color":"colors.border.error","--input-height":"var(--select-field-height)",height:"var(--select-field-height)",_disabled:{layerStyle:"disabled"},_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"},focusVisibleRing:"inside",lineHeight:"normal","& > option, & > optgroup":{bg:"bg"}},indicator:{position:"absolute",display:"inline-flex",alignItems:"center",justifyContent:"center",pointerEvents:"none",top:"50%",transform:"translateY(-50%)",height:"100%",color:"fg.muted",_disabled:{opacity:"0.5"},_invalid:{color:"fg.error"},_icon:{width:"1em",height:"1em"}}},variants:{variant:{outline:{field:di.variants?.variant.outline.trigger},subtle:{field:di.variants?.variant.subtle.trigger},plain:{field:{bg:"transparent",color:"fg",focusRingWidth:"2px"}}},size:{xs:{root:{"--select-field-height":"sizes.8"},field:{textStyle:"xs",ps:"2",pe:"6"},indicator:{textStyle:"sm",insetEnd:"1.5"}},sm:{root:{"--select-field-height":"sizes.9"},field:{textStyle:"sm",ps:"2.5",pe:"8"},indicator:{textStyle:"md",insetEnd:"2"}},md:{root:{"--select-field-height":"sizes.10"},field:{textStyle:"sm",ps:"3",pe:"8"},indicator:{textStyle:"lg",insetEnd:"2"}},lg:{root:{"--select-field-height":"sizes.11"},field:{textStyle:"md",ps:"4",pe:"8"},indicator:{textStyle:"xl",insetEnd:"3"}},xl:{root:{"--select-field-height":"sizes.12"},field:{textStyle:"md",ps:"4.5",pe:"10"},indicator:{textStyle:"xl",insetEnd:"3"}}}},defaultVariants:di.defaultVariants});function Ta(e,t){const n={};for(const r in e){const o=t(r,e[r]);n[o[0]]=o[1]}return n}const Uh=tr({display:"flex",justifyContent:"center",alignItems:"center",flex:"1",userSelect:"none",cursor:"button",lineHeight:"1",color:"fg.muted","--stepper-base-radius":"radii.l1","--stepper-radius":"calc(var(--stepper-base-radius) + 1px)",_icon:{boxSize:"1em"},_disabled:{opacity:"0.5"},_hover:{bg:"bg.muted"},_active:{bg:"bg.emphasized"}}),zS=B({className:"chakra-number-input",slots:Nd.keys(),base:{root:{position:"relative",zIndex:"0",isolation:"isolate"},input:{...he.base,verticalAlign:"top",pe:"calc(var(--stepper-width) + 0.5rem)"},control:{display:"flex",flexDirection:"column",position:"absolute",top:"0",insetEnd:"0px",margin:"1px",width:"var(--stepper-width)",height:"calc(100% - 2px)",zIndex:"1",borderStartWidth:"1px",divideY:"1px"},incrementTrigger:{...Uh,borderTopEndRadius:"var(--stepper-radius)"},decrementTrigger:{...Uh,borderBottomEndRadius:"var(--stepper-radius)"},valueText:{fontWeight:"medium",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums"}},variants:{size:{xs:{input:he.variants.size.xs,control:{fontSize:"2xs","--stepper-width":"sizes.4"}},sm:{input:he.variants.size.sm,control:{fontSize:"xs","--stepper-width":"sizes.5"}},md:{input:he.variants.size.md,control:{fontSize:"sm","--stepper-width":"sizes.6"}},lg:{input:he.variants.size.lg,control:{fontSize:"sm","--stepper-width":"sizes.6"}}},variant:Ta(he.variants.variant,(e,t)=>[e,{input:t}])},defaultVariants:{size:"md",variant:"outline"}}),{variants:Gh,defaultVariants:MS}=he,jS=B({className:"chakra-pin-input",slots:Ld.keys(),base:{input:{...he.base,textAlign:"center",width:"var(--input-height)"},control:{display:"inline-flex",gap:"2",isolation:"isolate"}},variants:{size:Ta(Gh.size,(e,t)=>[e,{input:{...t,px:"1"}}]),variant:Ta(Gh.variant,(e,t)=>[e,{input:t}]),attached:{true:{control:{gap:"0",spaceX:"-1px"},input:{_notFirst:{borderStartRadius:"0"},_notLast:{borderEndRadius:"0"},_focusVisible:{zIndex:"1"}}}}},defaultVariants:MS}),$S=B({className:"chakra-popover",slots:Kw.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--popover-bg":"colors.bg.panel",bg:"var(--popover-bg)",boxShadow:"lg","--popover-size":"sizes.xs","--popover-mobile-size":"calc(100dvw - 1rem)",width:{base:"min(var(--popover-mobile-size), var(--popover-size))",sm:"var(--popover-size)"},borderRadius:"l3","--popover-z-index":"zIndex.popover",zIndex:"calc(var(--popover-z-index) + var(--layer-index, 0))",outline:"0",transformOrigin:"var(--transform-origin)",maxHeight:"var(--available-height)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"faster"}},header:{paddingInline:"var(--popover-padding)",paddingTop:"var(--popover-padding)"},body:{padding:"var(--popover-padding)",flex:"1"},footer:{display:"flex",alignItems:"center",paddingInline:"var(--popover-padding)",paddingBottom:"var(--popover-padding)"},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--popover-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px"}},variants:{size:{xs:{content:{"--popover-padding":"spacing.3"}},sm:{content:{"--popover-padding":"spacing.4"}},md:{content:{"--popover-padding":"spacing.5"}},lg:{content:{"--popover-padding":"spacing.6"}}}},defaultVariants:{size:"md"}}),BS=B({slots:ha.keys(),className:"chakra-progress",base:{root:{textStyle:"sm",position:"relative"},track:{overflow:"hidden",position:"relative"},range:{display:"flex",alignItems:"center",justifyContent:"center",transitionProperty:"width, height",transitionDuration:"slow",height:"100%",bgColor:"var(--track-color)",_indeterminate:{"--animate-from-x":"-40%","--animate-to-x":"100%",position:"absolute",willChange:"left",minWidth:"50%",animation:"position 1s ease infinite normal none running",backgroundImage:"linear-gradient(to right, transparent 0%, var(--track-color) 50%, transparent 100%)"}},label:{display:"inline-flex",fontWeight:"medium",alignItems:"center",gap:"1"},valueText:{textStyle:"xs",lineHeight:"1",fontWeight:"medium"}},variants:{variant:{outline:{track:{shadow:"inset",bgColor:"bg.muted"},range:{bgColor:"colorPalette.solid"}},subtle:{track:{bgColor:"colorPalette.muted"},range:{bgColor:"colorPalette.solid/72"}}},shape:{square:{},rounded:{track:{borderRadius:"l1"}},full:{track:{borderRadius:"full"}}},striped:{true:{range:{backgroundImage:"linear-gradient(45deg, var(--stripe-color) 25%, transparent 25%, transparent 50%, var(--stripe-color) 50%, var(--stripe-color) 75%, transparent 75%, transparent)",backgroundSize:"var(--stripe-size) var(--stripe-size)","--stripe-size":"1rem","--stripe-color":{_light:"rgba(255, 255, 255, 0.3)",_dark:"rgba(0, 0, 0, 0.3)"}}}},animated:{true:{range:{"--animate-from":"var(--stripe-size)",animation:"bg-position 1s linear infinite"}}},size:{xs:{track:{h:"1.5"}},sm:{track:{h:"2"}},md:{track:{h:"2.5"}},lg:{track:{h:"3"}},xl:{track:{h:"4"}}}},defaultVariants:{variant:"outline",size:"md",shape:"rounded"}}),WS=B({className:"chakra-progress-circle",slots:ha.keys(),base:{root:{display:"inline-flex",textStyle:"sm",position:"relative"},circle:{_indeterminate:{animation:"spin 2s linear infinite"}},circleTrack:{"--track-color":"colors.colorPalette.muted",stroke:"var(--track-color)"},circleRange:{stroke:"colorPalette.solid",transitionProperty:"stroke-dashoffset, stroke-dasharray",transitionDuration:"0.6s",_indeterminate:{animation:"circular-progress 1.5s linear infinite"}},label:{display:"inline-flex"},valueText:{lineHeight:"1",fontWeight:"medium",letterSpacing:"tight",fontVariantNumeric:"tabular-nums"}},variants:{size:{xs:{circle:{"--size":"24px","--thickness":"4px"},valueText:{textStyle:"2xs"}},sm:{circle:{"--size":"32px","--thickness":"5px"},valueText:{textStyle:"2xs"}},md:{circle:{"--size":"40px","--thickness":"6px"},valueText:{textStyle:"xs"}},lg:{circle:{"--size":"48px","--thickness":"7px"},valueText:{textStyle:"sm"}},xl:{circle:{"--size":"64px","--thickness":"8px"},valueText:{textStyle:"sm"}}}},defaultVariants:{size:"md"}}),HS=B({slots:zd.keys(),className:"chakra-qr-code",base:{root:{position:"relative",width:"fit-content","--qr-code-overlay-size":"calc(var(--qr-code-size) / 3)"},frame:{width:"var(--qr-code-size)",height:"var(--qr-code-size)",fill:"currentColor"},overlay:{display:"flex",alignItems:"center",justifyContent:"center",width:"var(--qr-code-overlay-size)",height:"var(--qr-code-overlay-size)",padding:"1",bg:"bg",rounded:"l1"}},variants:{size:{"2xs":{root:{"--qr-code-size":"40px"}},xs:{root:{"--qr-code-size":"64px"}},sm:{root:{"--qr-code-size":"80px"}},md:{root:{"--qr-code-size":"120px"}},lg:{root:{"--qr-code-size":"160px"}},xl:{root:{"--qr-code-size":"200px"}},"2xl":{root:{"--qr-code-size":"240px"}},full:{root:{"--qr-code-size":"100%"}}}},defaultVariants:{size:"md"}}),US=B({className:"chakra-radio-card",slots:Yw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",isolation:"isolate"},item:{flex:"1",display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",_focus:{bg:"colorPalette.muted/20"},_disabled:{opacity:"0.8",borderColor:"border.disabled"},_checked:{zIndex:"1"}},label:{display:"inline-flex",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},itemText:{fontWeight:"medium",flex:"1"},itemDescription:{opacity:"0.64",textStyle:"sm"},itemControl:{display:"inline-flex",flex:"1",pos:"relative",rounded:"inherit",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)",_disabled:{bg:"bg.muted"}},itemIndicator:Ee.base,itemAddon:{roundedBottom:"inherit",_disabled:{color:"fg.muted"}},itemContent:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)"}},variants:{size:{sm:{item:{textStyle:"sm"},itemControl:{padding:"3",gap:"1.5"},itemAddon:{px:"3",py:"1.5",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.sm},md:{item:{textStyle:"sm"},itemControl:{padding:"4",gap:"2.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.md},lg:{item:{textStyle:"md"},itemControl:{padding:"4",gap:"3.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.lg}},variant:{surface:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"}},itemIndicator:Ee.variants?.variant.solid},subtle:{item:{bg:"bg.muted"},itemControl:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},itemIndicator:Ee.variants?.variant.outline},outline:{item:{borderWidth:"1px",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.solid},solid:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.inverted}},justify:{start:{item:{"--radio-card-justify":"flex-start"}},end:{item:{"--radio-card-justify":"flex-end"}},center:{item:{"--radio-card-justify":"center"}}},align:{start:{item:{"--radio-card-align":"flex-start"},itemControl:{textAlign:"start"}},end:{item:{"--radio-card-align":"flex-end"},itemControl:{textAlign:"end"}},center:{item:{"--radio-card-align":"center"},itemControl:{textAlign:"center"}}},orientation:{vertical:{itemControl:{flexDirection:"column"}},horizontal:{itemControl:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),GS=B({className:"chakra-radio-group",slots:Wh.keys(),base:{item:{display:"inline-flex",alignItems:"center",position:"relative",fontWeight:"medium",_disabled:{cursor:"disabled"}},itemControl:Ee.base,label:{userSelect:"none",textStyle:"sm",_disabled:{opacity:"0.5"}}},variants:{variant:{outline:{itemControl:Ee.variants?.variant?.outline},subtle:{itemControl:Ee.variants?.variant?.subtle},solid:{itemControl:Ee.variants?.variant?.solid}},size:{xs:{item:{textStyle:"xs",gap:"1.5"},itemControl:Ee.variants?.size?.xs},sm:{item:{textStyle:"sm",gap:"2"},itemControl:Ee.variants?.size?.sm},md:{item:{textStyle:"sm",gap:"2.5"},itemControl:Ee.variants?.size?.md},lg:{item:{textStyle:"md",gap:"3"},itemControl:Ee.variants?.size?.lg}}},defaultVariants:{size:"md",variant:"solid"}}),qS=B({className:"chakra-rating-group",slots:Xw.keys(),base:{root:{display:"inline-flex"},control:{display:"inline-flex",alignItems:"center"},item:{display:"inline-flex",alignItems:"center",justifyContent:"center",userSelect:"none"},itemIndicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",width:"1em",height:"1em",position:"relative","--clip-path":{base:"inset(0 50% 0 0)",_rtl:"inset(0 0 0 50%)"},_icon:{stroke:"currentColor",width:"100%",height:"100%",display:"inline-block",flexShrink:0,position:"absolute",left:0,top:0},"& [data-bg]":{color:"bg.emphasized"},"& [data-fg]":{color:"transparent"},"&[data-highlighted]:not([data-half])":{"& [data-fg]":{color:"colorPalette.solid"}},"&[data-half]":{"& [data-fg]":{color:"colorPalette.solid",clipPath:"var(--clip-path)"}}}},variants:{size:{xs:{item:{textStyle:"sm"}},sm:{item:{textStyle:"md"}},md:{item:{textStyle:"xl"}},lg:{item:{textStyle:"2xl"}}}},defaultVariants:{size:"md"}}),KS=B({className:"chakra-scroll-area",slots:jd.keys(),base:{root:{display:"flex",flexDirection:"column",width:"100%",height:"100%",position:"relative",overflow:"hidden","--scrollbar-margin":"2px","--scrollbar-click-area":"calc(var(--scrollbar-size) + calc(var(--scrollbar-margin) * 2))"},viewport:{display:"flex",flexDirection:"column",height:"100%",width:"100%",borderRadius:"inherit",WebkitOverflowScrolling:"touch",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}},content:{minWidth:"100%"},scrollbar:{display:"flex",userSelect:"none",touchAction:"none",borderRadius:"full",colorPalette:"gray",transition:"opacity 150ms 300ms",position:"relative",margin:"var(--scrollbar-margin)","&:not([data-overflow-x], [data-overflow-y])":{display:"none"},bg:"{colors.colorPalette.solid/10}","--thumb-bg":"{colors.colorPalette.solid/25}","&:is(:hover, :active)":{"--thumb-bg":"{colors.colorPalette.solid/50}"},_before:{content:'""',position:"absolute"},_vertical:{width:"var(--scrollbar-size)",flexDirection:"column","&::before":{width:"var(--scrollbar-click-area)",height:"100%",insetInlineStart:"calc(var(--scrollbar-margin) * -1)"}},_horizontal:{height:"var(--scrollbar-size)",flexDirection:"row","&::before":{height:"var(--scrollbar-click-area)",width:"100%",top:"calc(var(--scrollbar-margin) * -1)"}}},thumb:{borderRadius:"inherit",bg:"var(--thumb-bg)",transition:"backgrounds",_vertical:{width:"full"},_horizontal:{height:"full"}},corner:{bg:"bg.muted",margin:"var(--scrollbar-margin)",opacity:0,transition:"opacity 150ms 300ms","&[data-hover]":{transitionDelay:"0ms",opacity:1}}},variants:{variant:{hover:{scrollbar:{opacity:"0","&[data-hover], &[data-scrolling]":{opacity:"1",transitionDuration:"faster",transitionDelay:"0ms"}}},always:{scrollbar:{opacity:"1"}}},size:{xs:{root:{"--scrollbar-size":"sizes.1"}},sm:{root:{"--scrollbar-size":"sizes.1.5"}},md:{root:{"--scrollbar-size":"sizes.2"}},lg:{root:{"--scrollbar-size":"sizes.3"}}}},defaultVariants:{size:"md",variant:"hover"}}),YS=B({className:"chakra-segment-group",slots:$d.keys(),base:{root:{"--segment-radius":"radii.l2",borderRadius:"l2",display:"inline-flex",boxShadow:"inset",minW:"max-content",textAlign:"center",position:"relative",isolation:"isolate",bg:"bg.muted",_vertical:{flexDirection:"column"}},item:{display:"flex",alignItems:"center",justifyContent:"center",userSelect:"none",fontSize:"sm",position:"relative",color:"fg",borderRadius:"var(--segment-radius)",_disabled:{opacity:"0.5"},"&:has(input:focus-visible)":{focusRing:"outside"},_before:{content:'""',position:"absolute",bg:"border",transition:"opacity 0.2s"},_horizontal:{_before:{insetInlineStart:0,insetBlock:"1.5",width:"1px"}},_vertical:{_before:{insetBlockStart:0,insetInline:"1.5",height:"1px"}},"& + &[data-state=checked], &[data-state=checked] + &, &:first-of-type":{_before:{opacity:"0"}},"&[data-state=checked][data-ssr]":{shadow:"sm",bg:"bg",borderRadius:"var(--segment-radius)"}},indicator:{shadow:"sm",pos:"absolute",bg:{_light:"bg",_dark:"bg.emphasized"},width:"var(--width)",height:"var(--height)",top:"var(--top)",left:"var(--left)",zIndex:-1,borderRadius:"var(--segment-radius)"}},variants:{size:{xs:{item:{textStyle:"xs",px:"3",gap:"1",height:"6"}},sm:{item:{textStyle:"sm",px:"4",gap:"2",height:"8"}},md:{item:{textStyle:"sm",px:"4",gap:"2",height:"10"}},lg:{item:{textStyle:"md",px:"4.5",gap:"3",height:"11"}}}},defaultVariants:{size:"md"}}),XS=B({className:"chakra-slider",slots:Zw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",textStyle:"sm",position:"relative",isolation:"isolate",touchAction:"none"},label:{fontWeight:"medium",textStyle:"sm"},control:{display:"inline-flex",alignItems:"center",position:"relative"},track:{overflow:"hidden",borderRadius:"full",flex:"1"},range:{width:"inherit",height:"inherit",_disabled:{bg:"border.emphasized!"}},markerGroup:{position:"absolute!",zIndex:"1"},marker:{"--marker-bg":{base:"white",_underValue:"colors.bg"},display:"flex",alignItems:"center",gap:"calc(var(--slider-thumb-size) / 2)",color:"fg.muted",textStyle:"xs"},markerIndicator:{width:"var(--slider-marker-size)",height:"var(--slider-marker-size)",borderRadius:"full",bg:"var(--marker-bg)"},thumb:{width:"var(--slider-thumb-size)",height:"var(--slider-thumb-size)",display:"flex",alignItems:"center",justifyContent:"center",outline:0,zIndex:"2",borderRadius:"full",_focusVisible:{ring:"2px",ringColor:"colorPalette.focusRing",ringOffset:"2px",ringOffsetColor:"bg"}}},variants:{size:{sm:{root:{"--slider-thumb-size":"sizes.4","--slider-track-size":"sizes.1.5","--slider-marker-center":"6px","--slider-marker-size":"sizes.1","--slider-marker-inset":"3px"}},md:{root:{"--slider-thumb-size":"sizes.5","--slider-track-size":"sizes.2","--slider-marker-center":"8px","--slider-marker-size":"sizes.1","--slider-marker-inset":"4px"}},lg:{root:{"--slider-thumb-size":"sizes.6","--slider-track-size":"sizes.2.5","--slider-marker-center":"9px","--slider-marker-size":"sizes.1.5","--slider-marker-inset":"5px"}}},variant:{outline:{track:{shadow:"inset",bg:"bg.emphasized/72"},range:{bg:"colorPalette.solid"},thumb:{borderWidth:"2px",borderColor:"colorPalette.solid",bg:"bg",_disabled:{bg:"border.emphasized",borderColor:"border.emphasized"}}},solid:{track:{bg:"colorPalette.subtle",_disabled:{bg:"bg.muted"}},range:{bg:"colorPalette.solid"},thumb:{bg:"colorPalette.solid",_disabled:{bg:"border.emphasized"}}}},orientation:{vertical:{root:{display:"inline-flex"},control:{flexDirection:"column",height:"100%",minWidth:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginEnd:"4"}},track:{width:"var(--slider-track-size)"},thumb:{left:"50%",translate:"-50% 0"},markerGroup:{insetStart:"var(--slider-marker-center)",insetBlock:"var(--slider-marker-inset)"},marker:{flexDirection:"row"}},horizontal:{control:{flexDirection:"row",width:"100%",minHeight:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginBottom:"4"}},track:{height:"var(--slider-track-size)"},thumb:{top:"50%",translate:"0 -50%"},markerGroup:{top:"var(--slider-marker-center)",insetInline:"var(--slider-marker-inset)"},marker:{flexDirection:"column"}}}},defaultVariants:{size:"md",variant:"outline",orientation:"horizontal"}}),QS=B({className:"chakra-stat",slots:eS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",position:"relative",flex:"1"},label:{display:"inline-flex",gap:"1.5",alignItems:"center",color:"fg.muted",textStyle:"sm"},helpText:{color:"fg.muted",textStyle:"xs"},valueUnit:{color:"fg.muted",textStyle:"xs",fontWeight:"initial",letterSpacing:"initial"},valueText:{verticalAlign:"baseline",fontWeight:"semibold",letterSpacing:"tight",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums",display:"inline-flex",gap:"1"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",marginEnd:1,"& :where(svg)":{w:"1em",h:"1em"},"&[data-type=up]":{color:"fg.success"},"&[data-type=down]":{color:"fg.error"}}},variants:{size:{sm:{valueText:{textStyle:"xl"}},md:{valueText:{textStyle:"2xl"}},lg:{valueText:{textStyle:"3xl"}}}},defaultVariants:{size:"md"}}),JS=B({className:"chakra-status",slots:tS.keys(),base:{root:{display:"inline-flex",alignItems:"center",gap:"2"},indicator:{width:"0.64em",height:"0.64em",flexShrink:0,borderRadius:"full",forcedColorAdjust:"none",bg:"colorPalette.solid"}},variants:{size:{sm:{root:{textStyle:"xs"}},md:{root:{textStyle:"sm"}},lg:{root:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),ZS=B({className:"chakra-steps",slots:nS.keys(),base:{root:{display:"flex",width:"full"},list:{display:"flex",justifyContent:"space-between","--steps-gutter":"spacing.3","--steps-thickness":"2px"},title:{fontWeight:"medium",color:"fg"},description:{color:"fg.muted"},separator:{bg:"border",flex:"1"},indicator:{display:"flex",justifyContent:"center",alignItems:"center",flexShrink:"0",borderRadius:"full",fontWeight:"medium",width:"var(--steps-size)",height:"var(--steps-size)",_icon:{flexShrink:"0",width:"var(--steps-icon-size)",height:"var(--steps-icon-size)"}},item:{position:"relative",display:"flex",gap:"3",flex:"1 0 0","&:last-of-type":{flex:"initial","& [data-part=separator]":{display:"none"}}},trigger:{display:"flex",alignItems:"center",gap:"3",textAlign:"start",focusVisibleRing:"outside",borderRadius:"l2"},content:{focusVisibleRing:"outside"}},variants:{orientation:{vertical:{root:{flexDirection:"row",height:"100%"},list:{flexDirection:"column",alignItems:"flex-start"},separator:{position:"absolute",width:"var(--steps-thickness)",height:"100%",maxHeight:"calc(100% - var(--steps-size) - var(--steps-gutter) * 2)",top:"calc(var(--steps-size) + var(--steps-gutter))",insetStart:"calc(var(--steps-size) / 2 - 1px)"},item:{alignItems:"flex-start"}},horizontal:{root:{flexDirection:"column",width:"100%"},list:{flexDirection:"row",alignItems:"center"},separator:{width:"100%",height:"var(--steps-thickness)",marginX:"var(--steps-gutter)"},item:{alignItems:"center"}}},variant:{solid:{indicator:{_incomplete:{borderWidth:"var(--steps-thickness)"},_current:{bg:"colorPalette.muted",borderWidth:"var(--steps-thickness)",borderColor:"colorPalette.solid",color:"colorPalette.fg"},_complete:{bg:"colorPalette.solid",borderColor:"colorPalette.solid",color:"colorPalette.contrast"}},separator:{_complete:{bg:"colorPalette.solid"}}},subtle:{indicator:{_incomplete:{bg:"bg.muted"},_current:{bg:"colorPalette.muted",color:"colorPalette.fg"},_complete:{bg:"colorPalette.emphasized",color:"colorPalette.fg"}},separator:{_complete:{bg:"colorPalette.emphasized"}}}},size:{xs:{root:{gap:"2.5"},list:{"--steps-size":"sizes.6","--steps-icon-size":"sizes.3.5",textStyle:"xs"},title:{textStyle:"sm"}},sm:{root:{gap:"3"},list:{"--steps-size":"sizes.8","--steps-icon-size":"sizes.4",textStyle:"xs"},title:{textStyle:"sm"}},md:{root:{gap:"4"},list:{"--steps-size":"sizes.10","--steps-icon-size":"sizes.4",textStyle:"sm"},title:{textStyle:"sm"}},lg:{root:{gap:"6"},list:{"--steps-size":"sizes.11","--steps-icon-size":"sizes.5",textStyle:"md"},title:{textStyle:"md"}}}},defaultVariants:{size:"md",variant:"solid",orientation:"horizontal"}}),eE=B({slots:rS.keys(),className:"chakra-switch",base:{root:{display:"inline-flex",gap:"2.5",alignItems:"center",position:"relative",verticalAlign:"middle","--switch-diff":"calc(var(--switch-width) - var(--switch-height))","--switch-x":{base:"var(--switch-diff)",_rtl:"calc(var(--switch-diff) * -1)"}},label:{lineHeight:"1",userSelect:"none",fontSize:"sm",fontWeight:"medium",_disabled:{opacity:"0.5"}},indicator:{position:"absolute",height:"var(--switch-height)",width:"var(--switch-height)",fontSize:"var(--switch-indicator-font-size)",fontWeight:"medium",flexShrink:0,userSelect:"none",display:"grid",placeContent:"center",transition:"inset-inline-start 0.12s ease",insetInlineStart:"calc(var(--switch-x) - 2px)",_checked:{insetInlineStart:"2px"}},control:{display:"inline-flex",gap:"0.5rem",flexShrink:0,justifyContent:"flex-start",cursor:"switch",borderRadius:"full",position:"relative",width:"var(--switch-width)",height:"var(--switch-height)",transition:"backgrounds",_disabled:{opacity:"0.5",cursor:"not-allowed"},_invalid:{outline:"2px solid",outlineColor:"border.error",outlineOffset:"2px"}},thumb:{display:"flex",alignItems:"center",justifyContent:"center",flexShrink:0,transitionProperty:"translate",transitionDuration:"fast",borderRadius:"inherit",_checked:{translate:"var(--switch-x) 0"}}},variants:{variant:{solid:{control:{borderRadius:"full",bg:"bg.emphasized",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}},thumb:{bg:"white",width:"var(--switch-height)",height:"var(--switch-height)",scale:"0.8",boxShadow:"sm",_checked:{bg:"colorPalette.contrast"}}},raised:{control:{borderRadius:"full",height:"calc(var(--switch-height) / 2)",bg:"bg.muted",boxShadow:"inset",_checked:{bg:"colorPalette.solid/60"}},thumb:{width:"var(--switch-height)",height:"var(--switch-height)",position:"relative",top:"calc(var(--switch-height) * -0.25)",bg:"white",boxShadow:"xs",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}}}},size:{xs:{root:{"--switch-width":"sizes.6","--switch-height":"sizes.3","--switch-indicator-font-size":"fontSizes.xs"}},sm:{root:{"--switch-width":"sizes.8","--switch-height":"sizes.4","--switch-indicator-font-size":"fontSizes.xs"}},md:{root:{"--switch-width":"sizes.10","--switch-height":"sizes.5","--switch-indicator-font-size":"fontSizes.sm"}},lg:{root:{"--switch-width":"sizes.12","--switch-height":"sizes.6","--switch-indicator-font-size":"fontSizes.md"}}}},defaultVariants:{variant:"solid",size:"md"}}),tE=B({className:"chakra-table",slots:oS.keys(),base:{root:{fontVariantNumeric:"lining-nums tabular-nums",borderCollapse:"collapse",width:"full",textAlign:"start",verticalAlign:"top"},row:{_selected:{bg:"colorPalette.subtle"}},cell:{textAlign:"start",alignItems:"center"},columnHeader:{fontWeight:"medium",textAlign:"start",color:"fg"},caption:{fontWeight:"medium",textStyle:"xs"},footer:{fontWeight:"medium"}},variants:{interactive:{true:{body:{"& tr":{_hover:{bg:"colorPalette.subtle"}}}}},stickyHeader:{true:{header:{"& :where(tr)":{top:"var(--table-sticky-offset, 0)",position:"sticky",zIndex:1}}}},striped:{true:{row:{"&:nth-of-type(odd) td":{bg:"bg.muted"}}}},showColumnBorder:{true:{columnHeader:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}},cell:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}}}},variant:{line:{columnHeader:{borderBottomWidth:"1px"},cell:{borderBottomWidth:"1px"},row:{bg:"bg"}},outline:{root:{boxShadow:"0 0 0 1px {colors.border}",overflow:"hidden"},columnHeader:{borderBottomWidth:"1px"},header:{bg:"bg.muted"},row:{"&:not(:last-of-type)":{borderBottomWidth:"1px"}},footer:{borderTopWidth:"1px"}}},size:{sm:{root:{textStyle:"sm"},columnHeader:{px:"2",py:"2"},cell:{px:"2",py:"2"}},md:{root:{textStyle:"sm"},columnHeader:{px:"3",py:"3"},cell:{px:"3",py:"3"}},lg:{root:{textStyle:"md"},columnHeader:{px:"4",py:"3"},cell:{px:"4",py:"3"}}}},defaultVariants:{variant:"line",size:"md"}}),nE=B({slots:sS.keys(),className:"chakra-tabs",base:{root:{"--tabs-trigger-radius":"radii.l2",position:"relative",_horizontal:{display:"block"},_vertical:{display:"flex"}},list:{display:"inline-flex",position:"relative",isolation:"isolate","--tabs-indicator-shadow":"shadows.xs","--tabs-indicator-bg":"colors.bg",minH:"var(--tabs-height)",_horizontal:{flexDirection:"row"},_vertical:{flexDirection:"column"}},trigger:{outline:"0",minW:"var(--tabs-height)",height:"var(--tabs-height)",display:"flex",alignItems:"center",fontWeight:"medium",position:"relative",cursor:"button",gap:"2",_focusVisible:{zIndex:1,outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{cursor:"not-allowed",opacity:.5}},content:{focusVisibleRing:"inside",_horizontal:{width:"100%",pt:"var(--tabs-content-padding)"},_vertical:{height:"100%",ps:"var(--tabs-content-padding)"}},indicator:{width:"var(--width)",height:"var(--height)",borderRadius:"var(--tabs-indicator-radius)",bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",zIndex:-1}},variants:{fitted:{true:{list:{display:"flex"},trigger:{flex:1,textAlign:"center",justifyContent:"center"}}},justify:{start:{list:{justifyContent:"flex-start"}},center:{list:{justifyContent:"center"}},end:{list:{justifyContent:"flex-end"}}},size:{sm:{root:{"--tabs-height":"sizes.9","--tabs-content-padding":"spacing.3"},trigger:{py:"1",px:"3",textStyle:"sm"}},md:{root:{"--tabs-height":"sizes.10","--tabs-content-padding":"spacing.4"},trigger:{py:"2",px:"4",textStyle:"sm"}},lg:{root:{"--tabs-height":"sizes.11","--tabs-content-padding":"spacing.4.5"},trigger:{py:"2",px:"4.5",textStyle:"md"}}},variant:{line:{list:{display:"flex",borderColor:"border",_horizontal:{borderBottomWidth:"1px"},_vertical:{borderEndWidth:"1px"}},trigger:{color:"fg.muted",_disabled:{_active:{bg:"initial"}},_selected:{color:"fg",_horizontal:{layerStyle:"indicator.bottom","--indicator-offset-y":"-1px","--indicator-color":"colors.colorPalette.solid"},_vertical:{layerStyle:"indicator.end","--indicator-offset-x":"-1px"}}}},subtle:{trigger:{borderRadius:"var(--tabs-trigger-radius)",color:"fg.muted",_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}},enclosed:{list:{bg:"bg.muted",padding:"1",borderRadius:"l3",minH:"calc(var(--tabs-height) - 4px)"},trigger:{justifyContent:"center",color:"fg.muted",borderRadius:"var(--tabs-trigger-radius)",_selected:{bg:"bg",color:"colorPalette.fg",shadow:"xs"}}},outline:{list:{"--line-thickness":"1px","--line-offset":"calc(var(--line-thickness) * -1)",borderColor:"border",display:"flex",_horizontal:{_before:{content:'""',position:"absolute",bottom:"0px",width:"100%",borderBottomWidth:"var(--line-thickness)",borderBottomColor:"border"}},_vertical:{_before:{content:'""',position:"absolute",insetInline:"var(--line-offset)",height:"calc(100% - calc(var(--line-thickness) * 2))",borderEndWidth:"var(--line-thickness)",borderEndColor:"border"}}},trigger:{color:"fg.muted",borderWidth:"1px",borderColor:"transparent",_selected:{bg:"currentBg",color:"colorPalette.fg"},_horizontal:{borderTopRadius:"var(--tabs-trigger-radius)",marginBottom:"var(--line-offset)",marginEnd:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderBottomColor:"transparent"}},_vertical:{borderStartRadius:"var(--tabs-trigger-radius)",marginEnd:"var(--line-offset)",marginBottom:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderEndColor:"transparent"}}}},plain:{trigger:{color:"fg.muted",_selected:{color:"colorPalette.fg"},borderRadius:"var(--tabs-trigger-radius)","&[data-selected][data-ssr]":{bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",borderRadius:"var(--tabs-indicator-radius)"}}}}},defaultVariants:{size:"md",variant:"line"}}),hi=Ia.variants?.variant,rE=B({slots:aS.keys(),className:"chakra-tag",base:{root:{display:"inline-flex",alignItems:"center",verticalAlign:"top",maxWidth:"100%",userSelect:"none",borderRadius:"l2",focusVisibleRing:"outside"},label:{lineClamp:"1"},closeTrigger:{display:"flex",alignItems:"center",justifyContent:"center",outline:"0",borderRadius:"l1",color:"currentColor",focusVisibleRing:"inside",focusRingWidth:"2px"},startElement:{flexShrink:0,boxSize:"var(--tag-element-size)",ms:"var(--tag-element-offset)","&:has([data-scope=avatar])":{boxSize:"var(--tag-avatar-size)",ms:"calc(var(--tag-element-offset) * 1.5)"},_icon:{boxSize:"100%"}},endElement:{flexShrink:0,boxSize:"var(--tag-element-size)",me:"var(--tag-element-offset)",_icon:{boxSize:"100%"},"&:has(button)":{ms:"calc(var(--tag-element-offset) * -1)"}}},variants:{size:{sm:{root:{px:"1.5",minH:"4.5",gap:"1","--tag-avatar-size":"spacing.3","--tag-element-size":"spacing.3","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},md:{root:{px:"1.5",minH:"5",gap:"1","--tag-avatar-size":"spacing.3.5","--tag-element-size":"spacing.3.5","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},lg:{root:{px:"2",minH:"6",gap:"1.5","--tag-avatar-size":"spacing.4.5","--tag-element-size":"spacing.4","--tag-element-offset":"-3px"},label:{textStyle:"sm"}},xl:{root:{px:"2.5",minH:"8",gap:"1.5","--tag-avatar-size":"spacing.6","--tag-element-size":"spacing.4.5","--tag-element-offset":"-4px"},label:{textStyle:"sm"}}},variant:{subtle:{root:hi?.subtle},solid:{root:hi?.solid},outline:{root:hi?.outline},surface:{root:hi?.surface}}},defaultVariants:{size:"md",variant:"surface"}}),oE=B({slots:lS.keys(),className:"chakra-timeline",base:{root:{display:"flex",flexDirection:"column",width:"full","--timeline-thickness":"1px","--timeline-gutter":"4px"},item:{display:"flex",position:"relative",alignItems:"flex-start",flexShrink:0,gap:"4",_last:{"& :where(.chakra-timeline__separator)":{display:"none"}}},separator:{position:"absolute",borderStartWidth:"var(--timeline-thickness)",ms:"calc(-1 * var(--timeline-thickness) / 2)",insetInlineStart:"calc(var(--timeline-indicator-size) / 2)",insetBlock:"0",borderColor:"border"},indicator:{outline:"2px solid {colors.bg}",position:"relative",flexShrink:"0",boxSize:"var(--timeline-indicator-size)",fontSize:"var(--timeline-font-size)",display:"flex",alignItems:"center",justifyContent:"center",borderRadius:"full",fontWeight:"medium"},connector:{alignSelf:"stretch",position:"relative"},content:{pb:"6",display:"flex",flexDirection:"column",width:"full",gap:"2"},title:{display:"flex",fontWeight:"medium",flexWrap:"wrap",gap:"1.5",alignItems:"center",mt:"var(--timeline-margin)"},description:{color:"fg.muted",textStyle:"xs"}},variants:{variant:{subtle:{indicator:{bg:"colorPalette.muted"}},solid:{indicator:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},outline:{indicator:{bg:"currentBg",borderWidth:"1px",borderColor:"colorPalette.muted"}},plain:{}},size:{sm:{root:{"--timeline-indicator-size":"sizes.4","--timeline-font-size":"fontSizes.2xs"},title:{textStyle:"xs"}},md:{root:{"--timeline-indicator-size":"sizes.5","--timeline-font-size":"fontSizes.xs"},title:{textStyle:"sm"}},lg:{root:{"--timeline-indicator-size":"sizes.6","--timeline-font-size":"fontSizes.xs"},title:{mt:"0.5",textStyle:"sm"}},xl:{root:{"--timeline-indicator-size":"sizes.8","--timeline-font-size":"fontSizes.sm"},title:{mt:"1.5",textStyle:"sm"}}}},defaultVariants:{size:"md",variant:"solid"}}),iE=B({slots:iS.keys(),className:"chakra-toast",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",gap:"3",py:"4",ps:"4",pe:"6",borderRadius:"l2",translate:"var(--x) var(--y)",scale:"var(--scale)",zIndex:"var(--z-index)",height:"var(--height)",opacity:"var(--opacity)",willChange:"translate, opacity, scale",transition:"translate 400ms, scale 400ms, opacity 400ms, height 400ms, box-shadow 200ms",transitionTimingFunction:"cubic-bezier(0.21, 1.02, 0.73, 1)",_closed:{transition:"translate 400ms, scale 400ms, opacity 200ms",transitionTimingFunction:"cubic-bezier(0.06, 0.71, 0.55, 1)"},bg:"bg.panel",color:"fg",boxShadow:"xl","--toast-trigger-bg":"colors.bg.muted","&[data-type=warning]":{bg:"orange.solid",color:"orange.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=success]":{bg:"green.solid",color:"green.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=error]":{bg:"red.solid",color:"red.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"}},title:{fontWeight:"medium",textStyle:"sm",marginEnd:"2"},description:{display:"inline",textStyle:"sm",opacity:"0.8"},indicator:{flexShrink:"0",boxSize:"5"},actionTrigger:{textStyle:"sm",fontWeight:"medium",height:"8",px:"3",borderRadius:"l2",alignSelf:"center",borderWidth:"1px",borderColor:"var(--toast-border-color, inherit)",transition:"background 200ms",_hover:{bg:"var(--toast-trigger-bg)"}},closeTrigger:{position:"absolute",top:"1",insetEnd:"1",padding:"1",display:"inline-flex",alignItems:"center",justifyContent:"center",color:"{currentColor/60}",borderRadius:"l2",textStyle:"md",transition:"background 200ms",_icon:{boxSize:"1em"}}}}),sE=B({slots:Kd.keys(),className:"chakra-tooltip",base:{content:{"--tooltip-bg":"colors.bg.inverted",bg:"var(--tooltip-bg)",color:"fg.inverted",px:"2.5",py:"1",borderRadius:"l2",fontWeight:"medium",textStyle:"xs",boxShadow:"md",maxW:"xs",zIndex:"tooltip",transformOrigin:"var(--transform-origin)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"fast"}},arrow:{"--arrow-size":"sizes.2","--arrow-background":"var(--tooltip-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px",borderColor:"var(--tooltip-bg)"}}}),qh=tr({display:"flex",alignItems:"center",gap:"var(--tree-item-gap)",rounded:"l2",userSelect:"none",position:"relative","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-icon-offset":"calc(var(--tree-icon-size) * var(--tree-depth) * 0.5)","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset) + var(--tree-icon-offset))",ps:"var(--tree-offset)",pe:"var(--tree-padding-inline)",py:"var(--tree-padding-block)",focusVisibleRing:"inside",focusRingColor:"border.emphasized",focusRingWidth:"2px","&:hover, &:focus-visible":{bg:"bg.muted"},_disabled:{layerStyle:"disabled"}}),Kh=tr({flex:"1"}),Yh=tr({_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}),Xh=tr({_selected:{layerStyle:"fill.solid"}}),aE=B({slots:vd.keys(),className:"chakra-tree-view",base:{root:{width:"full",display:"flex",flexDirection:"column",gap:"2"},tree:{display:"flex",flexDirection:"column","--tree-item-gap":"spacing.2",_icon:{boxSize:"var(--tree-icon-size)"}},label:{fontWeight:"medium",textStyle:"sm"},branch:{position:"relative"},branchContent:{position:"relative"},branchIndentGuide:{height:"100%",width:"1px",bg:"border",position:"absolute","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset))","--tree-icon-offset":"calc(var(--tree-icon-size) * 0.5 * var(--depth))",insetInlineStart:"calc(var(--tree-offset) + var(--tree-icon-offset))",zIndex:"1"},branchIndicator:{color:"fg.muted",transformOrigin:"center",transitionDuration:"normal",transitionProperty:"transform",transitionTimingFunction:"default",_open:{transform:"rotate(90deg)"}},branchTrigger:{display:"inline-flex",alignItems:"center",justifyContent:"center"},branchControl:qh,item:qh,itemText:Kh,branchText:Kh,nodeCheckbox:{display:"inline-flex"}},variants:{size:{md:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1.5","--tree-icon-size":"spacing.4"}},sm:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}},xs:{tree:{textStyle:"xs","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.2","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}}},variant:{subtle:{branchControl:Yh,item:Yh},solid:{branchControl:Xh,item:Xh}},animateContent:{true:{branchContent:{_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}},defaultVariants:{size:"md",variant:"subtle"}}),lE={accordion:hS,actionBar:fS,alert:gS,avatar:pS,blockquote:mS,breadcrumb:vS,card:bS,checkbox:yS,checkboxCard:xS,codeBlock:kS,collapsible:CS,dataList:ES,dialog:OS,drawer:PS,editable:RS,emptyState:IS,field:TS,fieldset:NS,fileUpload:AS,hoverCard:_S,list:VS,listbox:FS,menu:LS,nativeSelect:DS,numberInput:zS,pinInput:jS,popover:$S,progress:BS,progressCircle:WS,radioCard:US,radioGroup:GS,ratingGroup:qS,scrollArea:KS,segmentGroup:YS,select:di,combobox:SS,slider:XS,stat:QS,steps:ZS,switch:eE,table:tE,tabs:nE,tag:rE,toast:iE,tooltip:sE,status:JS,timeline:oE,colorPicker:wS,qrCode:HS,treeView:aE},cE=Hk({"2xs":{value:{fontSize:"2xs",lineHeight:"0.75rem"}},xs:{value:{fontSize:"xs",lineHeight:"1rem"}},sm:{value:{fontSize:"sm",lineHeight:"1.25rem"}},md:{value:{fontSize:"md",lineHeight:"1.5rem"}},lg:{value:{fontSize:"lg",lineHeight:"1.75rem"}},xl:{value:{fontSize:"xl",lineHeight:"1.875rem"}},"2xl":{value:{fontSize:"2xl",lineHeight:"2rem"}},"3xl":{value:{fontSize:"3xl",lineHeight:"2.375rem"}},"4xl":{value:{fontSize:"4xl",lineHeight:"2.75rem",letterSpacing:"-0.025em"}},"5xl":{value:{fontSize:"5xl",lineHeight:"3.75rem",letterSpacing:"-0.025em"}},"6xl":{value:{fontSize:"6xl",lineHeight:"4.5rem",letterSpacing:"-0.025em"}},"7xl":{value:{fontSize:"7xl",lineHeight:"5.75rem",letterSpacing:"-0.025em"}},none:{value:{}},label:{value:{fontSize:"sm",lineHeight:"1.25rem",fontWeight:"medium"}}}),uE=de.animations({spin:{value:"spin 1s linear infinite"},ping:{value:"ping 1s cubic-bezier(0, 0, 0.2, 1) infinite"},pulse:{value:"pulse 2s cubic-bezier(0.4, 0, 0.6, 1) infinite"},bounce:{value:"bounce 1s infinite"}}),dE=de.aspectRatios({square:{value:"1 / 1"},landscape:{value:"4 / 3"},portrait:{value:"3 / 4"},wide:{value:"16 / 9"},ultrawide:{value:"18 / 5"},golden:{value:"1.618 / 1"}}),hE=de.blurs({none:{value:" "},sm:{value:"4px"},md:{value:"8px"},lg:{value:"12px"},xl:{value:"16px"},"2xl":{value:"24px"},"3xl":{value:"40px"},"4xl":{value:"64px"}}),fE=de.borders({xs:{value:"0.5px solid"},sm:{value:"1px solid"},md:{value:"2px solid"},lg:{value:"4px solid"},xl:{value:"8px solid"}}),gE=de.colors({transparent:{value:"transparent"},current:{value:"currentColor"},black:{value:"#09090B"},white:{value:"#FFFFFF"},whiteAlpha:{50:{value:"rgba(255, 255, 255, 0.04)"},100:{value:"rgba(255, 255, 255, 0.06)"},200:{value:"rgba(255, 255, 255, 0.08)"},300:{value:"rgba(255, 255, 255, 0.16)"},400:{value:"rgba(255, 255, 255, 0.24)"},500:{value:"rgba(255, 255, 255, 0.36)"},600:{value:"rgba(255, 255, 255, 0.48)"},700:{value:"rgba(255, 255, 255, 0.64)"},800:{value:"rgba(255, 255, 255, 0.80)"},900:{value:"rgba(255, 255, 255, 0.92)"},950:{value:"rgba(255, 255, 255, 0.95)"}},blackAlpha:{50:{value:"rgba(0, 0, 0, 0.04)"},100:{value:"rgba(0, 0, 0, 0.06)"},200:{value:"rgba(0, 0, 0, 0.08)"},300:{value:"rgba(0, 0, 0, 0.16)"},400:{value:"rgba(0, 0, 0, 0.24)"},500:{value:"rgba(0, 0, 0, 0.36)"},600:{value:"rgba(0, 0, 0, 0.48)"},700:{value:"rgba(0, 0, 0, 0.64)"},800:{value:"rgba(0, 0, 0, 0.80)"},900:{value:"rgba(0, 0, 0, 0.92)"},950:{value:"rgba(0, 0, 0, 0.95)"}},gray:{50:{value:"#fafafa"},100:{value:"#f4f4f5"},200:{value:"#e4e4e7"},300:{value:"#d4d4d8"},400:{value:"#a1a1aa"},500:{value:"#71717a"},600:{value:"#52525b"},700:{value:"#3f3f46"},800:{value:"#27272a"},900:{value:"#18181b"},950:{value:"#111111"}},red:{50:{value:"#fef2f2"},100:{value:"#fee2e2"},200:{value:"#fecaca"},300:{value:"#fca5a5"},400:{value:"#f87171"},500:{value:"#ef4444"},600:{value:"#dc2626"},700:{value:"#991919"},800:{value:"#511111"},900:{value:"#300c0c"},950:{value:"#1f0808"}},orange:{50:{value:"#fff7ed"},100:{value:"#ffedd5"},200:{value:"#fed7aa"},300:{value:"#fdba74"},400:{value:"#fb923c"},500:{value:"#f97316"},600:{value:"#ea580c"},700:{value:"#92310a"},800:{value:"#6c2710"},900:{value:"#3b1106"},950:{value:"#220a04"}},yellow:{50:{value:"#fefce8"},100:{value:"#fef9c3"},200:{value:"#fef08a"},300:{value:"#fde047"},400:{value:"#facc15"},500:{value:"#eab308"},600:{value:"#ca8a04"},700:{value:"#845209"},800:{value:"#713f12"},900:{value:"#422006"},950:{value:"#281304"}},green:{50:{value:"#f0fdf4"},100:{value:"#dcfce7"},200:{value:"#bbf7d0"},300:{value:"#86efac"},400:{value:"#4ade80"},500:{value:"#22c55e"},600:{value:"#16a34a"},700:{value:"#116932"},800:{value:"#124a28"},900:{value:"#042713"},950:{value:"#03190c"}},teal:{50:{value:"#f0fdfa"},100:{value:"#ccfbf1"},200:{value:"#99f6e4"},300:{value:"#5eead4"},400:{value:"#2dd4bf"},500:{value:"#14b8a6"},600:{value:"#0d9488"},700:{value:"#0c5d56"},800:{value:"#114240"},900:{value:"#032726"},950:{value:"#021716"}},blue:{50:{value:"#eff6ff"},100:{value:"#dbeafe"},200:{value:"#bfdbfe"},300:{value:"#a3cfff"},400:{value:"#60a5fa"},500:{value:"#3b82f6"},600:{value:"#2563eb"},700:{value:"#173da6"},800:{value:"#1a3478"},900:{value:"#14204a"},950:{value:"#0c142e"}},cyan:{50:{value:"#ecfeff"},100:{value:"#cffafe"},200:{value:"#a5f3fc"},300:{value:"#67e8f9"},400:{value:"#22d3ee"},500:{value:"#06b6d4"},600:{value:"#0891b2"},700:{value:"#0c5c72"},800:{value:"#134152"},900:{value:"#072a38"},950:{value:"#051b24"}},purple:{50:{value:"#faf5ff"},100:{value:"#f3e8ff"},200:{value:"#e9d5ff"},300:{value:"#d8b4fe"},400:{value:"#c084fc"},500:{value:"#a855f7"},600:{value:"#9333ea"},700:{value:"#641ba3"},800:{value:"#4a1772"},900:{value:"#2f0553"},950:{value:"#1a032e"}},pink:{50:{value:"#fdf2f8"},100:{value:"#fce7f3"},200:{value:"#fbcfe8"},300:{value:"#f9a8d4"},400:{value:"#f472b6"},500:{value:"#ec4899"},600:{value:"#db2777"},700:{value:"#a41752"},800:{value:"#6d0e34"},900:{value:"#45061f"},950:{value:"#2c0514"}}}),pE=de.cursor({button:{value:"pointer"},checkbox:{value:"default"},disabled:{value:"not-allowed"},menuitem:{value:"default"},option:{value:"default"},radio:{value:"default"},slider:{value:"default"},switch:{value:"pointer"}}),mE=de.durations({fastest:{value:"50ms"},faster:{value:"100ms"},fast:{value:"150ms"},moderate:{value:"200ms"},slow:{value:"300ms"},slower:{value:"400ms"},slowest:{value:"500ms"}}),vE=de.easings({"ease-in":{value:"cubic-bezier(0.42, 0, 1, 1)"},"ease-out":{value:"cubic-bezier(0, 0, 0.58, 1)"},"ease-in-out":{value:"cubic-bezier(0.42, 0, 0.58, 1)"},"ease-in-smooth":{value:"cubic-bezier(0.32, 0.72, 0, 1)"}}),bE=de.fontSizes({"2xs":{value:"0.625rem"},xs:{value:"0.75rem"},sm:{value:"0.875rem"},md:{value:"1rem"},lg:{value:"1.125rem"},xl:{value:"1.25rem"},"2xl":{value:"1.5rem"},"3xl":{value:"1.875rem"},"4xl":{value:"2.25rem"},"5xl":{value:"3rem"},"6xl":{value:"3.75rem"},"7xl":{value:"4.5rem"},"8xl":{value:"6rem"},"9xl":{value:"8rem"}}),yE=de.fontWeights({thin:{value:"100"},extralight:{value:"200"},light:{value:"300"},normal:{value:"400"},medium:{value:"500"},semibold:{value:"600"},bold:{value:"700"},extrabold:{value:"800"},black:{value:"900"}}),Qh='-apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol"',xE=de.fonts({heading:{value:`Inter, ${Qh}`},body:{value:`Inter, ${Qh}`},mono:{value:'SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace'}}),kE=Bk({spin:{"0%":{transform:"rotate(0deg)"},"100%":{transform:"rotate(360deg)"}},pulse:{"50%":{opacity:"0.5"}},ping:{"75%, 100%":{transform:"scale(2)",opacity:"0"}},bounce:{"0%, 100%":{transform:"translateY(-25%)",animationTimingFunction:"cubic-bezier(0.8,0,1,1)"},"50%":{transform:"none",animationTimingFunction:"cubic-bezier(0,0,0.2,1)"}},"bg-position":{from:{backgroundPosition:"var(--animate-from, 1rem) 0"},to:{backgroundPosition:"var(--animate-to, 0) 0"}},position:{from:{insetInlineStart:"var(--animate-from-x)",insetBlockStart:"var(--animate-from-y)"},to:{insetInlineStart:"var(--animate-to-x)",insetBlockStart:"var(--animate-to-y)"}},"circular-progress":{"0%":{strokeDasharray:"1, 400",strokeDashoffset:"0"},"50%":{strokeDasharray:"400, 400",strokeDashoffset:"-100%"},"100%":{strokeDasharray:"400, 400",strokeDashoffset:"-260%"}},"expand-height":{from:{height:"0"},to:{height:"var(--height)"}},"collapse-height":{from:{height:"var(--height)"},to:{height:"0"}},"expand-width":{from:{width:"0"},to:{width:"var(--width)"}},"collapse-width":{from:{height:"var(--width)"},to:{height:"0"}},"fade-in":{from:{opacity:0},to:{opacity:1}},"fade-out":{from:{opacity:1},to:{opacity:0}},"slide-from-left-full":{from:{translate:"-100% 0"},to:{translate:"0 0"}},"slide-from-right-full":{from:{translate:"100% 0"},to:{translate:"0 0"}},"slide-from-top-full":{from:{translate:"0 -100%"},to:{translate:"0 0"}},"slide-from-bottom-full":{from:{translate:"0 100%"},to:{translate:"0 0"}},"slide-to-left-full":{from:{translate:"0 0"},to:{translate:"-100% 0"}},"slide-to-right-full":{from:{translate:"0 0"},to:{translate:"100% 0"}},"slide-to-top-full":{from:{translate:"0 0"},to:{translate:"0 -100%"}},"slide-to-bottom-full":{from:{translate:"0 0"},to:{translate:"0 100%"}},"slide-from-top":{"0%":{translate:"0 -0.5rem"},to:{translate:"0"}},"slide-from-bottom":{"0%":{translate:"0 0.5rem"},to:{translate:"0"}},"slide-from-left":{"0%":{translate:"-0.5rem 0"},to:{translate:"0"}},"slide-from-right":{"0%":{translate:"0.5rem 0"},to:{translate:"0"}},"slide-to-top":{"0%":{translate:"0"},to:{translate:"0 -0.5rem"}},"slide-to-bottom":{"0%":{translate:"0"},to:{translate:"0 0.5rem"}},"slide-to-left":{"0%":{translate:"0"},to:{translate:"-0.5rem 0"}},"slide-to-right":{"0%":{translate:"0"},to:{translate:"0.5rem 0"}},"scale-in":{from:{scale:"0.95"},to:{scale:"1"}},"scale-out":{from:{scale:"1"},to:{scale:"0.95"}}}),CE=de.letterSpacings({tighter:{value:"-0.05em"},tight:{value:"-0.025em"},wide:{value:"0.025em"},wider:{value:"0.05em"},widest:{value:"0.1em"}}),wE=de.lineHeights({shorter:{value:1.25},short:{value:1.375},moderate:{value:1.5},tall:{value:1.625},taller:{value:2}}),SE=de.radii({none:{value:"0"},"2xs":{value:"0.0625rem"},xs:{value:"0.125rem"},sm:{value:"0.25rem"},md:{value:"0.375rem"},lg:{value:"0.5rem"},xl:{value:"0.75rem"},"2xl":{value:"1rem"},"3xl":{value:"1.5rem"},"4xl":{value:"2rem"},full:{value:"9999px"}}),Jh=de.spacing({.5:{value:"0.125rem"},1:{value:"0.25rem"},1.5:{value:"0.375rem"},2:{value:"0.5rem"},2.5:{value:"0.625rem"},3:{value:"0.75rem"},3.5:{value:"0.875rem"},4:{value:"1rem"},4.5:{value:"1.125rem"},5:{value:"1.25rem"},6:{value:"1.5rem"},7:{value:"1.75rem"},8:{value:"2rem"},9:{value:"2.25rem"},10:{value:"2.5rem"},11:{value:"2.75rem"},12:{value:"3rem"},14:{value:"3.5rem"},16:{value:"4rem"},20:{value:"5rem"},24:{value:"6rem"},28:{value:"7rem"},32:{value:"8rem"},36:{value:"9rem"},40:{value:"10rem"},44:{value:"11rem"},48:{value:"12rem"},52:{value:"13rem"},56:{value:"14rem"},60:{value:"15rem"},64:{value:"16rem"},72:{value:"18rem"},80:{value:"20rem"},96:{value:"24rem"}}),EE=de.sizes({"3xs":{value:"14rem"},"2xs":{value:"16rem"},xs:{value:"20rem"},sm:{value:"24rem"},md:{value:"28rem"},lg:{value:"32rem"},xl:{value:"36rem"},"2xl":{value:"42rem"},"3xl":{value:"48rem"},"4xl":{value:"56rem"},"5xl":{value:"64rem"},"6xl":{value:"72rem"},"7xl":{value:"80rem"},"8xl":{value:"90rem"}}),OE=de.sizes({max:{value:"max-content"},min:{value:"min-content"},fit:{value:"fit-content"},prose:{value:"60ch"},full:{value:"100%"},dvh:{value:"100dvh"},svh:{value:"100svh"},lvh:{value:"100lvh"},dvw:{value:"100dvw"},svw:{value:"100svw"},lvw:{value:"100lvw"},vw:{value:"100vw"},vh:{value:"100vh"}}),PE=de.sizes({"1/2":{value:"50%"},"1/3":{value:"33.333333%"},"2/3":{value:"66.666667%"},"1/4":{value:"25%"},"3/4":{value:"75%"},"1/5":{value:"20%"},"2/5":{value:"40%"},"3/5":{value:"60%"},"4/5":{value:"80%"},"1/6":{value:"16.666667%"},"2/6":{value:"33.333333%"},"3/6":{value:"50%"},"4/6":{value:"66.666667%"},"5/6":{value:"83.333333%"},"1/12":{value:"8.333333%"},"2/12":{value:"16.666667%"},"3/12":{value:"25%"},"4/12":{value:"33.333333%"},"5/12":{value:"41.666667%"},"6/12":{value:"50%"},"7/12":{value:"58.333333%"},"8/12":{value:"66.666667%"},"9/12":{value:"75%"},"10/12":{value:"83.333333%"},"11/12":{value:"91.666667%"}}),RE=de.sizes({...EE,...Jh,...PE,...OE}),IE=de.zIndex({hide:{value:-1},base:{value:0},docked:{value:10},dropdown:{value:1e3},sticky:{value:1100},banner:{value:1200},overlay:{value:1300},modal:{value:1400},popover:{value:1500},skipNav:{value:1600},toast:{value:1700},tooltip:{value:1800},max:{value:2147483647}}),TE=ka({preflight:!0,cssVarsPrefix:"chakra",cssVarsRoot:":where(html, .chakra-theme)",globalCss:aw,theme:{breakpoints:sw,keyframes:kE,tokens:{aspectRatios:dE,animations:uE,blurs:hE,borders:fE,colors:gE,durations:mE,easings:vE,fonts:xE,fontSizes:bE,fontWeights:yE,letterSpacings:CE,lineHeights:wE,radii:SE,spacing:Jh,sizes:RE,zIndex:IE,cursor:pE},semanticTokens:{colors:Pw,shadows:Iw,radii:Rw},recipes:Ow,slotRecipes:lE,textStyles:cE,layerStyles:lw,animationStyles:cw}}),Zh=Zd(eC,TE);zh(Zh);function NE(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getSlotRecipe(t):{});return r.sva(structuredClone(o))},[t,n,r])}const AE=e=>e.charAt(0).toUpperCase()+e.slice(1),fi=e=>{const{key:t,recipe:n}=e,r=AE(t||n.className||"Component"),[o,i]=Fn({name:`${r}StylesContext`,errorMessage:`use${r}Styles returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `}),[s,a]=Fn({name:`${r}ClassNameContext`,errorMessage:`use${r}ClassNames returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `,strict:!1}),[l,c]=Fn({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`,defaultValue:{}});function u(f){const{unstyled:p,...v}=f,b=NE({key:t,recipe:v.recipe||n}),[y,x]=P.useMemo(()=>b.splitVariantProps(v),[v,b]);return{styles:P.useMemo(()=>p?$m:b(y),[p,y,b]),classNames:b.classNameMap,props:x}}function h(f,p={}){const{defaultProps:v}=p,b=y=>{const x=c(),w=P.useMemo(()=>br(v,x,y),[x,y]),{styles:k,classNames:O,props:R}=u(w);return g.jsx(o,{value:k,children:g.jsx(s,{value:O,children:g.jsx(f,{...R})})})};return b.displayName=f.displayName||f.name,b}return{StylesProvider:o,ClassNamesProvider:s,PropsProvider:l,usePropsContext:c,useRecipeResult:u,withProvider:(f,p,v)=>{const{defaultProps:b,...y}=v??{},x=ve(f,{},y),w=P.forwardRef((k,O)=>{const R=c(),I=P.useMemo(()=>br(b??{},R,k),[R,k]),{styles:N,props:A,classNames:T}=u(I),S=T[p],_=g.jsx(o,{value:N,children:g.jsx(s,{value:T,children:g.jsx(x,{ref:O,...A,css:[N[p],I.css],className:Ge(I.className,S)})})});return v?.wrapElement?.(_,I)??_});return w.displayName=f.displayName||f.name,w},withContext:(f,p,v)=>{const b=ve(f,{},v),y=P.forwardRef((x,w)=>{const{unstyled:k,...O}=x,R=i(),N=a()?.[p];return g.jsx(b,{...O,css:[!k&&p?R[p]:void 0,x.css],ref:w,className:Ge(x.className,N)})});return y.displayName=f.displayName||f.name,y},withRootProvider:h,useStyles:i,useClassNames:a}},ef=ve("div",{base:{position:"absolute",display:"flex",alignItems:"center",justifyContent:"center"},variants:{axis:{horizontal:{insetStart:"50%",translate:"-50%",_rtl:{translate:"50%"}},vertical:{top:"50%",translate:"0 -50%"},both:{insetStart:"50%",top:"50%",translate:"-50% -50%",_rtl:{translate:"50% -50%"}}}},defaultVariants:{axis:"both"}});ef.displayName="AbsoluteCenter";const _E=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11.0026 16L6.75999 11.7574L8.17421 10.3431L11.0026 13.1716L16.6595 7.51472L18.0737 8.92893L11.0026 16Z"})}),tf=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 15H13V17H11V15ZM11 7H13V13H11V7Z"})}),nf=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",stroke:"currentColor",strokeWidth:"0",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 7H13V9H11V7ZM11 11H13V17H11V11Z"})}),VE=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",...e,children:g.jsx("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M18.7071 6.70711C19.0976 6.31658 19.0976 5.68342 18.7071 5.29289C18.3166 4.90237 17.6834 4.90237 17.2929 5.29289L12 10.5858L6.70711 5.29289C6.31658 4.90237 5.68342 4.90237 5.29289 5.29289C4.90237 5.68342 4.90237 6.31658 5.29289 6.70711L10.5858 12L5.29289 17.2929C4.90237 17.6834 4.90237 18.3166 5.29289 18.7071C5.68342 19.0976 6.31658 19.0976 6.70711 18.7071L12 13.4142L17.2929 18.7071C17.6834 19.0976 18.3166 19.0976 18.7071 18.7071C19.0976 18.3166 19.0976 17.6834 18.7071 17.2929L13.4142 12L18.7071 6.70711Z"})}),[FE,LE]=Fn({name:"AlertStatusContext",hookName:"useAlertStatusContext",providerName:""}),{withProvider:DE,withContext:Na,useStyles:zE}=fi({key:"alert"}),ME=DE("div","root",{forwardAsChild:!0,wrapElement(e,t){return g.jsx(FE,{value:{status:t.status||"info"},children:e})}}),rf=Na("div","title"),jE=Na("div","description"),$E=Na("div","content"),BE={info:nf,warning:tf,success:_E,error:tf,neutral:nf},WE=P.forwardRef(function(t,n){const r=LE(),o=zE(),i=typeof r.status=="string"?BE[r.status]:P.Fragment,{children:s=g.jsx(i,{}),...a}=t;return g.jsx(ve.span,{ref:n,...a,css:[o.indicator,t.css],children:s})}),HE=e=>e?"":void 0,{withContext:UE}=Lt({key:"badge"}),of=UE("span"),{withContext:GE}=Lt({key:"spinner"}),qE=GE("span"),KE=C.forwardRef(function(t,n){const{spinner:r=g.jsx(qE,{size:"inherit",borderWidth:"0.125em",color:"inherit"}),spinnerPlacement:o="start",children:i,text:s,visible:a=!0,...l}=t;return a?s?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[o==="start"&&r,s,o==="end"&&r]}):r?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[g.jsx(ef,{display:"inline-flex",children:r}),g.jsx(Vo,{visibility:"hidden",display:"contents",children:i})]}):g.jsx(Vo,{ref:n,display:"contents",...l,children:i}):i}),{useRecipeResult:YE,usePropsContext:XE}=Lt({key:"button"}),Re=P.forwardRef(function(t,n){const r=XE(),o=P.useMemo(()=>br(r,t),[r,t]),i=YE(o),{loading:s,loadingText:a,children:l,spinner:c,spinnerPlacement:u,...h}=i.props;return g.jsx(ve.button,{type:"button",ref:n,...h,"data-loading":HE(s),disabled:s||h.disabled,className:Ge(i.className,o.className),css:[i.styles,o.css],children:!o.asChild&&s?g.jsx(KE,{spinner:c,text:a,spinnerPlacement:u,children:l}):l})}),Et=P.forwardRef(function(t,n){return g.jsx(Re,{px:"0",py:"0",_icon:{fontSize:"1.2em"},ref:n,...t})}),fn=C.forwardRef(function(t,n){return g.jsx(Et,{variant:"ghost","aria-label":"Close",ref:n,...t,children:t.children??g.jsx(VE,{})})}),sf=ve("div",{base:{display:"flex",alignItems:"center",justifyContent:"center"},variants:{inline:{true:{display:"inline-flex"}}}});sf.displayName="Center";const{withContext:QE}=Lt({key:"code"}),JE=QE("code");function ZE(e){const{gap:t,direction:n}=e,r={column:{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},"column-reverse":{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},row:{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0},"row-reverse":{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0}};return{"&":Jd(n,o=>r[o])}}function e2(e){return P.Children.toArray(e).filter(t=>P.isValidElement(t))}const af=P.forwardRef(function(t,n){const{direction:r="column",align:o,justify:i,gap:s="0.5rem",wrap:a,children:l,separator:c,className:u,...h}=t,d=P.useMemo(()=>ZE({gap:s,direction:r}),[s,r]),m=P.useMemo(()=>P.isValidElement(c)?e2(l).map((f,p,v)=>{const b=typeof f.key<"u"?f.key:p,y=c,x=P.cloneElement(y,{css:[d,y.props.css]});return g.jsxs(P.Fragment,{children:[f,p===v.length-1?null:x]},b)}):l,[l,c,d]);return g.jsx(ve.div,{ref:n,display:"flex",alignItems:o,justifyContent:i,flexDirection:r,flexWrap:a,gap:c?void 0:s,className:Ge("chakra-stack",u),...h,children:m})}),{withRootProvider:lf,withContext:pt}=fi({key:"dialog"});lf(O1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});const gn=lf(E1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});pt(ad,"trigger",{forwardAsChild:!0});const pn=pt(Zu,"positioner",{forwardAsChild:!0}),mn=pt(Qu,"content",{forwardAsChild:!0});pt(Ju,"description",{forwardAsChild:!0});const vn=pt(sd,"title",{forwardAsChild:!0}),bn=pt(Xu,"closeTrigger",{forwardAsChild:!0}),yn=P.forwardRef(function(t,n){const r=Ht();return g.jsx(ve.button,{...t,ref:n,onClick:()=>r.setOpen(!1)})}),xn=pt(Yu,"backdrop",{forwardAsChild:!0}),kn=pt("div","body"),Cn=pt("div","footer"),wn=pt("div","header");function t2(e){const{each:t,fallback:n,children:r}=e;return t?.length===0?n||null:t?.map(r)}const gi=P.forwardRef(function(t,n){const{direction:r,align:o,justify:i,wrap:s,basis:a,grow:l,shrink:c,inline:u,...h}=t;return g.jsx(ve.div,{ref:n,...h,css:{display:u?"inline-flex":"flex",flexDirection:r,alignItems:o,justifyContent:i,flexWrap:s,flexBasis:a,flexGrow:l,flexShrink:c,...t.css}})}),{withContext:n2}=Lt({key:"input"}),r2=n2(cd),{withContext:o2}=Lt({key:"link"}),Br=o2("a"),{withProvider:i2,withContext:cf}=fi({key:"list"}),uf=i2("ul","root",{defaultProps:{role:"list"}}),df=cf("li","item");cf("span","indicator");const s2=P.forwardRef(function(t,n){return g.jsx(af,{align:"center",...t,direction:"row",ref:n})}),pi=P.forwardRef(function(t,n){return g.jsx(af,{align:"center",...t,direction:"column",ref:n})}),{StylesProvider:a2,ClassNamesProvider:l2,useRecipeResult:c2,withContext:Ot}=fi({key:"table"}),hf=P.forwardRef(function({native:t,...n},r){const{styles:o,props:i,classNames:s}=c2(n),a=P.useMemo(()=>t?{...o.root,"& thead":o.header,"& tbody":o.body,"& tfoot":o.footer,"& thead th":o.columnHeader,"& tr":o.row,"& td":o.cell,"& caption":o.caption}:o.root,[o,t]);return g.jsx(l2,{value:s,children:g.jsx(a2,{value:o,children:g.jsx(ve.table,{ref:r,...i,css:[a,n.css],className:Ge(s?.root,n.className)})})})}),mi=Ot("tr","row");ve("div",{base:{display:"block",whiteSpace:"nowrap",WebkitOverflowScrolling:"touch",overflow:"auto",maxWidth:"100%"}});const ff=Ot("thead","header");Ot("tfoot","footer");const ke=Ot("th","columnHeader"),Ce=Ot("td","cell");Ot("caption","caption",{defaultProps:{captionSide:"bottom"}});const gf=Ot("tbody","body");Ot("colgroup"),Ot("col");const{withContext:u2}=Lt({key:"textarea"}),pf=u2(dd);var ir=class{constructor(){this.listeners=new Set,this.subscribe=this.subscribe.bind(this)}subscribe(e){return this.listeners.add(e),this.onSubscribe(),()=>{this.listeners.delete(e),this.onUnsubscribe()}}hasListeners(){return this.listeners.size>0}onSubscribe(){}onUnsubscribe(){}},d2={setTimeout:(e,t)=>setTimeout(e,t),clearTimeout:e=>clearTimeout(e),setInterval:(e,t)=>setInterval(e,t),clearInterval:e=>clearInterval(e)},h2=class{#e=d2;#t=!1;setTimeoutProvider(e){this.#e=e}setTimeout(e,t){return this.#e.setTimeout(e,t)}clearTimeout(e){this.#e.clearTimeout(e)}setInterval(e,t){return this.#e.setInterval(e,t)}clearInterval(e){this.#e.clearInterval(e)}},Sn=new h2;function f2(e){setTimeout(e,0)}var En=typeof window>"u"||"Deno"in globalThis;function Ne(){}function g2(e,t){return typeof e=="function"?e(t):e}function Aa(e){return typeof e=="number"&&e>=0&&e!==1/0}function mf(e,t){return Math.max(e+(t||0)-Date.now(),0)}function qt(e,t){return typeof e=="function"?e(t):e}function Qe(e,t){return typeof e=="function"?e(t):e}function vf(e,t){const{type:n="all",exact:r,fetchStatus:o,predicate:i,queryKey:s,stale:a}=e;if(s){if(r){if(t.queryHash!==_a(s,t.options))return!1}else if(!Wr(t.queryKey,s))return!1}if(n!=="all"){const l=t.isActive();if(n==="active"&&!l||n==="inactive"&&l)return!1}return!(typeof a=="boolean"&&t.isStale()!==a||o&&o!==t.state.fetchStatus||i&&!i(t))}function bf(e,t){const{exact:n,status:r,predicate:o,mutationKey:i}=e;if(i){if(!t.options.mutationKey)return!1;if(n){if(On(t.options.mutationKey)!==On(i))return!1}else if(!Wr(t.options.mutationKey,i))return!1}return!(r&&t.state.status!==r||o&&!o(t))}function _a(e,t){return(t?.queryKeyHashFn||On)(e)}function On(e){return JSON.stringify(e,(t,n)=>Va(n)?Object.keys(n).sort().reduce((r,o)=>(r[o]=n[o],r),{}):n)}function Wr(e,t){return e===t?!0:typeof e!=typeof t?!1:e&&t&&typeof e=="object"&&typeof t=="object"?Object.keys(t).every(n=>Wr(e[n],t[n])):!1}var p2=Object.prototype.hasOwnProperty;function yf(e,t){if(e===t)return e;const n=xf(e)&&xf(t);if(!n&&!(Va(e)&&Va(t)))return t;const o=(n?e:Object.keys(e)).length,i=n?t:Object.keys(t),s=i.length,a=n?new Array(s):{};let l=0;for(let c=0;c{Sn.setTimeout(t,e)})}function Fa(e,t,n){return typeof n.structuralSharing=="function"?n.structuralSharing(e,t):n.structuralSharing!==!1?yf(e,t):t}function v2(e,t,n=0){const r=[...e,t];return n&&r.length>n?r.slice(1):r}function b2(e,t,n=0){const r=[t,...e];return n&&r.length>n?r.slice(0,-1):r}var La=Symbol();function Cf(e,t){return!e.queryFn&&t?.initialPromise?()=>t.initialPromise:!e.queryFn||e.queryFn===La?()=>Promise.reject(new Error(`Missing queryFn: '${e.queryHash}'`)):e.queryFn}function wf(e,t){return typeof e=="function"?e(...t):!!e}var y2=class extends ir{#e;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e();return window.addEventListener("visibilitychange",t,!1),()=>{window.removeEventListener("visibilitychange",t)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(t=>{typeof t=="boolean"?this.setFocused(t):this.onFocus()})}setFocused(e){this.#e!==e&&(this.#e=e,this.onFocus())}onFocus(){const e=this.isFocused();this.listeners.forEach(t=>{t(e)})}isFocused(){return typeof this.#e=="boolean"?this.#e:globalThis.document?.visibilityState!=="hidden"}},Da=new y2;function za(){let e,t;const n=new Promise((o,i)=>{e=o,t=i});n.status="pending",n.catch(()=>{});function r(o){Object.assign(n,o),delete n.resolve,delete n.reject}return n.resolve=o=>{r({status:"fulfilled",value:o}),e(o)},n.reject=o=>{r({status:"rejected",reason:o}),t(o)},n}var x2=f2;function k2(){let e=[],t=0,n=a=>{a()},r=a=>{a()},o=x2;const i=a=>{t?e.push(a):o(()=>{n(a)})},s=()=>{const a=e;e=[],a.length&&o(()=>{r(()=>{a.forEach(l=>{n(l)})})})};return{batch:a=>{let l;t++;try{l=a()}finally{t--,t||s()}return l},batchCalls:a=>(...l)=>{i(()=>{a(...l)})},schedule:i,setNotifyFunction:a=>{n=a},setBatchNotifyFunction:a=>{r=a},setScheduler:a=>{o=a}}}var ge=k2(),C2=class extends ir{#e=!0;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e(!0),n=()=>e(!1);return window.addEventListener("online",t,!1),window.addEventListener("offline",n,!1),()=>{window.removeEventListener("online",t),window.removeEventListener("offline",n)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(this.setOnline.bind(this))}setOnline(e){this.#e!==e&&(this.#e=e,this.listeners.forEach(n=>{n(e)}))}isOnline(){return this.#e}},bi=new C2;function w2(e){return Math.min(1e3*2**e,3e4)}function Sf(e){return(e??"online")==="online"?bi.isOnline():!0}var Ma=class extends Error{constructor(e){super("CancelledError"),this.revert=e?.revert,this.silent=e?.silent}};function Ef(e){let t=!1,n=0,r;const o=za(),i=()=>o.status!=="pending",s=p=>{if(!i()){const v=new Ma(p);d(v),e.onCancel?.(v)}},a=()=>{t=!0},l=()=>{t=!1},c=()=>Da.isFocused()&&(e.networkMode==="always"||bi.isOnline())&&e.canRun(),u=()=>Sf(e.networkMode)&&e.canRun(),h=p=>{i()||(r?.(),o.resolve(p))},d=p=>{i()||(r?.(),o.reject(p))},m=()=>new Promise(p=>{r=v=>{(i()||c())&&p(v)},e.onPause?.()}).then(()=>{r=void 0,i()||e.onContinue?.()}),f=()=>{if(i())return;let p;const v=n===0?e.initialPromise:void 0;try{p=v??e.fn()}catch(b){p=Promise.reject(b)}Promise.resolve(p).then(h).catch(b=>{if(i())return;const y=e.retry??(En?0:3),x=e.retryDelay??w2,w=typeof x=="function"?x(n,b):x,k=y===!0||typeof y=="number"&&nc()?void 0:m()).then(()=>{t?d(b):f()})})};return{promise:o,status:()=>o.status,cancel:s,continue:()=>(r?.(),o),cancelRetry:a,continueRetry:l,canStart:u,start:()=>(u()?f():m().then(f),o)}}var Of=class{#e;destroy(){this.clearGcTimeout()}scheduleGc(){this.clearGcTimeout(),Aa(this.gcTime)&&(this.#e=Sn.setTimeout(()=>{this.optionalRemove()},this.gcTime))}updateGcTime(e){this.gcTime=Math.max(this.gcTime||0,e??(En?1/0:300*1e3))}clearGcTimeout(){this.#e&&(Sn.clearTimeout(this.#e),this.#e=void 0)}},S2=class extends Of{#e;#t;#n;#o;#r;#s;#a;constructor(e){super(),this.#a=!1,this.#s=e.defaultOptions,this.setOptions(e.options),this.observers=[],this.#o=e.client,this.#n=this.#o.getQueryCache(),this.queryKey=e.queryKey,this.queryHash=e.queryHash,this.#e=Rf(this.options),this.state=e.state??this.#e,this.scheduleGc()}get meta(){return this.options.meta}get promise(){return this.#r?.promise}setOptions(e){if(this.options={...this.#s,...e},this.updateGcTime(this.options.gcTime),this.state&&this.state.data===void 0){const t=Rf(this.options);t.data!==void 0&&(this.setData(t.data,{updatedAt:t.dataUpdatedAt,manual:!0}),this.#e=t)}}optionalRemove(){!this.observers.length&&this.state.fetchStatus==="idle"&&this.#n.remove(this)}setData(e,t){const n=Fa(this.state.data,e,this.options);return this.#i({data:n,type:"success",dataUpdatedAt:t?.updatedAt,manual:t?.manual}),n}setState(e,t){this.#i({type:"setState",state:e,setStateOptions:t})}cancel(e){const t=this.#r?.promise;return this.#r?.cancel(e),t?t.then(Ne).catch(Ne):Promise.resolve()}destroy(){super.destroy(),this.cancel({silent:!0})}reset(){this.destroy(),this.setState(this.#e)}isActive(){return this.observers.some(e=>Qe(e.options.enabled,this)!==!1)}isDisabled(){return this.getObserversCount()>0?!this.isActive():this.options.queryFn===La||this.state.dataUpdateCount+this.state.errorUpdateCount===0}isStatic(){return this.getObserversCount()>0?this.observers.some(e=>qt(e.options.staleTime,this)==="static"):!1}isStale(){return this.getObserversCount()>0?this.observers.some(e=>e.getCurrentResult().isStale):this.state.data===void 0||this.state.isInvalidated}isStaleByTime(e=0){return this.state.data===void 0?!0:e==="static"?!1:this.state.isInvalidated?!0:!mf(this.state.dataUpdatedAt,e)}onFocus(){this.observers.find(t=>t.shouldFetchOnWindowFocus())?.refetch({cancelRefetch:!1}),this.#r?.continue()}onOnline(){this.observers.find(t=>t.shouldFetchOnReconnect())?.refetch({cancelRefetch:!1}),this.#r?.continue()}addObserver(e){this.observers.includes(e)||(this.observers.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",query:this,observer:e}))}removeObserver(e){this.observers.includes(e)&&(this.observers=this.observers.filter(t=>t!==e),this.observers.length||(this.#r&&(this.#a?this.#r.cancel({revert:!0}):this.#r.cancelRetry()),this.scheduleGc()),this.#n.notify({type:"observerRemoved",query:this,observer:e}))}getObserversCount(){return this.observers.length}invalidate(){this.state.isInvalidated||this.#i({type:"invalidate"})}async fetch(e,t){if(this.state.fetchStatus!=="idle"&&this.#r?.status()!=="rejected"){if(this.state.data!==void 0&&t?.cancelRefetch)this.cancel({silent:!0});else if(this.#r)return this.#r.continueRetry(),this.#r.promise}if(e&&this.setOptions(e),!this.options.queryFn){const a=this.observers.find(l=>l.options.queryFn);a&&this.setOptions(a.options)}const n=new AbortController,r=a=>{Object.defineProperty(a,"signal",{enumerable:!0,get:()=>(this.#a=!0,n.signal)})},o=()=>{const a=Cf(this.options,t),c=(()=>{const u={client:this.#o,queryKey:this.queryKey,meta:this.meta};return r(u),u})();return this.#a=!1,this.options.persister?this.options.persister(a,c,this):a(c)},s=(()=>{const a={fetchOptions:t,options:this.options,queryKey:this.queryKey,client:this.#o,state:this.state,fetchFn:o};return r(a),a})();this.options.behavior?.onFetch(s,this),this.#t=this.state,(this.state.fetchStatus==="idle"||this.state.fetchMeta!==s.fetchOptions?.meta)&&this.#i({type:"fetch",meta:s.fetchOptions?.meta}),this.#r=Ef({initialPromise:t?.initialPromise,fn:s.fetchFn,onCancel:a=>{a instanceof Ma&&a.revert&&this.setState({...this.#t,fetchStatus:"idle"}),n.abort()},onFail:(a,l)=>{this.#i({type:"failed",failureCount:a,error:l})},onPause:()=>{this.#i({type:"pause"})},onContinue:()=>{this.#i({type:"continue"})},retry:s.options.retry,retryDelay:s.options.retryDelay,networkMode:s.options.networkMode,canRun:()=>!0});try{const a=await this.#r.start();if(a===void 0)throw new Error(`${this.queryHash} data is undefined`);return this.setData(a),this.#n.config.onSuccess?.(a,this),this.#n.config.onSettled?.(a,this.state.error,this),a}catch(a){if(a instanceof Ma){if(a.silent)return this.#r.promise;if(a.revert){if(this.state.data===void 0)throw a;return this.state.data}}throw this.#i({type:"error",error:a}),this.#n.config.onError?.(a,this),this.#n.config.onSettled?.(this.state.data,a,this),a}finally{this.scheduleGc()}}#i(e){const t=n=>{switch(e.type){case"failed":return{...n,fetchFailureCount:e.failureCount,fetchFailureReason:e.error};case"pause":return{...n,fetchStatus:"paused"};case"continue":return{...n,fetchStatus:"fetching"};case"fetch":return{...n,...Pf(n.data,this.options),fetchMeta:e.meta??null};case"success":const r={...n,data:e.data,dataUpdateCount:n.dataUpdateCount+1,dataUpdatedAt:e.dataUpdatedAt??Date.now(),error:null,isInvalidated:!1,status:"success",...!e.manual&&{fetchStatus:"idle",fetchFailureCount:0,fetchFailureReason:null}};return this.#t=e.manual?r:void 0,r;case"error":const o=e.error;return{...n,error:o,errorUpdateCount:n.errorUpdateCount+1,errorUpdatedAt:Date.now(),fetchFailureCount:n.fetchFailureCount+1,fetchFailureReason:o,fetchStatus:"idle",status:"error"};case"invalidate":return{...n,isInvalidated:!0};case"setState":return{...n,...e.state}}};this.state=t(this.state),ge.batch(()=>{this.observers.forEach(n=>{n.onQueryUpdate()}),this.#n.notify({query:this,type:"updated",action:e})})}};function Pf(e,t){return{fetchFailureCount:0,fetchFailureReason:null,fetchStatus:Sf(t.networkMode)?"fetching":"paused",...e===void 0&&{error:null,status:"pending"}}}function Rf(e){const t=typeof e.initialData=="function"?e.initialData():e.initialData,n=t!==void 0,r=n?typeof e.initialDataUpdatedAt=="function"?e.initialDataUpdatedAt():e.initialDataUpdatedAt:0;return{data:t,dataUpdateCount:0,dataUpdatedAt:n?r??Date.now():0,error:null,errorUpdateCount:0,errorUpdatedAt:0,fetchFailureCount:0,fetchFailureReason:null,fetchMeta:null,isInvalidated:!1,status:n?"success":"pending",fetchStatus:"idle"}}var E2=class extends ir{constructor(e,t){super(),this.options=t,this.#e=e,this.#i=null,this.#a=za(),this.bindMethods(),this.setOptions(t)}#e;#t=void 0;#n=void 0;#o=void 0;#r;#s;#a;#i;#p;#h;#f;#c;#u;#l;#g=new Set;bindMethods(){this.refetch=this.refetch.bind(this)}onSubscribe(){this.listeners.size===1&&(this.#t.addObserver(this),If(this.#t,this.options)?this.#d():this.updateResult(),this.#y())}onUnsubscribe(){this.hasListeners()||this.destroy()}shouldFetchOnReconnect(){return ja(this.#t,this.options,this.options.refetchOnReconnect)}shouldFetchOnWindowFocus(){return ja(this.#t,this.options,this.options.refetchOnWindowFocus)}destroy(){this.listeners=new Set,this.#x(),this.#k(),this.#t.removeObserver(this)}setOptions(e){const t=this.options,n=this.#t;if(this.options=this.#e.defaultQueryOptions(e),this.options.enabled!==void 0&&typeof this.options.enabled!="boolean"&&typeof this.options.enabled!="function"&&typeof Qe(this.options.enabled,this.#t)!="boolean")throw new Error("Expected enabled to be a boolean or a callback that returns a boolean");this.#C(),this.#t.setOptions(this.options),t._defaulted&&!vi(this.options,t)&&this.#e.getQueryCache().notify({type:"observerOptionsUpdated",query:this.#t,observer:this});const r=this.hasListeners();r&&Tf(this.#t,n,this.options,t)&&this.#d(),this.updateResult(),r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||qt(this.options.staleTime,this.#t)!==qt(t.staleTime,this.#t))&&this.#m();const o=this.#v();r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||o!==this.#l)&&this.#b(o)}getOptimisticResult(e){const t=this.#e.getQueryCache().build(this.#e,e),n=this.createResult(t,e);return P2(this,n)&&(this.#o=n,this.#s=this.options,this.#r=this.#t.state),n}getCurrentResult(){return this.#o}trackResult(e,t){return new Proxy(e,{get:(n,r)=>(this.trackProp(r),t?.(r),r==="promise"&&!this.options.experimental_prefetchInRender&&this.#a.status==="pending"&&this.#a.reject(new Error("experimental_prefetchInRender feature flag is not enabled")),Reflect.get(n,r))})}trackProp(e){this.#g.add(e)}getCurrentQuery(){return this.#t}refetch({...e}={}){return this.fetch({...e})}fetchOptimistic(e){const t=this.#e.defaultQueryOptions(e),n=this.#e.getQueryCache().build(this.#e,t);return n.fetch().then(()=>this.createResult(n,t))}fetch(e){return this.#d({...e,cancelRefetch:e.cancelRefetch??!0}).then(()=>(this.updateResult(),this.#o))}#d(e){this.#C();let t=this.#t.fetch(this.options,e);return e?.throwOnError||(t=t.catch(Ne)),t}#m(){this.#x();const e=qt(this.options.staleTime,this.#t);if(En||this.#o.isStale||!Aa(e))return;const n=mf(this.#o.dataUpdatedAt,e)+1;this.#c=Sn.setTimeout(()=>{this.#o.isStale||this.updateResult()},n)}#v(){return(typeof this.options.refetchInterval=="function"?this.options.refetchInterval(this.#t):this.options.refetchInterval)??!1}#b(e){this.#k(),this.#l=e,!(En||Qe(this.options.enabled,this.#t)===!1||!Aa(this.#l)||this.#l===0)&&(this.#u=Sn.setInterval(()=>{(this.options.refetchIntervalInBackground||Da.isFocused())&&this.#d()},this.#l))}#y(){this.#m(),this.#b(this.#v())}#x(){this.#c&&(Sn.clearTimeout(this.#c),this.#c=void 0)}#k(){this.#u&&(Sn.clearInterval(this.#u),this.#u=void 0)}createResult(e,t){const n=this.#t,r=this.options,o=this.#o,i=this.#r,s=this.#s,l=e!==n?e.state:this.#n,{state:c}=e;let u={...c},h=!1,d;if(t._optimisticResults){const I=this.hasListeners(),N=!I&&If(e,t),A=I&&Tf(e,n,t,r);(N||A)&&(u={...u,...Pf(c.data,e.options)}),t._optimisticResults==="isRestoring"&&(u.fetchStatus="idle")}let{error:m,errorUpdatedAt:f,status:p}=u;d=u.data;let v=!1;if(t.placeholderData!==void 0&&d===void 0&&p==="pending"){let I;o?.isPlaceholderData&&t.placeholderData===s?.placeholderData?(I=o.data,v=!0):I=typeof t.placeholderData=="function"?t.placeholderData(this.#f?.state.data,this.#f):t.placeholderData,I!==void 0&&(p="success",d=Fa(o?.data,I,t),h=!0)}if(t.select&&d!==void 0&&!v)if(o&&d===i?.data&&t.select===this.#p)d=this.#h;else try{this.#p=t.select,d=t.select(d),d=Fa(o?.data,d,t),this.#h=d,this.#i=null}catch(I){this.#i=I}this.#i&&(m=this.#i,d=this.#h,f=Date.now(),p="error");const b=u.fetchStatus==="fetching",y=p==="pending",x=p==="error",w=y&&b,k=d!==void 0,R={status:p,fetchStatus:u.fetchStatus,isPending:y,isSuccess:p==="success",isError:x,isInitialLoading:w,isLoading:w,data:d,dataUpdatedAt:u.dataUpdatedAt,error:m,errorUpdatedAt:f,failureCount:u.fetchFailureCount,failureReason:u.fetchFailureReason,errorUpdateCount:u.errorUpdateCount,isFetched:u.dataUpdateCount>0||u.errorUpdateCount>0,isFetchedAfterMount:u.dataUpdateCount>l.dataUpdateCount||u.errorUpdateCount>l.errorUpdateCount,isFetching:b,isRefetching:b&&!y,isLoadingError:x&&!k,isPaused:u.fetchStatus==="paused",isPlaceholderData:h,isRefetchError:x&&k,isStale:$a(e,t),refetch:this.refetch,promise:this.#a,isEnabled:Qe(t.enabled,e)!==!1};if(this.options.experimental_prefetchInRender){const I=T=>{R.status==="error"?T.reject(R.error):R.data!==void 0&&T.resolve(R.data)},N=()=>{const T=this.#a=R.promise=za();I(T)},A=this.#a;switch(A.status){case"pending":e.queryHash===n.queryHash&&I(A);break;case"fulfilled":(R.status==="error"||R.data!==A.value)&&N();break;case"rejected":(R.status!=="error"||R.error!==A.reason)&&N();break}}return R}updateResult(){const e=this.#o,t=this.createResult(this.#t,this.options);if(this.#r=this.#t.state,this.#s=this.options,this.#r.data!==void 0&&(this.#f=this.#t),vi(t,e))return;this.#o=t;const n=()=>{if(!e)return!0;const{notifyOnChangeProps:r}=this.options,o=typeof r=="function"?r():r;if(o==="all"||!o&&!this.#g.size)return!0;const i=new Set(o??this.#g);return this.options.throwOnError&&i.add("error"),Object.keys(this.#o).some(s=>{const a=s;return this.#o[a]!==e[a]&&i.has(a)})};this.#w({listeners:n()})}#C(){const e=this.#e.getQueryCache().build(this.#e,this.options);if(e===this.#t)return;const t=this.#t;this.#t=e,this.#n=e.state,this.hasListeners()&&(t?.removeObserver(this),e.addObserver(this))}onQueryUpdate(){this.updateResult(),this.hasListeners()&&this.#y()}#w(e){ge.batch(()=>{e.listeners&&this.listeners.forEach(t=>{t(this.#o)}),this.#e.getQueryCache().notify({query:this.#t,type:"observerResultsUpdated"})})}};function O2(e,t){return Qe(t.enabled,e)!==!1&&e.state.data===void 0&&!(e.state.status==="error"&&t.retryOnMount===!1)}function If(e,t){return O2(e,t)||e.state.data!==void 0&&ja(e,t,t.refetchOnMount)}function ja(e,t,n){if(Qe(t.enabled,e)!==!1&&qt(t.staleTime,e)!=="static"){const r=typeof n=="function"?n(e):n;return r==="always"||r!==!1&&$a(e,t)}return!1}function Tf(e,t,n,r){return(e!==t||Qe(r.enabled,e)===!1)&&(!n.suspense||e.state.status!=="error")&&$a(e,n)}function $a(e,t){return Qe(t.enabled,e)!==!1&&e.isStaleByTime(qt(t.staleTime,e))}function P2(e,t){return!vi(e.getCurrentResult(),t)}function Nf(e){return{onFetch:(t,n)=>{const r=t.options,o=t.fetchOptions?.meta?.fetchMore?.direction,i=t.state.data?.pages||[],s=t.state.data?.pageParams||[];let a={pages:[],pageParams:[]},l=0;const c=async()=>{let u=!1;const h=f=>{Object.defineProperty(f,"signal",{enumerable:!0,get:()=>(t.signal.aborted?u=!0:t.signal.addEventListener("abort",()=>{u=!0}),t.signal)})},d=Cf(t.options,t.fetchOptions),m=async(f,p,v)=>{if(u)return Promise.reject();if(p==null&&f.pages.length)return Promise.resolve(f);const y=(()=>{const O={client:t.client,queryKey:t.queryKey,pageParam:p,direction:v?"backward":"forward",meta:t.options.meta};return h(O),O})(),x=await d(y),{maxPages:w}=t.options,k=v?b2:v2;return{pages:k(f.pages,x,w),pageParams:k(f.pageParams,p,w)}};if(o&&i.length){const f=o==="backward",p=f?R2:Af,v={pages:i,pageParams:s},b=p(r,v);a=await m(v,b,f)}else{const f=e??i.length;do{const p=l===0?s[0]??r.initialPageParam:Af(r,a);if(l>0&&p==null)break;a=await m(a,p),l++}while(lt.options.persister?.(c,{client:t.client,queryKey:t.queryKey,meta:t.options.meta,signal:t.signal},n):t.fetchFn=c}}}function Af(e,{pages:t,pageParams:n}){const r=t.length-1;return t.length>0?e.getNextPageParam(t[r],t,n[r],n):void 0}function R2(e,{pages:t,pageParams:n}){return t.length>0?e.getPreviousPageParam?.(t[0],t,n[0],n):void 0}var I2=class extends Of{#e;#t;#n;#o;constructor(e){super(),this.#e=e.client,this.mutationId=e.mutationId,this.#n=e.mutationCache,this.#t=[],this.state=e.state||_f(),this.setOptions(e.options),this.scheduleGc()}setOptions(e){this.options=e,this.updateGcTime(this.options.gcTime)}get meta(){return this.options.meta}addObserver(e){this.#t.includes(e)||(this.#t.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",mutation:this,observer:e}))}removeObserver(e){this.#t=this.#t.filter(t=>t!==e),this.scheduleGc(),this.#n.notify({type:"observerRemoved",mutation:this,observer:e})}optionalRemove(){this.#t.length||(this.state.status==="pending"?this.scheduleGc():this.#n.remove(this))}continue(){return this.#o?.continue()??this.execute(this.state.variables)}async execute(e){const t=()=>{this.#r({type:"continue"})},n={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};this.#o=Ef({fn:()=>this.options.mutationFn?this.options.mutationFn(e,n):Promise.reject(new Error("No mutationFn found")),onFail:(i,s)=>{this.#r({type:"failed",failureCount:i,error:s})},onPause:()=>{this.#r({type:"pause"})},onContinue:t,retry:this.options.retry??0,retryDelay:this.options.retryDelay,networkMode:this.options.networkMode,canRun:()=>this.#n.canRun(this)});const r=this.state.status==="pending",o=!this.#o.canStart();try{if(r)t();else{this.#r({type:"pending",variables:e,isPaused:o}),await this.#n.config.onMutate?.(e,this,n);const s=await this.options.onMutate?.(e,n);s!==this.state.context&&this.#r({type:"pending",context:s,variables:e,isPaused:o})}const i=await this.#o.start();return await this.#n.config.onSuccess?.(i,e,this.state.context,this,n),await this.options.onSuccess?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(i,null,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(i,null,e,this.state.context,n),this.#r({type:"success",data:i}),i}catch(i){try{throw await this.#n.config.onError?.(i,e,this.state.context,this,n),await this.options.onError?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(void 0,i,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(void 0,i,e,this.state.context,n),i}finally{this.#r({type:"error",error:i})}}finally{this.#n.runNext(this)}}#r(e){const t=n=>{switch(e.type){case"failed":return{...n,failureCount:e.failureCount,failureReason:e.error};case"pause":return{...n,isPaused:!0};case"continue":return{...n,isPaused:!1};case"pending":return{...n,context:e.context,data:void 0,failureCount:0,failureReason:null,error:null,isPaused:e.isPaused,status:"pending",variables:e.variables,submittedAt:Date.now()};case"success":return{...n,data:e.data,failureCount:0,failureReason:null,error:null,status:"success",isPaused:!1};case"error":return{...n,data:void 0,error:e.error,failureCount:n.failureCount+1,failureReason:e.error,isPaused:!1,status:"error"}}};this.state=t(this.state),ge.batch(()=>{this.#t.forEach(n=>{n.onMutationUpdate(e)}),this.#n.notify({mutation:this,type:"updated",action:e})})}};function _f(){return{context:void 0,data:void 0,error:null,failureCount:0,failureReason:null,isPaused:!1,status:"idle",variables:void 0,submittedAt:0}}var T2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Set,this.#t=new Map,this.#n=0}#e;#t;#n;build(e,t,n){const r=new I2({client:e,mutationCache:this,mutationId:++this.#n,options:e.defaultMutationOptions(t),state:n});return this.add(r),r}add(e){this.#e.add(e);const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);n?n.push(e):this.#t.set(t,[e])}this.notify({type:"added",mutation:e})}remove(e){if(this.#e.delete(e)){const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);if(n)if(n.length>1){const r=n.indexOf(e);r!==-1&&n.splice(r,1)}else n[0]===e&&this.#t.delete(t)}}this.notify({type:"removed",mutation:e})}canRun(e){const t=yi(e);if(typeof t=="string"){const r=this.#t.get(t)?.find(o=>o.state.status==="pending");return!r||r===e}else return!0}runNext(e){const t=yi(e);return typeof t=="string"?this.#t.get(t)?.find(r=>r!==e&&r.state.isPaused)?.continue()??Promise.resolve():Promise.resolve()}clear(){ge.batch(()=>{this.#e.forEach(e=>{this.notify({type:"removed",mutation:e})}),this.#e.clear(),this.#t.clear()})}getAll(){return Array.from(this.#e)}find(e){const t={exact:!0,...e};return this.getAll().find(n=>bf(t,n))}findAll(e={}){return this.getAll().filter(t=>bf(e,t))}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}resumePausedMutations(){const e=this.getAll().filter(t=>t.state.isPaused);return ge.batch(()=>Promise.all(e.map(t=>t.continue().catch(Ne))))}};function yi(e){return e.options.scope?.id}var N2=class extends ir{#e;#t=void 0;#n;#o;constructor(e,t){super(),this.#e=e,this.setOptions(t),this.bindMethods(),this.#r()}bindMethods(){this.mutate=this.mutate.bind(this),this.reset=this.reset.bind(this)}setOptions(e){const t=this.options;this.options=this.#e.defaultMutationOptions(e),vi(this.options,t)||this.#e.getMutationCache().notify({type:"observerOptionsUpdated",mutation:this.#n,observer:this}),t?.mutationKey&&this.options.mutationKey&&On(t.mutationKey)!==On(this.options.mutationKey)?this.reset():this.#n?.state.status==="pending"&&this.#n.setOptions(this.options)}onUnsubscribe(){this.hasListeners()||this.#n?.removeObserver(this)}onMutationUpdate(e){this.#r(),this.#s(e)}getCurrentResult(){return this.#t}reset(){this.#n?.removeObserver(this),this.#n=void 0,this.#r(),this.#s()}mutate(e,t){return this.#o=t,this.#n?.removeObserver(this),this.#n=this.#e.getMutationCache().build(this.#e,this.options),this.#n.addObserver(this),this.#n.execute(e)}#r(){const e=this.#n?.state??_f();this.#t={...e,isPending:e.status==="pending",isSuccess:e.status==="success",isError:e.status==="error",isIdle:e.status==="idle",mutate:this.mutate,reset:this.reset}}#s(e){ge.batch(()=>{if(this.#o&&this.hasListeners()){const t=this.#t.variables,n=this.#t.context,r={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};e?.type==="success"?(this.#o.onSuccess?.(e.data,t,n,r),this.#o.onSettled?.(e.data,null,t,n,r)):e?.type==="error"&&(this.#o.onError?.(e.error,t,n,r),this.#o.onSettled?.(void 0,e.error,t,n,r))}this.listeners.forEach(t=>{t(this.#t)})})}},A2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Map}#e;build(e,t,n){const r=t.queryKey,o=t.queryHash??_a(r,t);let i=this.get(o);return i||(i=new S2({client:e,queryKey:r,queryHash:o,options:e.defaultQueryOptions(t),state:n,defaultOptions:e.getQueryDefaults(r)}),this.add(i)),i}add(e){this.#e.has(e.queryHash)||(this.#e.set(e.queryHash,e),this.notify({type:"added",query:e}))}remove(e){const t=this.#e.get(e.queryHash);t&&(e.destroy(),t===e&&this.#e.delete(e.queryHash),this.notify({type:"removed",query:e}))}clear(){ge.batch(()=>{this.getAll().forEach(e=>{this.remove(e)})})}get(e){return this.#e.get(e)}getAll(){return[...this.#e.values()]}find(e){const t={exact:!0,...e};return this.getAll().find(n=>vf(t,n))}findAll(e={}){const t=this.getAll();return Object.keys(e).length>0?t.filter(n=>vf(e,n)):t}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}onFocus(){ge.batch(()=>{this.getAll().forEach(e=>{e.onFocus()})})}onOnline(){ge.batch(()=>{this.getAll().forEach(e=>{e.onOnline()})})}},_2=class{#e;#t;#n;#o;#r;#s;#a;#i;constructor(e={}){this.#e=e.queryCache||new A2,this.#t=e.mutationCache||new T2,this.#n=e.defaultOptions||{},this.#o=new Map,this.#r=new Map,this.#s=0}mount(){this.#s++,this.#s===1&&(this.#a=Da.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onFocus())}),this.#i=bi.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onOnline())}))}unmount(){this.#s--,this.#s===0&&(this.#a?.(),this.#a=void 0,this.#i?.(),this.#i=void 0)}isFetching(e){return this.#e.findAll({...e,fetchStatus:"fetching"}).length}isMutating(e){return this.#t.findAll({...e,status:"pending"}).length}getQueryData(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state.data}ensureQueryData(e){const t=this.defaultQueryOptions(e),n=this.#e.build(this,t),r=n.state.data;return r===void 0?this.fetchQuery(e):(e.revalidateIfStale&&n.isStaleByTime(qt(t.staleTime,n))&&this.prefetchQuery(t),Promise.resolve(r))}getQueriesData(e){return this.#e.findAll(e).map(({queryKey:t,state:n})=>{const r=n.data;return[t,r]})}setQueryData(e,t,n){const r=this.defaultQueryOptions({queryKey:e}),i=this.#e.get(r.queryHash)?.state.data,s=g2(t,i);if(s!==void 0)return this.#e.build(this,r).setData(s,{...n,manual:!0})}setQueriesData(e,t,n){return ge.batch(()=>this.#e.findAll(e).map(({queryKey:r})=>[r,this.setQueryData(r,t,n)]))}getQueryState(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state}removeQueries(e){const t=this.#e;ge.batch(()=>{t.findAll(e).forEach(n=>{t.remove(n)})})}resetQueries(e,t){const n=this.#e;return ge.batch(()=>(n.findAll(e).forEach(r=>{r.reset()}),this.refetchQueries({type:"active",...e},t)))}cancelQueries(e,t={}){const n={revert:!0,...t},r=ge.batch(()=>this.#e.findAll(e).map(o=>o.cancel(n)));return Promise.all(r).then(Ne).catch(Ne)}invalidateQueries(e,t={}){return ge.batch(()=>(this.#e.findAll(e).forEach(n=>{n.invalidate()}),e?.refetchType==="none"?Promise.resolve():this.refetchQueries({...e,type:e?.refetchType??e?.type??"active"},t)))}refetchQueries(e,t={}){const n={...t,cancelRefetch:t.cancelRefetch??!0},r=ge.batch(()=>this.#e.findAll(e).filter(o=>!o.isDisabled()&&!o.isStatic()).map(o=>{let i=o.fetch(void 0,n);return n.throwOnError||(i=i.catch(Ne)),o.state.fetchStatus==="paused"?Promise.resolve():i}));return Promise.all(r).then(Ne)}fetchQuery(e){const t=this.defaultQueryOptions(e);t.retry===void 0&&(t.retry=!1);const n=this.#e.build(this,t);return n.isStaleByTime(qt(t.staleTime,n))?n.fetch(t):Promise.resolve(n.state.data)}prefetchQuery(e){return this.fetchQuery(e).then(Ne).catch(Ne)}fetchInfiniteQuery(e){return e.behavior=Nf(e.pages),this.fetchQuery(e)}prefetchInfiniteQuery(e){return this.fetchInfiniteQuery(e).then(Ne).catch(Ne)}ensureInfiniteQueryData(e){return e.behavior=Nf(e.pages),this.ensureQueryData(e)}resumePausedMutations(){return bi.isOnline()?this.#t.resumePausedMutations():Promise.resolve()}getQueryCache(){return this.#e}getMutationCache(){return this.#t}getDefaultOptions(){return this.#n}setDefaultOptions(e){this.#n=e}setQueryDefaults(e,t){this.#o.set(On(e),{queryKey:e,defaultOptions:t})}getQueryDefaults(e){const t=[...this.#o.values()],n={};return t.forEach(r=>{Wr(e,r.queryKey)&&Object.assign(n,r.defaultOptions)}),n}setMutationDefaults(e,t){this.#r.set(On(e),{mutationKey:e,defaultOptions:t})}getMutationDefaults(e){const t=[...this.#r.values()],n={};return t.forEach(r=>{Wr(e,r.mutationKey)&&Object.assign(n,r.defaultOptions)}),n}defaultQueryOptions(e){if(e._defaulted)return e;const t={...this.#n.queries,...this.getQueryDefaults(e.queryKey),...e,_defaulted:!0};return t.queryHash||(t.queryHash=_a(t.queryKey,t)),t.refetchOnReconnect===void 0&&(t.refetchOnReconnect=t.networkMode!=="always"),t.throwOnError===void 0&&(t.throwOnError=!!t.suspense),!t.networkMode&&t.persister&&(t.networkMode="offlineFirst"),t.queryFn===La&&(t.enabled=!1),t}defaultMutationOptions(e){return e?._defaulted?e:{...this.#n.mutations,...e?.mutationKey&&this.getMutationDefaults(e.mutationKey),...e,_defaulted:!0}}clear(){this.#e.clear(),this.#t.clear()}},Vf=C.createContext(void 0),Ff=e=>{const t=C.useContext(Vf);if(!t)throw new Error("No QueryClient set, use QueryClientProvider to set one");return t},V2=({client:e,children:t})=>(C.useEffect(()=>(e.mount(),()=>{e.unmount()}),[e]),g.jsx(Vf.Provider,{value:e,children:t})),Lf=C.createContext(!1),F2=()=>C.useContext(Lf);Lf.Provider;function L2(){let e=!1;return{clearReset:()=>{e=!1},reset:()=>{e=!0},isReset:()=>e}}var D2=C.createContext(L2()),z2=()=>C.useContext(D2),M2=(e,t)=>{(e.suspense||e.throwOnError||e.experimental_prefetchInRender)&&(t.isReset()||(e.retryOnMount=!1))},j2=e=>{C.useEffect(()=>{e.clearReset()},[e])},$2=({result:e,errorResetBoundary:t,throwOnError:n,query:r,suspense:o})=>e.isError&&!t.isReset()&&!e.isFetching&&r&&(o&&e.data===void 0||wf(n,[e.error,r])),B2=e=>{if(e.suspense){const n=o=>o==="static"?o:Math.max(o??1e3,1e3),r=e.staleTime;e.staleTime=typeof r=="function"?(...o)=>n(r(...o)):n(r),typeof e.gcTime=="number"&&(e.gcTime=Math.max(e.gcTime,1e3))}},W2=(e,t)=>e.isLoading&&e.isFetching&&!t,H2=(e,t)=>e?.suspense&&t.isPending,Df=(e,t,n)=>t.fetchOptimistic(e).catch(()=>{n.clearReset()});function U2(e,t,n){const r=F2(),o=z2(),i=Ff(),s=i.defaultQueryOptions(e);i.getDefaultOptions().queries?._experimental_beforeQuery?.(s),s._optimisticResults=r?"isRestoring":"optimistic",B2(s),M2(s,o),j2(o);const a=!i.getQueryCache().get(s.queryHash),[l]=C.useState(()=>new t(i,s)),c=l.getOptimisticResult(s),u=!r&&e.subscribed!==!1;if(C.useSyncExternalStore(C.useCallback(h=>{const d=u?l.subscribe(ge.batchCalls(h)):Ne;return l.updateResult(),d},[l,u]),()=>l.getCurrentResult(),()=>l.getCurrentResult()),C.useEffect(()=>{l.setOptions(s)},[s,l]),H2(s,c))throw Df(s,l,o);if($2({result:c,errorResetBoundary:o,throwOnError:s.throwOnError,query:i.getQueryCache().get(s.queryHash),suspense:s.suspense}))throw c.error;return i.getDefaultOptions().queries?._experimental_afterQuery?.(s,c),s.experimental_prefetchInRender&&!En&&W2(c,r)&&(a?Df(s,l,o):i.getQueryCache().get(s.queryHash)?.promise)?.catch(Ne).finally(()=>{l.updateResult()}),s.notifyOnChangeProps?c:l.trackResult(c)}function zf(e,t){return U2(e,E2)}function Pn(e,t){const n=Ff(),[r]=C.useState(()=>new N2(n,e));C.useEffect(()=>{r.setOptions(e)},[r,e]);const o=C.useSyncExternalStore(C.useCallback(s=>r.subscribe(ge.batchCalls(s)),[r]),()=>r.getCurrentResult(),()=>r.getCurrentResult()),i=C.useCallback((s,a)=>{r.mutate(s,a).catch(Ne)},[r]);if(o.error&&wf(r.options.throwOnError,[o.error]))throw o.error;return{...o,mutate:i,mutateAsync:o.mutate}}function Mf(e,t){return function(){return e.apply(t,arguments)}}const{toString:G2}=Object.prototype,{getPrototypeOf:Ba}=Object,{iterator:xi,toStringTag:jf}=Symbol,ki=(e=>t=>{const n=G2.call(t);return e[n]||(e[n]=n.slice(8,-1).toLowerCase())})(Object.create(null)),ot=e=>(e=e.toLowerCase(),t=>ki(t)===e),Ci=e=>t=>typeof t===e,{isArray:sr}=Array,ar=Ci("undefined");function Hr(e){return e!==null&&!ar(e)&&e.constructor!==null&&!ar(e.constructor)&&ze(e.constructor.isBuffer)&&e.constructor.isBuffer(e)}const $f=ot("ArrayBuffer");function q2(e){let t;return typeof ArrayBuffer<"u"&&ArrayBuffer.isView?t=ArrayBuffer.isView(e):t=e&&e.buffer&&$f(e.buffer),t}const K2=Ci("string"),ze=Ci("function"),Bf=Ci("number"),Ur=e=>e!==null&&typeof e=="object",Y2=e=>e===!0||e===!1,wi=e=>{if(ki(e)!=="object")return!1;const t=Ba(e);return(t===null||t===Object.prototype||Object.getPrototypeOf(t)===null)&&!(jf in e)&&!(xi in e)},X2=e=>{if(!Ur(e)||Hr(e))return!1;try{return Object.keys(e).length===0&&Object.getPrototypeOf(e)===Object.prototype}catch{return!1}},Q2=ot("Date"),J2=ot("File"),Z2=ot("Blob"),eO=ot("FileList"),tO=e=>Ur(e)&&ze(e.pipe),nO=e=>{let t;return e&&(typeof FormData=="function"&&e instanceof FormData||ze(e.append)&&((t=ki(e))==="formdata"||t==="object"&&ze(e.toString)&&e.toString()==="[object FormData]"))},rO=ot("URLSearchParams"),[oO,iO,sO,aO]=["ReadableStream","Request","Response","Headers"].map(ot),lO=e=>e.trim?e.trim():e.replace(/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,"");function Gr(e,t,{allOwnKeys:n=!1}={}){if(e===null||typeof e>"u")return;let r,o;if(typeof e!="object"&&(e=[e]),sr(e))for(r=0,o=e.length;r0;)if(o=n[r],t===o.toLowerCase())return o;return null}const Rn=typeof globalThis<"u"?globalThis:typeof self<"u"?self:typeof window<"u"?window:globalThis,Hf=e=>!ar(e)&&e!==Rn;function Wa(){const{caseless:e,skipUndefined:t}=Hf(this)&&this||{},n={},r=(o,i)=>{const s=e&&Wf(n,i)||i;wi(n[s])&&wi(o)?n[s]=Wa(n[s],o):wi(o)?n[s]=Wa({},o):sr(o)?n[s]=o.slice():(!t||!ar(o))&&(n[s]=o)};for(let o=0,i=arguments.length;o(Gr(t,(o,i)=>{n&&ze(o)?e[i]=Mf(o,n):e[i]=o},{allOwnKeys:r}),e),uO=e=>(e.charCodeAt(0)===65279&&(e=e.slice(1)),e),dO=(e,t,n,r)=>{e.prototype=Object.create(t.prototype,r),e.prototype.constructor=e,Object.defineProperty(e,"super",{value:t.prototype}),n&&Object.assign(e.prototype,n)},hO=(e,t,n,r)=>{let o,i,s;const a={};if(t=t||{},e==null)return t;do{for(o=Object.getOwnPropertyNames(e),i=o.length;i-- >0;)s=o[i],(!r||r(s,e,t))&&!a[s]&&(t[s]=e[s],a[s]=!0);e=n!==!1&&Ba(e)}while(e&&(!n||n(e,t))&&e!==Object.prototype);return t},fO=(e,t,n)=>{e=String(e),(n===void 0||n>e.length)&&(n=e.length),n-=t.length;const r=e.indexOf(t,n);return r!==-1&&r===n},gO=e=>{if(!e)return null;if(sr(e))return e;let t=e.length;if(!Bf(t))return null;const n=new Array(t);for(;t-- >0;)n[t]=e[t];return n},pO=(e=>t=>e&&t instanceof e)(typeof Uint8Array<"u"&&Ba(Uint8Array)),mO=(e,t)=>{const r=(e&&e[xi]).call(e);let o;for(;(o=r.next())&&!o.done;){const i=o.value;t.call(e,i[0],i[1])}},vO=(e,t)=>{let n;const r=[];for(;(n=e.exec(t))!==null;)r.push(n);return r},bO=ot("HTMLFormElement"),yO=e=>e.toLowerCase().replace(/[-_\s]([a-z\d])(\w*)/g,function(n,r,o){return r.toUpperCase()+o}),Uf=(({hasOwnProperty:e})=>(t,n)=>e.call(t,n))(Object.prototype),xO=ot("RegExp"),Gf=(e,t)=>{const n=Object.getOwnPropertyDescriptors(e),r={};Gr(n,(o,i)=>{let s;(s=t(o,i,e))!==!1&&(r[i]=s||o)}),Object.defineProperties(e,r)},kO=e=>{Gf(e,(t,n)=>{if(ze(e)&&["arguments","caller","callee"].indexOf(n)!==-1)return!1;const r=e[n];if(ze(r)){if(t.enumerable=!1,"writable"in t){t.writable=!1;return}t.set||(t.set=()=>{throw Error("Can not rewrite read-only method '"+n+"'")})}})},CO=(e,t)=>{const n={},r=o=>{o.forEach(i=>{n[i]=!0})};return sr(e)?r(e):r(String(e).split(t)),n},wO=()=>{},SO=(e,t)=>e!=null&&Number.isFinite(e=+e)?e:t;function EO(e){return!!(e&&ze(e.append)&&e[jf]==="FormData"&&e[xi])}const OO=e=>{const t=new Array(10),n=(r,o)=>{if(Ur(r)){if(t.indexOf(r)>=0)return;if(Hr(r))return r;if(!("toJSON"in r)){t[o]=r;const i=sr(r)?[]:{};return Gr(r,(s,a)=>{const l=n(s,o+1);!ar(l)&&(i[a]=l)}),t[o]=void 0,i}}return r};return n(e,0)},PO=ot("AsyncFunction"),RO=e=>e&&(Ur(e)||ze(e))&&ze(e.then)&&ze(e.catch),qf=((e,t)=>e?setImmediate:t?((n,r)=>(Rn.addEventListener("message",({source:o,data:i})=>{o===Rn&&i===n&&r.length&&r.shift()()},!1),o=>{r.push(o),Rn.postMessage(n,"*")}))(`axios@${Math.random()}`,[]):n=>setTimeout(n))(typeof setImmediate=="function",ze(Rn.postMessage)),IO=typeof queueMicrotask<"u"?queueMicrotask.bind(Rn):typeof process<"u"&&process.nextTick||qf,E={isArray:sr,isArrayBuffer:$f,isBuffer:Hr,isFormData:nO,isArrayBufferView:q2,isString:K2,isNumber:Bf,isBoolean:Y2,isObject:Ur,isPlainObject:wi,isEmptyObject:X2,isReadableStream:oO,isRequest:iO,isResponse:sO,isHeaders:aO,isUndefined:ar,isDate:Q2,isFile:J2,isBlob:Z2,isRegExp:xO,isFunction:ze,isStream:tO,isURLSearchParams:rO,isTypedArray:pO,isFileList:eO,forEach:Gr,merge:Wa,extend:cO,trim:lO,stripBOM:uO,inherits:dO,toFlatObject:hO,kindOf:ki,kindOfTest:ot,endsWith:fO,toArray:gO,forEachEntry:mO,matchAll:vO,isHTMLForm:bO,hasOwnProperty:Uf,hasOwnProp:Uf,reduceDescriptors:Gf,freezeMethods:kO,toObjectSet:CO,toCamelCase:yO,noop:wO,toFiniteNumber:SO,findKey:Wf,global:Rn,isContextDefined:Hf,isSpecCompliantForm:EO,toJSONObject:OO,isAsyncFn:PO,isThenable:RO,setImmediate:qf,asap:IO,isIterable:e=>e!=null&&ze(e[xi])};function H(e,t,n,r,o){Error.call(this),Error.captureStackTrace?Error.captureStackTrace(this,this.constructor):this.stack=new Error().stack,this.message=e,this.name="AxiosError",t&&(this.code=t),n&&(this.config=n),r&&(this.request=r),o&&(this.response=o,this.status=o.status?o.status:null)}E.inherits(H,Error,{toJSON:function(){return{message:this.message,name:this.name,description:this.description,number:this.number,fileName:this.fileName,lineNumber:this.lineNumber,columnNumber:this.columnNumber,stack:this.stack,config:E.toJSONObject(this.config),code:this.code,status:this.status}}});const Kf=H.prototype,Yf={};["ERR_BAD_OPTION_VALUE","ERR_BAD_OPTION","ECONNABORTED","ETIMEDOUT","ERR_NETWORK","ERR_FR_TOO_MANY_REDIRECTS","ERR_DEPRECATED","ERR_BAD_RESPONSE","ERR_BAD_REQUEST","ERR_CANCELED","ERR_NOT_SUPPORT","ERR_INVALID_URL"].forEach(e=>{Yf[e]={value:e}}),Object.defineProperties(H,Yf),Object.defineProperty(Kf,"isAxiosError",{value:!0}),H.from=(e,t,n,r,o,i)=>{const s=Object.create(Kf);E.toFlatObject(e,s,function(u){return u!==Error.prototype},c=>c!=="isAxiosError");const a=e&&e.message?e.message:"Error",l=t==null&&e?e.code:t;return H.call(s,a,l,n,r,o),e&&s.cause==null&&Object.defineProperty(s,"cause",{value:e,configurable:!0}),s.name=e&&e.name||"Error",i&&Object.assign(s,i),s};const TO=null;function Ha(e){return E.isPlainObject(e)||E.isArray(e)}function Xf(e){return E.endsWith(e,"[]")?e.slice(0,-2):e}function Qf(e,t,n){return e?e.concat(t).map(function(o,i){return o=Xf(o),!n&&i?"["+o+"]":o}).join(n?".":""):t}function NO(e){return E.isArray(e)&&!e.some(Ha)}const AO=E.toFlatObject(E,{},null,function(t){return/^is[A-Z]/.test(t)});function Si(e,t,n){if(!E.isObject(e))throw new TypeError("target must be an object");t=t||new FormData,n=E.toFlatObject(n,{metaTokens:!0,dots:!1,indexes:!1},!1,function(p,v){return!E.isUndefined(v[p])});const r=n.metaTokens,o=n.visitor||u,i=n.dots,s=n.indexes,l=(n.Blob||typeof Blob<"u"&&Blob)&&E.isSpecCompliantForm(t);if(!E.isFunction(o))throw new TypeError("visitor must be a function");function c(f){if(f===null)return"";if(E.isDate(f))return f.toISOString();if(E.isBoolean(f))return f.toString();if(!l&&E.isBlob(f))throw new H("Blob is not supported. Use a Buffer instead.");return E.isArrayBuffer(f)||E.isTypedArray(f)?l&&typeof Blob=="function"?new Blob([f]):Buffer.from(f):f}function u(f,p,v){let b=f;if(f&&!v&&typeof f=="object"){if(E.endsWith(p,"{}"))p=r?p:p.slice(0,-2),f=JSON.stringify(f);else if(E.isArray(f)&&NO(f)||(E.isFileList(f)||E.endsWith(p,"[]"))&&(b=E.toArray(f)))return p=Xf(p),b.forEach(function(x,w){!(E.isUndefined(x)||x===null)&&t.append(s===!0?Qf([p],w,i):s===null?p:p+"[]",c(x))}),!1}return Ha(f)?!0:(t.append(Qf(v,p,i),c(f)),!1)}const h=[],d=Object.assign(AO,{defaultVisitor:u,convertValue:c,isVisitable:Ha});function m(f,p){if(!E.isUndefined(f)){if(h.indexOf(f)!==-1)throw Error("Circular reference detected in "+p.join("."));h.push(f),E.forEach(f,function(b,y){(!(E.isUndefined(b)||b===null)&&o.call(t,b,E.isString(y)?y.trim():y,p,d))===!0&&m(b,p?p.concat(y):[y])}),h.pop()}}if(!E.isObject(e))throw new TypeError("data must be an object");return m(e),t}function Jf(e){const t={"!":"%21","'":"%27","(":"%28",")":"%29","~":"%7E","%20":"+","%00":"\0"};return encodeURIComponent(e).replace(/[!'()~]|%20|%00/g,function(r){return t[r]})}function Ua(e,t){this._pairs=[],e&&Si(e,this,t)}const Zf=Ua.prototype;Zf.append=function(t,n){this._pairs.push([t,n])},Zf.toString=function(t){const n=t?function(r){return t.call(this,r,Jf)}:Jf;return this._pairs.map(function(o){return n(o[0])+"="+n(o[1])},"").join("&")};function _O(e){return encodeURIComponent(e).replace(/%3A/gi,":").replace(/%24/g,"$").replace(/%2C/gi,",").replace(/%20/g,"+")}function eg(e,t,n){if(!t)return e;const r=n&&n.encode||_O;E.isFunction(n)&&(n={serialize:n});const o=n&&n.serialize;let i;if(o?i=o(t,n):i=E.isURLSearchParams(t)?t.toString():new Ua(t,n).toString(r),i){const s=e.indexOf("#");s!==-1&&(e=e.slice(0,s)),e+=(e.indexOf("?")===-1?"?":"&")+i}return e}class tg{constructor(){this.handlers=[]}use(t,n,r){return this.handlers.push({fulfilled:t,rejected:n,synchronous:r?r.synchronous:!1,runWhen:r?r.runWhen:null}),this.handlers.length-1}eject(t){this.handlers[t]&&(this.handlers[t]=null)}clear(){this.handlers&&(this.handlers=[])}forEach(t){E.forEach(this.handlers,function(r){r!==null&&t(r)})}}const ng={silentJSONParsing:!0,forcedJSONParsing:!0,clarifyTimeoutError:!1},VO={isBrowser:!0,classes:{URLSearchParams:typeof URLSearchParams<"u"?URLSearchParams:Ua,FormData:typeof FormData<"u"?FormData:null,Blob:typeof Blob<"u"?Blob:null},protocols:["http","https","file","blob","url","data"]},Ga=typeof window<"u"&&typeof document<"u",qa=typeof navigator=="object"&&navigator||void 0,FO=Ga&&(!qa||["ReactNative","NativeScript","NS"].indexOf(qa.product)<0),LO=typeof WorkerGlobalScope<"u"&&self instanceof WorkerGlobalScope&&typeof self.importScripts=="function",DO=Ga&&window.location.href||"http://localhost",Ie={...Object.freeze(Object.defineProperty({__proto__:null,hasBrowserEnv:Ga,hasStandardBrowserEnv:FO,hasStandardBrowserWebWorkerEnv:LO,navigator:qa,origin:DO},Symbol.toStringTag,{value:"Module"})),...VO};function zO(e,t){return Si(e,new Ie.classes.URLSearchParams,{visitor:function(n,r,o,i){return Ie.isNode&&E.isBuffer(n)?(this.append(r,n.toString("base64")),!1):i.defaultVisitor.apply(this,arguments)},...t})}function MO(e){return E.matchAll(/\w+|\[(\w*)]/g,e).map(t=>t[0]==="[]"?"":t[1]||t[0])}function jO(e){const t={},n=Object.keys(e);let r;const o=n.length;let i;for(r=0;r=n.length;return s=!s&&E.isArray(o)?o.length:s,l?(E.hasOwnProp(o,s)?o[s]=[o[s],r]:o[s]=r,!a):((!o[s]||!E.isObject(o[s]))&&(o[s]=[]),t(n,r,o[s],i)&&E.isArray(o[s])&&(o[s]=jO(o[s])),!a)}if(E.isFormData(e)&&E.isFunction(e.entries)){const n={};return E.forEachEntry(e,(r,o)=>{t(MO(r),o,n,0)}),n}return null}function $O(e,t,n){if(E.isString(e))try{return(t||JSON.parse)(e),E.trim(e)}catch(r){if(r.name!=="SyntaxError")throw r}return(n||JSON.stringify)(e)}const qr={transitional:ng,adapter:["xhr","http","fetch"],transformRequest:[function(t,n){const r=n.getContentType()||"",o=r.indexOf("application/json")>-1,i=E.isObject(t);if(i&&E.isHTMLForm(t)&&(t=new FormData(t)),E.isFormData(t))return o?JSON.stringify(rg(t)):t;if(E.isArrayBuffer(t)||E.isBuffer(t)||E.isStream(t)||E.isFile(t)||E.isBlob(t)||E.isReadableStream(t))return t;if(E.isArrayBufferView(t))return t.buffer;if(E.isURLSearchParams(t))return n.setContentType("application/x-www-form-urlencoded;charset=utf-8",!1),t.toString();let a;if(i){if(r.indexOf("application/x-www-form-urlencoded")>-1)return zO(t,this.formSerializer).toString();if((a=E.isFileList(t))||r.indexOf("multipart/form-data")>-1){const l=this.env&&this.env.FormData;return Si(a?{"files[]":t}:t,l&&new l,this.formSerializer)}}return i||o?(n.setContentType("application/json",!1),$O(t)):t}],transformResponse:[function(t){const n=this.transitional||qr.transitional,r=n&&n.forcedJSONParsing,o=this.responseType==="json";if(E.isResponse(t)||E.isReadableStream(t))return t;if(t&&E.isString(t)&&(r&&!this.responseType||o)){const s=!(n&&n.silentJSONParsing)&&o;try{return JSON.parse(t,this.parseReviver)}catch(a){if(s)throw a.name==="SyntaxError"?H.from(a,H.ERR_BAD_RESPONSE,this,null,this.response):a}}return t}],timeout:0,xsrfCookieName:"XSRF-TOKEN",xsrfHeaderName:"X-XSRF-TOKEN",maxContentLength:-1,maxBodyLength:-1,env:{FormData:Ie.classes.FormData,Blob:Ie.classes.Blob},validateStatus:function(t){return t>=200&&t<300},headers:{common:{Accept:"application/json, text/plain, */*","Content-Type":void 0}}};E.forEach(["delete","get","head","post","put","patch"],e=>{qr.headers[e]={}});const BO=E.toObjectSet(["age","authorization","content-length","content-type","etag","expires","from","host","if-modified-since","if-unmodified-since","last-modified","location","max-forwards","proxy-authorization","referer","retry-after","user-agent"]),WO=e=>{const t={};let n,r,o;return e&&e.split(` -`).forEach(function(s){o=s.indexOf(":"),n=s.substring(0,o).trim().toLowerCase(),r=s.substring(o+1).trim(),!(!n||t[n]&&BO[n])&&(n==="set-cookie"?t[n]?t[n].push(r):t[n]=[r]:t[n]=t[n]?t[n]+", "+r:r)}),t},og=Symbol("internals");function Kr(e){return e&&String(e).trim().toLowerCase()}function Ei(e){return e===!1||e==null?e:E.isArray(e)?e.map(Ei):String(e)}function HO(e){const t=Object.create(null),n=/([^\s,;=]+)\s*(?:=\s*([^,;]+))?/g;let r;for(;r=n.exec(e);)t[r[1]]=r[2];return t}const UO=e=>/^[-_a-zA-Z0-9^`|~,!#$%&'*+.]+$/.test(e.trim());function Ka(e,t,n,r,o){if(E.isFunction(r))return r.call(this,t,n);if(o&&(t=n),!!E.isString(t)){if(E.isString(r))return t.indexOf(r)!==-1;if(E.isRegExp(r))return r.test(t)}}function GO(e){return e.trim().toLowerCase().replace(/([a-z\d])(\w*)/g,(t,n,r)=>n.toUpperCase()+r)}function qO(e,t){const n=E.toCamelCase(" "+t);["get","set","has"].forEach(r=>{Object.defineProperty(e,r+n,{value:function(o,i,s){return this[r].call(this,t,o,i,s)},configurable:!0})})}let Me=class{constructor(t){t&&this.set(t)}set(t,n,r){const o=this;function i(a,l,c){const u=Kr(l);if(!u)throw new Error("header name must be a non-empty string");const h=E.findKey(o,u);(!h||o[h]===void 0||c===!0||c===void 0&&o[h]!==!1)&&(o[h||l]=Ei(a))}const s=(a,l)=>E.forEach(a,(c,u)=>i(c,u,l));if(E.isPlainObject(t)||t instanceof this.constructor)s(t,n);else if(E.isString(t)&&(t=t.trim())&&!UO(t))s(WO(t),n);else if(E.isObject(t)&&E.isIterable(t)){let a={},l,c;for(const u of t){if(!E.isArray(u))throw TypeError("Object iterator must return a key-value pair");a[c=u[0]]=(l=a[c])?E.isArray(l)?[...l,u[1]]:[l,u[1]]:u[1]}s(a,n)}else t!=null&&i(n,t,r);return this}get(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);if(r){const o=this[r];if(!n)return o;if(n===!0)return HO(o);if(E.isFunction(n))return n.call(this,o,r);if(E.isRegExp(n))return n.exec(o);throw new TypeError("parser must be boolean|regexp|function")}}}has(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);return!!(r&&this[r]!==void 0&&(!n||Ka(this,this[r],r,n)))}return!1}delete(t,n){const r=this;let o=!1;function i(s){if(s=Kr(s),s){const a=E.findKey(r,s);a&&(!n||Ka(r,r[a],a,n))&&(delete r[a],o=!0)}}return E.isArray(t)?t.forEach(i):i(t),o}clear(t){const n=Object.keys(this);let r=n.length,o=!1;for(;r--;){const i=n[r];(!t||Ka(this,this[i],i,t,!0))&&(delete this[i],o=!0)}return o}normalize(t){const n=this,r={};return E.forEach(this,(o,i)=>{const s=E.findKey(r,i);if(s){n[s]=Ei(o),delete n[i];return}const a=t?GO(i):String(i).trim();a!==i&&delete n[i],n[a]=Ei(o),r[a]=!0}),this}concat(...t){return this.constructor.concat(this,...t)}toJSON(t){const n=Object.create(null);return E.forEach(this,(r,o)=>{r!=null&&r!==!1&&(n[o]=t&&E.isArray(r)?r.join(", "):r)}),n}[Symbol.iterator](){return Object.entries(this.toJSON())[Symbol.iterator]()}toString(){return Object.entries(this.toJSON()).map(([t,n])=>t+": "+n).join(` -`)}getSetCookie(){return this.get("set-cookie")||[]}get[Symbol.toStringTag](){return"AxiosHeaders"}static from(t){return t instanceof this?t:new this(t)}static concat(t,...n){const r=new this(t);return n.forEach(o=>r.set(o)),r}static accessor(t){const r=(this[og]=this[og]={accessors:{}}).accessors,o=this.prototype;function i(s){const a=Kr(s);r[a]||(qO(o,s),r[a]=!0)}return E.isArray(t)?t.forEach(i):i(t),this}};Me.accessor(["Content-Type","Content-Length","Accept","Accept-Encoding","User-Agent","Authorization"]),E.reduceDescriptors(Me.prototype,({value:e},t)=>{let n=t[0].toUpperCase()+t.slice(1);return{get:()=>e,set(r){this[n]=r}}}),E.freezeMethods(Me);function Ya(e,t){const n=this||qr,r=t||n,o=Me.from(r.headers);let i=r.data;return E.forEach(e,function(a){i=a.call(n,i,o.normalize(),t?t.status:void 0)}),o.normalize(),i}function ig(e){return!!(e&&e.__CANCEL__)}function lr(e,t,n){H.call(this,e??"canceled",H.ERR_CANCELED,t,n),this.name="CanceledError"}E.inherits(lr,H,{__CANCEL__:!0});function sg(e,t,n){const r=n.config.validateStatus;!n.status||!r||r(n.status)?e(n):t(new H("Request failed with status code "+n.status,[H.ERR_BAD_REQUEST,H.ERR_BAD_RESPONSE][Math.floor(n.status/100)-4],n.config,n.request,n))}function KO(e){const t=/^([-+\w]{1,25})(:?\/\/|:)/.exec(e);return t&&t[1]||""}function YO(e,t){e=e||10;const n=new Array(e),r=new Array(e);let o=0,i=0,s;return t=t!==void 0?t:1e3,function(l){const c=Date.now(),u=r[i];s||(s=c),n[o]=l,r[o]=c;let h=i,d=0;for(;h!==o;)d+=n[h++],h=h%e;if(o=(o+1)%e,o===i&&(i=(i+1)%e),c-s{n=u,o=null,i&&(clearTimeout(i),i=null),e(...c)};return[(...c)=>{const u=Date.now(),h=u-n;h>=r?s(c,u):(o=c,i||(i=setTimeout(()=>{i=null,s(o)},r-h)))},()=>o&&s(o)]}const Oi=(e,t,n=3)=>{let r=0;const o=YO(50,250);return XO(i=>{const s=i.loaded,a=i.lengthComputable?i.total:void 0,l=s-r,c=o(l),u=s<=a;r=s;const h={loaded:s,total:a,progress:a?s/a:void 0,bytes:l,rate:c||void 0,estimated:c&&a&&u?(a-s)/c:void 0,event:i,lengthComputable:a!=null,[t?"download":"upload"]:!0};e(h)},n)},ag=(e,t)=>{const n=e!=null;return[r=>t[0]({lengthComputable:n,total:e,loaded:r}),t[1]]},lg=e=>(...t)=>E.asap(()=>e(...t)),QO=Ie.hasStandardBrowserEnv?((e,t)=>n=>(n=new URL(n,Ie.origin),e.protocol===n.protocol&&e.host===n.host&&(t||e.port===n.port)))(new URL(Ie.origin),Ie.navigator&&/(msie|trident)/i.test(Ie.navigator.userAgent)):()=>!0,JO=Ie.hasStandardBrowserEnv?{write(e,t,n,r,o,i){const s=[e+"="+encodeURIComponent(t)];E.isNumber(n)&&s.push("expires="+new Date(n).toGMTString()),E.isString(r)&&s.push("path="+r),E.isString(o)&&s.push("domain="+o),i===!0&&s.push("secure"),document.cookie=s.join("; ")},read(e){const t=document.cookie.match(new RegExp("(^|;\\s*)("+e+")=([^;]*)"));return t?decodeURIComponent(t[3]):null},remove(e){this.write(e,"",Date.now()-864e5)}}:{write(){},read(){return null},remove(){}};function ZO(e){return/^([a-z][a-z\d+\-.]*:)?\/\//i.test(e)}function eP(e,t){return t?e.replace(/\/?\/$/,"")+"/"+t.replace(/^\/+/,""):e}function cg(e,t,n){let r=!ZO(t);return e&&(r||n==!1)?eP(e,t):t}const ug=e=>e instanceof Me?{...e}:e;function In(e,t){t=t||{};const n={};function r(c,u,h,d){return E.isPlainObject(c)&&E.isPlainObject(u)?E.merge.call({caseless:d},c,u):E.isPlainObject(u)?E.merge({},u):E.isArray(u)?u.slice():u}function o(c,u,h,d){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c,h,d)}else return r(c,u,h,d)}function i(c,u){if(!E.isUndefined(u))return r(void 0,u)}function s(c,u){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c)}else return r(void 0,u)}function a(c,u,h){if(h in t)return r(c,u);if(h in e)return r(void 0,c)}const l={url:i,method:i,data:i,baseURL:s,transformRequest:s,transformResponse:s,paramsSerializer:s,timeout:s,timeoutMessage:s,withCredentials:s,withXSRFToken:s,adapter:s,responseType:s,xsrfCookieName:s,xsrfHeaderName:s,onUploadProgress:s,onDownloadProgress:s,decompress:s,maxContentLength:s,maxBodyLength:s,beforeRedirect:s,transport:s,httpAgent:s,httpsAgent:s,cancelToken:s,socketPath:s,responseEncoding:s,validateStatus:a,headers:(c,u,h)=>o(ug(c),ug(u),h,!0)};return E.forEach(Object.keys({...e,...t}),function(u){const h=l[u]||o,d=h(e[u],t[u],u);E.isUndefined(d)&&h!==a||(n[u]=d)}),n}const dg=e=>{const t=In({},e);let{data:n,withXSRFToken:r,xsrfHeaderName:o,xsrfCookieName:i,headers:s,auth:a}=t;if(t.headers=s=Me.from(s),t.url=eg(cg(t.baseURL,t.url,t.allowAbsoluteUrls),e.params,e.paramsSerializer),a&&s.set("Authorization","Basic "+btoa((a.username||"")+":"+(a.password?unescape(encodeURIComponent(a.password)):""))),E.isFormData(n)){if(Ie.hasStandardBrowserEnv||Ie.hasStandardBrowserWebWorkerEnv)s.setContentType(void 0);else if(E.isFunction(n.getHeaders)){const l=n.getHeaders(),c=["content-type","content-length"];Object.entries(l).forEach(([u,h])=>{c.includes(u.toLowerCase())&&s.set(u,h)})}}if(Ie.hasStandardBrowserEnv&&(r&&E.isFunction(r)&&(r=r(t)),r||r!==!1&&QO(t.url))){const l=o&&i&&JO.read(i);l&&s.set(o,l)}return t},tP=typeof XMLHttpRequest<"u"&&function(e){return new Promise(function(n,r){const o=dg(e);let i=o.data;const s=Me.from(o.headers).normalize();let{responseType:a,onUploadProgress:l,onDownloadProgress:c}=o,u,h,d,m,f;function p(){m&&m(),f&&f(),o.cancelToken&&o.cancelToken.unsubscribe(u),o.signal&&o.signal.removeEventListener("abort",u)}let v=new XMLHttpRequest;v.open(o.method.toUpperCase(),o.url,!0),v.timeout=o.timeout;function b(){if(!v)return;const x=Me.from("getAllResponseHeaders"in v&&v.getAllResponseHeaders()),k={data:!a||a==="text"||a==="json"?v.responseText:v.response,status:v.status,statusText:v.statusText,headers:x,config:e,request:v};sg(function(R){n(R),p()},function(R){r(R),p()},k),v=null}"onloadend"in v?v.onloadend=b:v.onreadystatechange=function(){!v||v.readyState!==4||v.status===0&&!(v.responseURL&&v.responseURL.indexOf("file:")===0)||setTimeout(b)},v.onabort=function(){v&&(r(new H("Request aborted",H.ECONNABORTED,e,v)),v=null)},v.onerror=function(w){const k=w&&w.message?w.message:"Network Error",O=new H(k,H.ERR_NETWORK,e,v);O.event=w||null,r(O),v=null},v.ontimeout=function(){let w=o.timeout?"timeout of "+o.timeout+"ms exceeded":"timeout exceeded";const k=o.transitional||ng;o.timeoutErrorMessage&&(w=o.timeoutErrorMessage),r(new H(w,k.clarifyTimeoutError?H.ETIMEDOUT:H.ECONNABORTED,e,v)),v=null},i===void 0&&s.setContentType(null),"setRequestHeader"in v&&E.forEach(s.toJSON(),function(w,k){v.setRequestHeader(k,w)}),E.isUndefined(o.withCredentials)||(v.withCredentials=!!o.withCredentials),a&&a!=="json"&&(v.responseType=o.responseType),c&&([d,f]=Oi(c,!0),v.addEventListener("progress",d)),l&&v.upload&&([h,m]=Oi(l),v.upload.addEventListener("progress",h),v.upload.addEventListener("loadend",m)),(o.cancelToken||o.signal)&&(u=x=>{v&&(r(!x||x.type?new lr(null,e,v):x),v.abort(),v=null)},o.cancelToken&&o.cancelToken.subscribe(u),o.signal&&(o.signal.aborted?u():o.signal.addEventListener("abort",u)));const y=KO(o.url);if(y&&Ie.protocols.indexOf(y)===-1){r(new H("Unsupported protocol "+y+":",H.ERR_BAD_REQUEST,e));return}v.send(i||null)})},nP=(e,t)=>{const{length:n}=e=e?e.filter(Boolean):[];if(t||n){let r=new AbortController,o;const i=function(c){if(!o){o=!0,a();const u=c instanceof Error?c:this.reason;r.abort(u instanceof H?u:new lr(u instanceof Error?u.message:u))}};let s=t&&setTimeout(()=>{s=null,i(new H(`timeout ${t} of ms exceeded`,H.ETIMEDOUT))},t);const a=()=>{e&&(s&&clearTimeout(s),s=null,e.forEach(c=>{c.unsubscribe?c.unsubscribe(i):c.removeEventListener("abort",i)}),e=null)};e.forEach(c=>c.addEventListener("abort",i));const{signal:l}=r;return l.unsubscribe=()=>E.asap(a),l}},rP=function*(e,t){let n=e.byteLength;if(n{const o=oP(e,t);let i=0,s,a=l=>{s||(s=!0,r&&r(l))};return new ReadableStream({async pull(l){try{const{done:c,value:u}=await o.next();if(c){a(),l.close();return}let h=u.byteLength;if(n){let d=i+=h;n(d)}l.enqueue(new Uint8Array(u))}catch(c){throw a(c),c}},cancel(l){return a(l),o.return()}},{highWaterMark:2})},fg=64*1024,{isFunction:Pi}=E,sP=(({Request:e,Response:t})=>({Request:e,Response:t}))(E.global),{ReadableStream:gg,TextEncoder:pg}=E.global,mg=(e,...t)=>{try{return!!e(...t)}catch{return!1}},aP=e=>{e=E.merge.call({skipUndefined:!0},sP,e);const{fetch:t,Request:n,Response:r}=e,o=t?Pi(t):typeof fetch=="function",i=Pi(n),s=Pi(r);if(!o)return!1;const a=o&&Pi(gg),l=o&&(typeof pg=="function"?(f=>p=>f.encode(p))(new pg):async f=>new Uint8Array(await new n(f).arrayBuffer())),c=i&&a&&mg(()=>{let f=!1;const p=new n(Ie.origin,{body:new gg,method:"POST",get duplex(){return f=!0,"half"}}).headers.has("Content-Type");return f&&!p}),u=s&&a&&mg(()=>E.isReadableStream(new r("").body)),h={stream:u&&(f=>f.body)};o&&["text","arrayBuffer","blob","formData","stream"].forEach(f=>{!h[f]&&(h[f]=(p,v)=>{let b=p&&p[f];if(b)return b.call(p);throw new H(`Response type '${f}' is not supported`,H.ERR_NOT_SUPPORT,v)})});const d=async f=>{if(f==null)return 0;if(E.isBlob(f))return f.size;if(E.isSpecCompliantForm(f))return(await new n(Ie.origin,{method:"POST",body:f}).arrayBuffer()).byteLength;if(E.isArrayBufferView(f)||E.isArrayBuffer(f))return f.byteLength;if(E.isURLSearchParams(f)&&(f=f+""),E.isString(f))return(await l(f)).byteLength},m=async(f,p)=>{const v=E.toFiniteNumber(f.getContentLength());return v??d(p)};return async f=>{let{url:p,method:v,data:b,signal:y,cancelToken:x,timeout:w,onDownloadProgress:k,onUploadProgress:O,responseType:R,headers:I,withCredentials:N="same-origin",fetchOptions:A}=dg(f),T=t||fetch;R=R?(R+"").toLowerCase():"text";let S=nP([y,x&&x.toAbortSignal()],w),_=null;const D=S&&S.unsubscribe&&(()=>{S.unsubscribe()});let z;try{if(O&&c&&v!=="get"&&v!=="head"&&(z=await m(I,b))!==0){let Z=new n(p,{method:"POST",body:b,duplex:"half"}),pe;if(E.isFormData(b)&&(pe=Z.headers.get("content-type"))&&I.setContentType(pe),Z.body){const[Je,yt]=ag(z,Oi(lg(O)));b=hg(Z.body,fg,Je,yt)}}E.isString(N)||(N=N?"include":"omit");const W=i&&"credentials"in n.prototype,J={...A,signal:S,method:v.toUpperCase(),headers:I.normalize().toJSON(),body:b,duplex:"half",credentials:W?N:void 0};_=i&&new n(p,J);let ee=await(i?T(_,A):T(p,J));const $=u&&(R==="stream"||R==="response");if(u&&(k||$&&D)){const Z={};["status","statusText","headers"].forEach(_t=>{Z[_t]=ee[_t]});const pe=E.toFiniteNumber(ee.headers.get("content-length")),[Je,yt]=k&&ag(pe,Oi(lg(k),!0))||[];ee=new r(hg(ee.body,fg,Je,()=>{yt&&yt(),D&&D()}),Z)}R=R||"text";let q=await h[E.findKey(h,R)||"text"](ee,f);return!$&&D&&D(),await new Promise((Z,pe)=>{sg(Z,pe,{data:q,headers:Me.from(ee.headers),status:ee.status,statusText:ee.statusText,config:f,request:_})})}catch(W){throw D&&D(),W&&W.name==="TypeError"&&/Load failed|fetch/i.test(W.message)?Object.assign(new H("Network Error",H.ERR_NETWORK,f,_),{cause:W.cause||W}):H.from(W,W&&W.code,f,_)}}},lP=new Map,vg=e=>{let t=e?e.env:{};const{fetch:n,Request:r,Response:o}=t,i=[r,o,n];let s=i.length,a=s,l,c,u=lP;for(;a--;)l=i[a],c=u.get(l),c===void 0&&u.set(l,c=a?new Map:aP(t)),u=c;return c};vg();const Xa={http:TO,xhr:tP,fetch:{get:vg}};E.forEach(Xa,(e,t)=>{if(e){try{Object.defineProperty(e,"name",{value:t})}catch{}Object.defineProperty(e,"adapterName",{value:t})}});const bg=e=>`- ${e}`,cP=e=>E.isFunction(e)||e===null||e===!1,yg={getAdapter:(e,t)=>{e=E.isArray(e)?e:[e];const{length:n}=e;let r,o;const i={};for(let s=0;s`adapter ${l} `+(c===!1?"is not supported by the environment":"is not available in the build"));let a=n?s.length>1?`since : -`+s.map(bg).join(` -`):" "+bg(s[0]):"as no adapter specified";throw new H("There is no suitable adapter to dispatch the request "+a,"ERR_NOT_SUPPORT")}return o},adapters:Xa};function Qa(e){if(e.cancelToken&&e.cancelToken.throwIfRequested(),e.signal&&e.signal.aborted)throw new lr(null,e)}function xg(e){return Qa(e),e.headers=Me.from(e.headers),e.data=Ya.call(e,e.transformRequest),["post","put","patch"].indexOf(e.method)!==-1&&e.headers.setContentType("application/x-www-form-urlencoded",!1),yg.getAdapter(e.adapter||qr.adapter,e)(e).then(function(r){return Qa(e),r.data=Ya.call(e,e.transformResponse,r),r.headers=Me.from(r.headers),r},function(r){return ig(r)||(Qa(e),r&&r.response&&(r.response.data=Ya.call(e,e.transformResponse,r.response),r.response.headers=Me.from(r.response.headers))),Promise.reject(r)})}const kg="1.12.2",Ri={};["object","boolean","number","function","string","symbol"].forEach((e,t)=>{Ri[e]=function(r){return typeof r===e||"a"+(t<1?"n ":" ")+e}});const Cg={};Ri.transitional=function(t,n,r){function o(i,s){return"[Axios v"+kg+"] Transitional option '"+i+"'"+s+(r?". "+r:"")}return(i,s,a)=>{if(t===!1)throw new H(o(s," has been removed"+(n?" in "+n:"")),H.ERR_DEPRECATED);return n&&!Cg[s]&&(Cg[s]=!0,console.warn(o(s," has been deprecated since v"+n+" and will be removed in the near future"))),t?t(i,s,a):!0}},Ri.spelling=function(t){return(n,r)=>(console.warn(`${r} is likely a misspelling of ${t}`),!0)};function uP(e,t,n){if(typeof e!="object")throw new H("options must be an object",H.ERR_BAD_OPTION_VALUE);const r=Object.keys(e);let o=r.length;for(;o-- >0;){const i=r[o],s=t[i];if(s){const a=e[i],l=a===void 0||s(a,i,e);if(l!==!0)throw new H("option "+i+" must be "+l,H.ERR_BAD_OPTION_VALUE);continue}if(n!==!0)throw new H("Unknown option "+i,H.ERR_BAD_OPTION)}}const Ii={assertOptions:uP,validators:Ri},mt=Ii.validators;let Tn=class{constructor(t){this.defaults=t||{},this.interceptors={request:new tg,response:new tg}}async request(t,n){try{return await this._request(t,n)}catch(r){if(r instanceof Error){let o={};Error.captureStackTrace?Error.captureStackTrace(o):o=new Error;const i=o.stack?o.stack.replace(/^.+\n/,""):"";try{r.stack?i&&!String(r.stack).endsWith(i.replace(/^.+\n.+\n/,""))&&(r.stack+=` -`+i):r.stack=i}catch{}}throw r}}_request(t,n){typeof t=="string"?(n=n||{},n.url=t):n=t||{},n=In(this.defaults,n);const{transitional:r,paramsSerializer:o,headers:i}=n;r!==void 0&&Ii.assertOptions(r,{silentJSONParsing:mt.transitional(mt.boolean),forcedJSONParsing:mt.transitional(mt.boolean),clarifyTimeoutError:mt.transitional(mt.boolean)},!1),o!=null&&(E.isFunction(o)?n.paramsSerializer={serialize:o}:Ii.assertOptions(o,{encode:mt.function,serialize:mt.function},!0)),n.allowAbsoluteUrls!==void 0||(this.defaults.allowAbsoluteUrls!==void 0?n.allowAbsoluteUrls=this.defaults.allowAbsoluteUrls:n.allowAbsoluteUrls=!0),Ii.assertOptions(n,{baseUrl:mt.spelling("baseURL"),withXsrfToken:mt.spelling("withXSRFToken")},!0),n.method=(n.method||this.defaults.method||"get").toLowerCase();let s=i&&E.merge(i.common,i[n.method]);i&&E.forEach(["delete","get","head","post","put","patch","common"],f=>{delete i[f]}),n.headers=Me.concat(s,i);const a=[];let l=!0;this.interceptors.request.forEach(function(p){typeof p.runWhen=="function"&&p.runWhen(n)===!1||(l=l&&p.synchronous,a.unshift(p.fulfilled,p.rejected))});const c=[];this.interceptors.response.forEach(function(p){c.push(p.fulfilled,p.rejected)});let u,h=0,d;if(!l){const f=[xg.bind(this),void 0];for(f.unshift(...a),f.push(...c),d=f.length,u=Promise.resolve(n);h{if(!r._listeners)return;let i=r._listeners.length;for(;i-- >0;)r._listeners[i](o);r._listeners=null}),this.promise.then=o=>{let i;const s=new Promise(a=>{r.subscribe(a),i=a}).then(o);return s.cancel=function(){r.unsubscribe(i)},s},t(function(i,s,a){r.reason||(r.reason=new lr(i,s,a),n(r.reason))})}throwIfRequested(){if(this.reason)throw this.reason}subscribe(t){if(this.reason){t(this.reason);return}this._listeners?this._listeners.push(t):this._listeners=[t]}unsubscribe(t){if(!this._listeners)return;const n=this._listeners.indexOf(t);n!==-1&&this._listeners.splice(n,1)}toAbortSignal(){const t=new AbortController,n=r=>{t.abort(r)};return this.subscribe(n),t.signal.unsubscribe=()=>this.unsubscribe(n),t.signal}static source(){let t;return{token:new wp(function(o){t=o}),cancel:t}}};function hP(e){return function(n){return e.apply(null,n)}}function fP(e){return E.isObject(e)&&e.isAxiosError===!0}const Ja={Continue:100,SwitchingProtocols:101,Processing:102,EarlyHints:103,Ok:200,Created:201,Accepted:202,NonAuthoritativeInformation:203,NoContent:204,ResetContent:205,PartialContent:206,MultiStatus:207,AlreadyReported:208,ImUsed:226,MultipleChoices:300,MovedPermanently:301,Found:302,SeeOther:303,NotModified:304,UseProxy:305,Unused:306,TemporaryRedirect:307,PermanentRedirect:308,BadRequest:400,Unauthorized:401,PaymentRequired:402,Forbidden:403,NotFound:404,MethodNotAllowed:405,NotAcceptable:406,ProxyAuthenticationRequired:407,RequestTimeout:408,Conflict:409,Gone:410,LengthRequired:411,PreconditionFailed:412,PayloadTooLarge:413,UriTooLong:414,UnsupportedMediaType:415,RangeNotSatisfiable:416,ExpectationFailed:417,ImATeapot:418,MisdirectedRequest:421,UnprocessableEntity:422,Locked:423,FailedDependency:424,TooEarly:425,UpgradeRequired:426,PreconditionRequired:428,TooManyRequests:429,RequestHeaderFieldsTooLarge:431,UnavailableForLegalReasons:451,InternalServerError:500,NotImplemented:501,BadGateway:502,ServiceUnavailable:503,GatewayTimeout:504,HttpVersionNotSupported:505,VariantAlsoNegotiates:506,InsufficientStorage:507,LoopDetected:508,NotExtended:510,NetworkAuthenticationRequired:511};Object.entries(Ja).forEach(([e,t])=>{Ja[t]=e});function wg(e){const t=new Tn(e),n=Mf(Tn.prototype.request,t);return E.extend(n,Tn.prototype,t,{allOwnKeys:!0}),E.extend(n,t,null,{allOwnKeys:!0}),n.create=function(o){return wg(In(e,o))},n}const ce=wg(qr);ce.Axios=Tn,ce.CanceledError=lr,ce.CancelToken=dP,ce.isCancel=ig,ce.VERSION=kg,ce.toFormData=Si,ce.AxiosError=H,ce.Cancel=ce.CanceledError,ce.all=function(t){return Promise.all(t)},ce.spread=hP,ce.isAxiosError=fP,ce.mergeConfig=In,ce.AxiosHeaders=Me,ce.formToJSON=e=>rg(E.isHTMLForm(e)?new FormData(e):e),ce.getAdapter=yg.getAdapter,ce.HttpStatusCode=Ja,ce.default=ce;const{Axios:HT,AxiosError:UT,CanceledError:GT,isCancel:qT,CancelToken:KT,VERSION:YT,all:XT,Cancel:QT,isAxiosError:JT,spread:ZT,toFormData:e5,AxiosHeaders:t5,HttpStatusCode:n5,formToJSON:r5,getAdapter:o5,mergeConfig:i5}=ce;var gP=(e,t,n,r,o,i,s,a)=>{let l=document.documentElement,c=["light","dark"];function u(m){(Array.isArray(e)?e:[e]).forEach(f=>{let p=f==="class",v=p&&i?o.map(b=>i[b]||b):o;p?(l.classList.remove(...v),l.classList.add(i&&i[m]?i[m]:m)):l.setAttribute(f,m)}),h(m)}function h(m){a&&c.includes(m)&&(l.style.colorScheme=m)}function d(){return window.matchMedia("(prefers-color-scheme: dark)").matches?"dark":"light"}if(r)u(r);else try{let m=localStorage.getItem(t)||n,f=s&&m==="system"?d():m;u(f)}catch{}},Sg=["light","dark"],Eg="(prefers-color-scheme: dark)",pP=typeof window>"u",Og=C.createContext(void 0),mP=e=>C.useContext(Og)?C.createElement(C.Fragment,null,e.children):C.createElement(bP,{...e}),vP=["light","dark"],bP=({forcedTheme:e,disableTransitionOnChange:t=!1,enableSystem:n=!0,enableColorScheme:r=!0,storageKey:o="theme",themes:i=vP,defaultTheme:s=n?"system":"light",attribute:a="data-theme",value:l,children:c,nonce:u,scriptProps:h})=>{let[d,m]=C.useState(()=>xP(o,s)),[f,p]=C.useState(()=>d==="system"?Za():d),v=l?Object.values(l):i,b=C.useCallback(k=>{let O=k;if(!O)return;k==="system"&&n&&(O=Za());let R=l?l[O]:O,I=t?kP(u):null,N=document.documentElement,A=T=>{T==="class"?(N.classList.remove(...v),R&&N.classList.add(R)):T.startsWith("data-")&&(R?N.setAttribute(T,R):N.removeAttribute(T))};if(Array.isArray(a)?a.forEach(A):A(a),r){let T=Sg.includes(s)?s:null,S=Sg.includes(O)?O:T;N.style.colorScheme=S}I?.()},[u]),y=C.useCallback(k=>{let O=typeof k=="function"?k(d):k;m(O);try{localStorage.setItem(o,O)}catch{}},[d]),x=C.useCallback(k=>{let O=Za(k);p(O),d==="system"&&n&&!e&&b("system")},[d,e]);C.useEffect(()=>{let k=window.matchMedia(Eg);return k.addListener(x),x(k),()=>k.removeListener(x)},[x]),C.useEffect(()=>{let k=O=>{O.key===o&&(O.newValue?m(O.newValue):y(s))};return window.addEventListener("storage",k),()=>window.removeEventListener("storage",k)},[y]),C.useEffect(()=>{b(e??d)},[e,d]);let w=C.useMemo(()=>({theme:d,setTheme:y,forcedTheme:e,resolvedTheme:d==="system"?f:d,themes:n?[...i,"system"]:i,systemTheme:n?f:void 0}),[d,y,e,f,n,i]);return C.createElement(Og.Provider,{value:w},C.createElement(yP,{forcedTheme:e,storageKey:o,attribute:a,enableSystem:n,enableColorScheme:r,defaultTheme:s,value:l,themes:i,nonce:u,scriptProps:h}),c)},yP=C.memo(({forcedTheme:e,storageKey:t,attribute:n,enableSystem:r,enableColorScheme:o,defaultTheme:i,value:s,themes:a,nonce:l,scriptProps:c})=>{let u=JSON.stringify([n,t,i,e,a,s,r,o]).slice(1,-1);return C.createElement("script",{...c,suppressHydrationWarning:!0,nonce:typeof window>"u"?l:"",dangerouslySetInnerHTML:{__html:`(${gP.toString()})(${u})`}})}),xP=(e,t)=>{if(pP)return;let n;try{n=localStorage.getItem(e)||void 0}catch{}return n||t},kP=e=>{let t=document.createElement("style");return e&&t.setAttribute("nonce",e),t.appendChild(document.createTextNode("*,*::before,*::after{-webkit-transition:none!important;-moz-transition:none!important;-o-transition:none!important;-ms-transition:none!important;transition:none!important}")),document.head.appendChild(t),()=>{window.getComputedStyle(document.body),setTimeout(()=>{document.head.removeChild(t)},1)}},Za=e=>(e||(e=window.matchMedia(Eg)),e.matches?"dark":"light");const CP=e=>g.jsx(mP,{attribute:"class",disableTransitionOnChange:!0,...e});/** - * react-router v7.9.2 +)+\\(\\s*min(-device)?-${e}`,"i"),max:new RegExp(`\\(\\s*max(-device)?-${e}`,"i")}),yC=dh("width"),xC=dh("height"),hh=e=>({isMin:bh(e.minMax,e.maxMin,e.min),isMax:bh(e.maxMin,e.minMax,e.max)}),{isMin:Sa,isMax:fh}=hh(yC),{isMin:Ea,isMax:gh}=hh(xC),ph=/print/i,mh=/^print$/i,kC=/(-?\d*\.?\d+)(ch|em|ex|px|rem)/,CC=/(\d)/,$r=Number.MAX_VALUE,wC={ch:8.8984375,em:16,rem:16,ex:8.296875,px:1};function vh(e){const t=kC.exec(e)||(Sa(e)||Ea(e)?CC.exec(e):null);if(!t)return $r;if(t[0]==="0")return 0;const n=parseFloat(t[1]),r=t[2];return n*(wC[r]||1)}function bh(e,t,n){return r=>e.test(r)||!t.test(r)&&n.test(r)}function SC(e,t){const n=ph.test(e),r=mh.test(e),o=ph.test(t),i=mh.test(t);return n&&o?!r&&i?1:r&&!i?-1:e.localeCompare(t):n?1:o?-1:null}const EC=rt((e,t)=>{const n=SC(e,t);if(n!==null)return n;const r=Sa(e)||Ea(e),o=fh(e)||gh(e),i=Sa(t)||Ea(t),s=fh(t)||gh(t);if(r&&s)return-1;if(o&&i)return 1;const a=vh(e),l=vh(t);return a===$r&&l===$r?e.localeCompare(t):a===$r?1:l===$r?-1:a!==l?a>l?o?-1:1:o?1:-1:e.localeCompare(t)});function yh(e){return e.sort(([t],[n])=>EC(t,n))}function xh(e){const t=[],n=[],r={};for(const[s,a]of Object.entries(e))s.startsWith("@media")?t.push([s,a]):s.startsWith("@container")?n.push([s,a]):_e(a)?r[s]=xh(a):r[s]=a;const o=yh(t),i=yh(n);return{...r,...Object.fromEntries(o),...Object.fromEntries(i)}}const kh=/\s*!(important)?/i,OC=e=>lt(e)?kh.test(e):!1,PC=e=>lt(e)?e.replace(kh,"").trim():e;function Ch(e){const{transform:t,conditions:n,normalize:r}=e,o=TC(e);return rt(function(...s){const a=o(...s),l=r(a),c=Object.create(null);return gt(l,(u,h)=>{const d=OC(u);if(u==null)return;const[m,...f]=n.sort(h).map(n.resolve);d&&(u=PC(u));let p=t(m,u)??Object.create(null);p=gt(p,v=>lt(v)&&d?`${v} !important`:v,{getKey:v=>n.expandAtRule(v)}),RC(c,f.flat(),p)}),xh(c)})}function RC(e,t,n){let r=e;for(const o of t)o&&(r[o]||(r[o]=Object.create(null)),r=r[o]);er(r,n)}function IC(...e){return e.filter(t=>_e(t)&&Object.keys(yr(t)).length>0)}function TC(e){function t(n){const r=IC(...n);return r.length===1?r:r.map(o=>e.normalize(o))}return rt(function(...r){return er({},...t(r))})}const wh=e=>({base:{},variants:{},defaultVariants:{},compoundVariants:[],...e});function NC(e){const{css:t,conditions:n,normalize:r,layers:o}=e;function i(a={}){const{base:l,variants:c,defaultVariants:u,compoundVariants:h}=wh(a),d=Ch({conditions:n,normalize:r,transform(y,x){return c[y]?.[x]}}),m=(y={})=>{const x=r({...u,...yr(y)});let w={...l};er(w,d(x));const k=s(h,x);return o.wrap("recipes",t(w,k))},f=Object.keys(c),p=y=>{const x=Jd(y,["recipe"]),[w,k]=Fn(x,f);return f.includes("colorPalette")||(w.colorPalette=y.colorPalette||u.colorPalette),f.includes("orientation")&&(k.orientation=y.orientation),[w,k]},v=Object.fromEntries(Object.entries(c).map(([y,x])=>[y,Object.keys(x)]));return Object.assign(y=>t(m(y)),{className:a.className,__cva__:!0,variantMap:v,variantKeys:f,raw:m,config:a,splitVariantProps:p,merge(y){return i(AC(e)(this,y))}})}function s(a,l){let c={};return a.forEach(u=>{Object.entries(u).every(([d,m])=>d==="css"?!0:(Array.isArray(m)?m:[m]).some(p=>l[d]===p))&&(c=t(c,u.css))}),c}return i}function AC(e){const{css:t}=e;return function(r,o){const i=wh(o.config),s=Wl(r.variantKeys,Object.keys(o.variants)),a=t(r.base,i.base),l=Object.fromEntries(s.map(d=>[d,t(r.config.variants[d],i.variants[d])])),c=er(r.config.defaultVariants,i.defaultVariants),u=[...r.compoundVariants,...i.compoundVariants];return{className:Ge(r.className,o.className),base:a,variants:l,defaultVariants:c,compoundVariants:u}}}const _C={reset:"reset",base:"base",tokens:"tokens",recipes:"recipes"},Sh={reset:0,base:1,tokens:2,recipes:3};function VC(e){const t=e.layers??_C,r=Object.values(t).sort((o,i)=>Sh[o]-Sh[i]);return{names:r,atRule:`@layer ${r.join(", ")};`,wrap(o,i){return e.disableLayers?i:{[`@layer ${t[o]}`]:i}}}}function LC(e){const{utility:t,normalize:n}=e,{hasShorthand:r,resolveShorthand:o}=t;return function(i){return gt(i,n,{stop:s=>Array.isArray(s),getKey:r?o:void 0})}}function FC(e){const{preflight:t}=e;if(!t)return{};const{scope:n="",level:r="parent"}=_e(t)?t:{};let o="";n&&r==="parent"?o=`${n} `:n&&r==="element"&&(o=`&${n}`);const i={"*":{margin:"0px",padding:"0px",font:"inherit",wordWrap:"break-word",WebkitTapHighlightColor:"transparent"},"*, *::before, *::after, *::backdrop":{boxSizing:"border-box",borderWidth:"0px",borderStyle:"solid",borderColor:"var(--global-color-border, currentColor)"},hr:{height:"0px",color:"inherit",borderTopWidth:"1px"},body:{minHeight:"100dvh",position:"relative"},img:{borderStyle:"none"},"img, svg, video, canvas, audio, iframe, embed, object":{display:"block",verticalAlign:"middle"},iframe:{border:"none"},"img, video":{maxWidth:"100%",height:"auto"},"p, h1, h2, h3, h4, h5, h6":{overflowWrap:"break-word"},"ol, ul":{listStyle:"none"},"code, kbd, pre, samp":{fontSize:"1em"},"button, [type='button'], [type='reset'], [type='submit']":{WebkitAppearance:"button",backgroundColor:"transparent",backgroundImage:"none"},"button, input, optgroup, select, textarea":{color:"inherit"},"button, select":{textTransform:"none"},table:{textIndent:"0px",borderColor:"inherit",borderCollapse:"collapse"},"*::placeholder":{opacity:"unset",color:"#9ca3af",userSelect:"none"},textarea:{resize:"vertical"},summary:{display:"list-item"},small:{fontSize:"80%"},"sub, sup":{fontSize:"75%",lineHeight:0,position:"relative",verticalAlign:"baseline"},sub:{bottom:"-0.25em"},sup:{top:"-0.5em"},dialog:{padding:"0px"},a:{color:"inherit",textDecoration:"inherit"},"abbr:where([title])":{textDecoration:"underline dotted"},"b, strong":{fontWeight:"bolder"},"code, kbd, samp, pre":{fontSize:"1em","--font-mono-fallback":"ui-monospace, SFMono-Regular, Menlo, Monaco, Consolas, 'Liberation Mono', 'Courier New'",fontFamily:"var(--global-font-mono, var(--font-mono-fallback))"},'input[type="text"], input[type="email"], input[type="search"], input[type="password"]':{WebkitAppearance:"none",MozAppearance:"none"},"input[type='search']":{WebkitAppearance:"textfield",outlineOffset:"-2px"},"::-webkit-search-decoration, ::-webkit-search-cancel-button":{WebkitAppearance:"none"},"::-webkit-file-upload-button":{WebkitAppearance:"button",font:"inherit"},'input[type="number"]::-webkit-inner-spin-button, input[type="number"]::-webkit-outer-spin-button':{height:"auto"},"input[type='number']":{MozAppearance:"textfield"},":-moz-ui-invalid":{boxShadow:"none"},":-moz-focusring":{outline:"auto"},"[hidden]:where(:not([hidden='until-found']))":{display:"none !important"}},s={[n||"html"]:{lineHeight:1.5,"--font-fallback":"ui-sans-serif, system-ui, -apple-system, BlinkMacSystemFont, 'Segoe UI', Roboto, 'Helvetica Neue', Arial, 'Noto Sans', sans-serif, 'Apple Color Emoji', 'Segoe UI Emoji', 'Segoe UI Symbol', 'Noto Color Emoji'",WebkitTextSizeAdjust:"100%",WebkitFontSmoothing:"antialiased",MozOsxFontSmoothing:"grayscale",textRendering:"optimizeLegibility",touchAction:"manipulation",MozTabSize:"4",tabSize:"4",fontFamily:"var(--global-font-body, var(--font-fallback))"}};if(r==="element"){const a=Object.entries(i).reduce((l,[c,u])=>(l[c]={[o]:u},l),{});Object.assign(s,a)}else o?s[o]=i:Object.assign(s,i);return s}function DC(e){const{conditions:t,isValidProperty:n}=e;return function(o){return gt(o,i=>i,{getKey:(i,s)=>_e(s)&&!t.has(i)&&!n(i)?zC(i).map(a=>"&"+a).join(", "):i})}}function zC(e){const t=[];let n=0,r="",o=!1;for(let i=0;i{const t=o=>({base:e.base?.[o]??{},variants:{},defaultVariants:e.defaultVariants??{},compoundVariants:e.compoundVariants?jC(e.compoundVariants,o):[]}),r=(e.slots??[]).map(o=>[o,t(o)]);for(const[o,i]of Object.entries(e.variants??{}))for(const[s,a]of Object.entries(i))r.forEach(([l,c])=>{var u;(u=c.variants)[o]??(u[o]={}),c.variants[o][s]=a[l]??{}});return Object.fromEntries(r)},jC=(e,t)=>e.filter(n=>n.css[t]).map(n=>({...n,css:n.css[t]}));function $C(e){const{cva:t}=e;return function(r={}){const o=Object.entries(MC(r)).map(([h,d])=>[h,t(d)]);function i(h){const d=o.map(([m,f])=>[m,f(h)]);return Object.fromEntries(d)}const s=r.variants??{},a=Object.keys(s);function l(h){const d=Jd(h,["recipe"]),[m,f]=Fn(d,a);return a.includes("colorPalette")||(m.colorPalette=h.colorPalette||r.defaultVariants?.colorPalette),a.includes("orientation")&&(f.orientation=h.orientation),[m,f]}const c=Object.fromEntries(Object.entries(s).map(([h,d])=>[h,Object.keys(d)]));let u={};return r.className&&(u=Object.fromEntries(r.slots.map(h=>[h,`${r.className}__${h}`]))),Object.assign(i,{variantMap:c,variantKeys:a,splitVariantProps:l,classNameMap:u})}}const BC=()=>e=>Array.from(new Set(e)),WC=/([\0-\x1f\x7f]|^-?\d)|^-$|^-|[^\x80-\uFFFF\w-]/g,HC=function(e,t){return t?e==="\0"?"�":e==="-"&&e.length===1?"\\-":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16):"\\"+e},Eh=e=>(e+"").replace(WC,HC),Oh=(e,t)=>{let n="",r=0,o="char",i="",s="";const a=[];for(;r{let t=0;const n=["("];for(;t{n instanceof Map?t[r]=Object.fromEntries(n):t[r]=n}),t}const Rh=/({([^}]*)})/g,GC=/[{}]/g,qC=/\w+\.\w+/,Ih=e=>{if(!lt(e))return[];const t=e.match(Rh);return t?t.map(n=>n.replace(GC,"")).map(n=>n.trim()):[]},KC=e=>Rh.test(e);function Th(e){if(!e.extensions?.references)return e.extensions?.cssVar?.ref??e.value;const t=e.extensions.references??{};return e.value=Object.keys(t).reduce((n,r)=>{const o=t[r];if(o.extensions.conditions)return n;const i=Th(o);return n.replace(`{${r}}`,i)},e.value),delete e.extensions.references,e.value}function Nh(e){return _e(e)&&e.reference?e.reference:String(e)}const ui=(e,...t)=>t.map(Nh).join(` ${e} `).replace(/calc/g,""),Ah=(...e)=>`calc(${ui("+",...e)})`,_h=(...e)=>`calc(${ui("-",...e)})`,Oa=(...e)=>`calc(${ui("*",...e)})`,Vh=(...e)=>`calc(${ui("/",...e)})`,Lh=e=>{const t=Nh(e);return t!=null&&!Number.isNaN(parseFloat(t))?String(t).startsWith("-")?String(t).slice(1):`-${t}`:Oa(t,-1)},or=Object.assign(e=>({add:(...t)=>or(Ah(e,...t)),subtract:(...t)=>or(_h(e,...t)),multiply:(...t)=>or(Oa(e,...t)),divide:(...t)=>or(Vh(e,...t)),negate:()=>or(Lh(e)),toString:()=>e.toString()}),{add:Ah,subtract:_h,multiply:Oa,divide:Vh,negate:Lh}),YC={enforce:"pre",transform(e){const{prefix:t,allTokens:n,formatCssVar:r,formatTokenName:o,registerToken:i}=e;n.filter(({extensions:a})=>a.category==="spacing").forEach(a=>{const l=a.path.slice(),c=r(l,t);if(lt(a.value)&&a.value==="0rem")return;const u=structuredClone(a);Object.assign(u.extensions,{negative:!0,prop:`-${a.extensions.prop}`,originalPath:l}),u.value=or.negate(c.ref);const h=u.path[u.path.length-1];h!=null&&(u.path[u.path.length-1]=`-${h}`),u.path&&(u.name=o(u.path)),i(u)})}},XC=new Set(["spacing","sizes","borderWidths","fontSizes","radii"]),QC=[YC,{enforce:"post",transform(e){const{allTokens:t,registerToken:n,formatTokenName:r}=e,o=t.filter(({extensions:a})=>a.category==="colors"),i=new Map,s=new Map;o.forEach(a=>{const{colorPalette:l}=a.extensions;l&&(l.keys.forEach(c=>{i.set(r(c),c)}),l.roots.forEach(c=>{const u=r(c),h=s.get(u)||[];if(h.push(a),s.set(u,h),a.extensions.default&&c.length===1){const d=l.keys[0]?.filter(Boolean);if(!d.length)return;const m=c.concat(d);i.set(r(m),[])}}))}),i.forEach(a=>{const l=["colors","colorPalette",...a].filter(Boolean),c=r(l),u=r(l.slice(1));n({name:c,value:c,originalValue:c,path:l,extensions:{condition:"base",originalPath:l,category:"colors",prop:u,virtual:!0}},"pre")})}},{enforce:"post",transform(e){e.allTokens.filter(n=>XC.has(n.extensions.category)&&!n.extensions.negative).forEach(n=>{Object.assign(n.extensions,{pixelValue:ch(n.value)})})}},{enforce:"post",transform(e){e.allTokens=e.allTokens.filter(t=>t.value!=="")}}],JC=[{type:"extensions",enforce:"pre",name:"tokens/css-var",transform(e,t){const{prefix:n,formatCssVar:r}=t,{negative:o,originalPath:i}=e.extensions,s=o?i:e.path;return{cssVar:r(s.filter(Boolean),n)}}},{enforce:"post",type:"value",name:"tokens/conditionals",transform(e,t){const{prefix:n,formatCssVar:r}=t,o=Ih(e.value);return o.length&&o.forEach(i=>{const s=r(i.split("."),n);e.value=e.value.replace(`{${s.ref}}`,s)}),e.value}},{type:"extensions",enforce:"pre",name:"tokens/colors/colorPalette",match(e){return e.extensions.category==="colors"&&!e.extensions.virtual},transform(e,t){let n=e.path.slice();if(n.pop(),n.shift(),n.length===0){const a=[...e.path];a.shift(),n=a}if(n.length===0)return{};const r=n.reduce((a,l,c,u)=>{const h=u.slice(0,c+1);return a.push(h),a},[]),o=n[0],i=t.formatTokenName(n),s=e.path.slice(e.path.indexOf(o)+1).reduce((a,l,c,u)=>(a.push(u.slice(c)),a),[]);return s.length===0&&s.push([""]),{colorPalette:{value:i,roots:r,keys:s}}}}],Fh=e=>_e(e)&&Object.prototype.hasOwnProperty.call(e,"value");function ZC(e){return e?{breakpoints:Zd(e,t=>({value:t})),sizes:Object.fromEntries(Object.entries(e).map(([t,n])=>[`breakpoint-${t}`,{value:n}]))}:{breakpoints:{},sizes:{}}}function ew(e){const{prefix:t="",tokens:n={},semanticTokens:r={},breakpoints:o={}}=e,i=F=>F.join("."),s=(F,L)=>nh(F.join("-"),{prefix:L}),a=[],l=new Map,c=new Map,u=new Map,h=new Map,d=new Map,m=new Map,f=new Map,p=new Map,v=[];function b(F,L){a.push(F),l.set(F.name,F),L&&p.forEach(K=>{K.enforce===L&&q(K,F)})}const y=ZC(o),x=yr({...n,breakpoints:y.breakpoints,sizes:{...n.sizes,...y.sizes}});function w(){gt(x,(F,L)=>{const K=L.includes("DEFAULT");L=Dh(L);const ie=L[0],me=i(L),Oe=lt(F)?{value:F}:F,Vt={value:Oe.value,originalValue:Oe.value,name:me,path:L,extensions:{condition:"base",originalPath:L,category:ie,prop:i(L.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh}),gt(r,(F,L)=>{const K=L.includes("DEFAULT");L=zh(Dh(L));const ie=L[0],me=i(L),Oe=lt(F.value)?{value:{base:F.value}}:F,Vt={value:Oe.value.base||"",originalValue:Oe.value.base||"",name:me,path:L,extensions:{originalPath:L,category:ie,conditions:Oe.value,condition:"base",prop:i(L.slice(1))}};K&&(Vt.extensions.default=!0),b(Vt)},{stop:Fh})}function k(F){return l.get(F)}function O(F){const{condition:L}=F.extensions;L&&(c.has(L)||c.set(L,new Set),c.get(L).add(F))}function R(F){const{category:L,prop:K}=F.extensions;L&&(f.has(L)||f.set(L,new Map),f.get(L).set(K,F))}function I(F){const{condition:L,negative:K,virtual:ie,cssVar:me}=F.extensions;K||ie||!L||!me||(u.has(L)||u.set(L,new Map),u.get(L).set(me.var,F.value))}function N(F){const{category:L,prop:K,cssVar:ie,negative:me}=F.extensions;if(!L)return;m.has(L)||m.set(L,new Map);const Oe=me?F.extensions.conditions?F.originalValue:F.value:ie.ref;m.get(L).set(K,Oe),d.set([L,K].join("."),Oe)}function A(F){const{colorPalette:L,virtual:K,default:ie}=F.extensions;!L||K||L.roots.forEach(me=>{const Oe=i(me);h.has(Oe)||h.set(Oe,new Map);const Vt=nw([...F.path],[...me]),Wi=i(Vt),gl=k(Wi);if(!gl||!gl.extensions.cssVar)return;const{var:mT}=gl.extensions.cssVar;if(h.get(Oe).set(mT,F.extensions.cssVar.ref),ie&&me.length===1){const vT=i(["colors","colorPalette"]),yp=k(vT);if(!yp)return;const bT=i(F.path),xp=k(bT);if(!xp)return;const kp=L.keys[0]?.filter(Boolean);if(!kp.length)return;const pl=i(me.concat(kp));h.has(pl)||h.set(pl,new Map),h.get(pl).set(yp.extensions.cssVar.var,xp.extensions.cssVar.ref)}})}let T={};function S(){a.forEach(F=>{O(F),R(F),I(F),N(F),A(F)}),T=Ph(m)}const _=(F,L)=>{if(!F||typeof F!="string")return{invalid:!0,value:F};const[K,ie]=F.split("/");if(!K||!ie)return{invalid:!0,value:K};const me=L(K),Oe=k(`opacity.${ie}`)?.value;if(!Oe&&isNaN(Number(ie)))return{invalid:!0,value:K};const Vt=Oe?Number(Oe)*100+"%":`${ie}%`,Wi=me??K;return{invalid:!1,color:Wi,value:`color-mix(in srgb, ${Wi} ${Vt}, transparent)`}},D=rt((F,L)=>d.get(F)??L),z=rt(F=>T[F]||null),W=rt(F=>Oh(F,L=>{if(!L)return;if(L.includes("/")){const ie=_(L,me=>D(me));if(ie.invalid)throw new Error("Invalid color mix at "+L+": "+ie.value);return ie.value}const K=D(L);return K||(qC.test(L)?Eh(L):L)})),J={prefix:t,allTokens:a,tokenMap:l,registerToken:b,getByName:k,formatTokenName:i,formatCssVar:s,flatMap:d,cssVarMap:u,categoryMap:f,colorPaletteMap:h,getVar:D,getCategoryValues:z,expandReferenceInValue:W};function ee(...F){F.forEach(L=>{p.set(L.name,L)})}function $(...F){v.push(...F)}function q(F,L){if(L.extensions.references||cs(F.match)&&!F.match(L))return;const ie=(me=>F.transform(me,J))(L);switch(!0){case F.type==="extensions":Object.assign(L.extensions,ie);break;case F.type==="value":L.value=ie;break;default:L[F.type]=ie;break}}function Z(F){v.forEach(L=>{L.enforce===F&&L.transform(J)})}function pe(F){p.forEach(L=>{L.enforce===F&&a.forEach(K=>{q(L,K)})})}function Je(){a.forEach(F=>{const L=tw(F);!L||L.length===0||L.forEach(K=>{b(K)})})}function xt(F){return Ih(F).map(K=>k(K)).filter(Boolean)}function _t(){a.forEach(F=>{if(!KC(F.value))return;const L=xt(F.value);F.extensions.references=L.reduce((K,ie)=>(K[ie.name]=ie,K),{})})}function fl(){a.forEach(F=>{Th(F)})}function pT(){Z("pre"),pe("pre"),Je(),_t(),fl(),Z("post"),pe("post"),S()}return w(),ee(...JC),$(...QC),pT(),J}function Dh(e){return e[0]==="DEFAULT"?e:e.filter(t=>t!=="DEFAULT")}function zh(e){return e.filter(t=>t!=="base")}function tw(e){if(!e.extensions.conditions)return;const{conditions:t}=e.extensions,n=[];return gt(t,(r,o)=>{const i=zh(o);if(!i.length)return;const s=structuredClone(e);s.value=r,s.extensions.condition=i.join(":"),n.push(s)}),n}function nw(e,t){const n=e.findIndex((r,o)=>t.every((i,s)=>e[o+s]===i));return n===-1||(e.splice(n,t.length),e.splice(n,0,"colorPalette")),e}BC()(["aspectRatios","zIndex","opacity","colors","fonts","fontSizes","fontWeights","lineHeights","letterSpacings","sizes","shadows","spacing","radii","cursor","borders","borderWidths","borderStyles","durations","easings","animations","blurs","gradients","breakpoints","assets"]);function _T(e){return e}function rw(e){return Object.fromEntries(Object.entries(e).map(([t,n])=>[t,n]))}function ow(e){const t=rw(e.config),n=e.tokens,r=new Map,o=new Map;function i(I,N){t[I]=N,s(I,N)}const s=(I,N)=>{const A=p(N);A&&(o.set(I,A),h(I,N))},a=()=>{for(const[I,N]of Object.entries(t))N&&s(I,N)},l=()=>{for(const[I,N]of Object.entries(t)){const{shorthand:A}=N??{};if(!A)continue;(Array.isArray(A)?A:[A]).forEach(S=>r.set(S,I))}},c=()=>{const I=Ph(n.colorPaletteMap);i("colorPalette",{values:Object.keys(I),transform:rt(N=>I[N])})},u=new Map,h=(I,N)=>{if(!N)return;const A=p(N,S=>`type:Tokens["${S}"]`);if(typeof A=="object"&&A.type){u.set(I,new Set([`type:${A.type}`]));return}if(A){const S=new Set(Object.keys(A));u.set(I,S)}const T=u.get(I)??new Set;N.property&&u.set(I,T.add(`CssProperties["${N.property}"]`))},d=()=>{for(const[I,N]of Object.entries(t))N&&h(I,N)},m=(I,N)=>{const A=u.get(I)??new Set;u.set(I,new Set([...A,...N]))},f=()=>{const I=new Map;for(const[N,A]of u.entries()){if(A.size===0){I.set(N,["string"]);continue}const T=Array.from(A).map(S=>S.startsWith("CssProperties")?S:S.startsWith("type:")?S.replace("type:",""):JSON.stringify(S));I.set(N,T)}return I},p=(I,N)=>{const{values:A}=I,T=S=>{const _=N?.(S);return _?{[_]:_}:void 0};return lt(A)?T?.(A)??n.getCategoryValues(A)??{}:Array.isArray(A)?A.reduce((S,_)=>(S[_]=_,S),{}):cs(A)?A(N?T:n.getCategoryValues):A},v=rt((I,N)=>({[I]:I.startsWith("--")?n.getVar(N,N):N})),b=Object.assign(n.getVar,{raw:I=>n.getByName(I)}),y=rt((I,N)=>{const A=k(I);lt(N)&&!N.includes("_EMO_")&&(N=n.expandReferenceInValue(N));const T=t[A];if(!T)return v(A,N);const S=o.get(A)?.[N];if(!T.transform)return v(I,S??N);const _=D=>Mk(D,b);return T.transform(S??N,{raw:N,token:b,utils:{colorMix:_}})});function x(){l(),c(),a(),d()}x();const w=r.size>0,k=rt(I=>r.get(I)??I);return{keys:()=>[...Array.from(r.keys()),...Object.keys(t)],hasShorthand:w,transform:y,shorthands:r,resolveShorthand:k,register:i,getTypes:f,addPropertyType:m}}const De={};function Mh(...e){const t=eh(...e),{theme:n={},utilities:r={},globalCss:o={},cssVarsRoot:i=":where(:root, :host)",cssVarsPrefix:s="chakra",preflight:a}=t,l=VC(t),c=ew({breakpoints:n.breakpoints,tokens:n.tokens,semanticTokens:n.semanticTokens,prefix:s}),u=gC(n.breakpoints??De),h=bC({conditions:t.conditions??De,breakpoints:u}),d=ow({config:r,tokens:c});function m(){const{textStyles:$,layerStyles:q,animationStyles:Z}=n,pe=yr({textStyle:$,layerStyle:q,animationStyle:Z});for(const[Je,xt]of Object.entries(pe)){const _t=ih(xt??De,jh);d.register(Je,{values:Object.keys(_t),transform(fl){return x(_t[fl])}})}}m(),d.addPropertyType("animationName",Object.keys(n.keyframes??De));const f=new Set(["css",...d.keys(),...h.keys()]),p=rt($=>f.has($)||cC($)),v=$=>Array.isArray($)?$.reduce((q,Z,pe)=>{const Je=h.breakpoints[pe];return Z!=null&&(q[Je]=Z),q},{}):$,b=LC({utility:d,normalize:v}),y=DC({conditions:h,isValidProperty:p}),x=Ch({transform:d.transform,conditions:h,normalize:b}),w=NC({css:x,conditions:h,normalize:b,layers:l}),k=$C({cva:w});function O(){const $={};for(const[q,Z]of c.cssVarMap.entries()){const pe=Object.fromEntries(Z);if(Object.keys(pe).length===0)continue;const Je=q==="base"?i:h.resolve(q),xt=Je.startsWith("@"),_t=x(y({[Je]:xt?{[i]:pe}:pe}));er($,_t)}return l.wrap("tokens",$)}function R(){const $=Object.fromEntries(Object.entries(n.keyframes??De).map(([Z,pe])=>[`@keyframes ${Z}`,pe])),q=Object.assign({},$,x(y(o)));return l.wrap("base",q)}function I($){return Fn($,p)}function N(){const $=FC({preflight:a});return l.wrap("reset",$)}const A=iw(c),T=($,q)=>A.get($)?.value||q;T.var=($,q)=>A.get($)?.variable||q;function S($,q){return n.recipes?.[$]??q}function _($,q){return n.slotRecipes?.[$]??q}function D($){return Object.hasOwnProperty.call(n.recipes??De,$)}function z($){return Object.hasOwnProperty.call(n.slotRecipes??De,$)}function W($){return D($)||z($)}const J=[N(),R(),O()],ee={layerStyles:Pa(n.layerStyles??De),textStyles:Pa(n.textStyles??De),animationStyles:Pa(n.animationStyles??De),tokens:$h(c,Object.keys(n.tokens??De),($,q)=>!$.extensions.conditions&&!q.includes("colorPalette")),semanticTokens:$h(c,Object.keys(n.semanticTokens??De),$=>!!$.extensions.conditions),keyframes:Bh(n.keyframes??De),breakpoints:Bh(n.breakpoints??De)};return{$$chakra:!0,_config:t,_global:J,breakpoints:u,tokens:c,conditions:h,utility:d,token:T,properties:f,layers:l,isValidProperty:p,splitCssProps:I,normalizeValue:v,getTokenCss:O,getGlobalCss:R,getPreflightCss:N,css:x,cva:w,sva:k,getRecipe:S,getSlotRecipe:_,hasRecipe:W,isRecipe:D,isSlotRecipe:z,query:ee}}function iw(e){const t=new Map;return e.allTokens.forEach(n=>{const{cssVar:r,virtual:o,conditions:i}=n.extensions,s=i||o?r.ref:n.value;t.set(n.name,{value:s,variable:r.ref})}),t}const jh=e=>_e(e)&&"value"in e,Pa=e=>({list(){return Object.keys(ih(e,jh))},search(t){return this.list().filter(n=>n.includes(t))}}),$h=(e,t,n)=>({categoryKeys:t,list(r){return Array.from(e.categoryMap.get(r)?.entries()??[]).reduce((o,[i,s])=>(n(s,i)&&o.push(i),o),[])},search(r,o){return this.list(r).filter(i=>i.includes(o))}}),Bh=e=>({list(){return Object.keys(e)},search(t){return this.list().filter(n=>n.includes(t))}}),sw={sm:"480px",md:"768px",lg:"1024px",xl:"1280px","2xl":"1536px"},Ra="var(--chakra-empty,/*!*/ /*!*/)",aw=Wk({"*":{fontFeatureSettings:'"cv11"',"--ring-inset":Ra,"--ring-offset-width":"0px","--ring-offset-color":"#fff","--ring-color":"rgba(66, 153, 225, 0.6)","--ring-offset-shadow":"0 0 #0000","--ring-shadow":"0 0 #0000",...Object.fromEntries(["brightness","contrast","grayscale","hue-rotate","invert","saturate","sepia","drop-shadow"].map(e=>[`--${e}`,Ra])),...Object.fromEntries(["blur","brightness","contrast","grayscale","hue-rotate","invert","opacity","saturate","sepia"].map(e=>[`--backdrop-${e}`,Ra])),"--global-font-mono":"fonts.mono","--global-font-body":"fonts.body","--global-color-border":"colors.border"},html:{color:"fg",bg:"bg",lineHeight:"1.5",colorPalette:"gray"},"*::placeholder, *[data-placeholder]":{color:"fg.muted/80"},"*::selection":{bg:"colorPalette.emphasized/80"}}),lw=Gk({"fill.muted":{value:{background:"colorPalette.muted",color:"colorPalette.fg"}},"fill.subtle":{value:{background:"colorPalette.subtle",color:"colorPalette.fg"}},"fill.surface":{value:{background:"colorPalette.subtle",color:"colorPalette.fg",boxShadow:"0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.muted"}},"fill.solid":{value:{background:"colorPalette.solid",color:"colorPalette.contrast"}},"outline.subtle":{value:{color:"colorPalette.fg",boxShadow:"inset 0 0 0px 1px var(--shadow-color)",boxShadowColor:"colorPalette.subtle"}},"outline.solid":{value:{borderWidth:"1px",borderColor:"colorPalette.solid",color:"colorPalette.fg"}},"indicator.bottom":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",bottom:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.top":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",top:"var(--indicator-offset-y, 0)",insetInline:"var(--indicator-offset-x, 0)",height:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.start":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineStart:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},"indicator.end":{value:{position:"relative","--indicator-color-fallback":"colors.colorPalette.solid",_before:{content:'""',position:"absolute",insetInlineEnd:"var(--indicator-offset-x, 0)",insetBlock:"var(--indicator-offset-y, 0)",width:"var(--indicator-thickness, 2px)",background:"var(--indicator-color, var(--indicator-color-fallback))"}}},disabled:{value:{opacity:"0.5",cursor:"not-allowed"}},none:{value:{}}}),cw=Uk({"slide-fade-in":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-from-bottom, fade-in"},"&[data-placement^=bottom]":{animationName:"slide-from-top, fade-in"},"&[data-placement^=left]":{animationName:"slide-from-right, fade-in"},"&[data-placement^=right]":{animationName:"slide-from-left, fade-in"}}},"slide-fade-out":{value:{transformOrigin:"var(--transform-origin)","&[data-placement^=top]":{animationName:"slide-to-bottom, fade-out"},"&[data-placement^=bottom]":{animationName:"slide-to-top, fade-out"},"&[data-placement^=left]":{animationName:"slide-to-right, fade-out"},"&[data-placement^=right]":{animationName:"slide-to-left, fade-out"}}},"scale-fade-in":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-in, fade-in"}},"scale-fade-out":{value:{transformOrigin:"var(--transform-origin)",animationName:"scale-out, fade-out"}}}),Ia=fe({className:"chakra-badge",base:{display:"inline-flex",alignItems:"center",borderRadius:"l2",gap:"1",fontWeight:"medium",fontVariantNumeric:"tabular-nums",whiteSpace:"nowrap",userSelect:"none"},variants:{variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg"},outline:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},plain:{color:"colorPalette.fg"}},size:{xs:{textStyle:"2xs",px:"1",minH:"4"},sm:{textStyle:"xs",px:"1.5",minH:"5"},md:{textStyle:"sm",px:"2",minH:"6"},lg:{textStyle:"sm",px:"2.5",minH:"7"}}},defaultVariants:{variant:"subtle",size:"sm"}}),uw=fe({className:"chakra-button",base:{display:"inline-flex",appearance:"none",alignItems:"center",justifyContent:"center",userSelect:"none",position:"relative",borderRadius:"l2",whiteSpace:"nowrap",verticalAlign:"middle",borderWidth:"1px",borderColor:"transparent",cursor:"button",flexShrink:"0",outline:"0",lineHeight:"1.2",isolation:"isolate",fontWeight:"medium",transitionProperty:"common",transitionDuration:"moderate",focusVisibleRing:"outside",_disabled:{layerStyle:"disabled"},_icon:{flexShrink:"0"}},variants:{size:{"2xs":{h:"6",minW:"6",textStyle:"xs",px:"2",gap:"1",_icon:{width:"3.5",height:"3.5"}},xs:{h:"8",minW:"8",textStyle:"xs",px:"2.5",gap:"1",_icon:{width:"4",height:"4"}},sm:{h:"9",minW:"9",px:"3.5",textStyle:"sm",gap:"2",_icon:{width:"4",height:"4"}},md:{h:"10",minW:"10",textStyle:"sm",px:"4",gap:"2",_icon:{width:"5",height:"5"}},lg:{h:"11",minW:"11",textStyle:"md",px:"5",gap:"3",_icon:{width:"5",height:"5"}},xl:{h:"12",minW:"12",textStyle:"md",px:"5",gap:"2.5",_icon:{width:"5",height:"5"}},"2xl":{h:"16",minW:"16",textStyle:"lg",px:"7",gap:"3",_icon:{width:"6",height:"6"}}},variant:{solid:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"transparent",_hover:{bg:"colorPalette.solid/90"},_expanded:{bg:"colorPalette.solid/90"}},subtle:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"transparent",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},surface:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted",_hover:{bg:"colorPalette.muted"},_expanded:{bg:"colorPalette.muted"}},outline:{borderWidth:"1px",borderColor:"colorPalette.muted",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},ghost:{bg:"transparent",color:"colorPalette.fg",_hover:{bg:"colorPalette.subtle"},_expanded:{bg:"colorPalette.subtle"}},plain:{color:"colorPalette.fg"}}},defaultVariants:{size:"md",variant:"solid"}}),Se=fe({className:"chakra-checkmark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"l1",cursor:"checkbox",focusVisibleRing:"outside",_icon:{boxSize:"full"},_invalid:{colorPalette:"red",borderColor:"border.error"},_disabled:{opacity:"0.5",cursor:"disabled"}},variants:{size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5",p:"0.5"},lg:{boxSize:"6",p:"0.5"}},variant:{solid:{borderColor:"border.emphasized","&:is([data-state=checked], [data-state=indeterminate])":{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},outline:{borderColor:"border","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg",borderColor:"colorPalette.solid"}},subtle:{bg:"colorPalette.muted",borderColor:"colorPalette.muted","&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},plain:{"&:is([data-state=checked], [data-state=indeterminate])":{color:"colorPalette.fg"}},inverted:{borderColor:"border",color:"colorPalette.fg","&:is([data-state=checked], [data-state=indeterminate])":{borderColor:"colorPalette.solid"}}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),{variants:dw,defaultVariants:hw}=Ia,fw=fe({className:"chakra-code",base:{fontFamily:"mono",alignItems:"center",display:"inline-flex",borderRadius:"l2"},variants:dw,defaultVariants:hw}),Wh=fe({className:"color-swatch",base:{boxSize:"var(--swatch-size)",shadow:"inset 0 0 0 1px rgba(0, 0, 0, 0.1)","--checker-size":"8px","--checker-bg":"colors.bg","--checker-fg":"colors.bg.emphasized",background:"linear-gradient(var(--color), var(--color)), repeating-conic-gradient(var(--checker-fg) 0%, var(--checker-fg) 25%, var(--checker-bg) 0%, var(--checker-bg) 50%) 0% 50% / var(--checker-size) var(--checker-size) !important",display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},variants:{size:{"2xs":{"--swatch-size":"sizes.3.5"},xs:{"--swatch-size":"sizes.4"},sm:{"--swatch-size":"sizes.4.5"},md:{"--swatch-size":"sizes.5"},lg:{"--swatch-size":"sizes.6"},xl:{"--swatch-size":"sizes.7"},"2xl":{"--swatch-size":"sizes.8"},inherit:{"--swatch-size":"inherit"},full:{"--swatch-size":"100%"}},shape:{square:{borderRadius:"none"},circle:{borderRadius:"full"},rounded:{borderRadius:"l1"}}},defaultVariants:{size:"md",shape:"rounded"}}),gw=fe({className:"chakra-container",base:{position:"relative",maxWidth:"8xl",w:"100%",mx:"auto",px:{base:"4",md:"6",lg:"8"}},variants:{centerContent:{true:{display:"flex",flexDirection:"column",alignItems:"center"}},fluid:{true:{maxWidth:"full"}}}}),pw=fe({className:"chakra-heading",base:{fontFamily:"heading",fontWeight:"semibold"},variants:{size:{xs:{textStyle:"xs"},sm:{textStyle:"sm"},md:{textStyle:"md"},lg:{textStyle:"lg"},xl:{textStyle:"xl"},"2xl":{textStyle:"2xl"},"3xl":{textStyle:"3xl"},"4xl":{textStyle:"4xl"},"5xl":{textStyle:"5xl"},"6xl":{textStyle:"6xl"},"7xl":{textStyle:"7xl"}}},defaultVariants:{size:"xl"}}),mw=fe({className:"chakra-icon",base:{display:"inline-block",lineHeight:"1em",flexShrink:"0",color:"currentcolor",verticalAlign:"middle"},variants:{size:{inherit:{},xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"},xl:{boxSize:"7"},"2xl":{boxSize:"8"}}},defaultVariants:{size:"inherit"}}),he=fe({className:"chakra-input",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},height:"var(--input-height)",minW:"var(--input-height)","--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{"2xs":{textStyle:"xs",px:"2","--input-height":"sizes.7"},xs:{textStyle:"xs",px:"2","--input-height":"sizes.8"},sm:{textStyle:"sm",px:"2.5","--input-height":"sizes.9"},md:{textStyle:"sm",px:"3","--input-height":"sizes.10"},lg:{textStyle:"md",px:"4","--input-height":"sizes.11"},xl:{textStyle:"md",px:"4.5","--input-height":"sizes.12"},"2xl":{textStyle:"lg",px:"5","--input-height":"sizes.16"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside",focusRingColor:"var(--focus-color)"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)",_invalid:{borderColor:"var(--error-color)",boxShadow:"0px 1px 0px 0px var(--error-color)"}}}}},defaultVariants:{size:"md",variant:"outline"}}),vw=fe({className:"chakra-input-addon",base:{flex:"0 0 auto",width:"auto",display:"flex",alignItems:"center",whiteSpace:"nowrap",alignSelf:"stretch",borderRadius:"l2"},variants:{size:he.variants.size,variant:{outline:{borderWidth:"1px",borderColor:"border",bg:"bg.muted"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.emphasized"},flushed:{borderBottom:"1px solid",borderColor:"inherit",borderRadius:"0",px:"0",bg:"transparent"}}},defaultVariants:{size:"md",variant:"outline"}}),bw=fe({className:"chakra-kbd",base:{display:"inline-flex",alignItems:"center",fontWeight:"medium",fontFamily:"mono",flexShrink:"0",whiteSpace:"nowrap",wordSpacing:"-0.5em",userSelect:"none",px:"1",borderRadius:"l2"},variants:{variant:{raised:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderWidth:"1px",borderBottomWidth:"2px",borderColor:"colorPalette.muted"},outline:{borderWidth:"1px",color:"colorPalette.fg"},subtle:{bg:"colorPalette.muted",color:"colorPalette.fg"},plain:{color:"colorPalette.fg"}},size:{sm:{textStyle:"xs",height:"4.5"},md:{textStyle:"sm",height:"5"},lg:{textStyle:"md",height:"6"}}},defaultVariants:{size:"md",variant:"raised"}}),yw=fe({className:"chakra-link",base:{display:"inline-flex",alignItems:"center",outline:"none",gap:"1.5",cursor:"pointer",borderRadius:"l1",focusRing:"outside"},variants:{variant:{underline:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"},plain:{color:"colorPalette.fg",_hover:{textDecoration:"underline",textUnderlineOffset:"3px",textDecorationColor:"currentColor/20"}}}},defaultVariants:{variant:"plain"}}),xw=fe({className:"chakra-mark",base:{bg:"transparent",color:"inherit",whiteSpace:"nowrap"},variants:{variant:{subtle:{bg:"colorPalette.subtle",color:"inherit"},solid:{bg:"colorPalette.solid",color:"colorPalette.contrast"},text:{fontWeight:"medium"},plain:{}}}}),Ee=fe({className:"chakra-radiomark",base:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:0,verticalAlign:"top",color:"white",borderWidth:"1px",borderColor:"transparent",borderRadius:"full",cursor:"radio",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing",outlineOffset:"2px"},_invalid:{colorPalette:"red",borderColor:"red.500"},_disabled:{opacity:"0.5",cursor:"disabled"},"& .dot":{height:"100%",width:"100%",borderRadius:"full",bg:"currentColor",scale:"0.4"}},variants:{variant:{solid:{borderWidth:"1px",borderColor:"border.emphasized",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},subtle:{borderWidth:"1px",bg:"colorPalette.muted",borderColor:"colorPalette.muted",color:"transparent",_checked:{color:"colorPalette.fg"}},outline:{borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.fg",borderColor:"colorPalette.solid"},"& .dot":{scale:"0.6"}},inverted:{bg:"bg",borderWidth:"1px",borderColor:"inherit",_checked:{color:"colorPalette.solid",borderColor:"currentcolor"}}},size:{xs:{boxSize:"3"},sm:{boxSize:"4"},md:{boxSize:"5"},lg:{boxSize:"6"}},filled:{true:{bg:"bg"}}},defaultVariants:{variant:"solid",size:"md"}}),kw=fe({className:"chakra-separator",base:{display:"block",borderColor:"border"},variants:{variant:{solid:{borderStyle:"solid"},dashed:{borderStyle:"dashed"},dotted:{borderStyle:"dotted"}},orientation:{vertical:{borderInlineStartWidth:"var(--separator-thickness)"},horizontal:{borderTopWidth:"var(--separator-thickness)"}},size:{xs:{"--separator-thickness":"0.5px"},sm:{"--separator-thickness":"1px"},md:{"--separator-thickness":"2px"},lg:{"--separator-thickness":"3px"}}},defaultVariants:{size:"sm",variant:"solid",orientation:"horizontal"}}),Cw=fe({className:"chakra-skeleton",base:{},variants:{loading:{true:{borderRadius:"l2",boxShadow:"none",backgroundClip:"padding-box",cursor:"default",color:"transparent",pointerEvents:"none",userSelect:"none",flexShrink:"0","&::before, &::after, *":{visibility:"hidden"}},false:{background:"unset",animation:"fade-in var(--fade-duration, 0.1s) ease-out !important"}},variant:{pulse:{background:"bg.emphasized",animation:"pulse",animationDuration:"var(--duration, 1.2s)"},shine:{"--animate-from":"200%","--animate-to":"-200%","--start-color":"colors.bg.muted","--end-color":"colors.bg.emphasized",backgroundImage:"linear-gradient(270deg,var(--start-color),var(--end-color),var(--end-color),var(--start-color))",backgroundSize:"400% 100%",animation:"bg-position var(--duration, 5s) ease-in-out infinite"},none:{animation:"none"}}},defaultVariants:{variant:"pulse",loading:!0}}),ww=fe({className:"chakra-skip-nav",base:{display:"inline-flex",bg:"bg.panel",padding:"2.5",borderRadius:"l2",fontWeight:"semibold",focusVisibleRing:"outside",textStyle:"sm",userSelect:"none",border:"0",height:"1px",width:"1px",margin:"-1px",outline:"0",overflow:"hidden",position:"absolute",clip:"rect(0 0 0 0)",_focusVisible:{clip:"auto",width:"auto",height:"auto",position:"fixed",top:"6",insetStart:"6"}}}),Sw=fe({className:"chakra-spinner",base:{display:"inline-block",borderColor:"currentColor",borderStyle:"solid",borderWidth:"2px",borderRadius:"full",width:"var(--spinner-size)",height:"var(--spinner-size)",animation:"spin",animationDuration:"slowest","--spinner-track-color":"transparent",borderBottomColor:"var(--spinner-track-color)",borderInlineStartColor:"var(--spinner-track-color)"},variants:{size:{inherit:{"--spinner-size":"1em"},xs:{"--spinner-size":"sizes.3"},sm:{"--spinner-size":"sizes.4"},md:{"--spinner-size":"sizes.5"},lg:{"--spinner-size":"sizes.8"},xl:{"--spinner-size":"sizes.10"}}},defaultVariants:{size:"md"}}),Ew=fe({className:"chakra-textarea",base:{width:"100%",minWidth:"0",outline:"0",position:"relative",appearance:"none",textAlign:"start",borderRadius:"l2",_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},variants:{size:{xs:{textStyle:"xs",px:"2",py:"1.5",scrollPaddingBottom:"1.5"},sm:{textStyle:"sm",px:"2.5",py:"2",scrollPaddingBottom:"2"},md:{textStyle:"sm",px:"3",py:"2",scrollPaddingBottom:"2"},lg:{textStyle:"md",px:"4",py:"3",scrollPaddingBottom:"3"},xl:{textStyle:"md",px:"4.5",py:"3.5",scrollPaddingBottom:"3.5"}},variant:{outline:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"},subtle:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"},flushed:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}}}},defaultVariants:{size:"md",variant:"outline"}}),Ow={badge:Ia,button:uw,code:fw,container:gw,heading:pw,input:he,inputAddon:vw,kbd:bw,link:yw,mark:xw,separator:kw,skeleton:Cw,skipNavLink:ww,spinner:Sw,textarea:Ew,icon:mw,checkmark:Se,radiomark:Ee,colorSwatch:Wh},Pw=xa.colors({bg:{DEFAULT:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},emphasized:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},inverted:{value:{_light:"{colors.black}",_dark:"{colors.white}"}},panel:{value:{_light:"{colors.white}",_dark:"{colors.gray.950}"}},error:{value:{_light:"{colors.red.50}",_dark:"{colors.red.950}"}},warning:{value:{_light:"{colors.orange.50}",_dark:"{colors.orange.950}"}},success:{value:{_light:"{colors.green.50}",_dark:"{colors.green.950}"}},info:{value:{_light:"{colors.blue.50}",_dark:"{colors.blue.950}"}}},fg:{DEFAULT:{value:{_light:"{colors.black}",_dark:"{colors.gray.50}"}},muted:{value:{_light:"{colors.gray.600}",_dark:"{colors.gray.400}"}},subtle:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.500}"}},inverted:{value:{_light:"{colors.gray.50}",_dark:"{colors.black}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.300}"}},success:{value:{_light:"{colors.green.600}",_dark:"{colors.green.300}"}},info:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.300}"}}},border:{DEFAULT:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},muted:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},subtle:{value:{_light:"{colors.gray.50}",_dark:"{colors.gray.950}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},inverted:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},error:{value:{_light:"{colors.red.500}",_dark:"{colors.red.400}"}},warning:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.400}"}},success:{value:{_light:"{colors.green.500}",_dark:"{colors.green.400}"}},info:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.400}"}}},gray:{contrast:{value:{_light:"{colors.white}",_dark:"{colors.black}"}},fg:{value:{_light:"{colors.gray.800}",_dark:"{colors.gray.200}"}},subtle:{value:{_light:"{colors.gray.100}",_dark:"{colors.gray.900}"}},muted:{value:{_light:"{colors.gray.200}",_dark:"{colors.gray.800}"}},emphasized:{value:{_light:"{colors.gray.300}",_dark:"{colors.gray.700}"}},solid:{value:{_light:"{colors.gray.900}",_dark:"{colors.white}"}},focusRing:{value:{_light:"{colors.gray.400}",_dark:"{colors.gray.400}"}}},red:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.red.700}",_dark:"{colors.red.300}"}},subtle:{value:{_light:"{colors.red.100}",_dark:"{colors.red.900}"}},muted:{value:{_light:"{colors.red.200}",_dark:"{colors.red.800}"}},emphasized:{value:{_light:"{colors.red.300}",_dark:"{colors.red.700}"}},solid:{value:{_light:"{colors.red.600}",_dark:"{colors.red.600}"}},focusRing:{value:{_light:"{colors.red.500}",_dark:"{colors.red.500}"}}},orange:{contrast:{value:{_light:"white",_dark:"black"}},fg:{value:{_light:"{colors.orange.700}",_dark:"{colors.orange.300}"}},subtle:{value:{_light:"{colors.orange.100}",_dark:"{colors.orange.900}"}},muted:{value:{_light:"{colors.orange.200}",_dark:"{colors.orange.800}"}},emphasized:{value:{_light:"{colors.orange.300}",_dark:"{colors.orange.700}"}},solid:{value:{_light:"{colors.orange.600}",_dark:"{colors.orange.500}"}},focusRing:{value:{_light:"{colors.orange.500}",_dark:"{colors.orange.500}"}}},green:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.green.700}",_dark:"{colors.green.300}"}},subtle:{value:{_light:"{colors.green.100}",_dark:"{colors.green.900}"}},muted:{value:{_light:"{colors.green.200}",_dark:"{colors.green.800}"}},emphasized:{value:{_light:"{colors.green.300}",_dark:"{colors.green.700}"}},solid:{value:{_light:"{colors.green.600}",_dark:"{colors.green.600}"}},focusRing:{value:{_light:"{colors.green.500}",_dark:"{colors.green.500}"}}},blue:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.blue.700}",_dark:"{colors.blue.300}"}},subtle:{value:{_light:"{colors.blue.100}",_dark:"{colors.blue.900}"}},muted:{value:{_light:"{colors.blue.200}",_dark:"{colors.blue.800}"}},emphasized:{value:{_light:"{colors.blue.300}",_dark:"{colors.blue.700}"}},solid:{value:{_light:"{colors.blue.600}",_dark:"{colors.blue.600}"}},focusRing:{value:{_light:"{colors.blue.500}",_dark:"{colors.blue.500}"}}},yellow:{contrast:{value:{_light:"black",_dark:"black"}},fg:{value:{_light:"{colors.yellow.800}",_dark:"{colors.yellow.300}"}},subtle:{value:{_light:"{colors.yellow.100}",_dark:"{colors.yellow.900}"}},muted:{value:{_light:"{colors.yellow.200}",_dark:"{colors.yellow.800}"}},emphasized:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.700}"}},solid:{value:{_light:"{colors.yellow.300}",_dark:"{colors.yellow.300}"}},focusRing:{value:{_light:"{colors.yellow.500}",_dark:"{colors.yellow.500}"}}},teal:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.teal.700}",_dark:"{colors.teal.300}"}},subtle:{value:{_light:"{colors.teal.100}",_dark:"{colors.teal.900}"}},muted:{value:{_light:"{colors.teal.200}",_dark:"{colors.teal.800}"}},emphasized:{value:{_light:"{colors.teal.300}",_dark:"{colors.teal.700}"}},solid:{value:{_light:"{colors.teal.600}",_dark:"{colors.teal.600}"}},focusRing:{value:{_light:"{colors.teal.500}",_dark:"{colors.teal.500}"}}},purple:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.purple.700}",_dark:"{colors.purple.300}"}},subtle:{value:{_light:"{colors.purple.100}",_dark:"{colors.purple.900}"}},muted:{value:{_light:"{colors.purple.200}",_dark:"{colors.purple.800}"}},emphasized:{value:{_light:"{colors.purple.300}",_dark:"{colors.purple.700}"}},solid:{value:{_light:"{colors.purple.600}",_dark:"{colors.purple.600}"}},focusRing:{value:{_light:"{colors.purple.500}",_dark:"{colors.purple.500}"}}},pink:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.pink.700}",_dark:"{colors.pink.300}"}},subtle:{value:{_light:"{colors.pink.100}",_dark:"{colors.pink.900}"}},muted:{value:{_light:"{colors.pink.200}",_dark:"{colors.pink.800}"}},emphasized:{value:{_light:"{colors.pink.300}",_dark:"{colors.pink.700}"}},solid:{value:{_light:"{colors.pink.600}",_dark:"{colors.pink.600}"}},focusRing:{value:{_light:"{colors.pink.500}",_dark:"{colors.pink.500}"}}},cyan:{contrast:{value:{_light:"white",_dark:"white"}},fg:{value:{_light:"{colors.cyan.700}",_dark:"{colors.cyan.300}"}},subtle:{value:{_light:"{colors.cyan.100}",_dark:"{colors.cyan.900}"}},muted:{value:{_light:"{colors.cyan.200}",_dark:"{colors.cyan.800}"}},emphasized:{value:{_light:"{colors.cyan.300}",_dark:"{colors.cyan.700}"}},solid:{value:{_light:"{colors.cyan.600}",_dark:"{colors.cyan.600}"}},focusRing:{value:{_light:"{colors.cyan.500}",_dark:"{colors.cyan.500}"}}}}),Rw=xa.radii({l1:{value:"{radii.xs}"},l2:{value:"{radii.sm}"},l3:{value:"{radii.md}"}}),Iw=xa.shadows({xs:{value:{_light:"0px 1px 2px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/20}",_dark:"0px 1px 1px {black/64}, 0px 0px 1px inset {colors.gray.300/20}"}},sm:{value:{_light:"0px 2px 4px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 2px 4px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},md:{value:{_light:"0px 4px 8px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 4px 8px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},lg:{value:{_light:"0px 8px 16px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 8px 16px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},xl:{value:{_light:"0px 16px 24px {colors.gray.900/10}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 16px 24px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},"2xl":{value:{_light:"0px 24px 40px {colors.gray.900/16}, 0px 0px 1px {colors.gray.900/30}",_dark:"0px 24px 40px {black/64}, 0px 0px 1px inset {colors.gray.300/30}"}},inner:{value:{_light:"inset 0 2px 4px 0 {black/5}",_dark:"inset 0 2px 4px 0 black"}},inset:{value:{_light:"inset 0 0 0 1px {black/5}",_dark:"inset 0 0 0 1px {colors.gray.300/5}"}}}),Tw=xu.extendWith("itemBody"),Nw=j("action-bar").parts("positioner","content","separator","selectionTrigger","closeTrigger"),Aw=j("alert").parts("title","description","root","indicator","content"),_w=j("breadcrumb").parts("link","currentLink","item","list","root","ellipsis","separator"),Vw=j("blockquote").parts("root","icon","content","caption"),Lw=j("card").parts("root","header","body","footer","title","description"),Fw=j("checkbox-card",["root","control","label","description","addon","indicator","content"]),Dw=j("data-list").parts("root","item","itemLabel","itemValue"),zw=ea.extendWith("header","body","footer","backdrop"),Mw=ea.extendWith("header","body","footer","backdrop"),jw=ld.extendWith("textarea"),$w=j("empty-state",["root","content","indicator","title","description"]),Bw=ud.extendWith("requiredIndicator"),Ww=hd.extendWith("content"),Hw=fd.extendWith("itemContent","dropzoneContent","fileText"),Uw=j("list").parts("root","item","indicator"),Gw=xd.extendWith("itemCommand"),qw=j("select").parts("root","field","indicator"),Kw=zd.extendWith("header","body","footer"),Hh=fa.extendWith("itemAddon","itemIndicator"),Yw=Hh.extendWith("itemContent","itemDescription"),Xw=jd.extendWith("itemIndicator"),Qw=Wd.extendWith("indicatorGroup"),Jw=m1.extendWith("indicatorGroup","empty"),Zw=qd.extendWith("markerIndicator"),eS=j("stat").parts("root","label","helpText","valueText","valueUnit","indicator"),tS=j("status").parts("root","indicator"),nS=j("steps",["root","list","item","trigger","indicator","separator","content","title","description","nextTrigger","prevTrigger","progress"]),rS=Kd.extendWith("indicator"),oS=j("table").parts("root","header","body","row","columnHeader","cell","footer","caption"),iS=j("toast").parts("root","title","description","indicator","closeTrigger","actionTrigger"),sS=j("tabs").parts("root","trigger","list","content","contentGroup","indicator"),aS=j("tag").parts("root","label","closeTrigger","startElement","endElement"),lS=j("timeline").parts("root","item","content","separator","indicator","connector","title","description"),cS=jx.extendWith("channelText"),uS=j("code-block",["root","content","title","header","footer","control","overlay","code","codeText","copyTrigger","copyIndicator","collapseTrigger","collapseIndicator","collapseText"]);Vu.extendWith("valueText");const dS=L1,hS=B({className:"chakra-accordion",slots:Tw.keys(),base:{root:{width:"full","--accordion-radius":"radii.l2"},item:{overflowAnchor:"none"},itemTrigger:{display:"flex",alignItems:"center",textAlign:"start",width:"full",outline:"0",gap:"3",fontWeight:"medium",borderRadius:"var(--accordion-radius)",_focusVisible:{outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{layerStyle:"disabled"}},itemBody:{pt:"var(--accordion-padding-y)",pb:"calc(var(--accordion-padding-y) * 2)"},itemContent:{overflow:"hidden",borderRadius:"var(--accordion-radius)",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}},itemIndicator:{transition:"rotate 0.2s",transformOrigin:"center",color:"fg.subtle",_open:{rotate:"180deg"},_icon:{width:"1.2em",height:"1.2em"}}},variants:{variant:{outline:{item:{borderBottomWidth:"1px"}},subtle:{itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{borderRadius:"var(--accordion-radius)",_open:{bg:"colorPalette.subtle"}}},enclosed:{root:{borderWidth:"1px",borderRadius:"var(--accordion-radius)",divideY:"1px",overflow:"hidden"},itemTrigger:{px:"var(--accordion-padding-x)"},itemContent:{px:"var(--accordion-padding-x)"},item:{_open:{bg:"bg.subtle"}}},plain:{}},size:{sm:{root:{"--accordion-padding-x":"spacing.3","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"sm",py:"var(--accordion-padding-y)"}},md:{root:{"--accordion-padding-x":"spacing.4","--accordion-padding-y":"spacing.2"},itemTrigger:{textStyle:"md",py:"var(--accordion-padding-y)"}},lg:{root:{"--accordion-padding-x":"spacing.4.5","--accordion-padding-y":"spacing.2.5"},itemTrigger:{textStyle:"lg",py:"var(--accordion-padding-y)"}}}},defaultVariants:{size:"md",variant:"outline"}}),fS=B({className:"chakra-action-bar",slots:Nw.keys(),base:{positioner:{position:"fixed",display:"flex",justifyContent:"center",pointerEvents:"none",insetInline:"0",top:"unset",bottom:"calc(env(safe-area-inset-bottom) + 20px)"},content:{bg:"bg.panel",shadow:"md",display:"flex",alignItems:"center",gap:"3",borderRadius:"l3",py:"2.5",px:"3",pointerEvents:"auto",translate:"calc(-1 * var(--scrollbar-width) / 2) 0px",_open:{animationName:"slide-from-bottom, fade-in",animationDuration:"moderate"},_closed:{animationName:"slide-to-bottom, fade-out",animationDuration:"faster"}},separator:{width:"1px",height:"5",bg:"border"},selectionTrigger:{display:"inline-flex",alignItems:"center",gap:"2",alignSelf:"stretch",textStyle:"sm",px:"4",py:"1",borderRadius:"l2",borderWidth:"1px",borderStyle:"dashed"}}}),gS=B({slots:Aw.keys(),className:"chakra-alert",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",borderRadius:"l3"},title:{fontWeight:"medium"},description:{display:"inline"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0",width:"1em",height:"1em",_icon:{boxSize:"full"}},content:{display:"flex",flex:"1",gap:"1"}},variants:{status:{info:{root:{colorPalette:"blue"}},warning:{root:{colorPalette:"orange"}},success:{root:{colorPalette:"green"}},error:{root:{colorPalette:"red"}},neutral:{root:{colorPalette:"gray"}}},inline:{true:{content:{display:"inline-flex",flexDirection:"row",alignItems:"center"}},false:{content:{display:"flex",flexDirection:"column"}}},variant:{subtle:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg"}},surface:{root:{bg:"colorPalette.subtle",color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",shadow:"inset 0 0 0px 1px var(--shadow-color)",shadowColor:"colorPalette.muted"},indicator:{color:"colorPalette.fg"}},solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"},indicator:{color:"colorPalette.contrast"}}},size:{sm:{root:{gap:"2",px:"3",py:"3",textStyle:"xs"},indicator:{textStyle:"lg"}},md:{root:{gap:"3",px:"4",py:"4",textStyle:"sm"},indicator:{textStyle:"xl"}},lg:{root:{gap:"3",px:"4",py:"4",textStyle:"md"},indicator:{textStyle:"2xl"}}}},defaultVariants:{status:"info",variant:"subtle",size:"md",inline:!1}}),pS=B({slots:wu.keys(),className:"chakra-avatar",base:{root:{display:"inline-flex",alignItems:"center",justifyContent:"center",fontWeight:"medium",position:"relative",verticalAlign:"top",flexShrink:"0",userSelect:"none",width:"var(--avatar-size)",height:"var(--avatar-size)",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)","&[data-group-item]":{borderWidth:"2px",borderColor:"bg"}},image:{width:"100%",height:"100%",objectFit:"cover",borderRadius:"var(--avatar-radius)"},fallback:{lineHeight:"1",textTransform:"uppercase",fontWeight:"medium",fontSize:"var(--avatar-font-size)",borderRadius:"var(--avatar-radius)"}},variants:{size:{full:{root:{"--avatar-size":"100%","--avatar-font-size":"100%"}},"2xs":{root:{"--avatar-font-size":"fontSizes.2xs","--avatar-size":"sizes.6"}},xs:{root:{"--avatar-font-size":"fontSizes.xs","--avatar-size":"sizes.8"}},sm:{root:{"--avatar-font-size":"fontSizes.sm","--avatar-size":"sizes.9"}},md:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.10"}},lg:{root:{"--avatar-font-size":"fontSizes.md","--avatar-size":"sizes.11"}},xl:{root:{"--avatar-font-size":"fontSizes.lg","--avatar-size":"sizes.12"}},"2xl":{root:{"--avatar-font-size":"fontSizes.xl","--avatar-size":"sizes.16"}}},variant:{solid:{root:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},subtle:{root:{bg:"colorPalette.muted",color:"colorPalette.fg"}},outline:{root:{color:"colorPalette.fg",borderWidth:"1px",borderColor:"colorPalette.muted"}}},shape:{square:{},rounded:{root:{"--avatar-radius":"radii.l3"}},full:{root:{"--avatar-radius":"radii.full"}}},borderless:{true:{root:{"&[data-group-item]":{borderWidth:"0px"}}}}},defaultVariants:{size:"md",shape:"full",variant:"subtle"}}),mS=B({className:"chakra-blockquote",slots:Vw.keys(),base:{root:{position:"relative",display:"flex",flexDirection:"column",gap:"2"},caption:{textStyle:"sm",color:"fg.muted"},icon:{boxSize:"5"}},variants:{justify:{start:{root:{alignItems:"flex-start",textAlign:"start"}},center:{root:{alignItems:"center",textAlign:"center"}},end:{root:{alignItems:"flex-end",textAlign:"end"}}},variant:{subtle:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.muted"},icon:{color:"colorPalette.fg"}},solid:{root:{paddingX:"5",borderStartWidth:"4px",borderStartColor:"colorPalette.solid"},icon:{color:"colorPalette.solid"}},plain:{root:{paddingX:"5"},icon:{color:"colorPalette.solid"}}}},defaultVariants:{variant:"subtle",justify:"start"}}),vS=B({className:"chakra-breadcrumb",slots:_w.keys(),base:{list:{display:"flex",alignItems:"center",wordBreak:"break-word",color:"fg.muted",listStyle:"none"},link:{outline:"0",textDecoration:"none",borderRadius:"l1",focusRing:"outside",display:"inline-flex",alignItems:"center",gap:"2"},item:{display:"inline-flex",alignItems:"center"},separator:{color:"fg.muted",opacity:"0.8",_icon:{boxSize:"1em"},_rtl:{rotate:"180deg"}},ellipsis:{display:"inline-flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"1em"}}},variants:{variant:{underline:{link:{color:"colorPalette.fg",textDecoration:"underline",textUnderlineOffset:"0.2em",textDecorationColor:"colorPalette.muted"},currentLink:{color:"colorPalette.fg"}},plain:{link:{color:"fg.muted",_hover:{color:"fg"}},currentLink:{color:"fg"}}},size:{sm:{list:{gap:"1",textStyle:"xs"}},md:{list:{gap:"1.5",textStyle:"sm"}},lg:{list:{gap:"2",textStyle:"md"}}}},defaultVariants:{variant:"plain",size:"md"}}),bS=B({className:"chakra-card",slots:Lw.keys(),base:{root:{display:"flex",flexDirection:"column",position:"relative",minWidth:"0",wordWrap:"break-word",borderRadius:"l3",color:"fg",textAlign:"start"},title:{fontWeight:"semibold"},description:{color:"fg.muted",fontSize:"sm"},header:{paddingInline:"var(--card-padding)",paddingTop:"var(--card-padding)",display:"flex",flexDirection:"column",gap:"1.5"},body:{padding:"var(--card-padding)",flex:"1",display:"flex",flexDirection:"column"},footer:{display:"flex",alignItems:"center",gap:"2",paddingInline:"var(--card-padding)",paddingBottom:"var(--card-padding)"}},variants:{size:{sm:{root:{"--card-padding":"spacing.4"},title:{textStyle:"md"}},md:{root:{"--card-padding":"spacing.6"},title:{textStyle:"lg"}},lg:{root:{"--card-padding":"spacing.7"},title:{textStyle:"xl"}}},variant:{elevated:{root:{bg:"bg.panel",boxShadow:"md"}},outline:{root:{bg:"bg.panel",borderWidth:"1px",borderColor:"border"}},subtle:{root:{bg:"bg.muted"}}}},defaultVariants:{variant:"outline",size:"md"}}),yS=B({slots:Mx.keys(),className:"chakra-checkbox",base:{root:{display:"inline-flex",gap:"2",alignItems:"center",verticalAlign:"top",position:"relative"},control:Se.base,label:{fontWeight:"medium",userSelect:"none",_disabled:{opacity:"0.5"}}},variants:{size:{xs:{root:{gap:"1.5"},label:{textStyle:"xs"},control:Se.variants?.size?.xs},sm:{root:{gap:"2"},label:{textStyle:"sm"},control:Se.variants?.size?.sm},md:{root:{gap:"2.5"},label:{textStyle:"sm"},control:Se.variants?.size?.md},lg:{root:{gap:"3"},label:{textStyle:"md"},control:Se.variants?.size?.lg}},variant:{outline:{control:Se.variants?.variant?.outline},solid:{control:Se.variants?.variant?.solid},subtle:{control:Se.variants?.variant?.subtle}}},defaultVariants:{variant:"solid",size:"md"}}),xS=B({slots:Fw.keys(),className:"chakra-checkbox-card",base:{root:{display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",flex:"1",focusVisibleRing:"outside",_disabled:{opacity:"0.8"},_invalid:{outline:"2px solid",outlineColor:"border.error"}},control:{display:"inline-flex",flex:"1",position:"relative",borderRadius:"inherit",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"},label:{fontWeight:"medium",display:"flex",alignItems:"center",gap:"2",flex:"1",_disabled:{opacity:"0.5"}},description:{opacity:"0.64",textStyle:"sm",_disabled:{opacity:"0.5"}},addon:{_disabled:{opacity:"0.5"}},indicator:Se.base,content:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--checkbox-card-justify)",alignItems:"var(--checkbox-card-align)"}},variants:{size:{sm:{root:{textStyle:"sm"},control:{padding:"3",gap:"1.5"},addon:{px:"3",py:"1.5",borderTopWidth:"1px"},indicator:Se.variants?.size.sm},md:{root:{textStyle:"sm"},control:{padding:"4",gap:"2.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.md},lg:{root:{textStyle:"md"},control:{padding:"4",gap:"3.5"},addon:{px:"4",py:"2",borderTopWidth:"1px"},indicator:Se.variants?.size.lg}},variant:{surface:{root:{borderWidth:"1px",borderColor:"border",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"},_disabled:{bg:"bg.muted"}},indicator:Se.variants?.variant.solid},subtle:{root:{bg:"bg.muted"},control:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},indicator:Se.variants?.variant.plain},outline:{root:{borderWidth:"1px",borderColor:"border",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.solid},solid:{root:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},indicator:Se.variants?.variant.inverted}},justify:{start:{root:{"--checkbox-card-justify":"flex-start"}},end:{root:{"--checkbox-card-justify":"flex-end"}},center:{root:{"--checkbox-card-justify":"center"}}},align:{start:{root:{"--checkbox-card-align":"flex-start"},content:{textAlign:"start"}},end:{root:{"--checkbox-card-align":"flex-end"},content:{textAlign:"end"}},center:{root:{"--checkbox-card-align":"center"},content:{textAlign:"center"}}},orientation:{vertical:{control:{flexDirection:"column"}},horizontal:{control:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),kS=B({slots:uS.keys(),className:"code-block",base:{root:{colorPalette:"gray",rounded:"var(--code-block-radius)",overflow:"hidden",bg:"bg",color:"fg",borderWidth:"1px","--code-block-max-height":"320px","--code-block-bg":"colors.bg","--code-block-fg":"colors.fg","--code-block-obscured-opacity":"0.5","--code-block-obscured-blur":"1px","--code-block-line-number-width":"sizes.3","--code-block-line-number-margin":"spacing.4","--code-block-highlight-bg":"{colors.teal.focusRing/20}","--code-block-highlight-border":"colors.teal.focusRing","--code-block-highlight-added-bg":"{colors.green.focusRing/20}","--code-block-highlight-added-border":"colors.green.focusRing","--code-block-highlight-removed-bg":"{colors.red.focusRing/20}","--code-block-highlight-removed-border":"colors.red.focusRing"},header:{display:"flex",alignItems:"center",gap:"2",position:"relative",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)",mb:"calc(var(--code-block-padding) / 2 * -1)"},title:{display:"inline-flex",alignItems:"center",gap:"1.5",flex:"1",color:"fg.muted"},control:{gap:"1.5",display:"inline-flex",alignItems:"center"},footer:{display:"flex",alignItems:"center",justifyContent:"center",gap:"2",px:"var(--code-block-padding)",minH:"var(--code-block-header-height)"},content:{position:"relative",colorScheme:"dark",overflowX:"auto",overflowY:"hidden",borderBottomRadius:"var(--code-block-radius)",maxHeight:"var(--code-block-max-height)","& ::selection":{bg:"blue.500/40"},_expanded:{maxHeight:"unset"}},overlay:{"--bg":"{colors.black/50}",display:"flex",alignItems:"flex-end",justifyContent:"center",padding:"4",bgImage:"linear-gradient(0deg,var(--bg) 25%,transparent 100%)",color:"white",minH:"5rem",pos:"absolute",bottom:"0",insetInline:"0",zIndex:"1",fontWeight:"medium",_expanded:{display:"none"}},code:{fontFamily:"mono",lineHeight:"tall",whiteSpace:"pre",counterReset:"line 0"},codeText:{px:"var(--code-block-padding)",py:"var(--code-block-padding)",position:"relative",display:"block",width:"100%","&[data-has-focused]":{"& [data-line]:not([data-focused])":{transitionProperty:"opacity, filter",transitionDuration:"moderate",transitionTimingFunction:"ease-in-out",opacity:"var(--code-block-obscured-opacity)",filter:"blur(var(--code-block-obscured-blur))"},"&:hover":{"--code-block-obscured-opacity":"1","--code-block-obscured-blur":"0px"}},"&[data-has-line-numbers][data-plaintext]":{paddingInlineStart:"calc(var(--code-block-line-number-width) + var(--code-block-line-number-margin) + var(--code-block-padding))"},"& [data-line]":{position:"relative","--highlight-bg":"var(--code-block-highlight-bg)","--highlight-border":"var(--code-block-highlight-border)","&[data-highlight], &[data-diff]":{display:"inline-block",width:"full","&:after":{content:"''",display:"block",position:"absolute",insetStart:"calc(var(--code-block-padding) * -1)",insetEnd:"0px",width:"calc(100% + var(--code-block-padding) * 2)",height:"100%",bg:"var(--highlight-bg)",borderStartWidth:"2px",borderStartColor:"var(--highlight-border)"}},"&[data-diff='added']":{"--highlight-bg":"var(--code-block-highlight-added-bg)","--highlight-border":"var(--code-block-highlight-added-border)"},"&[data-diff='removed']":{"--highlight-bg":"var(--code-block-highlight-removed-bg)","--highlight-border":"var(--code-block-highlight-removed-border)"}},"&[data-word-wrap]":{"&[data-plaintext], & [data-line]":{whiteSpace:"pre-wrap",wordBreak:"break-all"}},"&[data-has-line-numbers]":{"--content":"counter(line)","& [data-line]:before":{content:"var(--content)",counterIncrement:"line",width:"var(--code-block-line-number-width)",marginRight:"var(--code-block-line-number-margin)",display:"inline-block",textAlign:"end",userSelect:"none",whiteSpace:"nowrap",opacity:.4},"& [data-diff='added']:before":{content:"'+'"},"& [data-diff='removed']:before":{content:"'-'"}}}},variants:{size:{sm:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.md","--code-block-header-height":"sizes.8"},title:{textStyle:"xs"},code:{fontSize:"xs"}},md:{root:{"--code-block-padding":"spacing.4","--code-block-radius":"radii.lg","--code-block-header-height":"sizes.10"},title:{textStyle:"xs"},code:{fontSize:"sm"}},lg:{root:{"--code-block-padding":"spacing.5","--code-block-radius":"radii.xl","--code-block-header-height":"sizes.12"},title:{textStyle:"sm"},code:{fontSize:"sm"}}}},defaultVariants:{size:"md"}}),CS=B({slots:Ac.keys(),className:"chakra-collapsible",base:{content:{overflow:"hidden",_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}),wS=B({className:"colorPicker",slots:cS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5"},label:{color:"fg",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},valueText:{textAlign:"start"},control:{display:"flex",alignItems:"center",flexDirection:"row",gap:"2",position:"relative"},swatchTrigger:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{display:"flex",alignItems:"center",justifyContent:"center",flexDirection:"row",flexShrink:"0",gap:"2",textStyle:"sm",minH:"var(--input-height)",minW:"var(--input-height)",px:"1",rounded:"l2",_disabled:{opacity:"0.5"},"--focus-color":"colors.colorPalette.focusRing","&:focus-visible":{borderColor:"var(--focus-color)",outline:"1px solid var(--focus-color)"},"&[data-fit-content]":{"--input-height":"unset",px:"0",border:"0"}},content:{display:"flex",flexDirection:"column",bg:"bg.panel",borderRadius:"l3",boxShadow:"lg",width:"64",p:"4",gap:"3",zIndex:"dropdown",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},area:{height:"180px",borderRadius:"l2",overflow:"hidden"},areaThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",focusVisibleRing:"mixed",focusRingColor:"white"},areaBackground:{height:"full"},channelSlider:{borderRadius:"l2",flex:"1"},channelSliderTrack:{height:"var(--slider-height)",borderRadius:"inherit",boxShadow:"inset 0 0 0 1px rgba(0,0,0,0.1)"},channelText:{textStyle:"xs",color:"fg.muted",fontWeight:"medium",textTransform:"capitalize"},swatchGroup:{display:"flex",flexDirection:"row",flexWrap:"wrap",gap:"2"},swatch:{...Wh.base,borderRadius:"l1"},swatchIndicator:{color:"white",rounded:"full"},channelSliderThumb:{borderRadius:"full",height:"var(--thumb-size)",width:"var(--thumb-size)",borderWidth:"2px",borderColor:"white",shadow:"sm",transform:"translate(-50%, -50%)",focusVisibleRing:"outside",focusRingOffset:"1px"},channelInput:{...he.base,"&::-webkit-inner-spin-button, &::-webkit-outer-spin-button":{WebkitAppearance:"none",margin:0}},formatSelect:{textStyle:"xs",textTransform:"uppercase",borderWidth:"1px",minH:"6",focusRing:"inside",rounded:"l2"},transparencyGrid:{borderRadius:"l2"},view:{display:"flex",flexDirection:"column",gap:"2"}},variants:{size:{"2xs":{channelInput:he.variants?.size?.["2xs"],swatch:{"--swatch-size":"sizes.4.5"},trigger:{"--input-height":"sizes.7"},area:{"--thumb-size":"sizes.3"},channelSlider:{"--slider-height":"sizes.3","--thumb-size":"sizes.3"}},xs:{channelInput:he.variants?.size?.xs,swatch:{"--swatch-size":"sizes.5"},trigger:{"--input-height":"sizes.8"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},sm:{channelInput:he.variants?.size?.sm,swatch:{"--swatch-size":"sizes.6"},trigger:{"--input-height":"sizes.9"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},md:{channelInput:he.variants?.size?.md,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.10"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},lg:{channelInput:he.variants?.size?.lg,swatch:{"--swatch-size":"sizes.7"},trigger:{"--input-height":"sizes.11"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},xl:{channelInput:he.variants?.size?.xl,swatch:{"--swatch-size":"sizes.8"},trigger:{"--input-height":"sizes.12"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}},"2xl":{channelInput:he.variants?.size?.["2xl"],swatch:{"--swatch-size":"sizes.10"},trigger:{"--input-height":"sizes.16"},area:{"--thumb-size":"sizes.3.5"},channelSlider:{"--slider-height":"sizes.3.5","--thumb-size":"sizes.3.5"}}},variant:{outline:{channelInput:he.variants?.variant?.outline,trigger:{borderWidth:"1px"}},subtle:{channelInput:he.variants?.variant?.subtle,trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}}},defaultVariants:{size:"md",variant:"outline"}}),SS=B({className:"chakra-combobox",slots:Jw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},input:{display:"flex",alignItems:"center",justifyContent:"space-between",background:"bg.panel",width:"full",minH:"var(--combobox-input-height)",px:"var(--combobox-input-padding-x)","--input-height":"var(--combobox-input-height)",borderRadius:"l2",outline:0,userSelect:"none",textAlign:"start",_placeholderShown:{color:"fg.muted"},_disabled:{layerStyle:"disabled"},"--focus-color":"colors.colorPalette.focusRing","--error-color":"colors.border.error",_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"}},trigger:{display:"inline-flex",alignItems:"center",justifyContent:"center","--input-height":"var(--combobox-input-height)"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"},control:{pos:"relative"},indicatorGroup:{display:"flex",alignItems:"center",justifyContent:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--combobox-input-padding-x)",_icon:{boxSize:"var(--combobox-indicator-size)"},"[data-disabled] &":{opacity:.5}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"0s"},"&[data-empty]:not(:has([data-scope=combobox][data-part=empty]))":{opacity:0}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{boxSize:"var(--combobox-indicator-size)"}},empty:{py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{pb:"var(--combobox-item-padding-y)",_last:{pb:"0"}},itemGroupLabel:{fontWeight:"medium",py:"var(--combobox-item-padding-y)",px:"var(--combobox-item-padding-x)"}},variants:{variant:{outline:{input:{bg:"transparent",borderWidth:"1px",borderColor:"border",focusVisibleRing:"inside"}},subtle:{input:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted",focusVisibleRing:"inside"}},flushed:{input:{bg:"transparent",borderBottomWidth:"1px",borderBottomColor:"border",borderRadius:"0",px:"0",_focusVisible:{borderColor:"var(--focus-color)",boxShadow:"0px 1px 0px 0px var(--focus-color)"}},indicatorGroup:{px:"0"}}},size:{xs:{root:{"--combobox-input-height":"sizes.8","--combobox-input-padding-x":"spacing.2","--combobox-indicator-size":"sizes.3.5"},input:{textStyle:"xs"},content:{"--combobox-item-padding-x":"spacing.1.5","--combobox-item-padding-y":"spacing.1","--combobox-indicator-size":"sizes.3.5",p:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"}},sm:{root:{"--combobox-input-height":"sizes.9","--combobox-input-padding-x":"spacing.2.5","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"}},md:{root:{"--combobox-input-height":"sizes.10","--combobox-input-padding-x":"spacing.3","--combobox-indicator-size":"sizes.4"},input:{textStyle:"sm"},content:{"--combobox-item-padding-x":"spacing.2","--combobox-item-padding-y":"spacing.1.5","--combobox-indicator-size":"sizes.4",p:"1",textStyle:"sm"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},trigger:{textStyle:"sm",gap:"2"}},lg:{root:{"--combobox-input-height":"sizes.12","--combobox-input-padding-x":"spacing.4","--combobox-indicator-size":"sizes.5"},input:{textStyle:"md"},content:{"--combobox-item-padding-y":"spacing.2","--combobox-item-padding-x":"spacing.3","--combobox-indicator-size":"sizes.5",p:"1.5",textStyle:"md"},trigger:{textStyle:"md",py:"3",gap:"2"}}}},defaultVariants:{size:"md",variant:"outline"}}),ES=B({slots:Dw.keys(),className:"chakra-data-list",base:{itemLabel:{display:"flex",alignItems:"center",gap:"1"},itemValue:{display:"flex",minWidth:"0",flex:"1"}},variants:{orientation:{horizontal:{root:{display:"flex",flexDirection:"column"},item:{display:"inline-flex",alignItems:"center",gap:"4"},itemLabel:{minWidth:"120px"}},vertical:{root:{display:"flex",flexDirection:"column"},item:{display:"flex",flexDirection:"column",gap:"1"}}},size:{sm:{root:{gap:"3"},item:{textStyle:"xs"}},md:{root:{gap:"4"},item:{textStyle:"sm"}},lg:{root:{gap:"5"},item:{textStyle:"md"}}},variant:{subtle:{itemLabel:{color:"fg.muted"}},bold:{itemLabel:{fontWeight:"medium"},itemValue:{color:"fg.muted"}}}},defaultVariants:{size:"md",orientation:"vertical",variant:"subtle"}}),OS=B({slots:zw.keys(),className:"chakra-dialog",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",left:0,top:0,w:"100dvw",h:"100dvh",zIndex:"var(--z-index)",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100dvw",height:"100dvh",position:"fixed",left:0,top:0,"--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",justifyContent:"center",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,borderRadius:"l3",textStyle:"sm",my:"var(--dialog-margin, var(--dialog-base-margin))","--dialog-z-index":"zIndex.modal",zIndex:"calc(var(--dialog-z-index) + var(--layer-index, 0))",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"moderate"},_closed:{animationDuration:"faster"}},header:{display:"flex",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{flex:"1",px:"6",pt:"2",pb:"6"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"2",insetEnd:"2"}},variants:{placement:{center:{positioner:{alignItems:"center"},content:{"--dialog-base-margin":"auto",mx:"auto"}},top:{positioner:{alignItems:"flex-start"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}},bottom:{positioner:{alignItems:"flex-end"},content:{"--dialog-base-margin":"spacing.16",mx:"auto"}}},scrollBehavior:{inside:{positioner:{overflow:"hidden"},content:{maxH:"calc(100% - 7.5rem)"},body:{overflow:"auto"}},outside:{positioner:{overflow:"auto",pointerEvents:"auto"}}},size:{xs:{content:{maxW:"sm"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},cover:{positioner:{padding:"10"},content:{width:"100%",height:"100%","--dialog-margin":"0"}},full:{content:{maxW:"100dvw",minH:"100dvh","--dialog-margin":"0",borderRadius:"0"}}},motionPreset:{scale:{content:{_open:{animationName:"scale-in, fade-in"},_closed:{animationName:"scale-out, fade-out"}}},"slide-in-bottom":{content:{_open:{animationName:"slide-from-bottom, fade-in"},_closed:{animationName:"slide-to-bottom, fade-out"}}},"slide-in-top":{content:{_open:{animationName:"slide-from-top, fade-in"},_closed:{animationName:"slide-to-top, fade-out"}}},"slide-in-left":{content:{_open:{animationName:"slide-from-left, fade-in"},_closed:{animationName:"slide-to-left, fade-out"}}},"slide-in-right":{content:{_open:{animationName:"slide-from-right, fade-in"},_closed:{animationName:"slide-to-right, fade-out"}}},none:{}}},defaultVariants:{size:"md",scrollBehavior:"outside",placement:"top",motionPreset:"scale"}}),PS=B({slots:Mw.keys(),className:"chakra-drawer",base:{backdrop:{bg:"blackAlpha.500",pos:"fixed",insetInlineStart:0,top:0,w:"100vw",h:"100dvh",zIndex:"overlay",_open:{animationName:"fade-in",animationDuration:"slow"},_closed:{animationName:"fade-out",animationDuration:"moderate"}},positioner:{display:"flex",width:"100vw",height:"100dvh",position:"fixed",insetInlineStart:0,top:0,zIndex:"modal",overscrollBehaviorY:"none"},content:{display:"flex",flexDirection:"column",position:"relative",width:"100%",outline:0,zIndex:"modal",textStyle:"sm",maxH:"100dvh",color:"inherit",bg:"bg.panel",boxShadow:"lg",_open:{animationDuration:"slowest",animationTimingFunction:"ease-in-smooth"},_closed:{animationDuration:"slower",animationTimingFunction:"ease-in-smooth"}},header:{display:"flex",alignItems:"center",gap:"2",flex:0,px:"6",pt:"6",pb:"4"},body:{px:"6",py:"2",flex:"1",overflow:"auto"},footer:{display:"flex",alignItems:"center",justifyContent:"flex-end",gap:"3",px:"6",pt:"2",pb:"4"},title:{flex:"1",textStyle:"lg",fontWeight:"semibold"},description:{color:"fg.muted"},closeTrigger:{pos:"absolute",top:"3",insetEnd:"2"}},variants:{size:{xs:{content:{maxW:"xs"}},sm:{content:{maxW:"md"}},md:{content:{maxW:"lg"}},lg:{content:{maxW:"2xl"}},xl:{content:{maxW:"4xl"}},full:{content:{maxW:"100vw",h:"100dvh"}}},placement:{start:{positioner:{justifyContent:"flex-start",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-left-full, fade-in",_rtl:"slide-from-right-full, fade-in"}},_closed:{animationName:{base:"slide-to-left-full, fade-out",_rtl:"slide-to-right-full, fade-out"}}}},end:{positioner:{justifyContent:"flex-end",alignItems:"stretch"},content:{_open:{animationName:{base:"slide-from-right-full, fade-in",_rtl:"slide-from-left-full, fade-in"}},_closed:{animationName:{base:"slide-to-right-full, fade-out",_rtl:"slide-to-left-full, fade-out"}}}},top:{positioner:{justifyContent:"stretch",alignItems:"flex-start"},content:{maxW:"100%",_open:{animationName:"slide-from-top-full, fade-in"},_closed:{animationName:"slide-to-top-full, fade-out"}}},bottom:{positioner:{justifyContent:"stretch",alignItems:"flex-end"},content:{maxW:"100%",_open:{animationName:"slide-from-bottom-full, fade-in"},_closed:{animationName:"slide-to-bottom-full, fade-out"}}}},contained:{true:{positioner:{padding:"4"},content:{borderRadius:"l3"}}}},defaultVariants:{size:"xs",placement:"end"}}),Uh=tr({fontSize:"inherit",fontWeight:"inherit",textAlign:"inherit",bg:"transparent",borderRadius:"l2"}),RS=B({slots:jw.keys(),className:"chakra-editable",base:{root:{display:"inline-flex",alignItems:"center",position:"relative",gap:"1.5",width:"full"},preview:{...Uh,py:"1",px:"1",display:"inline-flex",alignItems:"center",transitionProperty:"common",transitionDuration:"moderate",cursor:"text",_hover:{bg:"bg.muted"},_disabled:{userSelect:"none"}},input:{...Uh,outline:"0",py:"1",px:"1",transitionProperty:"common",transitionDuration:"normal",width:"full",focusVisibleRing:"inside",focusRingWidth:"2px",_placeholder:{opacity:.6}},control:{display:"inline-flex",alignItems:"center",gap:"1.5"}},variants:{size:{sm:{root:{textStyle:"sm"},preview:{minH:"8"},input:{minH:"8"}},md:{root:{textStyle:"sm"},preview:{minH:"9"},input:{minH:"9"}},lg:{root:{textStyle:"md"},preview:{minH:"10"},input:{minH:"10"}}}},defaultVariants:{size:"md"}}),IS=B({slots:$w.keys(),className:"chakra-empty-state",base:{root:{width:"full"},content:{display:"flex",flexDirection:"column",alignItems:"center",justifyContent:"center"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:"fg.subtle",_icon:{boxSize:"1em"}},title:{fontWeight:"semibold"},description:{textStyle:"sm",color:"fg.muted"}},variants:{size:{sm:{root:{px:"4",py:"6"},title:{textStyle:"md"},content:{gap:"4"},indicator:{textStyle:"2xl"}},md:{root:{px:"8",py:"12"},title:{textStyle:"lg"},content:{gap:"6"},indicator:{textStyle:"4xl"}},lg:{root:{px:"12",py:"16"},title:{textStyle:"xl"},content:{gap:"8"},indicator:{textStyle:"6xl"}}}},defaultVariants:{size:"md"}}),TS=B({className:"chakra-field",slots:Bw.keys(),base:{requiredIndicator:{color:"fg.error",lineHeight:"1"},root:{display:"flex",width:"100%",position:"relative",gap:"1.5"},label:{display:"flex",alignItems:"center",textAlign:"start",textStyle:"sm",fontWeight:"medium",gap:"1",userSelect:"none",_disabled:{opacity:"0.5"}},errorText:{display:"inline-flex",alignItems:"center",fontWeight:"medium",gap:"1",color:"fg.error",textStyle:"xs"},helperText:{color:"fg.muted",textStyle:"xs"}},variants:{orientation:{vertical:{root:{flexDirection:"column",alignItems:"flex-start"}},horizontal:{root:{flexDirection:"row",alignItems:"center",justifyContent:"space-between"},label:{flex:"0 0 var(--field-label-width, 80px)"}}}},defaultVariants:{orientation:"vertical"}}),NS=B({className:"fieldset",slots:Ww.keys(),base:{root:{display:"flex",flexDirection:"column",width:"full"},content:{display:"flex",flexDirection:"column",width:"full"},legend:{color:"fg",fontWeight:"medium",_disabled:{opacity:"0.5"}},helperText:{color:"fg.muted",textStyle:"sm"},errorText:{display:"inline-flex",alignItems:"center",color:"fg.error",gap:"2",fontWeight:"medium",textStyle:"sm"}},variants:{size:{sm:{root:{spaceY:"2"},content:{gap:"1.5"},legend:{textStyle:"sm"}},md:{root:{spaceY:"4"},content:{gap:"4"},legend:{textStyle:"sm"}},lg:{root:{spaceY:"6"},content:{gap:"4"},legend:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),AS=B({className:"chakra-file-upload",slots:Hw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"4",width:"100%",alignItems:"flex-start"},label:{fontWeight:"medium",textStyle:"sm"},dropzone:{background:"bg",borderRadius:"l3",borderWidth:"2px",borderStyle:"dashed",display:"flex",alignItems:"center",flexDirection:"column",gap:"4",justifyContent:"center",minHeight:"2xs",px:"3",py:"2",transition:"backgrounds",focusVisibleRing:"outside",_hover:{bg:"bg.subtle"},_dragging:{bg:"colorPalette.subtle",borderStyle:"solid",borderColor:"colorPalette.solid"}},dropzoneContent:{display:"flex",flexDirection:"column",alignItems:"center",textAlign:"center",gap:"1",textStyle:"sm"},item:{pos:"relative",textStyle:"sm",animationName:"fade-in",animationDuration:"moderate",background:"bg",borderRadius:"l2",borderWidth:"1px",width:"100%",display:"flex",alignItems:"center",gap:"3",p:"4"},itemGroup:{width:"100%",display:"flex",flexDirection:"column",gap:"3",_empty:{display:"none"}},itemName:{color:"fg",fontWeight:"medium",lineClamp:"1"},itemContent:{display:"flex",flexDirection:"column",gap:"0.5",flex:"1"},itemSizeText:{color:"fg.muted",textStyle:"xs"},itemDeleteTrigger:{display:"flex",alignItems:"center",justifyContent:"center",alignSelf:"flex-start",boxSize:"5",p:"2px",color:"fg.muted",cursor:"button"},itemPreview:{color:"fg.muted",_icon:{boxSize:"4.5"}}},defaultVariants:{}}),_S=B({className:"chakra-hover-card",slots:gd.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--hovercard-bg":"colors.bg.panel",bg:"var(--hovercard-bg)",boxShadow:"lg",maxWidth:"80",borderRadius:"l3",zIndex:"popover",transformOrigin:"var(--transform-origin)",outline:"0",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--hovercard-bg)"},arrowTip:{borderTopWidth:"0.5px",borderInlineStartWidth:"0.5px"}},variants:{size:{xs:{content:{padding:"3"}},sm:{content:{padding:"4"}},md:{content:{padding:"5"}},lg:{content:{padding:"6"}}}},defaultVariants:{size:"md"}}),VS=B({className:"chakra-list",slots:Uw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"var(--list-gap)","& :where(ul, ol)":{marginTop:"var(--list-gap)"}},item:{whiteSpace:"normal",display:"list-item"},indicator:{marginEnd:"2",minHeight:"1lh",flexShrink:0,display:"inline-block",verticalAlign:"middle"}},variants:{variant:{marker:{root:{listStyle:"revert"},item:{_marker:{color:"fg.subtle"}}},plain:{item:{alignItems:"flex-start",display:"inline-flex"}}},align:{center:{item:{alignItems:"center"}},start:{item:{alignItems:"flex-start"}},end:{item:{alignItems:"flex-end"}}}},defaultVariants:{variant:"marker"}}),LS=B({className:"chakra-listbox",slots:dS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},content:{display:"flex",maxH:"96",p:"1",gap:"1",textStyle:"sm",outline:"none",scrollPadding:"1",_horizontal:{flexDirection:"row",overflowX:"auto"},_vertical:{flexDirection:"column",overflowY:"auto"},"--listbox-item-padding-x":"spacing.2","--listbox-item-padding-y":"spacing.1.5"},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"pointer",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)",_highlighted:{outline:"2px solid",outlineColor:"border.emphasized"},_disabled:{pointerEvents:"none",opacity:"0.5"}},empty:{py:"var(--listbox-item-padding-y)",px:"var(--listbox-item-padding-x)"},itemText:{flex:"1"},itemGroup:{mt:"1.5",_first:{mt:"0"}},itemGroupLabel:{py:"1.5",px:"2",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center",_icon:{boxSize:"4"}}},variants:{variant:{subtle:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_hover:{bg:"bg.emphasized/60"},_selected:{bg:"bg.muted"}}},solid:{content:{bg:"bg.panel",borderWidth:"1px",borderRadius:"l2"},item:{_selected:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}},plain:{}}},defaultVariants:{variant:"subtle"}}),FS=B({className:"chakra-menu",slots:Gw.keys(),base:{content:{outline:0,bg:"bg.panel",boxShadow:"lg",color:"fg",maxHeight:"var(--available-height)","--menu-z-index":"zIndex.dropdown",zIndex:"calc(var(--menu-z-index) + var(--layer-index, 0))",borderRadius:"l2",overflow:"hidden",overflowY:"auto",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"faster"}},item:{textDecoration:"none",color:"fg",userSelect:"none",borderRadius:"l1",width:"100%",display:"flex",cursor:"menuitem",alignItems:"center",textAlign:"start",position:"relative",flex:"0 0 auto",outline:0,_disabled:{layerStyle:"disabled"},"&[data-type]":{ps:"8"}},itemText:{flex:"1"},itemIndicator:{position:"absolute",insetStart:"2",transform:"translateY(-50%)",top:"50%"},itemGroupLabel:{px:"2",py:"1.5",fontWeight:"semibold",textStyle:"sm"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",flexShrink:"0"},itemCommand:{opacity:"0.6",textStyle:"xs",ms:"auto",ps:"4",letterSpacing:"widest",fontFamily:"inherit"},separator:{height:"1px",bg:"bg.muted",my:"1",mx:"-1"}},variants:{variant:{subtle:{item:{_highlighted:{bg:"bg.emphasized/60"}}},solid:{item:{_highlighted:{bg:"colorPalette.solid",color:"colorPalette.contrast"}}}},size:{sm:{content:{minW:"8rem",padding:"1",scrollPadding:"1"},item:{gap:"1",textStyle:"xs",py:"1",px:"1.5"}},md:{content:{minW:"8rem",padding:"1.5",scrollPadding:"1.5"},item:{gap:"2",textStyle:"sm",py:"1.5",px:"2"}}}},defaultVariants:{size:"md",variant:"subtle"}}),di=B({className:"chakra-select",slots:Qw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",width:"full"},trigger:{display:"flex",alignItems:"center",justifyContent:"space-between",width:"full",minH:"var(--select-trigger-height)","--input-height":"var(--select-trigger-height)",px:"var(--select-trigger-padding-x)",borderRadius:"l2",userSelect:"none",textAlign:"start",focusVisibleRing:"inside",_placeholderShown:{color:"fg.muted/80"},_disabled:{layerStyle:"disabled"},_invalid:{borderColor:"border.error"}},indicatorGroup:{display:"flex",alignItems:"center",gap:"1",pos:"absolute",insetEnd:"0",top:"0",bottom:"0",px:"var(--select-trigger-padding-x)",pointerEvents:"none"},indicator:{display:"flex",alignItems:"center",justifyContent:"center",color:{base:"fg.muted",_disabled:"fg.subtle",_invalid:"fg.error"}},content:{background:"bg.panel",display:"flex",flexDirection:"column",zIndex:"dropdown",borderRadius:"l2",outline:0,maxH:"96",overflowY:"auto",boxShadow:"md",_open:{animationStyle:"slide-fade-in",animationDuration:"fast"},_closed:{animationStyle:"slide-fade-out",animationDuration:"fastest"}},item:{position:"relative",userSelect:"none",display:"flex",alignItems:"center",gap:"2",cursor:"option",justifyContent:"space-between",flex:"1",textAlign:"start",borderRadius:"l1",_highlighted:{bg:"bg.emphasized/60"},_disabled:{pointerEvents:"none",opacity:"0.5"},_icon:{width:"4",height:"4"}},control:{pos:"relative"},itemText:{flex:"1"},itemGroup:{_first:{mt:"0"}},itemGroupLabel:{py:"1",fontWeight:"medium"},label:{fontWeight:"medium",userSelect:"none",textStyle:"sm",_disabled:{layerStyle:"disabled"}},valueText:{lineClamp:"1",maxW:"80%"},clearTrigger:{color:"fg.muted",pointerEvents:"auto",focusVisibleRing:"inside",focusRingWidth:"2px",rounded:"l1"}},variants:{variant:{outline:{trigger:{bg:"transparent",borderWidth:"1px",borderColor:"border",_expanded:{borderColor:"border.emphasized"}}},subtle:{trigger:{borderWidth:"1px",borderColor:"transparent",bg:"bg.muted"}}},size:{xs:{root:{"--select-trigger-height":"sizes.8","--select-trigger-padding-x":"spacing.2"},content:{p:"1",gap:"1",textStyle:"xs"},trigger:{textStyle:"xs",gap:"1"},item:{py:"1",px:"2"},itemGroupLabel:{py:"1",px:"2"},indicator:{_icon:{width:"3.5",height:"3.5"}}},sm:{root:{"--select-trigger-height":"sizes.9","--select-trigger-padding-x":"spacing.2.5"},content:{p:"1",textStyle:"sm"},trigger:{textStyle:"sm",gap:"1"},indicator:{_icon:{width:"4",height:"4"}},item:{py:"1",px:"1.5"},itemGroup:{mt:"1"},itemGroupLabel:{py:"1",px:"1.5"}},md:{root:{"--select-trigger-height":"sizes.10","--select-trigger-padding-x":"spacing.3"},content:{p:"1",textStyle:"sm"},itemGroup:{mt:"1.5"},item:{py:"1.5",px:"2"},itemIndicator:{display:"flex",alignItems:"center",justifyContent:"center"},itemGroupLabel:{py:"1.5",px:"2"},trigger:{textStyle:"sm",gap:"2"},indicator:{_icon:{width:"4",height:"4"}}},lg:{root:{"--select-trigger-height":"sizes.12","--select-trigger-padding-x":"spacing.4"},content:{p:"1.5",textStyle:"md"},itemGroup:{mt:"2"},item:{py:"2",px:"3"},itemGroupLabel:{py:"2",px:"3"},trigger:{textStyle:"md",py:"3",gap:"2"},indicator:{_icon:{width:"5",height:"5"}}}}},defaultVariants:{size:"md",variant:"outline"}}),DS=B({className:"chakra-native-select",slots:qw.keys(),base:{root:{height:"fit-content",display:"flex",width:"100%",position:"relative"},field:{width:"100%",minWidth:"0",outline:"0",appearance:"none",borderRadius:"l2","--error-color":"colors.border.error","--input-height":"var(--select-field-height)",height:"var(--select-field-height)",_disabled:{layerStyle:"disabled"},_invalid:{focusRingColor:"var(--error-color)",borderColor:"var(--error-color)"},focusVisibleRing:"inside",lineHeight:"normal","& > option, & > optgroup":{bg:"bg"}},indicator:{position:"absolute",display:"inline-flex",alignItems:"center",justifyContent:"center",pointerEvents:"none",top:"50%",transform:"translateY(-50%)",height:"100%",color:"fg.muted",_disabled:{opacity:"0.5"},_invalid:{color:"fg.error"},_icon:{width:"1em",height:"1em"}}},variants:{variant:{outline:{field:di.variants?.variant.outline.trigger},subtle:{field:di.variants?.variant.subtle.trigger},plain:{field:{bg:"transparent",color:"fg",focusRingWidth:"2px"}}},size:{xs:{root:{"--select-field-height":"sizes.8"},field:{textStyle:"xs",ps:"2",pe:"6"},indicator:{textStyle:"sm",insetEnd:"1.5"}},sm:{root:{"--select-field-height":"sizes.9"},field:{textStyle:"sm",ps:"2.5",pe:"8"},indicator:{textStyle:"md",insetEnd:"2"}},md:{root:{"--select-field-height":"sizes.10"},field:{textStyle:"sm",ps:"3",pe:"8"},indicator:{textStyle:"lg",insetEnd:"2"}},lg:{root:{"--select-field-height":"sizes.11"},field:{textStyle:"md",ps:"4",pe:"8"},indicator:{textStyle:"xl",insetEnd:"3"}},xl:{root:{"--select-field-height":"sizes.12"},field:{textStyle:"md",ps:"4.5",pe:"10"},indicator:{textStyle:"xl",insetEnd:"3"}}}},defaultVariants:di.defaultVariants});function Ta(e,t){const n={};for(const r in e){const o=t(r,e[r]);n[o[0]]=o[1]}return n}const Gh=tr({display:"flex",justifyContent:"center",alignItems:"center",flex:"1",userSelect:"none",cursor:"button",lineHeight:"1",color:"fg.muted","--stepper-base-radius":"radii.l1","--stepper-radius":"calc(var(--stepper-base-radius) + 1px)",_icon:{boxSize:"1em"},_disabled:{opacity:"0.5"},_hover:{bg:"bg.muted"},_active:{bg:"bg.emphasized"}}),zS=B({className:"chakra-number-input",slots:Nd.keys(),base:{root:{position:"relative",zIndex:"0",isolation:"isolate"},input:{...he.base,verticalAlign:"top",pe:"calc(var(--stepper-width) + 0.5rem)"},control:{display:"flex",flexDirection:"column",position:"absolute",top:"0",insetEnd:"0px",margin:"1px",width:"var(--stepper-width)",height:"calc(100% - 2px)",zIndex:"1",borderStartWidth:"1px",divideY:"1px"},incrementTrigger:{...Gh,borderTopEndRadius:"var(--stepper-radius)"},decrementTrigger:{...Gh,borderBottomEndRadius:"var(--stepper-radius)"},valueText:{fontWeight:"medium",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums"}},variants:{size:{xs:{input:he.variants.size.xs,control:{fontSize:"2xs","--stepper-width":"sizes.4"}},sm:{input:he.variants.size.sm,control:{fontSize:"xs","--stepper-width":"sizes.5"}},md:{input:he.variants.size.md,control:{fontSize:"sm","--stepper-width":"sizes.6"}},lg:{input:he.variants.size.lg,control:{fontSize:"sm","--stepper-width":"sizes.6"}}},variant:Ta(he.variants.variant,(e,t)=>[e,{input:t}])},defaultVariants:{size:"md",variant:"outline"}}),{variants:qh,defaultVariants:MS}=he,jS=B({className:"chakra-pin-input",slots:Dd.keys(),base:{input:{...he.base,textAlign:"center",width:"var(--input-height)"},control:{display:"inline-flex",gap:"2",isolation:"isolate"}},variants:{size:Ta(qh.size,(e,t)=>[e,{input:{...t,px:"1"}}]),variant:Ta(qh.variant,(e,t)=>[e,{input:t}]),attached:{true:{control:{gap:"0",spaceX:"-1px"},input:{_notFirst:{borderStartRadius:"0"},_notLast:{borderEndRadius:"0"},_focusVisible:{zIndex:"1"}}}}},defaultVariants:MS}),$S=B({className:"chakra-popover",slots:Kw.keys(),base:{content:{position:"relative",display:"flex",flexDirection:"column",textStyle:"sm","--popover-bg":"colors.bg.panel",bg:"var(--popover-bg)",boxShadow:"lg","--popover-size":"sizes.xs","--popover-mobile-size":"calc(100dvw - 1rem)",width:{base:"min(var(--popover-mobile-size), var(--popover-size))",sm:"var(--popover-size)"},borderRadius:"l3","--popover-z-index":"zIndex.popover",zIndex:"calc(var(--popover-z-index) + var(--layer-index, 0))",outline:"0",transformOrigin:"var(--transform-origin)",maxHeight:"var(--available-height)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"faster"}},header:{paddingInline:"var(--popover-padding)",paddingTop:"var(--popover-padding)"},body:{padding:"var(--popover-padding)",flex:"1"},footer:{display:"flex",alignItems:"center",paddingInline:"var(--popover-padding)",paddingBottom:"var(--popover-padding)"},arrow:{"--arrow-size":"sizes.3","--arrow-background":"var(--popover-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px"}},variants:{size:{xs:{content:{"--popover-padding":"spacing.3"}},sm:{content:{"--popover-padding":"spacing.4"}},md:{content:{"--popover-padding":"spacing.5"}},lg:{content:{"--popover-padding":"spacing.6"}}}},defaultVariants:{size:"md"}}),BS=B({slots:ha.keys(),className:"chakra-progress",base:{root:{textStyle:"sm",position:"relative"},track:{overflow:"hidden",position:"relative"},range:{display:"flex",alignItems:"center",justifyContent:"center",transitionProperty:"width, height",transitionDuration:"slow",height:"100%",bgColor:"var(--track-color)",_indeterminate:{"--animate-from-x":"-40%","--animate-to-x":"100%",position:"absolute",willChange:"left",minWidth:"50%",animation:"position 1s ease infinite normal none running",backgroundImage:"linear-gradient(to right, transparent 0%, var(--track-color) 50%, transparent 100%)"}},label:{display:"inline-flex",fontWeight:"medium",alignItems:"center",gap:"1"},valueText:{textStyle:"xs",lineHeight:"1",fontWeight:"medium"}},variants:{variant:{outline:{track:{shadow:"inset",bgColor:"bg.muted"},range:{bgColor:"colorPalette.solid"}},subtle:{track:{bgColor:"colorPalette.muted"},range:{bgColor:"colorPalette.solid/72"}}},shape:{square:{},rounded:{track:{borderRadius:"l1"}},full:{track:{borderRadius:"full"}}},striped:{true:{range:{backgroundImage:"linear-gradient(45deg, var(--stripe-color) 25%, transparent 25%, transparent 50%, var(--stripe-color) 50%, var(--stripe-color) 75%, transparent 75%, transparent)",backgroundSize:"var(--stripe-size) var(--stripe-size)","--stripe-size":"1rem","--stripe-color":{_light:"rgba(255, 255, 255, 0.3)",_dark:"rgba(0, 0, 0, 0.3)"}}}},animated:{true:{range:{"--animate-from":"var(--stripe-size)",animation:"bg-position 1s linear infinite"}}},size:{xs:{track:{h:"1.5"}},sm:{track:{h:"2"}},md:{track:{h:"2.5"}},lg:{track:{h:"3"}},xl:{track:{h:"4"}}}},defaultVariants:{variant:"outline",size:"md",shape:"rounded"}}),WS=B({className:"chakra-progress-circle",slots:ha.keys(),base:{root:{display:"inline-flex",textStyle:"sm",position:"relative"},circle:{_indeterminate:{animation:"spin 2s linear infinite"}},circleTrack:{"--track-color":"colors.colorPalette.muted",stroke:"var(--track-color)"},circleRange:{stroke:"colorPalette.solid",transitionProperty:"stroke-dashoffset, stroke-dasharray",transitionDuration:"0.6s",_indeterminate:{animation:"circular-progress 1.5s linear infinite"}},label:{display:"inline-flex"},valueText:{lineHeight:"1",fontWeight:"medium",letterSpacing:"tight",fontVariantNumeric:"tabular-nums"}},variants:{size:{xs:{circle:{"--size":"24px","--thickness":"4px"},valueText:{textStyle:"2xs"}},sm:{circle:{"--size":"32px","--thickness":"5px"},valueText:{textStyle:"2xs"}},md:{circle:{"--size":"40px","--thickness":"6px"},valueText:{textStyle:"xs"}},lg:{circle:{"--size":"48px","--thickness":"7px"},valueText:{textStyle:"sm"}},xl:{circle:{"--size":"64px","--thickness":"8px"},valueText:{textStyle:"sm"}}}},defaultVariants:{size:"md"}}),HS=B({slots:Md.keys(),className:"chakra-qr-code",base:{root:{position:"relative",width:"fit-content","--qr-code-overlay-size":"calc(var(--qr-code-size) / 3)"},frame:{width:"var(--qr-code-size)",height:"var(--qr-code-size)",fill:"currentColor"},overlay:{display:"flex",alignItems:"center",justifyContent:"center",width:"var(--qr-code-overlay-size)",height:"var(--qr-code-overlay-size)",padding:"1",bg:"bg",rounded:"l1"}},variants:{size:{"2xs":{root:{"--qr-code-size":"40px"}},xs:{root:{"--qr-code-size":"64px"}},sm:{root:{"--qr-code-size":"80px"}},md:{root:{"--qr-code-size":"120px"}},lg:{root:{"--qr-code-size":"160px"}},xl:{root:{"--qr-code-size":"200px"}},"2xl":{root:{"--qr-code-size":"240px"}},full:{root:{"--qr-code-size":"100%"}}}},defaultVariants:{size:"md"}}),US=B({className:"chakra-radio-card",slots:Yw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1.5",isolation:"isolate"},item:{flex:"1",display:"flex",flexDirection:"column",userSelect:"none",position:"relative",borderRadius:"l2",_focus:{bg:"colorPalette.muted/20"},_disabled:{opacity:"0.8",borderColor:"border.disabled"},_checked:{zIndex:"1"}},label:{display:"inline-flex",fontWeight:"medium",textStyle:"sm",_disabled:{opacity:"0.5"}},itemText:{fontWeight:"medium",flex:"1"},itemDescription:{opacity:"0.64",textStyle:"sm"},itemControl:{display:"inline-flex",flex:"1",pos:"relative",rounded:"inherit",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)",_disabled:{bg:"bg.muted"}},itemIndicator:Ee.base,itemAddon:{roundedBottom:"inherit",_disabled:{color:"fg.muted"}},itemContent:{display:"flex",flexDirection:"column",flex:"1",gap:"1",justifyContent:"var(--radio-card-justify)",alignItems:"var(--radio-card-align)"}},variants:{size:{sm:{item:{textStyle:"sm"},itemControl:{padding:"3",gap:"1.5"},itemAddon:{px:"3",py:"1.5",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.sm},md:{item:{textStyle:"sm"},itemControl:{padding:"4",gap:"2.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.md},lg:{item:{textStyle:"md"},itemControl:{padding:"4",gap:"3.5"},itemAddon:{px:"4",py:"2",borderTopWidth:"1px"},itemIndicator:Ee.variants?.size.lg}},variant:{surface:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.subtle",color:"colorPalette.fg",borderColor:"colorPalette.muted"}},itemIndicator:Ee.variants?.variant.solid},subtle:{item:{bg:"bg.muted"},itemControl:{_checked:{bg:"colorPalette.muted",color:"colorPalette.fg"}},itemIndicator:Ee.variants?.variant.outline},outline:{item:{borderWidth:"1px",_checked:{boxShadow:"0 0 0 1px var(--shadow-color)",boxShadowColor:"colorPalette.solid",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.solid},solid:{item:{borderWidth:"1px",_checked:{bg:"colorPalette.solid",color:"colorPalette.contrast",borderColor:"colorPalette.solid"}},itemIndicator:Ee.variants?.variant.inverted}},justify:{start:{item:{"--radio-card-justify":"flex-start"}},end:{item:{"--radio-card-justify":"flex-end"}},center:{item:{"--radio-card-justify":"center"}}},align:{start:{item:{"--radio-card-align":"flex-start"},itemControl:{textAlign:"start"}},end:{item:{"--radio-card-align":"flex-end"},itemControl:{textAlign:"end"}},center:{item:{"--radio-card-align":"center"},itemControl:{textAlign:"center"}}},orientation:{vertical:{itemControl:{flexDirection:"column"}},horizontal:{itemControl:{flexDirection:"row"}}}},defaultVariants:{size:"md",variant:"outline",align:"start",orientation:"horizontal"}}),GS=B({className:"chakra-radio-group",slots:Hh.keys(),base:{item:{display:"inline-flex",alignItems:"center",position:"relative",fontWeight:"medium",_disabled:{cursor:"disabled"}},itemControl:Ee.base,label:{userSelect:"none",textStyle:"sm",_disabled:{opacity:"0.5"}}},variants:{variant:{outline:{itemControl:Ee.variants?.variant?.outline},subtle:{itemControl:Ee.variants?.variant?.subtle},solid:{itemControl:Ee.variants?.variant?.solid}},size:{xs:{item:{textStyle:"xs",gap:"1.5"},itemControl:Ee.variants?.size?.xs},sm:{item:{textStyle:"sm",gap:"2"},itemControl:Ee.variants?.size?.sm},md:{item:{textStyle:"sm",gap:"2.5"},itemControl:Ee.variants?.size?.md},lg:{item:{textStyle:"md",gap:"3"},itemControl:Ee.variants?.size?.lg}}},defaultVariants:{size:"md",variant:"solid"}}),qS=B({className:"chakra-rating-group",slots:Xw.keys(),base:{root:{display:"inline-flex"},control:{display:"inline-flex",alignItems:"center"},item:{display:"inline-flex",alignItems:"center",justifyContent:"center",userSelect:"none"},itemIndicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",width:"1em",height:"1em",position:"relative","--clip-path":{base:"inset(0 50% 0 0)",_rtl:"inset(0 0 0 50%)"},_icon:{stroke:"currentColor",width:"100%",height:"100%",display:"inline-block",flexShrink:0,position:"absolute",left:0,top:0},"& [data-bg]":{color:"bg.emphasized"},"& [data-fg]":{color:"transparent"},"&[data-highlighted]:not([data-half])":{"& [data-fg]":{color:"colorPalette.solid"}},"&[data-half]":{"& [data-fg]":{color:"colorPalette.solid",clipPath:"var(--clip-path)"}}}},variants:{size:{xs:{item:{textStyle:"sm"}},sm:{item:{textStyle:"md"}},md:{item:{textStyle:"xl"}},lg:{item:{textStyle:"2xl"}}}},defaultVariants:{size:"md"}}),KS=B({className:"chakra-scroll-area",slots:$d.keys(),base:{root:{display:"flex",flexDirection:"column",width:"100%",height:"100%",position:"relative",overflow:"hidden","--scrollbar-margin":"2px","--scrollbar-click-area":"calc(var(--scrollbar-size) + calc(var(--scrollbar-margin) * 2))"},viewport:{display:"flex",flexDirection:"column",height:"100%",width:"100%",borderRadius:"inherit",WebkitOverflowScrolling:"touch",scrollbarWidth:"none","&::-webkit-scrollbar":{display:"none"}},content:{minWidth:"100%"},scrollbar:{display:"flex",userSelect:"none",touchAction:"none",borderRadius:"full",colorPalette:"gray",transition:"opacity 150ms 300ms",position:"relative",margin:"var(--scrollbar-margin)","&:not([data-overflow-x], [data-overflow-y])":{display:"none"},bg:"{colors.colorPalette.solid/10}","--thumb-bg":"{colors.colorPalette.solid/25}","&:is(:hover, :active)":{"--thumb-bg":"{colors.colorPalette.solid/50}"},_before:{content:'""',position:"absolute"},_vertical:{width:"var(--scrollbar-size)",flexDirection:"column","&::before":{width:"var(--scrollbar-click-area)",height:"100%",insetInlineStart:"calc(var(--scrollbar-margin) * -1)"}},_horizontal:{height:"var(--scrollbar-size)",flexDirection:"row","&::before":{height:"var(--scrollbar-click-area)",width:"100%",top:"calc(var(--scrollbar-margin) * -1)"}}},thumb:{borderRadius:"inherit",bg:"var(--thumb-bg)",transition:"backgrounds",_vertical:{width:"full"},_horizontal:{height:"full"}},corner:{bg:"bg.muted",margin:"var(--scrollbar-margin)",opacity:0,transition:"opacity 150ms 300ms","&[data-hover]":{transitionDelay:"0ms",opacity:1}}},variants:{variant:{hover:{scrollbar:{opacity:"0","&[data-hover], &[data-scrolling]":{opacity:"1",transitionDuration:"faster",transitionDelay:"0ms"}}},always:{scrollbar:{opacity:"1"}}},size:{xs:{root:{"--scrollbar-size":"sizes.1"}},sm:{root:{"--scrollbar-size":"sizes.1.5"}},md:{root:{"--scrollbar-size":"sizes.2"}},lg:{root:{"--scrollbar-size":"sizes.3"}}}},defaultVariants:{size:"md",variant:"hover"}}),YS=B({className:"chakra-segment-group",slots:Bd.keys(),base:{root:{"--segment-radius":"radii.l2",borderRadius:"l2",display:"inline-flex",boxShadow:"inset",minW:"max-content",textAlign:"center",position:"relative",isolation:"isolate",bg:"bg.muted",_vertical:{flexDirection:"column"}},item:{display:"flex",alignItems:"center",justifyContent:"center",userSelect:"none",fontSize:"sm",position:"relative",color:"fg",borderRadius:"var(--segment-radius)",_disabled:{opacity:"0.5"},"&:has(input:focus-visible)":{focusRing:"outside"},_before:{content:'""',position:"absolute",bg:"border",transition:"opacity 0.2s"},_horizontal:{_before:{insetInlineStart:0,insetBlock:"1.5",width:"1px"}},_vertical:{_before:{insetBlockStart:0,insetInline:"1.5",height:"1px"}},"& + &[data-state=checked], &[data-state=checked] + &, &:first-of-type":{_before:{opacity:"0"}},"&[data-state=checked][data-ssr]":{shadow:"sm",bg:"bg",borderRadius:"var(--segment-radius)"}},indicator:{shadow:"sm",pos:"absolute",bg:{_light:"bg",_dark:"bg.emphasized"},width:"var(--width)",height:"var(--height)",top:"var(--top)",left:"var(--left)",zIndex:-1,borderRadius:"var(--segment-radius)"}},variants:{size:{xs:{item:{textStyle:"xs",px:"3",gap:"1",height:"6"}},sm:{item:{textStyle:"sm",px:"4",gap:"2",height:"8"}},md:{item:{textStyle:"sm",px:"4",gap:"2",height:"10"}},lg:{item:{textStyle:"md",px:"4.5",gap:"3",height:"11"}}}},defaultVariants:{size:"md"}}),XS=B({className:"chakra-slider",slots:Zw.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",textStyle:"sm",position:"relative",isolation:"isolate",touchAction:"none"},label:{fontWeight:"medium",textStyle:"sm"},control:{display:"inline-flex",alignItems:"center",position:"relative"},track:{overflow:"hidden",borderRadius:"full",flex:"1"},range:{width:"inherit",height:"inherit",_disabled:{bg:"border.emphasized!"}},markerGroup:{position:"absolute!",zIndex:"1"},marker:{"--marker-bg":{base:"white",_underValue:"colors.bg"},display:"flex",alignItems:"center",gap:"calc(var(--slider-thumb-size) / 2)",color:"fg.muted",textStyle:"xs"},markerIndicator:{width:"var(--slider-marker-size)",height:"var(--slider-marker-size)",borderRadius:"full",bg:"var(--marker-bg)"},thumb:{width:"var(--slider-thumb-size)",height:"var(--slider-thumb-size)",display:"flex",alignItems:"center",justifyContent:"center",outline:0,zIndex:"2",borderRadius:"full",_focusVisible:{ring:"2px",ringColor:"colorPalette.focusRing",ringOffset:"2px",ringOffsetColor:"bg"}}},variants:{size:{sm:{root:{"--slider-thumb-size":"sizes.4","--slider-track-size":"sizes.1.5","--slider-marker-center":"6px","--slider-marker-size":"sizes.1","--slider-marker-inset":"3px"}},md:{root:{"--slider-thumb-size":"sizes.5","--slider-track-size":"sizes.2","--slider-marker-center":"8px","--slider-marker-size":"sizes.1","--slider-marker-inset":"4px"}},lg:{root:{"--slider-thumb-size":"sizes.6","--slider-track-size":"sizes.2.5","--slider-marker-center":"9px","--slider-marker-size":"sizes.1.5","--slider-marker-inset":"5px"}}},variant:{outline:{track:{shadow:"inset",bg:"bg.emphasized/72"},range:{bg:"colorPalette.solid"},thumb:{borderWidth:"2px",borderColor:"colorPalette.solid",bg:"bg",_disabled:{bg:"border.emphasized",borderColor:"border.emphasized"}}},solid:{track:{bg:"colorPalette.subtle",_disabled:{bg:"bg.muted"}},range:{bg:"colorPalette.solid"},thumb:{bg:"colorPalette.solid",_disabled:{bg:"border.emphasized"}}}},orientation:{vertical:{root:{display:"inline-flex"},control:{flexDirection:"column",height:"100%",minWidth:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginEnd:"4"}},track:{width:"var(--slider-track-size)"},thumb:{left:"50%",translate:"-50% 0"},markerGroup:{insetStart:"var(--slider-marker-center)",insetBlock:"var(--slider-marker-inset)"},marker:{flexDirection:"row"}},horizontal:{control:{flexDirection:"row",width:"100%",minHeight:"var(--slider-thumb-size)","&[data-has-mark-label], &:has(.chakra-slider__marker-label)":{marginBottom:"4"}},track:{height:"var(--slider-track-size)"},thumb:{top:"50%",translate:"0 -50%"},markerGroup:{top:"var(--slider-marker-center)",insetInline:"var(--slider-marker-inset)"},marker:{flexDirection:"column"}}}},defaultVariants:{size:"md",variant:"outline",orientation:"horizontal"}}),QS=B({className:"chakra-stat",slots:eS.keys(),base:{root:{display:"flex",flexDirection:"column",gap:"1",position:"relative",flex:"1"},label:{display:"inline-flex",gap:"1.5",alignItems:"center",color:"fg.muted",textStyle:"sm"},helpText:{color:"fg.muted",textStyle:"xs"},valueUnit:{color:"fg.muted",textStyle:"xs",fontWeight:"initial",letterSpacing:"initial"},valueText:{verticalAlign:"baseline",fontWeight:"semibold",letterSpacing:"tight",fontFeatureSettings:"pnum",fontVariantNumeric:"proportional-nums",display:"inline-flex",gap:"1"},indicator:{display:"inline-flex",alignItems:"center",justifyContent:"center",marginEnd:1,"& :where(svg)":{w:"1em",h:"1em"},"&[data-type=up]":{color:"fg.success"},"&[data-type=down]":{color:"fg.error"}}},variants:{size:{sm:{valueText:{textStyle:"xl"}},md:{valueText:{textStyle:"2xl"}},lg:{valueText:{textStyle:"3xl"}}}},defaultVariants:{size:"md"}}),JS=B({className:"chakra-status",slots:tS.keys(),base:{root:{display:"inline-flex",alignItems:"center",gap:"2"},indicator:{width:"0.64em",height:"0.64em",flexShrink:0,borderRadius:"full",forcedColorAdjust:"none",bg:"colorPalette.solid"}},variants:{size:{sm:{root:{textStyle:"xs"}},md:{root:{textStyle:"sm"}},lg:{root:{textStyle:"md"}}}},defaultVariants:{size:"md"}}),ZS=B({className:"chakra-steps",slots:nS.keys(),base:{root:{display:"flex",width:"full"},list:{display:"flex",justifyContent:"space-between","--steps-gutter":"spacing.3","--steps-thickness":"2px"},title:{fontWeight:"medium",color:"fg"},description:{color:"fg.muted"},separator:{bg:"border",flex:"1"},indicator:{display:"flex",justifyContent:"center",alignItems:"center",flexShrink:"0",borderRadius:"full",fontWeight:"medium",width:"var(--steps-size)",height:"var(--steps-size)",_icon:{flexShrink:"0",width:"var(--steps-icon-size)",height:"var(--steps-icon-size)"}},item:{position:"relative",display:"flex",gap:"3",flex:"1 0 0","&:last-of-type":{flex:"initial","& [data-part=separator]":{display:"none"}}},trigger:{display:"flex",alignItems:"center",gap:"3",textAlign:"start",focusVisibleRing:"outside",borderRadius:"l2"},content:{focusVisibleRing:"outside"}},variants:{orientation:{vertical:{root:{flexDirection:"row",height:"100%"},list:{flexDirection:"column",alignItems:"flex-start"},separator:{position:"absolute",width:"var(--steps-thickness)",height:"100%",maxHeight:"calc(100% - var(--steps-size) - var(--steps-gutter) * 2)",top:"calc(var(--steps-size) + var(--steps-gutter))",insetStart:"calc(var(--steps-size) / 2 - 1px)"},item:{alignItems:"flex-start"}},horizontal:{root:{flexDirection:"column",width:"100%"},list:{flexDirection:"row",alignItems:"center"},separator:{width:"100%",height:"var(--steps-thickness)",marginX:"var(--steps-gutter)"},item:{alignItems:"center"}}},variant:{solid:{indicator:{_incomplete:{borderWidth:"var(--steps-thickness)"},_current:{bg:"colorPalette.muted",borderWidth:"var(--steps-thickness)",borderColor:"colorPalette.solid",color:"colorPalette.fg"},_complete:{bg:"colorPalette.solid",borderColor:"colorPalette.solid",color:"colorPalette.contrast"}},separator:{_complete:{bg:"colorPalette.solid"}}},subtle:{indicator:{_incomplete:{bg:"bg.muted"},_current:{bg:"colorPalette.muted",color:"colorPalette.fg"},_complete:{bg:"colorPalette.emphasized",color:"colorPalette.fg"}},separator:{_complete:{bg:"colorPalette.emphasized"}}}},size:{xs:{root:{gap:"2.5"},list:{"--steps-size":"sizes.6","--steps-icon-size":"sizes.3.5",textStyle:"xs"},title:{textStyle:"sm"}},sm:{root:{gap:"3"},list:{"--steps-size":"sizes.8","--steps-icon-size":"sizes.4",textStyle:"xs"},title:{textStyle:"sm"}},md:{root:{gap:"4"},list:{"--steps-size":"sizes.10","--steps-icon-size":"sizes.4",textStyle:"sm"},title:{textStyle:"sm"}},lg:{root:{gap:"6"},list:{"--steps-size":"sizes.11","--steps-icon-size":"sizes.5",textStyle:"md"},title:{textStyle:"md"}}}},defaultVariants:{size:"md",variant:"solid",orientation:"horizontal"}}),eE=B({slots:rS.keys(),className:"chakra-switch",base:{root:{display:"inline-flex",gap:"2.5",alignItems:"center",position:"relative",verticalAlign:"middle","--switch-diff":"calc(var(--switch-width) - var(--switch-height))","--switch-x":{base:"var(--switch-diff)",_rtl:"calc(var(--switch-diff) * -1)"}},label:{lineHeight:"1",userSelect:"none",fontSize:"sm",fontWeight:"medium",_disabled:{opacity:"0.5"}},indicator:{position:"absolute",height:"var(--switch-height)",width:"var(--switch-height)",fontSize:"var(--switch-indicator-font-size)",fontWeight:"medium",flexShrink:0,userSelect:"none",display:"grid",placeContent:"center",transition:"inset-inline-start 0.12s ease",insetInlineStart:"calc(var(--switch-x) - 2px)",_checked:{insetInlineStart:"2px"}},control:{display:"inline-flex",gap:"0.5rem",flexShrink:0,justifyContent:"flex-start",cursor:"switch",borderRadius:"full",position:"relative",width:"var(--switch-width)",height:"var(--switch-height)",transition:"backgrounds",_disabled:{opacity:"0.5",cursor:"not-allowed"},_invalid:{outline:"2px solid",outlineColor:"border.error",outlineOffset:"2px"}},thumb:{display:"flex",alignItems:"center",justifyContent:"center",flexShrink:0,transitionProperty:"translate",transitionDuration:"fast",borderRadius:"inherit",_checked:{translate:"var(--switch-x) 0"}}},variants:{variant:{solid:{control:{borderRadius:"full",bg:"bg.emphasized",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}},thumb:{bg:"white",width:"var(--switch-height)",height:"var(--switch-height)",scale:"0.8",boxShadow:"sm",_checked:{bg:"colorPalette.contrast"}}},raised:{control:{borderRadius:"full",height:"calc(var(--switch-height) / 2)",bg:"bg.muted",boxShadow:"inset",_checked:{bg:"colorPalette.solid/60"}},thumb:{width:"var(--switch-height)",height:"var(--switch-height)",position:"relative",top:"calc(var(--switch-height) * -0.25)",bg:"white",boxShadow:"xs",focusVisibleRing:"outside",_checked:{bg:"colorPalette.solid"}}}},size:{xs:{root:{"--switch-width":"sizes.6","--switch-height":"sizes.3","--switch-indicator-font-size":"fontSizes.xs"}},sm:{root:{"--switch-width":"sizes.8","--switch-height":"sizes.4","--switch-indicator-font-size":"fontSizes.xs"}},md:{root:{"--switch-width":"sizes.10","--switch-height":"sizes.5","--switch-indicator-font-size":"fontSizes.sm"}},lg:{root:{"--switch-width":"sizes.12","--switch-height":"sizes.6","--switch-indicator-font-size":"fontSizes.md"}}}},defaultVariants:{variant:"solid",size:"md"}}),tE=B({className:"chakra-table",slots:oS.keys(),base:{root:{fontVariantNumeric:"lining-nums tabular-nums",borderCollapse:"collapse",width:"full",textAlign:"start",verticalAlign:"top"},row:{_selected:{bg:"colorPalette.subtle"}},cell:{textAlign:"start",alignItems:"center"},columnHeader:{fontWeight:"medium",textAlign:"start",color:"fg"},caption:{fontWeight:"medium",textStyle:"xs"},footer:{fontWeight:"medium"}},variants:{interactive:{true:{body:{"& tr":{_hover:{bg:"colorPalette.subtle"}}}}},stickyHeader:{true:{header:{"& :where(tr)":{top:"var(--table-sticky-offset, 0)",position:"sticky",zIndex:1}}}},striped:{true:{row:{"&:nth-of-type(odd) td":{bg:"bg.muted"}}}},showColumnBorder:{true:{columnHeader:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}},cell:{"&:not(:last-of-type)":{borderInlineEndWidth:"1px"}}}},variant:{line:{columnHeader:{borderBottomWidth:"1px"},cell:{borderBottomWidth:"1px"},row:{bg:"bg"}},outline:{root:{boxShadow:"0 0 0 1px {colors.border}",overflow:"hidden"},columnHeader:{borderBottomWidth:"1px"},header:{bg:"bg.muted"},row:{"&:not(:last-of-type)":{borderBottomWidth:"1px"}},footer:{borderTopWidth:"1px"}}},size:{sm:{root:{textStyle:"sm"},columnHeader:{px:"2",py:"2"},cell:{px:"2",py:"2"}},md:{root:{textStyle:"sm"},columnHeader:{px:"3",py:"3"},cell:{px:"3",py:"3"}},lg:{root:{textStyle:"md"},columnHeader:{px:"4",py:"3"},cell:{px:"4",py:"3"}}}},defaultVariants:{variant:"line",size:"md"}}),nE=B({slots:sS.keys(),className:"chakra-tabs",base:{root:{"--tabs-trigger-radius":"radii.l2",position:"relative",_horizontal:{display:"block"},_vertical:{display:"flex"}},list:{display:"inline-flex",position:"relative",isolation:"isolate","--tabs-indicator-shadow":"shadows.xs","--tabs-indicator-bg":"colors.bg",minH:"var(--tabs-height)",_horizontal:{flexDirection:"row"},_vertical:{flexDirection:"column"}},trigger:{outline:"0",minW:"var(--tabs-height)",height:"var(--tabs-height)",display:"flex",alignItems:"center",fontWeight:"medium",position:"relative",cursor:"button",gap:"2",_focusVisible:{zIndex:1,outline:"2px solid",outlineColor:"colorPalette.focusRing"},_disabled:{cursor:"not-allowed",opacity:.5}},content:{focusVisibleRing:"inside",_horizontal:{width:"100%",pt:"var(--tabs-content-padding)"},_vertical:{height:"100%",ps:"var(--tabs-content-padding)"}},indicator:{width:"var(--width)",height:"var(--height)",borderRadius:"var(--tabs-indicator-radius)",bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",zIndex:-1}},variants:{fitted:{true:{list:{display:"flex"},trigger:{flex:1,textAlign:"center",justifyContent:"center"}}},justify:{start:{list:{justifyContent:"flex-start"}},center:{list:{justifyContent:"center"}},end:{list:{justifyContent:"flex-end"}}},size:{sm:{root:{"--tabs-height":"sizes.9","--tabs-content-padding":"spacing.3"},trigger:{py:"1",px:"3",textStyle:"sm"}},md:{root:{"--tabs-height":"sizes.10","--tabs-content-padding":"spacing.4"},trigger:{py:"2",px:"4",textStyle:"sm"}},lg:{root:{"--tabs-height":"sizes.11","--tabs-content-padding":"spacing.4.5"},trigger:{py:"2",px:"4.5",textStyle:"md"}}},variant:{line:{list:{display:"flex",borderColor:"border",_horizontal:{borderBottomWidth:"1px"},_vertical:{borderEndWidth:"1px"}},trigger:{color:"fg.muted",_disabled:{_active:{bg:"initial"}},_selected:{color:"fg",_horizontal:{layerStyle:"indicator.bottom","--indicator-offset-y":"-1px","--indicator-color":"colors.colorPalette.solid"},_vertical:{layerStyle:"indicator.end","--indicator-offset-x":"-1px"}}}},subtle:{trigger:{borderRadius:"var(--tabs-trigger-radius)",color:"fg.muted",_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}},enclosed:{list:{bg:"bg.muted",padding:"1",borderRadius:"l3",minH:"calc(var(--tabs-height) - 4px)"},trigger:{justifyContent:"center",color:"fg.muted",borderRadius:"var(--tabs-trigger-radius)",_selected:{bg:"bg",color:"colorPalette.fg",shadow:"xs"}}},outline:{list:{"--line-thickness":"1px","--line-offset":"calc(var(--line-thickness) * -1)",borderColor:"border",display:"flex",_horizontal:{_before:{content:'""',position:"absolute",bottom:"0px",width:"100%",borderBottomWidth:"var(--line-thickness)",borderBottomColor:"border"}},_vertical:{_before:{content:'""',position:"absolute",insetInline:"var(--line-offset)",height:"calc(100% - calc(var(--line-thickness) * 2))",borderEndWidth:"var(--line-thickness)",borderEndColor:"border"}}},trigger:{color:"fg.muted",borderWidth:"1px",borderColor:"transparent",_selected:{bg:"currentBg",color:"colorPalette.fg"},_horizontal:{borderTopRadius:"var(--tabs-trigger-radius)",marginBottom:"var(--line-offset)",marginEnd:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderBottomColor:"transparent"}},_vertical:{borderStartRadius:"var(--tabs-trigger-radius)",marginEnd:"var(--line-offset)",marginBottom:{_notLast:"var(--line-offset)"},_selected:{borderColor:"border",borderEndColor:"transparent"}}}},plain:{trigger:{color:"fg.muted",_selected:{color:"colorPalette.fg"},borderRadius:"var(--tabs-trigger-radius)","&[data-selected][data-ssr]":{bg:"var(--tabs-indicator-bg)",shadow:"var(--tabs-indicator-shadow)",borderRadius:"var(--tabs-indicator-radius)"}}}}},defaultVariants:{size:"md",variant:"line"}}),hi=Ia.variants?.variant,rE=B({slots:aS.keys(),className:"chakra-tag",base:{root:{display:"inline-flex",alignItems:"center",verticalAlign:"top",maxWidth:"100%",userSelect:"none",borderRadius:"l2",focusVisibleRing:"outside"},label:{lineClamp:"1"},closeTrigger:{display:"flex",alignItems:"center",justifyContent:"center",outline:"0",borderRadius:"l1",color:"currentColor",focusVisibleRing:"inside",focusRingWidth:"2px"},startElement:{flexShrink:0,boxSize:"var(--tag-element-size)",ms:"var(--tag-element-offset)","&:has([data-scope=avatar])":{boxSize:"var(--tag-avatar-size)",ms:"calc(var(--tag-element-offset) * 1.5)"},_icon:{boxSize:"100%"}},endElement:{flexShrink:0,boxSize:"var(--tag-element-size)",me:"var(--tag-element-offset)",_icon:{boxSize:"100%"},"&:has(button)":{ms:"calc(var(--tag-element-offset) * -1)"}}},variants:{size:{sm:{root:{px:"1.5",minH:"4.5",gap:"1","--tag-avatar-size":"spacing.3","--tag-element-size":"spacing.3","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},md:{root:{px:"1.5",minH:"5",gap:"1","--tag-avatar-size":"spacing.3.5","--tag-element-size":"spacing.3.5","--tag-element-offset":"-2px"},label:{textStyle:"xs"}},lg:{root:{px:"2",minH:"6",gap:"1.5","--tag-avatar-size":"spacing.4.5","--tag-element-size":"spacing.4","--tag-element-offset":"-3px"},label:{textStyle:"sm"}},xl:{root:{px:"2.5",minH:"8",gap:"1.5","--tag-avatar-size":"spacing.6","--tag-element-size":"spacing.4.5","--tag-element-offset":"-4px"},label:{textStyle:"sm"}}},variant:{subtle:{root:hi?.subtle},solid:{root:hi?.solid},outline:{root:hi?.outline},surface:{root:hi?.surface}}},defaultVariants:{size:"md",variant:"surface"}}),oE=B({slots:lS.keys(),className:"chakra-timeline",base:{root:{display:"flex",flexDirection:"column",width:"full","--timeline-thickness":"1px","--timeline-gutter":"4px"},item:{display:"flex",position:"relative",alignItems:"flex-start",flexShrink:0,gap:"4",_last:{"& :where(.chakra-timeline__separator)":{display:"none"}}},separator:{position:"absolute",borderStartWidth:"var(--timeline-thickness)",ms:"calc(-1 * var(--timeline-thickness) / 2)",insetInlineStart:"calc(var(--timeline-indicator-size) / 2)",insetBlock:"0",borderColor:"border"},indicator:{outline:"2px solid {colors.bg}",position:"relative",flexShrink:"0",boxSize:"var(--timeline-indicator-size)",fontSize:"var(--timeline-font-size)",display:"flex",alignItems:"center",justifyContent:"center",borderRadius:"full",fontWeight:"medium"},connector:{alignSelf:"stretch",position:"relative"},content:{pb:"6",display:"flex",flexDirection:"column",width:"full",gap:"2"},title:{display:"flex",fontWeight:"medium",flexWrap:"wrap",gap:"1.5",alignItems:"center",mt:"var(--timeline-margin)"},description:{color:"fg.muted",textStyle:"xs"}},variants:{variant:{subtle:{indicator:{bg:"colorPalette.muted"}},solid:{indicator:{bg:"colorPalette.solid",color:"colorPalette.contrast"}},outline:{indicator:{bg:"currentBg",borderWidth:"1px",borderColor:"colorPalette.muted"}},plain:{}},size:{sm:{root:{"--timeline-indicator-size":"sizes.4","--timeline-font-size":"fontSizes.2xs"},title:{textStyle:"xs"}},md:{root:{"--timeline-indicator-size":"sizes.5","--timeline-font-size":"fontSizes.xs"},title:{textStyle:"sm"}},lg:{root:{"--timeline-indicator-size":"sizes.6","--timeline-font-size":"fontSizes.xs"},title:{mt:"0.5",textStyle:"sm"}},xl:{root:{"--timeline-indicator-size":"sizes.8","--timeline-font-size":"fontSizes.sm"},title:{mt:"1.5",textStyle:"sm"}}}},defaultVariants:{size:"md",variant:"solid"}}),iE=B({slots:iS.keys(),className:"chakra-toast",base:{root:{width:"full",display:"flex",alignItems:"flex-start",position:"relative",gap:"3",py:"4",ps:"4",pe:"6",borderRadius:"l2",translate:"var(--x) var(--y)",scale:"var(--scale)",zIndex:"var(--z-index)",height:"var(--height)",opacity:"var(--opacity)",willChange:"translate, opacity, scale",transition:"translate 400ms, scale 400ms, opacity 400ms, height 400ms, box-shadow 200ms",transitionTimingFunction:"cubic-bezier(0.21, 1.02, 0.73, 1)",_closed:{transition:"translate 400ms, scale 400ms, opacity 200ms",transitionTimingFunction:"cubic-bezier(0.06, 0.71, 0.55, 1)"},bg:"bg.panel",color:"fg",boxShadow:"xl","--toast-trigger-bg":"colors.bg.muted","&[data-type=warning]":{bg:"orange.solid",color:"orange.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=success]":{bg:"green.solid",color:"green.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"},"&[data-type=error]":{bg:"red.solid",color:"red.contrast","--toast-trigger-bg":"{white/10}","--toast-border-color":"{white/40}"}},title:{fontWeight:"medium",textStyle:"sm",marginEnd:"2"},description:{display:"inline",textStyle:"sm",opacity:"0.8"},indicator:{flexShrink:"0",boxSize:"5"},actionTrigger:{textStyle:"sm",fontWeight:"medium",height:"8",px:"3",borderRadius:"l2",alignSelf:"center",borderWidth:"1px",borderColor:"var(--toast-border-color, inherit)",transition:"background 200ms",_hover:{bg:"var(--toast-trigger-bg)"}},closeTrigger:{position:"absolute",top:"1",insetEnd:"1",padding:"1",display:"inline-flex",alignItems:"center",justifyContent:"center",color:"{currentColor/60}",borderRadius:"l2",textStyle:"md",transition:"background 200ms",_icon:{boxSize:"1em"}}}}),sE=B({slots:Yd.keys(),className:"chakra-tooltip",base:{content:{"--tooltip-bg":"colors.bg.inverted",bg:"var(--tooltip-bg)",color:"fg.inverted",px:"2.5",py:"1",borderRadius:"l2",fontWeight:"medium",textStyle:"xs",boxShadow:"md",maxW:"xs",zIndex:"tooltip",transformOrigin:"var(--transform-origin)",_open:{animationStyle:"scale-fade-in",animationDuration:"fast"},_closed:{animationStyle:"scale-fade-out",animationDuration:"fast"}},arrow:{"--arrow-size":"sizes.2","--arrow-background":"var(--tooltip-bg)"},arrowTip:{borderTopWidth:"1px",borderInlineStartWidth:"1px",borderColor:"var(--tooltip-bg)"}}}),Kh=tr({display:"flex",alignItems:"center",gap:"var(--tree-item-gap)",rounded:"l2",userSelect:"none",position:"relative","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-icon-offset":"calc(var(--tree-icon-size) * var(--tree-depth) * 0.5)","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset) + var(--tree-icon-offset))",ps:"var(--tree-offset)",pe:"var(--tree-padding-inline)",py:"var(--tree-padding-block)",focusVisibleRing:"inside",focusRingColor:"border.emphasized",focusRingWidth:"2px","&:hover, &:focus-visible":{bg:"bg.muted"},_disabled:{layerStyle:"disabled"}}),Yh=tr({flex:"1"}),Xh=tr({_selected:{bg:"colorPalette.subtle",color:"colorPalette.fg"}}),Qh=tr({_selected:{layerStyle:"fill.solid"}}),aE=B({slots:vd.keys(),className:"chakra-tree-view",base:{root:{width:"full",display:"flex",flexDirection:"column",gap:"2"},tree:{display:"flex",flexDirection:"column","--tree-item-gap":"spacing.2",_icon:{boxSize:"var(--tree-icon-size)"}},label:{fontWeight:"medium",textStyle:"sm"},branch:{position:"relative"},branchContent:{position:"relative"},branchIndentGuide:{height:"100%",width:"1px",bg:"border",position:"absolute","--tree-depth":"calc(var(--depth) - 1)","--tree-indentation-offset":"calc(var(--tree-indentation) * var(--tree-depth))","--tree-offset":"calc(var(--tree-padding-inline) + var(--tree-indentation-offset))","--tree-icon-offset":"calc(var(--tree-icon-size) * 0.5 * var(--depth))",insetInlineStart:"calc(var(--tree-offset) + var(--tree-icon-offset))",zIndex:"1"},branchIndicator:{color:"fg.muted",transformOrigin:"center",transitionDuration:"normal",transitionProperty:"transform",transitionTimingFunction:"default",_open:{transform:"rotate(90deg)"}},branchTrigger:{display:"inline-flex",alignItems:"center",justifyContent:"center"},branchControl:Kh,item:Kh,itemText:Yh,branchText:Yh,nodeCheckbox:{display:"inline-flex"}},variants:{size:{md:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1.5","--tree-icon-size":"spacing.4"}},sm:{tree:{textStyle:"sm","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.3","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}},xs:{tree:{textStyle:"xs","--tree-indentation":"spacing.4","--tree-padding-inline":"spacing.2","--tree-padding-block":"spacing.1","--tree-icon-size":"spacing.3"}}},variant:{subtle:{branchControl:Xh,item:Xh},solid:{branchControl:Qh,item:Qh}},animateContent:{true:{branchContent:{_open:{animationName:"expand-height, fade-in",animationDuration:"moderate"},_closed:{animationName:"collapse-height, fade-out",animationDuration:"moderate"}}}}},defaultVariants:{size:"md",variant:"subtle"}}),lE={accordion:hS,actionBar:fS,alert:gS,avatar:pS,blockquote:mS,breadcrumb:vS,card:bS,checkbox:yS,checkboxCard:xS,codeBlock:kS,collapsible:CS,dataList:ES,dialog:OS,drawer:PS,editable:RS,emptyState:IS,field:TS,fieldset:NS,fileUpload:AS,hoverCard:_S,list:VS,listbox:LS,menu:FS,nativeSelect:DS,numberInput:zS,pinInput:jS,popover:$S,progress:BS,progressCircle:WS,radioCard:US,radioGroup:GS,ratingGroup:qS,scrollArea:KS,segmentGroup:YS,select:di,combobox:SS,slider:XS,stat:QS,steps:ZS,switch:eE,table:tE,tabs:nE,tag:rE,toast:iE,tooltip:sE,status:JS,timeline:oE,colorPicker:wS,qrCode:HS,treeView:aE},cE=Hk({"2xs":{value:{fontSize:"2xs",lineHeight:"0.75rem"}},xs:{value:{fontSize:"xs",lineHeight:"1rem"}},sm:{value:{fontSize:"sm",lineHeight:"1.25rem"}},md:{value:{fontSize:"md",lineHeight:"1.5rem"}},lg:{value:{fontSize:"lg",lineHeight:"1.75rem"}},xl:{value:{fontSize:"xl",lineHeight:"1.875rem"}},"2xl":{value:{fontSize:"2xl",lineHeight:"2rem"}},"3xl":{value:{fontSize:"3xl",lineHeight:"2.375rem"}},"4xl":{value:{fontSize:"4xl",lineHeight:"2.75rem",letterSpacing:"-0.025em"}},"5xl":{value:{fontSize:"5xl",lineHeight:"3.75rem",letterSpacing:"-0.025em"}},"6xl":{value:{fontSize:"6xl",lineHeight:"4.5rem",letterSpacing:"-0.025em"}},"7xl":{value:{fontSize:"7xl",lineHeight:"5.75rem",letterSpacing:"-0.025em"}},none:{value:{}},label:{value:{fontSize:"sm",lineHeight:"1.25rem",fontWeight:"medium"}}}),uE=de.animations({spin:{value:"spin 1s linear infinite"},ping:{value:"ping 1s cubic-bezier(0, 0, 0.2, 1) infinite"},pulse:{value:"pulse 2s cubic-bezier(0.4, 0, 0.6, 1) infinite"},bounce:{value:"bounce 1s infinite"}}),dE=de.aspectRatios({square:{value:"1 / 1"},landscape:{value:"4 / 3"},portrait:{value:"3 / 4"},wide:{value:"16 / 9"},ultrawide:{value:"18 / 5"},golden:{value:"1.618 / 1"}}),hE=de.blurs({none:{value:" "},sm:{value:"4px"},md:{value:"8px"},lg:{value:"12px"},xl:{value:"16px"},"2xl":{value:"24px"},"3xl":{value:"40px"},"4xl":{value:"64px"}}),fE=de.borders({xs:{value:"0.5px solid"},sm:{value:"1px solid"},md:{value:"2px solid"},lg:{value:"4px solid"},xl:{value:"8px solid"}}),gE=de.colors({transparent:{value:"transparent"},current:{value:"currentColor"},black:{value:"#09090B"},white:{value:"#FFFFFF"},whiteAlpha:{50:{value:"rgba(255, 255, 255, 0.04)"},100:{value:"rgba(255, 255, 255, 0.06)"},200:{value:"rgba(255, 255, 255, 0.08)"},300:{value:"rgba(255, 255, 255, 0.16)"},400:{value:"rgba(255, 255, 255, 0.24)"},500:{value:"rgba(255, 255, 255, 0.36)"},600:{value:"rgba(255, 255, 255, 0.48)"},700:{value:"rgba(255, 255, 255, 0.64)"},800:{value:"rgba(255, 255, 255, 0.80)"},900:{value:"rgba(255, 255, 255, 0.92)"},950:{value:"rgba(255, 255, 255, 0.95)"}},blackAlpha:{50:{value:"rgba(0, 0, 0, 0.04)"},100:{value:"rgba(0, 0, 0, 0.06)"},200:{value:"rgba(0, 0, 0, 0.08)"},300:{value:"rgba(0, 0, 0, 0.16)"},400:{value:"rgba(0, 0, 0, 0.24)"},500:{value:"rgba(0, 0, 0, 0.36)"},600:{value:"rgba(0, 0, 0, 0.48)"},700:{value:"rgba(0, 0, 0, 0.64)"},800:{value:"rgba(0, 0, 0, 0.80)"},900:{value:"rgba(0, 0, 0, 0.92)"},950:{value:"rgba(0, 0, 0, 0.95)"}},gray:{50:{value:"#fafafa"},100:{value:"#f4f4f5"},200:{value:"#e4e4e7"},300:{value:"#d4d4d8"},400:{value:"#a1a1aa"},500:{value:"#71717a"},600:{value:"#52525b"},700:{value:"#3f3f46"},800:{value:"#27272a"},900:{value:"#18181b"},950:{value:"#111111"}},red:{50:{value:"#fef2f2"},100:{value:"#fee2e2"},200:{value:"#fecaca"},300:{value:"#fca5a5"},400:{value:"#f87171"},500:{value:"#ef4444"},600:{value:"#dc2626"},700:{value:"#991919"},800:{value:"#511111"},900:{value:"#300c0c"},950:{value:"#1f0808"}},orange:{50:{value:"#fff7ed"},100:{value:"#ffedd5"},200:{value:"#fed7aa"},300:{value:"#fdba74"},400:{value:"#fb923c"},500:{value:"#f97316"},600:{value:"#ea580c"},700:{value:"#92310a"},800:{value:"#6c2710"},900:{value:"#3b1106"},950:{value:"#220a04"}},yellow:{50:{value:"#fefce8"},100:{value:"#fef9c3"},200:{value:"#fef08a"},300:{value:"#fde047"},400:{value:"#facc15"},500:{value:"#eab308"},600:{value:"#ca8a04"},700:{value:"#845209"},800:{value:"#713f12"},900:{value:"#422006"},950:{value:"#281304"}},green:{50:{value:"#f0fdf4"},100:{value:"#dcfce7"},200:{value:"#bbf7d0"},300:{value:"#86efac"},400:{value:"#4ade80"},500:{value:"#22c55e"},600:{value:"#16a34a"},700:{value:"#116932"},800:{value:"#124a28"},900:{value:"#042713"},950:{value:"#03190c"}},teal:{50:{value:"#f0fdfa"},100:{value:"#ccfbf1"},200:{value:"#99f6e4"},300:{value:"#5eead4"},400:{value:"#2dd4bf"},500:{value:"#14b8a6"},600:{value:"#0d9488"},700:{value:"#0c5d56"},800:{value:"#114240"},900:{value:"#032726"},950:{value:"#021716"}},blue:{50:{value:"#eff6ff"},100:{value:"#dbeafe"},200:{value:"#bfdbfe"},300:{value:"#a3cfff"},400:{value:"#60a5fa"},500:{value:"#3b82f6"},600:{value:"#2563eb"},700:{value:"#173da6"},800:{value:"#1a3478"},900:{value:"#14204a"},950:{value:"#0c142e"}},cyan:{50:{value:"#ecfeff"},100:{value:"#cffafe"},200:{value:"#a5f3fc"},300:{value:"#67e8f9"},400:{value:"#22d3ee"},500:{value:"#06b6d4"},600:{value:"#0891b2"},700:{value:"#0c5c72"},800:{value:"#134152"},900:{value:"#072a38"},950:{value:"#051b24"}},purple:{50:{value:"#faf5ff"},100:{value:"#f3e8ff"},200:{value:"#e9d5ff"},300:{value:"#d8b4fe"},400:{value:"#c084fc"},500:{value:"#a855f7"},600:{value:"#9333ea"},700:{value:"#641ba3"},800:{value:"#4a1772"},900:{value:"#2f0553"},950:{value:"#1a032e"}},pink:{50:{value:"#fdf2f8"},100:{value:"#fce7f3"},200:{value:"#fbcfe8"},300:{value:"#f9a8d4"},400:{value:"#f472b6"},500:{value:"#ec4899"},600:{value:"#db2777"},700:{value:"#a41752"},800:{value:"#6d0e34"},900:{value:"#45061f"},950:{value:"#2c0514"}}}),pE=de.cursor({button:{value:"pointer"},checkbox:{value:"default"},disabled:{value:"not-allowed"},menuitem:{value:"default"},option:{value:"default"},radio:{value:"default"},slider:{value:"default"},switch:{value:"pointer"}}),mE=de.durations({fastest:{value:"50ms"},faster:{value:"100ms"},fast:{value:"150ms"},moderate:{value:"200ms"},slow:{value:"300ms"},slower:{value:"400ms"},slowest:{value:"500ms"}}),vE=de.easings({"ease-in":{value:"cubic-bezier(0.42, 0, 1, 1)"},"ease-out":{value:"cubic-bezier(0, 0, 0.58, 1)"},"ease-in-out":{value:"cubic-bezier(0.42, 0, 0.58, 1)"},"ease-in-smooth":{value:"cubic-bezier(0.32, 0.72, 0, 1)"}}),bE=de.fontSizes({"2xs":{value:"0.625rem"},xs:{value:"0.75rem"},sm:{value:"0.875rem"},md:{value:"1rem"},lg:{value:"1.125rem"},xl:{value:"1.25rem"},"2xl":{value:"1.5rem"},"3xl":{value:"1.875rem"},"4xl":{value:"2.25rem"},"5xl":{value:"3rem"},"6xl":{value:"3.75rem"},"7xl":{value:"4.5rem"},"8xl":{value:"6rem"},"9xl":{value:"8rem"}}),yE=de.fontWeights({thin:{value:"100"},extralight:{value:"200"},light:{value:"300"},normal:{value:"400"},medium:{value:"500"},semibold:{value:"600"},bold:{value:"700"},extrabold:{value:"800"},black:{value:"900"}}),Jh='-apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol"',xE=de.fonts({heading:{value:`Inter, ${Jh}`},body:{value:`Inter, ${Jh}`},mono:{value:'SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace'}}),kE=Bk({spin:{"0%":{transform:"rotate(0deg)"},"100%":{transform:"rotate(360deg)"}},pulse:{"50%":{opacity:"0.5"}},ping:{"75%, 100%":{transform:"scale(2)",opacity:"0"}},bounce:{"0%, 100%":{transform:"translateY(-25%)",animationTimingFunction:"cubic-bezier(0.8,0,1,1)"},"50%":{transform:"none",animationTimingFunction:"cubic-bezier(0,0,0.2,1)"}},"bg-position":{from:{backgroundPosition:"var(--animate-from, 1rem) 0"},to:{backgroundPosition:"var(--animate-to, 0) 0"}},position:{from:{insetInlineStart:"var(--animate-from-x)",insetBlockStart:"var(--animate-from-y)"},to:{insetInlineStart:"var(--animate-to-x)",insetBlockStart:"var(--animate-to-y)"}},"circular-progress":{"0%":{strokeDasharray:"1, 400",strokeDashoffset:"0"},"50%":{strokeDasharray:"400, 400",strokeDashoffset:"-100%"},"100%":{strokeDasharray:"400, 400",strokeDashoffset:"-260%"}},"expand-height":{from:{height:"0"},to:{height:"var(--height)"}},"collapse-height":{from:{height:"var(--height)"},to:{height:"0"}},"expand-width":{from:{width:"0"},to:{width:"var(--width)"}},"collapse-width":{from:{height:"var(--width)"},to:{height:"0"}},"fade-in":{from:{opacity:0},to:{opacity:1}},"fade-out":{from:{opacity:1},to:{opacity:0}},"slide-from-left-full":{from:{translate:"-100% 0"},to:{translate:"0 0"}},"slide-from-right-full":{from:{translate:"100% 0"},to:{translate:"0 0"}},"slide-from-top-full":{from:{translate:"0 -100%"},to:{translate:"0 0"}},"slide-from-bottom-full":{from:{translate:"0 100%"},to:{translate:"0 0"}},"slide-to-left-full":{from:{translate:"0 0"},to:{translate:"-100% 0"}},"slide-to-right-full":{from:{translate:"0 0"},to:{translate:"100% 0"}},"slide-to-top-full":{from:{translate:"0 0"},to:{translate:"0 -100%"}},"slide-to-bottom-full":{from:{translate:"0 0"},to:{translate:"0 100%"}},"slide-from-top":{"0%":{translate:"0 -0.5rem"},to:{translate:"0"}},"slide-from-bottom":{"0%":{translate:"0 0.5rem"},to:{translate:"0"}},"slide-from-left":{"0%":{translate:"-0.5rem 0"},to:{translate:"0"}},"slide-from-right":{"0%":{translate:"0.5rem 0"},to:{translate:"0"}},"slide-to-top":{"0%":{translate:"0"},to:{translate:"0 -0.5rem"}},"slide-to-bottom":{"0%":{translate:"0"},to:{translate:"0 0.5rem"}},"slide-to-left":{"0%":{translate:"0"},to:{translate:"-0.5rem 0"}},"slide-to-right":{"0%":{translate:"0"},to:{translate:"0.5rem 0"}},"scale-in":{from:{scale:"0.95"},to:{scale:"1"}},"scale-out":{from:{scale:"1"},to:{scale:"0.95"}}}),CE=de.letterSpacings({tighter:{value:"-0.05em"},tight:{value:"-0.025em"},wide:{value:"0.025em"},wider:{value:"0.05em"},widest:{value:"0.1em"}}),wE=de.lineHeights({shorter:{value:1.25},short:{value:1.375},moderate:{value:1.5},tall:{value:1.625},taller:{value:2}}),SE=de.radii({none:{value:"0"},"2xs":{value:"0.0625rem"},xs:{value:"0.125rem"},sm:{value:"0.25rem"},md:{value:"0.375rem"},lg:{value:"0.5rem"},xl:{value:"0.75rem"},"2xl":{value:"1rem"},"3xl":{value:"1.5rem"},"4xl":{value:"2rem"},full:{value:"9999px"}}),Zh=de.spacing({.5:{value:"0.125rem"},1:{value:"0.25rem"},1.5:{value:"0.375rem"},2:{value:"0.5rem"},2.5:{value:"0.625rem"},3:{value:"0.75rem"},3.5:{value:"0.875rem"},4:{value:"1rem"},4.5:{value:"1.125rem"},5:{value:"1.25rem"},6:{value:"1.5rem"},7:{value:"1.75rem"},8:{value:"2rem"},9:{value:"2.25rem"},10:{value:"2.5rem"},11:{value:"2.75rem"},12:{value:"3rem"},14:{value:"3.5rem"},16:{value:"4rem"},20:{value:"5rem"},24:{value:"6rem"},28:{value:"7rem"},32:{value:"8rem"},36:{value:"9rem"},40:{value:"10rem"},44:{value:"11rem"},48:{value:"12rem"},52:{value:"13rem"},56:{value:"14rem"},60:{value:"15rem"},64:{value:"16rem"},72:{value:"18rem"},80:{value:"20rem"},96:{value:"24rem"}}),EE=de.sizes({"3xs":{value:"14rem"},"2xs":{value:"16rem"},xs:{value:"20rem"},sm:{value:"24rem"},md:{value:"28rem"},lg:{value:"32rem"},xl:{value:"36rem"},"2xl":{value:"42rem"},"3xl":{value:"48rem"},"4xl":{value:"56rem"},"5xl":{value:"64rem"},"6xl":{value:"72rem"},"7xl":{value:"80rem"},"8xl":{value:"90rem"}}),OE=de.sizes({max:{value:"max-content"},min:{value:"min-content"},fit:{value:"fit-content"},prose:{value:"60ch"},full:{value:"100%"},dvh:{value:"100dvh"},svh:{value:"100svh"},lvh:{value:"100lvh"},dvw:{value:"100dvw"},svw:{value:"100svw"},lvw:{value:"100lvw"},vw:{value:"100vw"},vh:{value:"100vh"}}),PE=de.sizes({"1/2":{value:"50%"},"1/3":{value:"33.333333%"},"2/3":{value:"66.666667%"},"1/4":{value:"25%"},"3/4":{value:"75%"},"1/5":{value:"20%"},"2/5":{value:"40%"},"3/5":{value:"60%"},"4/5":{value:"80%"},"1/6":{value:"16.666667%"},"2/6":{value:"33.333333%"},"3/6":{value:"50%"},"4/6":{value:"66.666667%"},"5/6":{value:"83.333333%"},"1/12":{value:"8.333333%"},"2/12":{value:"16.666667%"},"3/12":{value:"25%"},"4/12":{value:"33.333333%"},"5/12":{value:"41.666667%"},"6/12":{value:"50%"},"7/12":{value:"58.333333%"},"8/12":{value:"66.666667%"},"9/12":{value:"75%"},"10/12":{value:"83.333333%"},"11/12":{value:"91.666667%"}}),RE=de.sizes({...EE,...Zh,...PE,...OE}),IE=de.zIndex({hide:{value:-1},base:{value:0},docked:{value:10},dropdown:{value:1e3},sticky:{value:1100},banner:{value:1200},overlay:{value:1300},modal:{value:1400},popover:{value:1500},skipNav:{value:1600},toast:{value:1700},tooltip:{value:1800},max:{value:2147483647}}),TE=ka({preflight:!0,cssVarsPrefix:"chakra",cssVarsRoot:":where(html, .chakra-theme)",globalCss:aw,theme:{breakpoints:sw,keyframes:kE,tokens:{aspectRatios:dE,animations:uE,blurs:hE,borders:fE,colors:gE,durations:mE,easings:vE,fonts:xE,fontSizes:bE,fontWeights:yE,letterSpacings:CE,lineHeights:wE,radii:SE,spacing:Zh,sizes:RE,zIndex:IE,cursor:pE},semanticTokens:{colors:Pw,shadows:Iw,radii:Rw},recipes:Ow,slotRecipes:lE,textStyles:cE,layerStyles:lw,animationStyles:cw}}),ef=eh(eC,TE);Mh(ef);function NE(e){const{key:t,recipe:n}=e,r=ho();return P.useMemo(()=>{const o=n||(t!=null?r.getSlotRecipe(t):{});return r.sva(structuredClone(o))},[t,n,r])}const AE=e=>e.charAt(0).toUpperCase()+e.slice(1),fi=e=>{const{key:t,recipe:n}=e,r=AE(t||n.className||"Component"),[o,i]=Ln({name:`${r}StylesContext`,errorMessage:`use${r}Styles returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `}),[s,a]=Ln({name:`${r}ClassNameContext`,errorMessage:`use${r}ClassNames returned is 'undefined'. Seems you forgot to wrap the components in "<${r}.Root />" `,strict:!1}),[l,c]=Ln({strict:!1,name:`${r}PropsContext`,providerName:`${r}PropsContext`,defaultValue:{}});function u(f){const{unstyled:p,...v}=f,b=NE({key:t,recipe:v.recipe||n}),[y,x]=P.useMemo(()=>b.splitVariantProps(v),[v,b]);return{styles:P.useMemo(()=>p?Bm:b(y),[p,y,b]),classNames:b.classNameMap,props:x}}function h(f,p={}){const{defaultProps:v}=p,b=y=>{const x=c(),w=P.useMemo(()=>br(v,x,y),[x,y]),{styles:k,classNames:O,props:R}=u(w);return g.jsx(o,{value:k,children:g.jsx(s,{value:O,children:g.jsx(f,{...R})})})};return b.displayName=f.displayName||f.name,b}return{StylesProvider:o,ClassNamesProvider:s,PropsProvider:l,usePropsContext:c,useRecipeResult:u,withProvider:(f,p,v)=>{const{defaultProps:b,...y}=v??{},x=ve(f,{},y),w=P.forwardRef((k,O)=>{const R=c(),I=P.useMemo(()=>br(b??{},R,k),[R,k]),{styles:N,props:A,classNames:T}=u(I),S=T[p],_=g.jsx(o,{value:N,children:g.jsx(s,{value:T,children:g.jsx(x,{ref:O,...A,css:[N[p],I.css],className:Ge(I.className,S)})})});return v?.wrapElement?.(_,I)??_});return w.displayName=f.displayName||f.name,w},withContext:(f,p,v)=>{const b=ve(f,{},v),y=P.forwardRef((x,w)=>{const{unstyled:k,...O}=x,R=i(),N=a()?.[p];return g.jsx(b,{...O,css:[!k&&p?R[p]:void 0,x.css],ref:w,className:Ge(x.className,N)})});return y.displayName=f.displayName||f.name,y},withRootProvider:h,useStyles:i,useClassNames:a}},tf=ve("div",{base:{position:"absolute",display:"flex",alignItems:"center",justifyContent:"center"},variants:{axis:{horizontal:{insetStart:"50%",translate:"-50%",_rtl:{translate:"50%"}},vertical:{top:"50%",translate:"0 -50%"},both:{insetStart:"50%",top:"50%",translate:"-50% -50%",_rtl:{translate:"50% -50%"}}}},defaultVariants:{axis:"both"}});tf.displayName="AbsoluteCenter";const _E=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11.0026 16L6.75999 11.7574L8.17421 10.3431L11.0026 13.1716L16.6595 7.51472L18.0737 8.92893L11.0026 16Z"})}),nf=e=>g.jsx(ve.svg,{stroke:"currentColor",fill:"currentColor",strokeWidth:"0",viewBox:"0 0 24 24",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 15H13V17H11V15ZM11 7H13V13H11V7Z"})}),rf=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",stroke:"currentColor",strokeWidth:"0",...e,children:g.jsx("path",{d:"M12 22C6.47715 22 2 17.5228 2 12C2 6.47715 6.47715 2 12 2C17.5228 2 22 6.47715 22 12C22 17.5228 17.5228 22 12 22ZM12 20C16.4183 20 20 16.4183 20 12C20 7.58172 16.4183 4 12 4C7.58172 4 4 7.58172 4 12C4 16.4183 7.58172 20 12 20ZM11 7H13V9H11V7ZM11 11H13V17H11V11Z"})}),VE=e=>g.jsx(ve.svg,{viewBox:"0 0 24 24",fill:"currentColor",...e,children:g.jsx("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M18.7071 6.70711C19.0976 6.31658 19.0976 5.68342 18.7071 5.29289C18.3166 4.90237 17.6834 4.90237 17.2929 5.29289L12 10.5858L6.70711 5.29289C6.31658 4.90237 5.68342 4.90237 5.29289 5.29289C4.90237 5.68342 4.90237 6.31658 5.29289 6.70711L10.5858 12L5.29289 17.2929C4.90237 17.6834 4.90237 18.3166 5.29289 18.7071C5.68342 19.0976 6.31658 19.0976 6.70711 18.7071L12 13.4142L17.2929 18.7071C17.6834 19.0976 18.3166 19.0976 18.7071 18.7071C19.0976 18.3166 19.0976 17.6834 18.7071 17.2929L13.4142 12L18.7071 6.70711Z"})}),[LE,FE]=Ln({name:"AlertStatusContext",hookName:"useAlertStatusContext",providerName:""}),{withProvider:DE,withContext:Na,useStyles:zE}=fi({key:"alert"}),ME=DE("div","root",{forwardAsChild:!0,wrapElement(e,t){return g.jsx(LE,{value:{status:t.status||"info"},children:e})}}),of=Na("div","title"),jE=Na("div","description"),$E=Na("div","content"),BE={info:rf,warning:nf,success:_E,error:nf,neutral:rf},WE=P.forwardRef(function(t,n){const r=FE(),o=zE(),i=typeof r.status=="string"?BE[r.status]:P.Fragment,{children:s=g.jsx(i,{}),...a}=t;return g.jsx(ve.span,{ref:n,...a,css:[o.indicator,t.css],children:s})}),HE=e=>e?"":void 0,{withContext:UE}=Ft({key:"badge"}),sf=UE("span"),{withContext:GE}=Ft({key:"spinner"}),qE=GE("span"),KE=C.forwardRef(function(t,n){const{spinner:r=g.jsx(qE,{size:"inherit",borderWidth:"0.125em",color:"inherit"}),spinnerPlacement:o="start",children:i,text:s,visible:a=!0,...l}=t;return a?s?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[o==="start"&&r,s,o==="end"&&r]}):r?g.jsxs(Vo,{ref:n,display:"contents",...l,children:[g.jsx(tf,{display:"inline-flex",children:r}),g.jsx(Vo,{visibility:"hidden",display:"contents",children:i})]}):g.jsx(Vo,{ref:n,display:"contents",...l,children:i}):i}),{useRecipeResult:YE,usePropsContext:XE}=Ft({key:"button"}),Re=P.forwardRef(function(t,n){const r=XE(),o=P.useMemo(()=>br(r,t),[r,t]),i=YE(o),{loading:s,loadingText:a,children:l,spinner:c,spinnerPlacement:u,...h}=i.props;return g.jsx(ve.button,{type:"button",ref:n,...h,"data-loading":HE(s),disabled:s||h.disabled,className:Ge(i.className,o.className),css:[i.styles,o.css],children:!o.asChild&&s?g.jsx(KE,{spinner:c,text:a,spinnerPlacement:u,children:l}):l})}),Ot=P.forwardRef(function(t,n){return g.jsx(Re,{px:"0",py:"0",_icon:{fontSize:"1.2em"},ref:n,...t})}),fn=C.forwardRef(function(t,n){return g.jsx(Ot,{variant:"ghost","aria-label":"Close",ref:n,...t,children:t.children??g.jsx(VE,{})})}),af=ve("div",{base:{display:"flex",alignItems:"center",justifyContent:"center"},variants:{inline:{true:{display:"inline-flex"}}}});af.displayName="Center";const{withContext:QE}=Ft({key:"code"}),JE=QE("code");function ZE(e){const{gap:t,direction:n}=e,r={column:{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},"column-reverse":{marginY:t,marginX:0,borderInlineStartWidth:0,borderTopWidth:"1px"},row:{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0},"row-reverse":{marginX:t,marginY:0,borderInlineStartWidth:"1px",borderTopWidth:0}};return{"&":Zd(n,o=>r[o])}}function e2(e){return P.Children.toArray(e).filter(t=>P.isValidElement(t))}const lf=P.forwardRef(function(t,n){const{direction:r="column",align:o,justify:i,gap:s="0.5rem",wrap:a,children:l,separator:c,className:u,...h}=t,d=P.useMemo(()=>ZE({gap:s,direction:r}),[s,r]),m=P.useMemo(()=>P.isValidElement(c)?e2(l).map((f,p,v)=>{const b=typeof f.key<"u"?f.key:p,y=c,x=P.cloneElement(y,{css:[d,y.props.css]});return g.jsxs(P.Fragment,{children:[f,p===v.length-1?null:x]},b)}):l,[l,c,d]);return g.jsx(ve.div,{ref:n,display:"flex",alignItems:o,justifyContent:i,flexDirection:r,flexWrap:a,gap:c?void 0:s,className:Ge("chakra-stack",u),...h,children:m})}),{withRootProvider:cf,withContext:pt}=fi({key:"dialog"});cf(P1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});const gn=cf(O1,{defaultProps:{unmountOnExit:!0,lazyMount:!0}});pt(ad,"trigger",{forwardAsChild:!0});const pn=pt(td,"positioner",{forwardAsChild:!0}),mn=pt(Zu,"content",{forwardAsChild:!0});pt(ed,"description",{forwardAsChild:!0});const vn=pt(sd,"title",{forwardAsChild:!0}),bn=pt(Ju,"closeTrigger",{forwardAsChild:!0}),yn=P.forwardRef(function(t,n){const r=Ht();return g.jsx(ve.button,{...t,ref:n,onClick:()=>r.setOpen(!1)})}),xn=pt(Qu,"backdrop",{forwardAsChild:!0}),kn=pt("div","body"),Cn=pt("div","footer"),wn=pt("div","header");function t2(e){const{each:t,fallback:n,children:r}=e;return t?.length===0?n||null:t?.map(r)}const gi=P.forwardRef(function(t,n){const{direction:r,align:o,justify:i,wrap:s,basis:a,grow:l,shrink:c,inline:u,...h}=t;return g.jsx(ve.div,{ref:n,...h,css:{display:u?"inline-flex":"flex",flexDirection:r,alignItems:o,justifyContent:i,flexWrap:s,flexBasis:a,flexGrow:l,flexShrink:c,...t.css}})}),{withContext:n2}=Ft({key:"input"}),r2=n2(cd),{withContext:o2}=Ft({key:"link"}),Br=o2("a"),{withProvider:i2,withContext:uf}=fi({key:"list"}),df=i2("ul","root",{defaultProps:{role:"list"}}),hf=uf("li","item");uf("span","indicator");const s2=P.forwardRef(function(t,n){return g.jsx(lf,{align:"center",...t,direction:"row",ref:n})}),pi=P.forwardRef(function(t,n){return g.jsx(lf,{align:"center",...t,direction:"column",ref:n})}),{StylesProvider:a2,ClassNamesProvider:l2,useRecipeResult:c2,withContext:Pt}=fi({key:"table"}),ff=P.forwardRef(function({native:t,...n},r){const{styles:o,props:i,classNames:s}=c2(n),a=P.useMemo(()=>t?{...o.root,"& thead":o.header,"& tbody":o.body,"& tfoot":o.footer,"& thead th":o.columnHeader,"& tr":o.row,"& td":o.cell,"& caption":o.caption}:o.root,[o,t]);return g.jsx(l2,{value:s,children:g.jsx(a2,{value:o,children:g.jsx(ve.table,{ref:r,...i,css:[a,n.css],className:Ge(s?.root,n.className)})})})}),mi=Pt("tr","row");ve("div",{base:{display:"block",whiteSpace:"nowrap",WebkitOverflowScrolling:"touch",overflow:"auto",maxWidth:"100%"}});const gf=Pt("thead","header");Pt("tfoot","footer");const ke=Pt("th","columnHeader"),Ce=Pt("td","cell");Pt("caption","caption",{defaultProps:{captionSide:"bottom"}});const pf=Pt("tbody","body");Pt("colgroup"),Pt("col");const{withContext:u2}=Ft({key:"textarea"}),mf=u2(dd);var ir=class{constructor(){this.listeners=new Set,this.subscribe=this.subscribe.bind(this)}subscribe(e){return this.listeners.add(e),this.onSubscribe(),()=>{this.listeners.delete(e),this.onUnsubscribe()}}hasListeners(){return this.listeners.size>0}onSubscribe(){}onUnsubscribe(){}},d2={setTimeout:(e,t)=>setTimeout(e,t),clearTimeout:e=>clearTimeout(e),setInterval:(e,t)=>setInterval(e,t),clearInterval:e=>clearInterval(e)},h2=class{#e=d2;#t=!1;setTimeoutProvider(e){this.#e=e}setTimeout(e,t){return this.#e.setTimeout(e,t)}clearTimeout(e){this.#e.clearTimeout(e)}setInterval(e,t){return this.#e.setInterval(e,t)}clearInterval(e){this.#e.clearInterval(e)}},Sn=new h2;function f2(e){setTimeout(e,0)}var En=typeof window>"u"||"Deno"in globalThis;function Ne(){}function g2(e,t){return typeof e=="function"?e(t):e}function Aa(e){return typeof e=="number"&&e>=0&&e!==1/0}function vf(e,t){return Math.max(e+(t||0)-Date.now(),0)}function qt(e,t){return typeof e=="function"?e(t):e}function Qe(e,t){return typeof e=="function"?e(t):e}function bf(e,t){const{type:n="all",exact:r,fetchStatus:o,predicate:i,queryKey:s,stale:a}=e;if(s){if(r){if(t.queryHash!==_a(s,t.options))return!1}else if(!Wr(t.queryKey,s))return!1}if(n!=="all"){const l=t.isActive();if(n==="active"&&!l||n==="inactive"&&l)return!1}return!(typeof a=="boolean"&&t.isStale()!==a||o&&o!==t.state.fetchStatus||i&&!i(t))}function yf(e,t){const{exact:n,status:r,predicate:o,mutationKey:i}=e;if(i){if(!t.options.mutationKey)return!1;if(n){if(On(t.options.mutationKey)!==On(i))return!1}else if(!Wr(t.options.mutationKey,i))return!1}return!(r&&t.state.status!==r||o&&!o(t))}function _a(e,t){return(t?.queryKeyHashFn||On)(e)}function On(e){return JSON.stringify(e,(t,n)=>Va(n)?Object.keys(n).sort().reduce((r,o)=>(r[o]=n[o],r),{}):n)}function Wr(e,t){return e===t?!0:typeof e!=typeof t?!1:e&&t&&typeof e=="object"&&typeof t=="object"?Object.keys(t).every(n=>Wr(e[n],t[n])):!1}var p2=Object.prototype.hasOwnProperty;function xf(e,t){if(e===t)return e;const n=kf(e)&&kf(t);if(!n&&!(Va(e)&&Va(t)))return t;const o=(n?e:Object.keys(e)).length,i=n?t:Object.keys(t),s=i.length,a=n?new Array(s):{};let l=0;for(let c=0;c{Sn.setTimeout(t,e)})}function La(e,t,n){return typeof n.structuralSharing=="function"?n.structuralSharing(e,t):n.structuralSharing!==!1?xf(e,t):t}function v2(e,t,n=0){const r=[...e,t];return n&&r.length>n?r.slice(1):r}function b2(e,t,n=0){const r=[t,...e];return n&&r.length>n?r.slice(0,-1):r}var Fa=Symbol();function wf(e,t){return!e.queryFn&&t?.initialPromise?()=>t.initialPromise:!e.queryFn||e.queryFn===Fa?()=>Promise.reject(new Error(`Missing queryFn: '${e.queryHash}'`)):e.queryFn}function Sf(e,t){return typeof e=="function"?e(...t):!!e}var y2=class extends ir{#e;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e();return window.addEventListener("visibilitychange",t,!1),()=>{window.removeEventListener("visibilitychange",t)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(t=>{typeof t=="boolean"?this.setFocused(t):this.onFocus()})}setFocused(e){this.#e!==e&&(this.#e=e,this.onFocus())}onFocus(){const e=this.isFocused();this.listeners.forEach(t=>{t(e)})}isFocused(){return typeof this.#e=="boolean"?this.#e:globalThis.document?.visibilityState!=="hidden"}},Da=new y2;function za(){let e,t;const n=new Promise((o,i)=>{e=o,t=i});n.status="pending",n.catch(()=>{});function r(o){Object.assign(n,o),delete n.resolve,delete n.reject}return n.resolve=o=>{r({status:"fulfilled",value:o}),e(o)},n.reject=o=>{r({status:"rejected",reason:o}),t(o)},n}var x2=f2;function k2(){let e=[],t=0,n=a=>{a()},r=a=>{a()},o=x2;const i=a=>{t?e.push(a):o(()=>{n(a)})},s=()=>{const a=e;e=[],a.length&&o(()=>{r(()=>{a.forEach(l=>{n(l)})})})};return{batch:a=>{let l;t++;try{l=a()}finally{t--,t||s()}return l},batchCalls:a=>(...l)=>{i(()=>{a(...l)})},schedule:i,setNotifyFunction:a=>{n=a},setBatchNotifyFunction:a=>{r=a},setScheduler:a=>{o=a}}}var ge=k2(),C2=class extends ir{#e=!0;#t;#n;constructor(){super(),this.#n=e=>{if(!En&&window.addEventListener){const t=()=>e(!0),n=()=>e(!1);return window.addEventListener("online",t,!1),window.addEventListener("offline",n,!1),()=>{window.removeEventListener("online",t),window.removeEventListener("offline",n)}}}}onSubscribe(){this.#t||this.setEventListener(this.#n)}onUnsubscribe(){this.hasListeners()||(this.#t?.(),this.#t=void 0)}setEventListener(e){this.#n=e,this.#t?.(),this.#t=e(this.setOnline.bind(this))}setOnline(e){this.#e!==e&&(this.#e=e,this.listeners.forEach(n=>{n(e)}))}isOnline(){return this.#e}},bi=new C2;function w2(e){return Math.min(1e3*2**e,3e4)}function Ef(e){return(e??"online")==="online"?bi.isOnline():!0}var Ma=class extends Error{constructor(e){super("CancelledError"),this.revert=e?.revert,this.silent=e?.silent}};function Of(e){let t=!1,n=0,r;const o=za(),i=()=>o.status!=="pending",s=p=>{if(!i()){const v=new Ma(p);d(v),e.onCancel?.(v)}},a=()=>{t=!0},l=()=>{t=!1},c=()=>Da.isFocused()&&(e.networkMode==="always"||bi.isOnline())&&e.canRun(),u=()=>Ef(e.networkMode)&&e.canRun(),h=p=>{i()||(r?.(),o.resolve(p))},d=p=>{i()||(r?.(),o.reject(p))},m=()=>new Promise(p=>{r=v=>{(i()||c())&&p(v)},e.onPause?.()}).then(()=>{r=void 0,i()||e.onContinue?.()}),f=()=>{if(i())return;let p;const v=n===0?e.initialPromise:void 0;try{p=v??e.fn()}catch(b){p=Promise.reject(b)}Promise.resolve(p).then(h).catch(b=>{if(i())return;const y=e.retry??(En?0:3),x=e.retryDelay??w2,w=typeof x=="function"?x(n,b):x,k=y===!0||typeof y=="number"&&nc()?void 0:m()).then(()=>{t?d(b):f()})})};return{promise:o,status:()=>o.status,cancel:s,continue:()=>(r?.(),o),cancelRetry:a,continueRetry:l,canStart:u,start:()=>(u()?f():m().then(f),o)}}var Pf=class{#e;destroy(){this.clearGcTimeout()}scheduleGc(){this.clearGcTimeout(),Aa(this.gcTime)&&(this.#e=Sn.setTimeout(()=>{this.optionalRemove()},this.gcTime))}updateGcTime(e){this.gcTime=Math.max(this.gcTime||0,e??(En?1/0:300*1e3))}clearGcTimeout(){this.#e&&(Sn.clearTimeout(this.#e),this.#e=void 0)}},S2=class extends Pf{#e;#t;#n;#o;#r;#s;#a;constructor(e){super(),this.#a=!1,this.#s=e.defaultOptions,this.setOptions(e.options),this.observers=[],this.#o=e.client,this.#n=this.#o.getQueryCache(),this.queryKey=e.queryKey,this.queryHash=e.queryHash,this.#e=If(this.options),this.state=e.state??this.#e,this.scheduleGc()}get meta(){return this.options.meta}get promise(){return this.#r?.promise}setOptions(e){if(this.options={...this.#s,...e},this.updateGcTime(this.options.gcTime),this.state&&this.state.data===void 0){const t=If(this.options);t.data!==void 0&&(this.setData(t.data,{updatedAt:t.dataUpdatedAt,manual:!0}),this.#e=t)}}optionalRemove(){!this.observers.length&&this.state.fetchStatus==="idle"&&this.#n.remove(this)}setData(e,t){const n=La(this.state.data,e,this.options);return this.#i({data:n,type:"success",dataUpdatedAt:t?.updatedAt,manual:t?.manual}),n}setState(e,t){this.#i({type:"setState",state:e,setStateOptions:t})}cancel(e){const t=this.#r?.promise;return this.#r?.cancel(e),t?t.then(Ne).catch(Ne):Promise.resolve()}destroy(){super.destroy(),this.cancel({silent:!0})}reset(){this.destroy(),this.setState(this.#e)}isActive(){return this.observers.some(e=>Qe(e.options.enabled,this)!==!1)}isDisabled(){return this.getObserversCount()>0?!this.isActive():this.options.queryFn===Fa||this.state.dataUpdateCount+this.state.errorUpdateCount===0}isStatic(){return this.getObserversCount()>0?this.observers.some(e=>qt(e.options.staleTime,this)==="static"):!1}isStale(){return this.getObserversCount()>0?this.observers.some(e=>e.getCurrentResult().isStale):this.state.data===void 0||this.state.isInvalidated}isStaleByTime(e=0){return this.state.data===void 0?!0:e==="static"?!1:this.state.isInvalidated?!0:!vf(this.state.dataUpdatedAt,e)}onFocus(){this.observers.find(t=>t.shouldFetchOnWindowFocus())?.refetch({cancelRefetch:!1}),this.#r?.continue()}onOnline(){this.observers.find(t=>t.shouldFetchOnReconnect())?.refetch({cancelRefetch:!1}),this.#r?.continue()}addObserver(e){this.observers.includes(e)||(this.observers.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",query:this,observer:e}))}removeObserver(e){this.observers.includes(e)&&(this.observers=this.observers.filter(t=>t!==e),this.observers.length||(this.#r&&(this.#a?this.#r.cancel({revert:!0}):this.#r.cancelRetry()),this.scheduleGc()),this.#n.notify({type:"observerRemoved",query:this,observer:e}))}getObserversCount(){return this.observers.length}invalidate(){this.state.isInvalidated||this.#i({type:"invalidate"})}async fetch(e,t){if(this.state.fetchStatus!=="idle"&&this.#r?.status()!=="rejected"){if(this.state.data!==void 0&&t?.cancelRefetch)this.cancel({silent:!0});else if(this.#r)return this.#r.continueRetry(),this.#r.promise}if(e&&this.setOptions(e),!this.options.queryFn){const a=this.observers.find(l=>l.options.queryFn);a&&this.setOptions(a.options)}const n=new AbortController,r=a=>{Object.defineProperty(a,"signal",{enumerable:!0,get:()=>(this.#a=!0,n.signal)})},o=()=>{const a=wf(this.options,t),c=(()=>{const u={client:this.#o,queryKey:this.queryKey,meta:this.meta};return r(u),u})();return this.#a=!1,this.options.persister?this.options.persister(a,c,this):a(c)},s=(()=>{const a={fetchOptions:t,options:this.options,queryKey:this.queryKey,client:this.#o,state:this.state,fetchFn:o};return r(a),a})();this.options.behavior?.onFetch(s,this),this.#t=this.state,(this.state.fetchStatus==="idle"||this.state.fetchMeta!==s.fetchOptions?.meta)&&this.#i({type:"fetch",meta:s.fetchOptions?.meta}),this.#r=Of({initialPromise:t?.initialPromise,fn:s.fetchFn,onCancel:a=>{a instanceof Ma&&a.revert&&this.setState({...this.#t,fetchStatus:"idle"}),n.abort()},onFail:(a,l)=>{this.#i({type:"failed",failureCount:a,error:l})},onPause:()=>{this.#i({type:"pause"})},onContinue:()=>{this.#i({type:"continue"})},retry:s.options.retry,retryDelay:s.options.retryDelay,networkMode:s.options.networkMode,canRun:()=>!0});try{const a=await this.#r.start();if(a===void 0)throw new Error(`${this.queryHash} data is undefined`);return this.setData(a),this.#n.config.onSuccess?.(a,this),this.#n.config.onSettled?.(a,this.state.error,this),a}catch(a){if(a instanceof Ma){if(a.silent)return this.#r.promise;if(a.revert){if(this.state.data===void 0)throw a;return this.state.data}}throw this.#i({type:"error",error:a}),this.#n.config.onError?.(a,this),this.#n.config.onSettled?.(this.state.data,a,this),a}finally{this.scheduleGc()}}#i(e){const t=n=>{switch(e.type){case"failed":return{...n,fetchFailureCount:e.failureCount,fetchFailureReason:e.error};case"pause":return{...n,fetchStatus:"paused"};case"continue":return{...n,fetchStatus:"fetching"};case"fetch":return{...n,...Rf(n.data,this.options),fetchMeta:e.meta??null};case"success":const r={...n,data:e.data,dataUpdateCount:n.dataUpdateCount+1,dataUpdatedAt:e.dataUpdatedAt??Date.now(),error:null,isInvalidated:!1,status:"success",...!e.manual&&{fetchStatus:"idle",fetchFailureCount:0,fetchFailureReason:null}};return this.#t=e.manual?r:void 0,r;case"error":const o=e.error;return{...n,error:o,errorUpdateCount:n.errorUpdateCount+1,errorUpdatedAt:Date.now(),fetchFailureCount:n.fetchFailureCount+1,fetchFailureReason:o,fetchStatus:"idle",status:"error"};case"invalidate":return{...n,isInvalidated:!0};case"setState":return{...n,...e.state}}};this.state=t(this.state),ge.batch(()=>{this.observers.forEach(n=>{n.onQueryUpdate()}),this.#n.notify({query:this,type:"updated",action:e})})}};function Rf(e,t){return{fetchFailureCount:0,fetchFailureReason:null,fetchStatus:Ef(t.networkMode)?"fetching":"paused",...e===void 0&&{error:null,status:"pending"}}}function If(e){const t=typeof e.initialData=="function"?e.initialData():e.initialData,n=t!==void 0,r=n?typeof e.initialDataUpdatedAt=="function"?e.initialDataUpdatedAt():e.initialDataUpdatedAt:0;return{data:t,dataUpdateCount:0,dataUpdatedAt:n?r??Date.now():0,error:null,errorUpdateCount:0,errorUpdatedAt:0,fetchFailureCount:0,fetchFailureReason:null,fetchMeta:null,isInvalidated:!1,status:n?"success":"pending",fetchStatus:"idle"}}var E2=class extends ir{constructor(e,t){super(),this.options=t,this.#e=e,this.#i=null,this.#a=za(),this.bindMethods(),this.setOptions(t)}#e;#t=void 0;#n=void 0;#o=void 0;#r;#s;#a;#i;#p;#h;#f;#c;#u;#l;#g=new Set;bindMethods(){this.refetch=this.refetch.bind(this)}onSubscribe(){this.listeners.size===1&&(this.#t.addObserver(this),Tf(this.#t,this.options)?this.#d():this.updateResult(),this.#y())}onUnsubscribe(){this.hasListeners()||this.destroy()}shouldFetchOnReconnect(){return ja(this.#t,this.options,this.options.refetchOnReconnect)}shouldFetchOnWindowFocus(){return ja(this.#t,this.options,this.options.refetchOnWindowFocus)}destroy(){this.listeners=new Set,this.#x(),this.#k(),this.#t.removeObserver(this)}setOptions(e){const t=this.options,n=this.#t;if(this.options=this.#e.defaultQueryOptions(e),this.options.enabled!==void 0&&typeof this.options.enabled!="boolean"&&typeof this.options.enabled!="function"&&typeof Qe(this.options.enabled,this.#t)!="boolean")throw new Error("Expected enabled to be a boolean or a callback that returns a boolean");this.#C(),this.#t.setOptions(this.options),t._defaulted&&!vi(this.options,t)&&this.#e.getQueryCache().notify({type:"observerOptionsUpdated",query:this.#t,observer:this});const r=this.hasListeners();r&&Nf(this.#t,n,this.options,t)&&this.#d(),this.updateResult(),r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||qt(this.options.staleTime,this.#t)!==qt(t.staleTime,this.#t))&&this.#m();const o=this.#v();r&&(this.#t!==n||Qe(this.options.enabled,this.#t)!==Qe(t.enabled,this.#t)||o!==this.#l)&&this.#b(o)}getOptimisticResult(e){const t=this.#e.getQueryCache().build(this.#e,e),n=this.createResult(t,e);return P2(this,n)&&(this.#o=n,this.#s=this.options,this.#r=this.#t.state),n}getCurrentResult(){return this.#o}trackResult(e,t){return new Proxy(e,{get:(n,r)=>(this.trackProp(r),t?.(r),r==="promise"&&!this.options.experimental_prefetchInRender&&this.#a.status==="pending"&&this.#a.reject(new Error("experimental_prefetchInRender feature flag is not enabled")),Reflect.get(n,r))})}trackProp(e){this.#g.add(e)}getCurrentQuery(){return this.#t}refetch({...e}={}){return this.fetch({...e})}fetchOptimistic(e){const t=this.#e.defaultQueryOptions(e),n=this.#e.getQueryCache().build(this.#e,t);return n.fetch().then(()=>this.createResult(n,t))}fetch(e){return this.#d({...e,cancelRefetch:e.cancelRefetch??!0}).then(()=>(this.updateResult(),this.#o))}#d(e){this.#C();let t=this.#t.fetch(this.options,e);return e?.throwOnError||(t=t.catch(Ne)),t}#m(){this.#x();const e=qt(this.options.staleTime,this.#t);if(En||this.#o.isStale||!Aa(e))return;const n=vf(this.#o.dataUpdatedAt,e)+1;this.#c=Sn.setTimeout(()=>{this.#o.isStale||this.updateResult()},n)}#v(){return(typeof this.options.refetchInterval=="function"?this.options.refetchInterval(this.#t):this.options.refetchInterval)??!1}#b(e){this.#k(),this.#l=e,!(En||Qe(this.options.enabled,this.#t)===!1||!Aa(this.#l)||this.#l===0)&&(this.#u=Sn.setInterval(()=>{(this.options.refetchIntervalInBackground||Da.isFocused())&&this.#d()},this.#l))}#y(){this.#m(),this.#b(this.#v())}#x(){this.#c&&(Sn.clearTimeout(this.#c),this.#c=void 0)}#k(){this.#u&&(Sn.clearInterval(this.#u),this.#u=void 0)}createResult(e,t){const n=this.#t,r=this.options,o=this.#o,i=this.#r,s=this.#s,l=e!==n?e.state:this.#n,{state:c}=e;let u={...c},h=!1,d;if(t._optimisticResults){const I=this.hasListeners(),N=!I&&Tf(e,t),A=I&&Nf(e,n,t,r);(N||A)&&(u={...u,...Rf(c.data,e.options)}),t._optimisticResults==="isRestoring"&&(u.fetchStatus="idle")}let{error:m,errorUpdatedAt:f,status:p}=u;d=u.data;let v=!1;if(t.placeholderData!==void 0&&d===void 0&&p==="pending"){let I;o?.isPlaceholderData&&t.placeholderData===s?.placeholderData?(I=o.data,v=!0):I=typeof t.placeholderData=="function"?t.placeholderData(this.#f?.state.data,this.#f):t.placeholderData,I!==void 0&&(p="success",d=La(o?.data,I,t),h=!0)}if(t.select&&d!==void 0&&!v)if(o&&d===i?.data&&t.select===this.#p)d=this.#h;else try{this.#p=t.select,d=t.select(d),d=La(o?.data,d,t),this.#h=d,this.#i=null}catch(I){this.#i=I}this.#i&&(m=this.#i,d=this.#h,f=Date.now(),p="error");const b=u.fetchStatus==="fetching",y=p==="pending",x=p==="error",w=y&&b,k=d!==void 0,R={status:p,fetchStatus:u.fetchStatus,isPending:y,isSuccess:p==="success",isError:x,isInitialLoading:w,isLoading:w,data:d,dataUpdatedAt:u.dataUpdatedAt,error:m,errorUpdatedAt:f,failureCount:u.fetchFailureCount,failureReason:u.fetchFailureReason,errorUpdateCount:u.errorUpdateCount,isFetched:u.dataUpdateCount>0||u.errorUpdateCount>0,isFetchedAfterMount:u.dataUpdateCount>l.dataUpdateCount||u.errorUpdateCount>l.errorUpdateCount,isFetching:b,isRefetching:b&&!y,isLoadingError:x&&!k,isPaused:u.fetchStatus==="paused",isPlaceholderData:h,isRefetchError:x&&k,isStale:$a(e,t),refetch:this.refetch,promise:this.#a,isEnabled:Qe(t.enabled,e)!==!1};if(this.options.experimental_prefetchInRender){const I=T=>{R.status==="error"?T.reject(R.error):R.data!==void 0&&T.resolve(R.data)},N=()=>{const T=this.#a=R.promise=za();I(T)},A=this.#a;switch(A.status){case"pending":e.queryHash===n.queryHash&&I(A);break;case"fulfilled":(R.status==="error"||R.data!==A.value)&&N();break;case"rejected":(R.status!=="error"||R.error!==A.reason)&&N();break}}return R}updateResult(){const e=this.#o,t=this.createResult(this.#t,this.options);if(this.#r=this.#t.state,this.#s=this.options,this.#r.data!==void 0&&(this.#f=this.#t),vi(t,e))return;this.#o=t;const n=()=>{if(!e)return!0;const{notifyOnChangeProps:r}=this.options,o=typeof r=="function"?r():r;if(o==="all"||!o&&!this.#g.size)return!0;const i=new Set(o??this.#g);return this.options.throwOnError&&i.add("error"),Object.keys(this.#o).some(s=>{const a=s;return this.#o[a]!==e[a]&&i.has(a)})};this.#w({listeners:n()})}#C(){const e=this.#e.getQueryCache().build(this.#e,this.options);if(e===this.#t)return;const t=this.#t;this.#t=e,this.#n=e.state,this.hasListeners()&&(t?.removeObserver(this),e.addObserver(this))}onQueryUpdate(){this.updateResult(),this.hasListeners()&&this.#y()}#w(e){ge.batch(()=>{e.listeners&&this.listeners.forEach(t=>{t(this.#o)}),this.#e.getQueryCache().notify({query:this.#t,type:"observerResultsUpdated"})})}};function O2(e,t){return Qe(t.enabled,e)!==!1&&e.state.data===void 0&&!(e.state.status==="error"&&t.retryOnMount===!1)}function Tf(e,t){return O2(e,t)||e.state.data!==void 0&&ja(e,t,t.refetchOnMount)}function ja(e,t,n){if(Qe(t.enabled,e)!==!1&&qt(t.staleTime,e)!=="static"){const r=typeof n=="function"?n(e):n;return r==="always"||r!==!1&&$a(e,t)}return!1}function Nf(e,t,n,r){return(e!==t||Qe(r.enabled,e)===!1)&&(!n.suspense||e.state.status!=="error")&&$a(e,n)}function $a(e,t){return Qe(t.enabled,e)!==!1&&e.isStaleByTime(qt(t.staleTime,e))}function P2(e,t){return!vi(e.getCurrentResult(),t)}function Af(e){return{onFetch:(t,n)=>{const r=t.options,o=t.fetchOptions?.meta?.fetchMore?.direction,i=t.state.data?.pages||[],s=t.state.data?.pageParams||[];let a={pages:[],pageParams:[]},l=0;const c=async()=>{let u=!1;const h=f=>{Object.defineProperty(f,"signal",{enumerable:!0,get:()=>(t.signal.aborted?u=!0:t.signal.addEventListener("abort",()=>{u=!0}),t.signal)})},d=wf(t.options,t.fetchOptions),m=async(f,p,v)=>{if(u)return Promise.reject();if(p==null&&f.pages.length)return Promise.resolve(f);const y=(()=>{const O={client:t.client,queryKey:t.queryKey,pageParam:p,direction:v?"backward":"forward",meta:t.options.meta};return h(O),O})(),x=await d(y),{maxPages:w}=t.options,k=v?b2:v2;return{pages:k(f.pages,x,w),pageParams:k(f.pageParams,p,w)}};if(o&&i.length){const f=o==="backward",p=f?R2:_f,v={pages:i,pageParams:s},b=p(r,v);a=await m(v,b,f)}else{const f=e??i.length;do{const p=l===0?s[0]??r.initialPageParam:_f(r,a);if(l>0&&p==null)break;a=await m(a,p),l++}while(lt.options.persister?.(c,{client:t.client,queryKey:t.queryKey,meta:t.options.meta,signal:t.signal},n):t.fetchFn=c}}}function _f(e,{pages:t,pageParams:n}){const r=t.length-1;return t.length>0?e.getNextPageParam(t[r],t,n[r],n):void 0}function R2(e,{pages:t,pageParams:n}){return t.length>0?e.getPreviousPageParam?.(t[0],t,n[0],n):void 0}var I2=class extends Pf{#e;#t;#n;#o;constructor(e){super(),this.#e=e.client,this.mutationId=e.mutationId,this.#n=e.mutationCache,this.#t=[],this.state=e.state||Vf(),this.setOptions(e.options),this.scheduleGc()}setOptions(e){this.options=e,this.updateGcTime(this.options.gcTime)}get meta(){return this.options.meta}addObserver(e){this.#t.includes(e)||(this.#t.push(e),this.clearGcTimeout(),this.#n.notify({type:"observerAdded",mutation:this,observer:e}))}removeObserver(e){this.#t=this.#t.filter(t=>t!==e),this.scheduleGc(),this.#n.notify({type:"observerRemoved",mutation:this,observer:e})}optionalRemove(){this.#t.length||(this.state.status==="pending"?this.scheduleGc():this.#n.remove(this))}continue(){return this.#o?.continue()??this.execute(this.state.variables)}async execute(e){const t=()=>{this.#r({type:"continue"})},n={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};this.#o=Of({fn:()=>this.options.mutationFn?this.options.mutationFn(e,n):Promise.reject(new Error("No mutationFn found")),onFail:(i,s)=>{this.#r({type:"failed",failureCount:i,error:s})},onPause:()=>{this.#r({type:"pause"})},onContinue:t,retry:this.options.retry??0,retryDelay:this.options.retryDelay,networkMode:this.options.networkMode,canRun:()=>this.#n.canRun(this)});const r=this.state.status==="pending",o=!this.#o.canStart();try{if(r)t();else{this.#r({type:"pending",variables:e,isPaused:o}),await this.#n.config.onMutate?.(e,this,n);const s=await this.options.onMutate?.(e,n);s!==this.state.context&&this.#r({type:"pending",context:s,variables:e,isPaused:o})}const i=await this.#o.start();return await this.#n.config.onSuccess?.(i,e,this.state.context,this,n),await this.options.onSuccess?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(i,null,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(i,null,e,this.state.context,n),this.#r({type:"success",data:i}),i}catch(i){try{throw await this.#n.config.onError?.(i,e,this.state.context,this,n),await this.options.onError?.(i,e,this.state.context,n),await this.#n.config.onSettled?.(void 0,i,this.state.variables,this.state.context,this,n),await this.options.onSettled?.(void 0,i,e,this.state.context,n),i}finally{this.#r({type:"error",error:i})}}finally{this.#n.runNext(this)}}#r(e){const t=n=>{switch(e.type){case"failed":return{...n,failureCount:e.failureCount,failureReason:e.error};case"pause":return{...n,isPaused:!0};case"continue":return{...n,isPaused:!1};case"pending":return{...n,context:e.context,data:void 0,failureCount:0,failureReason:null,error:null,isPaused:e.isPaused,status:"pending",variables:e.variables,submittedAt:Date.now()};case"success":return{...n,data:e.data,failureCount:0,failureReason:null,error:null,status:"success",isPaused:!1};case"error":return{...n,data:void 0,error:e.error,failureCount:n.failureCount+1,failureReason:e.error,isPaused:!1,status:"error"}}};this.state=t(this.state),ge.batch(()=>{this.#t.forEach(n=>{n.onMutationUpdate(e)}),this.#n.notify({mutation:this,type:"updated",action:e})})}};function Vf(){return{context:void 0,data:void 0,error:null,failureCount:0,failureReason:null,isPaused:!1,status:"idle",variables:void 0,submittedAt:0}}var T2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Set,this.#t=new Map,this.#n=0}#e;#t;#n;build(e,t,n){const r=new I2({client:e,mutationCache:this,mutationId:++this.#n,options:e.defaultMutationOptions(t),state:n});return this.add(r),r}add(e){this.#e.add(e);const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);n?n.push(e):this.#t.set(t,[e])}this.notify({type:"added",mutation:e})}remove(e){if(this.#e.delete(e)){const t=yi(e);if(typeof t=="string"){const n=this.#t.get(t);if(n)if(n.length>1){const r=n.indexOf(e);r!==-1&&n.splice(r,1)}else n[0]===e&&this.#t.delete(t)}}this.notify({type:"removed",mutation:e})}canRun(e){const t=yi(e);if(typeof t=="string"){const r=this.#t.get(t)?.find(o=>o.state.status==="pending");return!r||r===e}else return!0}runNext(e){const t=yi(e);return typeof t=="string"?this.#t.get(t)?.find(r=>r!==e&&r.state.isPaused)?.continue()??Promise.resolve():Promise.resolve()}clear(){ge.batch(()=>{this.#e.forEach(e=>{this.notify({type:"removed",mutation:e})}),this.#e.clear(),this.#t.clear()})}getAll(){return Array.from(this.#e)}find(e){const t={exact:!0,...e};return this.getAll().find(n=>yf(t,n))}findAll(e={}){return this.getAll().filter(t=>yf(e,t))}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}resumePausedMutations(){const e=this.getAll().filter(t=>t.state.isPaused);return ge.batch(()=>Promise.all(e.map(t=>t.continue().catch(Ne))))}};function yi(e){return e.options.scope?.id}var N2=class extends ir{#e;#t=void 0;#n;#o;constructor(e,t){super(),this.#e=e,this.setOptions(t),this.bindMethods(),this.#r()}bindMethods(){this.mutate=this.mutate.bind(this),this.reset=this.reset.bind(this)}setOptions(e){const t=this.options;this.options=this.#e.defaultMutationOptions(e),vi(this.options,t)||this.#e.getMutationCache().notify({type:"observerOptionsUpdated",mutation:this.#n,observer:this}),t?.mutationKey&&this.options.mutationKey&&On(t.mutationKey)!==On(this.options.mutationKey)?this.reset():this.#n?.state.status==="pending"&&this.#n.setOptions(this.options)}onUnsubscribe(){this.hasListeners()||this.#n?.removeObserver(this)}onMutationUpdate(e){this.#r(),this.#s(e)}getCurrentResult(){return this.#t}reset(){this.#n?.removeObserver(this),this.#n=void 0,this.#r(),this.#s()}mutate(e,t){return this.#o=t,this.#n?.removeObserver(this),this.#n=this.#e.getMutationCache().build(this.#e,this.options),this.#n.addObserver(this),this.#n.execute(e)}#r(){const e=this.#n?.state??Vf();this.#t={...e,isPending:e.status==="pending",isSuccess:e.status==="success",isError:e.status==="error",isIdle:e.status==="idle",mutate:this.mutate,reset:this.reset}}#s(e){ge.batch(()=>{if(this.#o&&this.hasListeners()){const t=this.#t.variables,n=this.#t.context,r={client:this.#e,meta:this.options.meta,mutationKey:this.options.mutationKey};e?.type==="success"?(this.#o.onSuccess?.(e.data,t,n,r),this.#o.onSettled?.(e.data,null,t,n,r)):e?.type==="error"&&(this.#o.onError?.(e.error,t,n,r),this.#o.onSettled?.(void 0,e.error,t,n,r))}this.listeners.forEach(t=>{t(this.#t)})})}},A2=class extends ir{constructor(e={}){super(),this.config=e,this.#e=new Map}#e;build(e,t,n){const r=t.queryKey,o=t.queryHash??_a(r,t);let i=this.get(o);return i||(i=new S2({client:e,queryKey:r,queryHash:o,options:e.defaultQueryOptions(t),state:n,defaultOptions:e.getQueryDefaults(r)}),this.add(i)),i}add(e){this.#e.has(e.queryHash)||(this.#e.set(e.queryHash,e),this.notify({type:"added",query:e}))}remove(e){const t=this.#e.get(e.queryHash);t&&(e.destroy(),t===e&&this.#e.delete(e.queryHash),this.notify({type:"removed",query:e}))}clear(){ge.batch(()=>{this.getAll().forEach(e=>{this.remove(e)})})}get(e){return this.#e.get(e)}getAll(){return[...this.#e.values()]}find(e){const t={exact:!0,...e};return this.getAll().find(n=>bf(t,n))}findAll(e={}){const t=this.getAll();return Object.keys(e).length>0?t.filter(n=>bf(e,n)):t}notify(e){ge.batch(()=>{this.listeners.forEach(t=>{t(e)})})}onFocus(){ge.batch(()=>{this.getAll().forEach(e=>{e.onFocus()})})}onOnline(){ge.batch(()=>{this.getAll().forEach(e=>{e.onOnline()})})}},_2=class{#e;#t;#n;#o;#r;#s;#a;#i;constructor(e={}){this.#e=e.queryCache||new A2,this.#t=e.mutationCache||new T2,this.#n=e.defaultOptions||{},this.#o=new Map,this.#r=new Map,this.#s=0}mount(){this.#s++,this.#s===1&&(this.#a=Da.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onFocus())}),this.#i=bi.subscribe(async e=>{e&&(await this.resumePausedMutations(),this.#e.onOnline())}))}unmount(){this.#s--,this.#s===0&&(this.#a?.(),this.#a=void 0,this.#i?.(),this.#i=void 0)}isFetching(e){return this.#e.findAll({...e,fetchStatus:"fetching"}).length}isMutating(e){return this.#t.findAll({...e,status:"pending"}).length}getQueryData(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state.data}ensureQueryData(e){const t=this.defaultQueryOptions(e),n=this.#e.build(this,t),r=n.state.data;return r===void 0?this.fetchQuery(e):(e.revalidateIfStale&&n.isStaleByTime(qt(t.staleTime,n))&&this.prefetchQuery(t),Promise.resolve(r))}getQueriesData(e){return this.#e.findAll(e).map(({queryKey:t,state:n})=>{const r=n.data;return[t,r]})}setQueryData(e,t,n){const r=this.defaultQueryOptions({queryKey:e}),i=this.#e.get(r.queryHash)?.state.data,s=g2(t,i);if(s!==void 0)return this.#e.build(this,r).setData(s,{...n,manual:!0})}setQueriesData(e,t,n){return ge.batch(()=>this.#e.findAll(e).map(({queryKey:r})=>[r,this.setQueryData(r,t,n)]))}getQueryState(e){const t=this.defaultQueryOptions({queryKey:e});return this.#e.get(t.queryHash)?.state}removeQueries(e){const t=this.#e;ge.batch(()=>{t.findAll(e).forEach(n=>{t.remove(n)})})}resetQueries(e,t){const n=this.#e;return ge.batch(()=>(n.findAll(e).forEach(r=>{r.reset()}),this.refetchQueries({type:"active",...e},t)))}cancelQueries(e,t={}){const n={revert:!0,...t},r=ge.batch(()=>this.#e.findAll(e).map(o=>o.cancel(n)));return Promise.all(r).then(Ne).catch(Ne)}invalidateQueries(e,t={}){return ge.batch(()=>(this.#e.findAll(e).forEach(n=>{n.invalidate()}),e?.refetchType==="none"?Promise.resolve():this.refetchQueries({...e,type:e?.refetchType??e?.type??"active"},t)))}refetchQueries(e,t={}){const n={...t,cancelRefetch:t.cancelRefetch??!0},r=ge.batch(()=>this.#e.findAll(e).filter(o=>!o.isDisabled()&&!o.isStatic()).map(o=>{let i=o.fetch(void 0,n);return n.throwOnError||(i=i.catch(Ne)),o.state.fetchStatus==="paused"?Promise.resolve():i}));return Promise.all(r).then(Ne)}fetchQuery(e){const t=this.defaultQueryOptions(e);t.retry===void 0&&(t.retry=!1);const n=this.#e.build(this,t);return n.isStaleByTime(qt(t.staleTime,n))?n.fetch(t):Promise.resolve(n.state.data)}prefetchQuery(e){return this.fetchQuery(e).then(Ne).catch(Ne)}fetchInfiniteQuery(e){return e.behavior=Af(e.pages),this.fetchQuery(e)}prefetchInfiniteQuery(e){return this.fetchInfiniteQuery(e).then(Ne).catch(Ne)}ensureInfiniteQueryData(e){return e.behavior=Af(e.pages),this.ensureQueryData(e)}resumePausedMutations(){return bi.isOnline()?this.#t.resumePausedMutations():Promise.resolve()}getQueryCache(){return this.#e}getMutationCache(){return this.#t}getDefaultOptions(){return this.#n}setDefaultOptions(e){this.#n=e}setQueryDefaults(e,t){this.#o.set(On(e),{queryKey:e,defaultOptions:t})}getQueryDefaults(e){const t=[...this.#o.values()],n={};return t.forEach(r=>{Wr(e,r.queryKey)&&Object.assign(n,r.defaultOptions)}),n}setMutationDefaults(e,t){this.#r.set(On(e),{mutationKey:e,defaultOptions:t})}getMutationDefaults(e){const t=[...this.#r.values()],n={};return t.forEach(r=>{Wr(e,r.mutationKey)&&Object.assign(n,r.defaultOptions)}),n}defaultQueryOptions(e){if(e._defaulted)return e;const t={...this.#n.queries,...this.getQueryDefaults(e.queryKey),...e,_defaulted:!0};return t.queryHash||(t.queryHash=_a(t.queryKey,t)),t.refetchOnReconnect===void 0&&(t.refetchOnReconnect=t.networkMode!=="always"),t.throwOnError===void 0&&(t.throwOnError=!!t.suspense),!t.networkMode&&t.persister&&(t.networkMode="offlineFirst"),t.queryFn===Fa&&(t.enabled=!1),t}defaultMutationOptions(e){return e?._defaulted?e:{...this.#n.mutations,...e?.mutationKey&&this.getMutationDefaults(e.mutationKey),...e,_defaulted:!0}}clear(){this.#e.clear(),this.#t.clear()}},Lf=C.createContext(void 0),Ff=e=>{const t=C.useContext(Lf);if(!t)throw new Error("No QueryClient set, use QueryClientProvider to set one");return t},V2=({client:e,children:t})=>(C.useEffect(()=>(e.mount(),()=>{e.unmount()}),[e]),g.jsx(Lf.Provider,{value:e,children:t})),Df=C.createContext(!1),L2=()=>C.useContext(Df);Df.Provider;function F2(){let e=!1;return{clearReset:()=>{e=!1},reset:()=>{e=!0},isReset:()=>e}}var D2=C.createContext(F2()),z2=()=>C.useContext(D2),M2=(e,t)=>{(e.suspense||e.throwOnError||e.experimental_prefetchInRender)&&(t.isReset()||(e.retryOnMount=!1))},j2=e=>{C.useEffect(()=>{e.clearReset()},[e])},$2=({result:e,errorResetBoundary:t,throwOnError:n,query:r,suspense:o})=>e.isError&&!t.isReset()&&!e.isFetching&&r&&(o&&e.data===void 0||Sf(n,[e.error,r])),B2=e=>{if(e.suspense){const n=o=>o==="static"?o:Math.max(o??1e3,1e3),r=e.staleTime;e.staleTime=typeof r=="function"?(...o)=>n(r(...o)):n(r),typeof e.gcTime=="number"&&(e.gcTime=Math.max(e.gcTime,1e3))}},W2=(e,t)=>e.isLoading&&e.isFetching&&!t,H2=(e,t)=>e?.suspense&&t.isPending,zf=(e,t,n)=>t.fetchOptimistic(e).catch(()=>{n.clearReset()});function U2(e,t,n){const r=L2(),o=z2(),i=Ff(),s=i.defaultQueryOptions(e);i.getDefaultOptions().queries?._experimental_beforeQuery?.(s),s._optimisticResults=r?"isRestoring":"optimistic",B2(s),M2(s,o),j2(o);const a=!i.getQueryCache().get(s.queryHash),[l]=C.useState(()=>new t(i,s)),c=l.getOptimisticResult(s),u=!r&&e.subscribed!==!1;if(C.useSyncExternalStore(C.useCallback(h=>{const d=u?l.subscribe(ge.batchCalls(h)):Ne;return l.updateResult(),d},[l,u]),()=>l.getCurrentResult(),()=>l.getCurrentResult()),C.useEffect(()=>{l.setOptions(s)},[s,l]),H2(s,c))throw zf(s,l,o);if($2({result:c,errorResetBoundary:o,throwOnError:s.throwOnError,query:i.getQueryCache().get(s.queryHash),suspense:s.suspense}))throw c.error;return i.getDefaultOptions().queries?._experimental_afterQuery?.(s,c),s.experimental_prefetchInRender&&!En&&W2(c,r)&&(a?zf(s,l,o):i.getQueryCache().get(s.queryHash)?.promise)?.catch(Ne).finally(()=>{l.updateResult()}),s.notifyOnChangeProps?c:l.trackResult(c)}function Mf(e,t){return U2(e,E2)}function Pn(e,t){const n=Ff(),[r]=C.useState(()=>new N2(n,e));C.useEffect(()=>{r.setOptions(e)},[r,e]);const o=C.useSyncExternalStore(C.useCallback(s=>r.subscribe(ge.batchCalls(s)),[r]),()=>r.getCurrentResult(),()=>r.getCurrentResult()),i=C.useCallback((s,a)=>{r.mutate(s,a).catch(Ne)},[r]);if(o.error&&Sf(r.options.throwOnError,[o.error]))throw o.error;return{...o,mutate:i,mutateAsync:o.mutate}}function jf(e,t){return function(){return e.apply(t,arguments)}}const{toString:G2}=Object.prototype,{getPrototypeOf:Ba}=Object,{iterator:xi,toStringTag:$f}=Symbol,ki=(e=>t=>{const n=G2.call(t);return e[n]||(e[n]=n.slice(8,-1).toLowerCase())})(Object.create(null)),ot=e=>(e=e.toLowerCase(),t=>ki(t)===e),Ci=e=>t=>typeof t===e,{isArray:sr}=Array,ar=Ci("undefined");function Hr(e){return e!==null&&!ar(e)&&e.constructor!==null&&!ar(e.constructor)&&ze(e.constructor.isBuffer)&&e.constructor.isBuffer(e)}const Bf=ot("ArrayBuffer");function q2(e){let t;return typeof ArrayBuffer<"u"&&ArrayBuffer.isView?t=ArrayBuffer.isView(e):t=e&&e.buffer&&Bf(e.buffer),t}const K2=Ci("string"),ze=Ci("function"),Wf=Ci("number"),Ur=e=>e!==null&&typeof e=="object",Y2=e=>e===!0||e===!1,wi=e=>{if(ki(e)!=="object")return!1;const t=Ba(e);return(t===null||t===Object.prototype||Object.getPrototypeOf(t)===null)&&!($f in e)&&!(xi in e)},X2=e=>{if(!Ur(e)||Hr(e))return!1;try{return Object.keys(e).length===0&&Object.getPrototypeOf(e)===Object.prototype}catch{return!1}},Q2=ot("Date"),J2=ot("File"),Z2=ot("Blob"),eO=ot("FileList"),tO=e=>Ur(e)&&ze(e.pipe),nO=e=>{let t;return e&&(typeof FormData=="function"&&e instanceof FormData||ze(e.append)&&((t=ki(e))==="formdata"||t==="object"&&ze(e.toString)&&e.toString()==="[object FormData]"))},rO=ot("URLSearchParams"),[oO,iO,sO,aO]=["ReadableStream","Request","Response","Headers"].map(ot),lO=e=>e.trim?e.trim():e.replace(/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,"");function Gr(e,t,{allOwnKeys:n=!1}={}){if(e===null||typeof e>"u")return;let r,o;if(typeof e!="object"&&(e=[e]),sr(e))for(r=0,o=e.length;r0;)if(o=n[r],t===o.toLowerCase())return o;return null}const Rn=typeof globalThis<"u"?globalThis:typeof self<"u"?self:typeof window<"u"?window:globalThis,Uf=e=>!ar(e)&&e!==Rn;function Wa(){const{caseless:e,skipUndefined:t}=Uf(this)&&this||{},n={},r=(o,i)=>{const s=e&&Hf(n,i)||i;wi(n[s])&&wi(o)?n[s]=Wa(n[s],o):wi(o)?n[s]=Wa({},o):sr(o)?n[s]=o.slice():(!t||!ar(o))&&(n[s]=o)};for(let o=0,i=arguments.length;o(Gr(t,(o,i)=>{n&&ze(o)?e[i]=jf(o,n):e[i]=o},{allOwnKeys:r}),e),uO=e=>(e.charCodeAt(0)===65279&&(e=e.slice(1)),e),dO=(e,t,n,r)=>{e.prototype=Object.create(t.prototype,r),e.prototype.constructor=e,Object.defineProperty(e,"super",{value:t.prototype}),n&&Object.assign(e.prototype,n)},hO=(e,t,n,r)=>{let o,i,s;const a={};if(t=t||{},e==null)return t;do{for(o=Object.getOwnPropertyNames(e),i=o.length;i-- >0;)s=o[i],(!r||r(s,e,t))&&!a[s]&&(t[s]=e[s],a[s]=!0);e=n!==!1&&Ba(e)}while(e&&(!n||n(e,t))&&e!==Object.prototype);return t},fO=(e,t,n)=>{e=String(e),(n===void 0||n>e.length)&&(n=e.length),n-=t.length;const r=e.indexOf(t,n);return r!==-1&&r===n},gO=e=>{if(!e)return null;if(sr(e))return e;let t=e.length;if(!Wf(t))return null;const n=new Array(t);for(;t-- >0;)n[t]=e[t];return n},pO=(e=>t=>e&&t instanceof e)(typeof Uint8Array<"u"&&Ba(Uint8Array)),mO=(e,t)=>{const r=(e&&e[xi]).call(e);let o;for(;(o=r.next())&&!o.done;){const i=o.value;t.call(e,i[0],i[1])}},vO=(e,t)=>{let n;const r=[];for(;(n=e.exec(t))!==null;)r.push(n);return r},bO=ot("HTMLFormElement"),yO=e=>e.toLowerCase().replace(/[-_\s]([a-z\d])(\w*)/g,function(n,r,o){return r.toUpperCase()+o}),Gf=(({hasOwnProperty:e})=>(t,n)=>e.call(t,n))(Object.prototype),xO=ot("RegExp"),qf=(e,t)=>{const n=Object.getOwnPropertyDescriptors(e),r={};Gr(n,(o,i)=>{let s;(s=t(o,i,e))!==!1&&(r[i]=s||o)}),Object.defineProperties(e,r)},kO=e=>{qf(e,(t,n)=>{if(ze(e)&&["arguments","caller","callee"].indexOf(n)!==-1)return!1;const r=e[n];if(ze(r)){if(t.enumerable=!1,"writable"in t){t.writable=!1;return}t.set||(t.set=()=>{throw Error("Can not rewrite read-only method '"+n+"'")})}})},CO=(e,t)=>{const n={},r=o=>{o.forEach(i=>{n[i]=!0})};return sr(e)?r(e):r(String(e).split(t)),n},wO=()=>{},SO=(e,t)=>e!=null&&Number.isFinite(e=+e)?e:t;function EO(e){return!!(e&&ze(e.append)&&e[$f]==="FormData"&&e[xi])}const OO=e=>{const t=new Array(10),n=(r,o)=>{if(Ur(r)){if(t.indexOf(r)>=0)return;if(Hr(r))return r;if(!("toJSON"in r)){t[o]=r;const i=sr(r)?[]:{};return Gr(r,(s,a)=>{const l=n(s,o+1);!ar(l)&&(i[a]=l)}),t[o]=void 0,i}}return r};return n(e,0)},PO=ot("AsyncFunction"),RO=e=>e&&(Ur(e)||ze(e))&&ze(e.then)&&ze(e.catch),Kf=((e,t)=>e?setImmediate:t?((n,r)=>(Rn.addEventListener("message",({source:o,data:i})=>{o===Rn&&i===n&&r.length&&r.shift()()},!1),o=>{r.push(o),Rn.postMessage(n,"*")}))(`axios@${Math.random()}`,[]):n=>setTimeout(n))(typeof setImmediate=="function",ze(Rn.postMessage)),IO=typeof queueMicrotask<"u"?queueMicrotask.bind(Rn):typeof process<"u"&&process.nextTick||Kf,E={isArray:sr,isArrayBuffer:Bf,isBuffer:Hr,isFormData:nO,isArrayBufferView:q2,isString:K2,isNumber:Wf,isBoolean:Y2,isObject:Ur,isPlainObject:wi,isEmptyObject:X2,isReadableStream:oO,isRequest:iO,isResponse:sO,isHeaders:aO,isUndefined:ar,isDate:Q2,isFile:J2,isBlob:Z2,isRegExp:xO,isFunction:ze,isStream:tO,isURLSearchParams:rO,isTypedArray:pO,isFileList:eO,forEach:Gr,merge:Wa,extend:cO,trim:lO,stripBOM:uO,inherits:dO,toFlatObject:hO,kindOf:ki,kindOfTest:ot,endsWith:fO,toArray:gO,forEachEntry:mO,matchAll:vO,isHTMLForm:bO,hasOwnProperty:Gf,hasOwnProp:Gf,reduceDescriptors:qf,freezeMethods:kO,toObjectSet:CO,toCamelCase:yO,noop:wO,toFiniteNumber:SO,findKey:Hf,global:Rn,isContextDefined:Uf,isSpecCompliantForm:EO,toJSONObject:OO,isAsyncFn:PO,isThenable:RO,setImmediate:Kf,asap:IO,isIterable:e=>e!=null&&ze(e[xi])};function H(e,t,n,r,o){Error.call(this),Error.captureStackTrace?Error.captureStackTrace(this,this.constructor):this.stack=new Error().stack,this.message=e,this.name="AxiosError",t&&(this.code=t),n&&(this.config=n),r&&(this.request=r),o&&(this.response=o,this.status=o.status?o.status:null)}E.inherits(H,Error,{toJSON:function(){return{message:this.message,name:this.name,description:this.description,number:this.number,fileName:this.fileName,lineNumber:this.lineNumber,columnNumber:this.columnNumber,stack:this.stack,config:E.toJSONObject(this.config),code:this.code,status:this.status}}});const Yf=H.prototype,Xf={};["ERR_BAD_OPTION_VALUE","ERR_BAD_OPTION","ECONNABORTED","ETIMEDOUT","ERR_NETWORK","ERR_FR_TOO_MANY_REDIRECTS","ERR_DEPRECATED","ERR_BAD_RESPONSE","ERR_BAD_REQUEST","ERR_CANCELED","ERR_NOT_SUPPORT","ERR_INVALID_URL"].forEach(e=>{Xf[e]={value:e}}),Object.defineProperties(H,Xf),Object.defineProperty(Yf,"isAxiosError",{value:!0}),H.from=(e,t,n,r,o,i)=>{const s=Object.create(Yf);E.toFlatObject(e,s,function(u){return u!==Error.prototype},c=>c!=="isAxiosError");const a=e&&e.message?e.message:"Error",l=t==null&&e?e.code:t;return H.call(s,a,l,n,r,o),e&&s.cause==null&&Object.defineProperty(s,"cause",{value:e,configurable:!0}),s.name=e&&e.name||"Error",i&&Object.assign(s,i),s};const TO=null;function Ha(e){return E.isPlainObject(e)||E.isArray(e)}function Qf(e){return E.endsWith(e,"[]")?e.slice(0,-2):e}function Jf(e,t,n){return e?e.concat(t).map(function(o,i){return o=Qf(o),!n&&i?"["+o+"]":o}).join(n?".":""):t}function NO(e){return E.isArray(e)&&!e.some(Ha)}const AO=E.toFlatObject(E,{},null,function(t){return/^is[A-Z]/.test(t)});function Si(e,t,n){if(!E.isObject(e))throw new TypeError("target must be an object");t=t||new FormData,n=E.toFlatObject(n,{metaTokens:!0,dots:!1,indexes:!1},!1,function(p,v){return!E.isUndefined(v[p])});const r=n.metaTokens,o=n.visitor||u,i=n.dots,s=n.indexes,l=(n.Blob||typeof Blob<"u"&&Blob)&&E.isSpecCompliantForm(t);if(!E.isFunction(o))throw new TypeError("visitor must be a function");function c(f){if(f===null)return"";if(E.isDate(f))return f.toISOString();if(E.isBoolean(f))return f.toString();if(!l&&E.isBlob(f))throw new H("Blob is not supported. Use a Buffer instead.");return E.isArrayBuffer(f)||E.isTypedArray(f)?l&&typeof Blob=="function"?new Blob([f]):Buffer.from(f):f}function u(f,p,v){let b=f;if(f&&!v&&typeof f=="object"){if(E.endsWith(p,"{}"))p=r?p:p.slice(0,-2),f=JSON.stringify(f);else if(E.isArray(f)&&NO(f)||(E.isFileList(f)||E.endsWith(p,"[]"))&&(b=E.toArray(f)))return p=Qf(p),b.forEach(function(x,w){!(E.isUndefined(x)||x===null)&&t.append(s===!0?Jf([p],w,i):s===null?p:p+"[]",c(x))}),!1}return Ha(f)?!0:(t.append(Jf(v,p,i),c(f)),!1)}const h=[],d=Object.assign(AO,{defaultVisitor:u,convertValue:c,isVisitable:Ha});function m(f,p){if(!E.isUndefined(f)){if(h.indexOf(f)!==-1)throw Error("Circular reference detected in "+p.join("."));h.push(f),E.forEach(f,function(b,y){(!(E.isUndefined(b)||b===null)&&o.call(t,b,E.isString(y)?y.trim():y,p,d))===!0&&m(b,p?p.concat(y):[y])}),h.pop()}}if(!E.isObject(e))throw new TypeError("data must be an object");return m(e),t}function Zf(e){const t={"!":"%21","'":"%27","(":"%28",")":"%29","~":"%7E","%20":"+","%00":"\0"};return encodeURIComponent(e).replace(/[!'()~]|%20|%00/g,function(r){return t[r]})}function Ua(e,t){this._pairs=[],e&&Si(e,this,t)}const eg=Ua.prototype;eg.append=function(t,n){this._pairs.push([t,n])},eg.toString=function(t){const n=t?function(r){return t.call(this,r,Zf)}:Zf;return this._pairs.map(function(o){return n(o[0])+"="+n(o[1])},"").join("&")};function _O(e){return encodeURIComponent(e).replace(/%3A/gi,":").replace(/%24/g,"$").replace(/%2C/gi,",").replace(/%20/g,"+")}function tg(e,t,n){if(!t)return e;const r=n&&n.encode||_O;E.isFunction(n)&&(n={serialize:n});const o=n&&n.serialize;let i;if(o?i=o(t,n):i=E.isURLSearchParams(t)?t.toString():new Ua(t,n).toString(r),i){const s=e.indexOf("#");s!==-1&&(e=e.slice(0,s)),e+=(e.indexOf("?")===-1?"?":"&")+i}return e}class ng{constructor(){this.handlers=[]}use(t,n,r){return this.handlers.push({fulfilled:t,rejected:n,synchronous:r?r.synchronous:!1,runWhen:r?r.runWhen:null}),this.handlers.length-1}eject(t){this.handlers[t]&&(this.handlers[t]=null)}clear(){this.handlers&&(this.handlers=[])}forEach(t){E.forEach(this.handlers,function(r){r!==null&&t(r)})}}const rg={silentJSONParsing:!0,forcedJSONParsing:!0,clarifyTimeoutError:!1},VO={isBrowser:!0,classes:{URLSearchParams:typeof URLSearchParams<"u"?URLSearchParams:Ua,FormData:typeof FormData<"u"?FormData:null,Blob:typeof Blob<"u"?Blob:null},protocols:["http","https","file","blob","url","data"]},Ga=typeof window<"u"&&typeof document<"u",qa=typeof navigator=="object"&&navigator||void 0,LO=Ga&&(!qa||["ReactNative","NativeScript","NS"].indexOf(qa.product)<0),FO=typeof WorkerGlobalScope<"u"&&self instanceof WorkerGlobalScope&&typeof self.importScripts=="function",DO=Ga&&window.location.href||"http://localhost",Ie={...Object.freeze(Object.defineProperty({__proto__:null,hasBrowserEnv:Ga,hasStandardBrowserEnv:LO,hasStandardBrowserWebWorkerEnv:FO,navigator:qa,origin:DO},Symbol.toStringTag,{value:"Module"})),...VO};function zO(e,t){return Si(e,new Ie.classes.URLSearchParams,{visitor:function(n,r,o,i){return Ie.isNode&&E.isBuffer(n)?(this.append(r,n.toString("base64")),!1):i.defaultVisitor.apply(this,arguments)},...t})}function MO(e){return E.matchAll(/\w+|\[(\w*)]/g,e).map(t=>t[0]==="[]"?"":t[1]||t[0])}function jO(e){const t={},n=Object.keys(e);let r;const o=n.length;let i;for(r=0;r=n.length;return s=!s&&E.isArray(o)?o.length:s,l?(E.hasOwnProp(o,s)?o[s]=[o[s],r]:o[s]=r,!a):((!o[s]||!E.isObject(o[s]))&&(o[s]=[]),t(n,r,o[s],i)&&E.isArray(o[s])&&(o[s]=jO(o[s])),!a)}if(E.isFormData(e)&&E.isFunction(e.entries)){const n={};return E.forEachEntry(e,(r,o)=>{t(MO(r),o,n,0)}),n}return null}function $O(e,t,n){if(E.isString(e))try{return(t||JSON.parse)(e),E.trim(e)}catch(r){if(r.name!=="SyntaxError")throw r}return(n||JSON.stringify)(e)}const qr={transitional:rg,adapter:["xhr","http","fetch"],transformRequest:[function(t,n){const r=n.getContentType()||"",o=r.indexOf("application/json")>-1,i=E.isObject(t);if(i&&E.isHTMLForm(t)&&(t=new FormData(t)),E.isFormData(t))return o?JSON.stringify(og(t)):t;if(E.isArrayBuffer(t)||E.isBuffer(t)||E.isStream(t)||E.isFile(t)||E.isBlob(t)||E.isReadableStream(t))return t;if(E.isArrayBufferView(t))return t.buffer;if(E.isURLSearchParams(t))return n.setContentType("application/x-www-form-urlencoded;charset=utf-8",!1),t.toString();let a;if(i){if(r.indexOf("application/x-www-form-urlencoded")>-1)return zO(t,this.formSerializer).toString();if((a=E.isFileList(t))||r.indexOf("multipart/form-data")>-1){const l=this.env&&this.env.FormData;return Si(a?{"files[]":t}:t,l&&new l,this.formSerializer)}}return i||o?(n.setContentType("application/json",!1),$O(t)):t}],transformResponse:[function(t){const n=this.transitional||qr.transitional,r=n&&n.forcedJSONParsing,o=this.responseType==="json";if(E.isResponse(t)||E.isReadableStream(t))return t;if(t&&E.isString(t)&&(r&&!this.responseType||o)){const s=!(n&&n.silentJSONParsing)&&o;try{return JSON.parse(t,this.parseReviver)}catch(a){if(s)throw a.name==="SyntaxError"?H.from(a,H.ERR_BAD_RESPONSE,this,null,this.response):a}}return t}],timeout:0,xsrfCookieName:"XSRF-TOKEN",xsrfHeaderName:"X-XSRF-TOKEN",maxContentLength:-1,maxBodyLength:-1,env:{FormData:Ie.classes.FormData,Blob:Ie.classes.Blob},validateStatus:function(t){return t>=200&&t<300},headers:{common:{Accept:"application/json, text/plain, */*","Content-Type":void 0}}};E.forEach(["delete","get","head","post","put","patch"],e=>{qr.headers[e]={}});const BO=E.toObjectSet(["age","authorization","content-length","content-type","etag","expires","from","host","if-modified-since","if-unmodified-since","last-modified","location","max-forwards","proxy-authorization","referer","retry-after","user-agent"]),WO=e=>{const t={};let n,r,o;return e&&e.split(` +`).forEach(function(s){o=s.indexOf(":"),n=s.substring(0,o).trim().toLowerCase(),r=s.substring(o+1).trim(),!(!n||t[n]&&BO[n])&&(n==="set-cookie"?t[n]?t[n].push(r):t[n]=[r]:t[n]=t[n]?t[n]+", "+r:r)}),t},ig=Symbol("internals");function Kr(e){return e&&String(e).trim().toLowerCase()}function Ei(e){return e===!1||e==null?e:E.isArray(e)?e.map(Ei):String(e)}function HO(e){const t=Object.create(null),n=/([^\s,;=]+)\s*(?:=\s*([^,;]+))?/g;let r;for(;r=n.exec(e);)t[r[1]]=r[2];return t}const UO=e=>/^[-_a-zA-Z0-9^`|~,!#$%&'*+.]+$/.test(e.trim());function Ka(e,t,n,r,o){if(E.isFunction(r))return r.call(this,t,n);if(o&&(t=n),!!E.isString(t)){if(E.isString(r))return t.indexOf(r)!==-1;if(E.isRegExp(r))return r.test(t)}}function GO(e){return e.trim().toLowerCase().replace(/([a-z\d])(\w*)/g,(t,n,r)=>n.toUpperCase()+r)}function qO(e,t){const n=E.toCamelCase(" "+t);["get","set","has"].forEach(r=>{Object.defineProperty(e,r+n,{value:function(o,i,s){return this[r].call(this,t,o,i,s)},configurable:!0})})}let Me=class{constructor(t){t&&this.set(t)}set(t,n,r){const o=this;function i(a,l,c){const u=Kr(l);if(!u)throw new Error("header name must be a non-empty string");const h=E.findKey(o,u);(!h||o[h]===void 0||c===!0||c===void 0&&o[h]!==!1)&&(o[h||l]=Ei(a))}const s=(a,l)=>E.forEach(a,(c,u)=>i(c,u,l));if(E.isPlainObject(t)||t instanceof this.constructor)s(t,n);else if(E.isString(t)&&(t=t.trim())&&!UO(t))s(WO(t),n);else if(E.isObject(t)&&E.isIterable(t)){let a={},l,c;for(const u of t){if(!E.isArray(u))throw TypeError("Object iterator must return a key-value pair");a[c=u[0]]=(l=a[c])?E.isArray(l)?[...l,u[1]]:[l,u[1]]:u[1]}s(a,n)}else t!=null&&i(n,t,r);return this}get(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);if(r){const o=this[r];if(!n)return o;if(n===!0)return HO(o);if(E.isFunction(n))return n.call(this,o,r);if(E.isRegExp(n))return n.exec(o);throw new TypeError("parser must be boolean|regexp|function")}}}has(t,n){if(t=Kr(t),t){const r=E.findKey(this,t);return!!(r&&this[r]!==void 0&&(!n||Ka(this,this[r],r,n)))}return!1}delete(t,n){const r=this;let o=!1;function i(s){if(s=Kr(s),s){const a=E.findKey(r,s);a&&(!n||Ka(r,r[a],a,n))&&(delete r[a],o=!0)}}return E.isArray(t)?t.forEach(i):i(t),o}clear(t){const n=Object.keys(this);let r=n.length,o=!1;for(;r--;){const i=n[r];(!t||Ka(this,this[i],i,t,!0))&&(delete this[i],o=!0)}return o}normalize(t){const n=this,r={};return E.forEach(this,(o,i)=>{const s=E.findKey(r,i);if(s){n[s]=Ei(o),delete n[i];return}const a=t?GO(i):String(i).trim();a!==i&&delete n[i],n[a]=Ei(o),r[a]=!0}),this}concat(...t){return this.constructor.concat(this,...t)}toJSON(t){const n=Object.create(null);return E.forEach(this,(r,o)=>{r!=null&&r!==!1&&(n[o]=t&&E.isArray(r)?r.join(", "):r)}),n}[Symbol.iterator](){return Object.entries(this.toJSON())[Symbol.iterator]()}toString(){return Object.entries(this.toJSON()).map(([t,n])=>t+": "+n).join(` +`)}getSetCookie(){return this.get("set-cookie")||[]}get[Symbol.toStringTag](){return"AxiosHeaders"}static from(t){return t instanceof this?t:new this(t)}static concat(t,...n){const r=new this(t);return n.forEach(o=>r.set(o)),r}static accessor(t){const r=(this[ig]=this[ig]={accessors:{}}).accessors,o=this.prototype;function i(s){const a=Kr(s);r[a]||(qO(o,s),r[a]=!0)}return E.isArray(t)?t.forEach(i):i(t),this}};Me.accessor(["Content-Type","Content-Length","Accept","Accept-Encoding","User-Agent","Authorization"]),E.reduceDescriptors(Me.prototype,({value:e},t)=>{let n=t[0].toUpperCase()+t.slice(1);return{get:()=>e,set(r){this[n]=r}}}),E.freezeMethods(Me);function Ya(e,t){const n=this||qr,r=t||n,o=Me.from(r.headers);let i=r.data;return E.forEach(e,function(a){i=a.call(n,i,o.normalize(),t?t.status:void 0)}),o.normalize(),i}function sg(e){return!!(e&&e.__CANCEL__)}function lr(e,t,n){H.call(this,e??"canceled",H.ERR_CANCELED,t,n),this.name="CanceledError"}E.inherits(lr,H,{__CANCEL__:!0});function ag(e,t,n){const r=n.config.validateStatus;!n.status||!r||r(n.status)?e(n):t(new H("Request failed with status code "+n.status,[H.ERR_BAD_REQUEST,H.ERR_BAD_RESPONSE][Math.floor(n.status/100)-4],n.config,n.request,n))}function KO(e){const t=/^([-+\w]{1,25})(:?\/\/|:)/.exec(e);return t&&t[1]||""}function YO(e,t){e=e||10;const n=new Array(e),r=new Array(e);let o=0,i=0,s;return t=t!==void 0?t:1e3,function(l){const c=Date.now(),u=r[i];s||(s=c),n[o]=l,r[o]=c;let h=i,d=0;for(;h!==o;)d+=n[h++],h=h%e;if(o=(o+1)%e,o===i&&(i=(i+1)%e),c-s{n=u,o=null,i&&(clearTimeout(i),i=null),e(...c)};return[(...c)=>{const u=Date.now(),h=u-n;h>=r?s(c,u):(o=c,i||(i=setTimeout(()=>{i=null,s(o)},r-h)))},()=>o&&s(o)]}const Oi=(e,t,n=3)=>{let r=0;const o=YO(50,250);return XO(i=>{const s=i.loaded,a=i.lengthComputable?i.total:void 0,l=s-r,c=o(l),u=s<=a;r=s;const h={loaded:s,total:a,progress:a?s/a:void 0,bytes:l,rate:c||void 0,estimated:c&&a&&u?(a-s)/c:void 0,event:i,lengthComputable:a!=null,[t?"download":"upload"]:!0};e(h)},n)},lg=(e,t)=>{const n=e!=null;return[r=>t[0]({lengthComputable:n,total:e,loaded:r}),t[1]]},cg=e=>(...t)=>E.asap(()=>e(...t)),QO=Ie.hasStandardBrowserEnv?((e,t)=>n=>(n=new URL(n,Ie.origin),e.protocol===n.protocol&&e.host===n.host&&(t||e.port===n.port)))(new URL(Ie.origin),Ie.navigator&&/(msie|trident)/i.test(Ie.navigator.userAgent)):()=>!0,JO=Ie.hasStandardBrowserEnv?{write(e,t,n,r,o,i){const s=[e+"="+encodeURIComponent(t)];E.isNumber(n)&&s.push("expires="+new Date(n).toGMTString()),E.isString(r)&&s.push("path="+r),E.isString(o)&&s.push("domain="+o),i===!0&&s.push("secure"),document.cookie=s.join("; ")},read(e){const t=document.cookie.match(new RegExp("(^|;\\s*)("+e+")=([^;]*)"));return t?decodeURIComponent(t[3]):null},remove(e){this.write(e,"",Date.now()-864e5)}}:{write(){},read(){return null},remove(){}};function ZO(e){return/^([a-z][a-z\d+\-.]*:)?\/\//i.test(e)}function eP(e,t){return t?e.replace(/\/?\/$/,"")+"/"+t.replace(/^\/+/,""):e}function ug(e,t,n){let r=!ZO(t);return e&&(r||n==!1)?eP(e,t):t}const dg=e=>e instanceof Me?{...e}:e;function In(e,t){t=t||{};const n={};function r(c,u,h,d){return E.isPlainObject(c)&&E.isPlainObject(u)?E.merge.call({caseless:d},c,u):E.isPlainObject(u)?E.merge({},u):E.isArray(u)?u.slice():u}function o(c,u,h,d){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c,h,d)}else return r(c,u,h,d)}function i(c,u){if(!E.isUndefined(u))return r(void 0,u)}function s(c,u){if(E.isUndefined(u)){if(!E.isUndefined(c))return r(void 0,c)}else return r(void 0,u)}function a(c,u,h){if(h in t)return r(c,u);if(h in e)return r(void 0,c)}const l={url:i,method:i,data:i,baseURL:s,transformRequest:s,transformResponse:s,paramsSerializer:s,timeout:s,timeoutMessage:s,withCredentials:s,withXSRFToken:s,adapter:s,responseType:s,xsrfCookieName:s,xsrfHeaderName:s,onUploadProgress:s,onDownloadProgress:s,decompress:s,maxContentLength:s,maxBodyLength:s,beforeRedirect:s,transport:s,httpAgent:s,httpsAgent:s,cancelToken:s,socketPath:s,responseEncoding:s,validateStatus:a,headers:(c,u,h)=>o(dg(c),dg(u),h,!0)};return E.forEach(Object.keys({...e,...t}),function(u){const h=l[u]||o,d=h(e[u],t[u],u);E.isUndefined(d)&&h!==a||(n[u]=d)}),n}const hg=e=>{const t=In({},e);let{data:n,withXSRFToken:r,xsrfHeaderName:o,xsrfCookieName:i,headers:s,auth:a}=t;if(t.headers=s=Me.from(s),t.url=tg(ug(t.baseURL,t.url,t.allowAbsoluteUrls),e.params,e.paramsSerializer),a&&s.set("Authorization","Basic "+btoa((a.username||"")+":"+(a.password?unescape(encodeURIComponent(a.password)):""))),E.isFormData(n)){if(Ie.hasStandardBrowserEnv||Ie.hasStandardBrowserWebWorkerEnv)s.setContentType(void 0);else if(E.isFunction(n.getHeaders)){const l=n.getHeaders(),c=["content-type","content-length"];Object.entries(l).forEach(([u,h])=>{c.includes(u.toLowerCase())&&s.set(u,h)})}}if(Ie.hasStandardBrowserEnv&&(r&&E.isFunction(r)&&(r=r(t)),r||r!==!1&&QO(t.url))){const l=o&&i&&JO.read(i);l&&s.set(o,l)}return t},tP=typeof XMLHttpRequest<"u"&&function(e){return new Promise(function(n,r){const o=hg(e);let i=o.data;const s=Me.from(o.headers).normalize();let{responseType:a,onUploadProgress:l,onDownloadProgress:c}=o,u,h,d,m,f;function p(){m&&m(),f&&f(),o.cancelToken&&o.cancelToken.unsubscribe(u),o.signal&&o.signal.removeEventListener("abort",u)}let v=new XMLHttpRequest;v.open(o.method.toUpperCase(),o.url,!0),v.timeout=o.timeout;function b(){if(!v)return;const x=Me.from("getAllResponseHeaders"in v&&v.getAllResponseHeaders()),k={data:!a||a==="text"||a==="json"?v.responseText:v.response,status:v.status,statusText:v.statusText,headers:x,config:e,request:v};ag(function(R){n(R),p()},function(R){r(R),p()},k),v=null}"onloadend"in v?v.onloadend=b:v.onreadystatechange=function(){!v||v.readyState!==4||v.status===0&&!(v.responseURL&&v.responseURL.indexOf("file:")===0)||setTimeout(b)},v.onabort=function(){v&&(r(new H("Request aborted",H.ECONNABORTED,e,v)),v=null)},v.onerror=function(w){const k=w&&w.message?w.message:"Network Error",O=new H(k,H.ERR_NETWORK,e,v);O.event=w||null,r(O),v=null},v.ontimeout=function(){let w=o.timeout?"timeout of "+o.timeout+"ms exceeded":"timeout exceeded";const k=o.transitional||rg;o.timeoutErrorMessage&&(w=o.timeoutErrorMessage),r(new H(w,k.clarifyTimeoutError?H.ETIMEDOUT:H.ECONNABORTED,e,v)),v=null},i===void 0&&s.setContentType(null),"setRequestHeader"in v&&E.forEach(s.toJSON(),function(w,k){v.setRequestHeader(k,w)}),E.isUndefined(o.withCredentials)||(v.withCredentials=!!o.withCredentials),a&&a!=="json"&&(v.responseType=o.responseType),c&&([d,f]=Oi(c,!0),v.addEventListener("progress",d)),l&&v.upload&&([h,m]=Oi(l),v.upload.addEventListener("progress",h),v.upload.addEventListener("loadend",m)),(o.cancelToken||o.signal)&&(u=x=>{v&&(r(!x||x.type?new lr(null,e,v):x),v.abort(),v=null)},o.cancelToken&&o.cancelToken.subscribe(u),o.signal&&(o.signal.aborted?u():o.signal.addEventListener("abort",u)));const y=KO(o.url);if(y&&Ie.protocols.indexOf(y)===-1){r(new H("Unsupported protocol "+y+":",H.ERR_BAD_REQUEST,e));return}v.send(i||null)})},nP=(e,t)=>{const{length:n}=e=e?e.filter(Boolean):[];if(t||n){let r=new AbortController,o;const i=function(c){if(!o){o=!0,a();const u=c instanceof Error?c:this.reason;r.abort(u instanceof H?u:new lr(u instanceof Error?u.message:u))}};let s=t&&setTimeout(()=>{s=null,i(new H(`timeout ${t} of ms exceeded`,H.ETIMEDOUT))},t);const a=()=>{e&&(s&&clearTimeout(s),s=null,e.forEach(c=>{c.unsubscribe?c.unsubscribe(i):c.removeEventListener("abort",i)}),e=null)};e.forEach(c=>c.addEventListener("abort",i));const{signal:l}=r;return l.unsubscribe=()=>E.asap(a),l}},rP=function*(e,t){let n=e.byteLength;if(n{const o=oP(e,t);let i=0,s,a=l=>{s||(s=!0,r&&r(l))};return new ReadableStream({async pull(l){try{const{done:c,value:u}=await o.next();if(c){a(),l.close();return}let h=u.byteLength;if(n){let d=i+=h;n(d)}l.enqueue(new Uint8Array(u))}catch(c){throw a(c),c}},cancel(l){return a(l),o.return()}},{highWaterMark:2})},gg=64*1024,{isFunction:Pi}=E,sP=(({Request:e,Response:t})=>({Request:e,Response:t}))(E.global),{ReadableStream:pg,TextEncoder:mg}=E.global,vg=(e,...t)=>{try{return!!e(...t)}catch{return!1}},aP=e=>{e=E.merge.call({skipUndefined:!0},sP,e);const{fetch:t,Request:n,Response:r}=e,o=t?Pi(t):typeof fetch=="function",i=Pi(n),s=Pi(r);if(!o)return!1;const a=o&&Pi(pg),l=o&&(typeof mg=="function"?(f=>p=>f.encode(p))(new mg):async f=>new Uint8Array(await new n(f).arrayBuffer())),c=i&&a&&vg(()=>{let f=!1;const p=new n(Ie.origin,{body:new pg,method:"POST",get duplex(){return f=!0,"half"}}).headers.has("Content-Type");return f&&!p}),u=s&&a&&vg(()=>E.isReadableStream(new r("").body)),h={stream:u&&(f=>f.body)};o&&["text","arrayBuffer","blob","formData","stream"].forEach(f=>{!h[f]&&(h[f]=(p,v)=>{let b=p&&p[f];if(b)return b.call(p);throw new H(`Response type '${f}' is not supported`,H.ERR_NOT_SUPPORT,v)})});const d=async f=>{if(f==null)return 0;if(E.isBlob(f))return f.size;if(E.isSpecCompliantForm(f))return(await new n(Ie.origin,{method:"POST",body:f}).arrayBuffer()).byteLength;if(E.isArrayBufferView(f)||E.isArrayBuffer(f))return f.byteLength;if(E.isURLSearchParams(f)&&(f=f+""),E.isString(f))return(await l(f)).byteLength},m=async(f,p)=>{const v=E.toFiniteNumber(f.getContentLength());return v??d(p)};return async f=>{let{url:p,method:v,data:b,signal:y,cancelToken:x,timeout:w,onDownloadProgress:k,onUploadProgress:O,responseType:R,headers:I,withCredentials:N="same-origin",fetchOptions:A}=hg(f),T=t||fetch;R=R?(R+"").toLowerCase():"text";let S=nP([y,x&&x.toAbortSignal()],w),_=null;const D=S&&S.unsubscribe&&(()=>{S.unsubscribe()});let z;try{if(O&&c&&v!=="get"&&v!=="head"&&(z=await m(I,b))!==0){let Z=new n(p,{method:"POST",body:b,duplex:"half"}),pe;if(E.isFormData(b)&&(pe=Z.headers.get("content-type"))&&I.setContentType(pe),Z.body){const[Je,xt]=lg(z,Oi(cg(O)));b=fg(Z.body,gg,Je,xt)}}E.isString(N)||(N=N?"include":"omit");const W=i&&"credentials"in n.prototype,J={...A,signal:S,method:v.toUpperCase(),headers:I.normalize().toJSON(),body:b,duplex:"half",credentials:W?N:void 0};_=i&&new n(p,J);let ee=await(i?T(_,A):T(p,J));const $=u&&(R==="stream"||R==="response");if(u&&(k||$&&D)){const Z={};["status","statusText","headers"].forEach(_t=>{Z[_t]=ee[_t]});const pe=E.toFiniteNumber(ee.headers.get("content-length")),[Je,xt]=k&&lg(pe,Oi(cg(k),!0))||[];ee=new r(fg(ee.body,gg,Je,()=>{xt&&xt(),D&&D()}),Z)}R=R||"text";let q=await h[E.findKey(h,R)||"text"](ee,f);return!$&&D&&D(),await new Promise((Z,pe)=>{ag(Z,pe,{data:q,headers:Me.from(ee.headers),status:ee.status,statusText:ee.statusText,config:f,request:_})})}catch(W){throw D&&D(),W&&W.name==="TypeError"&&/Load failed|fetch/i.test(W.message)?Object.assign(new H("Network Error",H.ERR_NETWORK,f,_),{cause:W.cause||W}):H.from(W,W&&W.code,f,_)}}},lP=new Map,bg=e=>{let t=e?e.env:{};const{fetch:n,Request:r,Response:o}=t,i=[r,o,n];let s=i.length,a=s,l,c,u=lP;for(;a--;)l=i[a],c=u.get(l),c===void 0&&u.set(l,c=a?new Map:aP(t)),u=c;return c};bg();const Xa={http:TO,xhr:tP,fetch:{get:bg}};E.forEach(Xa,(e,t)=>{if(e){try{Object.defineProperty(e,"name",{value:t})}catch{}Object.defineProperty(e,"adapterName",{value:t})}});const yg=e=>`- ${e}`,cP=e=>E.isFunction(e)||e===null||e===!1,xg={getAdapter:(e,t)=>{e=E.isArray(e)?e:[e];const{length:n}=e;let r,o;const i={};for(let s=0;s`adapter ${l} `+(c===!1?"is not supported by the environment":"is not available in the build"));let a=n?s.length>1?`since : +`+s.map(yg).join(` +`):" "+yg(s[0]):"as no adapter specified";throw new H("There is no suitable adapter to dispatch the request "+a,"ERR_NOT_SUPPORT")}return o},adapters:Xa};function Qa(e){if(e.cancelToken&&e.cancelToken.throwIfRequested(),e.signal&&e.signal.aborted)throw new lr(null,e)}function kg(e){return Qa(e),e.headers=Me.from(e.headers),e.data=Ya.call(e,e.transformRequest),["post","put","patch"].indexOf(e.method)!==-1&&e.headers.setContentType("application/x-www-form-urlencoded",!1),xg.getAdapter(e.adapter||qr.adapter,e)(e).then(function(r){return Qa(e),r.data=Ya.call(e,e.transformResponse,r),r.headers=Me.from(r.headers),r},function(r){return sg(r)||(Qa(e),r&&r.response&&(r.response.data=Ya.call(e,e.transformResponse,r.response),r.response.headers=Me.from(r.response.headers))),Promise.reject(r)})}const Cg="1.12.2",Ri={};["object","boolean","number","function","string","symbol"].forEach((e,t)=>{Ri[e]=function(r){return typeof r===e||"a"+(t<1?"n ":" ")+e}});const wg={};Ri.transitional=function(t,n,r){function o(i,s){return"[Axios v"+Cg+"] Transitional option '"+i+"'"+s+(r?". "+r:"")}return(i,s,a)=>{if(t===!1)throw new H(o(s," has been removed"+(n?" in "+n:"")),H.ERR_DEPRECATED);return n&&!wg[s]&&(wg[s]=!0,console.warn(o(s," has been deprecated since v"+n+" and will be removed in the near future"))),t?t(i,s,a):!0}},Ri.spelling=function(t){return(n,r)=>(console.warn(`${r} is likely a misspelling of ${t}`),!0)};function uP(e,t,n){if(typeof e!="object")throw new H("options must be an object",H.ERR_BAD_OPTION_VALUE);const r=Object.keys(e);let o=r.length;for(;o-- >0;){const i=r[o],s=t[i];if(s){const a=e[i],l=a===void 0||s(a,i,e);if(l!==!0)throw new H("option "+i+" must be "+l,H.ERR_BAD_OPTION_VALUE);continue}if(n!==!0)throw new H("Unknown option "+i,H.ERR_BAD_OPTION)}}const Ii={assertOptions:uP,validators:Ri},mt=Ii.validators;let Tn=class{constructor(t){this.defaults=t||{},this.interceptors={request:new ng,response:new ng}}async request(t,n){try{return await this._request(t,n)}catch(r){if(r instanceof Error){let o={};Error.captureStackTrace?Error.captureStackTrace(o):o=new Error;const i=o.stack?o.stack.replace(/^.+\n/,""):"";try{r.stack?i&&!String(r.stack).endsWith(i.replace(/^.+\n.+\n/,""))&&(r.stack+=` +`+i):r.stack=i}catch{}}throw r}}_request(t,n){typeof t=="string"?(n=n||{},n.url=t):n=t||{},n=In(this.defaults,n);const{transitional:r,paramsSerializer:o,headers:i}=n;r!==void 0&&Ii.assertOptions(r,{silentJSONParsing:mt.transitional(mt.boolean),forcedJSONParsing:mt.transitional(mt.boolean),clarifyTimeoutError:mt.transitional(mt.boolean)},!1),o!=null&&(E.isFunction(o)?n.paramsSerializer={serialize:o}:Ii.assertOptions(o,{encode:mt.function,serialize:mt.function},!0)),n.allowAbsoluteUrls!==void 0||(this.defaults.allowAbsoluteUrls!==void 0?n.allowAbsoluteUrls=this.defaults.allowAbsoluteUrls:n.allowAbsoluteUrls=!0),Ii.assertOptions(n,{baseUrl:mt.spelling("baseURL"),withXsrfToken:mt.spelling("withXSRFToken")},!0),n.method=(n.method||this.defaults.method||"get").toLowerCase();let s=i&&E.merge(i.common,i[n.method]);i&&E.forEach(["delete","get","head","post","put","patch","common"],f=>{delete i[f]}),n.headers=Me.concat(s,i);const a=[];let l=!0;this.interceptors.request.forEach(function(p){typeof p.runWhen=="function"&&p.runWhen(n)===!1||(l=l&&p.synchronous,a.unshift(p.fulfilled,p.rejected))});const c=[];this.interceptors.response.forEach(function(p){c.push(p.fulfilled,p.rejected)});let u,h=0,d;if(!l){const f=[kg.bind(this),void 0];for(f.unshift(...a),f.push(...c),d=f.length,u=Promise.resolve(n);h{if(!r._listeners)return;let i=r._listeners.length;for(;i-- >0;)r._listeners[i](o);r._listeners=null}),this.promise.then=o=>{let i;const s=new Promise(a=>{r.subscribe(a),i=a}).then(o);return s.cancel=function(){r.unsubscribe(i)},s},t(function(i,s,a){r.reason||(r.reason=new lr(i,s,a),n(r.reason))})}throwIfRequested(){if(this.reason)throw this.reason}subscribe(t){if(this.reason){t(this.reason);return}this._listeners?this._listeners.push(t):this._listeners=[t]}unsubscribe(t){if(!this._listeners)return;const n=this._listeners.indexOf(t);n!==-1&&this._listeners.splice(n,1)}toAbortSignal(){const t=new AbortController,n=r=>{t.abort(r)};return this.subscribe(n),t.signal.unsubscribe=()=>this.unsubscribe(n),t.signal}static source(){let t;return{token:new Sp(function(o){t=o}),cancel:t}}};function hP(e){return function(n){return e.apply(null,n)}}function fP(e){return E.isObject(e)&&e.isAxiosError===!0}const Ja={Continue:100,SwitchingProtocols:101,Processing:102,EarlyHints:103,Ok:200,Created:201,Accepted:202,NonAuthoritativeInformation:203,NoContent:204,ResetContent:205,PartialContent:206,MultiStatus:207,AlreadyReported:208,ImUsed:226,MultipleChoices:300,MovedPermanently:301,Found:302,SeeOther:303,NotModified:304,UseProxy:305,Unused:306,TemporaryRedirect:307,PermanentRedirect:308,BadRequest:400,Unauthorized:401,PaymentRequired:402,Forbidden:403,NotFound:404,MethodNotAllowed:405,NotAcceptable:406,ProxyAuthenticationRequired:407,RequestTimeout:408,Conflict:409,Gone:410,LengthRequired:411,PreconditionFailed:412,PayloadTooLarge:413,UriTooLong:414,UnsupportedMediaType:415,RangeNotSatisfiable:416,ExpectationFailed:417,ImATeapot:418,MisdirectedRequest:421,UnprocessableEntity:422,Locked:423,FailedDependency:424,TooEarly:425,UpgradeRequired:426,PreconditionRequired:428,TooManyRequests:429,RequestHeaderFieldsTooLarge:431,UnavailableForLegalReasons:451,InternalServerError:500,NotImplemented:501,BadGateway:502,ServiceUnavailable:503,GatewayTimeout:504,HttpVersionNotSupported:505,VariantAlsoNegotiates:506,InsufficientStorage:507,LoopDetected:508,NotExtended:510,NetworkAuthenticationRequired:511};Object.entries(Ja).forEach(([e,t])=>{Ja[t]=e});function Sg(e){const t=new Tn(e),n=jf(Tn.prototype.request,t);return E.extend(n,Tn.prototype,t,{allOwnKeys:!0}),E.extend(n,t,null,{allOwnKeys:!0}),n.create=function(o){return Sg(In(e,o))},n}const ce=Sg(qr);ce.Axios=Tn,ce.CanceledError=lr,ce.CancelToken=dP,ce.isCancel=sg,ce.VERSION=Cg,ce.toFormData=Si,ce.AxiosError=H,ce.Cancel=ce.CanceledError,ce.all=function(t){return Promise.all(t)},ce.spread=hP,ce.isAxiosError=fP,ce.mergeConfig=In,ce.AxiosHeaders=Me,ce.formToJSON=e=>og(E.isHTMLForm(e)?new FormData(e):e),ce.getAdapter=xg.getAdapter,ce.HttpStatusCode=Ja,ce.default=ce;const{Axios:HT,AxiosError:UT,CanceledError:GT,isCancel:qT,CancelToken:KT,VERSION:YT,all:XT,Cancel:QT,isAxiosError:JT,spread:ZT,toFormData:e5,AxiosHeaders:t5,HttpStatusCode:n5,formToJSON:r5,getAdapter:o5,mergeConfig:i5}=ce;class Eg{_fns;constructor(){this._fns=[]}eject(t){const n=this._fns.indexOf(t);n!==-1&&(this._fns=[...this._fns.slice(0,n),...this._fns.slice(n+1)])}use(t){this._fns=[...this._fns,t]}}const vt={BASE:"",CREDENTIALS:"include",ENCODE_PATH:void 0,HEADERS:void 0,PASSWORD:void 0,TOKEN:void 0,USERNAME:void 0,VERSION:"0.1.0",WITH_CREDENTIALS:!1,interceptors:{request:new Eg,response:new Eg}};var gP=(e,t,n,r,o,i,s,a)=>{let l=document.documentElement,c=["light","dark"];function u(m){(Array.isArray(e)?e:[e]).forEach(f=>{let p=f==="class",v=p&&i?o.map(b=>i[b]||b):o;p?(l.classList.remove(...v),l.classList.add(i&&i[m]?i[m]:m)):l.setAttribute(f,m)}),h(m)}function h(m){a&&c.includes(m)&&(l.style.colorScheme=m)}function d(){return window.matchMedia("(prefers-color-scheme: dark)").matches?"dark":"light"}if(r)u(r);else try{let m=localStorage.getItem(t)||n,f=s&&m==="system"?d():m;u(f)}catch{}},Og=["light","dark"],Pg="(prefers-color-scheme: dark)",pP=typeof window>"u",Rg=C.createContext(void 0),mP=e=>C.useContext(Rg)?C.createElement(C.Fragment,null,e.children):C.createElement(bP,{...e}),vP=["light","dark"],bP=({forcedTheme:e,disableTransitionOnChange:t=!1,enableSystem:n=!0,enableColorScheme:r=!0,storageKey:o="theme",themes:i=vP,defaultTheme:s=n?"system":"light",attribute:a="data-theme",value:l,children:c,nonce:u,scriptProps:h})=>{let[d,m]=C.useState(()=>xP(o,s)),[f,p]=C.useState(()=>d==="system"?Za():d),v=l?Object.values(l):i,b=C.useCallback(k=>{let O=k;if(!O)return;k==="system"&&n&&(O=Za());let R=l?l[O]:O,I=t?kP(u):null,N=document.documentElement,A=T=>{T==="class"?(N.classList.remove(...v),R&&N.classList.add(R)):T.startsWith("data-")&&(R?N.setAttribute(T,R):N.removeAttribute(T))};if(Array.isArray(a)?a.forEach(A):A(a),r){let T=Og.includes(s)?s:null,S=Og.includes(O)?O:T;N.style.colorScheme=S}I?.()},[u]),y=C.useCallback(k=>{let O=typeof k=="function"?k(d):k;m(O);try{localStorage.setItem(o,O)}catch{}},[d]),x=C.useCallback(k=>{let O=Za(k);p(O),d==="system"&&n&&!e&&b("system")},[d,e]);C.useEffect(()=>{let k=window.matchMedia(Pg);return k.addListener(x),x(k),()=>k.removeListener(x)},[x]),C.useEffect(()=>{let k=O=>{O.key===o&&(O.newValue?m(O.newValue):y(s))};return window.addEventListener("storage",k),()=>window.removeEventListener("storage",k)},[y]),C.useEffect(()=>{b(e??d)},[e,d]);let w=C.useMemo(()=>({theme:d,setTheme:y,forcedTheme:e,resolvedTheme:d==="system"?f:d,themes:n?[...i,"system"]:i,systemTheme:n?f:void 0}),[d,y,e,f,n,i]);return C.createElement(Rg.Provider,{value:w},C.createElement(yP,{forcedTheme:e,storageKey:o,attribute:a,enableSystem:n,enableColorScheme:r,defaultTheme:s,value:l,themes:i,nonce:u,scriptProps:h}),c)},yP=C.memo(({forcedTheme:e,storageKey:t,attribute:n,enableSystem:r,enableColorScheme:o,defaultTheme:i,value:s,themes:a,nonce:l,scriptProps:c})=>{let u=JSON.stringify([n,t,i,e,a,s,r,o]).slice(1,-1);return C.createElement("script",{...c,suppressHydrationWarning:!0,nonce:typeof window>"u"?l:"",dangerouslySetInnerHTML:{__html:`(${gP.toString()})(${u})`}})}),xP=(e,t)=>{if(pP)return;let n;try{n=localStorage.getItem(e)||void 0}catch{}return n||t},kP=e=>{let t=document.createElement("style");return e&&t.setAttribute("nonce",e),t.appendChild(document.createTextNode("*,*::before,*::after{-webkit-transition:none!important;-moz-transition:none!important;-o-transition:none!important;-ms-transition:none!important;transition:none!important}")),document.head.appendChild(t),()=>{window.getComputedStyle(document.body),setTimeout(()=>{document.head.removeChild(t)},1)}},Za=e=>(e||(e=window.matchMedia(Pg)),e.matches?"dark":"light");const CP=e=>g.jsx(mP,{attribute:"class",disableTransitionOnChange:!0,...e});/** + * react-router v7.9.3 * * Copyright (c) Remix Software Inc. * @@ -35,9 +35,9 @@ * LICENSE.md file in the root directory of this source tree. * * @license MIT - */var Pg="popstate";function wP(e={}){function t(r,o){let{pathname:i,search:s,hash:a}=r.location;return el("",{pathname:i,search:s,hash:a},o.state&&o.state.usr||null,o.state&&o.state.key||"default")}function n(r,o){return typeof o=="string"?o:Yr(o)}return EP(t,n,null,e)}function le(e,t){if(e===!1||e===null||typeof e>"u")throw new Error(t)}function vt(e,t){if(!e){typeof console<"u"&&console.warn(t);try{throw new Error(t)}catch{}}}function SP(){return Math.random().toString(36).substring(2,10)}function Rg(e,t){return{usr:e.state,key:e.key,idx:t}}function el(e,t,n=null,r){return{pathname:typeof e=="string"?e:e.pathname,search:"",hash:"",...typeof t=="string"?cr(t):t,state:n,key:t&&t.key||r||SP()}}function Yr({pathname:e="/",search:t="",hash:n=""}){return t&&t!=="?"&&(e+=t.charAt(0)==="?"?t:"?"+t),n&&n!=="#"&&(e+=n.charAt(0)==="#"?n:"#"+n),e}function cr(e){let t={};if(e){let n=e.indexOf("#");n>=0&&(t.hash=e.substring(n),e=e.substring(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substring(r),e=e.substring(0,r)),e&&(t.pathname=e)}return t}function EP(e,t,n,r={}){let{window:o=document.defaultView,v5Compat:i=!1}=r,s=o.history,a="POP",l=null,c=u();c==null&&(c=0,s.replaceState({...s.state,idx:c},""));function u(){return(s.state||{idx:null}).idx}function h(){a="POP";let v=u(),b=v==null?null:v-c;c=v,l&&l({action:a,location:p.location,delta:b})}function d(v,b){a="PUSH";let y=el(p.location,v,b);c=u()+1;let x=Rg(y,c),w=p.createHref(y);try{s.pushState(x,"",w)}catch(k){if(k instanceof DOMException&&k.name==="DataCloneError")throw k;o.location.assign(w)}i&&l&&l({action:a,location:p.location,delta:1})}function m(v,b){a="REPLACE";let y=el(p.location,v,b);c=u();let x=Rg(y,c),w=p.createHref(y);s.replaceState(x,"",w),i&&l&&l({action:a,location:p.location,delta:0})}function f(v){return OP(v)}let p={get action(){return a},get location(){return e(o,s)},listen(v){if(l)throw new Error("A history only accepts one active listener");return o.addEventListener(Pg,h),l=v,()=>{o.removeEventListener(Pg,h),l=null}},createHref(v){return t(o,v)},createURL:f,encodeLocation(v){let b=f(v);return{pathname:b.pathname,search:b.search,hash:b.hash}},push:d,replace:m,go(v){return s.go(v)}};return p}function OP(e,t=!1){let n="http://localhost";typeof window<"u"&&(n=window.location.origin!=="null"?window.location.origin:window.location.href),le(n,"No window.location.(origin|href) available to create URL");let r=typeof e=="string"?e:Yr(e);return r=r.replace(/ $/,"%20"),!t&&r.startsWith("//")&&(r=n+r),new URL(r,n)}function Ig(e,t,n="/"){return PP(e,t,n,!1)}function PP(e,t,n,r){let o=typeof t=="string"?cr(t):t,i=Pt(o.pathname||"/",n);if(i==null)return null;let s=Tg(e);RP(s);let a=null;for(let l=0;a==null&&l{let u={relativePath:c===void 0?s.path||"":c,caseSensitive:s.caseSensitive===!0,childrenIndex:a,route:s};if(u.relativePath.startsWith("/")){if(!u.relativePath.startsWith(r)&&l)return;le(u.relativePath.startsWith(r),`Absolute route path "${u.relativePath}" nested under path "${r}" is not valid. An absolute child route path must start with the combined path of all its parent routes.`),u.relativePath=u.relativePath.slice(r.length)}let h=Rt([r,u.relativePath]),d=n.concat(u);s.children&&s.children.length>0&&(le(s.index!==!0,`Index routes must not have child routes. Please remove all child routes from route path "${h}".`),Tg(s.children,t,d,h,l)),!(s.path==null&&!s.index)&&t.push({path:h,score:FP(h,s.index),routesMeta:d})};return e.forEach((s,a)=>{if(s.path===""||!s.path?.includes("?"))i(s,a);else for(let l of Ng(s.path))i(s,a,!0,l)}),t}function Ng(e){let t=e.split("/");if(t.length===0)return[];let[n,...r]=t,o=n.endsWith("?"),i=n.replace(/\?$/,"");if(r.length===0)return o?[i,""]:[i];let s=Ng(r.join("/")),a=[];return a.push(...s.map(l=>l===""?i:[i,l].join("/"))),o&&a.push(...s),a.map(l=>e.startsWith("/")&&l===""?"/":l)}function RP(e){e.sort((t,n)=>t.score!==n.score?n.score-t.score:LP(t.routesMeta.map(r=>r.childrenIndex),n.routesMeta.map(r=>r.childrenIndex)))}var IP=/^:[\w-]+$/,TP=3,NP=2,AP=1,_P=10,VP=-2,Ag=e=>e==="*";function FP(e,t){let n=e.split("/"),r=n.length;return n.some(Ag)&&(r+=VP),t&&(r+=NP),n.filter(o=>!Ag(o)).reduce((o,i)=>o+(IP.test(i)?TP:i===""?AP:_P),r)}function LP(e,t){return e.length===t.length&&e.slice(0,-1).every((r,o)=>r===t[o])?e[e.length-1]-t[t.length-1]:0}function DP(e,t,n=!1){let{routesMeta:r}=e,o={},i="/",s=[];for(let a=0;a{if(u==="*"){let f=a[d]||"";s=i.slice(0,i.length-f.length).replace(/(.)\/+$/,"$1")}const m=a[d];return h&&!m?c[u]=void 0:c[u]=(m||"").replace(/%2F/g,"/"),c},{}),pathname:i,pathnameBase:s,pattern:e}}function zP(e,t=!1,n=!0){vt(e==="*"||!e.endsWith("*")||e.endsWith("/*"),`Route path "${e}" will be treated as if it were "${e.replace(/\*$/,"/*")}" because the \`*\` character must always follow a \`/\` in the pattern. To get rid of this warning, please change the route path to "${e.replace(/\*$/,"/*")}".`);let r=[],o="^"+e.replace(/\/*\*?$/,"").replace(/^\/*/,"/").replace(/[\\.*+^${}|()[\]]/g,"\\$&").replace(/\/:([\w-]+)(\?)?/g,(s,a,l)=>(r.push({paramName:a,isOptional:l!=null}),l?"/?([^\\/]+)?":"/([^\\/]+)")).replace(/\/([\w-]+)\?(\/|$)/g,"(/$1)?$2");return e.endsWith("*")?(r.push({paramName:"*"}),o+=e==="*"||e==="/*"?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?o+="\\/*$":e!==""&&e!=="/"&&(o+="(?:(?=\\/|$))"),[new RegExp(o,t?void 0:"i"),r]}function MP(e){try{return e.split("/").map(t=>decodeURIComponent(t).replace(/\//g,"%2F")).join("/")}catch(t){return vt(!1,`The URL path "${e}" could not be decoded because it is a malformed URL segment. This is probably due to a bad percent encoding (${t}).`),e}}function Pt(e,t){if(t==="/")return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&r!=="/"?null:e.slice(n)||"/"}function jP(e,t="/"){let{pathname:n,search:r="",hash:o=""}=typeof e=="string"?cr(e):e;return{pathname:n?n.startsWith("/")?n:$P(n,t):t,search:HP(r),hash:UP(o)}}function $P(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach(o=>{o===".."?n.length>1&&n.pop():o!=="."&&n.push(o)}),n.length>1?n.join("/"):"/"}function tl(e,t,n,r){return`Cannot include a '${e}' character in a manually specified \`to.${t}\` field [${JSON.stringify(r)}]. Please separate it out to the \`to.${n}\` field. Alternatively you may provide the full path as a string in and the router will parse it for you.`}function BP(e){return e.filter((t,n)=>n===0||t.route.path&&t.route.path.length>0)}function _g(e){let t=BP(e);return t.map((n,r)=>r===t.length-1?n.pathname:n.pathnameBase)}function Vg(e,t,n,r=!1){let o;typeof e=="string"?o=cr(e):(o={...e},le(!o.pathname||!o.pathname.includes("?"),tl("?","pathname","search",o)),le(!o.pathname||!o.pathname.includes("#"),tl("#","pathname","hash",o)),le(!o.search||!o.search.includes("#"),tl("#","search","hash",o)));let i=e===""||o.pathname==="",s=i?"/":o.pathname,a;if(s==null)a=n;else{let h=t.length-1;if(!r&&s.startsWith("..")){let d=s.split("/");for(;d[0]==="..";)d.shift(),h-=1;o.pathname=d.join("/")}a=h>=0?t[h]:"/"}let l=jP(o,a),c=s&&s!=="/"&&s.endsWith("/"),u=(i||s===".")&&n.endsWith("/");return!l.pathname.endsWith("/")&&(c||u)&&(l.pathname+="/"),l}var Rt=e=>e.join("/").replace(/\/\/+/g,"/"),WP=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),HP=e=>!e||e==="?"?"":e.startsWith("?")?e:"?"+e,UP=e=>!e||e==="#"?"":e.startsWith("#")?e:"#"+e;function GP(e){return e!=null&&typeof e.status=="number"&&typeof e.statusText=="string"&&typeof e.internal=="boolean"&&"data"in e}var Fg=["POST","PUT","PATCH","DELETE"];new Set(Fg);var qP=["GET",...Fg];new Set(qP);var ur=C.createContext(null);ur.displayName="DataRouter";var Ni=C.createContext(null);Ni.displayName="DataRouterState",C.createContext(!1);var Lg=C.createContext({isTransitioning:!1});Lg.displayName="ViewTransition";var KP=C.createContext(new Map);KP.displayName="Fetchers";var YP=C.createContext(null);YP.displayName="Await";var bt=C.createContext(null);bt.displayName="Navigation";var Xr=C.createContext(null);Xr.displayName="Location";var It=C.createContext({outlet:null,matches:[],isDataRoute:!1});It.displayName="Route";var nl=C.createContext(null);nl.displayName="RouteError";function XP(e,{relative:t}={}){le(Qr(),"useHref() may be used only in the context of a component.");let{basename:n,navigator:r}=C.useContext(bt),{hash:o,pathname:i,search:s}=Jr(e,{relative:t}),a=i;return n!=="/"&&(a=i==="/"?n:Rt([n,i])),r.createHref({pathname:a,search:s,hash:o})}function Qr(){return C.useContext(Xr)!=null}function Nn(){return le(Qr(),"useLocation() may be used only in the context of a component."),C.useContext(Xr).location}var Dg="You should call navigate() in a React.useEffect(), not when your component is first rendered.";function zg(e){C.useContext(bt).static||C.useLayoutEffect(e)}function QP(){let{isDataRoute:e}=C.useContext(It);return e?uR():JP()}function JP(){le(Qr(),"useNavigate() may be used only in the context of a component.");let e=C.useContext(ur),{basename:t,navigator:n}=C.useContext(bt),{matches:r}=C.useContext(It),{pathname:o}=Nn(),i=JSON.stringify(_g(r)),s=C.useRef(!1);return zg(()=>{s.current=!0}),C.useCallback((l,c={})=>{if(vt(s.current,Dg),!s.current)return;if(typeof l=="number"){n.go(l);return}let u=Vg(l,JSON.parse(i),o,c.relative==="path");e==null&&t!=="/"&&(u.pathname=u.pathname==="/"?t:Rt([t,u.pathname])),(c.replace?n.replace:n.push)(u,c.state,c)},[t,n,i,o,e])}C.createContext(null);function Jr(e,{relative:t}={}){let{matches:n}=C.useContext(It),{pathname:r}=Nn(),o=JSON.stringify(_g(n));return C.useMemo(()=>Vg(e,JSON.parse(o),r,t==="path"),[e,o,r,t])}function ZP(e,t){return Mg(e,t)}function Mg(e,t,n,r,o){le(Qr(),"useRoutes() may be used only in the context of a component.");let{navigator:i}=C.useContext(bt),{matches:s}=C.useContext(It),a=s[s.length-1],l=a?a.params:{},c=a?a.pathname:"/",u=a?a.pathnameBase:"/",h=a&&a.route;{let y=h&&h.path||"";$g(c,!h||y.endsWith("*")||y.endsWith("*?"),`You rendered descendant (or called \`useRoutes()\`) at "${c}" (under ) but the parent route path has no trailing "*". This means if you navigate deeper, the parent won't match anymore and therefore the child routes will never render. + */var Ig="popstate";function wP(e={}){function t(r,o){let{pathname:i,search:s,hash:a}=r.location;return el("",{pathname:i,search:s,hash:a},o.state&&o.state.usr||null,o.state&&o.state.key||"default")}function n(r,o){return typeof o=="string"?o:Yr(o)}return EP(t,n,null,e)}function le(e,t){if(e===!1||e===null||typeof e>"u")throw new Error(t)}function bt(e,t){if(!e){typeof console<"u"&&console.warn(t);try{throw new Error(t)}catch{}}}function SP(){return Math.random().toString(36).substring(2,10)}function Tg(e,t){return{usr:e.state,key:e.key,idx:t}}function el(e,t,n=null,r){return{pathname:typeof e=="string"?e:e.pathname,search:"",hash:"",...typeof t=="string"?cr(t):t,state:n,key:t&&t.key||r||SP()}}function Yr({pathname:e="/",search:t="",hash:n=""}){return t&&t!=="?"&&(e+=t.charAt(0)==="?"?t:"?"+t),n&&n!=="#"&&(e+=n.charAt(0)==="#"?n:"#"+n),e}function cr(e){let t={};if(e){let n=e.indexOf("#");n>=0&&(t.hash=e.substring(n),e=e.substring(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substring(r),e=e.substring(0,r)),e&&(t.pathname=e)}return t}function EP(e,t,n,r={}){let{window:o=document.defaultView,v5Compat:i=!1}=r,s=o.history,a="POP",l=null,c=u();c==null&&(c=0,s.replaceState({...s.state,idx:c},""));function u(){return(s.state||{idx:null}).idx}function h(){a="POP";let v=u(),b=v==null?null:v-c;c=v,l&&l({action:a,location:p.location,delta:b})}function d(v,b){a="PUSH";let y=el(p.location,v,b);c=u()+1;let x=Tg(y,c),w=p.createHref(y);try{s.pushState(x,"",w)}catch(k){if(k instanceof DOMException&&k.name==="DataCloneError")throw k;o.location.assign(w)}i&&l&&l({action:a,location:p.location,delta:1})}function m(v,b){a="REPLACE";let y=el(p.location,v,b);c=u();let x=Tg(y,c),w=p.createHref(y);s.replaceState(x,"",w),i&&l&&l({action:a,location:p.location,delta:0})}function f(v){return OP(v)}let p={get action(){return a},get location(){return e(o,s)},listen(v){if(l)throw new Error("A history only accepts one active listener");return o.addEventListener(Ig,h),l=v,()=>{o.removeEventListener(Ig,h),l=null}},createHref(v){return t(o,v)},createURL:f,encodeLocation(v){let b=f(v);return{pathname:b.pathname,search:b.search,hash:b.hash}},push:d,replace:m,go(v){return s.go(v)}};return p}function OP(e,t=!1){let n="http://localhost";typeof window<"u"&&(n=window.location.origin!=="null"?window.location.origin:window.location.href),le(n,"No window.location.(origin|href) available to create URL");let r=typeof e=="string"?e:Yr(e);return r=r.replace(/ $/,"%20"),!t&&r.startsWith("//")&&(r=n+r),new URL(r,n)}function Ng(e,t,n="/"){return PP(e,t,n,!1)}function PP(e,t,n,r){let o=typeof t=="string"?cr(t):t,i=Rt(o.pathname||"/",n);if(i==null)return null;let s=Ag(e);RP(s);let a=null;for(let l=0;a==null&&l{let u={relativePath:c===void 0?s.path||"":c,caseSensitive:s.caseSensitive===!0,childrenIndex:a,route:s};if(u.relativePath.startsWith("/")){if(!u.relativePath.startsWith(r)&&l)return;le(u.relativePath.startsWith(r),`Absolute route path "${u.relativePath}" nested under path "${r}" is not valid. An absolute child route path must start with the combined path of all its parent routes.`),u.relativePath=u.relativePath.slice(r.length)}let h=It([r,u.relativePath]),d=n.concat(u);s.children&&s.children.length>0&&(le(s.index!==!0,`Index routes must not have child routes. Please remove all child routes from route path "${h}".`),Ag(s.children,t,d,h,l)),!(s.path==null&&!s.index)&&t.push({path:h,score:LP(h,s.index),routesMeta:d})};return e.forEach((s,a)=>{if(s.path===""||!s.path?.includes("?"))i(s,a);else for(let l of _g(s.path))i(s,a,!0,l)}),t}function _g(e){let t=e.split("/");if(t.length===0)return[];let[n,...r]=t,o=n.endsWith("?"),i=n.replace(/\?$/,"");if(r.length===0)return o?[i,""]:[i];let s=_g(r.join("/")),a=[];return a.push(...s.map(l=>l===""?i:[i,l].join("/"))),o&&a.push(...s),a.map(l=>e.startsWith("/")&&l===""?"/":l)}function RP(e){e.sort((t,n)=>t.score!==n.score?n.score-t.score:FP(t.routesMeta.map(r=>r.childrenIndex),n.routesMeta.map(r=>r.childrenIndex)))}var IP=/^:[\w-]+$/,TP=3,NP=2,AP=1,_P=10,VP=-2,Vg=e=>e==="*";function LP(e,t){let n=e.split("/"),r=n.length;return n.some(Vg)&&(r+=VP),t&&(r+=NP),n.filter(o=>!Vg(o)).reduce((o,i)=>o+(IP.test(i)?TP:i===""?AP:_P),r)}function FP(e,t){return e.length===t.length&&e.slice(0,-1).every((r,o)=>r===t[o])?e[e.length-1]-t[t.length-1]:0}function DP(e,t,n=!1){let{routesMeta:r}=e,o={},i="/",s=[];for(let a=0;a{if(u==="*"){let f=a[d]||"";s=i.slice(0,i.length-f.length).replace(/(.)\/+$/,"$1")}const m=a[d];return h&&!m?c[u]=void 0:c[u]=(m||"").replace(/%2F/g,"/"),c},{}),pathname:i,pathnameBase:s,pattern:e}}function zP(e,t=!1,n=!0){bt(e==="*"||!e.endsWith("*")||e.endsWith("/*"),`Route path "${e}" will be treated as if it were "${e.replace(/\*$/,"/*")}" because the \`*\` character must always follow a \`/\` in the pattern. To get rid of this warning, please change the route path to "${e.replace(/\*$/,"/*")}".`);let r=[],o="^"+e.replace(/\/*\*?$/,"").replace(/^\/*/,"/").replace(/[\\.*+^${}|()[\]]/g,"\\$&").replace(/\/:([\w-]+)(\?)?/g,(s,a,l)=>(r.push({paramName:a,isOptional:l!=null}),l?"/?([^\\/]+)?":"/([^\\/]+)")).replace(/\/([\w-]+)\?(\/|$)/g,"(/$1)?$2");return e.endsWith("*")?(r.push({paramName:"*"}),o+=e==="*"||e==="/*"?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?o+="\\/*$":e!==""&&e!=="/"&&(o+="(?:(?=\\/|$))"),[new RegExp(o,t?void 0:"i"),r]}function MP(e){try{return e.split("/").map(t=>decodeURIComponent(t).replace(/\//g,"%2F")).join("/")}catch(t){return bt(!1,`The URL path "${e}" could not be decoded because it is a malformed URL segment. This is probably due to a bad percent encoding (${t}).`),e}}function Rt(e,t){if(t==="/")return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&r!=="/"?null:e.slice(n)||"/"}function jP(e,t="/"){let{pathname:n,search:r="",hash:o=""}=typeof e=="string"?cr(e):e;return{pathname:n?n.startsWith("/")?n:$P(n,t):t,search:HP(r),hash:UP(o)}}function $P(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach(o=>{o===".."?n.length>1&&n.pop():o!=="."&&n.push(o)}),n.length>1?n.join("/"):"/"}function tl(e,t,n,r){return`Cannot include a '${e}' character in a manually specified \`to.${t}\` field [${JSON.stringify(r)}]. Please separate it out to the \`to.${n}\` field. Alternatively you may provide the full path as a string in and the router will parse it for you.`}function BP(e){return e.filter((t,n)=>n===0||t.route.path&&t.route.path.length>0)}function Lg(e){let t=BP(e);return t.map((n,r)=>r===t.length-1?n.pathname:n.pathnameBase)}function Fg(e,t,n,r=!1){let o;typeof e=="string"?o=cr(e):(o={...e},le(!o.pathname||!o.pathname.includes("?"),tl("?","pathname","search",o)),le(!o.pathname||!o.pathname.includes("#"),tl("#","pathname","hash",o)),le(!o.search||!o.search.includes("#"),tl("#","search","hash",o)));let i=e===""||o.pathname==="",s=i?"/":o.pathname,a;if(s==null)a=n;else{let h=t.length-1;if(!r&&s.startsWith("..")){let d=s.split("/");for(;d[0]==="..";)d.shift(),h-=1;o.pathname=d.join("/")}a=h>=0?t[h]:"/"}let l=jP(o,a),c=s&&s!=="/"&&s.endsWith("/"),u=(i||s===".")&&n.endsWith("/");return!l.pathname.endsWith("/")&&(c||u)&&(l.pathname+="/"),l}var It=e=>e.join("/").replace(/\/\/+/g,"/"),WP=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),HP=e=>!e||e==="?"?"":e.startsWith("?")?e:"?"+e,UP=e=>!e||e==="#"?"":e.startsWith("#")?e:"#"+e;function GP(e){return e!=null&&typeof e.status=="number"&&typeof e.statusText=="string"&&typeof e.internal=="boolean"&&"data"in e}var Dg=["POST","PUT","PATCH","DELETE"];new Set(Dg);var qP=["GET",...Dg];new Set(qP);var ur=C.createContext(null);ur.displayName="DataRouter";var Ni=C.createContext(null);Ni.displayName="DataRouterState",C.createContext(!1);var zg=C.createContext({isTransitioning:!1});zg.displayName="ViewTransition";var KP=C.createContext(new Map);KP.displayName="Fetchers";var YP=C.createContext(null);YP.displayName="Await";var yt=C.createContext(null);yt.displayName="Navigation";var Xr=C.createContext(null);Xr.displayName="Location";var Tt=C.createContext({outlet:null,matches:[],isDataRoute:!1});Tt.displayName="Route";var nl=C.createContext(null);nl.displayName="RouteError";function XP(e,{relative:t}={}){le(Qr(),"useHref() may be used only in the context of a component.");let{basename:n,navigator:r}=C.useContext(yt),{hash:o,pathname:i,search:s}=Jr(e,{relative:t}),a=i;return n!=="/"&&(a=i==="/"?n:It([n,i])),r.createHref({pathname:a,search:s,hash:o})}function Qr(){return C.useContext(Xr)!=null}function Nn(){return le(Qr(),"useLocation() may be used only in the context of a component."),C.useContext(Xr).location}var Mg="You should call navigate() in a React.useEffect(), not when your component is first rendered.";function jg(e){C.useContext(yt).static||C.useLayoutEffect(e)}function QP(){let{isDataRoute:e}=C.useContext(Tt);return e?uR():JP()}function JP(){le(Qr(),"useNavigate() may be used only in the context of a component.");let e=C.useContext(ur),{basename:t,navigator:n}=C.useContext(yt),{matches:r}=C.useContext(Tt),{pathname:o}=Nn(),i=JSON.stringify(Lg(r)),s=C.useRef(!1);return jg(()=>{s.current=!0}),C.useCallback((l,c={})=>{if(bt(s.current,Mg),!s.current)return;if(typeof l=="number"){n.go(l);return}let u=Fg(l,JSON.parse(i),o,c.relative==="path");e==null&&t!=="/"&&(u.pathname=u.pathname==="/"?t:It([t,u.pathname])),(c.replace?n.replace:n.push)(u,c.state,c)},[t,n,i,o,e])}C.createContext(null);function Jr(e,{relative:t}={}){let{matches:n}=C.useContext(Tt),{pathname:r}=Nn(),o=JSON.stringify(Lg(n));return C.useMemo(()=>Fg(e,JSON.parse(o),r,t==="path"),[e,o,r,t])}function ZP(e,t){return $g(e,t)}function $g(e,t,n,r,o){le(Qr(),"useRoutes() may be used only in the context of a component.");let{navigator:i}=C.useContext(yt),{matches:s}=C.useContext(Tt),a=s[s.length-1],l=a?a.params:{},c=a?a.pathname:"/",u=a?a.pathnameBase:"/",h=a&&a.route;{let y=h&&h.path||"";Wg(c,!h||y.endsWith("*")||y.endsWith("*?"),`You rendered descendant (or called \`useRoutes()\`) at "${c}" (under ) but the parent route path has no trailing "*". This means if you navigate deeper, the parent won't match anymore and therefore the child routes will never render. -Please change the parent to .`)}let d=Nn(),m;if(t){let y=typeof t=="string"?cr(t):t;le(u==="/"||y.pathname?.startsWith(u),`When overriding the location using \`\` or \`useRoutes(routes, location)\`, the location pathname must begin with the portion of the URL pathname that was matched by all parent routes. The current pathname base is "${u}" but pathname "${y.pathname}" was given in the \`location\` prop.`),m=y}else m=d;let f=m.pathname||"/",p=f;if(u!=="/"){let y=u.replace(/^\//,"").split("/");p="/"+f.replace(/^\//,"").split("/").slice(y.length).join("/")}let v=Ig(e,{pathname:p});vt(h||v!=null,`No routes matched location "${m.pathname}${m.search}${m.hash}" `),vt(v==null||v[v.length-1].route.element!==void 0||v[v.length-1].route.Component!==void 0||v[v.length-1].route.lazy!==void 0,`Matched leaf route at location "${m.pathname}${m.search}${m.hash}" does not have an element or Component. This means it will render an with a null value by default resulting in an "empty" page.`);let b=oR(v&&v.map(y=>Object.assign({},y,{params:Object.assign({},l,y.params),pathname:Rt([u,i.encodeLocation?i.encodeLocation(y.pathname.replace(/\?/g,"%3F").replace(/#/g,"%23")).pathname:y.pathname]),pathnameBase:y.pathnameBase==="/"?u:Rt([u,i.encodeLocation?i.encodeLocation(y.pathnameBase.replace(/\?/g,"%3F").replace(/#/g,"%23")).pathname:y.pathnameBase])})),s,n,r,o);return t&&b?C.createElement(Xr.Provider,{value:{location:{pathname:"/",search:"",hash:"",state:null,key:"default",...m},navigationType:"POP"}},b):b}function eR(){let e=cR(),t=GP(e)?`${e.status} ${e.statusText}`:e instanceof Error?e.message:JSON.stringify(e),n=e instanceof Error?e.stack:null,r="rgba(200,200,200, 0.5)",o={padding:"0.5rem",backgroundColor:r},i={padding:"2px 4px",backgroundColor:r},s=null;return console.error("Error handled by React Router default ErrorBoundary:",e),s=C.createElement(C.Fragment,null,C.createElement("p",null,"💿 Hey developer 👋"),C.createElement("p",null,"You can provide a way better UX than this when your app throws errors by providing your own ",C.createElement("code",{style:i},"ErrorBoundary")," or"," ",C.createElement("code",{style:i},"errorElement")," prop on your route.")),C.createElement(C.Fragment,null,C.createElement("h2",null,"Unexpected Application Error!"),C.createElement("h3",{style:{fontStyle:"italic"}},t),n?C.createElement("pre",{style:o},n):null,s)}var tR=C.createElement(eR,null),nR=class extends C.Component{constructor(e){super(e),this.state={location:e.location,revalidation:e.revalidation,error:e.error}}static getDerivedStateFromError(e){return{error:e}}static getDerivedStateFromProps(e,t){return t.location!==e.location||t.revalidation!=="idle"&&e.revalidation==="idle"?{error:e.error,location:e.location,revalidation:e.revalidation}:{error:e.error!==void 0?e.error:t.error,location:t.location,revalidation:e.revalidation||t.revalidation}}componentDidCatch(e,t){this.props.unstable_onError?this.props.unstable_onError(e,t):console.error("React Router caught the following error during render",e)}render(){return this.state.error!==void 0?C.createElement(It.Provider,{value:this.props.routeContext},C.createElement(nl.Provider,{value:this.state.error,children:this.props.component})):this.props.children}};function rR({routeContext:e,match:t,children:n}){let r=C.useContext(ur);return r&&r.static&&r.staticContext&&(t.route.errorElement||t.route.ErrorBoundary)&&(r.staticContext._deepestRenderedBoundaryId=t.route.id),C.createElement(It.Provider,{value:e},n)}function oR(e,t=[],n=null,r=null,o=null){if(e==null){if(!n)return null;if(n.errors)e=n.matches;else if(t.length===0&&!n.initialized&&n.matches.length>0)e=n.matches;else return null}let i=e,s=n?.errors;if(s!=null){let c=i.findIndex(u=>u.route.id&&s?.[u.route.id]!==void 0);le(c>=0,`Could not find a matching route for errors on route IDs: ${Object.keys(s).join(",")}`),i=i.slice(0,Math.min(i.length,c+1))}let a=!1,l=-1;if(n)for(let c=0;c=0?i=i.slice(0,l+1):i=[i[0]];break}}}return i.reduceRight((c,u,h)=>{let d,m=!1,f=null,p=null;n&&(d=s&&u.route.id?s[u.route.id]:void 0,f=u.route.errorElement||tR,a&&(l<0&&h===0?($g("route-fallback",!1,"No `HydrateFallback` element provided to render during initial hydration"),m=!0,p=null):l===h&&(m=!0,p=u.route.hydrateFallbackElement||null)));let v=t.concat(i.slice(0,h+1)),b=()=>{let y;return d?y=f:m?y=p:u.route.Component?y=C.createElement(u.route.Component,null):u.route.element?y=u.route.element:y=c,C.createElement(rR,{match:u,routeContext:{outlet:c,matches:v,isDataRoute:n!=null},children:y})};return n&&(u.route.ErrorBoundary||u.route.errorElement||h===0)?C.createElement(nR,{location:n.location,revalidation:n.revalidation,component:f,error:d,children:b(),routeContext:{outlet:null,matches:v,isDataRoute:!0},unstable_onError:r}):b()},null)}function rl(e){return`${e} must be used within a data router. See https://reactrouter.com/en/main/routers/picking-a-router.`}function iR(e){let t=C.useContext(ur);return le(t,rl(e)),t}function sR(e){let t=C.useContext(Ni);return le(t,rl(e)),t}function aR(e){let t=C.useContext(It);return le(t,rl(e)),t}function ol(e){let t=aR(e),n=t.matches[t.matches.length-1];return le(n.route.id,`${e} can only be used on routes that contain a unique "id"`),n.route.id}function lR(){return ol("useRouteId")}function cR(){let e=C.useContext(nl),t=sR("useRouteError"),n=ol("useRouteError");return e!==void 0?e:t.errors?.[n]}function uR(){let{router:e}=iR("useNavigate"),t=ol("useNavigate"),n=C.useRef(!1);return zg(()=>{n.current=!0}),C.useCallback(async(o,i={})=>{vt(n.current,Dg),n.current&&(typeof o=="number"?e.navigate(o):await e.navigate(o,{fromRouteId:t,...i}))},[e,t])}var jg={};function $g(e,t,n){!t&&!jg[e]&&(jg[e]=!0,vt(!1,n))}C.memo(dR);function dR({routes:e,future:t,state:n,unstable_onError:r}){return Mg(e,void 0,n,r,t)}function il(e){le(!1,"A is only ever to be used as the child of element, never rendered directly. Please wrap your in a .")}function hR({basename:e="/",children:t=null,location:n,navigationType:r="POP",navigator:o,static:i=!1}){le(!Qr(),"You cannot render a inside another . You should never have more than one in your app.");let s=e.replace(/^\/*/,"/"),a=C.useMemo(()=>({basename:s,navigator:o,static:i,future:{}}),[s,o,i]);typeof n=="string"&&(n=cr(n));let{pathname:l="/",search:c="",hash:u="",state:h=null,key:d="default"}=n,m=C.useMemo(()=>{let f=Pt(l,s);return f==null?null:{location:{pathname:f,search:c,hash:u,state:h,key:d},navigationType:r}},[s,l,c,u,h,d,r]);return vt(m!=null,` is not able to match the URL "${l}${c}${u}" because it does not start with the basename, so the won't render anything.`),m==null?null:C.createElement(bt.Provider,{value:a},C.createElement(Xr.Provider,{children:t,value:m}))}function fR({children:e,location:t}){return ZP(sl(e),t)}function sl(e,t=[]){let n=[];return C.Children.forEach(e,(r,o)=>{if(!C.isValidElement(r))return;let i=[...t,o];if(r.type===C.Fragment){n.push.apply(n,sl(r.props.children,i));return}le(r.type===il,`[${typeof r.type=="string"?r.type:r.type.name}] is not a component. All component children of must be a or `),le(!r.props.index||!r.props.children,"An index route cannot have child routes.");let s={id:r.props.id||i.join("-"),caseSensitive:r.props.caseSensitive,element:r.props.element,Component:r.props.Component,index:r.props.index,path:r.props.path,middleware:r.props.middleware,loader:r.props.loader,action:r.props.action,hydrateFallbackElement:r.props.hydrateFallbackElement,HydrateFallback:r.props.HydrateFallback,errorElement:r.props.errorElement,ErrorBoundary:r.props.ErrorBoundary,hasErrorBoundary:r.props.hasErrorBoundary===!0||r.props.ErrorBoundary!=null||r.props.errorElement!=null,shouldRevalidate:r.props.shouldRevalidate,handle:r.props.handle,lazy:r.props.lazy};r.props.children&&(s.children=sl(r.props.children,i)),n.push(s)}),n}var Ai="get",_i="application/x-www-form-urlencoded";function Vi(e){return e!=null&&typeof e.tagName=="string"}function gR(e){return Vi(e)&&e.tagName.toLowerCase()==="button"}function pR(e){return Vi(e)&&e.tagName.toLowerCase()==="form"}function mR(e){return Vi(e)&&e.tagName.toLowerCase()==="input"}function vR(e){return!!(e.metaKey||e.altKey||e.ctrlKey||e.shiftKey)}function bR(e,t){return e.button===0&&(!t||t==="_self")&&!vR(e)}var Fi=null;function yR(){if(Fi===null)try{new FormData(document.createElement("form"),0),Fi=!1}catch{Fi=!0}return Fi}var xR=new Set(["application/x-www-form-urlencoded","multipart/form-data","text/plain"]);function al(e){return e!=null&&!xR.has(e)?(vt(!1,`"${e}" is not a valid \`encType\` for \`\`/\`\` and will default to "${_i}"`),null):e}function kR(e,t){let n,r,o,i,s;if(pR(e)){let a=e.getAttribute("action");r=a?Pt(a,t):null,n=e.getAttribute("method")||Ai,o=al(e.getAttribute("enctype"))||_i,i=new FormData(e)}else if(gR(e)||mR(e)&&(e.type==="submit"||e.type==="image")){let a=e.form;if(a==null)throw new Error('Cannot submit a