Why Gemfury? Push, build, and install  RubyGems npm packages Python packages Maven artifacts PHP packages Go Modules Debian packages RPM packages NuGet packages

Repository URL to install this package:

Details    
ray / dag / class_node.py
Size: Mime:
from typing import Any, Dict, List, Optional, Tuple, Union
from weakref import ReferenceType

import ray
from ray.dag.constants import (
    BIND_INDEX_KEY,
    IS_CLASS_METHOD_OUTPUT_KEY,
    PARENT_CLASS_NODE_KEY,
    PREV_CLASS_METHOD_CALL_KEY,
)
from ray.dag.dag_node import DAGNode
from ray.dag.format_utils import get_dag_node_str
from ray.dag.input_node import InputNode
from ray.util.annotations import DeveloperAPI


@DeveloperAPI
class ClassNode(DAGNode):
    """Represents an actor creation in a Ray task DAG."""

    def __init__(
        self,
        cls,
        cls_args,
        cls_kwargs,
        cls_options,
        other_args_to_resolve=None,
    ):
        self._body = cls
        self._last_call: Optional["ClassMethodNode"] = None
        super().__init__(
            cls_args,
            cls_kwargs,
            cls_options,
            other_args_to_resolve=other_args_to_resolve,
        )

        if self._contains_input_node():
            raise ValueError(
                "InputNode handles user dynamic input the DAG, and "
                "cannot be used as args, kwargs, or other_args_to_resolve "
                "in ClassNode constructor because it is not available at "
                "class construction or binding time."
            )

    def _copy_impl(
        self,
        new_args: List[Any],
        new_kwargs: Dict[str, Any],
        new_options: Dict[str, Any],
        new_other_args_to_resolve: Dict[str, Any],
    ):
        return ClassNode(
            self._body,
            new_args,
            new_kwargs,
            new_options,
            other_args_to_resolve=new_other_args_to_resolve,
        )

    def _execute_impl(self, *args, **kwargs):
        """Executor of ClassNode by ray.remote()

        Args and kwargs are to match base class signature, but not in the
        implementation. All args and kwargs should be resolved and replaced
        with value in bound_args and bound_kwargs via bottom-up recursion when
        current node is executed.
        """
        return (
            ray.remote(self._body)
            .options(**self._bound_options)
            .remote(*self._bound_args, **self._bound_kwargs)
        )

    def _contains_input_node(self) -> bool:
        """Check if InputNode is used in children DAGNodes with current node
        as the root.
        """
        children_dag_nodes = self._get_all_child_nodes()
        for child in children_dag_nodes:
            if isinstance(child, InputNode):
                return True
        return False

    def __getattr__(self, method_name: str):
        # User trying to call .bind() without a bind class method
        if method_name == "bind" and "bind" not in dir(self._body):
            raise AttributeError(f".bind() cannot be used again on {type(self)} ")
        # Raise an error if the method is invalid.
        getattr(self._body, method_name)
        call_node = _UnboundClassMethodNode(self, method_name, {})
        return call_node

    def __str__(self) -> str:
        return get_dag_node_str(self, str(self._body))


class _UnboundClassMethodNode(object):
    def __init__(self, actor: ClassNode, method_name: str, options: dict):
        # TODO(sang): Theoretically, We should use weakref cuz it is
        # a circular dependency but when I used weakref, it fails
        # because we cannot serialize the weakref.
        self._actor = actor
        self._method_name = method_name
        self._options = options

    def bind(self, *args, **kwargs):
        other_args_to_resolve = {
            PARENT_CLASS_NODE_KEY: self._actor,
            PREV_CLASS_METHOD_CALL_KEY: self._actor._last_call,
        }

        node = ClassMethodNode(
            self._method_name,
            args,
            kwargs,
            self._options,
            other_args_to_resolve=other_args_to_resolve,
        )
        self._actor._last_call = node
        return node

    def __getattr__(self, attr: str):
        if attr == "remote":
            raise AttributeError(
                ".remote() cannot be used on ClassMethodNodes. Use .bind() instead "
                "to express an symbolic actor call."
            )
        else:
            return self.__getattribute__(attr)

    def options(self, **options):
        self._options = options
        return self


class _ClassMethodOutput:
    """Represents a class method output in a Ray function DAG."""

    def __init__(self, class_method_call: "ClassMethodNode", output_idx: int):
        # The upstream class method call that returns multiple values.
        self._class_method_call = class_method_call
        # The output index of the return value from the upstream class method call.
        self._output_idx = output_idx

    @property
    def class_method_call(self) -> "ClassMethodNode":
        return self._class_method_call

    @property
    def output_idx(self) -> int:
        return self._output_idx


