diff --git a/.gitignore b/.gitignore index 11cabc8..2ea79bb 100644 --- a/.gitignore +++ b/.gitignore @@ -67,3 +67,6 @@ sample/dagster_project/storage/ sample/dagster_project/logs/ sample/dagster_project/history/ sample/dagster_project/schedules/ +tests/temp/ + +debug_runs/ diff --git a/Makefile b/Makefile index 2987092..4b2ac8a 100644 --- a/Makefile +++ b/Makefile @@ -6,11 +6,19 @@ PYTHON_VENV_NAME := ".venv" VENV_NAME := .venv PYTHON := python -PYTHON_CMD := $(CURDIR)/$(VENV_NAME)/bin/python -SQLMESH_CMD := $(CURDIR)/$(VENV_NAME)/bin/sqlmesh -UV_CMD := uv -ACTIVATE := source $(CURDIR)/$(VENV_NAME)/bin/activate -DEACTIVATE := deactivate +ifeq ($(OS),Windows_NT) + PYTHON_CMD := $(CURDIR)/$(VENV_NAME)/Scripts/python + SQLMESH_CMD := $(CURDIR)/$(VENV_NAME)/Scripts/sqlmesh + UV_CMD := "$(subst \,/,$(USERPROFILE))/.local/bin/uv.exe" + ACTIVATE := source $(CURDIR)/$(VENV_NAME)/Scripts/activate + DEACTIVATE := source $(CURDIR)/$(VENV_NAME)/Scripts/deactivate +else + PYTHON_CMD := $(CURDIR)/$(VENV_NAME)/bin/python + SQLMESH_CMD := $(CURDIR)/$(VENV_NAME)/bin/sqlmesh + UV_CMD := uv + ACTIVATE := source $(CURDIR)/$(VENV_NAME)/bin/activate + DEACTIVATE := deactivate +endif init-python: @if [ ! -d "$(PYTHON_VENV_NAME)" ]; then \ @@ -49,15 +57,18 @@ clean-dagster: rm -rf sample/dagster_project/storage sample/dagster_project/logs sample/dagster_project/history clean-db: - $(PYTHON_CMD) -c "import duckdb; conn = duckdb.connect('db.db'); [conn.execute(cmd[0]) for cmd in conn.execute(\"\"\"SELECT 'DROP TABLE ' || table_schema || '.' || table_name || ' CASCADE;' as drop_cmd FROM information_schema.tables WHERE table_schema != 'sources' AND table_schema != 'information_schema' AND table_type = 'BASE TABLE'\"\"\").fetchall()]; [conn.execute(cmd[0]) for cmd in conn.execute(\"\"\"SELECT 'DROP VIEW ' || table_schema || '.' || table_name || ' CASCADE;' as drop_cmd FROM information_schema.tables WHERE table_schema != 'sources' AND table_schema != 'information_schema' AND table_type = 'VIEW'\"\"\").fetchall()]; conn.close()" + "$(PYTHON_CMD)" -c "import duckdb; conn = duckdb.connect('db.db'); [conn.execute(cmd[0]) for cmd in conn.execute(\"\"\"SELECT 'DROP TABLE ' || table_schema || '.' || table_name || ' CASCADE;' as drop_cmd FROM information_schema.tables WHERE table_schema != 'sources' AND table_schema != 'information_schema' AND table_type = 'BASE TABLE'\"\"").fetchall()]; [conn.execute(cmd[0]) for cmd in conn.execute(\"\"\"SELECT 'DROP VIEW ' || table_schema || '.' || table_name || ' CASCADE;' as drop_cmd FROM information_schema.tables WHERE table_schema != 'sources' AND table_schema != 'information_schema' AND table_type = 'VIEW'\"\"").fetchall()]; conn.close()" dagster-dev: clean-dagster @DAGSTER_HOME="$(subst \,/,$(CURDIR))/sample/dagster_project" "$(PYTHON_CMD)" -m dagster dev -f sample/dagster_project/definitions.py -h 0.0.0.0 dev: dagster-dev # Alias for dagster-dev -dagster-materialize: - $(PYTHON_CMD) -m dagster asset materialize -f sample/dagster_project/definitions.py --select '*' +mat-test-source: + "$(PYTHON_CMD)" -m dagster asset materialize \ + -f "$(CURDIR)/sample/dagster_project/definitions.py" \ + --select "test_source" \ + --config-json '{"resources": {"sqlmesh": {"config": {"config": {"gateway": "local", "path": "C:\\\\Users\\\\kevin\\\\git_repos\\\\dagster-sqlmesh - Copy\\\\sample\\\\sqlmesh_project"}, "plan_options_override": {"skip_backfill": false}}}}}' sqlmesh-plan: cd sample/sqlmesh_project && $(SQLMESH_CMD) plan diff --git a/dagster_sqlmesh/conftest.py b/dagster_sqlmesh/conftest.py index 9556b09..44a7fc9 100644 --- a/dagster_sqlmesh/conftest.py +++ b/dagster_sqlmesh/conftest.py @@ -1,10 +1,20 @@ +import datetime as dt +import io +import json import logging import os +import queue import shutil +import subprocess import sys import tempfile +import threading import typing as t +from contextlib import contextmanager +from dataclasses import dataclass, field +import duckdb +import polars import pytest from sqlmesh.core.config import ( Config as SQLMeshConfig, @@ -12,9 +22,13 @@ GatewayConfig, ModelDefaultsConfig, ) +from sqlmesh.core.console import get_console +from sqlmesh.utils.date import TimeLike from dagster_sqlmesh.config import SQLMeshContextConfig -from dagster_sqlmesh.testing import SQLMeshTestContext +from dagster_sqlmesh.controller.base import PlanOptions, RunOptions +from dagster_sqlmesh.controller.dagster import DagsterSQLMeshController +from dagster_sqlmesh.events import ConsoleRecorder logger = logging.getLogger(__name__) @@ -28,18 +42,401 @@ def setup_debug_logging_for_tests() -> None: @pytest.fixture -def sample_sqlmesh_project() -> t.Iterator[str]: - """Creates a temporary sqlmesh project by copying the sample project""" +def sample_project_root() -> t.Iterator[str]: + """Creates a temporary project directory containing both SQLMesh and Dagster projects""" with tempfile.TemporaryDirectory() as tmp_dir: - project_dir = shutil.copytree( - "sample/sqlmesh_project", os.path.join(tmp_dir, "project") + project_dir = shutil.copytree("sample", tmp_dir, dirs_exist_ok=True) + + yield project_dir + + # Create debug directory with timestamp AFTER test run + debug_dir = os.path.join( + os.path.dirname(os.path.dirname(__file__)), "debug_runs" + ) + os.makedirs(debug_dir, exist_ok=True) + timestamp = dt.datetime.now().strftime("%Y%m%d_%H%M%S") + run_debug_dir = os.path.join(debug_dir, f"run_{timestamp}") + + # Copy contents to debug directory + try: + shutil.copytree(tmp_dir, run_debug_dir, dirs_exist_ok=True) + logger.info( + f"Copied final test project contents to {run_debug_dir} for debugging" + ) + except FileNotFoundError: + logger.warning( + f"Temporary directory {tmp_dir} not found during cleanup copy." + ) + except Exception as e: + logger.error( + f"Error copying temporary directory {tmp_dir} to {run_debug_dir}: {e}" + ) + + +@pytest.fixture +def sample_sqlmesh_project(sample_project_root: str) -> t.Iterator[str]: + """Returns path to the SQLMesh project within the sample project""" + sqlmesh_project_dir = os.path.join(sample_project_root, "sqlmesh_project") + db_path = os.path.join(sqlmesh_project_dir, "db.db") + if os.path.exists(db_path): + os.remove(db_path) + yield sqlmesh_project_dir + + +@pytest.fixture +def sample_dagster_project(sample_project_root: str) -> t.Iterator[str]: + """Returns path to the Dagster project within the sample project""" + dagster_project_dir = os.path.join(sample_project_root, "dagster_project") + sqlmesh_project_dir = os.path.join(sample_project_root, "sqlmesh_project") + + db_path = os.path.join(sqlmesh_project_dir, "db.db") + if os.path.exists(db_path): + os.remove(db_path) + + yield dagster_project_dir + + +@dataclass +class SQLMeshTestContext: + """A test context for running SQLMesh""" + + db_path: str + context_config: SQLMeshContextConfig + project_path: str + + # Internal state for backup/restore + _backed_up_files: set[str] = field(default_factory=set, init=False) + + def create_controller( + self, enable_debug_console: bool = False + ) -> DagsterSQLMeshController: + console = None + if enable_debug_console: + console = get_console() + return DagsterSQLMeshController.setup_with_config( + self.context_config, debug_console=console + ) + + def get_model_path(self, model_name: str) -> str: + """Get the full path to a model file. + + Args: + model_name: The name of the model file (e.g. 'staging_model_1.sql') + + Returns: + str: Full path to the model file + """ + # Common model directories to search + model_dirs = [ + os.path.join(self.project_path, "models"), + os.path.join(self.project_path, "models", "staging"), + os.path.join(self.project_path, "models", "intermediate"), + os.path.join(self.project_path, "models", "mart"), + ] + + for directory in model_dirs: + if not os.path.exists(directory): + continue + for root, _, files in os.walk(directory): + if model_name in files: + return os.path.join(root, model_name) + + raise FileNotFoundError(f"Model file {model_name} not found in project") + + def backup_model_file(self, model_name: str) -> None: + """Create a backup of a model file. + + Args: + model_name: The name of the model file to backup + """ + model_path = self.get_model_path(model_name) + backup_path = f"{model_path}.bak" + shutil.copy2(model_path, backup_path) + + def restore_model_file(self, model_name: str) -> None: + """Restore a model file from its backup. + + Args: + model_name: The name of the model file to restore + """ + model_path = self.get_model_path(model_name) + backup_path = f"{model_path}.bak" + if os.path.exists(backup_path): + shutil.copy2(backup_path, model_path) + os.remove(backup_path) + + def modify_model_file(self, model_name: str, new_content: str) -> None: + """Modify a model file with new content, creating a backup first. + + Args: + model_name: The name of the model file to modify + new_content: The new content for the model file + """ + model_path = self.get_model_path(model_name) + if not hasattr(self, "_backed_up_files"): + self._backed_up_files: set[str] = set() + + # Create backup if not already done + if model_name not in self._backed_up_files: + self.backup_model_file(model_name) + self._backed_up_files.add(model_name) + + # Write new content + with open(model_path, "w") as f: + f.write(new_content) + + def cleanup_modified_files(self) -> None: + """Restore all modified model files from their backups.""" + if hasattr(self, "_backed_up_files"): + for model_name in self._backed_up_files: + self.restore_model_file(model_name) + self._backed_up_files.clear() + + def save_sqlmesh_debug_state(self, name_suffix: str = "manual_save") -> str: + """Saves the current state of the SQLMesh project to the debug directory. + + Copies the contents of the SQLMesh project directory (self.project_path) + to a timestamped sub-directory within the 'debug_runs' folder. + + Args: + name_suffix: An optional suffix to append to the debug directory name + to distinguish this save point (e.g., 'before_change', + 'after_plan'). Defaults to 'manual_save'. + + Returns: + The path to the created debug state directory. + """ + debug_dir_base = os.path.join( + os.path.dirname(self.project_path), "..", "debug_runs" ) - db_path = os.path.join(project_dir, "db.db") - if os.path.exists(db_path): - os.remove(os.path.join(project_dir, "db.db")) + os.makedirs(debug_dir_base, exist_ok=True) + timestamp = dt.datetime.now().strftime("%Y%m%d_%H%M%S") + run_debug_dir = os.path.join( + debug_dir_base, f"sqlmesh_state_{timestamp}_{name_suffix}" + ) + + try: + shutil.copytree(self.project_path, run_debug_dir, dirs_exist_ok=True) + logger.info(f"Saved SQLMesh project debug state to {run_debug_dir}") + return run_debug_dir + except Exception as e: + logger.error( + f"Error saving SQLMesh project debug state to {run_debug_dir}: {e}" + ) + raise + + def query(self, *args: t.Any, return_df: bool = False, **kwargs: t.Any) -> t.Any: + """Execute a query against the test database. + + Args: + *args: Arguments to pass to DuckDB's sql method + **kwargs: Keyword arguments to pass to DuckDB's sql method + + Returns: + For SELECT queries: Query results as a list of tuples + For DDL/DML queries: None + """ + with duckdb.connect(self.db_path) as conn: + result = conn.sql(*args, **kwargs) + # Only try to fetch results if it's a SELECT query + if result is not None: + if return_df: + return result.to_df() + return result.fetchall() + return None + + def initialize_test_source(self) -> None: + conn = duckdb.connect(self.db_path) + conn.sql( + """ + CREATE SCHEMA sources; + """ + ) + conn.sql( + """ + CREATE TABLE sources.test_source (id INTEGER, name VARCHAR); + """ + ) + conn.sql( + """ + INSERT INTO sources.test_source (id, name) + VALUES (1, 'abc'), (2, 'def'); + """ + ) + conn.close() + + def append_to_test_source(self, df: polars.DataFrame): + logger.debug("appending data to the test source") + conn = duckdb.connect(self.db_path) + conn.sql( + """ + INSERT INTO sources.test_source + SELECT * FROM df + """ + ) + + def plan( + self, + *, + environment: str, + execution_time: TimeLike | None = None, + enable_debug_console: bool = False, + start: TimeLike | None = None, + end: TimeLike | None = None, + plan_options: PlanOptions | None = None, + restate_models: list[str] | None = None, + ) -> None: + """Runs plan and run on SQLMesh with the given configuration and record all of the generated events. + + Args: + environment (str): The environment to run SQLMesh in. + execution_time (TimeLike, optional): The execution timestamp for the run. Defaults to None. + enable_debug_console (bool, optional): Flag to enable debug console. Defaults to False. + start (TimeLike, optional): Start time for the run interval. Defaults to None. + end (TimeLike, optional): End time for the run interval. Defaults to None. + plan_options (PlanOptions, optional): Plan options for the plan. Defaults to None. + restate_models (List[str], optional): List of models to restate. Defaults to None. + + Returns: + None: The function records events to a debug console but doesn't return anything. + + Note: + TimeLike can be any time-like object that SQLMesh accepts (datetime, str, etc.). + The function creates a controller and recorder to capture all SQLMesh events during execution. + """ + controller = self.create_controller(enable_debug_console=enable_debug_console) + recorder = ConsoleRecorder() + # controller.add_event_handler(ConsoleRecorder()) + if plan_options is None: + plan_options = PlanOptions( + enable_preview=True, + ) + + if execution_time: + plan_options["execution_time"] = execution_time + if restate_models: + plan_options["restate_models"] = restate_models + if start: + plan_options["start"] = start + if end: + plan_options["end"] = end + + for event in controller.plan( + environment, + plan_options=plan_options, + categorizer=None, + default_catalog=None, + ): + recorder(event) + + def run( + self, + *, + environment: str, + execution_time: TimeLike | None = None, + enable_debug_console: bool = False, + start: TimeLike | None = None, + end: TimeLike | None = None, + run_options: RunOptions | None = None, + restate_models: list[str] | None = None, + ) -> None: + """Runs plan and run on SQLMesh with the given configuration and record all of the generated events. - # Initialize the "source" data - yield str(project_dir) + Args: + environment (str): The environment to run SQLMesh in. + execution_time (TimeLike, optional): The execution timestamp for the run. Defaults to None. + enable_debug_console (bool, optional): Flag to enable debug console. Defaults to False. + start (TimeLike, optional): Start time for the run interval. Defaults to None. + end (TimeLike, optional): End time for the run interval. Defaults to None. + run_options (RunOptions, optional): Run options for the run. Defaults to None. + restate_models (List[str], optional): List of models to restate. Defaults to None. + + Returns: + None: The function records events to a debug console but doesn't return anything. + + Note: + TimeLike can be any time-like object that SQLMesh accepts (datetime, str, etc.). + The function creates a controller and recorder to capture all SQLMesh events during execution. + """ + controller = self.create_controller(enable_debug_console=enable_debug_console) + recorder = ConsoleRecorder() + # controller.add_event_handler(ConsoleRecorder()) + if run_options is None: + run_options = RunOptions() + + if execution_time: + run_options["execution_time"] = execution_time + if start: + run_options["start"] = start + if end: + run_options["end"] = end + + for event in controller.run( + environment, + **run_options, + ): + recorder(event) + + def plan_and_run( + self, + *, + environment: str, + enable_debug_console: bool = False, + start: TimeLike | None = None, + end: TimeLike | None = None, + select_models: list[str] | None = None, + restate_selected: bool = False, + skip_run: bool = False, + plan_options: PlanOptions | None = None, + run_options: RunOptions | None = None, + ) -> None: + """Runs plan and run on SQLMesh with the given configuration and record all of the generated events. + + Args: + environment (str): The environment to run SQLMesh in. + enable_debug_console (bool, optional): Flag to enable debug console. Defaults to False. + start (TimeLike, optional): Start time for the run interval. Defaults to None. + end (TimeLike, optional): End time for the run interval. Defaults to None. + execution_time (TimeLike, optional): The execution timestamp for the run. Defaults to None. + select_models (List[str], optional): List of models to select. Defaults to None. + restate_selected (bool, optional): Flag to restate selected models. Defaults to False. + skip_run (bool, optional): Flag to skip the run. Defaults to False. + plan_options (PlanOptions, optional): Plan options for the plan. Defaults to None. + run_options (RunOptions, optional): Run options for the run. Defaults to None. + + Returns: + None: The function records events to a debug console but doesn't return anything. + + Note: + TimeLike can be any time-like object that SQLMesh accepts (datetime, str, etc.). + The function creates a controller and recorder to capture all SQLMesh events during execution. + """ + controller = self.create_controller(enable_debug_console=enable_debug_console) + recorder = ConsoleRecorder() + + plan_options = plan_options or PlanOptions(enable_preview=True) + run_options = run_options or RunOptions() + + if plan_options.get("select_models") or run_options.get("select_models"): + raise ValueError( + "select_models should not be set in plan_options or run_options use the `select_models` or `select_models_func` arguments instead" + ) + if plan_options.get("restate_models"): + raise ValueError( + "restate_models should not be set in plan_options use the `restate_selected` argument with `select_models` or `select_models_func` instead" + ) + + for event in controller.plan_and_run( + environment, + start=start, + end=end, + select_models=select_models, + restate_selected=restate_selected, + plan_options=plan_options, + run_options=run_options, + skip_run=skip_run, + ): + recorder(event) @pytest.fixture @@ -58,6 +455,339 @@ def sample_sqlmesh_test_context( context_config = SQLMeshContextConfig( path=sample_sqlmesh_project, gateway="local", config_override=config_as_dict ) - test_context = SQLMeshTestContext(db_path=db_path, context_config=context_config) + test_context = SQLMeshTestContext( + db_path=db_path, + context_config=context_config, + project_path=sample_sqlmesh_project, + ) test_context.initialize_test_source() yield test_context + + +@pytest.fixture +def permanent_sqlmesh_project() -> str: + """FOR DEBUGGING ONLY: Returns the path to the permanent sample SQLMesh project. + + This fixture provides access to the sample project without copying to a temp directory, + which is useful for debugging and investigating issues with file handling. + It creates a permanent copy of the sample project in tests/temp/sqlmesh_project + if it doesn't exist. + + Returns: + str: Absolute path to the sample SQLMesh project directory + """ + # Define source and target paths + source_dir = os.path.abspath("sample/sqlmesh_project") + project_dir = os.path.abspath("tests/temp/sqlmesh_project") + + # Create the temp directory if it doesn't exist + os.makedirs(os.path.dirname(project_dir), exist_ok=True) + + # If project directory doesn't exist or is empty, copy from sample + if not os.path.exists(project_dir) or not os.listdir(project_dir): + if os.path.exists(project_dir): + shutil.rmtree(project_dir) + shutil.copytree(source_dir, project_dir) + + # Clean up any existing db file + db_path = os.path.join(project_dir, "db.db") + if os.path.exists(db_path): + os.remove(db_path) + + return project_dir + + +@pytest.fixture +def model_change_test_context( + permanent_sqlmesh_project: str, +) -> t.Iterator[SQLMeshTestContext]: + """FOR DEBUGGING ONLY: Creates a SQLMesh test context specifically for testing model code changes. + + This fixture provides a context that allows modifying SQL model files and ensures + they are properly restored after the test completes. It uses a permanent project + directory instead of a temporary one for better debugging and investigation. + + Args: + permanent_sqlmesh_project: The permanent project directory + + Yields: + SQLMeshTestContext: A test context with additional methods for modifying model files + """ + db_path = os.path.join(permanent_sqlmesh_project, "db.db") + config = SQLMeshConfig( + gateways={ + "local": GatewayConfig(connection=DuckDBConnectionConfig(database=db_path)), + }, + default_gateway="local", + model_defaults=ModelDefaultsConfig(dialect="duckdb"), + ) + config_as_dict = config.dict() + context_config = SQLMeshContextConfig( + path=permanent_sqlmesh_project, gateway="local", config_override=config_as_dict + ) + test_context = SQLMeshTestContext( + db_path=db_path, + context_config=context_config, + project_path=permanent_sqlmesh_project, + ) + test_context.initialize_test_source() + + yield test_context + + # Cleanup: restore any modified files + # test_context.cleanup_modified_files() + + +@dataclass +class DagsterTestContext: + """A test context for running Dagster""" + + dagster_project_path: str + sqlmesh_project_path: str + + def asset_materialisation( + self, + assets: list[str], + plan_options: PlanOptions | None = None, + run_options: RunOptions | None = None, + ) -> None: + """Materialises the given Dagster assets using CLI command. + + Args: + assets: String of comma-separated asset names to materialize + plan_options: Optional SQLMesh plan options to pass to the config + run_options: Optional SQLMesh run options to pass to the config + """ + config: dict[str, t.Any] = { + "resources": { + "sqlmesh": { + "config": { + "config": { + "gateway": "local", + "path": self.sqlmesh_project_path, + } + } + } + } + } + + # Add plan options if provided + if plan_options: + config["resources"]["sqlmesh"]["config"]["plan_options_override"] = { + k: v for k, v in plan_options.items() if v is not None + } + + if run_options: + config["resources"]["sqlmesh"]["config"]["run_options_override"] = { + k: v for k, v in run_options.items() if v is not None + } + + # Convert config to JSON string, escaping backslashes for Windows paths + config_json = json.dumps(config).replace("\\", "\\\\") + + cmd = [ + sys.executable, + "-m", + "dagster", + "asset", + "materialize", + "-f", + os.path.join(self.dagster_project_path, "definitions.py"), + "--select", + ",".join(assets), + "--config-json", + config_json, + ] + + self._run_command(cmd=cmd, cwd=self.sqlmesh_project_path) + + def _run_command(self, cmd: list[str], cwd: str | None = None) -> None: + """Execute a command and stream its output in real-time. + + Args: + cmd: List of command parts to execute + cwd: Optional directory to change to before running the command. + + Raises: + subprocess.CalledProcessError: If the command returns non-zero exit code + RuntimeError: If subprocess pipes cannot be opened + """ + with self._manage_working_directory(cwd): + process = self._create_subprocess(cmd) + self._stream_process_output(process, cmd) + + def _manage_working_directory( + self, cwd: str | None = None + ) -> t.ContextManager[None]: + """Context manager to handle directory changes safely. + + Args: + cwd: Optional directory to change to before running the command. + """ + + @contextmanager + def _directory_context(): + original_cwd = os.getcwd() + try: + if cwd: + print(f"Changing to directory: {cwd}") + os.chdir(cwd) + else: + print(f"Running in current directory: {original_cwd}") + yield + finally: + if os.getcwd() != original_cwd: + print(f"Changing back to original directory: {original_cwd}") + os.chdir(original_cwd) + + return _directory_context() + + def _create_subprocess(self, cmd: list[str]) -> "subprocess.Popen[str]": + """Create and return a subprocess with proper pipe configuration. + + Args: + cmd: List of command parts to execute + + Returns: + subprocess.Popen: The created subprocess with stdout/stderr pipes + + Raises: + RuntimeError: If subprocess pipes cannot be opened + """ + print(f"Running command: {' '.join(cmd)}") + process = subprocess.Popen( + cmd, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + text=True, + universal_newlines=True, + encoding="utf-8", + errors="replace", + ) + if not process.stdout or not process.stderr: + raise RuntimeError("Failed to open subprocess pipes") + return process + + def _stream_process_output( + self, process: "subprocess.Popen[str]", cmd: list[str] + ) -> None: + """Handle the streaming of process output from both stdout and stderr. + + Args: + process: The subprocess whose output to stream + cmd: The original command (for error reporting) + + Raises: + subprocess.CalledProcessError: If the process returns non-zero exit code + """ + output_queue: queue.Queue[tuple[str, str | None]] = queue.Queue() + + # Start output capture threads + threads = self._start_output_threads(process, output_queue) + + # Process output until both streams are done + active_streams = {"stdout", "stderr"} + while active_streams: + try: + stream_type, content = output_queue.get(timeout=0.1) + if content is None: + active_streams.remove(stream_type) + else: + print(content, end="", flush=True) + except queue.Empty: + continue + + # Wait for completion + for thread in threads: + thread.join() + process.wait() + + if process.returncode != 0: + raise subprocess.CalledProcessError(process.returncode, cmd) + + def _start_output_threads( + self, + process: "subprocess.Popen[str]", + output_queue: queue.Queue[tuple[str, str | None]], + ) -> list[threading.Thread]: + """Start and return the stdout/stderr capture threads. + + Args: + process: The subprocess whose output to capture + output_queue: Queue to write captured output to + + Returns: + list[threading.Thread]: List of started capture threads + """ + threads = [] + for pipe in [process.stdout, process.stderr]: + thread = threading.Thread( + target=self._stream_output, + args=(pipe, output_queue, process.stdout), + ) + thread.daemon = True + thread.start() + threads.append(thread) + return threads + + def _stream_output( + self, + pipe: t.IO[str], + output_queue: queue.Queue[tuple[str, str | None]], + process_stdout: t.IO[str], + ) -> None: + """Stream output from a pipe to a queue. + + Args: + pipe: The pipe to read from (stdout or stderr) + output_queue: Queue to write output to, as (stream_type, line) tuples + process_stdout: The stdout pipe from the process, used to determine stream type + """ + # Use a StringIO buffer to accumulate characters into lines + buffer = io.StringIO() + stream_type = "stdout" if pipe is process_stdout else "stderr" + + try: + while True: + char = pipe.read(1) + if not char: + # Flush any remaining content in buffer + remaining = buffer.getvalue() + if remaining: + output_queue.put((stream_type, remaining)) + break + + buffer.write(char) + + # If we hit a newline, flush the buffer + if char == "\n": + output_queue.put((stream_type, buffer.getvalue())) + buffer = io.StringIO() + finally: + buffer.close() + output_queue.put((stream_type, None)) # Signal EOF + + def reset_assets(self) -> None: + """Resets the assets to the original state""" + self.asset_materialisation(assets=["reset_asset"]) + + def init_test_source(self) -> None: + """Initialises the test source""" + self.asset_materialisation(assets=["test_source"]) + + +@pytest.fixture +def sample_dagster_test_context( + sample_dagster_project: str, +) -> t.Iterator[DagsterTestContext]: + test_context = DagsterTestContext( + dagster_project_path=os.path.join(sample_dagster_project), + sqlmesh_project_path=os.path.join( + sample_dagster_project.replace("dagster_project", "sqlmesh_project") + ), + ) + yield test_context + + +if __name__ == "__main__": + pytest.main([__file__]) diff --git a/dagster_sqlmesh/controller/base.py b/dagster_sqlmesh/controller/base.py index 6f4cbf1..cdf84e4 100644 --- a/dagster_sqlmesh/controller/base.py +++ b/dagster_sqlmesh/controller/base.py @@ -10,7 +10,7 @@ from sqlmesh.core.console import Console, set_console from sqlmesh.core.context import Context from sqlmesh.core.model import Model -from sqlmesh.core.plan import PlanBuilder +from sqlmesh.core.plan import Plan, PlanBuilder from sqlmesh.utils.dag import DAG from sqlmesh.utils.date import TimeLike @@ -133,12 +133,112 @@ def console_context(self, handler: ConsoleEventHandler) -> t.Iterator[None]: yield self.console.remove_handler(id) + def _get_plan_summary(self, plan: Plan) -> str: + """ + Get a summary of the plan and return it as a string. + + Args: + plan (Plan): The plan to summarize. + + Returns: + str: A summary of the plan. + """ + directly_modified = len(plan.directly_modified) + indirectly_modified = sum( + len(deps) for deps in plan.indirectly_modified.values() + ) + + plan_summary = [ + "SQLMesh Plan Summary:", + f"• Models: {directly_modified} direct changes, {indirectly_modified} indirect changes", + "• Direct Modifications:", + ] + + # Add directly modified models + for model in sorted(plan.directly_modified): + plan_summary.append(f" - {model}") + + # Add indirectly modified models and their parents + if plan.indirectly_modified: + plan_summary.append("• Indirect Modifications:") + for parent, children in sorted(plan.indirectly_modified.items()): + plan_summary.append(f" - Due to {parent}:") + for child in sorted(children): + plan_summary.append(f" • {child}") + + # Add restatements if any exist + if plan.restatements: + plan_summary.append("• Restatements:") + for model, interval in sorted(plan.restatements.items()): + plan_summary.append(f" - {model}: {interval}") + + plan_summary.extend( + [ + f"• Time Range: {plan.provided_start or 'default'} → {plan.provided_end or 'default'}", + "• Configuration:", + f" - Skip Backfill: {plan.skip_backfill}", + f" - Forward Only: {plan.forward_only}", + f" - No Gaps: {plan.no_gaps}", + f" - Include Unmodified: {plan.include_unmodified}", + f" - Empty Backfill: {plan.empty_backfill}", + f" - End Bounded: {plan.end_bounded}", + f" - Is Dev Environment: {plan.is_dev}", + ] + ) + + if plan.skip_backfill: + plan_summary.append("• Backfill: DISABLED (skip_backfill=True)") + else: + plan_summary.append("• Backfill:") + if plan.selected_models_to_backfill: + plan_summary.append( + f" - User Selected: {sorted(plan.selected_models_to_backfill)}" + ) + if plan.models_to_backfill: + additional = plan.models_to_backfill - ( + plan.selected_models_to_backfill or set() + ) + if additional: + plan_summary.append(f" - Auto-detected: {sorted(additional)}") + if not (plan.selected_models_to_backfill or plan.models_to_backfill): + plan_summary.append(" - None required") + + return "\n".join(plan_summary) + + def _get_builder( + self, context: Context, environment: str, plan_options: PlanOptions + ) -> PlanBuilder: + return context.plan_builder( + environment=environment, + **plan_options, + ) + + def _build_plan( + self, + builder: PlanBuilder, + ) -> Plan: + """Build a SQLMesh plan without applying it. + + Args: + builder: PlanBuilder instance to use for building the plan + + Returns: + Plan + """ + plan: Plan = builder.build() + plan_str = self._get_plan_summary(plan) + + logger.debug("dagster-sqlmesh: plan") + logger.info(f"Plan Summary: {plan_str}") + + return plan + def plan( self, categorizer: SnapshotCategorizer | None = None, default_catalog: str | None = None, **plan_options: t.Unpack[PlanOptions], - ) -> t.Iterator[ConsoleEvent]: + ) -> t.Generator[ConsoleEvent, None, None]: """ Executes a sqlmesh plan operation in a separate thread and yields console events. @@ -174,14 +274,22 @@ def run_sqlmesh_thread( ) -> None: logger.debug("dagster-sqlmesh: thread started") try: - builder = t.cast( - PlanBuilder, - context.plan_builder( - environment=environment, - **plan_options, - ), + builder: PlanBuilder = self._get_builder( + context=context, + environment=environment, + plan_options=plan_options, + ) + + plan: Plan = self._build_plan( + builder=builder, ) + plan_str = self._get_plan_summary(plan) + + print(f"plan_str: {plan_str}") + logger.debug("dagster-sqlmesh: plan") + logger.info(f"Plan Summary: {plan_str}") + controller.console.plan( builder, auto_apply=True, diff --git a/dagster_sqlmesh/controller/tests_base/test__build_plan.py b/dagster_sqlmesh/controller/tests_base/test__build_plan.py new file mode 100644 index 0000000..776184a --- /dev/null +++ b/dagster_sqlmesh/controller/tests_base/test__build_plan.py @@ -0,0 +1,195 @@ +import pytest + +from dagster_sqlmesh.conftest import SQLMeshTestContext +from dagster_sqlmesh.controller.base import PlanOptions +from dagster_sqlmesh.utils import snapshot_id_to_model_name + + +def test_given_basic_plan_when_building_then_returns_expected_plan_properties( + sample_sqlmesh_test_context: SQLMeshTestContext, +) -> None: + """Test basic plan creation with default options.""" + controller = sample_sqlmesh_test_context.create_controller( + enable_debug_console=True + ) + + with controller.instance("dev") as mesh: + builder = mesh._get_builder( + context=mesh.context, + environment="dev", + plan_options=PlanOptions(), + ) + plan = mesh._build_plan( + builder=builder, + ) + + assert plan is not None + # Verify plan properties + assert plan.is_dev is True + assert not plan.skip_backfill + assert not plan.forward_only + assert not plan.include_unmodified + assert plan.end_bounded + + +def test_given_skip_backfill_when_building_then_plan_reflects_backfill_status( + sample_sqlmesh_test_context: SQLMeshTestContext, +) -> None: + """Test plan creation with skip_backfill option.""" + controller = sample_sqlmesh_test_context.create_controller( + enable_debug_console=True + ) + + with controller.instance("dev") as mesh: + builder = mesh._get_builder( + context=mesh.context, + environment="dev", + plan_options=PlanOptions(skip_backfill=True), + ) + plan = mesh._build_plan( + builder=builder, + ) + + assert plan is not None + assert plan.skip_backfill is True + # When skip_backfill is True, no backfill should be executed even if models need it + assert not plan.empty_backfill # Models do need backfill + assert plan.skip_backfill # But backfill is skipped + + +def test_given_selected_models_when_building_then_plan_shows_correct_model_selection( + sample_sqlmesh_test_context: SQLMeshTestContext, +) -> None: + """Test plan creation with specific model selection.""" + controller = sample_sqlmesh_test_context.create_controller( + enable_debug_console=True + ) + selected_models = { + "sqlmesh_example.staging_model_1", + "sqlmesh_example.staging_model_3", + } + + with controller.instance("dev") as mesh: + builder = mesh._get_builder( + context=mesh.context, + environment="dev", + plan_options=PlanOptions(select_models=list(selected_models)), + ) + plan = mesh._build_plan( + builder=builder, + ) + + assert plan is not None + assert plan.selected_models_to_backfill is not None + + # Convert selected models to the fully qualified format + fully_qualified_models = { + f'"db"."{schema}"."{name}"' + for model in selected_models + for schema, name in [model.split(".")] + } + + # Verify selected models match what's in the plan + assert plan.selected_models_to_backfill == fully_qualified_models + + # Verify the number of directly modified models matches + assert len(plan.directly_modified) == len(selected_models) + + print(f"Selected models to backfill: {plan.selected_models_to_backfill}") + print(f"Directly modified models: {plan.directly_modified}") + + # Extract just the model names from SnapshotId objects + directly_modified_models = { + snapshot_id_to_model_name(snapshot_id) for snapshot_id in plan.directly_modified + } + + # Verify the models appear in directly modified (they should be the same models) + assert directly_modified_models == fully_qualified_models + + +def test_given_all_plan_options_when_building_then_plan_reflects_all_settings( + sample_sqlmesh_test_context: SQLMeshTestContext, +) -> None: + """Test plan creation with all possible options set to verify complete configuration.""" + controller = sample_sqlmesh_test_context.create_controller( + enable_debug_console=True + ) + + # Create a comprehensive set of plan options + plan_options = PlanOptions( + skip_backfill=True, + forward_only=True, + include_unmodified=True, + select_models=[ + "sqlmesh_example.staging_model_1", + "sqlmesh_example.staging_model_3", + ], + no_gaps=True, + allow_destructive_models={ + "sqlmesh_example.staging_model_1", + "sqlmesh_example.staging_model_3", + }, + start="2024-01-01", + end="2024-01-02", + execution_time="2024-01-02", + effective_from="2024-01-01", + no_auto_categorization=True, + enable_preview=True, + ) + + with controller.instance("dev") as mesh: + builder = mesh._get_builder( + context=mesh.context, + environment="dev", + plan_options=plan_options, + ) + plan = mesh._build_plan( + builder=builder, + ) + + # Basic plan properties + assert plan.is_dev is True + assert plan.skip_backfill is True + assert plan.forward_only is True + assert plan.include_unmodified is True + assert plan.end_bounded is True + assert plan.no_gaps is True + + # Environment and timing properties + assert plan.provided_start == "2024-01-01" + assert plan.provided_end == "2024-01-02" + assert plan.execution_time == "2024-01-02" + assert plan.effective_from == "2024-01-01" + + # Model selection and backfill properties + selected_models = plan.selected_models_to_backfill + assert selected_models is not None + assert len(selected_models) == 2 + assert '"db"."sqlmesh_example"."staging_model_1"' in selected_models + assert '"db"."sqlmesh_example"."staging_model_3"' in selected_models + + # Destructive models + assert plan.allow_destructive_models == { + '"db"."sqlmesh_example"."staging_model_1"', + '"db"."sqlmesh_example"."staging_model_3"', + } + + # Verify computed properties have expected values + assert plan.start == "2024-01-01" # Should match provided_start + assert plan.end == "2024-01-02" # Should match provided_end + assert not plan.requires_backfill # Should be False since skip_backfill is True + assert plan.has_changes # Should be True since we selected models + assert ( + not plan.has_unmodified_unpromoted + ) # Should be False since we're not including unmodified + assert len(plan.categorized) > 0 # Should have categorized models + assert len(plan.uncategorized) == 0 # Should have no uncategorized models + assert not plan.metadata_updated # Should have no metadata updates + assert len(plan.new_snapshots) > 0 # Should have new snapshots + assert ( + len(plan.missing_intervals) > 0 + ) # Should have missing intervals even with skip_backfill + + +if __name__ == "__main__": + pytest.main([__file__]) diff --git a/dagster_sqlmesh/controller/tests_plan_and_run/test_models.py b/dagster_sqlmesh/controller/tests_plan_and_run/test_models.py new file mode 100644 index 0000000..e1e8e1e --- /dev/null +++ b/dagster_sqlmesh/controller/tests_plan_and_run/test_models.py @@ -0,0 +1,180 @@ +import logging + +import pytest + +from dagster_sqlmesh.conftest import SQLMeshTestContext +from dagster_sqlmesh.controller.base import PlanOptions, RunOptions + +logger = logging.getLogger(__name__) + + +@pytest.mark.parametrize( + "no_auto_upstream,skip_backfill,expected_changes", + [ + ( + True, + True, + { + "staging_1": "==", # No change expected + "staging_2": "==", # No change expected (seed unchanged) + "intermediate": "==", # No change expected + "full": "==", # No change expected + }, + ), + ( + False, + True, + { + "staging_1": ">=", # Should increase (no_auto_upstream disabled) + "staging_2": "==", # No change expected (seed unchanged) + "intermediate": ">=", # Should increase (no_auto_upstream disabled) + "full": ">=", # Should increase (upstream changes) + }, + ), + ( + True, + False, + { + "staging_1": ">=", # Should increase (skip_backfill disabled) + "staging_2": "==", # No change expected (seed unchanged) + "intermediate": ">=", # Should increase (skip_backfill disabled) + "full": ">=", # Should increase (upstream changes) + }, + ), + ( + False, + False, + { + "staging_1": ">=", # Should increase (both disabled) + "staging_2": "==", # No change expected (seed unchanged) + "intermediate": ">=", # Should increase (both disabled) + "full": ">=", # Should increase (upstream changes) + }, + ), + ], + ids=[ + "both_enabled", + "only_skip_backfill", + "only_no_auto_upstream", + "both_disabled", + ], +) +def test_given_model_chain_when_running_with_different_flags_then_behaves_as_expected( + sample_sqlmesh_test_context: SQLMeshTestContext, + no_auto_upstream: bool, + skip_backfill: bool, + expected_changes: dict[str, str], +): + """Test how no_auto_upstream and skip_backfill flags affect model chain updates. + + Model chain: + 1. staging_model_1 (INCREMENTAL_BY_TIME_RANGE) reads from seed_model_1 + - Contains id, item_id, event_date + 2. staging_model_2 (VIEW) reads from seed_model_2 + - Contains id, item_name + 3. intermediate_model_1 (INCREMENTAL_BY_TIME_RANGE) + - Joins staging_1 and staging_2 on id + 4. full_model (FULL) + - Groups by item_id and counts distinct ids + - Count will remain same unless new item_ids are added to seed data + + Args: + sample_sqlmesh_test_context: Test context fixture + no_auto_upstream: Whether to disable automatic upstream model updates + skip_backfill: Whether to skip backfilling data + expected_changes: Dict mapping model names to expected count changes + "==" means counts should be equal + ">=" means final count should be greater than or equal to initial + """ + # Initial run to set up all models + sample_sqlmesh_test_context.plan_and_run( + environment="dev", + start="2023-01-01", + end="2024-01-01", + plan_options=PlanOptions( + execution_time="2024-01-02", + ), + run_options=RunOptions( + execution_time="2024-01-02", + ), + ) + + # Get initial counts for the model chain + initial_counts = { + "staging_1": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_1" + )[0][0], + "staging_2": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_2" + )[0][0], + "intermediate": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.intermediate_model_1" + )[0][0], + "full": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.full_model" + )[0][0], + } + + print(f"initial_counts: {initial_counts}") + + # Run with specified flags + sample_sqlmesh_test_context.plan_and_run( + environment="dev", + start="2023-01-01", # TODO SQLMesh may have a bug where it doesn't update full_model if start is not set (full model has the wrong snapshot as a parent) + end="2024-07-15", + select_models=["sqlmesh_example.full_model"], + plan_options=PlanOptions( + skip_backfill=skip_backfill, + enable_preview=True, + execution_time="2024-07-15", + ), + run_options=RunOptions( + no_auto_upstream=no_auto_upstream, + execution_time="2024-07-15", + ), + ) + + # Get final counts and debug info + final_counts = { + "seed_1": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.seed_model_1" + )[0][0], + "staging_1": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_1" + )[0][0], + "staging_2": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_2" + )[0][0], + "intermediate": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.intermediate_model_1" + )[0][0], + "full": sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.full_model" + )[0][0], + } + print(f"final_counts: {final_counts}") + print( + f"intermediate_model_1: {sample_sqlmesh_test_context.query('SELECT * FROM sqlmesh_example__dev.intermediate_model_1', return_df=True)}" + ) + print( + f"full_model: {sample_sqlmesh_test_context.query('SELECT * FROM sqlmesh_example__dev.full_model', return_df=True)}" + ) + + # Verify counts match expectations + for model, expected_change in expected_changes.items(): + if expected_change == "==": + assert final_counts[model] == initial_counts[model], ( + f"{model} count should remain unchanged when " + f"no_auto_upstream={no_auto_upstream} and skip_backfill={skip_backfill}" + ) + elif expected_change == ">=": + assert final_counts[model] >= initial_counts[model], ( + f"{model} count should increase when " + f"no_auto_upstream={no_auto_upstream} and skip_backfill={skip_backfill}" + ) + else: + raise ValueError(f"Invalid expected change: {expected_change}") + + +if __name__ == "__main__": + pytest.main([__file__]) diff --git a/dagster_sqlmesh/controller/tests_plan_and_run/test_seeds.py b/dagster_sqlmesh/controller/tests_plan_and_run/test_seeds.py new file mode 100644 index 0000000..73e5eaa --- /dev/null +++ b/dagster_sqlmesh/controller/tests_plan_and_run/test_seeds.py @@ -0,0 +1,75 @@ +import logging + +import pytest + +from dagster_sqlmesh.conftest import SQLMeshTestContext +from dagster_sqlmesh.controller.base import PlanOptions, RunOptions + +logger = logging.getLogger(__name__) + + +@pytest.mark.parametrize( + "skip_backfill", + [True, False], + ids=["skip_backfill_enabled", "skip_backfill_disabled"], +) +def test_given_seed_model_when_selected_then_only_builds_seed( + sample_sqlmesh_test_context: SQLMeshTestContext, + skip_backfill: bool, +): + """Test that selecting a seed model only builds that model. + + This test verifies that when selecting only a seed model: + 1. Only the selected seed model is built + 2. No other models are built or affected + 3. The behavior is consistent regardless of skip_backfill setting + """ + # Run with only seed_model_1 selected - no need for initial setup since + # we're testing what gets built in the first place + sample_sqlmesh_test_context.plan_and_run( + environment="dev", + start="2023-01-01", + end="2024-07-15", + plan_options=PlanOptions( + skip_backfill=skip_backfill, + enable_preview=True, + execution_time="2024-07-15", + ), + run_options=RunOptions( + execution_time="2024-07-15", + ), + select_models=["sqlmesh_example.seed_model_1"], + ) + + # Verify seed_model_1 exists and has data + seed_1_count = sample_sqlmesh_test_context.query( + "SELECT COUNT(*) FROM sqlmesh_example__dev.seed_model_1" + )[0][0] + assert seed_1_count > 0, "Selected seed model should be built with data" + + # Verify no other models were built by checking if their tables exist + other_models = [ + "seed_model_2", + "staging_model_1", + "staging_model_2", + "intermediate_model_1", + "full_model", + ] + + for model in other_models: + # Using information_schema to check if table exists + table_exists = sample_sqlmesh_test_context.query( + f""" + SELECT COUNT(*) + FROM information_schema.tables + WHERE table_schema = 'sqlmesh_example__dev' + AND table_name = '{model}' + """ + )[0][0] + assert table_exists == 0, ( + f"{model} should not be built when only seed_model_1 is selected" + ) + + +if __name__ == "__main__": + pytest.main([__file__]) diff --git a/dagster_sqlmesh/test_sqlmesh_context.py b/dagster_sqlmesh/controller/tests_plan_and_run/test_sqlmesh_context.py similarity index 74% rename from dagster_sqlmesh/test_sqlmesh_context.py rename to dagster_sqlmesh/controller/tests_plan_and_run/test_sqlmesh_context.py index 075a544..cc10c1e 100644 --- a/dagster_sqlmesh/test_sqlmesh_context.py +++ b/dagster_sqlmesh/controller/tests_plan_and_run/test_sqlmesh_context.py @@ -1,8 +1,10 @@ import logging import polars +import pytest -from .testing import SQLMeshTestContext +from dagster_sqlmesh.conftest import SQLMeshTestContext +from dagster_sqlmesh.controller.base import PlanOptions, RunOptions logger = logging.getLogger(__name__) @@ -17,7 +19,7 @@ def test_basic_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): SELECT COUNT(*) as items FROM sqlmesh_example__dev.staging_model_1 """ ) - assert staging_model_count[0][0] == 7 + assert staging_model_count[0][0] == 8 def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): @@ -26,7 +28,12 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): environment="dev", start="2023-01-01", end="2024-01-01", - execution_time="2024-01-02", + plan_options=PlanOptions( + execution_time="2024-01-02", + ), + run_options=RunOptions( + execution_time="2024-01-02", + ), ) staging_model_count = sample_sqlmesh_test_context.query( @@ -41,7 +48,12 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): environment="dev", start="2024-01-01", end="2024-07-07", - execution_time="2024-07-08", + plan_options=PlanOptions( + execution_time="2024-07-08", + ), + run_options=RunOptions( + execution_time="2024-07-08", + ), ) staging_model_count = sample_sqlmesh_test_context.query( @@ -49,7 +61,7 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_1 """ ) - assert staging_model_count[0][0] == 7 + assert staging_model_count[0][0] == 8 test_source_model_count = sample_sqlmesh_test_context.query( """ @@ -70,7 +82,12 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): sample_sqlmesh_test_context.plan_and_run( environment="dev", end="2024-07-10", - execution_time="2024-07-10", + plan_options=PlanOptions( + execution_time="2024-07-10", + ), + run_options=RunOptions( + execution_time="2024-07-10", + ), # restate_models=["sqlmesh_example.staging_model_3"], ) @@ -79,7 +96,7 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): SELECT COUNT(*) FROM sqlmesh_example__dev.staging_model_1 """ ) - assert staging_model_count[0][0] == 7 + assert staging_model_count[0][0] == 8 test_source_model_count = sample_sqlmesh_test_context.query( """ @@ -92,7 +109,12 @@ def test_sqlmesh_context(sample_sqlmesh_test_context: SQLMeshTestContext): sample_sqlmesh_test_context.plan_and_run( environment="dev", end="2024-07-10", - execution_time="2024-07-10", + plan_options=PlanOptions( + execution_time="2024-07-10", + ), + run_options=RunOptions( + execution_time="2024-07-10", + ), ) logger.debug("SQLMESH MATERIALIZATION 5") @@ -130,7 +152,12 @@ def test_restating_models(sample_sqlmesh_test_context: SQLMeshTestContext): environment="dev", start="2023-01-01", end="2024-01-01", - execution_time="2024-01-02", + plan_options=PlanOptions( + execution_time="2024-01-02", + ), + run_options=RunOptions( + execution_time="2024-01-02", + ), ) count_query = sample_sqlmesh_test_context.query( @@ -161,7 +188,12 @@ def test_restating_models(sample_sqlmesh_test_context: SQLMeshTestContext): environment="dev", start="2023-03-01", end="2023-03-31", - execution_time="2024-01-02", + plan_options=PlanOptions( + execution_time="2024-01-02", + ), + run_options=RunOptions( + execution_time="2024-01-02", + ), select_models=["sqlmesh_example.staging_model_4"], restate_selected=True, skip_run=True, @@ -184,12 +216,16 @@ def test_restating_models(sample_sqlmesh_test_context: SQLMeshTestContext): """ ) - assert ( - feb_sum_query_restate[0][0] == feb_sum_query[0][0] - ), "February sum should not change" - assert ( - march_sum_query_restate[0][0] != march_sum_query[0][0] - ), "March sum should change" - assert ( - intermediate_2_query_restate[0][0] == intermediate_2_query[0][0] - ), "Intermediate model should not change during restate" + assert feb_sum_query_restate[0][0] == feb_sum_query[0][0], ( + "February sum should not change" + ) + assert march_sum_query_restate[0][0] != march_sum_query[0][0], ( + "March sum should change" + ) + assert intermediate_2_query_restate[0][0] == intermediate_2_query[0][0], ( + "Intermediate model should not change during restate" + ) + + +if __name__ == "__main__": + pytest.main([__file__]) diff --git a/dagster_sqlmesh/resource.py b/dagster_sqlmesh/resource.py index 02665a9..c59f783 100644 --- a/dagster_sqlmesh/resource.py +++ b/dagster_sqlmesh/resource.py @@ -30,7 +30,7 @@ def __init__(self, sorted_dag: list[str], logger: logging.Logger) -> None: def plan(self, batches: dict[Snapshot, int]) -> None: self._batches = batches - self._count: dict[Snapshot, int] = {} + self._count = {} incomplete_names = set() for snapshot, count in self._batches.items(): @@ -228,7 +228,9 @@ def update_stage(self, stage: str): class SQLMeshResource(ConfigurableResource): config: SQLMeshContextConfig - + plan_options_override: dict | None = None + run_options_override: dict | None = None + def run( self, context: AssetExecutionContext, @@ -244,6 +246,20 @@ def run( plan_options = plan_options or {} run_options = run_options or {} + if self.plan_options_override: + assert isinstance(self.plan_options_override, dict) + plan_options = PlanOptions(**self.plan_options_override) + + if plan_options.get("select_models"): + raise ValueError("select_models is not allowed in plan_options") + + if self.run_options_override: + assert isinstance(self.run_options_override, dict) + run_options = RunOptions(**self.run_options_override) + + if run_options.get("select_models"): + raise ValueError("select_models is not allowed in run_options") + logger = context.log controller = self.get_controller(logger) @@ -251,7 +267,7 @@ def run( with controller.instance(environment) as mesh: dag = mesh.models_dag() - select_models = [] + select_models: list[str] | None = None models = mesh.models() models_map = models.copy() @@ -268,7 +284,12 @@ def run( logger.info(f"selected model: {model.name}") models_map[key] = model + + if select_models is None: + select_models = [] + select_models.append(model.name) + selected_models_set = set(models_map.keys()) if all_available_models == selected_models_set: diff --git a/package.json b/package.json index cda3ff5..b296f48 100644 --- a/package.json +++ b/package.json @@ -12,7 +12,7 @@ "scripts": { "build": "turbo run build --concurrency=100%", "format:staged": "lint-staged", - "pyright": "pyright --pythonpath $(echo 'import sys; print(sys.prefix)' | uv run -)/bin/python", + "pyright": "uv run -q -c \"import sys; print(sys.executable)\" | xargs -r pyright --pythonpath", "prepare": "husky install" }, "devDependencies": { diff --git a/pyproject.toml b/pyproject.toml index b78da07..b9198bc 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -25,6 +25,7 @@ dev = [ "dagster-duckdb-polars>=0.24.2", "fastapi", # this is for sqlmesh ui "sse-starlette", + "pandas-stubs" ] [build-system] diff --git a/sample/dagster_project/definitions.py b/sample/dagster_project/definitions.py index 345de54..b056cbe 100644 --- a/sample/dagster_project/definitions.py +++ b/sample/dagster_project/definitions.py @@ -16,7 +16,7 @@ CURR_DIR = os.path.dirname(__file__) SQLMESH_PROJECT_PATH = os.path.abspath(os.path.join(CURR_DIR, "../sqlmesh_project")) -DUCKDB_PATH = os.path.join(CURR_DIR, "../../db.db") +DUCKDB_PATH = os.path.join(SQLMESH_PROJECT_PATH, "db.db") sqlmesh_config = SQLMeshContextConfig(path=SQLMESH_PROJECT_PATH, gateway="local") @@ -53,7 +53,9 @@ def test_source() -> pl.DataFrame: @sqlmesh_assets(environment="dev", config=sqlmesh_config, enabled_subsetting=True) -def sqlmesh_project(context: AssetExecutionContext, sqlmesh: SQLMeshResource) -> t.Iterator[MaterializeResult]: +def sqlmesh_project( + context: AssetExecutionContext, sqlmesh: SQLMeshResource +) -> t.Iterator[MaterializeResult]: yield from sqlmesh.run(context) diff --git a/sample/sqlmesh_project/models/marts/full_model.sql b/sample/sqlmesh_project/models/marts/full_model.sql index 4a70afd..52d8bf1 100644 --- a/sample/sqlmesh_project/models/marts/full_model.sql +++ b/sample/sqlmesh_project/models/marts/full_model.sql @@ -1,6 +1,6 @@ -MODEL ( - name sqlmesh_example.full_model, - kind FULL, +model ( + NAME sqlmesh_example.full_model, + kind full, cron '@daily', grain item_id, audits (assert_positive_order_ids), @@ -9,10 +9,14 @@ MODEL ( "full", ) ); - SELECT item_id, - COUNT(DISTINCT id) AS num_orders, + item_name, + COUNT( + DISTINCT id + ) AS num_orders, + now() AS last_updated_at FROM sqlmesh_example.intermediate_model_1 -GROUP BY item_id +GROUP BY + ALL diff --git a/sample/sqlmesh_project/seeds/seed_data_1.csv b/sample/sqlmesh_project/seeds/seed_data_1.csv index d70190f..1e4e6cd 100644 --- a/sample/sqlmesh_project/seeds/seed_data_1.csv +++ b/sample/sqlmesh_project/seeds/seed_data_1.csv @@ -6,3 +6,4 @@ id,item_id,event_date 5,1,2023-06-05 6,1,2024-07-06 7,1,2024-07-07 +8,4,2024-06-07 diff --git a/sample/sqlmesh_project/tests/test_full_model.yaml b/sample/sqlmesh_project/tests/test_full_model.yaml index 4285166..3a79c8c 100644 --- a/sample/sqlmesh_project/tests/test_full_model.yaml +++ b/sample/sqlmesh_project/tests/test_full_model.yaml @@ -1,5 +1,6 @@ test_example_full_model: model: sqlmesh_example.full_model + exclude_columns: [last_updated_at] inputs: sqlmesh_example.intermediate_model_1: rows: @@ -11,6 +12,7 @@ test_example_full_model: item_id: 2 outputs: query: + partial: true rows: - item_id: 1 num_orders: 2 diff --git a/uv.lock b/uv.lock index 9b1daee..3b6e331 100644 --- a/uv.lock +++ b/uv.lock @@ -8,16 +8,16 @@ resolution-markers = [ [[package]] name = "alembic" -version = "1.15.1" +version = "1.15.2" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "mako" }, { name = "sqlalchemy" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/4a/ed/901044acb892caa5604bf818d2da9ab0df94ef606c6059fdf367894ebf60/alembic-1.15.1.tar.gz", hash = "sha256:e1a1c738577bca1f27e68728c910cd389b9a92152ff91d902da649c192e30c49", size = 1924789 } +sdist = { url = "https://files.pythonhosted.org/packages/e6/57/e314c31b261d1e8a5a5f1908065b4ff98270a778ce7579bd4254477209a7/alembic-1.15.2.tar.gz", hash = "sha256:1c72391bbdeffccfe317eefba686cb9a3c078005478885413b95c3b26c57a8a7", size = 1925573 } wheels = [ - { url = "https://files.pythonhosted.org/packages/99/f7/d398fae160568472ddce0b3fde9c4581afc593019a6adc91006a66406991/alembic-1.15.1-py3-none-any.whl", hash = "sha256:197de710da4b3e91cf66a826a5b31b5d59a127ab41bd0fc42863e2902ce2bbbe", size = 231753 }, + { url = "https://files.pythonhosted.org/packages/41/18/d89a443ed1ab9bcda16264716f809c663866d4ca8de218aa78fd50b38ead/alembic-1.15.2-py3-none-any.whl", hash = "sha256:2e76bd916d547f6900ec4bb5a90aeac1485d2c92536923d0b138c02b126edc53", size = 231911 }, ] [[package]] @@ -40,16 +40,16 @@ wheels = [ [[package]] name = "anyio" -version = "4.8.0" +version = "4.9.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "idna" }, { name = "sniffio" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/a3/73/199a98fc2dae33535d6b8e8e6ec01f8c1d76c9adb096c6b7d64823038cde/anyio-4.8.0.tar.gz", hash = "sha256:1d9fe889df5212298c0c0723fa20479d1b94883a2df44bd3897aa91083316f7a", size = 181126 } +sdist = { url = "https://files.pythonhosted.org/packages/95/7d/4c1bd541d4dffa1b52bd83fb8527089e097a106fc90b467a7313b105f840/anyio-4.9.0.tar.gz", hash = "sha256:673c0c244e15788651a4ff38710fea9675823028a6f08a5eda409e0c9840a028", size = 190949 } wheels = [ - { url = "https://files.pythonhosted.org/packages/46/eb/e7f063ad1fec6b3178a3cd82d1a3c4de82cccf283fc42746168188e1cdd5/anyio-4.8.0-py3-none-any.whl", hash = "sha256:b5011f270ab5eb0abf13385f851315585cc37ef330dd88e27ec3d34d651fd47a", size = 96041 }, + { url = "https://files.pythonhosted.org/packages/a1/ee/48ca1a7c89ffec8b6a0c5d02b89c305671d5ffd8d3c94acf8b8c408575bb/anyio-4.9.0-py3-none-any.whl", hash = "sha256:9f76d541cad6e36af7beb62e978876f3b41e3e04f2c1fbf0884604c0a9c4d93c", size = 100916 }, ] [[package]] @@ -72,11 +72,11 @@ wheels = [ [[package]] name = "attrs" -version = "25.2.0" +version = "25.3.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/69/82/3c4e1d44f3cbaa2a578127d641fe385ba3bff6c38b789447ae11a21fa413/attrs-25.2.0.tar.gz", hash = "sha256:18a06db706db43ac232cce80443fcd9f2500702059ecf53489e3c5a3f417acaf", size = 812038 } +sdist = { url = "https://files.pythonhosted.org/packages/5a/b0/1367933a8532ee6ff8d63537de4f1177af4bff9f3e829baf7331f595bb24/attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b", size = 812032 } wheels = [ - { url = "https://files.pythonhosted.org/packages/03/33/7a7388b9ef94aab40539939d94461ec682afbd895458945ed25be07f03f6/attrs-25.2.0-py3-none-any.whl", hash = "sha256:611344ff0a5fed735d86d7784610c84f8126b95e549bcad9ff61b4242f2d386b", size = 64016 }, + { url = "https://files.pythonhosted.org/packages/77/06/bb80f5f86020c4551da315d78b3ab75e8228f89f0162f2c3a819e407941a/attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3", size = 63815 }, ] [[package]] @@ -192,7 +192,7 @@ wheels = [ [[package]] name = "dagster" -version = "1.10.4" +version = "1.10.7" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "alembic" }, @@ -200,19 +200,17 @@ dependencies = [ { name = "click" }, { name = "coloredlogs" }, { name = "dagster-pipes" }, + { name = "dagster-shared" }, { name = "docstring-parser" }, { name = "filelock" }, { name = "grpcio" }, { name = "grpcio-health-checking" }, { name = "jinja2" }, - { name = "packaging" }, { name = "protobuf" }, { name = "psutil", marker = "sys_platform == 'win32'" }, - { name = "pydantic" }, { name = "python-dotenv" }, { name = "pytz" }, { name = "pywin32", marker = "sys_platform == 'win32'" }, - { name = "pyyaml" }, { name = "requests" }, { name = "rich" }, { name = "setuptools" }, @@ -223,46 +221,45 @@ dependencies = [ { name = "tomli" }, { name = "toposort" }, { name = "tqdm" }, - { name = "typing-extensions" }, { name = "tzdata", marker = "sys_platform == 'win32'" }, { name = "universal-pathlib" }, { name = "watchdog" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/91/2e/a5f71129ec2a233781000f64398f1a5297a1b0981b3cb01204ed84ae9b4e/dagster-1.10.4.tar.gz", hash = "sha256:00c047abe2299a16d09b6b235a0f18ec904ffa2789f9ca98625cd1e48ba6d9b9", size = 1440355 } +sdist = { url = "https://files.pythonhosted.org/packages/26/72/a5965548050506b5381e8077bac7ed679276c8b8ba0832f2c84de30807c6/dagster-1.10.7.tar.gz", hash = "sha256:976aca6f9ef18b2cf63c4e583c58521d7981c1b1b70f0fe3661ab3453f1f5669", size = 1402907 } wheels = [ - { url = "https://files.pythonhosted.org/packages/96/3b/ad11467f0329dcdbd83ceee5591aa93807944bcb108e973dc00a1bd453b5/dagster-1.10.4-py3-none-any.whl", hash = "sha256:df6305d7e82fbd4d2c7f490467d1acf0eae6a138853c1bbdefd7c60715b29803", size = 1761024 }, + { url = "https://files.pythonhosted.org/packages/af/10/635c6307018d4d95b5f9a301121a1644e24b7221e3f00fdeffac5a8c98e2/dagster-1.10.7-py3-none-any.whl", hash = "sha256:b9790999d226f690067cfc913bfd4111f532bbd72cfad01b78473fb7259892e1", size = 1719757 }, ] [[package]] name = "dagster-duckdb" -version = "0.26.4" +version = "0.26.7" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "dagster" }, { name = "duckdb" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/db/b4/0a9624b58f50649ab2b699a87cc945e19d2fe268d072660e5088b0932fd4/dagster-duckdb-0.26.4.tar.gz", hash = "sha256:d87170b0726f60c408c36700b52a3bc08b89e0503f96736142539276c91731fc", size = 9292 } +sdist = { url = "https://files.pythonhosted.org/packages/13/c3/83a8361a1dba10d23656bb27f6ede5e29872cc2ddf209f9b6447f99b2eca/dagster-duckdb-0.26.7.tar.gz", hash = "sha256:e21bded1c1451544a0f8f1f310fb22f89864300853d6b744e6349c2e169c748f", size = 9304 } wheels = [ - { url = "https://files.pythonhosted.org/packages/9b/12/ae9313975437e62c05c9724aff46ee15ed2ade2fdc3876b9c4ffbd5f08f1/dagster_duckdb-0.26.4-py3-none-any.whl", hash = "sha256:73a91f722c290f5d0876ccf313312b94dc3e09cc362da803659acc97bf218074", size = 10483 }, + { url = "https://files.pythonhosted.org/packages/50/b6/175910bc2664beb05738e6b70ca6dc7bb231dbaa9dac55bf3a09d0c43def/dagster_duckdb-0.26.7-py3-none-any.whl", hash = "sha256:60bcb3f9acaac688f18db907aad74bda041a212490c616ffe94be96602112716", size = 10519 }, ] [[package]] name = "dagster-duckdb-polars" -version = "0.26.4" +version = "0.26.7" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "dagster" }, { name = "dagster-duckdb" }, { name = "polars", extra = ["pyarrow"] }, ] -sdist = { url = "https://files.pythonhosted.org/packages/60/c6/fd498cf0aed7f61fadfb4394de8757a6014ac90ba9b274fb8b54ccfdf336/dagster-duckdb-polars-0.26.4.tar.gz", hash = "sha256:589177f09bbd40c1cad73b0a37e3fbf0090b107746718e27cdfef3ed3695afa4", size = 7906 } +sdist = { url = "https://files.pythonhosted.org/packages/cd/6d/8d17fb3bf79a46d8b3c2e9596c35dea1c759230e51030432b20e8b412b43/dagster-duckdb-polars-0.26.7.tar.gz", hash = "sha256:6b18d9d4b570a51b62f4343db4b5e078609c1b5f37a938b1d752233c5f2aa3af", size = 7915 } wheels = [ - { url = "https://files.pythonhosted.org/packages/c8/96/9ee2d70f83acac8c4252912b344fcb8edbd321035e4f824e9053dff4cbf8/dagster_duckdb_polars-0.26.4-py3-none-any.whl", hash = "sha256:d52feb8f1ca06357e4a3769ad209cc7bc83e94c48c7451e1f2adff9f81208551", size = 8563 }, + { url = "https://files.pythonhosted.org/packages/55/97/cafd45ad3bb3e94457e9b512971e8af688c590519db0a71115171d9526f2/dagster_duckdb_polars-0.26.7-py3-none-any.whl", hash = "sha256:176ad0a7cc11b5e5c20d6bd1b10e76609f26d3f8325635396d0db44bab25a740", size = 8562 }, ] [[package]] name = "dagster-graphql" -version = "1.10.4" +version = "1.10.7" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "dagster" }, @@ -271,18 +268,33 @@ dependencies = [ { name = "requests" }, { name = "starlette" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/e3/0a/750e0f84de511cb65e32888f5c2eafda3b85ec914d47b7a71e7492b7aa84/dagster-graphql-1.10.4.tar.gz", hash = "sha256:69556db8aeb86ff9d82821c9808221004e593b9c263bff58f404682a5ccaee66", size = 146754 } +sdist = { url = "https://files.pythonhosted.org/packages/9a/94/d300c5ac3318266277770547a3ade424b29cfa5126fdbef4bd42d3320421/dagster-graphql-1.10.7.tar.gz", hash = "sha256:66c706a5becb1c23d0e6c6ee6f1cddd3d8019223a6e7d36eb8ac040181579736", size = 147998 } wheels = [ - { url = "https://files.pythonhosted.org/packages/cb/f4/f7fea2e06026f0ca4c4db8b5c576bc55a2c952800b14a2152577f5979b46/dagster_graphql-1.10.4-py3-none-any.whl", hash = "sha256:788c6d711cabdb2b4dcfb0324619268c28b558295acad870f074030fefd54fd8", size = 192852 }, + { url = "https://files.pythonhosted.org/packages/fc/d2/d1102292b3af66d5f7dae8d50015d6d03ea6b820b880fc747572c07fe4b4/dagster_graphql-1.10.7-py3-none-any.whl", hash = "sha256:3a7f8962d89ba7e7d43eaf6b66754c9302f35306b7cefa0338bfd4610924f0b7", size = 194185 }, ] [[package]] name = "dagster-pipes" -version = "1.10.4" +version = "1.10.7" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/eb/da/15294d7e95adef3df8be6ce6613afe9cdd21b43a77d0eca151c674099ad2/dagster-pipes-1.10.7.tar.gz", hash = "sha256:b8ee19a66e993165a0c6aa04a019a934beb299341cb21668b2a24ed2a1b58e71", size = 20476 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/5c/9b/e5f325c506790ca2f4e66acbe3965cf67053b5bb1e188cc130649da5f7db/dagster_pipes-1.10.7-py3-none-any.whl", hash = "sha256:caca2e18b9d993c71cb836a930c7c1a12be3d987637354a6ee2451d3ce7793b0", size = 20276 }, +] + +[[package]] +name = "dagster-shared" +version = "0.26.7" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/ad/37/d99a2f32cc829275680f75c3472fea474329af0ee649cf1e4154360af584/dagster-pipes-1.10.4.tar.gz", hash = "sha256:7f5f2098cca54d46cce0f38940405ed3226326ca14f667d16afbad5f605c16f6", size = 20457 } +dependencies = [ + { name = "packaging" }, + { name = "pydantic" }, + { name = "pyyaml" }, + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b8/47/b9b3d3958110b8e506b54e59a78ec25d2166d525a71d3f314ad55433ffec/dagster_shared-0.26.7.tar.gz", hash = "sha256:19c6542ee5d57d253de48435c2f280eabc7e1a2bee31169da0fbf0cf7ab9444a", size = 61501 } wheels = [ - { url = "https://files.pythonhosted.org/packages/fb/f4/7fa4fe5548f5f5e34dea155faf1c592c03e9877caf3b45a720eaa878bc87/dagster_pipes-1.10.4-py3-none-any.whl", hash = "sha256:31e8baaac97fcd46c683ee1499a435efd1c906cfe69e88ccafb5185d3630559c", size = 20254 }, + { url = "https://files.pythonhosted.org/packages/86/12/844d5a5f34708570e58997903a5c9309ada3b368924e6f52f603da27a402/dagster_shared-0.26.7-py3-none-any.whl", hash = "sha256:f966d5af45efd48a0b38c049b44d604594bc6b23c18d317eefb3b42a10818e6b", size = 71218 }, ] [[package]] @@ -302,6 +314,7 @@ dev = [ { name = "dagster-webserver" }, { name = "fastapi" }, { name = "ipython" }, + { name = "pandas-stubs" }, { name = "pdbpp" }, { name = "polars" }, { name = "ruff" }, @@ -322,6 +335,7 @@ dev = [ { name = "dagster-webserver", specifier = ">=1.8.1" }, { name = "fastapi" }, { name = "ipython", specifier = ">=8.26.0" }, + { name = "pandas-stubs" }, { name = "pdbpp", specifier = ">=0.10.3" }, { name = "polars", specifier = ">=1.5.0" }, { name = "ruff", specifier = "==0.10.0" }, @@ -330,7 +344,7 @@ dev = [ [[package]] name = "dagster-webserver" -version = "1.10.4" +version = "1.10.7" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "click" }, @@ -339,9 +353,9 @@ dependencies = [ { name = "starlette" }, { name = "uvicorn", extra = ["standard"] }, ] -sdist = { url = "https://files.pythonhosted.org/packages/21/f1/a1a9d07c716d17b0b4d7abe6ab17eba2babc78f0fe23a793f469fe4e1d22/dagster-webserver-1.10.4.tar.gz", hash = "sha256:9dcd56eee7b0218511d8353891b3390b3fd9775b0edaa024f825fa4a1a490006", size = 12014382 } +sdist = { url = "https://files.pythonhosted.org/packages/73/a5/c224e5ceac1b5f3a88d405d799c7a00f790292cfa69a4a5c265689adbc43/dagster-webserver-1.10.7.tar.gz", hash = "sha256:4732c9a288098b95ff0e767f5a79944a8dbeeabfd099fe4f75ae183e52590fd5", size = 12112730 } wheels = [ - { url = "https://files.pythonhosted.org/packages/69/93/83bca6aec5627e7ca57815cad9c984bf11d3140d2cf54da64b153fa3eccb/dagster_webserver-1.10.4-py3-none-any.whl", hash = "sha256:9182204f6c9a6b120c097d69e78a5f2d5b93d0318f5c7117a330e77060e12314", size = 12363015 }, + { url = "https://files.pythonhosted.org/packages/39/b0/0207d9657564e887f636dd7022b2d161ee04f8dd64bd9f6eaf14e54f8417/dagster_webserver-1.10.7-py3-none-any.whl", hash = "sha256:65f0f78b26763639a86897f4c99fd9ea17c2dc82ca2d919020f699ba345e2392", size = 12459915 }, ] [[package]] @@ -425,34 +439,34 @@ wheels = [ [[package]] name = "fastapi" -version = "0.115.11" +version = "0.115.12" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pydantic" }, { name = "starlette" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b5/28/c5d26e5860df807241909a961a37d45e10533acef95fc368066c7dd186cd/fastapi-0.115.11.tar.gz", hash = "sha256:cc81f03f688678b92600a65a5e618b93592c65005db37157147204d8924bf94f", size = 294441 } +sdist = { url = "https://files.pythonhosted.org/packages/f4/55/ae499352d82338331ca1e28c7f4a63bfd09479b16395dce38cf50a39e2c2/fastapi-0.115.12.tar.gz", hash = "sha256:1e2c2a2646905f9e83d32f04a3f86aff4a286669c6c950ca95b5fd68c2602681", size = 295236 } wheels = [ - { url = "https://files.pythonhosted.org/packages/b3/5d/4d8bbb94f0dbc22732350c06965e40740f4a92ca560e90bb566f4f73af41/fastapi-0.115.11-py3-none-any.whl", hash = "sha256:32e1541b7b74602e4ef4a0260ecaf3aadf9d4f19590bba3e1bf2ac4666aa2c64", size = 94926 }, + { url = "https://files.pythonhosted.org/packages/50/b3/b51f09c2ba432a576fe63758bddc81f78f0c6309d9e5c10d194313bf021e/fastapi-0.115.12-py3-none-any.whl", hash = "sha256:e94613d6c05e27be7ffebdd6ea5f388112e5e430c8f7d6494a9d1d88d43e814d", size = 95164 }, ] [[package]] name = "filelock" -version = "3.17.0" +version = "3.18.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/dc/9c/0b15fb47b464e1b663b1acd1253a062aa5feecb07d4e597daea542ebd2b5/filelock-3.17.0.tar.gz", hash = "sha256:ee4e77401ef576ebb38cd7f13b9b28893194acc20a8e68e18730ba9c0e54660e", size = 18027 } +sdist = { url = "https://files.pythonhosted.org/packages/0a/10/c23352565a6544bdc5353e0b15fc1c563352101f30e24bf500207a54df9a/filelock-3.18.0.tar.gz", hash = "sha256:adbc88eabb99d2fec8c9c1b229b171f18afa655400173ddc653d5d01501fb9f2", size = 18075 } wheels = [ - { url = "https://files.pythonhosted.org/packages/89/ec/00d68c4ddfedfe64159999e5f8a98fb8442729a63e2077eb9dcd89623d27/filelock-3.17.0-py3-none-any.whl", hash = "sha256:533dc2f7ba78dc2f0f531fc6c4940addf7b70a481e269a5a3b93be94ffbe8338", size = 16164 }, + { url = "https://files.pythonhosted.org/packages/4d/36/2a115987e2d8c300a974597416d9de88f2444426de9571f4b59b2cca3acc/filelock-3.18.0-py3-none-any.whl", hash = "sha256:c401f4f8377c4464e6db25fff06205fd89bdd83b65eb0488ed1b160f780e21de", size = 16215 }, ] [[package]] name = "fsspec" -version = "2025.3.0" +version = "2025.3.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/34/f4/5721faf47b8c499e776bc34c6a8fc17efdf7fdef0b00f398128bc5dcb4ac/fsspec-2025.3.0.tar.gz", hash = "sha256:a935fd1ea872591f2b5148907d103488fc523295e6c64b835cfad8c3eca44972", size = 298491 } +sdist = { url = "https://files.pythonhosted.org/packages/45/d8/8425e6ba5fcec61a1d16e41b1b71d2bf9344f1fe48012c2b48b9620feae5/fsspec-2025.3.2.tar.gz", hash = "sha256:e52c77ef398680bbd6a98c0e628fbc469491282981209907bbc8aea76a04fdc6", size = 299281 } wheels = [ - { url = "https://files.pythonhosted.org/packages/56/53/eb690efa8513166adef3e0669afd31e95ffde69fb3c52ec2ac7223ed6018/fsspec-2025.3.0-py3-none-any.whl", hash = "sha256:efb87af3efa9103f94ca91a7f8cb7a4df91af9f74fc106c9c7ea0efd7277c1b3", size = 193615 }, + { url = "https://files.pythonhosted.org/packages/44/4b/e0cfc1a6f17e990f3e64b7d941ddc4acdc7b19d6edd51abf495f32b1a9e4/fsspec-2025.3.2-py3-none-any.whl", hash = "sha256:2daf8dc3d1dfa65b6aa37748d112773a7a08416f6c70d96b264c96476ecaf711", size = 194435 }, ] [[package]] @@ -654,11 +668,11 @@ wheels = [ [[package]] name = "iniconfig" -version = "2.0.0" +version = "2.1.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/d7/4b/cbd8e699e64a6f16ca3a8220661b5f83792b3017d0f79807cb8708d33913/iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3", size = 4646 } +sdist = { url = "https://files.pythonhosted.org/packages/f2/97/ebf4da567aa6827c909642694d71c9fcf53e5b504f2d96afea02718862f3/iniconfig-2.1.0.tar.gz", hash = "sha256:3abbd2e30b36733fee78f9c7f7308f2d0050e88f0087fd25c2645f63c773e1c7", size = 4793 } wheels = [ - { url = "https://files.pythonhosted.org/packages/ef/a6/62565a6e1cf69e10f5727360368e451d4b7f58beeac6173dc9db836a5b46/iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374", size = 5892 }, + { url = "https://files.pythonhosted.org/packages/2c/e1/e6716421ea10d38022b952c159d5161ca1193197fb744506875fbb87ea7b/iniconfig-2.1.0-py3-none-any.whl", hash = "sha256:9deba5723312380e77435581c6bf4935c94cbfab9b1ed33ef8d238ea168eb760", size = 6050 }, ] [[package]] @@ -819,69 +833,69 @@ wheels = [ [[package]] name = "multidict" -version = "6.1.0" -source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/d6/be/504b89a5e9ca731cd47487e91c469064f8ae5af93b7259758dcfc2b9c848/multidict-6.1.0.tar.gz", hash = "sha256:22ae2ebf9b0c69d206c003e2f6a914ea33f0a932d4aa16f236afc049d9958f4a", size = 64002 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/93/13/df3505a46d0cd08428e4c8169a196131d1b0c4b515c3649829258843dde6/multidict-6.1.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:3efe2c2cb5763f2f1b275ad2bf7a287d3f7ebbef35648a9726e3b69284a4f3d6", size = 48570 }, - { url = "https://files.pythonhosted.org/packages/f0/e1/a215908bfae1343cdb72f805366592bdd60487b4232d039c437fe8f5013d/multidict-6.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c7053d3b0353a8b9de430a4f4b4268ac9a4fb3481af37dfe49825bf45ca24156", size = 29316 }, - { url = "https://files.pythonhosted.org/packages/70/0f/6dc70ddf5d442702ed74f298d69977f904960b82368532c88e854b79f72b/multidict-6.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:27e5fc84ccef8dfaabb09d82b7d179c7cf1a3fbc8a966f8274fcb4ab2eb4cadb", size = 29640 }, - { url = "https://files.pythonhosted.org/packages/d8/6d/9c87b73a13d1cdea30b321ef4b3824449866bd7f7127eceed066ccb9b9ff/multidict-6.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0e2b90b43e696f25c62656389d32236e049568b39320e2735d51f08fd362761b", size = 131067 }, - { url = "https://files.pythonhosted.org/packages/cc/1e/1b34154fef373371fd6c65125b3d42ff5f56c7ccc6bfff91b9b3c60ae9e0/multidict-6.1.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d83a047959d38a7ff552ff94be767b7fd79b831ad1cd9920662db05fec24fe72", size = 138507 }, - { url = "https://files.pythonhosted.org/packages/fb/e0/0bc6b2bac6e461822b5f575eae85da6aae76d0e2a79b6665d6206b8e2e48/multidict-6.1.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d1a9dd711d0877a1ece3d2e4fea11a8e75741ca21954c919406b44e7cf971304", size = 133905 }, - { url = "https://files.pythonhosted.org/packages/ba/af/73d13b918071ff9b2205fcf773d316e0f8fefb4ec65354bbcf0b10908cc6/multidict-6.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ec2abea24d98246b94913b76a125e855eb5c434f7c46546046372fe60f666351", size = 129004 }, - { url = "https://files.pythonhosted.org/packages/74/21/23960627b00ed39643302d81bcda44c9444ebcdc04ee5bedd0757513f259/multidict-6.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4867cafcbc6585e4b678876c489b9273b13e9fff9f6d6d66add5e15d11d926cb", size = 121308 }, - { url = "https://files.pythonhosted.org/packages/8b/5c/cf282263ffce4a596ed0bb2aa1a1dddfe1996d6a62d08842a8d4b33dca13/multidict-6.1.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:5b48204e8d955c47c55b72779802b219a39acc3ee3d0116d5080c388970b76e3", size = 132608 }, - { url = "https://files.pythonhosted.org/packages/d7/3e/97e778c041c72063f42b290888daff008d3ab1427f5b09b714f5a8eff294/multidict-6.1.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:d8fff389528cad1618fb4b26b95550327495462cd745d879a8c7c2115248e399", size = 127029 }, - { url = "https://files.pythonhosted.org/packages/47/ac/3efb7bfe2f3aefcf8d103e9a7162572f01936155ab2f7ebcc7c255a23212/multidict-6.1.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:a7a9541cd308eed5e30318430a9c74d2132e9a8cb46b901326272d780bf2d423", size = 137594 }, - { url = "https://files.pythonhosted.org/packages/42/9b/6c6e9e8dc4f915fc90a9b7798c44a30773dea2995fdcb619870e705afe2b/multidict-6.1.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:da1758c76f50c39a2efd5e9859ce7d776317eb1dd34317c8152ac9251fc574a3", size = 134556 }, - { url = "https://files.pythonhosted.org/packages/1d/10/8e881743b26aaf718379a14ac58572a240e8293a1c9d68e1418fb11c0f90/multidict-6.1.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:c943a53e9186688b45b323602298ab727d8865d8c9ee0b17f8d62d14b56f0753", size = 130993 }, - { url = "https://files.pythonhosted.org/packages/45/84/3eb91b4b557442802d058a7579e864b329968c8d0ea57d907e7023c677f2/multidict-6.1.0-cp311-cp311-win32.whl", hash = "sha256:90f8717cb649eea3504091e640a1b8568faad18bd4b9fcd692853a04475a4b80", size = 26405 }, - { url = "https://files.pythonhosted.org/packages/9f/0b/ad879847ecbf6d27e90a6eabb7eff6b62c129eefe617ea45eae7c1f0aead/multidict-6.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:82176036e65644a6cc5bd619f65f6f19781e8ec2e5330f51aa9ada7504cc1926", size = 28795 }, - { url = "https://files.pythonhosted.org/packages/fd/16/92057c74ba3b96d5e211b553895cd6dc7cc4d1e43d9ab8fafc727681ef71/multidict-6.1.0-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:b04772ed465fa3cc947db808fa306d79b43e896beb677a56fb2347ca1a49c1fa", size = 48713 }, - { url = "https://files.pythonhosted.org/packages/94/3d/37d1b8893ae79716179540b89fc6a0ee56b4a65fcc0d63535c6f5d96f217/multidict-6.1.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:6180c0ae073bddeb5a97a38c03f30c233e0a4d39cd86166251617d1bbd0af436", size = 29516 }, - { url = "https://files.pythonhosted.org/packages/a2/12/adb6b3200c363062f805275b4c1e656be2b3681aada66c80129932ff0bae/multidict-6.1.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:071120490b47aa997cca00666923a83f02c7fbb44f71cf7f136df753f7fa8761", size = 29557 }, - { url = "https://files.pythonhosted.org/packages/47/e9/604bb05e6e5bce1e6a5cf80a474e0f072e80d8ac105f1b994a53e0b28c42/multidict-6.1.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50b3a2710631848991d0bf7de077502e8994c804bb805aeb2925a981de58ec2e", size = 130170 }, - { url = "https://files.pythonhosted.org/packages/7e/13/9efa50801785eccbf7086b3c83b71a4fb501a4d43549c2f2f80b8787d69f/multidict-6.1.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b58c621844d55e71c1b7f7c498ce5aa6985d743a1a59034c57a905b3f153c1ef", size = 134836 }, - { url = "https://files.pythonhosted.org/packages/bf/0f/93808b765192780d117814a6dfcc2e75de6dcc610009ad408b8814dca3ba/multidict-6.1.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:55b6d90641869892caa9ca42ff913f7ff1c5ece06474fbd32fb2cf6834726c95", size = 133475 }, - { url = "https://files.pythonhosted.org/packages/d3/c8/529101d7176fe7dfe1d99604e48d69c5dfdcadb4f06561f465c8ef12b4df/multidict-6.1.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b820514bfc0b98a30e3d85462084779900347e4d49267f747ff54060cc33925", size = 131049 }, - { url = "https://files.pythonhosted.org/packages/ca/0c/fc85b439014d5a58063e19c3a158a889deec399d47b5269a0f3b6a2e28bc/multidict-6.1.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:10a9b09aba0c5b48c53761b7c720aaaf7cf236d5fe394cd399c7ba662d5f9966", size = 120370 }, - { url = "https://files.pythonhosted.org/packages/db/46/d4416eb20176492d2258fbd47b4abe729ff3b6e9c829ea4236f93c865089/multidict-6.1.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:1e16bf3e5fc9f44632affb159d30a437bfe286ce9e02754759be5536b169b305", size = 125178 }, - { url = "https://files.pythonhosted.org/packages/5b/46/73697ad7ec521df7de5531a32780bbfd908ded0643cbe457f981a701457c/multidict-6.1.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:76f364861c3bfc98cbbcbd402d83454ed9e01a5224bb3a28bf70002a230f73e2", size = 119567 }, - { url = "https://files.pythonhosted.org/packages/cd/ed/51f060e2cb0e7635329fa6ff930aa5cffa17f4c7f5c6c3ddc3500708e2f2/multidict-6.1.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:820c661588bd01a0aa62a1283f20d2be4281b086f80dad9e955e690c75fb54a2", size = 129822 }, - { url = "https://files.pythonhosted.org/packages/df/9e/ee7d1954b1331da3eddea0c4e08d9142da5f14b1321c7301f5014f49d492/multidict-6.1.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:0e5f362e895bc5b9e67fe6e4ded2492d8124bdf817827f33c5b46c2fe3ffaca6", size = 128656 }, - { url = "https://files.pythonhosted.org/packages/77/00/8538f11e3356b5d95fa4b024aa566cde7a38aa7a5f08f4912b32a037c5dc/multidict-6.1.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:3ec660d19bbc671e3a6443325f07263be452c453ac9e512f5eb935e7d4ac28b3", size = 125360 }, - { url = "https://files.pythonhosted.org/packages/be/05/5d334c1f2462d43fec2363cd00b1c44c93a78c3925d952e9a71caf662e96/multidict-6.1.0-cp312-cp312-win32.whl", hash = "sha256:58130ecf8f7b8112cdb841486404f1282b9c86ccb30d3519faf301b2e5659133", size = 26382 }, - { url = "https://files.pythonhosted.org/packages/a3/bf/f332a13486b1ed0496d624bcc7e8357bb8053823e8cd4b9a18edc1d97e73/multidict-6.1.0-cp312-cp312-win_amd64.whl", hash = "sha256:188215fc0aafb8e03341995e7c4797860181562380f81ed0a87ff455b70bf1f1", size = 28529 }, - { url = "https://files.pythonhosted.org/packages/99/b7/b9e70fde2c0f0c9af4cc5277782a89b66d35948ea3369ec9f598358c3ac5/multidict-6.1.0-py3-none-any.whl", hash = "sha256:48e171e52d1c4d33888e529b999e5900356b9ae588c2f09a52dcefb158b27506", size = 10051 }, +version = "6.3.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a1/d4/4e0e25aabd94f289b7d47da8293a3563e73ac1a4f7e9caddf11b6eeaf52d/multidict-6.3.1.tar.gz", hash = "sha256:3e18d6afe3f855736022748606def2000af18e90253fb8b4d698b51f61e21283", size = 86832 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b0/88/4609a79411529b054afe07907620624ed58a289cfabb82e8f9c87dfc9e87/multidict-6.3.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:818af177e8ed6040d980b2c931872fbda1cd96780540491324b7bd5b2791d012", size = 62658 }, + { url = "https://files.pythonhosted.org/packages/67/2d/542cb2ffaa39080f5bf5653fe8deae9e092fb519b071daa54b17d4a6c378/multidict-6.3.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:257a96af4b58f94ddc198d414bac15ae43ca55406b79d9e241d0c16da1419ea1", size = 37128 }, + { url = "https://files.pythonhosted.org/packages/cf/f7/6b0a993d3d48ca9a4d5ce008b1789a527dbbddaaf37a6612cd620bd22a85/multidict-6.3.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:b7e44e02c7f383571ec39b2532d6f1c28c7b1af637d1b46bb0a67c4d9a649baa", size = 36274 }, + { url = "https://files.pythonhosted.org/packages/16/3e/76bf90cd68ceb20e9b0576eea79e02885d94339ed176661c7bf199484472/multidict-6.3.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:60d576e62b498c888f83b6a924a560123fb7b76acae0fa629382635dd486112c", size = 242429 }, + { url = "https://files.pythonhosted.org/packages/03/18/006674268ea4edde7f14a1642f1a162a77412e7c34b40f61fb7441e9cbb4/multidict-6.3.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d7407537d36e294c794a7860e16978bb4c2211de2c7772a7e8e10eca47262d8b", size = 255752 }, + { url = "https://files.pythonhosted.org/packages/90/0d/9146ad1ec053bd78d36d75d16ce920075d93acad7306b304e2aa3e881dd3/multidict-6.3.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:81c6c87e015b381f2e3974614da1fdac45cafeaf74beb46821389edee408f540", size = 250856 }, + { url = "https://files.pythonhosted.org/packages/97/e4/924ed6212f1f628e588548d59248660d411c38c212602caab12807f57e08/multidict-6.3.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cad66c35d4d565952eaf55a91be50c2433068ea1e71f0647ffb25222e954e01c", size = 237043 }, + { url = "https://files.pythonhosted.org/packages/ef/70/b957c8ef3645c016ecfa23e37693611e07107ae080b4131fe2a0ecfe52f8/multidict-6.3.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a0a1f61aafda969979806795bd36310bf424ad00fa5ec5e949d13021080e5325", size = 229223 }, + { url = "https://files.pythonhosted.org/packages/c0/9d/4f6af43b54bba7d746c437e016804ec7a4f124e0ad6d17bf40d1280763dd/multidict-6.3.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:a54907daf359affa16a94d56044ac9b565b70b3c57bf6e2d52d2d8a61177eae1", size = 240250 }, + { url = "https://files.pythonhosted.org/packages/cc/e6/2a316f7d99b39b2ab35f6b251746abfb2f7ea9b9b9a75a1958850c819aff/multidict-6.3.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c640be28ee1a51f028793cac0d07414df9a19e0ca5d9ec3cfa872a0f266ca333", size = 235940 }, + { url = "https://files.pythonhosted.org/packages/02/55/86829bf2e81b198af6dbea27f9b621088d3b21e035986f4352e5ba315672/multidict-6.3.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:c3f7c8ce2451f4c820f4ea28c3a3bd794269903eaf0967f19081f2a345e46492", size = 249799 }, + { url = "https://files.pythonhosted.org/packages/ca/8c/cb86eecb10e4e381438ff7b21e42539e430df1bda44f14f5b81d2294284a/multidict-6.3.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:0bb725538b9feb05d5a0f2a657d14a3de2688ecfdb9c06c3773b1e0311352d6b", size = 241848 }, + { url = "https://files.pythonhosted.org/packages/29/1c/7f87a3cca58491c00a1028df35989fac4ea382b17353b9cc56e1c029732b/multidict-6.3.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:17398bf3ec71e24823910fb7ac671dcd29b4d48c54a22d9c65318f80e000256a", size = 237306 }, + { url = "https://files.pythonhosted.org/packages/60/2c/83ff5500901b80af1dc1cd3bf9c9f485a16091d62a61ba60cd63a335494c/multidict-6.3.1-cp311-cp311-win32.whl", hash = "sha256:48bd65b6d905ce11a707908391456fba2c8f3bcab2017ea5042ccd225e4d7d02", size = 34899 }, + { url = "https://files.pythonhosted.org/packages/ff/ca/5575f0e23bfcac4bbb50e72b497b72f9949d1da33b8a2c8933a3ec083789/multidict-6.3.1-cp311-cp311-win_amd64.whl", hash = "sha256:79bdf29cd48606bb55ea989a59a950226a66d3adb34b8d224d6bd0b54781b693", size = 38397 }, + { url = "https://files.pythonhosted.org/packages/62/74/acf7ecc3eb1195b3930008cfaae9ffbc54c71a5582a0c86db291307dd7cf/multidict-6.3.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:9679106633695b132ebc9191ec6230bfb1415d37c483833fcef2b35a2e8665ec", size = 62598 }, + { url = "https://files.pythonhosted.org/packages/d3/78/544442c1bb61986e5cbd529bfb7038b38f88fc39fe17962bc63bb0013199/multidict-6.3.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:73a43b3b2409aa395cce91b7471cff6b45814548063b18412162ba2222084201", size = 37323 }, + { url = "https://files.pythonhosted.org/packages/f5/d1/8260a5da38ac65e885a0adbd40b50113e6f9e8eeb94b91aeeac08e4ad36e/multidict-6.3.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1ce924e24c4f1c014f2ed8782e82a5232d5f61293fc5c204d8569f451191ffa8", size = 36053 }, + { url = "https://files.pythonhosted.org/packages/e7/51/cceca6f30954620b9b03f96e06a54f908367cf86ae0c61a3472aa3de9363/multidict-6.3.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:123b1d48eeed2ac1126be078deb88006f871559787cefc8759a884442a6f2cdc", size = 244641 }, + { url = "https://files.pythonhosted.org/packages/08/df/fb512545fa3cbf20f870175a4698c6ba58abf261ab19faf28d293e39621d/multidict-6.3.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6d98447906885e7f0f90456cde1d14ff41f30d9d7e127ab7140a45e784a0ff1b", size = 255743 }, + { url = "https://files.pythonhosted.org/packages/65/fb/a7648f5764e25e0a18e7d3bdda9fd67e86bb7e0c70a6ffee0348e1fb493f/multidict-6.3.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5072a9efe7f7f79d3dff1f26ac41e4893478f85ce55fe5318625f7eb703d76f8", size = 252217 }, + { url = "https://files.pythonhosted.org/packages/c0/98/32ef5e26956ee8cd7af8d7367b9da11f5a211438498e854bd5385e8eb812/multidict-6.3.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbc825b34595fe43966242e30b54d29617013e51b4310134aa2c16c3b3d00c91", size = 245195 }, + { url = "https://files.pythonhosted.org/packages/4d/e1/a8cf44bef56bee949ec68d993ecbc4b713338b3137fa42416f0e34a46c48/multidict-6.3.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:baec41c191855f92507f9e0bb182eea7eea5992d649f9c712c96a38076e59d00", size = 232457 }, + { url = "https://files.pythonhosted.org/packages/08/7c/81e91ef84b5df88d4780bcd03b08df423668b61ada7b387e0482ac19690a/multidict-6.3.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:eacd4036bb3d632828702a076458d660b53d12e049155eaeb7d11a91242d67b8", size = 252077 }, + { url = "https://files.pythonhosted.org/packages/6c/00/32b94a1b060f602aa9056189febefddfaf6cece4c6f4c5873668011fd67b/multidict-6.3.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:284737db826cc00fbd5292225717492f037afa404a2ddfea812cfbef7a3f0e93", size = 247211 }, + { url = "https://files.pythonhosted.org/packages/9c/78/9994ab4cc9b18c48e089f08c85028ea8c60ffa0d5868d7e42c842b9ca80e/multidict-6.3.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:ebd121433f5d8707379f4fc0e6b4bf67b0b7cd1a7132e097ead2713c8d661a41", size = 260515 }, + { url = "https://files.pythonhosted.org/packages/86/aa/d40c0bce043fa2903e7d3f9e5a2402fd55850933bc81f86a08efe78e303d/multidict-6.3.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:31f94d64672487570c7c2bbcff74311055066e013545714b938786843eb54ef8", size = 255476 }, + { url = "https://files.pythonhosted.org/packages/86/1b/20580f901b260c2d6733e5ec3e1e227e04330de966b567b3f3e102567bd0/multidict-6.3.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:739fe3fde8b8aca7219048f8bda17901afb8710c93307dc0d740014d3481b36b", size = 251311 }, + { url = "https://files.pythonhosted.org/packages/9c/44/db7cb84b604ee1912d5ba908def729adc60413448e789247db98992c149f/multidict-6.3.1-cp312-cp312-win32.whl", hash = "sha256:891a94a056de2d904cc30f40ec1d111aebb09abd33089a34631ff5a19e0167b2", size = 35031 }, + { url = "https://files.pythonhosted.org/packages/32/bb/2931b3d6a2b57b5a7dbb819c2b5c55d3170c54098009224872d4b6ae40d3/multidict-6.3.1-cp312-cp312-win_amd64.whl", hash = "sha256:d9844e0f93405a9c5bc2106d48cf82e022e18685baebea74cc5057ca2009799e", size = 38453 }, + { url = "https://files.pythonhosted.org/packages/57/99/624da94f4deb41a75b5e08752270ecdb1ff871c1b1539705c0aef02aa7a2/multidict-6.3.1-py3-none-any.whl", hash = "sha256:2d45b070b33fa1d0a9a7650469997713e3a4f5cd9eb564332d5d0206cf61efc5", size = 10348 }, ] [[package]] name = "numpy" -version = "2.2.3" -source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/fb/90/8956572f5c4ae52201fdec7ba2044b2c882832dcec7d5d0922c9e9acf2de/numpy-2.2.3.tar.gz", hash = "sha256:dbdc15f0c81611925f382dfa97b3bd0bc2c1ce19d4fe50482cb0ddc12ba30020", size = 20262700 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/96/86/453aa3949eab6ff54e2405f9cb0c01f756f031c3dc2a6d60a1d40cba5488/numpy-2.2.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:16372619ee728ed67a2a606a614f56d3eabc5b86f8b615c79d01957062826ca8", size = 21237256 }, - { url = "https://files.pythonhosted.org/packages/20/c3/93ecceadf3e155d6a9e4464dd2392d8d80cf436084c714dc8535121c83e8/numpy-2.2.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5521a06a3148686d9269c53b09f7d399a5725c47bbb5b35747e1cb76326b714b", size = 14408049 }, - { url = "https://files.pythonhosted.org/packages/8d/29/076999b69bd9264b8df5e56f2be18da2de6b2a2d0e10737e5307592e01de/numpy-2.2.3-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:7c8dde0ca2f77828815fd1aedfdf52e59071a5bae30dac3b4da2a335c672149a", size = 5408655 }, - { url = "https://files.pythonhosted.org/packages/e2/a7/b14f0a73eb0fe77cb9bd5b44534c183b23d4229c099e339c522724b02678/numpy-2.2.3-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:77974aba6c1bc26e3c205c2214f0d5b4305bdc719268b93e768ddb17e3fdd636", size = 6949996 }, - { url = "https://files.pythonhosted.org/packages/72/2f/8063da0616bb0f414b66dccead503bd96e33e43685c820e78a61a214c098/numpy-2.2.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d42f9c36d06440e34226e8bd65ff065ca0963aeecada587b937011efa02cdc9d", size = 14355789 }, - { url = "https://files.pythonhosted.org/packages/e6/d7/3cd47b00b8ea95ab358c376cf5602ad21871410950bc754cf3284771f8b6/numpy-2.2.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f2712c5179f40af9ddc8f6727f2bd910ea0eb50206daea75f58ddd9fa3f715bb", size = 16411356 }, - { url = "https://files.pythonhosted.org/packages/27/c0/a2379e202acbb70b85b41483a422c1e697ff7eee74db642ca478de4ba89f/numpy-2.2.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:c8b0451d2ec95010d1db8ca733afc41f659f425b7f608af569711097fd6014e2", size = 15576770 }, - { url = "https://files.pythonhosted.org/packages/bc/63/a13ee650f27b7999e5b9e1964ae942af50bb25606d088df4229283eda779/numpy-2.2.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:d9b4a8148c57ecac25a16b0e11798cbe88edf5237b0df99973687dd866f05e1b", size = 18200483 }, - { url = "https://files.pythonhosted.org/packages/4c/87/e71f89935e09e8161ac9c590c82f66d2321eb163893a94af749dfa8a3cf8/numpy-2.2.3-cp311-cp311-win32.whl", hash = "sha256:1f45315b2dc58d8a3e7754fe4e38b6fce132dab284a92851e41b2b344f6441c5", size = 6588415 }, - { url = "https://files.pythonhosted.org/packages/b9/c6/cd4298729826af9979c5f9ab02fcaa344b82621e7c49322cd2d210483d3f/numpy-2.2.3-cp311-cp311-win_amd64.whl", hash = "sha256:9f48ba6f6c13e5e49f3d3efb1b51c8193215c42ac82610a04624906a9270be6f", size = 12929604 }, - { url = "https://files.pythonhosted.org/packages/43/ec/43628dcf98466e087812142eec6d1c1a6c6bdfdad30a0aa07b872dc01f6f/numpy-2.2.3-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:12c045f43b1d2915eca6b880a7f4a256f59d62df4f044788c8ba67709412128d", size = 20929458 }, - { url = "https://files.pythonhosted.org/packages/9b/c0/2f4225073e99a5c12350954949ed19b5d4a738f541d33e6f7439e33e98e4/numpy-2.2.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:87eed225fd415bbae787f93a457af7f5990b92a334e346f72070bf569b9c9c95", size = 14115299 }, - { url = "https://files.pythonhosted.org/packages/ca/fa/d2c5575d9c734a7376cc1592fae50257ec95d061b27ee3dbdb0b3b551eb2/numpy-2.2.3-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:712a64103d97c404e87d4d7c47fb0c7ff9acccc625ca2002848e0d53288b90ea", size = 5145723 }, - { url = "https://files.pythonhosted.org/packages/eb/dc/023dad5b268a7895e58e791f28dc1c60eb7b6c06fcbc2af8538ad069d5f3/numpy-2.2.3-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:a5ae282abe60a2db0fd407072aff4599c279bcd6e9a2475500fc35b00a57c532", size = 6678797 }, - { url = "https://files.pythonhosted.org/packages/3f/19/bcd641ccf19ac25abb6fb1dcd7744840c11f9d62519d7057b6ab2096eb60/numpy-2.2.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5266de33d4c3420973cf9ae3b98b54a2a6d53a559310e3236c4b2b06b9c07d4e", size = 14067362 }, - { url = "https://files.pythonhosted.org/packages/39/04/78d2e7402fb479d893953fb78fa7045f7deb635ec095b6b4f0260223091a/numpy-2.2.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b787adbf04b0db1967798dba8da1af07e387908ed1553a0d6e74c084d1ceafe", size = 16116679 }, - { url = "https://files.pythonhosted.org/packages/d0/a1/e90f7aa66512be3150cb9d27f3d9995db330ad1b2046474a13b7040dfd92/numpy-2.2.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:34c1b7e83f94f3b564b35f480f5652a47007dd91f7c839f404d03279cc8dd021", size = 15264272 }, - { url = "https://files.pythonhosted.org/packages/dc/b6/50bd027cca494de4fa1fc7bf1662983d0ba5f256fa0ece2c376b5eb9b3f0/numpy-2.2.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:4d8335b5f1b6e2bce120d55fb17064b0262ff29b459e8493d1785c18ae2553b8", size = 17880549 }, - { url = "https://files.pythonhosted.org/packages/96/30/f7bf4acb5f8db10a96f73896bdeed7a63373137b131ca18bd3dab889db3b/numpy-2.2.3-cp312-cp312-win32.whl", hash = "sha256:4d9828d25fb246bedd31e04c9e75714a4087211ac348cb39c8c5f99dbb6683fe", size = 6293394 }, - { url = "https://files.pythonhosted.org/packages/42/6e/55580a538116d16ae7c9aa17d4edd56e83f42126cb1dfe7a684da7925d2c/numpy-2.2.3-cp312-cp312-win_amd64.whl", hash = "sha256:83807d445817326b4bcdaaaf8e8e9f1753da04341eceec705c001ff342002e5d", size = 12626357 }, +version = "2.2.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/e1/78/31103410a57bc2c2b93a3597340a8119588571f6a4539067546cb9a0bfac/numpy-2.2.4.tar.gz", hash = "sha256:9ba03692a45d3eef66559efe1d1096c4b9b75c0986b5dff5530c378fb8331d4f", size = 20270701 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/16/fb/09e778ee3a8ea0d4dc8329cca0a9c9e65fed847d08e37eba74cb7ed4b252/numpy-2.2.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:e9e0a277bb2eb5d8a7407e14688b85fd8ad628ee4e0c7930415687b6564207a4", size = 21254989 }, + { url = "https://files.pythonhosted.org/packages/a2/0a/1212befdbecab5d80eca3cde47d304cad986ad4eec7d85a42e0b6d2cc2ef/numpy-2.2.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9eeea959168ea555e556b8188da5fa7831e21d91ce031e95ce23747b7609f8a4", size = 14425910 }, + { url = "https://files.pythonhosted.org/packages/2b/3e/e7247c1d4f15086bb106c8d43c925b0b2ea20270224f5186fa48d4fb5cbd/numpy-2.2.4-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:bd3ad3b0a40e713fc68f99ecfd07124195333f1e689387c180813f0e94309d6f", size = 5426490 }, + { url = "https://files.pythonhosted.org/packages/5d/fa/aa7cd6be51419b894c5787a8a93c3302a1ed4f82d35beb0613ec15bdd0e2/numpy-2.2.4-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:cf28633d64294969c019c6df4ff37f5698e8326db68cc2b66576a51fad634880", size = 6967754 }, + { url = "https://files.pythonhosted.org/packages/d5/ee/96457c943265de9fadeb3d2ffdbab003f7fba13d971084a9876affcda095/numpy-2.2.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2fa8fa7697ad1646b5c93de1719965844e004fcad23c91228aca1cf0800044a1", size = 14373079 }, + { url = "https://files.pythonhosted.org/packages/c5/5c/ceefca458559f0ccc7a982319f37ed07b0d7b526964ae6cc61f8ad1b6119/numpy-2.2.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f4162988a360a29af158aeb4a2f4f09ffed6a969c9776f8f3bdee9b06a8ab7e5", size = 16428819 }, + { url = "https://files.pythonhosted.org/packages/22/31/9b2ac8eee99e001eb6add9fa27514ef5e9faf176169057a12860af52704c/numpy-2.2.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:892c10d6a73e0f14935c31229e03325a7b3093fafd6ce0af704be7f894d95687", size = 15881470 }, + { url = "https://files.pythonhosted.org/packages/f0/dc/8569b5f25ff30484b555ad8a3f537e0225d091abec386c9420cf5f7a2976/numpy-2.2.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:db1f1c22173ac1c58db249ae48aa7ead29f534b9a948bc56828337aa84a32ed6", size = 18218144 }, + { url = "https://files.pythonhosted.org/packages/5e/05/463c023a39bdeb9bb43a99e7dee2c664cb68d5bb87d14f92482b9f6011cc/numpy-2.2.4-cp311-cp311-win32.whl", hash = "sha256:ea2bb7e2ae9e37d96835b3576a4fa4b3a97592fbea8ef7c3587078b0068b8f09", size = 6606368 }, + { url = "https://files.pythonhosted.org/packages/8b/72/10c1d2d82101c468a28adc35de6c77b308f288cfd0b88e1070f15b98e00c/numpy-2.2.4-cp311-cp311-win_amd64.whl", hash = "sha256:f7de08cbe5551911886d1ab60de58448c6df0f67d9feb7d1fb21e9875ef95e91", size = 12947526 }, + { url = "https://files.pythonhosted.org/packages/a2/30/182db21d4f2a95904cec1a6f779479ea1ac07c0647f064dea454ec650c42/numpy-2.2.4-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:a7b9084668aa0f64e64bd00d27ba5146ef1c3a8835f3bd912e7a9e01326804c4", size = 20947156 }, + { url = "https://files.pythonhosted.org/packages/24/6d/9483566acfbda6c62c6bc74b6e981c777229d2af93c8eb2469b26ac1b7bc/numpy-2.2.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:dbe512c511956b893d2dacd007d955a3f03d555ae05cfa3ff1c1ff6df8851854", size = 14133092 }, + { url = "https://files.pythonhosted.org/packages/27/f6/dba8a258acbf9d2bed2525cdcbb9493ef9bae5199d7a9cb92ee7e9b2aea6/numpy-2.2.4-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:bb649f8b207ab07caebba230d851b579a3c8711a851d29efe15008e31bb4de24", size = 5163515 }, + { url = "https://files.pythonhosted.org/packages/62/30/82116199d1c249446723c68f2c9da40d7f062551036f50b8c4caa42ae252/numpy-2.2.4-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:f34dc300df798742b3d06515aa2a0aee20941c13579d7a2f2e10af01ae4901ee", size = 6696558 }, + { url = "https://files.pythonhosted.org/packages/0e/b2/54122b3c6df5df3e87582b2e9430f1bdb63af4023c739ba300164c9ae503/numpy-2.2.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3f7ac96b16955634e223b579a3e5798df59007ca43e8d451a0e6a50f6bfdfba", size = 14084742 }, + { url = "https://files.pythonhosted.org/packages/02/e2/e2cbb8d634151aab9528ef7b8bab52ee4ab10e076509285602c2a3a686e0/numpy-2.2.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f92084defa704deadd4e0a5ab1dc52d8ac9e8a8ef617f3fbb853e79b0ea3592", size = 16134051 }, + { url = "https://files.pythonhosted.org/packages/8e/21/efd47800e4affc993e8be50c1b768de038363dd88865920439ef7b422c60/numpy-2.2.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7a4e84a6283b36632e2a5b56e121961f6542ab886bc9e12f8f9818b3c266bfbb", size = 15578972 }, + { url = "https://files.pythonhosted.org/packages/04/1e/f8bb88f6157045dd5d9b27ccf433d016981032690969aa5c19e332b138c0/numpy-2.2.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:11c43995255eb4127115956495f43e9343736edb7fcdb0d973defd9de14cd84f", size = 17898106 }, + { url = "https://files.pythonhosted.org/packages/2b/93/df59a5a3897c1f036ae8ff845e45f4081bb06943039ae28a3c1c7c780f22/numpy-2.2.4-cp312-cp312-win32.whl", hash = "sha256:65ef3468b53269eb5fdb3a5c09508c032b793da03251d5f8722b1194f1790c00", size = 6311190 }, + { url = "https://files.pythonhosted.org/packages/46/69/8c4f928741c2a8efa255fdc7e9097527c6dc4e4df147e3cadc5d9357ce85/numpy-2.2.4-cp312-cp312-win_amd64.whl", hash = "sha256:2aad3c17ed2ff455b8eaafe06bcdae0062a1db77cb99f4b9cbb5f4ecb13c5146", size = 12644305 }, ] [[package]] @@ -921,6 +935,19 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/29/d4/1244ab8edf173a10fd601f7e13b9566c1b525c4f365d6bee918e68381889/pandas-2.2.3-cp312-cp312-win_amd64.whl", hash = "sha256:59ef3764d0fe818125a5097d2ae867ca3fa64df032331b7e0917cf5d7bf66b13", size = 11504248 }, ] +[[package]] +name = "pandas-stubs" +version = "2.2.3.250308" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "numpy" }, + { name = "types-pytz" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/2e/5a/261f5c67a73e46df2d5984fe7129d66a3ed4864fd7aa9d8721abb3fc802e/pandas_stubs-2.2.3.250308.tar.gz", hash = "sha256:3a6e9daf161f00b85c83772ed3d5cff9522028f07a94817472c07b91f46710fd", size = 103986 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ba/64/ab61d9ca06ff66c07eb804ec27dec1a2be1978b3c3767caaa91e363438cc/pandas_stubs-2.2.3.250308-py3-none-any.whl", hash = "sha256:a377edff3b61f8b268c82499fdbe7c00fdeed13235b8b71d6a1dc347aeddc74d", size = 158053 }, +] + [[package]] name = "parso" version = "0.8.4" @@ -967,16 +994,16 @@ wheels = [ [[package]] name = "polars" -version = "1.24.0" +version = "1.26.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/bf/d3/453bcecbe14a5aba6be47c99d81f4e1f941d3de729d5e0ce5c7d527c05ed/polars-1.24.0.tar.gz", hash = "sha256:6e7553789495081c998f5e4ad4ebc7e19e970a9cc83326d40461564e85ad226d", size = 4446066 } +sdist = { url = "https://files.pythonhosted.org/packages/72/c2/56a750b82e74c5af7b821215eab3ee992d9ef11310dad353f5b2086ef7db/polars-1.26.0.tar.gz", hash = "sha256:b5492d38e5ec2ae6a8853833c5a31549194a361b901134fc5f2f57b49bd563ea", size = 4522991 } wheels = [ - { url = "https://files.pythonhosted.org/packages/a9/3f/16f87d9ec4707d717a434bc54307506594522de99fdfe3d5d76233912c94/polars-1.24.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:563b99a6597fe77a3c89d478e4a6fb49c063f44ef84d4adefe490e14626e2f99", size = 33792674 }, - { url = "https://files.pythonhosted.org/packages/35/cd/27353d0b9331d60a95f5708370441348d4a3af0f609961ceaaa3b583190f/polars-1.24.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:6ad64d938d60b7fda39b60892ef67bc6a9942e0c7170db593a65d019e8730b09", size = 30469541 }, - { url = "https://files.pythonhosted.org/packages/a7/db/668d8328b1c3d8381023fc4ed905a88b93cca041c088f42a94dbd6822469/polars-1.24.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:331e737465b8d954bec51e6906bdc6e979a6ee52f97ffe5e8d0c10794a46bfd9", size = 34313540 }, - { url = "https://files.pythonhosted.org/packages/12/16/f95207616b2e802c381459cf01f0d233daa98bdc4e394ec88044af9e927f/polars-1.24.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:3c6c774aebdd5cd601839594986648789352f72b8893f4b7e34224e75b060c8d", size = 31490266 }, - { url = "https://files.pythonhosted.org/packages/8b/82/cb0512747ec5508a4f840a521feb27f28a81eac1aef6c92fc25643073579/polars-1.24.0-cp39-abi3-win_amd64.whl", hash = "sha256:a5a473ff44fe1b9e3e7a9013a9321efe841d858e89cf33d424e6f3fef3ea4d5e", size = 34678593 }, - { url = "https://files.pythonhosted.org/packages/67/00/db3810803938467a215c1f161ff21ad6fef581d5ac1381ee2990d0180c19/polars-1.24.0-cp39-abi3-win_arm64.whl", hash = "sha256:5ea781ca8e0a39c3b677171dbd852e5fa2d5c53417b5fbd69d711b6044a49eaa", size = 30892251 }, + { url = "https://files.pythonhosted.org/packages/4a/99/ce4427576a6134504e6dd58f5d411d55b228a05950c5fff5b75e90263cb1/polars-1.26.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:2afefcd356608981b2e15d46df9ddaa6e77f36095ebeb73c3261e198bd51c925", size = 34767938 }, + { url = "https://files.pythonhosted.org/packages/e3/0a/5c9455ff271c3583bf0fd505911e5787ca7bc0f247968853cb6dcfbedffb/polars-1.26.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:587eb3c5000423eb20be998f523e605ddba0d3c598ba4a7e2a4d0b92b1fd2a7e", size = 31612374 }, + { url = "https://files.pythonhosted.org/packages/6c/0a/c9e388b35533fc1827eaeb0f3940ba0a1058511bf77aaa689ebb1b0bef88/polars-1.26.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:66c30f4b7e060c2e7f3a45d6ac94ab3b179831a2f1e629401bf7912d54311529", size = 35404704 }, + { url = "https://files.pythonhosted.org/packages/e6/eb/b420131563a42ac0866224aa6284a356107d036c32d825c5478767a1446e/polars-1.26.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:110d6987d37ae954a5ef16d739fb717df9d39b144790d12d98fb3e72ed35621c", size = 32569800 }, + { url = "https://files.pythonhosted.org/packages/e9/8d/b28ae5d63a4bafdfe81d132efc6b8d076ef2c866ecbf5d24ca3b1f53b88b/polars-1.26.0-cp39-abi3-win_amd64.whl", hash = "sha256:189a58aaf393003515fa6d83e2dea815a2b448265f2007a926274ed12672583c", size = 35617074 }, + { url = "https://files.pythonhosted.org/packages/af/21/7a76c58203f0806cb5b1155e4994693e618d4de5ca6a34388ae85267ccc4/polars-1.26.0-cp39-abi3-win_arm64.whl", hash = "sha256:58db2dce39cad5f8fc8e8c5c923a250eb21eff4146b03514d570d1c205a4874c", size = 31895936 }, ] [package.optional-dependencies] @@ -998,57 +1025,57 @@ wheels = [ [[package]] name = "propcache" -version = "0.3.0" -source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/92/76/f941e63d55c0293ff7829dd21e7cf1147e90a526756869a9070f287a68c9/propcache-0.3.0.tar.gz", hash = "sha256:a8fd93de4e1d278046345f49e2238cdb298589325849b2645d4a94c53faeffc5", size = 42722 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/45/c9/cf09ff7e6d09f14149094f7cd50d2dec032b24e61af21fc4540da2b17bfb/propcache-0.3.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9ddd49258610499aab83b4f5b61b32e11fce873586282a0e972e5ab3bcadee51", size = 79568 }, - { url = "https://files.pythonhosted.org/packages/c8/32/2424d89da88cd81b7d148e0d2b3131461b570a02aa9d84a2e567509adb0d/propcache-0.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:2578541776769b500bada3f8a4eeaf944530516b6e90c089aa368266ed70c49e", size = 45895 }, - { url = "https://files.pythonhosted.org/packages/f6/91/ee5b6aa7aa31754fefcf0c5180e09223cac380ef195c4ddc8c266eb641ea/propcache-0.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d8074c5dd61c8a3e915fa8fc04754fa55cfa5978200d2daa1e2d4294c1f136aa", size = 45427 }, - { url = "https://files.pythonhosted.org/packages/bf/73/38f0128462b8b616181d8c53bd5d04eac41c50c449b07615c65d56ba0a9b/propcache-0.3.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b58229a844931bca61b3a20efd2be2a2acb4ad1622fc026504309a6883686fbf", size = 232427 }, - { url = "https://files.pythonhosted.org/packages/59/82/f3d4e84f4539dcfc9c3d338282b9e915f5b63c921986ecfdf7af2d12f87c/propcache-0.3.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e45377d5d6fefe1677da2a2c07b024a6dac782088e37c0b1efea4cfe2b1be19b", size = 239985 }, - { url = "https://files.pythonhosted.org/packages/42/e8/029f58cccbae83c9969a7ee7a06558d5b83a93dfc54e0f4f70234bbaea1b/propcache-0.3.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ec5060592d83454e8063e487696ac3783cc48c9a329498bafae0d972bc7816c9", size = 238827 }, - { url = "https://files.pythonhosted.org/packages/8b/a2/c373561777c0cb9b9e7b9b9a10b9b3a7b6bde75a2535b962231cecc8fdb8/propcache-0.3.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15010f29fbed80e711db272909a074dc79858c6d28e2915704cfc487a8ac89c6", size = 231348 }, - { url = "https://files.pythonhosted.org/packages/d7/d2/4673f715beedf6038b485bcd976813149231d9df5bb6196cb69a09c185c9/propcache-0.3.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a254537b9b696ede293bfdbc0a65200e8e4507bc9f37831e2a0318a9b333c85c", size = 220426 }, - { url = "https://files.pythonhosted.org/packages/e0/f6/1da65f900927bafd4675a16e890618ec7643f2f922bf0e4d84bb38645618/propcache-0.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2b975528998de037dfbc10144b8aed9b8dd5a99ec547f14d1cb7c5665a43f075", size = 220294 }, - { url = "https://files.pythonhosted.org/packages/ff/86/620451bdc02e91b1712cd71890c17077ee97e2a28493836a87e47b8e70ff/propcache-0.3.0-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:19d36bb351ad5554ff20f2ae75f88ce205b0748c38b146c75628577020351e3c", size = 212492 }, - { url = "https://files.pythonhosted.org/packages/6e/1b/e8f86921ed4016da80faf3b8f515f7829decabdbff106736bfff353bceba/propcache-0.3.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:6032231d4a5abd67c7f71168fd64a47b6b451fbcb91c8397c2f7610e67683810", size = 215113 }, - { url = "https://files.pythonhosted.org/packages/1a/95/a61d86cc49aa0945f6c06f3a4614fc543e311a50558c92861f5e9691a37c/propcache-0.3.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:6985a593417cdbc94c7f9c3403747335e450c1599da1647a5af76539672464d3", size = 228330 }, - { url = "https://files.pythonhosted.org/packages/8f/7d/10dbae48ff2bb189e92c2b3487a48f3229146a25941ad0d485934d1104d4/propcache-0.3.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:6a1948df1bb1d56b5e7b0553c0fa04fd0e320997ae99689488201f19fa90d2e7", size = 231942 }, - { url = "https://files.pythonhosted.org/packages/39/ce/82d16aec96c5513ae7db13ab901a65a1e54c915292fb5b2390e33275b61d/propcache-0.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:8319293e85feadbbfe2150a5659dbc2ebc4afdeaf7d98936fb9a2f2ba0d4c35c", size = 223077 }, - { url = "https://files.pythonhosted.org/packages/c8/e0/cb077e8e7a583c733df7f53327fcbdb92e42be59b976ce60bf1d904a0efe/propcache-0.3.0-cp311-cp311-win32.whl", hash = "sha256:63f26258a163c34542c24808f03d734b338da66ba91f410a703e505c8485791d", size = 40455 }, - { url = "https://files.pythonhosted.org/packages/d8/35/57abeb6146fe3c19081eeaf3d9d4cfea256f87f1e5101acf80d3332c1820/propcache-0.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:cacea77ef7a2195f04f9279297684955e3d1ae4241092ff0cfcef532bb7a1c32", size = 44705 }, - { url = "https://files.pythonhosted.org/packages/8d/2c/921f15dc365796ec23975b322b0078eae72995c7b4d49eba554c6a308d70/propcache-0.3.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:e53d19c2bf7d0d1e6998a7e693c7e87300dd971808e6618964621ccd0e01fe4e", size = 79867 }, - { url = "https://files.pythonhosted.org/packages/11/a5/4a6cc1a559d1f2fb57ea22edc4245158cdffae92f7f92afcee2913f84417/propcache-0.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:a61a68d630e812b67b5bf097ab84e2cd79b48c792857dc10ba8a223f5b06a2af", size = 46109 }, - { url = "https://files.pythonhosted.org/packages/e1/6d/28bfd3af3a567ad7d667348e7f46a520bda958229c4d545ba138a044232f/propcache-0.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:fb91d20fa2d3b13deea98a690534697742029f4fb83673a3501ae6e3746508b5", size = 45635 }, - { url = "https://files.pythonhosted.org/packages/73/20/d75b42eaffe5075eac2f4e168f6393d21c664c91225288811d85451b2578/propcache-0.3.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:67054e47c01b7b349b94ed0840ccae075449503cf1fdd0a1fdd98ab5ddc2667b", size = 242159 }, - { url = "https://files.pythonhosted.org/packages/a5/fb/4b537dd92f9fd4be68042ec51c9d23885ca5fafe51ec24c58d9401034e5f/propcache-0.3.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:997e7b8f173a391987df40f3b52c423e5850be6f6df0dcfb5376365440b56667", size = 248163 }, - { url = "https://files.pythonhosted.org/packages/e7/af/8a9db04ac596d531ca0ef7dde518feaadfcdabef7b17d6a5ec59ee3effc2/propcache-0.3.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8d663fd71491dde7dfdfc899d13a067a94198e90695b4321084c6e450743b8c7", size = 248794 }, - { url = "https://files.pythonhosted.org/packages/9d/c4/ecfc988879c0fd9db03228725b662d76cf484b6b46f7e92fee94e4b52490/propcache-0.3.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8884ba1a0fe7210b775106b25850f5e5a9dc3c840d1ae9924ee6ea2eb3acbfe7", size = 243912 }, - { url = "https://files.pythonhosted.org/packages/04/a2/298dd27184faa8b7d91cc43488b578db218b3cc85b54d912ed27b8c5597a/propcache-0.3.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:aa806bbc13eac1ab6291ed21ecd2dd426063ca5417dd507e6be58de20e58dfcf", size = 229402 }, - { url = "https://files.pythonhosted.org/packages/be/0d/efe7fec316ca92dbf4bc4a9ba49ca889c43ca6d48ab1d6fa99fc94e5bb98/propcache-0.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6f4d7a7c0aff92e8354cceca6fe223973ddf08401047920df0fcb24be2bd5138", size = 226896 }, - { url = "https://files.pythonhosted.org/packages/60/63/72404380ae1d9c96d96e165aa02c66c2aae6072d067fc4713da5cde96762/propcache-0.3.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:9be90eebc9842a93ef8335291f57b3b7488ac24f70df96a6034a13cb58e6ff86", size = 221447 }, - { url = "https://files.pythonhosted.org/packages/9d/18/b8392cab6e0964b67a30a8f4dadeaff64dc7022b5a34bb1d004ea99646f4/propcache-0.3.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:bf15fc0b45914d9d1b706f7c9c4f66f2b7b053e9517e40123e137e8ca8958b3d", size = 222440 }, - { url = "https://files.pythonhosted.org/packages/6f/be/105d9ceda0f97eff8c06bac1673448b2db2a497444de3646464d3f5dc881/propcache-0.3.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:5a16167118677d94bb48bfcd91e420088854eb0737b76ec374b91498fb77a70e", size = 234104 }, - { url = "https://files.pythonhosted.org/packages/cb/c9/f09a4ec394cfcce4053d8b2a04d622b5f22d21ba9bb70edd0cad061fa77b/propcache-0.3.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:41de3da5458edd5678b0f6ff66691507f9885f5fe6a0fb99a5d10d10c0fd2d64", size = 239086 }, - { url = "https://files.pythonhosted.org/packages/ea/aa/96f7f9ed6def82db67c972bdb7bd9f28b95d7d98f7e2abaf144c284bf609/propcache-0.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:728af36011bb5d344c4fe4af79cfe186729efb649d2f8b395d1572fb088a996c", size = 230991 }, - { url = "https://files.pythonhosted.org/packages/5a/11/bee5439de1307d06fad176f7143fec906e499c33d7aff863ea8428b8e98b/propcache-0.3.0-cp312-cp312-win32.whl", hash = "sha256:6b5b7fd6ee7b54e01759f2044f936dcf7dea6e7585f35490f7ca0420fe723c0d", size = 40337 }, - { url = "https://files.pythonhosted.org/packages/e4/17/e5789a54a0455a61cb9efc4ca6071829d992220c2998a27c59aeba749f6f/propcache-0.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:2d15bc27163cd4df433e75f546b9ac31c1ba7b0b128bfb1b90df19082466ff57", size = 44404 }, - { url = "https://files.pythonhosted.org/packages/b5/35/6c4c6fc8774a9e3629cd750dc24a7a4fb090a25ccd5c3246d127b70f9e22/propcache-0.3.0-py3-none-any.whl", hash = "sha256:67dda3c7325691c2081510e92c561f465ba61b975f481735aefdfc845d2cd043", size = 12101 }, +version = "0.3.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/07/c8/fdc6686a986feae3541ea23dcaa661bd93972d3940460646c6bb96e21c40/propcache-0.3.1.tar.gz", hash = "sha256:40d980c33765359098837527e18eddefc9a24cea5b45e078a7f3bb5b032c6ecf", size = 43651 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/90/0f/5a5319ee83bd651f75311fcb0c492c21322a7fc8f788e4eef23f44243427/propcache-0.3.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:7f30241577d2fef2602113b70ef7231bf4c69a97e04693bde08ddab913ba0ce5", size = 80243 }, + { url = "https://files.pythonhosted.org/packages/ce/84/3db5537e0879942783e2256616ff15d870a11d7ac26541336fe1b673c818/propcache-0.3.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:43593c6772aa12abc3af7784bff4a41ffa921608dd38b77cf1dfd7f5c4e71371", size = 46503 }, + { url = "https://files.pythonhosted.org/packages/e2/c8/b649ed972433c3f0d827d7f0cf9ea47162f4ef8f4fe98c5f3641a0bc63ff/propcache-0.3.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a75801768bbe65499495660b777e018cbe90c7980f07f8aa57d6be79ea6f71da", size = 45934 }, + { url = "https://files.pythonhosted.org/packages/59/f9/4c0a5cf6974c2c43b1a6810c40d889769cc8f84cea676cbe1e62766a45f8/propcache-0.3.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f6f1324db48f001c2ca26a25fa25af60711e09b9aaf4b28488602776f4f9a744", size = 233633 }, + { url = "https://files.pythonhosted.org/packages/e7/64/66f2f4d1b4f0007c6e9078bd95b609b633d3957fe6dd23eac33ebde4b584/propcache-0.3.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5cdb0f3e1eb6dfc9965d19734d8f9c481b294b5274337a8cb5cb01b462dcb7e0", size = 241124 }, + { url = "https://files.pythonhosted.org/packages/aa/bf/7b8c9fd097d511638fa9b6af3d986adbdf567598a567b46338c925144c1b/propcache-0.3.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1eb34d90aac9bfbced9a58b266f8946cb5935869ff01b164573a7634d39fbcb5", size = 240283 }, + { url = "https://files.pythonhosted.org/packages/fa/c9/e85aeeeaae83358e2a1ef32d6ff50a483a5d5248bc38510d030a6f4e2816/propcache-0.3.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f35c7070eeec2cdaac6fd3fe245226ed2a6292d3ee8c938e5bb645b434c5f256", size = 232498 }, + { url = "https://files.pythonhosted.org/packages/8e/66/acb88e1f30ef5536d785c283af2e62931cb934a56a3ecf39105887aa8905/propcache-0.3.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b23c11c2c9e6d4e7300c92e022046ad09b91fd00e36e83c44483df4afa990073", size = 221486 }, + { url = "https://files.pythonhosted.org/packages/f5/f9/233ddb05ffdcaee4448508ee1d70aa7deff21bb41469ccdfcc339f871427/propcache-0.3.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:3e19ea4ea0bf46179f8a3652ac1426e6dcbaf577ce4b4f65be581e237340420d", size = 222675 }, + { url = "https://files.pythonhosted.org/packages/98/b8/eb977e28138f9e22a5a789daf608d36e05ed93093ef12a12441030da800a/propcache-0.3.1-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:bd39c92e4c8f6cbf5f08257d6360123af72af9f4da75a690bef50da77362d25f", size = 215727 }, + { url = "https://files.pythonhosted.org/packages/89/2d/5f52d9c579f67b8ee1edd9ec073c91b23cc5b7ff7951a1e449e04ed8fdf3/propcache-0.3.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:b0313e8b923b3814d1c4a524c93dfecea5f39fa95601f6a9b1ac96cd66f89ea0", size = 217878 }, + { url = "https://files.pythonhosted.org/packages/7a/fd/5283e5ed8a82b00c7a989b99bb6ea173db1ad750bf0bf8dff08d3f4a4e28/propcache-0.3.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:e861ad82892408487be144906a368ddbe2dc6297074ade2d892341b35c59844a", size = 230558 }, + { url = "https://files.pythonhosted.org/packages/90/38/ab17d75938ef7ac87332c588857422ae126b1c76253f0f5b1242032923ca/propcache-0.3.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:61014615c1274df8da5991a1e5da85a3ccb00c2d4701ac6f3383afd3ca47ab0a", size = 233754 }, + { url = "https://files.pythonhosted.org/packages/06/5d/3b921b9c60659ae464137508d3b4c2b3f52f592ceb1964aa2533b32fcf0b/propcache-0.3.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:71ebe3fe42656a2328ab08933d420df5f3ab121772eef78f2dc63624157f0ed9", size = 226088 }, + { url = "https://files.pythonhosted.org/packages/54/6e/30a11f4417d9266b5a464ac5a8c5164ddc9dd153dfa77bf57918165eb4ae/propcache-0.3.1-cp311-cp311-win32.whl", hash = "sha256:58aa11f4ca8b60113d4b8e32d37e7e78bd8af4d1a5b5cb4979ed856a45e62005", size = 40859 }, + { url = "https://files.pythonhosted.org/packages/1d/3a/8a68dd867da9ca2ee9dfd361093e9cb08cb0f37e5ddb2276f1b5177d7731/propcache-0.3.1-cp311-cp311-win_amd64.whl", hash = "sha256:9532ea0b26a401264b1365146c440a6d78269ed41f83f23818d4b79497aeabe7", size = 45153 }, + { url = "https://files.pythonhosted.org/packages/41/aa/ca78d9be314d1e15ff517b992bebbed3bdfef5b8919e85bf4940e57b6137/propcache-0.3.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:f78eb8422acc93d7b69964012ad7048764bb45a54ba7a39bb9e146c72ea29723", size = 80430 }, + { url = "https://files.pythonhosted.org/packages/1a/d8/f0c17c44d1cda0ad1979af2e593ea290defdde9eaeb89b08abbe02a5e8e1/propcache-0.3.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:89498dd49c2f9a026ee057965cdf8192e5ae070ce7d7a7bd4b66a8e257d0c976", size = 46637 }, + { url = "https://files.pythonhosted.org/packages/ae/bd/c1e37265910752e6e5e8a4c1605d0129e5b7933c3dc3cf1b9b48ed83b364/propcache-0.3.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:09400e98545c998d57d10035ff623266927cb784d13dd2b31fd33b8a5316b85b", size = 46123 }, + { url = "https://files.pythonhosted.org/packages/d4/b0/911eda0865f90c0c7e9f0415d40a5bf681204da5fd7ca089361a64c16b28/propcache-0.3.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aa8efd8c5adc5a2c9d3b952815ff8f7710cefdcaf5f2c36d26aff51aeca2f12f", size = 243031 }, + { url = "https://files.pythonhosted.org/packages/0a/06/0da53397c76a74271621807265b6eb61fb011451b1ddebf43213df763669/propcache-0.3.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c2fe5c910f6007e716a06d269608d307b4f36e7babee5f36533722660e8c4a70", size = 249100 }, + { url = "https://files.pythonhosted.org/packages/f1/eb/13090e05bf6b963fc1653cdc922133ced467cb4b8dab53158db5a37aa21e/propcache-0.3.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a0ab8cf8cdd2194f8ff979a43ab43049b1df0b37aa64ab7eca04ac14429baeb7", size = 250170 }, + { url = "https://files.pythonhosted.org/packages/3b/4c/f72c9e1022b3b043ec7dc475a0f405d4c3e10b9b1d378a7330fecf0652da/propcache-0.3.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:563f9d8c03ad645597b8d010ef4e9eab359faeb11a0a2ac9f7b4bc8c28ebef25", size = 245000 }, + { url = "https://files.pythonhosted.org/packages/e8/fd/970ca0e22acc829f1adf5de3724085e778c1ad8a75bec010049502cb3a86/propcache-0.3.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fb6e0faf8cb6b4beea5d6ed7b5a578254c6d7df54c36ccd3d8b3eb00d6770277", size = 230262 }, + { url = "https://files.pythonhosted.org/packages/c4/42/817289120c6b9194a44f6c3e6b2c3277c5b70bbad39e7df648f177cc3634/propcache-0.3.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:1c5c7ab7f2bb3f573d1cb921993006ba2d39e8621019dffb1c5bc94cdbae81e8", size = 236772 }, + { url = "https://files.pythonhosted.org/packages/7c/9c/3b3942b302badd589ad6b672da3ca7b660a6c2f505cafd058133ddc73918/propcache-0.3.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:050b571b2e96ec942898f8eb46ea4bfbb19bd5502424747e83badc2d4a99a44e", size = 231133 }, + { url = "https://files.pythonhosted.org/packages/98/a1/75f6355f9ad039108ff000dfc2e19962c8dea0430da9a1428e7975cf24b2/propcache-0.3.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:e1c4d24b804b3a87e9350f79e2371a705a188d292fd310e663483af6ee6718ee", size = 230741 }, + { url = "https://files.pythonhosted.org/packages/67/0c/3e82563af77d1f8731132166da69fdfd95e71210e31f18edce08a1eb11ea/propcache-0.3.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:e4fe2a6d5ce975c117a6bb1e8ccda772d1e7029c1cca1acd209f91d30fa72815", size = 244047 }, + { url = "https://files.pythonhosted.org/packages/f7/50/9fb7cca01532a08c4d5186d7bb2da6c4c587825c0ae134b89b47c7d62628/propcache-0.3.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:feccd282de1f6322f56f6845bf1207a537227812f0a9bf5571df52bb418d79d5", size = 246467 }, + { url = "https://files.pythonhosted.org/packages/a9/02/ccbcf3e1c604c16cc525309161d57412c23cf2351523aedbb280eb7c9094/propcache-0.3.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ec314cde7314d2dd0510c6787326bbffcbdc317ecee6b7401ce218b3099075a7", size = 241022 }, + { url = "https://files.pythonhosted.org/packages/db/19/e777227545e09ca1e77a6e21274ae9ec45de0f589f0ce3eca2a41f366220/propcache-0.3.1-cp312-cp312-win32.whl", hash = "sha256:7d2d5a0028d920738372630870e7d9644ce437142197f8c827194fca404bf03b", size = 40647 }, + { url = "https://files.pythonhosted.org/packages/24/bb/3b1b01da5dd04c77a204c84e538ff11f624e31431cfde7201d9110b092b1/propcache-0.3.1-cp312-cp312-win_amd64.whl", hash = "sha256:88c423efef9d7a59dae0614eaed718449c09a5ac79a5f224a8b9664d603f04a3", size = 44784 }, + { url = "https://files.pythonhosted.org/packages/b8/d3/c3cb8f1d6ae3b37f83e1de806713a9b3642c5895f0215a62e1a4bd6e5e34/propcache-0.3.1-py3-none-any.whl", hash = "sha256:9a8ecf38de50a7f518c21568c80f985e776397b902f1ce0b01f799aba1608b40", size = 12376 }, ] [[package]] name = "protobuf" -version = "5.29.3" +version = "5.29.4" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/f7/d1/e0a911544ca9993e0f17ce6d3cc0932752356c1b0a834397f28e63479344/protobuf-5.29.3.tar.gz", hash = "sha256:5da0f41edaf117bde316404bad1a486cb4ededf8e4a54891296f648e8e076620", size = 424945 } +sdist = { url = "https://files.pythonhosted.org/packages/17/7d/b9dca7365f0e2c4fa7c193ff795427cfa6290147e5185ab11ece280a18e7/protobuf-5.29.4.tar.gz", hash = "sha256:4f1dfcd7997b31ef8f53ec82781ff434a28bf71d9102ddde14d076adcfc78c99", size = 424902 } wheels = [ - { url = "https://files.pythonhosted.org/packages/dc/7a/1e38f3cafa022f477ca0f57a1f49962f21ad25850c3ca0acd3b9d0091518/protobuf-5.29.3-cp310-abi3-win32.whl", hash = "sha256:3ea51771449e1035f26069c4c7fd51fba990d07bc55ba80701c78f886bf9c888", size = 422708 }, - { url = "https://files.pythonhosted.org/packages/61/fa/aae8e10512b83de633f2646506a6d835b151edf4b30d18d73afd01447253/protobuf-5.29.3-cp310-abi3-win_amd64.whl", hash = "sha256:a4fa6f80816a9a0678429e84973f2f98cbc218cca434abe8db2ad0bffc98503a", size = 434508 }, - { url = "https://files.pythonhosted.org/packages/dd/04/3eaedc2ba17a088961d0e3bd396eac764450f431621b58a04ce898acd126/protobuf-5.29.3-cp38-abi3-macosx_10_9_universal2.whl", hash = "sha256:a8434404bbf139aa9e1300dbf989667a83d42ddda9153d8ab76e0d5dcaca484e", size = 417825 }, - { url = "https://files.pythonhosted.org/packages/4f/06/7c467744d23c3979ce250397e26d8ad8eeb2bea7b18ca12ad58313c1b8d5/protobuf-5.29.3-cp38-abi3-manylinux2014_aarch64.whl", hash = "sha256:daaf63f70f25e8689c072cfad4334ca0ac1d1e05a92fc15c54eb9cf23c3efd84", size = 319573 }, - { url = "https://files.pythonhosted.org/packages/a8/45/2ebbde52ad2be18d3675b6bee50e68cd73c9e0654de77d595540b5129df8/protobuf-5.29.3-cp38-abi3-manylinux2014_x86_64.whl", hash = "sha256:c027e08a08be10b67c06bf2370b99c811c466398c357e615ca88c91c07f0910f", size = 319672 }, - { url = "https://files.pythonhosted.org/packages/fd/b2/ab07b09e0f6d143dfb839693aa05765257bceaa13d03bf1a696b78323e7a/protobuf-5.29.3-py3-none-any.whl", hash = "sha256:0a18ed4a24198528f2333802eb075e59dea9d679ab7a6c5efb017a59004d849f", size = 172550 }, + { url = "https://files.pythonhosted.org/packages/9a/b2/043a1a1a20edd134563699b0e91862726a0dc9146c090743b6c44d798e75/protobuf-5.29.4-cp310-abi3-win32.whl", hash = "sha256:13eb236f8eb9ec34e63fc8b1d6efd2777d062fa6aaa68268fb67cf77f6839ad7", size = 422709 }, + { url = "https://files.pythonhosted.org/packages/79/fc/2474b59570daa818de6124c0a15741ee3e5d6302e9d6ce0bdfd12e98119f/protobuf-5.29.4-cp310-abi3-win_amd64.whl", hash = "sha256:bcefcdf3976233f8a502d265eb65ea740c989bacc6c30a58290ed0e519eb4b8d", size = 434506 }, + { url = "https://files.pythonhosted.org/packages/46/de/7c126bbb06aa0f8a7b38aaf8bd746c514d70e6a2a3f6dd460b3b7aad7aae/protobuf-5.29.4-cp38-abi3-macosx_10_9_universal2.whl", hash = "sha256:307ecba1d852ec237e9ba668e087326a67564ef83e45a0189a772ede9e854dd0", size = 417826 }, + { url = "https://files.pythonhosted.org/packages/a2/b5/bade14ae31ba871a139aa45e7a8183d869efe87c34a4850c87b936963261/protobuf-5.29.4-cp38-abi3-manylinux2014_aarch64.whl", hash = "sha256:aec4962f9ea93c431d5714ed1be1c93f13e1a8618e70035ba2b0564d9e633f2e", size = 319574 }, + { url = "https://files.pythonhosted.org/packages/46/88/b01ed2291aae68b708f7d334288ad5fb3e7aa769a9c309c91a0d55cb91b0/protobuf-5.29.4-cp38-abi3-manylinux2014_x86_64.whl", hash = "sha256:d7d3f7d1d5a66ed4942d4fefb12ac4b14a29028b209d4bfb25c68ae172059922", size = 319672 }, + { url = "https://files.pythonhosted.org/packages/12/fb/a586e0c973c95502e054ac5f81f88394f24ccc7982dac19c515acd9e2c93/protobuf-5.29.4-py3-none-any.whl", hash = "sha256:3fde11b505e1597f71b875ef2fc52062b6a9740e5f7c8997ce878b6009145862", size = 172551 }, ] [[package]] @@ -1103,55 +1130,65 @@ wheels = [ [[package]] name = "pydantic" -version = "2.10.6" +version = "2.11.1" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "annotated-types" }, { name = "pydantic-core" }, { name = "typing-extensions" }, + { name = "typing-inspection" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b7/ae/d5220c5c52b158b1de7ca89fc5edb72f304a70a4c540c84c8844bf4008de/pydantic-2.10.6.tar.gz", hash = "sha256:ca5daa827cce33de7a42be142548b0096bf05a7e7b365aebfa5f8eeec7128236", size = 761681 } +sdist = { url = "https://files.pythonhosted.org/packages/93/a3/698b87a4d4d303d7c5f62ea5fbf7a79cab236ccfbd0a17847b7f77f8163e/pydantic-2.11.1.tar.gz", hash = "sha256:442557d2910e75c991c39f4b4ab18963d57b9b55122c8b2a9cd176d8c29ce968", size = 782817 } wheels = [ - { url = "https://files.pythonhosted.org/packages/f4/3c/8cc1cc84deffa6e25d2d0c688ebb80635dfdbf1dbea3e30c541c8cf4d860/pydantic-2.10.6-py3-none-any.whl", hash = "sha256:427d664bf0b8a2b34ff5dd0f5a18df00591adcee7198fbd71981054cef37b584", size = 431696 }, + { url = "https://files.pythonhosted.org/packages/cc/12/f9221a949f2419e2e23847303c002476c26fbcfd62dc7f3d25d0bec5ca99/pydantic-2.11.1-py3-none-any.whl", hash = "sha256:5b6c415eee9f8123a14d859be0c84363fec6b1feb6b688d6435801230b56e0b8", size = 442648 }, ] [[package]] name = "pydantic-core" -version = "2.27.2" +version = "2.33.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/fc/01/f3e5ac5e7c25833db5eb555f7b7ab24cd6f8c322d3a3ad2d67a952dc0abc/pydantic_core-2.27.2.tar.gz", hash = "sha256:eb026e5a4c1fee05726072337ff51d1efb6f59090b7da90d30ea58625b1ffb39", size = 413443 } -wheels = [ - { url = "https://files.pythonhosted.org/packages/c2/89/f3450af9d09d44eea1f2c369f49e8f181d742f28220f88cc4dfaae91ea6e/pydantic_core-2.27.2-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:8e10c99ef58cfdf2a66fc15d66b16c4a04f62bca39db589ae8cba08bc55331bc", size = 1893421 }, - { url = "https://files.pythonhosted.org/packages/9e/e3/71fe85af2021f3f386da42d291412e5baf6ce7716bd7101ea49c810eda90/pydantic_core-2.27.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:26f32e0adf166a84d0cb63be85c562ca8a6fa8de28e5f0d92250c6b7e9e2aff7", size = 1814998 }, - { url = "https://files.pythonhosted.org/packages/a6/3c/724039e0d848fd69dbf5806894e26479577316c6f0f112bacaf67aa889ac/pydantic_core-2.27.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8c19d1ea0673cd13cc2f872f6c9ab42acc4e4f492a7ca9d3795ce2b112dd7e15", size = 1826167 }, - { url = "https://files.pythonhosted.org/packages/2b/5b/1b29e8c1fb5f3199a9a57c1452004ff39f494bbe9bdbe9a81e18172e40d3/pydantic_core-2.27.2-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:5e68c4446fe0810e959cdff46ab0a41ce2f2c86d227d96dc3847af0ba7def306", size = 1865071 }, - { url = "https://files.pythonhosted.org/packages/89/6c/3985203863d76bb7d7266e36970d7e3b6385148c18a68cc8915fd8c84d57/pydantic_core-2.27.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d9640b0059ff4f14d1f37321b94061c6db164fbe49b334b31643e0528d100d99", size = 2036244 }, - { url = "https://files.pythonhosted.org/packages/0e/41/f15316858a246b5d723f7d7f599f79e37493b2e84bfc789e58d88c209f8a/pydantic_core-2.27.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:40d02e7d45c9f8af700f3452f329ead92da4c5f4317ca9b896de7ce7199ea459", size = 2737470 }, - { url = "https://files.pythonhosted.org/packages/a8/7c/b860618c25678bbd6d1d99dbdfdf0510ccb50790099b963ff78a124b754f/pydantic_core-2.27.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1c1fd185014191700554795c99b347d64f2bb637966c4cfc16998a0ca700d048", size = 1992291 }, - { url = "https://files.pythonhosted.org/packages/bf/73/42c3742a391eccbeab39f15213ecda3104ae8682ba3c0c28069fbcb8c10d/pydantic_core-2.27.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d81d2068e1c1228a565af076598f9e7451712700b673de8f502f0334f281387d", size = 1994613 }, - { url = "https://files.pythonhosted.org/packages/94/7a/941e89096d1175d56f59340f3a8ebaf20762fef222c298ea96d36a6328c5/pydantic_core-2.27.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:1a4207639fb02ec2dbb76227d7c751a20b1a6b4bc52850568e52260cae64ca3b", size = 2002355 }, - { url = "https://files.pythonhosted.org/packages/6e/95/2359937a73d49e336a5a19848713555605d4d8d6940c3ec6c6c0ca4dcf25/pydantic_core-2.27.2-cp311-cp311-musllinux_1_1_armv7l.whl", hash = "sha256:3de3ce3c9ddc8bbd88f6e0e304dea0e66d843ec9de1b0042b0911c1663ffd474", size = 2126661 }, - { url = "https://files.pythonhosted.org/packages/2b/4c/ca02b7bdb6012a1adef21a50625b14f43ed4d11f1fc237f9d7490aa5078c/pydantic_core-2.27.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:30c5f68ded0c36466acede341551106821043e9afaad516adfb6e8fa80a4e6a6", size = 2153261 }, - { url = "https://files.pythonhosted.org/packages/72/9d/a241db83f973049a1092a079272ffe2e3e82e98561ef6214ab53fe53b1c7/pydantic_core-2.27.2-cp311-cp311-win32.whl", hash = "sha256:c70c26d2c99f78b125a3459f8afe1aed4d9687c24fd677c6a4436bc042e50d6c", size = 1812361 }, - { url = "https://files.pythonhosted.org/packages/e8/ef/013f07248041b74abd48a385e2110aa3a9bbfef0fbd97d4e6d07d2f5b89a/pydantic_core-2.27.2-cp311-cp311-win_amd64.whl", hash = "sha256:08e125dbdc505fa69ca7d9c499639ab6407cfa909214d500897d02afb816e7cc", size = 1982484 }, - { url = "https://files.pythonhosted.org/packages/10/1c/16b3a3e3398fd29dca77cea0a1d998d6bde3902fa2706985191e2313cc76/pydantic_core-2.27.2-cp311-cp311-win_arm64.whl", hash = "sha256:26f0d68d4b235a2bae0c3fc585c585b4ecc51382db0e3ba402a22cbc440915e4", size = 1867102 }, - { url = "https://files.pythonhosted.org/packages/d6/74/51c8a5482ca447871c93e142d9d4a92ead74de6c8dc5e66733e22c9bba89/pydantic_core-2.27.2-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:9e0c8cfefa0ef83b4da9588448b6d8d2a2bf1a53c3f1ae5fca39eb3061e2f0b0", size = 1893127 }, - { url = "https://files.pythonhosted.org/packages/d3/f3/c97e80721735868313c58b89d2de85fa80fe8dfeeed84dc51598b92a135e/pydantic_core-2.27.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:83097677b8e3bd7eaa6775720ec8e0405f1575015a463285a92bfdfe254529ef", size = 1811340 }, - { url = "https://files.pythonhosted.org/packages/9e/91/840ec1375e686dbae1bd80a9e46c26a1e0083e1186abc610efa3d9a36180/pydantic_core-2.27.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:172fce187655fece0c90d90a678424b013f8fbb0ca8b036ac266749c09438cb7", size = 1822900 }, - { url = "https://files.pythonhosted.org/packages/f6/31/4240bc96025035500c18adc149aa6ffdf1a0062a4b525c932065ceb4d868/pydantic_core-2.27.2-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:519f29f5213271eeeeb3093f662ba2fd512b91c5f188f3bb7b27bc5973816934", size = 1869177 }, - { url = "https://files.pythonhosted.org/packages/fa/20/02fbaadb7808be578317015c462655c317a77a7c8f0ef274bc016a784c54/pydantic_core-2.27.2-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:05e3a55d124407fffba0dd6b0c0cd056d10e983ceb4e5dbd10dda135c31071d6", size = 2038046 }, - { url = "https://files.pythonhosted.org/packages/06/86/7f306b904e6c9eccf0668248b3f272090e49c275bc488a7b88b0823444a4/pydantic_core-2.27.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9c3ed807c7b91de05e63930188f19e921d1fe90de6b4f5cd43ee7fcc3525cb8c", size = 2685386 }, - { url = "https://files.pythonhosted.org/packages/8d/f0/49129b27c43396581a635d8710dae54a791b17dfc50c70164866bbf865e3/pydantic_core-2.27.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6fb4aadc0b9a0c063206846d603b92030eb6f03069151a625667f982887153e2", size = 1997060 }, - { url = "https://files.pythonhosted.org/packages/0d/0f/943b4af7cd416c477fd40b187036c4f89b416a33d3cc0ab7b82708a667aa/pydantic_core-2.27.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:28ccb213807e037460326424ceb8b5245acb88f32f3d2777427476e1b32c48c4", size = 2004870 }, - { url = "https://files.pythonhosted.org/packages/35/40/aea70b5b1a63911c53a4c8117c0a828d6790483f858041f47bab0b779f44/pydantic_core-2.27.2-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:de3cd1899e2c279b140adde9357c4495ed9d47131b4a4eaff9052f23398076b3", size = 1999822 }, - { url = "https://files.pythonhosted.org/packages/f2/b3/807b94fd337d58effc5498fd1a7a4d9d59af4133e83e32ae39a96fddec9d/pydantic_core-2.27.2-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:220f892729375e2d736b97d0e51466252ad84c51857d4d15f5e9692f9ef12be4", size = 2130364 }, - { url = "https://files.pythonhosted.org/packages/fc/df/791c827cd4ee6efd59248dca9369fb35e80a9484462c33c6649a8d02b565/pydantic_core-2.27.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:a0fcd29cd6b4e74fe8ddd2c90330fd8edf2e30cb52acda47f06dd615ae72da57", size = 2158303 }, - { url = "https://files.pythonhosted.org/packages/9b/67/4e197c300976af185b7cef4c02203e175fb127e414125916bf1128b639a9/pydantic_core-2.27.2-cp312-cp312-win32.whl", hash = "sha256:1e2cb691ed9834cd6a8be61228471d0a503731abfb42f82458ff27be7b2186fc", size = 1834064 }, - { url = "https://files.pythonhosted.org/packages/1f/ea/cd7209a889163b8dcca139fe32b9687dd05249161a3edda62860430457a5/pydantic_core-2.27.2-cp312-cp312-win_amd64.whl", hash = "sha256:cc3f1a99a4f4f9dd1de4fe0312c114e740b5ddead65bb4102884b384c15d8bc9", size = 1989046 }, - { url = "https://files.pythonhosted.org/packages/bc/49/c54baab2f4658c26ac633d798dab66b4c3a9bbf47cff5284e9c182f4137a/pydantic_core-2.27.2-cp312-cp312-win_arm64.whl", hash = "sha256:3911ac9284cd8a1792d3cb26a2da18f3ca26c6908cc434a18f730dc0db7bfa3b", size = 1885092 }, +sdist = { url = "https://files.pythonhosted.org/packages/b9/05/91ce14dfd5a3a99555fce436318cc0fd1f08c4daa32b3248ad63669ea8b4/pydantic_core-2.33.0.tar.gz", hash = "sha256:40eb8af662ba409c3cbf4a8150ad32ae73514cd7cb1f1a2113af39763dd616b3", size = 434080 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f0/93/9e97af2619b4026596487a79133e425c7d3c374f0a7f100f3d76bcdf9c83/pydantic_core-2.33.0-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:a608a75846804271cf9c83e40bbb4dab2ac614d33c6fd5b0c6187f53f5c593ef", size = 2042784 }, + { url = "https://files.pythonhosted.org/packages/42/b4/0bba8412fd242729feeb80e7152e24f0e1a1c19f4121ca3d4a307f4e6222/pydantic_core-2.33.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e1c69aa459f5609dec2fa0652d495353accf3eda5bdb18782bc5a2ae45c9273a", size = 1858179 }, + { url = "https://files.pythonhosted.org/packages/69/1f/c1c40305d929bd08af863df64b0a26203b70b352a1962d86f3bcd52950fe/pydantic_core-2.33.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b9ec80eb5a5f45a2211793f1c4aeddff0c3761d1c70d684965c1807e923a588b", size = 1909396 }, + { url = "https://files.pythonhosted.org/packages/0f/99/d2e727375c329c1e652b5d450fbb9d56e8c3933a397e4bd46e67c68c2cd5/pydantic_core-2.33.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:e925819a98318d17251776bd3d6aa9f3ff77b965762155bdad15d1a9265c4cfd", size = 1998264 }, + { url = "https://files.pythonhosted.org/packages/9c/2e/3119a33931278d96ecc2e9e1b9d50c240636cfeb0c49951746ae34e4de74/pydantic_core-2.33.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5bf68bb859799e9cec3d9dd8323c40c00a254aabb56fe08f907e437005932f2b", size = 2140588 }, + { url = "https://files.pythonhosted.org/packages/35/bd/9267bd1ba55f17c80ef6cb7e07b3890b4acbe8eb6014f3102092d53d9300/pydantic_core-2.33.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1b2ea72dea0825949a045fa4071f6d5b3d7620d2a208335207793cf29c5a182d", size = 2746296 }, + { url = "https://files.pythonhosted.org/packages/6f/ed/ef37de6478a412ee627cbebd73e7b72a680f45bfacce9ff1199de6e17e88/pydantic_core-2.33.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1583539533160186ac546b49f5cde9ffc928062c96920f58bd95de32ffd7bffd", size = 2005555 }, + { url = "https://files.pythonhosted.org/packages/dd/84/72c8d1439585d8ee7bc35eb8f88a04a4d302ee4018871f1f85ae1b0c6625/pydantic_core-2.33.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:23c3e77bf8a7317612e5c26a3b084c7edeb9552d645742a54a5867635b4f2453", size = 2124452 }, + { url = "https://files.pythonhosted.org/packages/a7/8f/cb13de30c6a3e303423751a529a3d1271c2effee4b98cf3e397a66ae8498/pydantic_core-2.33.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a7a7f2a3f628d2f7ef11cb6188bcf0b9e1558151d511b974dfea10a49afe192b", size = 2087001 }, + { url = "https://files.pythonhosted.org/packages/83/d0/e93dc8884bf288a63fedeb8040ac8f29cb71ca52e755f48e5170bb63e55b/pydantic_core-2.33.0-cp311-cp311-musllinux_1_1_armv7l.whl", hash = "sha256:f1fb026c575e16f673c61c7b86144517705865173f3d0907040ac30c4f9f5915", size = 2261663 }, + { url = "https://files.pythonhosted.org/packages/4c/ba/4b7739c95efa0b542ee45fd872c8f6b1884ab808cf04ce7ac6621b6df76e/pydantic_core-2.33.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:635702b2fed997e0ac256b2cfbdb4dd0bf7c56b5d8fba8ef03489c03b3eb40e2", size = 2257786 }, + { url = "https://files.pythonhosted.org/packages/cc/98/73cbca1d2360c27752cfa2fcdcf14d96230e92d7d48ecd50499865c56bf7/pydantic_core-2.33.0-cp311-cp311-win32.whl", hash = "sha256:07b4ced28fccae3f00626eaa0c4001aa9ec140a29501770a88dbbb0966019a86", size = 1925697 }, + { url = "https://files.pythonhosted.org/packages/9a/26/d85a40edeca5d8830ffc33667d6fef329fd0f4bc0c5181b8b0e206cfe488/pydantic_core-2.33.0-cp311-cp311-win_amd64.whl", hash = "sha256:4927564be53239a87770a5f86bdc272b8d1fbb87ab7783ad70255b4ab01aa25b", size = 1949859 }, + { url = "https://files.pythonhosted.org/packages/7e/0b/5a381605f0b9870465b805f2c86c06b0a7c191668ebe4117777306c2c1e5/pydantic_core-2.33.0-cp311-cp311-win_arm64.whl", hash = "sha256:69297418ad644d521ea3e1aa2e14a2a422726167e9ad22b89e8f1130d68e1e9a", size = 1907978 }, + { url = "https://files.pythonhosted.org/packages/a9/c4/c9381323cbdc1bb26d352bc184422ce77c4bc2f2312b782761093a59fafc/pydantic_core-2.33.0-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:6c32a40712e3662bebe524abe8abb757f2fa2000028d64cc5a1006016c06af43", size = 2025127 }, + { url = "https://files.pythonhosted.org/packages/6f/bd/af35278080716ecab8f57e84515c7dc535ed95d1c7f52c1c6f7b313a9dab/pydantic_core-2.33.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8ec86b5baa36f0a0bfb37db86c7d52652f8e8aa076ab745ef7725784183c3fdd", size = 1851687 }, + { url = "https://files.pythonhosted.org/packages/12/e4/a01461225809c3533c23bd1916b1e8c2e21727f0fea60ab1acbffc4e2fca/pydantic_core-2.33.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4deac83a8cc1d09e40683be0bc6d1fa4cde8df0a9bf0cda5693f9b0569ac01b6", size = 1892232 }, + { url = "https://files.pythonhosted.org/packages/51/17/3d53d62a328fb0a49911c2962036b9e7a4f781b7d15e9093c26299e5f76d/pydantic_core-2.33.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:175ab598fb457a9aee63206a1993874badf3ed9a456e0654273e56f00747bbd6", size = 1977896 }, + { url = "https://files.pythonhosted.org/packages/30/98/01f9d86e02ec4a38f4b02086acf067f2c776b845d43f901bd1ee1c21bc4b/pydantic_core-2.33.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5f36afd0d56a6c42cf4e8465b6441cf546ed69d3a4ec92724cc9c8c61bd6ecf4", size = 2127717 }, + { url = "https://files.pythonhosted.org/packages/3c/43/6f381575c61b7c58b0fd0b92134c5a1897deea4cdfc3d47567b3ff460a4e/pydantic_core-2.33.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0a98257451164666afafc7cbf5fb00d613e33f7e7ebb322fbcd99345695a9a61", size = 2680287 }, + { url = "https://files.pythonhosted.org/packages/01/42/c0d10d1451d161a9a0da9bbef023b8005aa26e9993a8cc24dc9e3aa96c93/pydantic_core-2.33.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ecc6d02d69b54a2eb83ebcc6f29df04957f734bcf309d346b4f83354d8376862", size = 2008276 }, + { url = "https://files.pythonhosted.org/packages/20/ca/e08df9dba546905c70bae44ced9f3bea25432e34448d95618d41968f40b7/pydantic_core-2.33.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:1a69b7596c6603afd049ce7f3835bcf57dd3892fc7279f0ddf987bebed8caa5a", size = 2115305 }, + { url = "https://files.pythonhosted.org/packages/03/1f/9b01d990730a98833113581a78e595fd40ed4c20f9693f5a658fb5f91eff/pydantic_core-2.33.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:ea30239c148b6ef41364c6f51d103c2988965b643d62e10b233b5efdca8c0099", size = 2068999 }, + { url = "https://files.pythonhosted.org/packages/20/18/fe752476a709191148e8b1e1139147841ea5d2b22adcde6ee6abb6c8e7cf/pydantic_core-2.33.0-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:abfa44cf2f7f7d7a199be6c6ec141c9024063205545aa09304349781b9a125e6", size = 2241488 }, + { url = "https://files.pythonhosted.org/packages/81/22/14738ad0a0bf484b928c9e52004f5e0b81dd8dabbdf23b843717b37a71d1/pydantic_core-2.33.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:20d4275f3c4659d92048c70797e5fdc396c6e4446caf517ba5cad2db60cd39d3", size = 2248430 }, + { url = "https://files.pythonhosted.org/packages/e8/27/be7571e215ac8d321712f2433c445b03dbcd645366a18f67b334df8912bc/pydantic_core-2.33.0-cp312-cp312-win32.whl", hash = "sha256:918f2013d7eadea1d88d1a35fd4a1e16aaf90343eb446f91cb091ce7f9b431a2", size = 1908353 }, + { url = "https://files.pythonhosted.org/packages/be/3a/be78f28732f93128bd0e3944bdd4b3970b389a1fbd44907c97291c8dcdec/pydantic_core-2.33.0-cp312-cp312-win_amd64.whl", hash = "sha256:aec79acc183865bad120b0190afac467c20b15289050648b876b07777e67ea48", size = 1955956 }, + { url = "https://files.pythonhosted.org/packages/21/26/b8911ac74faa994694b76ee6a22875cc7a4abea3c381fdba4edc6c6bef84/pydantic_core-2.33.0-cp312-cp312-win_arm64.whl", hash = "sha256:5461934e895968655225dfa8b3be79e7e927e95d4bd6c2d40edd2fa7052e71b6", size = 1903259 }, + { url = "https://files.pythonhosted.org/packages/2b/b2/553e42762e7b08771fca41c0230c1ac276f9e79e78f57628e1b7d328551d/pydantic_core-2.33.0-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:5d8dc9f63a26f7259b57f46a7aab5af86b2ad6fbe48487500bb1f4b27e051e4c", size = 2041207 }, + { url = "https://files.pythonhosted.org/packages/85/81/a91a57bbf3efe53525ab75f65944b8950e6ef84fe3b9a26c1ec173363263/pydantic_core-2.33.0-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:30369e54d6d0113d2aa5aee7a90d17f225c13d87902ace8fcd7bbf99b19124db", size = 1873736 }, + { url = "https://files.pythonhosted.org/packages/9c/d2/5ab52e9f551cdcbc1ee99a0b3ef595f56d031f66f88e5ca6726c49f9ce65/pydantic_core-2.33.0-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f3eb479354c62067afa62f53bb387827bee2f75c9c79ef25eef6ab84d4b1ae3b", size = 1903794 }, + { url = "https://files.pythonhosted.org/packages/2f/5f/a81742d3f3821b16f1265f057d6e0b68a3ab13a814fe4bffac536a1f26fd/pydantic_core-2.33.0-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0310524c833d91403c960b8a3cf9f46c282eadd6afd276c8c5edc617bd705dc9", size = 2083457 }, + { url = "https://files.pythonhosted.org/packages/b5/2f/e872005bc0fc47f9c036b67b12349a8522d32e3bda928e82d676e2a594d1/pydantic_core-2.33.0-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:eddb18a00bbb855325db27b4c2a89a4ba491cd6a0bd6d852b225172a1f54b36c", size = 2119537 }, + { url = "https://files.pythonhosted.org/packages/d3/13/183f13ce647202eaf3dada9e42cdfc59cbb95faedd44d25f22b931115c7f/pydantic_core-2.33.0-pp311-pypy311_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:ade5dbcf8d9ef8f4b28e682d0b29f3008df9842bb5ac48ac2c17bc55771cc976", size = 2080069 }, + { url = "https://files.pythonhosted.org/packages/23/8b/b6be91243da44a26558d9c3a9007043b3750334136c6550551e8092d6d96/pydantic_core-2.33.0-pp311-pypy311_pp73-musllinux_1_1_armv7l.whl", hash = "sha256:2c0afd34f928383e3fd25740f2050dbac9d077e7ba5adbaa2227f4d4f3c8da5c", size = 2251618 }, + { url = "https://files.pythonhosted.org/packages/aa/c5/fbcf1977035b834f63eb542e74cd6c807177f383386175b468f0865bcac4/pydantic_core-2.33.0-pp311-pypy311_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:7da333f21cd9df51d5731513a6d39319892947604924ddf2e24a4612975fb936", size = 2255374 }, + { url = "https://files.pythonhosted.org/packages/2f/f8/66f328e411f1c9574b13c2c28ab01f308b53688bbbe6ca8fb981e6cabc42/pydantic_core-2.33.0-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:4b6d77c75a57f041c5ee915ff0b0bb58eabb78728b69ed967bc5b780e8f701b8", size = 2082099 }, ] [[package]] @@ -1213,33 +1250,33 @@ wheels = [ [[package]] name = "python-dotenv" -version = "1.0.1" +version = "1.1.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/bc/57/e84d88dfe0aec03b7a2d4327012c1627ab5f03652216c63d49846d7a6c58/python-dotenv-1.0.1.tar.gz", hash = "sha256:e324ee90a023d808f1959c46bcbc04446a10ced277783dc6ee09987c37ec10ca", size = 39115 } +sdist = { url = "https://files.pythonhosted.org/packages/88/2c/7bb1416c5620485aa793f2de31d3df393d3686aa8a8506d11e10e13c5baf/python_dotenv-1.1.0.tar.gz", hash = "sha256:41f90bc6f5f177fb41f53e87666db362025010eb28f60a01c9143bfa33a2b2d5", size = 39920 } wheels = [ - { url = "https://files.pythonhosted.org/packages/6a/3e/b68c118422ec867fa7ab88444e1274aa40681c606d59ac27de5a5588f082/python_dotenv-1.0.1-py3-none-any.whl", hash = "sha256:f7b63ef50f1b690dddf550d03497b66d609393b40b564ed0d674909a68ebf16a", size = 19863 }, + { url = "https://files.pythonhosted.org/packages/1e/18/98a99ad95133c6a6e2005fe89faedf294a748bd5dc803008059409ac9b1e/python_dotenv-1.1.0-py3-none-any.whl", hash = "sha256:d7c01d9e2293916c18baf562d95698754b0dbbb5e74d457c45d4f6561fb9d55d", size = 20256 }, ] [[package]] name = "pytz" -version = "2025.1" +version = "2025.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/5f/57/df1c9157c8d5a05117e455d66fd7cf6dbc46974f832b1058ed4856785d8a/pytz-2025.1.tar.gz", hash = "sha256:c2db42be2a2518b28e65f9207c4d05e6ff547d1efa4086469ef855e4ab70178e", size = 319617 } +sdist = { url = "https://files.pythonhosted.org/packages/f8/bf/abbd3cdfb8fbc7fb3d4d38d320f2441b1e7cbe29be4f23797b4a2b5d8aac/pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3", size = 320884 } wheels = [ - { url = "https://files.pythonhosted.org/packages/eb/38/ac33370d784287baa1c3d538978b5e2ea064d4c1b93ffbd12826c190dd10/pytz-2025.1-py2.py3-none-any.whl", hash = "sha256:89dd22dca55b46eac6eda23b2d72721bf1bdfef212645d81513ef5d03038de57", size = 507930 }, + { url = "https://files.pythonhosted.org/packages/81/c4/34e93fe5f5429d7570ec1fa436f1986fb1f00c3e0f43a589fe2bbcd22c3f/pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00", size = 509225 }, ] [[package]] name = "pywin32" -version = "309" +version = "310" source = { registry = "https://pypi.org/simple" } wheels = [ - { url = "https://files.pythonhosted.org/packages/05/54/6409b1d98f2b8fed3bc2cc854859e48ae4a2dd956176664e38ee49c50a4c/pywin32-309-cp311-cp311-win32.whl", hash = "sha256:d5df6faa32b868baf9ade7c9b25337fa5eced28eb1ab89082c8dae9c48e4cd51", size = 8779225 }, - { url = "https://files.pythonhosted.org/packages/6a/f0/ae8ddb56771093dd2905baa852958fd65d42a8972aeefcf13578dfae69f4/pywin32-309-cp311-cp311-win_amd64.whl", hash = "sha256:e7ec2cef6df0926f8a89fd64959eba591a1eeaf0258082065f7bdbe2121228db", size = 9514129 }, - { url = "https://files.pythonhosted.org/packages/7a/4b/1f5e377a04448cf410e13040bc0e4c408bfa0a65705cabf96904178f18df/pywin32-309-cp311-cp311-win_arm64.whl", hash = "sha256:54ee296f6d11db1627216e9b4d4c3231856ed2d9f194c82f26c6cb5650163f4c", size = 8450450 }, - { url = "https://files.pythonhosted.org/packages/20/2c/b0240b14ff3dba7a8a7122dc9bbf7fbd21ed0e8b57c109633675b5d1761f/pywin32-309-cp312-cp312-win32.whl", hash = "sha256:de9acacced5fa82f557298b1fed5fef7bd49beee04190f68e1e4783fbdc19926", size = 8790648 }, - { url = "https://files.pythonhosted.org/packages/dd/11/c36884c732e2b3397deee808b5dac1abbb170ec37f94c6606fcb04d1e9d7/pywin32-309-cp312-cp312-win_amd64.whl", hash = "sha256:6ff9eebb77ffc3d59812c68db33c0a7817e1337e3537859499bd27586330fc9e", size = 9497399 }, - { url = "https://files.pythonhosted.org/packages/18/9f/79703972958f8ba3fd38bc9bf1165810bd75124982419b0cc433a2894d46/pywin32-309-cp312-cp312-win_arm64.whl", hash = "sha256:619f3e0a327b5418d833f44dc87859523635cf339f86071cc65a13c07be3110f", size = 8454122 }, + { url = "https://files.pythonhosted.org/packages/f7/b1/68aa2986129fb1011dabbe95f0136f44509afaf072b12b8f815905a39f33/pywin32-310-cp311-cp311-win32.whl", hash = "sha256:1e765f9564e83011a63321bb9d27ec456a0ed90d3732c4b2e312b855365ed8bd", size = 8784284 }, + { url = "https://files.pythonhosted.org/packages/b3/bd/d1592635992dd8db5bb8ace0551bc3a769de1ac8850200cfa517e72739fb/pywin32-310-cp311-cp311-win_amd64.whl", hash = "sha256:126298077a9d7c95c53823934f000599f66ec9296b09167810eb24875f32689c", size = 9520748 }, + { url = "https://files.pythonhosted.org/packages/90/b1/ac8b1ffce6603849eb45a91cf126c0fa5431f186c2e768bf56889c46f51c/pywin32-310-cp311-cp311-win_arm64.whl", hash = "sha256:19ec5fc9b1d51c4350be7bb00760ffce46e6c95eaf2f0b2f1150657b1a43c582", size = 8455941 }, + { url = "https://files.pythonhosted.org/packages/6b/ec/4fdbe47932f671d6e348474ea35ed94227fb5df56a7c30cbbb42cd396ed0/pywin32-310-cp312-cp312-win32.whl", hash = "sha256:8a75a5cc3893e83a108c05d82198880704c44bbaee4d06e442e471d3c9ea4f3d", size = 8796239 }, + { url = "https://files.pythonhosted.org/packages/e3/e5/b0627f8bb84e06991bea89ad8153a9e50ace40b2e1195d68e9dff6b03d0f/pywin32-310-cp312-cp312-win_amd64.whl", hash = "sha256:bf5c397c9a9a19a6f62f3fb821fbf36cac08f03770056711f765ec1503972060", size = 9503839 }, + { url = "https://files.pythonhosted.org/packages/1f/32/9ccf53748df72301a89713936645a664ec001abd35ecc8578beda593d37d/pywin32-310-cp312-cp312-win_arm64.whl", hash = "sha256:2349cc906eae872d0663d4d6290d13b90621eaf78964bb1578632ff20e152966", size = 8459470 }, ] [[package]] @@ -1335,15 +1372,15 @@ wheels = [ [[package]] name = "rich" -version = "13.9.4" +version = "14.0.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "markdown-it-py" }, { name = "pygments" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/ab/3a/0316b28d0761c6734d6bc14e770d85506c986c85ffb239e688eeaab2c2bc/rich-13.9.4.tar.gz", hash = "sha256:439594978a49a09530cff7ebc4b5c7103ef57baf48d5ea3184f21d9a2befa098", size = 223149 } +sdist = { url = "https://files.pythonhosted.org/packages/a1/53/830aa4c3066a8ab0ae9a9955976fb770fe9c6102117c8ec4ab3ea62d89e8/rich-14.0.0.tar.gz", hash = "sha256:82f1bc23a6a21ebca4ae0c45af9bdbc492ed20231dcb63f297d6d1021a9d5725", size = 224078 } wheels = [ - { url = "https://files.pythonhosted.org/packages/19/71/39c7c0d87f8d4e6c020a393182060eaefeeae6c01dab6a84ec346f2567df/rich-13.9.4-py3-none-any.whl", hash = "sha256:6049d5e6ec054bf2779ab3358186963bac2ea89175919d699e378b99738c2a90", size = 242424 }, + { url = "https://files.pythonhosted.org/packages/0d/9b/63f4c7ebc259242c89b3acafdb37b41d1185c07ff0011164674e9076b491/rich-14.0.0-py3-none-any.whl", hash = "sha256:1c9491e1951aac09caffd42f448ee3d04e58923ffe14993f6e83068dc395d7e0", size = 243229 }, ] [package.optional-dependencies] @@ -1416,11 +1453,11 @@ wheels = [ [[package]] name = "setuptools" -version = "76.0.0" +version = "78.1.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/32/d2/7b171caf085ba0d40d8391f54e1c75a1cda9255f542becf84575cfd8a732/setuptools-76.0.0.tar.gz", hash = "sha256:43b4ee60e10b0d0ee98ad11918e114c70701bc6051662a9a675a0496c1a158f4", size = 1349387 } +sdist = { url = "https://files.pythonhosted.org/packages/a9/5a/0db4da3bc908df06e5efae42b44e75c81dd52716e10192ff36d0c1c8e379/setuptools-78.1.0.tar.gz", hash = "sha256:18fd474d4a82a5f83dac888df697af65afa82dec7323d09c3e37d1f14288da54", size = 1367827 } wheels = [ - { url = "https://files.pythonhosted.org/packages/37/66/d2d7e6ad554f3a7c7297c3f8ef6e22643ad3d35ef5c63bf488bc89f32f31/setuptools-76.0.0-py3-none-any.whl", hash = "sha256:199466a166ff664970d0ee145839f5582cb9bca7a0a3a2e795b6a9cb2308e9c6", size = 1236106 }, + { url = "https://files.pythonhosted.org/packages/54/21/f43f0a1fa8b06b32812e0975981f4677d28e0f3271601dc88ac5a5b83220/setuptools-78.1.0-py3-none-any.whl", hash = "sha256:3e386e96793c8702ae83d17b853fb93d3e09ef82ec62722e61da5cd22376dcd8", size = 1256108 }, ] [[package]] @@ -1443,31 +1480,31 @@ wheels = [ [[package]] name = "sqlalchemy" -version = "2.0.39" +version = "2.0.40" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "greenlet", marker = "platform_machine == 'AMD64' or platform_machine == 'WIN32' or platform_machine == 'aarch64' or platform_machine == 'amd64' or platform_machine == 'ppc64le' or platform_machine == 'win32' or platform_machine == 'x86_64'" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/00/8e/e77fcaa67f8b9f504b4764570191e291524575ddbfe78a90fc656d671fdc/sqlalchemy-2.0.39.tar.gz", hash = "sha256:5d2d1fe548def3267b4c70a8568f108d1fed7cbbeccb9cc166e05af2abc25c22", size = 9644602 } +sdist = { url = "https://files.pythonhosted.org/packages/68/c3/3f2bfa5e4dcd9938405fe2fab5b6ab94a9248a4f9536ea2fd497da20525f/sqlalchemy-2.0.40.tar.gz", hash = "sha256:d827099289c64589418ebbcaead0145cd19f4e3e8a93919a0100247af245fa00", size = 9664299 } wheels = [ - { url = "https://files.pythonhosted.org/packages/59/23/d5f6a78cdec8b4c780b58bea2e243b80d324fb4733df5f31df695fe1fb2d/sqlalchemy-2.0.39-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a28f9c238f1e143ff42ab3ba27990dfb964e5d413c0eb001b88794c5c4a528a9", size = 2108611 }, - { url = "https://files.pythonhosted.org/packages/c6/25/6bec21ca66c36e3dcebe0141b86787eb17e3a918f4f583741831958775c2/sqlalchemy-2.0.39-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:08cf721bbd4391a0e765fe0fe8816e81d9f43cece54fdb5ac465c56efafecb3d", size = 2099011 }, - { url = "https://files.pythonhosted.org/packages/d8/75/e713f1827132fbf6e1ababa39235e486b5c96f911f55430986c17d46e546/sqlalchemy-2.0.39-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7a8517b6d4005facdbd7eb4e8cf54797dbca100a7df459fdaff4c5123265c1cd", size = 3217287 }, - { url = "https://files.pythonhosted.org/packages/ff/0a/46f3171f564a19a1daf6e7e0e6c8afc6ecd792f947c6de435519d4d16af3/sqlalchemy-2.0.39-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b2de1523d46e7016afc7e42db239bd41f2163316935de7c84d0e19af7e69538", size = 3217255 }, - { url = "https://files.pythonhosted.org/packages/80/57/12f5444c593536c1b61b34144d4ef4d4bac6e1919ce7b0626faecf1a959e/sqlalchemy-2.0.39-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:412c6c126369ddae171c13987b38df5122cb92015cba6f9ee1193b867f3f1530", size = 3154102 }, - { url = "https://files.pythonhosted.org/packages/3a/ad/1c681a46e0f10ecb5ca52d753416bffa8e80d62b3545e25f401a9b4082a5/sqlalchemy-2.0.39-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6b35e07f1d57b79b86a7de8ecdcefb78485dab9851b9638c2c793c50203b2ae8", size = 3174432 }, - { url = "https://files.pythonhosted.org/packages/7c/c2/948569a87bfc5c356fa9b66f03f6da6bd6d6c6d68a89d7c556ed87442c06/sqlalchemy-2.0.39-cp311-cp311-win32.whl", hash = "sha256:3eb14ba1a9d07c88669b7faf8f589be67871d6409305e73e036321d89f1d904e", size = 2079289 }, - { url = "https://files.pythonhosted.org/packages/cd/57/f0369e53e3031afabf5a7f7b78134037245dfed8b884d4d0e1a9c87b7c0f/sqlalchemy-2.0.39-cp311-cp311-win_amd64.whl", hash = "sha256:78f1b79132a69fe8bd6b5d91ef433c8eb40688ba782b26f8c9f3d2d9ca23626f", size = 2104704 }, - { url = "https://files.pythonhosted.org/packages/98/86/b2cb432aeb00a1eda7ed33ce86d943c2452dc1642f3ec51bfe9eaae9604b/sqlalchemy-2.0.39-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:c457a38351fb6234781d054260c60e531047e4d07beca1889b558ff73dc2014b", size = 2107210 }, - { url = "https://files.pythonhosted.org/packages/bf/b0/b2479edb3419ca763ba1b587161c292d181351a33642985506a530f9162b/sqlalchemy-2.0.39-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:018ee97c558b499b58935c5a152aeabf6d36b3d55d91656abeb6d93d663c0c4c", size = 2097599 }, - { url = "https://files.pythonhosted.org/packages/58/5e/c5b792a4abcc71e68d44cb531c4845ac539d558975cc61db1afbc8a73c96/sqlalchemy-2.0.39-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5493a8120d6fc185f60e7254fc056a6742f1db68c0f849cfc9ab46163c21df47", size = 3247012 }, - { url = "https://files.pythonhosted.org/packages/e0/a8/055fa8a7c5f85e6123b7e40ec2e9e87d63c566011d599b4a5ab75e033017/sqlalchemy-2.0.39-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b2cf5b5ddb69142511d5559c427ff00ec8c0919a1e6c09486e9c32636ea2b9dd", size = 3257851 }, - { url = "https://files.pythonhosted.org/packages/f6/40/aec16681e91a22ddf03dbaeb3c659bce96107c5f47d2a7c665eb7f24a014/sqlalchemy-2.0.39-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9f03143f8f851dd8de6b0c10784363712058f38209e926723c80654c1b40327a", size = 3193155 }, - { url = "https://files.pythonhosted.org/packages/21/9d/cef697b137b9eb0b66ab8e9cf193a7c7c048da3b4bb667e5fcea4d90c7a2/sqlalchemy-2.0.39-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:06205eb98cb3dd52133ca6818bf5542397f1dd1b69f7ea28aa84413897380b06", size = 3219770 }, - { url = "https://files.pythonhosted.org/packages/57/05/e109ca7dde837d8f2f1b235357e4e607f8af81ad8bc29c230fed8245687d/sqlalchemy-2.0.39-cp312-cp312-win32.whl", hash = "sha256:7f5243357e6da9a90c56282f64b50d29cba2ee1f745381174caacc50d501b109", size = 2077567 }, - { url = "https://files.pythonhosted.org/packages/97/c6/25ca068e38c29ed6be0fde2521888f19da923dbd58f5ff16af1b73ec9b58/sqlalchemy-2.0.39-cp312-cp312-win_amd64.whl", hash = "sha256:2ed107331d188a286611cea9022de0afc437dd2d3c168e368169f27aa0f61338", size = 2103136 }, - { url = "https://files.pythonhosted.org/packages/7b/0f/d69904cb7d17e65c65713303a244ec91fd3c96677baf1d6331457fd47e16/sqlalchemy-2.0.39-py3-none-any.whl", hash = "sha256:a1c6b0a5e3e326a466d809b651c63f278b1256146a377a528b6938a279da334f", size = 1898621 }, + { url = "https://files.pythonhosted.org/packages/77/7e/55044a9ec48c3249bb38d5faae93f09579c35e862bb318ebd1ed7a1994a5/sqlalchemy-2.0.40-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f6bacab7514de6146a1976bc56e1545bee247242fab030b89e5f70336fc0003e", size = 2114025 }, + { url = "https://files.pythonhosted.org/packages/77/0f/dcf7bba95f847aec72f638750747b12d37914f71c8cc7c133cf326ab945c/sqlalchemy-2.0.40-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5654d1ac34e922b6c5711631f2da497d3a7bffd6f9f87ac23b35feea56098011", size = 2104419 }, + { url = "https://files.pythonhosted.org/packages/75/70/c86a5c20715e4fe903dde4c2fd44fc7e7a0d5fb52c1b954d98526f65a3ea/sqlalchemy-2.0.40-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:35904d63412db21088739510216e9349e335f142ce4a04b69e2528020ee19ed4", size = 3222720 }, + { url = "https://files.pythonhosted.org/packages/12/cf/b891a8c1d0c27ce9163361664c2128c7a57de3f35000ea5202eb3a2917b7/sqlalchemy-2.0.40-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c7a80ed86d6aaacb8160a1caef6680d4ddd03c944d985aecee940d168c411d1", size = 3222682 }, + { url = "https://files.pythonhosted.org/packages/15/3f/7709d8c8266953d945435a96b7f425ae4172a336963756b58e996fbef7f3/sqlalchemy-2.0.40-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:519624685a51525ddaa7d8ba8265a1540442a2ec71476f0e75241eb8263d6f51", size = 3159542 }, + { url = "https://files.pythonhosted.org/packages/85/7e/717eaabaf0f80a0132dc2032ea8f745b7a0914451c984821a7c8737fb75a/sqlalchemy-2.0.40-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:2ee5f9999a5b0e9689bed96e60ee53c3384f1a05c2dd8068cc2e8361b0df5b7a", size = 3179864 }, + { url = "https://files.pythonhosted.org/packages/e4/cc/03eb5dfcdb575cbecd2bd82487b9848f250a4b6ecfb4707e834b4ce4ec07/sqlalchemy-2.0.40-cp311-cp311-win32.whl", hash = "sha256:c0cae71e20e3c02c52f6b9e9722bca70e4a90a466d59477822739dc31ac18b4b", size = 2084675 }, + { url = "https://files.pythonhosted.org/packages/9a/48/440946bf9dc4dc231f4f31ef0d316f7135bf41d4b86aaba0c0655150d370/sqlalchemy-2.0.40-cp311-cp311-win_amd64.whl", hash = "sha256:574aea2c54d8f1dd1699449f332c7d9b71c339e04ae50163a3eb5ce4c4325ee4", size = 2110099 }, + { url = "https://files.pythonhosted.org/packages/92/06/552c1f92e880b57d8b92ce6619bd569b25cead492389b1d84904b55989d8/sqlalchemy-2.0.40-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9d3b31d0a1c44b74d3ae27a3de422dfccd2b8f0b75e51ecb2faa2bf65ab1ba0d", size = 2112620 }, + { url = "https://files.pythonhosted.org/packages/01/72/a5bc6e76c34cebc071f758161dbe1453de8815ae6e662393910d3be6d70d/sqlalchemy-2.0.40-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:37f7a0f506cf78c80450ed1e816978643d3969f99c4ac6b01104a6fe95c5490a", size = 2103004 }, + { url = "https://files.pythonhosted.org/packages/bf/fd/0e96c8e6767618ed1a06e4d7a167fe13734c2f8113c4cb704443e6783038/sqlalchemy-2.0.40-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0bb933a650323e476a2e4fbef8997a10d0003d4da996aad3fd7873e962fdde4d", size = 3252440 }, + { url = "https://files.pythonhosted.org/packages/cd/6a/eb82e45b15a64266a2917a6833b51a334ea3c1991728fd905bfccbf5cf63/sqlalchemy-2.0.40-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6959738971b4745eea16f818a2cd086fb35081383b078272c35ece2b07012716", size = 3263277 }, + { url = "https://files.pythonhosted.org/packages/45/97/ebe41ab4530f50af99e3995ebd4e0204bf1b0dc0930f32250dde19c389fe/sqlalchemy-2.0.40-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:110179728e442dae85dd39591beb74072ae4ad55a44eda2acc6ec98ead80d5f2", size = 3198591 }, + { url = "https://files.pythonhosted.org/packages/e6/1c/a569c1b2b2f5ac20ba6846a1321a2bf52e9a4061001f282bf1c5528dcd69/sqlalchemy-2.0.40-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e8040680eaacdce4d635f12c55c714f3d4c7f57da2bc47a01229d115bd319191", size = 3225199 }, + { url = "https://files.pythonhosted.org/packages/8f/91/87cc71a6b10065ca0209d19a4bb575378abda6085e72fa0b61ffb2201b84/sqlalchemy-2.0.40-cp312-cp312-win32.whl", hash = "sha256:650490653b110905c10adac69408380688cefc1f536a137d0d69aca1069dc1d1", size = 2082959 }, + { url = "https://files.pythonhosted.org/packages/2a/9f/14c511cda174aa1ad9b0e42b64ff5a71db35d08b0d80dc044dae958921e5/sqlalchemy-2.0.40-cp312-cp312-win_amd64.whl", hash = "sha256:2be94d75ee06548d2fc591a3513422b873490efb124048f50556369a834853b0", size = 2108526 }, + { url = "https://files.pythonhosted.org/packages/d1/7c/5fc8e802e7506fe8b55a03a2e1dab156eae205c91bee46305755e086d2e2/sqlalchemy-2.0.40-py3-none-any.whl", hash = "sha256:32587e2e1e359276957e6fe5dad089758bc042a971a8a09ae8ecf7a8fe23d07a", size = 1903894 }, ] [[package]] @@ -1699,22 +1736,43 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/00/c0/8f5d070730d7836adc9c9b6408dec68c6ced86b304a9b26a14df072a6e8c/traitlets-5.14.3-py3-none-any.whl", hash = "sha256:b74e89e397b1ed28cc831db7aea759ba6640cb3de13090ca145426688ff1ac4f", size = 85359 }, ] +[[package]] +name = "types-pytz" +version = "2025.2.0.20250326" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/4b/66/38c89861242f2c61c8315ddbcc7d7bbf64979f4b0bdc48db0ba62aeec330/types_pytz-2025.2.0.20250326.tar.gz", hash = "sha256:deda02de24f527066fc8d6a19e284ab3f3ae716a42b4adb6b40e75e408c08d36", size = 10595 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4e/e0/17f3a6670db5c95dc195f346e2e7290f22ba8327c188133959389b578cbd/types_pytz-2025.2.0.20250326-py3-none-any.whl", hash = "sha256:3c397fd1b845cd2b3adc9398607764ced9e578a98a5d1fbb4a9bc9253edfb162", size = 10222 }, +] + [[package]] name = "typing-extensions" -version = "4.12.2" +version = "4.13.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/0e/3e/b00a62db91a83fff600de219b6ea9908e6918664899a2d85db222f4fbf19/typing_extensions-4.13.0.tar.gz", hash = "sha256:0a4ac55a5820789d87e297727d229866c9650f6521b64206413c4fbada24d95b", size = 106520 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e0/86/39b65d676ec5732de17b7e3c476e45bb80ec64eb50737a8dce1a4178aba1/typing_extensions-4.13.0-py3-none-any.whl", hash = "sha256:c8dd92cc0d6425a97c18fbb9d1954e5ff92c1ca881a309c45f06ebc0b79058e5", size = 45683 }, +] + +[[package]] +name = "typing-inspection" +version = "0.4.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/df/db/f35a00659bc03fec321ba8bce9420de607a1d37f8342eee1863174c69557/typing_extensions-4.12.2.tar.gz", hash = "sha256:1a7ead55c7e559dd4dee8856e3a88b41225abfe1ce8df57b7c13915fe121ffb8", size = 85321 } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/82/5c/e6082df02e215b846b4b8c0b887a64d7d08ffaba30605502639d44c06b82/typing_inspection-0.4.0.tar.gz", hash = "sha256:9765c87de36671694a67904bf2c96e395be9c6439bb6c87b5142569dcdd65122", size = 76222 } wheels = [ - { url = "https://files.pythonhosted.org/packages/26/9f/ad63fc0248c5379346306f8668cda6e2e2e9c95e01216d2b8ffd9ff037d0/typing_extensions-4.12.2-py3-none-any.whl", hash = "sha256:04e5ca0351e0f3f85c6853954072df659d0d13fac324d0072316b67d7794700d", size = 37438 }, + { url = "https://files.pythonhosted.org/packages/31/08/aa4fdfb71f7de5176385bd9e90852eaf6b5d622735020ad600f2bab54385/typing_inspection-0.4.0-py3-none-any.whl", hash = "sha256:50e72559fcd2a6367a19f7a7e610e6afcb9fac940c650290eed893d61386832f", size = 14125 }, ] [[package]] name = "tzdata" -version = "2025.1" +version = "2025.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/43/0f/fa4723f22942480be4ca9527bbde8d43f6c3f2fe8412f00e7f5f6746bc8b/tzdata-2025.1.tar.gz", hash = "sha256:24894909e88cdb28bd1636c6887801df64cb485bd593f2fd83ef29075a81d694", size = 194950 } +sdist = { url = "https://files.pythonhosted.org/packages/95/32/1a225d6164441be760d75c2c42e2780dc0873fe382da3e98a2e1e48361e5/tzdata-2025.2.tar.gz", hash = "sha256:b60a638fcc0daffadf82fe0f57e53d06bdec2f36c4df66280ae79bce6bd6f2b9", size = 196380 } wheels = [ - { url = "https://files.pythonhosted.org/packages/0f/dd/84f10e23edd882c6f968c21c2434fe67bd4a528967067515feca9e611e5e/tzdata-2025.1-py2.py3-none-any.whl", hash = "sha256:7e127113816800496f027041c570f50bcd464a020098a3b6b199517772303639", size = 346762 }, + { url = "https://files.pythonhosted.org/packages/5c/23/c7abc0ca0a1526a0774eca151daeb8de62ec457e77262b66b359c3c7679e/tzdata-2025.2-py2.py3-none-any.whl", hash = "sha256:1a403fada01ff9221ca8044d701868fa132215d84beb92242d9acd2147f667a8", size = 347839 }, ] [[package]]