ray/dashboard/modules/job/sdk.py
Alan Guo 5d6bc5360d
Fix the jobs tab in the beta dashboard and fill it with data from both "submission" jobs and "driver" jobs (#25902)
## Why are these changes needed?
- Fixes the jobs tab in the new dashboard. Previously it didn't load.
- Combines the old job concept, "driver jobs" and the new job submission conception into a single concept called "jobs". Jobs tab shows information about both jobs.

- Updates all job APIs: They now returns both submission jobs and driver jobs. They also contains additional data in the response including "id", "job_id", "submission_id", and "driver". They also accept either job_id or submission_id as input.

- Job ID is the same as the "ray core job id" concept. It is in the form of "0100000" and is the primary id to represent jobs.
- Submission ID is an ID that is generated for each ray job submission. It is in the form of "raysubmit_12345...". It is a secondary id that can be used if a client needs to provide a self-generated id. or if the job id doesn't exist (ex: if the submission job doesn't create a ray driver)

This PR has 2 deprecations
- The `submit_job` sdk now accepts a new kwarg `submission_id`. `job_id is deprecated.
- The `ray job submit` CLI now accepts `--submission-id`. `--job-id` is deprecated.

**This PR has 4 backwards incompatible changes:**
- list_jobs sdk now returns a list instead of a dictionary
- the `ray job list` CLI now prints a list instead of a dictionary
- The `/api/jobs` endpoint returns a list instead of a dictionary
- The `POST api/jobs` endpoint (submit job) now returns a json with `submission_id` field instead of `job_id`.
2022-07-27 02:39:52 -07:00

359 lines
13 KiB
Python

import dataclasses
import logging
from typing import Any, Dict, Iterator, List, Optional
try:
import aiohttp
import requests
from ray.dashboard.modules.job.pydantic_models import (
JobDetails,
)
except ImportError:
aiohttp = None
requests = None
JobDetails = None
from ray.dashboard.modules.job.common import (
JobStatus,
JobSubmitRequest,
JobSubmitResponse,
JobStopResponse,
JobLogsResponse,
)
from ray.dashboard.modules.dashboard_sdk import SubmissionClient
from ray.runtime_env import RuntimeEnv
from ray.util.annotations import PublicAPI
logger = logging.getLogger(__name__)
logger.setLevel(logging.INFO)
class JobSubmissionClient(SubmissionClient):
"""A local client for submitting and interacting with jobs on a remote cluster.
Submits requests over HTTP to the job server on the cluster using the REST API.
"""
def __init__(
self,
address: Optional[str] = None,
create_cluster_if_needed: bool = False,
cookies: Optional[Dict[str, Any]] = None,
metadata: Optional[Dict[str, Any]] = None,
headers: Optional[Dict[str, Any]] = None,
):
"""Initialize a JobSubmissionClient and check the connection to the cluster.
Args:
address: The IP address and port of the head node. Defaults to
http://localhost:8265.
create_cluster_if_needed: Indicates whether the cluster at the specified
address needs to already be running. Ray doesn't start a cluster
before interacting with jobs, but external job managers may do so.
cookies: Cookies to use when sending requests to the HTTP job server.
metadata: Arbitrary metadata to store along with all jobs. New metadata
specified per job will be merged with the global metadata provided here
via a simple dict update.
headers: Headers to use when sending requests to the HTTP job server, used
for cases like authentication to a remote cluster.
"""
if requests is None:
raise RuntimeError(
"The Ray jobs CLI & SDK require the ray[default] "
"installation: `pip install 'ray[default']``"
)
super().__init__(
address=address,
create_cluster_if_needed=create_cluster_if_needed,
cookies=cookies,
metadata=metadata,
headers=headers,
)
self._check_connection_and_version(
min_version="1.9",
version_error_message="Jobs API is not supported on the Ray "
"cluster. Please ensure the cluster is "
"running Ray 1.9 or higher.",
)
@PublicAPI(stability="beta")
def submit_job(
self,
*,
entrypoint: str,
job_id: Optional[str] = None,
runtime_env: Optional[Dict[str, Any]] = None,
metadata: Optional[Dict[str, str]] = None,
submission_id: Optional[str] = None,
) -> str:
"""Submit and execute a job asynchronously.
When a job is submitted, it runs once to completion or failure. Retries or
different runs with different parameters should be handled by the
submitter. Jobs are bound to the lifetime of a Ray cluster, so if the
cluster goes down, all running jobs on that cluster will be terminated.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> client.submit_job( # doctest: +SKIP
... entrypoint="python script.py",
... runtime_env={
... "working_dir": "./",
... "pip": ["requests==2.26.0"]
... }
... ) # doctest: +SKIP
'raysubmit_4LamXRuQpYdSMg7J'
Args:
entrypoint: The shell command to run for this job.
submission_id: A unique ID for this job.
runtime_env: The runtime environment to install and run this job in.
metadata: Arbitrary data to store along with this job.
job_id: DEPRECATED. This has been renamed to submission_id
Returns:
The submission ID of the submitted job. If not specified,
this is a randomly generated unique ID.
Raises:
RuntimeError: If the request to the job server fails, or if the specified
submission_id has already been used by a job on this cluster.
"""
if job_id:
logger.warning(
"job_id kwarg is deprecated. Please use submission_id instead."
)
runtime_env = runtime_env or {}
metadata = metadata or {}
metadata.update(self._default_metadata)
self._upload_working_dir_if_needed(runtime_env)
self._upload_py_modules_if_needed(runtime_env)
# Run the RuntimeEnv constructor to parse local pip/conda requirements files.
runtime_env = RuntimeEnv(**runtime_env).to_dict()
submission_id = submission_id or job_id
req = JobSubmitRequest(
entrypoint=entrypoint,
submission_id=submission_id,
runtime_env=runtime_env,
metadata=metadata,
)
logger.debug(f"Submitting job with submission_id={submission_id}.")
r = self._do_request("POST", "/api/jobs/", json_data=dataclasses.asdict(req))
if r.status_code == 200:
return JobSubmitResponse(**r.json()).submission_id
else:
self._raise_error(r)
@PublicAPI(stability="beta")
def stop_job(
self,
job_id: str,
) -> bool:
"""Request a job to exit asynchronously.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> sub_id = client.submit_job(entrypoint="sleep 10") # doctest: +SKIP
>>> client.stop_job(sub_id) # doctest: +SKIP
True
Args:
job_id: The job ID or submission ID for the job to be stopped.
Returns:
True if the job was running, otherwise False.
Raises:
RuntimeError: If the job does not exist or if the request to the
job server fails.
"""
logger.debug(f"Stopping job with job_id={job_id}.")
r = self._do_request("POST", f"/api/jobs/{job_id}/stop")
if r.status_code == 200:
return JobStopResponse(**r.json()).stopped
else:
self._raise_error(r)
@PublicAPI(stability="beta")
def get_job_info(
self,
job_id: str,
) -> JobDetails:
"""Get the latest status and other information associated with a job.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> submission_id = client.submit_job(entrypoint="sleep 1") # doctest: +SKIP
>>> job_submission_client.get_job_info(submission_id) # doctest: +SKIP
JobInfo(status='SUCCEEDED', message='Job finished successfully.',
error_type=None, start_time=1647388711, end_time=1647388712,
metadata={}, runtime_env={})
Args:
job_id: The job ID or submission ID of the job whose information
is being requested.
Returns:
The JobInfo for the job.
Raises:
RuntimeError: If the job does not exist or if the request to the
job server fails.
"""
r = self._do_request("GET", f"/api/jobs/{job_id}")
if r.status_code == 200:
return JobDetails(**r.json())
else:
self._raise_error(r)
@PublicAPI(stability="beta")
def list_jobs(self) -> List[JobDetails]:
"""List all jobs along with their status and other information.
Lists all jobs that have ever run on the cluster, including jobs that are
currently running and jobs that are no longer running.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> client.submit_job(entrypoint="echo hello") # doctest: +SKIP
>>> client.submit_job(entrypoint="sleep 2") # doctest: +SKIP
>>> client.list_jobs() # doctest: +SKIP
[JobDetails(status='SUCCEEDED',
job_id='03000000', type='submission',
submission_id='raysubmit_4LamXRuQpYdSMg7J',
message='Job finished successfully.', error_type=None,
start_time=1647388711, end_time=1647388712, metadata={}, runtime_env={}),
JobDetails(status='RUNNING',
job_id='04000000', type='submission',
submission_id='raysubmit_1dxCeNvG1fCMVNHG',
message='Job is currently running.', error_type=None,
start_time=1647454832, end_time=None, metadata={}, runtime_env={})]
Returns:
A dictionary mapping job_ids to their information.
Raises:
RuntimeError: If the request to the job server fails.
"""
r = self._do_request("GET", "/api/jobs/")
if r.status_code == 200:
jobs_info_json = r.json()
jobs_info = [
JobDetails(**job_info_json) for job_info_json in jobs_info_json
]
return jobs_info
else:
self._raise_error(r)
@PublicAPI(stability="beta")
def get_job_status(self, job_id: str) -> JobStatus:
"""Get the most recent status of a job.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> client.submit_job(entrypoint="echo hello") # doctest: +SKIP
>>> client.get_job_status("raysubmit_4LamXRuQpYdSMg7J") # doctest: +SKIP
'SUCCEEDED'
Args:
job_id: The job ID or submission ID of the job whose status is being
requested.
Returns:
The JobStatus of the job.
Raises:
RuntimeError: If the job does not exist or if the request to the
job server fails.
"""
return self.get_job_info(job_id).status
@PublicAPI(stability="beta")
def get_job_logs(self, job_id: str) -> str:
"""Get all logs produced by a job.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> sub_id = client.submit_job(entrypoint="echo hello") # doctest: +SKIP
>>> client.get_job_logs(sub_id) # doctest: +SKIP
'hello\\n'
Args:
job_id: The job ID or submission ID of the job whose logs are being
requested.
Returns:
A string containing the full logs of the job.
Raises:
RuntimeError: If the job does not exist or if the request to the
job server fails.
"""
r = self._do_request("GET", f"/api/jobs/{job_id}/logs")
if r.status_code == 200:
return JobLogsResponse(**r.json()).logs
else:
self._raise_error(r)
@PublicAPI(stability="beta")
async def tail_job_logs(self, job_id: str) -> Iterator[str]:
"""Get an iterator that follows the logs of a job.
Example:
>>> from ray.job_submission import JobSubmissionClient
>>> client = JobSubmissionClient("http://127.0.0.1:8265") # doctest: +SKIP
>>> submission_id = client.submit_job( # doctest: +SKIP
... entrypoint="echo hi && sleep 5 && echo hi2")
>>> async for lines in client.tail_job_logs( # doctest: +SKIP
... 'raysubmit_Xe7cvjyGJCyuCvm2'):
... print(lines, end="") # doctest: +SKIP
hi
hi2
Args:
job_id: The job ID or submission ID of the job whose logs are being
requested.
Returns:
The iterator.
Raises:
RuntimeError: If the job does not exist or if the request to the
job server fails.
"""
async with aiohttp.ClientSession(
cookies=self._cookies, headers=self._headers
) as session:
ws = await session.ws_connect(
f"{self._address}/api/jobs/{job_id}/logs/tail"
)
while True:
msg = await ws.receive()
if msg.type == aiohttp.WSMsgType.TEXT:
yield msg.data
elif msg.type == aiohttp.WSMsgType.CLOSED:
break
elif msg.type == aiohttp.WSMsgType.ERROR:
pass