ray/dashboard/modules/job/common.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

245 lines
8.5 KiB
Python
Raw Normal View History

import pickle
import time
from dataclasses import dataclass, replace
from enum import Enum
from pathlib import Path
from typing import Any, Dict, Optional, Tuple
from ray._private import ray_constants
from ray._private.runtime_env.packaging import parse_uri
from ray.experimental.internal_kv import (
_internal_kv_get,
_internal_kv_initialized,
_internal_kv_list,
_internal_kv_put,
)
# NOTE(edoakes): these constants should be considered a public API because
# they're exposed in the snapshot API.
JOB_ID_METADATA_KEY = "job_submission_id"
JOB_NAME_METADATA_KEY = "job_name"
class JobStatus(str, Enum):
"""An enumeration for describing the status of a job."""
#: The job has not started yet, likely waiting for the runtime_env to be set up.
PENDING = "PENDING"
#: The job is currently running.
RUNNING = "RUNNING"
#: The job was intentionally stopped by the user.
STOPPED = "STOPPED"
#: The job finished successfully.
SUCCEEDED = "SUCCEEDED"
#: The job failed.
FAILED = "FAILED"
def __str__(self) -> str:
return f"{self.value}"
def is_terminal(self) -> bool:
"""Return whether or not this status is terminal.
A terminal status is one that cannot transition to any other status.
The terminal statuses are "STOPPED", "SUCCEEDED", and "FAILED".
Returns:
True if this status is terminal, otherwise False.
"""
return self.value in {"STOPPED", "SUCCEEDED", "FAILED"}
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
# TODO(aguo): Convert to pydantic model
@dataclass
class JobInfo:
"""A class for recording information associated with a job and its execution."""
#: The status of the job.
status: JobStatus
#: The entrypoint command for this job.
entrypoint: str
#: A message describing the status in more detail.
message: Optional[str] = None
# TODO(architkulkarni): Populate this field with e.g. Runtime env setup failure,
# Internal error, user script error
error_type: Optional[str] = None
[Jobs] Change jobs start_time end_time from seconds to ms for consistency (#24123) In the snapshot, all timestamps are given in ms except for Jobs: ``` wget -q -O - http://127.0.0.1:8265/api/snapshot { "result":true, "msg":"hello", "data":{ "snapshot":{ "jobs":{ "01000000":{ "status":null, "statusMessage":null, "isDead":false, "startTime":1650315791249, "endTime":0, "config":{ "namespace":"_ray_internal_dashboard", "metadata":{ }, "runtimeEnv":{ } } } }, "jobSubmission":{ "raysubmit9Bsej1Rtxqqetxup":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650315925, "endTime":1650315926, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"ls" }, "raysubmitEibragqkyg16Hpcj":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316039, "endTime":1650316041, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" }, "raysubmitSh1U7Grdsbqrf6Je":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316354, "endTime":1650316355, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" } }, "actors":{ "8c8e28e642ba2cfd0457d45e01000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_9BSeJ1rTXQqEtXuP", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650315926620, "endTime":1650315927499, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"9628b5eb54e98353601413845fbca0a8c4e5379d1469ce95f3dfbace", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10003, "metadata":{ } }, "a7fd8354567129910c44298401000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_sh1u7grDsBQRf6je", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316355718, "endTime":1650316356620, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"f07fd7a393898bf7d9027a5de0b0f566bb64ae80c0fcbcc107185505", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10005, "metadata":{ } }, "19ca9ad190f47bae963592d601000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_eibRAGqKyG16HpCj", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316041089, "endTime":1650316041978, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"50b8e7e9a6981fe0270afd7f6387bc93788356822c9a664c2988f5ba", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10004, "metadata":{ } } }, "deployments":{ }, "sessionName":"session_2022-04-18_13-49-44_814862_139", "rayVersion":"1.12.0", "rayCommit":"f18fc31c7562990955556899090f8e8656b48d2d" } } } ``` This PR fixes the inconsistency by changing Jobs start/end timestamps to ms.
2022-04-26 08:37:41 -07:00
#: The time when the job was started. A Unix timestamp in ms.
start_time: Optional[int] = None
[Jobs] Change jobs start_time end_time from seconds to ms for consistency (#24123) In the snapshot, all timestamps are given in ms except for Jobs: ``` wget -q -O - http://127.0.0.1:8265/api/snapshot { "result":true, "msg":"hello", "data":{ "snapshot":{ "jobs":{ "01000000":{ "status":null, "statusMessage":null, "isDead":false, "startTime":1650315791249, "endTime":0, "config":{ "namespace":"_ray_internal_dashboard", "metadata":{ }, "runtimeEnv":{ } } } }, "jobSubmission":{ "raysubmit9Bsej1Rtxqqetxup":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650315925, "endTime":1650315926, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"ls" }, "raysubmitEibragqkyg16Hpcj":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316039, "endTime":1650316041, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" }, "raysubmitSh1U7Grdsbqrf6Je":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316354, "endTime":1650316355, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" } }, "actors":{ "8c8e28e642ba2cfd0457d45e01000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_9BSeJ1rTXQqEtXuP", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650315926620, "endTime":1650315927499, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"9628b5eb54e98353601413845fbca0a8c4e5379d1469ce95f3dfbace", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10003, "metadata":{ } }, "a7fd8354567129910c44298401000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_sh1u7grDsBQRf6je", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316355718, "endTime":1650316356620, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"f07fd7a393898bf7d9027a5de0b0f566bb64ae80c0fcbcc107185505", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10005, "metadata":{ } }, "19ca9ad190f47bae963592d601000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_eibRAGqKyG16HpCj", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316041089, "endTime":1650316041978, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"50b8e7e9a6981fe0270afd7f6387bc93788356822c9a664c2988f5ba", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10004, "metadata":{ } } }, "deployments":{ }, "sessionName":"session_2022-04-18_13-49-44_814862_139", "rayVersion":"1.12.0", "rayCommit":"f18fc31c7562990955556899090f8e8656b48d2d" } } } ``` This PR fixes the inconsistency by changing Jobs start/end timestamps to ms.
2022-04-26 08:37:41 -07:00
#: The time when the job moved into a terminal state. A Unix timestamp in ms.
end_time: Optional[int] = None
#: Arbitrary user-provided metadata for the job.
metadata: Optional[Dict[str, str]] = None
#: The runtime environment for the job.
runtime_env: Optional[Dict[str, Any]] = None
def __post_init__(self):
if self.message is None:
if self.status == JobStatus.PENDING:
self.message = (
"Job has not started yet, likely waiting "
"for the runtime_env to be set up."
)
elif self.status == JobStatus.RUNNING:
self.message = "Job is currently running."
elif self.status == JobStatus.STOPPED:
self.message = "Job was intentionally stopped."
elif self.status == JobStatus.SUCCEEDED:
self.message = "Job finished successfully."
elif self.status == JobStatus.FAILED:
self.message = "Job failed."
class JobInfoStorageClient:
"""
Interface to put and get job data from the Internal KV store.
"""
JOB_DATA_KEY_PREFIX = f"{ray_constants.RAY_INTERNAL_NAMESPACE_PREFIX}job_info_"
JOB_DATA_KEY = f"{JOB_DATA_KEY_PREFIX}{{job_id}}"
def __init__(self):
assert _internal_kv_initialized()
def put_info(self, job_id: str, data: JobInfo):
_internal_kv_put(
self.JOB_DATA_KEY.format(job_id=job_id),
pickle.dumps(data),
namespace=ray_constants.KV_NAMESPACE_JOB,
)
def get_info(self, job_id: str) -> Optional[JobInfo]:
pickled_info = _internal_kv_get(
self.JOB_DATA_KEY.format(job_id=job_id),
namespace=ray_constants.KV_NAMESPACE_JOB,
)
if pickled_info is None:
return None
else:
return pickle.loads(pickled_info)
def put_status(self, job_id: str, status: JobStatus, message: Optional[str] = None):
"""Puts or updates job status. Sets end_time if status is terminal."""
old_info = self.get_info(job_id)
if old_info is not None:
if status != old_info.status and old_info.status.is_terminal():
assert False, "Attempted to change job status from a terminal state."
new_info = replace(old_info, status=status, message=message)
else:
new_info = JobInfo(
entrypoint="Entrypoint not found.", status=status, message=message
)
if status.is_terminal():
[Jobs] Change jobs start_time end_time from seconds to ms for consistency (#24123) In the snapshot, all timestamps are given in ms except for Jobs: ``` wget -q -O - http://127.0.0.1:8265/api/snapshot { "result":true, "msg":"hello", "data":{ "snapshot":{ "jobs":{ "01000000":{ "status":null, "statusMessage":null, "isDead":false, "startTime":1650315791249, "endTime":0, "config":{ "namespace":"_ray_internal_dashboard", "metadata":{ }, "runtimeEnv":{ } } } }, "jobSubmission":{ "raysubmit9Bsej1Rtxqqetxup":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650315925, "endTime":1650315926, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"ls" }, "raysubmitEibragqkyg16Hpcj":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316039, "endTime":1650316041, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" }, "raysubmitSh1U7Grdsbqrf6Je":{ "status":"SUCCEEDED", "message":"Job finished successfully.", "errorType":null, "startTime":1650316354, "endTime":1650316355, "metadata":{ "creatorId":"usr_f6tgCaaFBJC6tZz1ZVzzAVf4" }, "runtimeEnv":{ "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "entrypoint":"echo hi" } }, "actors":{ "8c8e28e642ba2cfd0457d45e01000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_9BSeJ1rTXQqEtXuP", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650315926620, "endTime":1650315927499, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"9628b5eb54e98353601413845fbca0a8c4e5379d1469ce95f3dfbace", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10003, "metadata":{ } }, "a7fd8354567129910c44298401000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_sh1u7grDsBQRf6je", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316355718, "endTime":1650316356620, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"f07fd7a393898bf7d9027a5de0b0f566bb64ae80c0fcbcc107185505", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10005, "metadata":{ } }, "19ca9ad190f47bae963592d601000000":{ "jobId":"01000000", "state":"DEAD", "name":"_ray_internal_job_actor_raysubmit_eibRAGqKyG16HpCj", "namespace":"_ray_internal_dashboard", "runtimeEnv":{ "uris":{ "workingDirUri":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "workingDir":"gcs://_ray_pkg_6068c19fb3b8530f.zip" }, "startTime":1650316041089, "endTime":1650316041978, "isDetached":true, "resources":{ "node:172.31.73.39":0.001 }, "actorClass":"JobSupervisor", "currentWorkerId":"50b8e7e9a6981fe0270afd7f6387bc93788356822c9a664c2988f5ba", "currentRayletId":"61ab3958258c82266b222f4691a53e71b6315e312408a21cb3350bc7", "ipAddress":"172.31.73.39", "port":10004, "metadata":{ } } }, "deployments":{ }, "sessionName":"session_2022-04-18_13-49-44_814862_139", "rayVersion":"1.12.0", "rayCommit":"f18fc31c7562990955556899090f8e8656b48d2d" } } } ``` This PR fixes the inconsistency by changing Jobs start/end timestamps to ms.
2022-04-26 08:37:41 -07:00
new_info.end_time = int(time.time() * 1000)
self.put_info(job_id, new_info)
def get_status(self, job_id: str) -> Optional[JobStatus]:
job_info = self.get_info(job_id)
if job_info is None:
return None
else:
return job_info.status
def get_all_jobs(self) -> Dict[str, JobInfo]:
raw_job_ids_with_prefixes = _internal_kv_list(
self.JOB_DATA_KEY_PREFIX, namespace=ray_constants.KV_NAMESPACE_JOB
)
job_ids_with_prefixes = [
job_id.decode() for job_id in raw_job_ids_with_prefixes
]
job_ids = []
for job_id_with_prefix in job_ids_with_prefixes:
assert job_id_with_prefix.startswith(
self.JOB_DATA_KEY_PREFIX
), "Unexpected format for internal_kv key for Job submission"
job_ids.append(job_id_with_prefix[len(self.JOB_DATA_KEY_PREFIX) :])
return {job_id: self.get_info(job_id) for job_id in job_ids}
def uri_to_http_components(package_uri: str) -> Tuple[str, str]:
suffix = Path(package_uri).suffix
if suffix not in {".zip", ".whl"}:
raise ValueError(f"package_uri ({package_uri}) does not end in .zip or .whl")
# We need to strip the <protocol>:// prefix to make it possible to pass
# the package_uri over HTTP.
protocol, package_name = parse_uri(package_uri)
return protocol.value, package_name
def http_uri_components_to_uri(protocol: str, package_name: str) -> str:
return f"{protocol}://{package_name}"
def validate_request_type(json_data: Dict[str, Any], request_type: dataclass) -> Any:
return request_type(**json_data)
@dataclass
class JobSubmitRequest:
# Command to start execution, ex: "python script.py"
entrypoint: str
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
# Optional submission_id to specify for the job. If the submission_id
# is not specified, one will be generated. If a job with the same
# submission_id already exists, it will be rejected.
submission_id: Optional[str] = None
# Dict to setup execution environment.
runtime_env: Optional[Dict[str, Any]] = None
# Metadata to pass in to the JobConfig.
metadata: Optional[Dict[str, str]] = None
def __post_init__(self):
if not isinstance(self.entrypoint, str):
raise TypeError(f"entrypoint must be a string, got {type(self.entrypoint)}")
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
if self.submission_id is not None and not isinstance(self.submission_id, str):
raise TypeError(
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
"submission_id must be a string if provided, "
f"got {type(self.submission_id)}"
)
if self.runtime_env is not None:
if not isinstance(self.runtime_env, dict):
raise TypeError(
f"runtime_env must be a dict, got {type(self.runtime_env)}"
)
else:
for k in self.runtime_env.keys():
if not isinstance(k, str):
raise TypeError(
f"runtime_env keys must be strings, got {type(k)}"
)
if self.metadata is not None:
if not isinstance(self.metadata, dict):
raise TypeError(f"metadata must be a dict, got {type(self.metadata)}")
else:
for k in self.metadata.keys():
if not isinstance(k, str):
raise TypeError(f"metadata keys must be strings, got {type(k)}")
for v in self.metadata.values():
if not isinstance(v, str):
raise TypeError(
f"metadata values must be strings, got {type(v)}"
)
@dataclass
class JobSubmitResponse:
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
# DEPRECATED: Use submission_id instead.
job_id: str
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
submission_id: str
@dataclass
class JobStopResponse:
stopped: bool
# TODO(jiaodong): Support log streaming #19415
@dataclass
class JobLogsResponse:
logs: str