diff --git a/README.md b/README.md index 87cd26e..c4fe989 100644 --- a/README.md +++ b/README.md @@ -45,6 +45,7 @@ There are different options available for running Dagster code on Ray. The follo | Enabled per-asset | ❌ | ❌ | ✅ | ✅ | | Configurable per-asset | ❌ | ✅ | ✅ | ✅ | | Runs asset/op body on Ray | ✅ | ✅ | ❌ | ❌ | +| Requires configuring Dagster in the Ray cluster | ❌ | ❌ | ✅ | ✅ | # Examples @@ -78,8 +79,9 @@ run_launcher: module: dagster_ray class: RayRunLauncher config: - num_cpus: 1 - num_gpus: 0 + ray: + num_cpus: 1 + num_gpus: 0 ``` Individual Runs can **override** Ray configuration: diff --git a/dagster_ray/executor.py b/dagster_ray/executor.py index 4f8af90..d53b3e7 100644 --- a/dagster_ray/executor.py +++ b/dagster_ray/executor.py @@ -25,12 +25,17 @@ from dagster_ray.config import RayExecutionConfig, RayJobSubmissionClientConfig from dagster_ray.kuberay.resources import get_k8s_object_name from dagster_ray.run_launcher import RayRunLauncher +from dagster_ray.utils import resolve_env_vars_list if TYPE_CHECKING: from ray.job_submission import JobSubmissionClient class RayExecutorConfig(RayExecutionConfig, RayJobSubmissionClientConfig): + env_vars: Optional[List[str]] = Field( + default=None, + description="A list of environment variables to inject into the Job. Each can be of the form KEY=VALUE or just KEY (in which case the value will be pulled from the current process).", + ) address: Optional[str] = Field(default=None, description="The address of the Ray cluster to connect to.") # type: ignore # sorry for the long name, but it has to be very clear what this is doing inherit_job_submission_client_from_ray_run_launcher: bool = True @@ -74,6 +79,7 @@ def ray_executor(init_context: InitExecutorContext) -> Executor: return StepDelegatingExecutor( RayStepHandler( client=client, + env_vars=ray_cfg.env_vars, runtime_env=ray_cfg.runtime_env, num_cpus=ray_cfg.num_cpus, num_gpus=ray_cfg.num_gpus, @@ -95,6 +101,7 @@ def name(self): def __init__( self, client: "JobSubmissionClient", + env_vars: Optional[List[str]], runtime_env: Optional[Dict[str, Any]], num_cpus: Optional[float], num_gpus: Optional[float], @@ -104,6 +111,7 @@ def __init__( super().__init__() self.client = client + self.env_vars = env_vars or [] self.runtime_env = runtime_env or {} self.num_cpus = num_cpus self.num_gpus = num_gpus @@ -148,6 +156,7 @@ def launch_step(self, step_handler_context: StepHandlerContext) -> Iterator[Dags user_provided_config = RayExecutionConfig.from_tags({**step_handler_context.step_tags[step_key]}) + # note! ray modifies the user-provided runtime_env, so we copy it runtime_env = (user_provided_config.runtime_env or self.runtime_env).copy() dagster_env_vars = { @@ -158,6 +167,8 @@ def launch_step(self, step_handler_context: StepHandlerContext) -> Iterator[Dags runtime_env["env_vars"] = {**dagster_env_vars, **runtime_env.get("env_vars", {})} # type: ignore + runtime_env["runtime_env"].update(resolve_env_vars_list(self.env_vars)) + num_cpus = self.num_cpus or user_provided_config.num_cpus num_gpus = self.num_gpus or user_provided_config.num_gpus memory = self.memory or user_provided_config.memory diff --git a/dagster_ray/kuberay/client/base.py b/dagster_ray/kuberay/client/base.py index 2439b64..1bc8bb4 100644 --- a/dagster_ray/kuberay/client/base.py +++ b/dagster_ray/kuberay/client/base.py @@ -1,10 +1,8 @@ import time from typing import TYPE_CHECKING, Any, Dict, Optional -from kubernetes import client -from kubernetes.client import ApiException - if TYPE_CHECKING: + from kubernetes import client from kubernetes.client.models.v1_endpoints import V1Endpoints @@ -27,8 +25,10 @@ def __init__( version: str, kind: str, plural: str, - api_client: Optional[client.ApiClient] = None, + api_client: Optional["client.ApiClient"] = None, ): + from kubernetes import client + self.group = group self.version = version self.kind = kind @@ -38,6 +38,8 @@ def __init__( self._core_v1_api = client.CoreV1Api(api_client=api_client) def wait_for_service_endpoints(self, service_name: str, namespace: str, poll_interval: int = 5, timeout: int = 60): + from kubernetes.client import ApiException + start_time = time.time() while True: @@ -62,6 +64,8 @@ def wait_for_service_endpoints(self, service_name: str, namespace: str, poll_int time.sleep(poll_interval) def get_status(self, name: str, namespace: str, timeout: int = 60, poll_interval: int = 5) -> Dict[str, Any]: + from kubernetes.client import ApiException + while timeout > 0: try: resource: Any = self._api.get_namespaced_custom_object_status( @@ -83,6 +87,8 @@ def get_status(self, name: str, namespace: str, timeout: int = 60, poll_interval raise TimeoutError(f"Timed out waiting for status of {self.kind} {name} in namespace {namespace}") def list(self, namespace: str, label_selector: str = "", async_req: bool = False) -> Dict[str, Any]: + from kubernetes.client import ApiException + try: resource: Any = self._api.list_namespaced_custom_object( group=self.group, @@ -103,6 +109,8 @@ def list(self, namespace: str, label_selector: str = "", async_req: bool = False raise def get(self, name: str, namespace: str) -> Dict[str, Any]: + from kubernetes.client import ApiException + try: resource: Any = self._api.get_namespaced_custom_object( group=self.group, diff --git a/dagster_ray/kuberay/client/raycluster/client.py b/dagster_ray/kuberay/client/raycluster/client.py index 3ebd0f9..5e67b32 100644 --- a/dagster_ray/kuberay/client/raycluster/client.py +++ b/dagster_ray/kuberay/client/raycluster/client.py @@ -17,7 +17,6 @@ cast, ) -from kubernetes import watch from typing_extensions import NotRequired from dagster_ray.kuberay.client.base import BaseKubeRayClient @@ -104,6 +103,8 @@ def wait_until_ready( timeout: int, image: Optional[str] = None, ) -> Tuple[str, Dict[str, str]]: + from kubernetes import watch + """ If ready, returns service ip address and a dictionary of ports. Dictionary keys: ["client", "dashboard", "metrics", "redis", "serve"] diff --git a/dagster_ray/kuberay/resources.py b/dagster_ray/kuberay/resources.py index 9e31881..53cd60b 100644 --- a/dagster_ray/kuberay/resources.py +++ b/dagster_ray/kuberay/resources.py @@ -4,10 +4,9 @@ import re import string import sys -from typing import Any, Dict, Generator, Optional, cast +from typing import TYPE_CHECKING, Any, Dict, Generator, Optional, cast import dagster._check as check -import kubernetes from dagster import ConfigurableResource, InitResourceContext from dagster._annotations import experimental from pydantic import Field, PrivateAttr @@ -30,6 +29,9 @@ from dagster_ray._base.resources import BaseRayResource from dagster_ray.kuberay.client.base import load_kubeconfig +if TYPE_CHECKING: + import kubernetes + @experimental class RayClusterClientResource(ConfigurableResource): @@ -37,8 +39,8 @@ class RayClusterClientResource(ConfigurableResource): kubeconfig_file: Optional[str] = None _raycluster_client: RayClusterClient = PrivateAttr() - _k8s_api: kubernetes.client.CustomObjectsApi = PrivateAttr() - _k8s_core_api: kubernetes.client.CoreV1Api = PrivateAttr() + _k8s_api: "kubernetes.client.CustomObjectsApi" = PrivateAttr() + _k8s_core_api: "kubernetes.client.CoreV1Api" = PrivateAttr() @property def client(self) -> RayClusterClient: @@ -47,18 +49,20 @@ def client(self) -> RayClusterClient: return self._raycluster_client @property - def k8s(self) -> kubernetes.client.CustomObjectsApi: + def k8s(self) -> "kubernetes.client.CustomObjectsApi": if self._k8s_api is None: raise ValueError(f"{self.__class__.__name__} not initialized") return self._k8s_api @property - def k8s_core(self) -> kubernetes.client.CoreV1Api: + def k8s_core(self) -> "kubernetes.client.CoreV1Api": if self._k8s_core_api is None: raise ValueError(f"{self.__class__.__name__} not initialized") return self._k8s_core_api def setup_for_execution(self, context: InitResourceContext) -> None: + import kubernetes + load_kubeconfig(context=self.kube_context, config_file=self.kubeconfig_file) self._raycluster_client = RayClusterClient(context=self.kube_context, config_file=self.kubeconfig_file) @@ -218,6 +222,8 @@ def update_group_spec(group_spec: Dict[str, Any]): } def _wait_raycluster_ready(self): + import kubernetes + self.client.client.wait_until_ready(self.cluster_name, namespace=self.namespace, timeout=self.timeout) # the above code only checks for RayCluster creation diff --git a/dagster_ray/run_launcher.py b/dagster_ray/run_launcher.py index ffad0fb..e5c0462 100644 --- a/dagster_ray/run_launcher.py +++ b/dagster_ray/run_launcher.py @@ -1,6 +1,6 @@ import logging import sys -from typing import Any, Dict, Optional +from typing import TYPE_CHECKING, Any, Dict, List, Optional from dagster import _check as check from dagster._cli.api import ExecuteRunArgs # type: ignore @@ -12,15 +12,24 @@ from dagster._grpc.types import ResumeRunArgs from dagster._serdes import ConfigurableClass, ConfigurableClassData from dagster._utils.error import serializable_error_info_from_exc_info +from pydantic import Field from dagster_ray.config import RayExecutionConfig, RayJobSubmissionClientConfig +from dagster_ray.utils import resolve_env_vars_list + +if TYPE_CHECKING: + from ray.job_submission import JobSubmissionClient def get_job_submission_id_from_run_id(run_id: str, resume_attempt_number=None): return f"dagster-run-{run_id}" + ("" if not resume_attempt_number else f"-{resume_attempt_number}") -class RayLauncherConfig(RayExecutionConfig, RayJobSubmissionClientConfig): ... +class RayLauncherConfig(RayExecutionConfig, RayJobSubmissionClientConfig): + env_vars: Optional[List[str]] = Field( + default=None, + description="A list of environment variables to inject into the Job. Each can be of the form KEY=VALUE or just KEY (in which case the value will be pulled from the current process).", + ) class RayRunLauncher(RunLauncher, ConfigurableClass): @@ -30,6 +39,7 @@ def __init__( metadata: Optional[Dict[str, Any]] = None, headers: Optional[Dict[str, Any]] = None, cookies: Optional[Dict[str, Any]] = None, + env_vars: Optional[List[str]] = None, runtime_env: Optional[Dict[str, Any]] = None, num_cpus: Optional[int] = None, num_gpus: Optional[int] = None, @@ -55,17 +65,13 @@ def __init__( Fields such as `num_cpus` set via `dagster-ray/config` Run tag will override the yaml configuration. """ - - from ray.job_submission import JobSubmissionClient - self._inst_data = check.opt_inst_param(inst_data, "inst_data", ConfigurableClassData) - self.client = JobSubmissionClient(address, metadata=metadata, headers=headers, cookies=cookies) - self.address = address self.metadata = metadata self.headers = headers self.cookies = cookies + self.env_vars = env_vars self.runtime_env = runtime_env self.num_cpus = num_cpus self.num_gpus = num_gpus @@ -74,17 +80,23 @@ def __init__( super().__init__() + @property + def client(self) -> "JobSubmissionClient": # note: this must be a property + from ray.job_submission import JobSubmissionClient + + return JobSubmissionClient(self.address, metadata=self.metadata, headers=self.headers, cookies=self.cookies) + @property def inst_data(self) -> Optional[ConfigurableClassData]: return self._inst_data @classmethod def config_type(cls) -> UserConfigSchema: - return RayLauncherConfig.to_fields_dict() + return {"ray": RayLauncherConfig.to_config_schema().as_field()} @classmethod def from_config_value(cls, inst_data, config_value): - return cls(inst_data=inst_data, **config_value) + return cls(inst_data=inst_data, **config_value["ray"]) @property def supports_resume_run(self): @@ -132,13 +144,16 @@ def _launch_ray_job(self, submission_id: str, entrypoint: str, run: DagsterRun): cfg_from_tags = RayLauncherConfig.from_tags(run.tags) - runtime_env = cfg_from_tags.runtime_env or self.runtime_env or {} + env_vars = cfg_from_tags.env_vars or self.env_vars or [] + # note! ray modifies the user-provided runtime_env, so we copy it + runtime_env = (cfg_from_tags.runtime_env or self.runtime_env or {}).copy() num_cpus = cfg_from_tags.num_cpus or self.num_cpus num_gpus = cfg_from_tags.num_gpus or self.num_gpus memory = cfg_from_tags.memory or self.memory resources = cfg_from_tags.resources or self.resources runtime_env["env_vars"] = runtime_env.get("env_vars", {}) + runtime_env["env_vars"].update(resolve_env_vars_list(env_vars)) runtime_env["env_vars"].update( { "DAGSTER_RUN_JOB_NAME": job_origin.job_name, diff --git a/dagster_ray/utils.py b/dagster_ray/utils.py new file mode 100644 index 0000000..d4be922 --- /dev/null +++ b/dagster_ray/utils.py @@ -0,0 +1,17 @@ +import os +from typing import Dict, List, Optional + + +def resolve_env_vars_list(env_vars: Optional[List[str]]) -> Dict[str, str]: + res = {} + + if env_vars is not None: + for env_var in env_vars: + if "=" in env_var: + var, value = env_var.split("=", 1) + res[var] = value + else: + if value := os.getenv(env_var): + res[env_var] = value + + return res diff --git a/examples/docker/run_launcher/Dockerfile b/examples/docker/run_launcher/Dockerfile new file mode 100644 index 0000000..2e874fe --- /dev/null +++ b/examples/docker/run_launcher/Dockerfile @@ -0,0 +1,34 @@ +# Dagster libraries to run both dagster-webserver and the dagster-daemon. Does not +# need to have access to any pipeline code. + +FROM python:3.10-slim + +ENV UV_SYSTEM_PYTHON=1 + +# install uv +COPY --from=ghcr.io/astral-sh/uv:latest /uv /bin/uv + +RUN --mount=type=cache,target=/root/.cache/uv \ + uv pip install \ + dagster \ + dagster-graphql \ + dagster-webserver \ + dagster-postgres \ + dagster-docker \ + dagster-k8s \ + ray[all] + +WORKDIR /src + +COPY pyproject.toml README.md ./ +COPY dagster_ray ./dagster_ray + +RUN --mount=type=cache,target=/root/.cache/uv \ + uv pip install -e . + +# Set $DAGSTER_HOME and copy dagster instance and workspace YAML there +ENV DAGSTER_HOME=/dagster_home + +WORKDIR $DAGSTER_HOME + +COPY examples/docker/run_launcher/ ./ diff --git a/examples/docker/run_launcher/README.md b/examples/docker/run_launcher/README.md new file mode 100644 index 0000000..8855a69 --- /dev/null +++ b/examples/docker/run_launcher/README.md @@ -0,0 +1,13 @@ +# Example Dagster + Ray deployment with Docker + +1. Start all services with Docker Compose: + +```shell +docker compose up --build +``` + +2. In your browser, open `localhost:3000` to access the Dagster UI and `localhost:8265` to access the Ray dashboard. + +3. Launch a job. Observe how the steps are executed in separate Ray jobs in parallel. + +4. Make changes to `src/definitions.py`. diff --git a/examples/docker/run_launcher/dagster.yaml b/examples/docker/run_launcher/dagster.yaml new file mode 100644 index 0000000..f5b2fe6 --- /dev/null +++ b/examples/docker/run_launcher/dagster.yaml @@ -0,0 +1,65 @@ +scheduler: + module: dagster.core.scheduler + class: DagsterDaemonScheduler + +run_coordinator: + module: dagster.core.run_coordinator + class: QueuedRunCoordinator + +run_launcher: + module: dagster_ray + class: RayRunLauncher + config: + ray: + address: http://ray-head:8265 + runtime_env: + working_dir: /dagster_home + env_vars: + - DAGSTER_POSTGRES_USER + - DAGSTER_POSTGRES_PASSWORD + - DAGSTER_POSTGRES_DB + +run_storage: + module: dagster_postgres.run_storage + class: PostgresRunStorage + config: + postgres_db: + hostname: postgres + username: + env: DAGSTER_POSTGRES_USER + password: + env: DAGSTER_POSTGRES_PASSWORD + db_name: + env: DAGSTER_POSTGRES_DB + port: 5432 + +schedule_storage: + module: dagster_postgres.schedule_storage + class: PostgresScheduleStorage + config: + postgres_db: + hostname: postgres + username: + env: DAGSTER_POSTGRES_USER + password: + env: DAGSTER_POSTGRES_PASSWORD + db_name: + env: DAGSTER_POSTGRES_DB + port: 5432 + +event_log_storage: + module: dagster_postgres.event_log + class: PostgresEventLogStorage + config: + postgres_db: + hostname: postgres + username: + env: DAGSTER_POSTGRES_USER + password: + env: DAGSTER_POSTGRES_PASSWORD + db_name: + env: DAGSTER_POSTGRES_DB + port: 5432 + +telemetry: + enabled: false diff --git a/examples/docker/run_launcher/definitions.py b/examples/docker/run_launcher/definitions.py new file mode 100644 index 0000000..22b5d3d --- /dev/null +++ b/examples/docker/run_launcher/definitions.py @@ -0,0 +1,50 @@ +import time + +from dagster import Config, Definitions, OpExecutionContext, job, op + +from dagster_ray import RayIOManager + + +class MyConfig(Config): + sleep_for: int = 3 + + +@op +def return_one(context: OpExecutionContext, config: MyConfig) -> int: + context.log.info(f"sleeping for {config.sleep_for} seconds...") + time.sleep(config.sleep_for) + context.log.info("Waking up!") + + return 1 + + +@op +def return_two(context: OpExecutionContext, config: MyConfig) -> int: + context.log.info(f"sleeping for {config.sleep_for} seconds...") + time.sleep(config.sleep_for) + context.log.info("Waking up!") + + import os + + context.log.info(str(os.listdir("."))) + + return 2 + + +@op +def sum_one_and_two(a: int, b: int) -> int: + res = a + b + + assert res == 3 + + return res + + +@job(tags={"dagster-ray/config": {"num_cpus": 0.5}}) +def my_job(): + return_two_result = return_two() + return_one_result = return_one() + sum_one_and_two(return_one_result, return_two_result) + + +definitions = Definitions(jobs=[my_job], resources={"io_manager": RayIOManager()}) diff --git a/examples/docker/run_launcher/docker-compose.yml b/examples/docker/run_launcher/docker-compose.yml new file mode 100644 index 0000000..7a4267c --- /dev/null +++ b/examples/docker/run_launcher/docker-compose.yml @@ -0,0 +1,122 @@ +services: + ray-head: + build: + context: ../../../ + dockerfile: examples/docker/run_launcher/Dockerfile + command: ["ray", "start", "--block", "--head", "--dashboard-host", "0.0.0.0"] + ports: + - "8265:8265" + volumes: + - ./dagster.yaml:/dagster_home/dagster.yaml + - ./workspace.yaml:/dagster_home/workspace.yaml + - /tmp/io_manager_storage:/tmp/io_manager_storage + networks: + - db + + ray-worker: + build: + context: ../../../ + dockerfile: examples/docker/run_launcher/Dockerfile + command: ["ray", "start", "--block", "--address", "ray-head:6379"] + volumes: + - ./dagster.yaml:/dagster_home/dagster.yaml + - ./workspace.yaml:/dagster_home/workspace.yaml + - /tmp/io_manager_storage:/tmp/io_manager_storage + depends_on: + - ray-head + networks: + - db + deploy: + mode: replicated + replicas: 2 + + # This service runs the postgres DB used by dagster for run storage, schedule storage, + # and event log storage. + postgres: + image: postgres:11 + environment: + POSTGRES_USER: "postgres_user" + POSTGRES_PASSWORD: "postgres_password" + POSTGRES_DB: "postgres_db" + networks: + - db + + code-location: + build: + context: ../../../ + dockerfile: examples/docker/run_launcher/Dockerfile + restart: always + environment: + DAGSTER_POSTGRES_USER: "postgres_user" + DAGSTER_POSTGRES_PASSWORD: "postgres_password" + DAGSTER_POSTGRES_DB: "postgres_db" + command: + - dagster + - api + - grpc + - --python-file + - /dagster_home/definitions.py + - --host + - 0.0.0.0 + - --port + - "4266" + volumes: + - .:/dagster_home + networks: + - db + + webserver: + build: + context: ../../../ + dockerfile: examples/docker/run_launcher/Dockerfile + entrypoint: + - dagster-webserver + - -h + - "0.0.0.0" + - -p + - "3000" + - -w + - workspace.yaml + expose: + - "3000" + ports: + - "3000:3000" + environment: + DAGSTER_POSTGRES_USER: "postgres_user" + DAGSTER_POSTGRES_PASSWORD: "postgres_password" + DAGSTER_POSTGRES_DB: "postgres_db" + volumes: # Make docker client accessible so we can terminate containers from the webserver + - /var/run/docker.sock:/var/run/docker.sock + - /tmp/io_manager_storage:/tmp/io_manager_storage + networks: + - db + depends_on: + - postgres + - code-location + + daemon: + build: + context: ../../../ + dockerfile: examples/docker/run_launcher/Dockerfile + entrypoint: + - dagster-daemon + - run + restart: on-failure + environment: + DAGSTER_POSTGRES_USER: "postgres_user" + DAGSTER_POSTGRES_PASSWORD: "postgres_password" + DAGSTER_POSTGRES_DB: "postgres_db" + volumes: # Make docker client accessible so we can launch containers using host docker + - /var/run/docker.sock:/var/run/docker.sock + - /tmp/io_manager_storage:/tmp/io_manager_storage + - .:/dagster_home + networks: + - db + depends_on: + - postgres + - code-location + +networks: + db: + driver: bridge + name: db diff --git a/examples/docker/run_launcher/workspace.yaml b/examples/docker/run_launcher/workspace.yaml new file mode 100644 index 0000000..1a6d410 --- /dev/null +++ b/examples/docker/run_launcher/workspace.yaml @@ -0,0 +1,5 @@ +load_from: + - grpc_server: + host: code-location + port: 4266 + location_name: "example" diff --git a/examples/executor/README.md b/examples/local/executor/README.md similarity index 88% rename from examples/executor/README.md rename to examples/local/executor/README.md index 5ee6645..67e6ee1 100644 --- a/examples/executor/README.md +++ b/examples/local/executor/README.md @@ -9,7 +9,7 @@ ray start --head 2. Start Dagster in the context of this example directory: ```shell -cd examples/executor/workspace.yaml +cd examples/local/executor dagster dev ``` diff --git a/examples/executor/dagster.yaml b/examples/local/executor/dagster.yaml similarity index 100% rename from examples/executor/dagster.yaml rename to examples/local/executor/dagster.yaml diff --git a/examples/executor/definitions.py b/examples/local/executor/definitions.py similarity index 100% rename from examples/executor/definitions.py rename to examples/local/executor/definitions.py diff --git a/examples/executor/workspace.yaml b/examples/local/executor/workspace.yaml similarity index 100% rename from examples/executor/workspace.yaml rename to examples/local/executor/workspace.yaml diff --git a/examples/run_launcher/README.md b/examples/local/run_launcher/README.md similarity index 95% rename from examples/run_launcher/README.md rename to examples/local/run_launcher/README.md index 8d26787..d206f66 100644 --- a/examples/run_launcher/README.md +++ b/examples/local/run_launcher/README.md @@ -11,7 +11,7 @@ ray start --head 2. Start Dagster in the context of this example directory: ```shell -cd examples/run_launcher +cd examples/local/run_launcher dagster dev ``` diff --git a/examples/local/run_launcher/dagster.yaml b/examples/local/run_launcher/dagster.yaml new file mode 100644 index 0000000..6886cb4 --- /dev/null +++ b/examples/local/run_launcher/dagster.yaml @@ -0,0 +1,11 @@ +run_launcher: + module: dagster_ray + class: RayRunLauncher + config: + ray: + runtime_env: + working_dir: /home/dan/code/github/danielgafni/dagster-ray/examples/local/run_launcher + address: auto + num_cpus: 1 +telemetry: + enabled: false diff --git a/examples/run_launcher/definitions.py b/examples/local/run_launcher/definitions.py similarity index 99% rename from examples/run_launcher/definitions.py rename to examples/local/run_launcher/definitions.py index a6cadf5..bd1791b 100644 --- a/examples/run_launcher/definitions.py +++ b/examples/local/run_launcher/definitions.py @@ -37,6 +37,7 @@ def sum_one_and_two(a: int, b: int) -> int: def my_job(): return_two_result = return_two() return_one_result = return_one() + sum_one_and_two(return_one_result, return_two_result) diff --git a/examples/run_launcher/workspace.yaml b/examples/local/run_launcher/workspace.yaml similarity index 100% rename from examples/run_launcher/workspace.yaml rename to examples/local/run_launcher/workspace.yaml diff --git a/examples/run_launcher_and_executor/README.md b/examples/local/run_launcher_and_executor/README.md similarity index 90% rename from examples/run_launcher_and_executor/README.md rename to examples/local/run_launcher_and_executor/README.md index 119874d..9bdab9d 100644 --- a/examples/run_launcher_and_executor/README.md +++ b/examples/local/run_launcher_and_executor/README.md @@ -11,7 +11,7 @@ ray start --head 2. Start Dagster in the context of this example directory: ```shell -cd examples/run_launcher_and_executor +cd examples/local/run_launcher_and_executor dagster dev ``` diff --git a/examples/run_launcher_and_executor/dagster.yaml b/examples/local/run_launcher_and_executor/dagster.yaml similarity index 67% rename from examples/run_launcher_and_executor/dagster.yaml rename to examples/local/run_launcher_and_executor/dagster.yaml index d220b22..0be85c7 100644 --- a/examples/run_launcher_and_executor/dagster.yaml +++ b/examples/local/run_launcher_and_executor/dagster.yaml @@ -2,6 +2,8 @@ run_launcher: module: dagster_ray class: RayRunLauncher config: - address: auto + ray: + num_cpus: 1 + address: auto telemetry: enabled: false diff --git a/examples/run_launcher_and_executor/definitions.py b/examples/local/run_launcher_and_executor/definitions.py similarity index 100% rename from examples/run_launcher_and_executor/definitions.py rename to examples/local/run_launcher_and_executor/definitions.py diff --git a/examples/run_launcher_and_executor/workspace.yaml b/examples/local/run_launcher_and_executor/workspace.yaml similarity index 100% rename from examples/run_launcher_and_executor/workspace.yaml rename to examples/local/run_launcher_and_executor/workspace.yaml diff --git a/examples/run_launcher/dagster.yaml b/examples/run_launcher/dagster.yaml deleted file mode 100644 index 9c9ca87..0000000 --- a/examples/run_launcher/dagster.yaml +++ /dev/null @@ -1,8 +0,0 @@ -run_launcher: - module: dagster_ray - class: RayRunLauncher - config: - address: auto - num_cpus: 1 -telemetry: - enabled: false diff --git a/pyproject.toml b/pyproject.toml index 6103891..60d48a3 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,10 +26,8 @@ classifiers = [ ] dependencies = [ - "pyyaml>=4.0.0", - "tenacity>=8.0.0", "dagster>=1.8.0", - "tenacity>=8.0.0" + "tenacity>=8.0.0", ] dynamic = ["version"] @@ -41,10 +39,14 @@ run_launcher = [ executor = [ "ray[client]" ] +pipes = [ + "ray[client]" +] kuberay = [ "pyyaml", "kubernetes>=20.0.0", "dagster-k8s>=0.19.0", + "pyyaml>=4.0.0", ] [tool.uv] @@ -62,6 +64,7 @@ dev-dependencies = [ "hatch>=1.12.0", "dunamai>=1.22.0", "pip>=24.2", + "ipython>=8.12.3", ] @@ -112,7 +115,7 @@ known-first-party = ["dagster_ray", "tests"] [tool.ruff.lint.flake8-tidy-imports] # Disallow all relative imports. ban-relative-imports = "all" -banned-module-level-imports = ["ray"] +banned-module-level-imports = ["ray", "kubernetes", "torch"] [tool.pyright] reportPropertyTypeMismatch = true diff --git a/tests/examples/test_examples.py b/tests/examples/test_examples.py index 7a80f7f..3e81a36 100644 --- a/tests/examples/test_examples.py +++ b/tests/examples/test_examples.py @@ -6,11 +6,12 @@ import pytest EXAMPLES_DIR = Path(__file__).parent.parent.parent / "examples" +LOCAL_EXAMPLES_DIR = EXAMPLES_DIR / "local" -RUN_LAUNCHER_EXAMPLE_DIR = EXAMPLES_DIR / "run_launcher" -EXECUTOR_EXAMPLE_DIR = EXAMPLES_DIR / "executor" -RUN_LAUNCHER_AND_EXECUTOR_EXAMPLE_DIR = EXAMPLES_DIR / "run_launcher_and_executor" +RUN_LAUNCHER_EXAMPLE_DIR = LOCAL_EXAMPLES_DIR / "run_launcher" +EXECUTOR_EXAMPLE_DIR = LOCAL_EXAMPLES_DIR / "executor" +RUN_LAUNCHER_AND_EXECUTOR_EXAMPLE_DIR = LOCAL_EXAMPLES_DIR / "run_launcher_and_executor" @pytest.mark.parametrize( diff --git a/uv.lock b/uv.lock index 2b95913..c36c699 100644 --- a/uv.lock +++ b/uv.lock @@ -203,6 +203,27 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/7b/a2/10639a79341f6c019dedc95bd48a4928eed9f1d1197f4c04f546fc7ae0ff/anyio-4.4.0-py3-none-any.whl", hash = "sha256:c1b2d8f46a8a812513012e1107cb0e68c17159a7a594208005a57dc776e1bdc7", size = 86780 }, ] +[[package]] +name = "appnope" +version = "0.1.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/35/5d/752690df9ef5b76e169e68d6a129fa6d08a7100ca7f754c89495db3c6019/appnope-0.1.4.tar.gz", hash = "sha256:1de3860566df9caf38f01f86f65e0e13e379af54f9e4bee1e66b48f2efffd1ee", size = 4170 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/81/29/5ecc3a15d5a33e31b26c11426c45c501e439cb865d0bff96315d86443b78/appnope-0.1.4-py2.py3-none-any.whl", hash = "sha256:502575ee11cd7a28c0205f379b525beefebab9d161b7c964670864014ed7213c", size = 4321 }, +] + +[[package]] +name = "asttokens" +version = "2.4.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "six" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/45/1d/f03bcb60c4a3212e15f99a56085d93093a497718adf828d050b9d675da81/asttokens-2.4.1.tar.gz", hash = "sha256:b03869718ba9a6eb027e134bfdf69f38a236d681c83c160d510768af11254ba0", size = 62284 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/45/86/4736ac618d82a20d87d2f92ae19441ebc7ac9e7a581d7e58bbe79233b24a/asttokens-2.4.1-py2.py3-none-any.whl", hash = "sha256:051ed49c3dcae8913ea7cd08e46a606dba30b79993209636c4875bc1d637bc24", size = 27764 }, +] + [[package]] name = "async-timeout" version = "4.0.3" @@ -221,6 +242,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/6a/21/5b6702a7f963e95456c0de2d495f67bf5fd62840ac655dc451586d23d39a/attrs-24.2.0-py3-none-any.whl", hash = "sha256:81921eb96de3191c8258c199618104dd27ac608d9366f5e35d011eae1867ede2", size = 63001 }, ] +[[package]] +name = "backcall" +version = "0.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a2/40/764a663805d84deee23043e1426a9175567db89c8b3287b5c2ad9f71aa93/backcall-0.2.0.tar.gz", hash = "sha256:5cbdbf27be5e7cfadb448baf0aa95508f91f2bbc6c6437cd9cd06e2a4c215e1e", size = 18041 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4c/1c/ff6546b6c12603d8dd1070aa3c3d273ad4c07f5771689a7b69a550e8c951/backcall-0.2.0-py2.py3-none-any.whl", hash = "sha256:fbbce6a29f263178a1f7915c1940bde0ec2b2a967566fe1c65c1dfb7422bd255", size = 11157 }, +] + [[package]] name = "backoff" version = "2.2.1" @@ -661,7 +691,6 @@ version = "0.0.0" source = { editable = "." } dependencies = [ { name = "dagster" }, - { name = "pyyaml" }, { name = "tenacity" }, ] @@ -674,6 +703,9 @@ kuberay = [ { name = "kubernetes" }, { name = "pyyaml" }, ] +pipes = [ + { name = "ray", extra = ["client"] }, +] run-launcher = [ { name = "ray", extra = ["client"] }, ] @@ -685,6 +717,7 @@ dev = [ { name = "dunamai" }, { name = "hatch" }, { name = "hypothesis" }, + { name = "ipython" }, { name = "pip" }, { name = "pre-commit" }, { name = "pyright" }, @@ -700,9 +733,10 @@ requires-dist = [ { name = "dagster", specifier = ">=1.8.0" }, { name = "dagster-k8s", marker = "extra == 'kuberay'", specifier = ">=0.19.0" }, { name = "kubernetes", marker = "extra == 'kuberay'", specifier = ">=20.0.0" }, - { name = "pyyaml", specifier = ">=4.0.0" }, { name = "pyyaml", marker = "extra == 'kuberay'" }, + { name = "pyyaml", marker = "extra == 'kuberay'", specifier = ">=4.0.0" }, { name = "ray", extras = ["client"], marker = "extra == 'executor'" }, + { name = "ray", extras = ["client"], marker = "extra == 'pipes'" }, { name = "ray", extras = ["client"], marker = "extra == 'run-launcher'" }, { name = "tenacity", specifier = ">=8.0.0" }, ] @@ -714,6 +748,7 @@ dev = [ { name = "dunamai", specifier = ">=1.22.0" }, { name = "hatch", specifier = ">=1.12.0" }, { name = "hypothesis", specifier = ">=6.77.0,<7.0.0" }, + { name = "ipython", specifier = ">=8.12.3" }, { name = "pip", specifier = ">=24.2" }, { name = "pre-commit", specifier = ">=3.3.2,<4.0.0" }, { name = "pyright", specifier = ">=1.1.313,<2.0.0" }, @@ -752,6 +787,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/7b/fa/8e4a51e1afda8d4bd73d784bfe4a60cfdeeced9bea419eff5c271180377e/decopatch-1.4.10-py2.py3-none-any.whl", hash = "sha256:e151f7f93de2b1b3fd3f3272dcc7cefd1a69f68ec1c2d8e288ecd9deb36dc5f7", size = 18015 }, ] +[[package]] +name = "decorator" +version = "5.1.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/66/0c/8d907af351aa16b42caae42f9d6aa37b900c67308052d10fdce809f8d952/decorator-5.1.1.tar.gz", hash = "sha256:637996211036b6385ef91435e4fae22989472f9d571faba8927ba8253acbc330", size = 35016 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d5/50/83c593b07763e1161326b3b8c6686f0f4b0f24d5526546bee538c89837d6/decorator-5.1.1-py3-none-any.whl", hash = "sha256:b8c3f85900b9dc423225913c5aace94729fe1fa9763b38939a95226f02d37186", size = 9073 }, +] + [[package]] name = "distlib" version = "0.3.8" @@ -791,6 +835,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/02/cc/b7e31358aac6ed1ef2bb790a9746ac2c69bcb3c8588b41616914eb106eaf/exceptiongroup-1.2.2-py3-none-any.whl", hash = "sha256:3111b9d131c238bec2f8f516e123e14ba243563fb135d3fe885990585aa7795b", size = 16453 }, ] +[[package]] +name = "executing" +version = "2.1.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/8c/e3/7d45f492c2c4a0e8e0fad57d081a7c8a0286cdd86372b070cca1ec0caa1e/executing-2.1.0.tar.gz", hash = "sha256:8ea27ddd260da8150fa5a708269c4a10e76161e2496ec3e587da9e3c0fe4b9ab", size = 977485 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b5/fd/afcd0496feca3276f509df3dbd5dae726fcc756f1a08d9e25abe1733f962/executing-2.1.0-py2.py3-none-any.whl", hash = "sha256:8d63781349375b5ebccc3142f4b30350c0cd9c79f921cde38be2be4637e98eaf", size = 25805 }, +] + [[package]] name = "filelock" version = "3.16.0" @@ -1325,6 +1378,30 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/ef/a6/62565a6e1cf69e10f5727360368e451d4b7f58beeac6173dc9db836a5b46/iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374", size = 5892 }, ] +[[package]] +name = "ipython" +version = "8.12.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "appnope", marker = "sys_platform == 'darwin'" }, + { name = "backcall" }, + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "decorator" }, + { name = "jedi" }, + { name = "matplotlib-inline" }, + { name = "pexpect", marker = "sys_platform != 'win32'" }, + { name = "pickleshare" }, + { name = "prompt-toolkit" }, + { name = "pygments" }, + { name = "stack-data" }, + { name = "traitlets" }, + { name = "typing-extensions", marker = "python_full_version < '3.10'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/9e/6a/44ef299b1762f5a73841e87fae8a73a8cc8aee538d6dc8c77a5afe1fd2ce/ipython-8.12.3.tar.gz", hash = "sha256:3910c4b54543c2ad73d06579aa771041b7d5707b033bd488669b4cf544e3b363", size = 5470171 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8d/97/8fe103906cd81bc42d3b0175b5534a9f67dccae47d6451131cf8d0d70bb2/ipython-8.12.3-py3-none-any.whl", hash = "sha256:b0340d46a933d27c657b211a329d0be23793c36595acf9e6ef4164bc01a1804c", size = 798307 }, +] + [[package]] name = "jaraco-classes" version = "3.4.0" @@ -1361,6 +1438,18 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/9f/4f/24b319316142c44283d7540e76c7b5a6dbd5db623abd86bb7b3491c21018/jaraco.functools-4.1.0-py3-none-any.whl", hash = "sha256:ad159f13428bc4acbf5541ad6dec511f91573b90fba04df61dafa2a1231cf649", size = 10187 }, ] +[[package]] +name = "jedi" +version = "0.19.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "parso" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/d6/99/99b493cec4bf43176b678de30f81ed003fd6a647a301b9c927280c600f0a/jedi-0.19.1.tar.gz", hash = "sha256:cf0496f3651bc65d7174ac1b7d043eff454892c708a87d1b683e57b569927ffd", size = 1227821 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/20/9f/bc63f0f0737ad7a60800bfd472a4836661adae21f9c2535f3957b1e54ceb/jedi-0.19.1-py2.py3-none-any.whl", hash = "sha256:e983c654fe5c02867aef4cdfce5a2fbb4a50adc0af145f70504238f18ef5e7e0", size = 1569361 }, +] + [[package]] name = "jeepney" version = "0.8.0" @@ -1563,6 +1652,18 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/f6/f8/4da07de16f10551ca1f640c92b5f316f9394088b183c6a57183df6de5ae4/MarkupSafe-2.1.5-cp39-cp39-win_amd64.whl", hash = "sha256:fa173ec60341d6bb97a89f5ea19c85c5643c1e7dedebc22f5181eb73573142c5", size = 17211 }, ] +[[package]] +name = "matplotlib-inline" +version = "0.1.7" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "traitlets" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/99/5b/a36a337438a14116b16480db471ad061c36c3694df7c2084a0da7ba538b7/matplotlib_inline-0.1.7.tar.gz", hash = "sha256:8423b23ec666be3d16e16b60bdd8ac4e86e840ebd1dd11a30b9f117f2fa0ab90", size = 8159 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8f/8e/9ad090d3553c280a8060fbf6e24dc1c0c29704ee7d1c372f0c174aa59285/matplotlib_inline-0.1.7-py3-none-any.whl", hash = "sha256:df192d39a4ff8f21b1895d72e6a13f5fcc5099f00fa84384e0ea28c2cc0653ca", size = 9899 }, +] + [[package]] name = "mdit-py-plugins" version = "0.4.2" @@ -1923,6 +2024,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/9a/f2/0ad053856debbe90c83de1b4f05915f85fd2146f20faf9daa3b320d36df3/pandas-2.0.3-cp39-cp39-win_amd64.whl", hash = "sha256:1168574b036cd8b93abc746171c9b4f1b83467438a5e45909fed645cf8692dbc", size = 10755902 }, ] +[[package]] +name = "parso" +version = "0.8.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/66/94/68e2e17afaa9169cf6412ab0f28623903be73d1b32e208d9e8e541bb086d/parso-0.8.4.tar.gz", hash = "sha256:eb3a7b58240fb99099a345571deecc0f9540ea5f4dd2fe14c2a99d6b281ab92d", size = 400609 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c6/ac/dac4a63f978e4dcb3c6d3a78c4d8e0192a113d288502a1216950c41b1027/parso-0.8.4-py2.py3-none-any.whl", hash = "sha256:a418670a20291dacd2dddc80c377c5c3791378ee1e8d12bffc35420643d43f18", size = 103650 }, +] + [[package]] name = "pathspec" version = "0.12.1" @@ -1944,6 +2054,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772 }, ] +[[package]] +name = "pickleshare" +version = "0.7.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d8/b6/df3c1c9b616e9c0edbc4fbab6ddd09df9535849c64ba51fcb6531c32d4d8/pickleshare-0.7.5.tar.gz", hash = "sha256:87683d47965c1da65cdacaf31c8441d12b8044cdec9aca500cd78fc2c683afca", size = 6161 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9a/41/220f49aaea88bc6fa6cba8d05ecf24676326156c23b991e80b3f2fc24c77/pickleshare-0.7.5-py2.py3-none-any.whl", hash = "sha256:9649af414d74d4df115d5d718f82acb59c9d418196b7b4290ed47a12ce62df56", size = 6877 }, +] + [[package]] name = "pip" version = "24.2" @@ -2005,6 +2124,18 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/84/2d/46ed6436849c2c88228c3111865f44311cff784b4aabcdef4ea2545dbc3d/prometheus_client-0.21.0-py3-none-any.whl", hash = "sha256:4fa6b4dd0ac16d58bb587c04b1caae65b8c5043e85f778f42f5f632f6af2e166", size = 54686 }, ] +[[package]] +name = "prompt-toolkit" +version = "3.0.48" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "wcwidth" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/2d/4f/feb5e137aff82f7c7f3248267b97451da3644f6cdc218edfe549fb354127/prompt_toolkit-3.0.48.tar.gz", hash = "sha256:d6623ab0477a80df74e646bdbc93621143f5caf104206aa29294d53de1a03d90", size = 424684 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a9/6a/fd08d94654f7e67c52ca30523a178b3f8ccc4237fce4be90d39c938a831a/prompt_toolkit-3.0.48-py3-none-any.whl", hash = "sha256:f49a827f90062e411f1ce1f854f2aedb3c23353244f8108b89283587397ac10e", size = 386595 }, +] + [[package]] name = "proto-plus" version = "1.24.0" @@ -2056,6 +2187,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993 }, ] +[[package]] +name = "pure-eval" +version = "0.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cd/05/0a34433a064256a578f1783a10da6df098ceaa4a57bbeaa96a6c0352786b/pure_eval-0.2.3.tar.gz", hash = "sha256:5f4e983f40564c576c7c8635ae88db5956bb2229d7e9237d03b3c0b0190eaf42", size = 19752 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8e/37/efad0257dc6e593a18957422533ff0f87ede7c9c6ea010a2177d738fb82f/pure_eval-0.2.3-py3-none-any.whl", hash = "sha256:1db8e35b67b3d218d818ae653e27f06c3aa420901fa7b081ca98cbedc874e0d0", size = 11842 }, +] + [[package]] name = "py-spy" version = "0.3.14" @@ -2852,6 +2992,20 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/09/14/5c9b872fba29ccedeb905d0a5c203ad86287b8bb1bb8eda96bfe8a05f65b/SQLAlchemy-2.0.34-py3-none-any.whl", hash = "sha256:7286c353ee6475613d8beff83167374006c6b3e3f0e6491bfe8ca610eb1dec0f", size = 1880671 }, ] +[[package]] +name = "stack-data" +version = "0.6.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "asttokens" }, + { name = "executing" }, + { name = "pure-eval" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/28/e3/55dcc2cfbc3ca9c29519eb6884dd1415ecb53b0e934862d3559ddcb7e20b/stack_data-0.6.3.tar.gz", hash = "sha256:836a778de4fec4dcd1dcd89ed8abff8a221f58308462e1c4aa2a3cf30148f0b9", size = 44707 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f1/7b/ce1eafaf1a76852e2ec9b22edecf1daa58175c090266e9f6c64afcd81d91/stack_data-0.6.3-py3-none-any.whl", hash = "sha256:d5558e0c25a4cb0853cddad3d77da9891a08cb85dd9f9f91b9f8cd66e511e695", size = 24521 }, +] + [[package]] name = "starlette" version = "0.38.4" @@ -2955,6 +3109,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/48/5d/acf5905c36149bbaec41ccf7f2b68814647347b72075ac0b1fe3022fdc73/tqdm-4.66.5-py3-none-any.whl", hash = "sha256:90279a3770753eafc9194a0364852159802111925aa30eb3f9d85b0e805ac7cd", size = 78351 }, ] +[[package]] +name = "traitlets" +version = "5.14.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/eb/79/72064e6a701c2183016abbbfedaba506d81e30e232a68c9f0d6f6fcd1574/traitlets-5.14.3.tar.gz", hash = "sha256:9ed0579d3502c94b4b3732ac120375cda96f923114522847de4b3bb98b96b6b7", size = 161621 } +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 = "trove-classifiers" version = "2024.9.12" @@ -3261,6 +3424,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/fb/fc/885015d4a17ada85508e406c10d638808e7bfbb5622a2e342c868ede18c0/watchfiles-0.24.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:49fb58bcaa343fedc6a9e91f90195b20ccb3135447dc9e4e2570c3a39565853e", size = 428343 }, ] +[[package]] +name = "wcwidth" +version = "0.2.13" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6c/63/53559446a878410fc5a5974feb13d31d78d752eb18aeba59c7fef1af7598/wcwidth-0.2.13.tar.gz", hash = "sha256:72ea0c06399eb286d978fdedb6923a9eb47e1c486ce63e9b4e64fc18303972b5", size = 101301 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/fd/84/fd2ba7aafacbad3c4201d395674fc6348826569da3c0937e75505ead3528/wcwidth-0.2.13-py2.py3-none-any.whl", hash = "sha256:3da69048e4540d84af32131829ff948f1e022c1c6bdb8d6102117aac784f6859", size = 34166 }, +] + [[package]] name = "websocket-client" version = "1.8.0"