Skip to content

Commit

Permalink
[api] API deprecations and cleanups for 1.0 (internal_config and Chec…
Browse files Browse the repository at this point in the history
…kpointable actor) (ray-project#10333)

* remove

* internal config updates, remove Checkpointable

* Lower object timeout default

* remove json

* Fix flaky test

* Fix unit test
  • Loading branch information
stephanie-wang authored Aug 27, 2020
1 parent 0aec4cb commit f75dfd6
Show file tree
Hide file tree
Showing 56 changed files with 239 additions and 1,267 deletions.
5 changes: 2 additions & 3 deletions dashboard/tests/test_dashboard.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
import os
import json
import time
import logging

Expand Down Expand Up @@ -42,9 +41,9 @@ def prepare_test_files():

@pytest.mark.parametrize(
"ray_start_with_dashboard", [{
"_internal_config": json.dumps({
"_system_config": {
"agent_register_timeout_ms": 5000
})
}
}],
indirect=True)
def test_basic(ray_start_with_dashboard):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public void setUp() {
"--load-code-from-local",
"--include-java",
"--java-worker-options=" + workerOptions,
"--internal-config=" + new Gson().toJson(RayConfig.create().rayletConfigParameters)
"--system-config=" + new Gson().toJson(RayConfig.create().rayletConfigParameters)
);
if (!executeCommand(startCommand, 10, getRayStartEnv())) {
throw new RuntimeException("Couldn't start ray cluster.");
Expand Down
146 changes: 0 additions & 146 deletions python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,6 @@
import logging
import weakref

from abc import ABCMeta, abstractmethod
from collections import namedtuple
import ray.ray_constants as ray_constants
import ray._raylet
import ray.signature as signature
Expand Down Expand Up @@ -854,11 +852,6 @@ def modify_class(cls):
"classes. In Python 2, you must declare the class with "
"'class ClassName(object):' instead of 'class ClassName:'.")

if issubclass(cls, Checkpointable) and inspect.isabstract(cls):
raise TypeError(
"A checkpointable actor class should implement all abstract "
"methods in the `Checkpointable` interface.")

# Modify the class to have an additional method that will be used for
# terminating the worker.
class Class(cls):
Expand All @@ -869,20 +862,6 @@ def __ray_terminate__(self):
if worker.mode != ray.LOCAL_MODE:
ray.actor.exit_actor()

def __ray_checkpoint__(self):
"""Save a checkpoint.
This task saves the current state of the actor, the current task
frontier according to the raylet, and the checkpoint index
(number of tasks executed so far).
"""
worker = ray.worker.global_worker
if not isinstance(self, ray.actor.Checkpointable):
raise TypeError(
"__ray_checkpoint__.remote() may only be called on actors "
"that implement ray.actor.Checkpointable")
return worker._save_actor_checkpoint()

Class.__module__ = cls.__module__
Class.__name__ = cls.__name__

Expand Down Expand Up @@ -951,128 +930,3 @@ def exit_actor():
assert False, "This process should have terminated."
else:
raise TypeError("exit_actor called on a non-actor worker.")


CheckpointContext = namedtuple(
"CheckpointContext",
[
# Actor's ID.
"actor_id",
# Number of tasks executed since last checkpoint.
"num_tasks_since_last_checkpoint",
# Time elapsed since last checkpoint, in milliseconds.
"time_elapsed_ms_since_last_checkpoint",
],
)
"""A namedtuple that contains information about actor's last checkpoint."""

Checkpoint = namedtuple(
"Checkpoint",
[
# ID of this checkpoint.
"checkpoint_id",
# The timestamp at which this checkpoint was saved,
# represented as milliseconds elapsed since Unix epoch.
"timestamp",
],
)
"""A namedtuple that represents a checkpoint."""


class Checkpointable(metaclass=ABCMeta):
"""An interface that indicates an actor can be checkpointed."""

@abstractmethod
def should_checkpoint(self, checkpoint_context):
"""Whether this actor needs to be checkpointed.
This method will be called after every task. You should implement this
callback to decide whether this actor needs to be checkpointed at this
time, based on the checkpoint context, or any other factors.
Args:
checkpoint_context: A namedtuple that contains info about last
checkpoint.
Returns:
A boolean value that indicates whether this actor needs to be
checkpointed.
"""
pass

@abstractmethod
def save_checkpoint(self, actor_id, checkpoint_id):
"""Save a checkpoint to persistent storage.
If `should_checkpoint` returns true, this method will be called. You
should implement this callback to save actor's checkpoint and the given
checkpoint id to persistent storage.
Args:
actor_id: Actor's ID.
checkpoint_id: ID of this checkpoint. You should save it together
with actor's checkpoint data. And it will be used by the
`load_checkpoint` method.
Returns:
None.
"""
pass

@abstractmethod
def load_checkpoint(self, actor_id, available_checkpoints):
"""Load actor's previous checkpoint, and restore actor's state.
This method will be called when an actor is restarted, after
actor's constructor.
If the actor needs to restore from previous checkpoint, this function
should restore actor's state and return the checkpoint ID. Otherwise,
it should do nothing and return None.
Note, this method must return one of the checkpoint IDs in the
`available_checkpoints` list, or None. Otherwise, an exception will be
raised.
Args:
actor_id: Actor's ID.
available_checkpoints: A list of `Checkpoint` namedtuples that
contains all available checkpoint IDs and their timestamps,
sorted by timestamp in descending order.
Returns:
The ID of the checkpoint from which the actor was resumed, or None
if the actor should restart from the beginning.
"""
pass

@abstractmethod
def checkpoint_expired(self, actor_id, checkpoint_id):
"""Delete an expired checkpoint.
This method will be called when an checkpoint is expired. You should
implement this method to delete your application checkpoint data.
Note, the maximum number of checkpoints kept in the backend can be
configured at `RayConfig.num_actor_checkpoints_to_keep`.
Args:
actor_id: ID of the actor.
checkpoint_id: ID of the checkpoint that has expired.
Returns:
None.
"""
pass


def get_checkpoints_for_actor(actor_id):
"""Get the available checkpoints for the given actor ID, return a list
sorted by checkpoint timestamp in descending order.
"""
checkpoint_info = ray.state.state.actor_checkpoint_info(actor_id)
if checkpoint_info is None:
return []
checkpoints = [
Checkpoint(checkpoint_id, timestamp) for checkpoint_id, timestamp in
zip(checkpoint_info["CheckpointIds"], checkpoint_info["Timestamps"])
]
return sorted(
checkpoints,
key=lambda checkpoint: checkpoint.timestamp,
reverse=True,
)
4 changes: 0 additions & 4 deletions python/ray/cluster_utils.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
import json
import logging
import time

Expand Down Expand Up @@ -80,9 +79,6 @@ def add_node(self, **node_args):
"min_worker_port": 0,
"max_worker_port": 0,
}
if "_internal_config" in node_args:
node_args["_internal_config"] = json.loads(
node_args["_internal_config"])
ray_params = ray.parameter.RayParams(**node_args)
ray_params.update_if_absent(**default_kwargs)
if self.head_node is None:
Expand Down
125 changes: 6 additions & 119 deletions python/ray/function_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -544,43 +544,13 @@ def _make_actor_method_executor(self, method_name, method, actor_imported):
"""

def actor_method_executor(actor, *args, **kwargs):
# Update the actor's task counter to reflect the task we're about
# to execute.
self._worker.actor_task_counter += 1

# Execute the assigned method and save a checkpoint if necessary.
try:
is_bound = (is_class_method(method)
or is_static_method(type(actor), method_name))
if is_bound:
method_returns = method(*args, **kwargs)
else:
method_returns = method(actor, *args, **kwargs)
except Exception as e:
# Save the checkpoint before allowing the method exception
# to be thrown, but don't save the checkpoint for actor
# creation task.
if (isinstance(actor, ray.actor.Checkpointable)
and self._worker.actor_task_counter != 1):
self._save_and_log_checkpoint(actor)
raise e
# Execute the assigned method.
is_bound = (is_class_method(method)
or is_static_method(type(actor), method_name))
if is_bound:
return method(*args, **kwargs)
else:
# Handle any checkpointing operations before storing the
# method's return values.
# NOTE(swang): If method_returns is a pointer to the actor's
# state and the checkpointing operations can modify the return
# values if they mutate the actor's state. Is this okay?
if isinstance(actor, ray.actor.Checkpointable):
# If this is the first task to execute on the actor, try to
# resume from a checkpoint.
if self._worker.actor_task_counter == 1:
if actor_imported:
self._restore_and_log_checkpoint(actor)
else:
# Save the checkpoint before returning the method's
# return values.
self._save_and_log_checkpoint(actor)
return method_returns
return method(actor, *args, **kwargs)

# Set method_name and method as attributes to the executor clusore
# so we can make decision based on these attributes in task executor.
Expand All @@ -591,86 +561,3 @@ def actor_method_executor(actor, *args, **kwargs):
actor_method_executor.method = method

return actor_method_executor

def _save_and_log_checkpoint(self, actor):
"""Save an actor checkpoint if necessary and log any errors.
Args:
actor: The actor to checkpoint.
Returns:
The result of the actor's user-defined `save_checkpoint` method.
"""
actor_id = self._worker.actor_id
checkpoint_info = self._worker.actor_checkpoint_info[actor_id]
checkpoint_info.num_tasks_since_last_checkpoint += 1
now = int(1000 * time.time())
checkpoint_context = ray.actor.CheckpointContext(
actor_id, checkpoint_info.num_tasks_since_last_checkpoint,
now - checkpoint_info.last_checkpoint_timestamp)
# If we should take a checkpoint, notify raylet to prepare a checkpoint
# and then call `save_checkpoint`.
if actor.should_checkpoint(checkpoint_context):
try:
now = int(1000 * time.time())
checkpoint_id = (
self._worker.core_worker.prepare_actor_checkpoint(actor_id)
)
checkpoint_info.checkpoint_ids.append(checkpoint_id)
actor.save_checkpoint(actor_id, checkpoint_id)
if (len(checkpoint_info.checkpoint_ids) >
ray._config.num_actor_checkpoints_to_keep()):
actor.checkpoint_expired(
actor_id,
checkpoint_info.checkpoint_ids.pop(0),
)
checkpoint_info.num_tasks_since_last_checkpoint = 0
checkpoint_info.last_checkpoint_timestamp = now
except Exception:
# Checkpoint save or reload failed. Notify the driver.
traceback_str = ray.utils.format_error_message(
traceback.format_exc())
ray.utils.push_error_to_driver(
self._worker,
ray_constants.CHECKPOINT_PUSH_ERROR,
traceback_str,
job_id=self._worker.current_job_id)

def _restore_and_log_checkpoint(self, actor):
"""Restore an actor from a checkpoint if available and log any errors.
This should only be called on workers that have just executed an actor
creation task.
Args:
actor: The actor to restore from a checkpoint.
"""
actor_id = self._worker.actor_id
try:
checkpoints = ray.actor.get_checkpoints_for_actor(actor_id)
if len(checkpoints) > 0:
# If we found previously saved checkpoints for this actor,
# call the `load_checkpoint` callback.
checkpoint_id = actor.load_checkpoint(actor_id, checkpoints)
if checkpoint_id is not None:
# Check that the returned checkpoint id is in the
# `available_checkpoints` list.
msg = (
"`load_checkpoint` must return a checkpoint id that " +
"exists in the `available_checkpoints` list, or None.")
assert any(checkpoint_id == checkpoint.checkpoint_id
for checkpoint in checkpoints), msg
# Notify raylet that this actor has been resumed from
# a checkpoint.
(self._worker.core_worker.
notify_actor_resumed_from_checkpoint(
actor_id, checkpoint_id))
except Exception:
# Checkpoint save or reload failed. Notify the driver.
traceback_str = ray.utils.format_error_message(
traceback.format_exc())
ray.utils.push_error_to_driver(
self._worker,
ray_constants.CHECKPOINT_PUSH_ERROR,
traceback_str,
job_id=self._worker.current_job_id)
Loading

0 comments on commit f75dfd6

Please sign in to comment.