@DeveloperAPI
class ClassMethodNode(DAGNode):
    """Represents an actor method invocation in a Ray function DAG."""

    def __init__(
        self,
        method_name: str,
        method_args: Tuple[Any],
        method_kwargs: Dict[str, Any],
        method_options: Dict[str, Any],
        other_args_to_resolve: Dict[str, Any],
    ):
        self._bound_args = method_args or []
        self._bound_kwargs = method_kwargs or {}
        self._bound_options = method_options or {}
        self._method_name: str = method_name
        # Parse other_args_to_resolve and assign to variables
        self._parent_class_node: Union[
            ClassNode, ReferenceType["ray._private.actor.ActorHandle"]
        ] = other_args_to_resolve.get(PARENT_CLASS_NODE_KEY)
        # Used to track lineage of ClassMethodCall to preserve deterministic
        # submission and execution order.
        self._prev_class_method_call: Optional[
            ClassMethodNode
        ] = other_args_to_resolve.get(PREV_CLASS_METHOD_CALL_KEY, None)
        # The index/order when bind() is called on this class method
        self._bind_index: Optional[int] = other_args_to_resolve.get(
            BIND_INDEX_KEY, None
        )
        # Represent if the ClassMethodNode is a class method output. If True,
        # the node is a placeholder for a return value from the ClassMethodNode
        # that returns multiple values. If False, the node is a class method call.
        self._is_class_method_output: bool = other_args_to_resolve.get(
            IS_CLASS_METHOD_OUTPUT_KEY, False
        )
        # Represents the return value from the upstream ClassMethodNode that
        # returns multiple values. If the node is a class method call, this is None.
        self._class_method_output: Optional[_ClassMethodOutput] = None
        if self._is_class_method_output:
            # Set the upstream ClassMethodNode and the output index of the return
            # value from `method_args`.
            self._class_method_output = _ClassMethodOutput(
                method_args[0], method_args[1]
            )

        # The actor creation task dependency is encoded as the first argument,
        # and the ordering dependency as the second, which ensures they are
        # executed prior to this node.
        super().__init__(
            method_args,
            method_kwargs,
            method_options,
            other_args_to_resolve=other_args_to_resolve,
        )

    def _copy_impl(
        self,
        new_args: List[Any],
        new_kwargs: Dict[str, Any],
        new_options: Dict[str, Any],
        new_other_args_to_resolve: Dict[str, Any],
    ):
        return ClassMethodNode(
            self._method_name,
            new_args,
            new_kwargs,
            new_options,
            other_args_to_resolve=new_other_args_to_resolve,
        )

    def _execute_impl(self, *args, **kwargs):
        """Executor of ClassMethodNode by ray.remote()

        Args and kwargs are to match base class signature, but not in the
        implementation. All args and kwargs should be resolved and replaced
        with value in bound_args and bound_kwargs via bottom-up recursion when
        current node is executed.
        """
        if self.is_class_method_call:
            method_body = getattr(self._parent_class_node, self._method_name)
            # Execute with bound args.
            return method_body.options(**self._bound_options).remote(
                *self._bound_args,
                **self._bound_kwargs,
            )
        else:
            assert self._class_method_output is not None
            return self._bound_args[0][self._class_method_output.output_idx]

    def __str__(self) -> str:
        return get_dag_node_str(self, f"{self._method_name}()")

    def __repr__(self) -> str:
        return self.__str__()

    def get_method_name(self) -> str:
        return self._method_name

    def _get_bind_index(self) -> int:
        return self._bind_index

    def _get_remote_method(self, method_name):
        method_body = getattr(self._parent_class_node, method_name)
        return method_body

    def _get_actor_handle(self) -> Optional["ray.actor.ActorHandle"]:
        if not isinstance(self._parent_class_node, ray.actor.ActorHandle):
            return None
        return self._parent_class_node

    @property
    def num_returns(self) -> int:
        """
        Return the number of return values from the class method call. If the
        node is a class method output, return the number of return values from
        the upstream class method call.
        """

        if self.is_class_method_call:
            num_returns = self._bound_options.get("num_returns", None)
            if num_returns is None:
                method = self._get_remote_method(self._method_name)
                num_returns = method.__getstate__()["num_returns"]
            return num_returns
        else:
            assert self._class_method_output is not None
            return self._class_method_output.class_method_call.num_returns

    @property
    def is_class_method_call(self) -> bool:
        """
        Return True if the node is a class method call, False if the node is a
        class method output.
        """
        return not self._is_class_method_output

    @property
    def is_class_method_output(self) -> bool:
        """
        Return True if the node is a class method output, False if the node is a
        class method call.
        """
        return self._is_class_method_output

    @property
    def class_method_call(self) -> Optional["ClassMethodNode"]:
        """
        Return the upstream class method call that returns multiple values. If
        the node is a class method output, return None.
        """

        if self._class_method_output is None:
            return None
        return self._class_method_output.class_method_call

    @property
    def output_idx(self) -> Optional[int]:
        """
        Return the output index of the return value from the upstream class
        method call that returns multiple values. If the node is a class method
        call, return None.
        """

        if self._class_method_output is None:
            return None
        return self._class_method_output.output_idx