diff --git a/rllib/connectors/action/clip.py b/rllib/connectors/action/clip.py index 7a701833a78..da7c8b97bf9 100644 --- a/rllib/connectors/action/clip.py +++ b/rllib/connectors/action/clip.py @@ -7,10 +7,10 @@ from ray.rllib.connectors.registry import register_connector from ray.rllib.utils.spaces.space_utils import clip_action, get_base_struct_from_space from ray.rllib.utils.typing import ActionConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack class ClipActionsConnector(ActionConnector): def __init__(self, ctx: ConnectorContext): super().__init__(ctx) diff --git a/rllib/connectors/action/immutable.py b/rllib/connectors/action/immutable.py index 0e620ca7f63..3f5c8bbd197 100644 --- a/rllib/connectors/action/immutable.py +++ b/rllib/connectors/action/immutable.py @@ -9,10 +9,10 @@ from ray.rllib.connectors.registry import register_connector from ray.rllib.utils.numpy import make_action_immutable from ray.rllib.utils.typing import ActionConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack class ImmutableActionsConnector(ActionConnector): def transform(self, ac_data: ActionConnectorDataType) -> ActionConnectorDataType: assert isinstance( diff --git a/rllib/connectors/action/lambdas.py b/rllib/connectors/action/lambdas.py index 5f0c30e1676..3bf862dd834 100644 --- a/rllib/connectors/action/lambdas.py +++ b/rllib/connectors/action/lambdas.py @@ -12,10 +12,10 @@ StateBatches, TensorStructType, ) -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack def register_lambda_action_connector( name: str, fn: Callable[[TensorStructType, StateBatches, Dict], PolicyOutputType] ) -> Type[ActionConnector]: @@ -64,7 +64,7 @@ def from_state(ctx: ConnectorContext, params: Any): # Convert actions and states into numpy arrays if necessary. -ConvertToNumpyConnector = PublicAPI(stability="alpha")( +ConvertToNumpyConnector = OldAPIStack( register_lambda_action_connector( "ConvertToNumpyConnector", lambda actions, states, fetches: ( diff --git a/rllib/connectors/action/normalize.py b/rllib/connectors/action/normalize.py index 0ec644d471f..67c3731469a 100644 --- a/rllib/connectors/action/normalize.py +++ b/rllib/connectors/action/normalize.py @@ -10,10 +10,10 @@ unsquash_action, ) from ray.rllib.utils.typing import ActionConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack class NormalizeActionsConnector(ActionConnector): def __init__(self, ctx: ConnectorContext): super().__init__(ctx) diff --git a/rllib/connectors/action/pipeline.py b/rllib/connectors/action/pipeline.py index 740b2b5f849..b96296e7584 100644 --- a/rllib/connectors/action/pipeline.py +++ b/rllib/connectors/action/pipeline.py @@ -9,15 +9,15 @@ ConnectorPipeline, ) from ray.rllib.connectors.registry import get_connector, register_connector +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import ActionConnectorDataType -from ray.util.annotations import PublicAPI from ray.util.timer import _Timer logger = logging.getLogger(__name__) -@PublicAPI(stability="alpha") +@OldAPIStack class ActionConnectorPipeline(ConnectorPipeline, ActionConnector): def __init__(self, ctx: ConnectorContext, connectors: List[Connector]): super().__init__(ctx, connectors) diff --git a/rllib/connectors/agent/clip_reward.py b/rllib/connectors/agent/clip_reward.py index 7889cbd33a9..9d55e4aea24 100644 --- a/rllib/connectors/agent/clip_reward.py +++ b/rllib/connectors/agent/clip_reward.py @@ -9,10 +9,10 @@ from ray.rllib.connectors.registry import register_connector from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils.typing import AgentConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack class ClipRewardAgentConnector(AgentConnector): def __init__(self, ctx: ConnectorContext, sign=False, limit=None): super().__init__(ctx) diff --git a/rllib/connectors/agent/env_sampling.py b/rllib/connectors/agent/env_sampling.py index 56e2906007a..f0ba6f0a438 100644 --- a/rllib/connectors/agent/env_sampling.py +++ b/rllib/connectors/agent/env_sampling.py @@ -6,10 +6,10 @@ ) from ray.rllib.connectors.registry import register_connector from ray.rllib.utils.typing import AgentConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack class EnvSamplingAgentConnector(AgentConnector): def __init__(self, ctx: ConnectorContext, sign=False, limit=None): super().__init__(ctx) diff --git a/rllib/connectors/agent/lambdas.py b/rllib/connectors/agent/lambdas.py index 781a6bf671a..05a714a0df9 100644 --- a/rllib/connectors/agent/lambdas.py +++ b/rllib/connectors/agent/lambdas.py @@ -13,10 +13,10 @@ AgentConnectorDataType, AgentConnectorsOutput, ) -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -@PublicAPI(stability="alpha") +@OldAPIStack def register_lambda_agent_connector( name: str, fn: Callable[[Any], Any] ) -> Type[AgentConnector]: @@ -54,7 +54,7 @@ def from_state(ctx: ConnectorContext, params: Any): return LambdaAgentConnector -@PublicAPI(stability="alpha") +@OldAPIStack def flatten_data(data: AgentConnectorsOutput): assert isinstance( data, AgentConnectorsOutput @@ -81,6 +81,6 @@ def flatten_data(data: AgentConnectorsOutput): # Agent connector to build and return a flattened observation SampleBatch # in addition to the original input dict. -FlattenDataAgentConnector = PublicAPI(stability="alpha")( +FlattenDataAgentConnector = OldAPIStack( register_lambda_agent_connector("FlattenDataAgentConnector", flatten_data) ) diff --git a/rllib/connectors/agent/mean_std_filter.py b/rllib/connectors/agent/mean_std_filter.py index be8d929813b..64de9916426 100644 --- a/rllib/connectors/agent/mean_std_filter.py +++ b/rllib/connectors/agent/mean_std_filter.py @@ -11,15 +11,15 @@ ) from ray.rllib.connectors.registry import register_connector from ray.rllib.policy.sample_batch import SampleBatch +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.filter import Filter from ray.rllib.utils.filter import MeanStdFilter, ConcurrentMeanStdFilter from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.typing import AgentConnectorDataType -from ray.util.annotations import PublicAPI from ray.rllib.utils.filter import RunningStat -@PublicAPI(stability="alpha") +@OldAPIStack class MeanStdObservationFilterAgentConnector(SyncedFilterAgentConnector): """A connector used to mean-std-filter observations. @@ -149,7 +149,7 @@ def sync(self, other: "AgentConnector") -> None: return self.filter.sync(other.filter) -@PublicAPI(stability="alpha") +@OldAPIStack class ConcurrentMeanStdObservationFilterAgentConnector( MeanStdObservationFilterAgentConnector ): diff --git a/rllib/connectors/agent/obs_preproc.py b/rllib/connectors/agent/obs_preproc.py index 744b1bee99f..41b23745c05 100644 --- a/rllib/connectors/agent/obs_preproc.py +++ b/rllib/connectors/agent/obs_preproc.py @@ -8,13 +8,10 @@ from ray.rllib.models.preprocessors import get_preprocessor, NoPreprocessor from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils.typing import AgentConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack -# Bridging between current obs preprocessors and connector. -# We should not introduce any new preprocessors. -# TODO(jungong) : migrate and implement preprocessor library in Connector framework. -@PublicAPI(stability="alpha") +@OldAPIStack class ObsPreprocessorConnector(AgentConnector): """A connector that wraps around existing RLlib observation preprocessors. diff --git a/rllib/connectors/agent/pipeline.py b/rllib/connectors/agent/pipeline.py index 480161bc5a4..0113c1c4588 100644 --- a/rllib/connectors/agent/pipeline.py +++ b/rllib/connectors/agent/pipeline.py @@ -10,14 +10,14 @@ ) from ray.rllib.connectors.registry import get_connector, register_connector from ray.rllib.utils.typing import ActionConnectorDataType, AgentConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.util.timer import _Timer logger = logging.getLogger(__name__) -@PublicAPI(stability="alpha") +@OldAPIStack class AgentConnectorPipeline(ConnectorPipeline, AgentConnector): def __init__(self, ctx: ConnectorContext, connectors: List[Connector]): super().__init__(ctx, connectors) diff --git a/rllib/connectors/agent/state_buffer.py b/rllib/connectors/agent/state_buffer.py index a05b54388da..8d4c83bd4fe 100644 --- a/rllib/connectors/agent/state_buffer.py +++ b/rllib/connectors/agent/state_buffer.py @@ -18,13 +18,13 @@ from ray.rllib.core.models.base import STATE_OUT from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.typing import ActionConnectorDataType, AgentConnectorDataType -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack logger = logging.getLogger(__name__) -@PublicAPI(stability="alpha") +@OldAPIStack class StateBufferConnector(AgentConnector): def __init__(self, ctx: ConnectorContext, states: Any = None): super().__init__(ctx) diff --git a/rllib/connectors/agent/synced_filter.py b/rllib/connectors/agent/synced_filter.py index 08d147fbcb8..51c625d9ace 100644 --- a/rllib/connectors/agent/synced_filter.py +++ b/rllib/connectors/agent/synced_filter.py @@ -2,11 +2,11 @@ AgentConnector, ConnectorContext, ) -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.filter import Filter -@PublicAPI(stability="alpha") +@OldAPIStack class SyncedFilterAgentConnector(AgentConnector): """An agent connector that filters with synchronized parameters.""" diff --git a/rllib/connectors/agent/view_requirement.py b/rllib/connectors/agent/view_requirement.py index ffacecd27b0..1a079792bd3 100644 --- a/rllib/connectors/agent/view_requirement.py +++ b/rllib/connectors/agent/view_requirement.py @@ -11,11 +11,11 @@ AgentConnectorDataType, AgentConnectorsOutput, ) -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.evaluation.collectors.agent_collector import AgentCollector -@PublicAPI(stability="alpha") +@OldAPIStack class ViewRequirementAgentConnector(AgentConnector): """This connector does 2 things: 1. It filters data columns based on view_requirements for training and inference. diff --git a/rllib/connectors/connector.py b/rllib/connectors/connector.py index 5aac95accf6..80c5003085b 100644 --- a/rllib/connectors/connector.py +++ b/rllib/connectors/connector.py @@ -14,7 +14,7 @@ AlgorithmConfigDict, TensorType, ) -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack if TYPE_CHECKING: from ray.rllib.policy.policy import Policy @@ -22,7 +22,7 @@ logger = logging.getLogger(__name__) -@PublicAPI(stability="alpha") +@OldAPIStack class ConnectorContext: """Data bits that may be needed for running connectors. @@ -79,7 +79,7 @@ def from_policy(policy: "Policy") -> "ConnectorContext": ) -@PublicAPI(stability="alpha") +@OldAPIStack class Connector(abc.ABC): """Connector base class. @@ -137,7 +137,7 @@ def from_state(self, ctx: ConnectorContext, params: Any) -> "Connector": return NotImplementedError -@PublicAPI(stability="alpha") +@OldAPIStack class AgentConnector(Connector): """Connector connecting user environments to RLlib policies. @@ -277,7 +277,7 @@ def transform(self, ac_data: AgentConnectorDataType) -> AgentConnectorDataType: raise NotImplementedError -@PublicAPI(stability="alpha") +@OldAPIStack class ActionConnector(Connector): """Action connector connects policy outputs including actions, to user environments. @@ -332,7 +332,7 @@ def transform(self, ac_data: ActionConnectorDataType) -> ActionConnectorDataType raise NotImplementedError -@PublicAPI(stability="alpha") +@OldAPIStack class ConnectorPipeline(abc.ABC): """Utility class for quick manipulation of a connector pipeline.""" diff --git a/rllib/connectors/registry.py b/rllib/connectors/registry.py index bf4151318b0..8efe64515ee 100644 --- a/rllib/connectors/registry.py +++ b/rllib/connectors/registry.py @@ -1,14 +1,14 @@ """Registry of connector names for global access.""" from typing import Any -from ray.util.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.connectors.connector import Connector, ConnectorContext ALL_CONNECTORS = dict() -@PublicAPI(stability="alpha") +@OldAPIStack def register_connector(name: str, cls: Connector): """Register a connector for use with RLlib. @@ -28,7 +28,7 @@ def register_connector(name: str, cls: Connector): ALL_CONNECTORS[name] = cls -@PublicAPI(stability="alpha") +@OldAPIStack def get_connector(name: str, ctx: ConnectorContext, params: Any = None) -> Connector: # TODO(jungong) : switch the order of parameters man!! """Get a connector by its name and serialized config. diff --git a/rllib/connectors/util.py b/rllib/connectors/util.py index faa39d7982f..e0ffbcea29d 100644 --- a/rllib/connectors/util.py +++ b/rllib/connectors/util.py @@ -17,7 +17,7 @@ MeanStdObservationFilterAgentConnector, ConcurrentMeanStdObservationFilterAgentConnector, ) -from ray.util.annotations import PublicAPI, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.connectors.agent.synced_filter import SyncedFilterAgentConnector if TYPE_CHECKING: @@ -44,7 +44,7 @@ def __clip_rewards(config: "AlgorithmConfig"): return config.clip_rewards or config.is_atari -@PublicAPI(stability="alpha") +@OldAPIStack def get_agent_connectors_from_config( ctx: ConnectorContext, config: "AlgorithmConfig", @@ -78,7 +78,7 @@ def get_agent_connectors_from_config( return AgentConnectorPipeline(ctx, connectors) -@PublicAPI(stability="alpha") +@OldAPIStack def get_action_connectors_from_config( ctx: ConnectorContext, config: "AlgorithmConfig", @@ -98,7 +98,7 @@ def get_action_connectors_from_config( return ActionConnectorPipeline(ctx, connectors) -@PublicAPI(stability="alpha") +@OldAPIStack def create_connectors_for_policy(policy: "Policy", config: "AlgorithmConfig"): """Util to create agent and action connectors for a Policy. @@ -120,7 +120,7 @@ def create_connectors_for_policy(policy: "Policy", config: "AlgorithmConfig"): logger.info(policy.action_connectors.__str__(indentation=4)) -@PublicAPI(stability="alpha") +@OldAPIStack def restore_connectors_for_policy( policy: "Policy", connector_config: Tuple[str, Tuple[Any]] ) -> Connector: @@ -136,7 +136,7 @@ def restore_connectors_for_policy( # We need this filter selection mechanism temporarily to remain compatible to old API -@DeveloperAPI +@OldAPIStack def get_synced_filter_connector(ctx: ConnectorContext): filter_specifier = ctx.config.get("observation_filter") if filter_specifier == "MeanStdFilter": @@ -149,7 +149,7 @@ def get_synced_filter_connector(ctx: ConnectorContext): raise Exception("Unknown observation_filter: " + str(filter_specifier)) -@DeveloperAPI +@OldAPIStack def maybe_get_filters_for_syncing(rollout_worker, policy_id): # As long as the historic filter synchronization mechanism is in # place, we need to put filters into self.filters so that they get diff --git a/rllib/env/apis/task_settable_env.py b/rllib/env/apis/task_settable_env.py index 01cb1a3e9b2..c43999857be 100644 --- a/rllib/env/apis/task_settable_env.py +++ b/rllib/env/apis/task_settable_env.py @@ -1,9 +1,12 @@ import gymnasium as gym from typing import List, Any +from ray.rllib.utils.annotations import OldAPIStack + TaskType = Any # Can be different types depending on env, e.g., int or dict +@OldAPIStack class TaskSettableEnv(gym.Env): """ Extension of gym.Env to define a task-settable Env. diff --git a/rllib/env/base_env.py b/rllib/env/base_env.py index 3ce908728e8..f6bd9a3f983 100644 --- a/rllib/env/base_env.py +++ b/rllib/env/base_env.py @@ -3,7 +3,7 @@ import gymnasium as gym import ray -from ray.rllib.utils.annotations import Deprecated, DeveloperAPI, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import AgentID, EnvID, EnvType, MultiEnvDict if TYPE_CHECKING: @@ -14,7 +14,7 @@ logger = logging.getLogger(__name__) -@PublicAPI +@OldAPIStack class BaseEnv: """The lowest-level env interface used by RLlib for sampling. @@ -130,7 +130,6 @@ def to_base_env( """ return self - @PublicAPI def poll( self, ) -> Tuple[ @@ -163,7 +162,6 @@ def poll( """ raise NotImplementedError - @PublicAPI def send_actions(self, action_dict: MultiEnvDict) -> None: """Called to send actions back to running agents in this env. @@ -175,7 +173,6 @@ def send_actions(self, action_dict: MultiEnvDict) -> None: """ raise NotImplementedError - @PublicAPI def try_reset( self, env_id: Optional[EnvID] = None, @@ -210,7 +207,6 @@ def try_reset( """ return None, None - @DeveloperAPI def try_restart(self, env_id: Optional[EnvID] = None) -> None: """Attempt to restart the sub-env with the given id or all sub-envs. @@ -222,7 +218,6 @@ def try_restart(self, env_id: Optional[EnvID] = None) -> None: """ return None - @PublicAPI def get_sub_environments(self, as_dict: bool = False) -> Union[List[EnvType], dict]: """Return a reference to the underlying sub environments, if any. @@ -236,7 +231,6 @@ def get_sub_environments(self, as_dict: bool = False) -> Union[List[EnvType], di return {} return [] - @PublicAPI def get_agent_ids(self) -> Set[AgentID]: """Return the agent ids for the sub_environment. @@ -245,7 +239,6 @@ def get_agent_ids(self) -> Set[AgentID]: """ return {} - @PublicAPI def try_render(self, env_id: Optional[EnvID] = None) -> None: """Tries to render the sub-environment with the given id or all. @@ -257,7 +250,6 @@ def try_render(self, env_id: Optional[EnvID] = None) -> None: # By default, do nothing. pass - @PublicAPI def stop(self) -> None: """Releases all resources used.""" @@ -266,12 +258,7 @@ def stop(self) -> None: if hasattr(env, "close"): env.close() - @Deprecated(new="get_sub_environments", error=True) - def get_unwrapped(self) -> List[EnvType]: - return self.get_sub_environments() - @property - @PublicAPI def observation_space(self) -> gym.Space: """Returns the observation space for each agent. @@ -284,7 +271,6 @@ def observation_space(self) -> gym.Space: raise NotImplementedError @property - @PublicAPI def action_space(self) -> gym.Space: """Returns the action space for each agent. @@ -296,7 +282,6 @@ def action_space(self) -> gym.Space: """ raise NotImplementedError - @PublicAPI def action_space_sample(self, agent_id: list = None) -> MultiEnvDict: """Returns a random action for each environment, and potentially each agent in that environment. @@ -312,7 +297,6 @@ def action_space_sample(self, agent_id: list = None) -> MultiEnvDict: del agent_id return {} - @PublicAPI def observation_space_sample(self, agent_id: list = None) -> MultiEnvDict: """Returns a random observation for each environment, and potentially each agent in that environment. @@ -328,7 +312,6 @@ def observation_space_sample(self, agent_id: list = None) -> MultiEnvDict: del agent_id return {} - @PublicAPI def last( self, ) -> Tuple[MultiEnvDict, MultiEnvDict, MultiEnvDict, MultiEnvDict, MultiEnvDict]: @@ -343,7 +326,6 @@ def last( logger.warning("last has not been implemented for this environment.") return {}, {}, {}, {}, {} - @PublicAPI def observation_space_contains(self, x: MultiEnvDict) -> bool: """Checks if the given observation is valid for each environment. @@ -356,7 +338,6 @@ def observation_space_contains(self, x: MultiEnvDict) -> bool: """ return self._space_contains(self.observation_space, x) - @PublicAPI def action_space_contains(self, x: MultiEnvDict) -> bool: """Checks if the given actions is valid for each environment. @@ -399,7 +380,7 @@ def _space_contains(self, space: gym.Space, x: MultiEnvDict) -> bool: _DUMMY_AGENT_ID = "agent0" -@PublicAPI +@OldAPIStack def with_dummy_agent_id( env_id_to_values: Dict[EnvID, Any], dummy_id: "AgentID" = _DUMMY_AGENT_ID ) -> MultiEnvDict: @@ -412,7 +393,7 @@ def with_dummy_agent_id( return ret -@DeveloperAPI +@OldAPIStack def convert_to_base_env( env: EnvType, make_env: Callable[[int], EnvType] = None, @@ -524,10 +505,3 @@ def convert_to_base_env( assert isinstance(env, BaseEnv), env return env - - -@Deprecated(new="with_dummy_agent_id()", error=True) -def _with_dummy_agent_id( - env_id_to_values: Dict[EnvID, Any], dummy_id: "AgentID" = _DUMMY_AGENT_ID -) -> MultiEnvDict: - return {k: {dummy_id: v} for (k, v) in env_id_to_values.items()} diff --git a/rllib/env/env_context.py b/rllib/env/env_context.py index d39b6836171..bc1dcac587e 100644 --- a/rllib/env/env_context.py +++ b/rllib/env/env_context.py @@ -1,11 +1,11 @@ import copy from typing import Optional -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import EnvConfigDict -@PublicAPI +@OldAPIStack class EnvContext(dict): """Wraps env configurations to include extra rllib metadata. diff --git a/rllib/env/env_runner.py b/rllib/env/env_runner.py index a25f97535a4..7bab0621d3e 100644 --- a/rllib/env/env_runner.py +++ b/rllib/env/env_runner.py @@ -2,7 +2,7 @@ from typing import Any, Dict, TYPE_CHECKING from ray.rllib.utils.actor_manager import FaultAwareApply -from ray.rllib.utils.annotations import ExperimentalAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf if TYPE_CHECKING: @@ -11,7 +11,7 @@ tf1, _, _ = try_import_tf() -@ExperimentalAPI +@OldAPIStack class EnvRunner(FaultAwareApply, metaclass=abc.ABCMeta): """Base class for distributed RL-style data collection from an environment. diff --git a/rllib/env/external_env.py b/rllib/env/external_env.py index c0ae4555e60..41eb89d6c47 100644 --- a/rllib/env/external_env.py +++ b/rllib/env/external_env.py @@ -5,7 +5,7 @@ from typing import Callable, Tuple, Optional, TYPE_CHECKING from ray.rllib.env.base_env import BaseEnv -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import override, OldAPIStack from ray.rllib.utils.typing import ( EnvActionType, EnvInfoDict, @@ -19,7 +19,7 @@ from ray.rllib.models.preprocessors import Preprocessor -@PublicAPI +@OldAPIStack class ExternalEnv(threading.Thread): """An environment that interfaces with external agents. @@ -50,7 +50,6 @@ class ExternalEnv(threading.Thread): print(algo.train()) """ - @PublicAPI def __init__( self, action_space: gym.Space, @@ -81,7 +80,6 @@ def __init__( error=True, ) - @PublicAPI def run(self): """Override this to implement the run loop. @@ -96,7 +94,6 @@ def run(self): """ raise NotImplementedError - @PublicAPI def start_episode( self, episode_id: Optional[str] = None, training_enabled: bool = True ) -> str: @@ -127,7 +124,6 @@ def start_episode( return episode_id - @PublicAPI def get_action(self, episode_id: str, observation: EnvObsType) -> EnvActionType: """Record an observation and get the on-policy action. @@ -142,7 +138,6 @@ def get_action(self, episode_id: str, observation: EnvObsType) -> EnvActionType: episode = self._get(episode_id) return episode.wait_for_action(observation) - @PublicAPI def log_action( self, episode_id: str, observation: EnvObsType, action: EnvActionType ) -> None: @@ -157,7 +152,6 @@ def log_action( episode = self._get(episode_id) episode.log_action(observation, action) - @PublicAPI def log_returns( self, episode_id: str, reward: float, info: Optional[EnvInfoDict] = None ) -> None: @@ -179,7 +173,6 @@ def log_returns( if info: episode.cur_info = info or {} - @PublicAPI def end_episode(self, episode_id: str, observation: EnvObsType) -> None: """Records the end of an episode. @@ -248,6 +241,7 @@ def to_base_env( return env +@OldAPIStack class _ExternalEnvEpisode: """Tracked state for each active episode.""" @@ -354,7 +348,7 @@ def _send(self): self.results_avail_condition.notify() -@PublicAPI +@OldAPIStack class ExternalEnvWrapper(BaseEnv): """Internal adapter of ExternalEnv to BaseEnv.""" @@ -478,12 +472,10 @@ def fix(d, zero_val): @property @override(BaseEnv) - @PublicAPI def observation_space(self) -> gym.spaces.Dict: return self._observation_space @property @override(BaseEnv) - @PublicAPI def action_space(self) -> gym.Space: return self._action_space diff --git a/rllib/env/external_multi_agent_env.py b/rllib/env/external_multi_agent_env.py index c0fe8da08f7..1350d5c7c35 100644 --- a/rllib/env/external_multi_agent_env.py +++ b/rllib/env/external_multi_agent_env.py @@ -2,16 +2,15 @@ import gymnasium as gym from typing import Optional -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import override, OldAPIStack from ray.rllib.env.external_env import ExternalEnv, _ExternalEnvEpisode from ray.rllib.utils.typing import MultiAgentDict -@PublicAPI +@OldAPIStack class ExternalMultiAgentEnv(ExternalEnv): """This is the multi-agent version of ExternalEnv.""" - @PublicAPI def __init__( self, action_space: gym.Space, @@ -37,7 +36,6 @@ def __init__( ) ) - @PublicAPI def run(self): """Override this to implement the multi-agent run loop. @@ -54,7 +52,6 @@ def run(self): """ raise NotImplementedError - @PublicAPI @override(ExternalEnv) def start_episode( self, episode_id: Optional[str] = None, training_enabled: bool = True @@ -74,7 +71,6 @@ def start_episode( return episode_id - @PublicAPI @override(ExternalEnv) def get_action( self, episode_id: str, observation_dict: MultiAgentDict @@ -95,7 +91,6 @@ def get_action( episode = self._get(episode_id) return episode.wait_for_action(observation_dict) - @PublicAPI @override(ExternalEnv) def log_action( self, @@ -114,7 +109,6 @@ def log_action( episode = self._get(episode_id) episode.log_action(observation_dict, action_dict) - @PublicAPI @override(ExternalEnv) def log_returns( self, @@ -153,7 +147,6 @@ def log_returns( if info_dict: episode.cur_info_dict = info_dict or {} - @PublicAPI @override(ExternalEnv) def end_episode(self, episode_id: str, observation_dict: MultiAgentDict) -> None: """Record the end of an episode. diff --git a/rllib/env/multi_agent_env.py b/rllib/env/multi_agent_env.py index f768d8c3517..a10300d30d4 100644 --- a/rllib/env/multi_agent_env.py +++ b/rllib/env/multi_agent_env.py @@ -5,10 +5,9 @@ from ray.rllib.env.base_env import BaseEnv from ray.rllib.env.env_context import EnvContext from ray.rllib.utils.annotations import ( - ExperimentalAPI, + OldAPIStack, override, PublicAPI, - DeveloperAPI, ) from ray.rllib.utils.typing import ( AgentID, @@ -58,7 +57,6 @@ def __init__(self): if not hasattr(self, "_obs_space_in_preferred_format"): self._obs_space_in_preferred_format = None - @PublicAPI def reset( self, *, @@ -94,7 +92,6 @@ class MyMultiAgentEnv(MultiAgentEnv): # Call super's `reset()` method to (maybe) set the given `seed`. super().reset(seed=seed, options=options) - @PublicAPI def step( self, action_dict: MultiAgentDict ) -> Tuple[ @@ -147,7 +144,6 @@ def step( """ raise NotImplementedError - @ExperimentalAPI def observation_space_contains(self, x: MultiAgentDict) -> bool: """Checks if the observation space contains the given key. @@ -190,7 +186,6 @@ def observation_space_contains(self, x: MultiAgentDict) -> bool: ) return True - @ExperimentalAPI def action_space_contains(self, x: MultiAgentDict) -> bool: """Checks if the action space contains the given action. @@ -220,7 +215,6 @@ def action_space_contains(self, x: MultiAgentDict) -> bool: ) return True - @ExperimentalAPI def action_space_sample(self, agent_ids: list = None) -> MultiAgentDict: """Returns a random action for each environment, and potentially each agent in that environment. @@ -257,7 +251,6 @@ def action_space_sample(self, agent_ids: list = None) -> MultiAgentDict: ) return {} - @ExperimentalAPI def observation_space_sample(self, agent_ids: list = None) -> MultiEnvDict: """Returns a random observation from the observation space for each agent if agent_ids is None, otherwise returns a random observation for @@ -295,7 +288,6 @@ def observation_space_sample(self, agent_ids: list = None) -> MultiEnvDict: ) return {} - @PublicAPI def get_agent_ids(self) -> Set[AgentID]: """Returns a set of agent ids in the environment. @@ -306,7 +298,6 @@ def get_agent_ids(self) -> Set[AgentID]: self._agent_ids = set(self._agent_ids) return self._agent_ids - @PublicAPI def render(self) -> None: """Tries to render the environment.""" @@ -368,7 +359,7 @@ class MyMultiAgentEnv(MultiAgentEnv): # __grouping_doc_end__ # fmt: on - @PublicAPI + @OldAPIStack def to_base_env( self, make_env: Optional[Callable[[int], EnvType]] = None, @@ -427,7 +418,6 @@ def to_base_env( return env - @DeveloperAPI def _check_if_obs_space_maps_agent_id_to_sub_space(self) -> bool: """Checks if obs space maps from agent ids to spaces of individual agents.""" return ( @@ -436,7 +426,6 @@ def _check_if_obs_space_maps_agent_id_to_sub_space(self) -> bool: and set(self.observation_space.spaces.keys()) == self.get_agent_ids() ) - @DeveloperAPI def _check_if_action_space_maps_agent_id_to_sub_space(self) -> bool: """Checks if action space maps from agent ids to spaces of individual agents.""" return ( @@ -570,7 +559,7 @@ def render(self): return MultiEnv -@PublicAPI +@OldAPIStack class MultiAgentEnvWrapper(BaseEnv): """Internal adapter of MultiAgentEnv to BaseEnv. @@ -753,13 +742,11 @@ def try_render(self, env_id: Optional[EnvID] = None) -> None: @property @override(BaseEnv) - @PublicAPI def observation_space(self) -> gym.spaces.Dict: return self.envs[0].observation_space @property @override(BaseEnv) - @PublicAPI def action_space(self) -> gym.Space: return self.envs[0].action_space @@ -803,6 +790,7 @@ def _init_env_state(self, idx: Optional[int] = None) -> None: ) +@OldAPIStack class _MultiAgentEnvState: def __init__(self, env: MultiAgentEnv, return_error_as_obs: bool = False): assert isinstance(env, MultiAgentEnv) diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index a2a4c1bc2eb..bf8e39dac6b 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -13,12 +13,13 @@ from ray.rllib.env.utils import _gym_env_creator from ray.rllib.evaluation.metrics import RolloutMetrics from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import ExperimentalAPI, override +from ray.rllib.utils.annotations import override from ray.rllib.utils.typing import ModelWeights +from ray.util.annotations import PublicAPI from ray.tune.registry import ENV_CREATOR, _global_registry -@ExperimentalAPI +@PublicAPI(stability="alpha") class MultiAgentEnvRunner(EnvRunner): """The genetic environment runner for the multi-agent case.""" diff --git a/rllib/env/multi_agent_episode.py b/rllib/env/multi_agent_episode.py index 8484542c33f..3b2f979abdd 100644 --- a/rllib/env/multi_agent_episode.py +++ b/rllib/env/multi_agent_episode.py @@ -23,10 +23,12 @@ from ray.rllib.utils.error import MultiAgentEnvError from ray.rllib.utils.spaces.space_utils import batch from ray.rllib.utils.typing import AgentID, ModuleID, MultiAgentDict +from ray.util.annotations import PublicAPI # TODO (simon): Include cases in which the number of agents in an # episode are shrinking or growing during the episode itself. +@PublicAPI(stability="alpha") class MultiAgentEpisode: """Stores multi-agent episode data. diff --git a/rllib/env/remote_base_env.py b/rllib/env/remote_base_env.py index 54fb4312f3a..b9e388d50bc 100644 --- a/rllib/env/remote_base_env.py +++ b/rllib/env/remote_base_env.py @@ -5,7 +5,7 @@ import ray from ray.util import log_once from ray.rllib.env.base_env import BaseEnv, _DUMMY_AGENT_ID, ASYNC_RESET_RETURN -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import override, OldAPIStack from ray.rllib.utils.typing import AgentID, EnvID, EnvType, MultiEnvDict if TYPE_CHECKING: @@ -14,7 +14,7 @@ logger = logging.getLogger(__name__) -@PublicAPI +@OldAPIStack class RemoteBaseEnv(BaseEnv): """BaseEnv that executes its sub environments as @ray.remote actors. @@ -269,7 +269,6 @@ def poll( return obs, rewards, terminateds, truncateds, infos, {} @override(BaseEnv) - @PublicAPI def send_actions(self, action_dict: MultiEnvDict) -> None: for env_id, actions in action_dict.items(): actor = self.actors[env_id] @@ -285,7 +284,6 @@ def send_actions(self, action_dict: MultiEnvDict) -> None: self.pending[obj_ref] = actor @override(BaseEnv) - @PublicAPI def try_reset( self, env_id: Optional[EnvID] = None, @@ -322,14 +320,12 @@ def try_restart(self, env_id: Optional[EnvID] = None) -> None: self.actors[env_id] = self._make_sub_env(env_id) @override(BaseEnv) - @PublicAPI def stop(self) -> None: if self.actors is not None: for actor in self.actors: actor.__ray_terminate__.remote() @override(BaseEnv) - @PublicAPI def get_sub_environments(self, as_dict: bool = False) -> List[EnvType]: if as_dict: return {env_id: actor for env_id, actor in enumerate(self.actors)} @@ -337,13 +333,11 @@ def get_sub_environments(self, as_dict: bool = False) -> List[EnvType]: @property @override(BaseEnv) - @PublicAPI def observation_space(self) -> gym.spaces.Dict: return self._observation_space @property @override(BaseEnv) - @PublicAPI def action_space(self) -> gym.Space: return self._action_space diff --git a/rllib/env/remote_vector_env.py b/rllib/env/remote_vector_env.py deleted file mode 100644 index c6164ab57bd..00000000000 --- a/rllib/env/remote_vector_env.py +++ /dev/null @@ -1,10 +0,0 @@ -from ray.rllib.env.remote_base_env import RemoteBaseEnv -from ray.rllib.utils.deprecation import deprecation_warning - -deprecation_warning( - old="rllib.env.remote_vector_env.RemoteVectorEnv", - new="ray.rllib.env.remote_base_env.RemoteBaseEnv", - error=True, -) - -RemoteVectorEnv = RemoteBaseEnv diff --git a/rllib/env/single_agent_env_runner.py b/rllib/env/single_agent_env_runner.py index ac0f41812cc..f1fbc6a4fda 100644 --- a/rllib/env/single_agent_env_runner.py +++ b/rllib/env/single_agent_env_runner.py @@ -13,17 +13,18 @@ from ray.rllib.env.utils import _gym_env_creator from ray.rllib.evaluation.metrics import RolloutMetrics from ray.rllib.policy.sample_batch import DEFAULT_POLICY_ID, SampleBatch -from ray.rllib.utils.annotations import ExperimentalAPI, override +from ray.rllib.utils.annotations import override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.spaces.space_utils import unbatch from ray.rllib.utils.torch_utils import convert_to_torch_tensor from ray.rllib.utils.typing import TensorType, ModelWeights from ray.tune.registry import ENV_CREATOR, _global_registry +from ray.util.annotations import PublicAPI _, tf, _ = try_import_tf() -@ExperimentalAPI +@PublicAPI(stability="alpha") class SingleAgentEnvRunner(EnvRunner): """The generic environment runner for the single agent case.""" diff --git a/rllib/env/single_agent_episode.py b/rllib/env/single_agent_episode.py index 5d638ea963f..ee2533f0c9f 100644 --- a/rllib/env/single_agent_episode.py +++ b/rllib/env/single_agent_episode.py @@ -10,8 +10,10 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.env.utils.infinite_lookback_buffer import InfiniteLookbackBuffer from ray.rllib.utils.typing import AgentID, ModuleID +from ray.util.annotations import PublicAPI +@PublicAPI(stability="alpha") class SingleAgentEpisode: """A class representing RL environment episodes for individual agents. diff --git a/rllib/env/vector_env.py b/rllib/env/vector_env.py index e9bd879ab82..1c4f4348c4c 100644 --- a/rllib/env/vector_env.py +++ b/rllib/env/vector_env.py @@ -4,7 +4,7 @@ from typing import Callable, List, Optional, Tuple, Union, Set from ray.rllib.env.base_env import BaseEnv, _DUMMY_AGENT_ID -from ray.rllib.utils.annotations import Deprecated, override, PublicAPI +from ray.rllib.utils.annotations import Deprecated, OldAPIStack, override from ray.rllib.utils.typing import ( EnvActionType, EnvID, @@ -19,7 +19,7 @@ logger = logging.getLogger(__name__) -@PublicAPI +@OldAPIStack class VectorEnv: """An environment that supports batch evaluation using clones of sub-envs.""" @@ -81,7 +81,6 @@ def vectorize_gym_envs( restart_failed_sub_environments=restart_failed_sub_environments, ) - @PublicAPI def vector_reset( self, *, seeds: Optional[List[int]] = None, options: Optional[List[dict]] = None ) -> Tuple[List[EnvObsType], List[EnvInfoDict]]: @@ -100,7 +99,6 @@ def vector_reset( """ raise NotImplementedError - @PublicAPI def reset_at( self, index: Optional[int] = None, @@ -126,7 +124,6 @@ def reset_at( """ raise NotImplementedError - @PublicAPI def restart_at(self, index: Optional[int] = None) -> None: """Restarts a single sub-environment. @@ -135,7 +132,6 @@ def restart_at(self, index: Optional[int] = None) -> None: """ raise NotImplementedError - @PublicAPI def vector_step( self, actions: List[EnvActionType] ) -> Tuple[ @@ -156,7 +152,6 @@ def vector_step( """ raise NotImplementedError - @PublicAPI def get_sub_environments(self) -> List[EnvType]: """Returns the underlying sub environments. @@ -178,7 +173,6 @@ def try_render_at(self, index: Optional[int] = None) -> Optional[np.ndarray]: """ pass - @PublicAPI def to_base_env( self, make_env: Optional[Callable[[int], EnvType]] = None, @@ -224,6 +218,7 @@ def get_unwrapped(self) -> List[EnvType]: pass +@OldAPIStack class _VectorizedGymEnv(VectorEnv): """Internal wrapper to translate any gym.Envs into a VectorEnv object.""" @@ -383,7 +378,7 @@ def try_render_at(self, index: Optional[int] = None): return self.envs[index].render() -@PublicAPI +@OldAPIStack class VectorEnvWrapper(BaseEnv): """Internal adapter of VectorEnv to BaseEnv. @@ -512,30 +507,25 @@ def try_render(self, env_id: Optional[EnvID] = None) -> None: @property @override(BaseEnv) - @PublicAPI def observation_space(self) -> gym.Space: return self._observation_space @property @override(BaseEnv) - @PublicAPI def action_space(self) -> gym.Space: return self._action_space @override(BaseEnv) - @PublicAPI def action_space_sample(self, agent_id: list = None) -> MultiEnvDict: del agent_id return {0: {_DUMMY_AGENT_ID: self._action_space.sample()}} @override(BaseEnv) - @PublicAPI def observation_space_sample(self, agent_id: list = None) -> MultiEnvDict: del agent_id return {0: {_DUMMY_AGENT_ID: self._observation_space.sample()}} @override(BaseEnv) - @PublicAPI def get_agent_ids(self) -> Set[AgentID]: return {_DUMMY_AGENT_ID} diff --git a/rllib/evaluation/collectors/agent_collector.py b/rllib/evaluation/collectors/agent_collector.py index 0ea3d7d5b48..0628cbcb971 100644 --- a/rllib/evaluation/collectors/agent_collector.py +++ b/rllib/evaluation/collectors/agent_collector.py @@ -9,6 +9,7 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.spaces.space_utils import ( flatten_to_single_ndarray, @@ -20,7 +21,6 @@ TensorType, ViewRequirementsDict, ) -from ray.util.annotations import PublicAPI logger = logging.getLogger(__name__) @@ -47,7 +47,7 @@ def _get_buffered_slice_with_paddings(d, inds): return element_at_t -@PublicAPI +@OldAPIStack class AgentCollector: """Collects samples for one agent in one trajectory (episode). diff --git a/rllib/evaluation/collectors/sample_collector.py b/rllib/evaluation/collectors/sample_collector.py index 892c2080058..867b7ec1ae2 100644 --- a/rllib/evaluation/collectors/sample_collector.py +++ b/rllib/evaluation/collectors/sample_collector.py @@ -5,7 +5,7 @@ from ray.rllib.evaluation.episode import Episode from ray.rllib.policy.policy_map import PolicyMap from ray.rllib.policy.sample_batch import MultiAgentBatch, SampleBatch -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import AgentID, EnvID, EpisodeID, PolicyID, TensorType if TYPE_CHECKING: @@ -16,7 +16,7 @@ # fmt: off # __sphinx_doc_begin__ -@PublicAPI +@OldAPIStack class SampleCollector(metaclass=ABCMeta): """Collects samples for all policies and agents from a multi-agent env. diff --git a/rllib/evaluation/collectors/simple_list_collector.py b/rllib/evaluation/collectors/simple_list_collector.py index 28da7e2bc6b..68dc3f63865 100644 --- a/rllib/evaluation/collectors/simple_list_collector.py +++ b/rllib/evaluation/collectors/simple_list_collector.py @@ -12,7 +12,7 @@ from ray.rllib.policy.policy import Policy from ray.rllib.policy.policy_map import PolicyMap from ray.rllib.policy.sample_batch import SampleBatch, MultiAgentBatch, concat_samples -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.debug import summarize from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.spaces.space_utils import get_dummy_batch_for_space @@ -35,6 +35,7 @@ logger = logging.getLogger(__name__) +@OldAPIStack class _PolicyCollector: """Collects already postprocessed (single agent) samples for one policy. @@ -110,7 +111,7 @@ def __init__(self, policy_map): self.agent_steps = 0 -@PublicAPI +@OldAPIStack class SimpleListCollector(SampleCollector): """Util to build SampleBatches for each policy in a multi-agent env. diff --git a/rllib/evaluation/env_runner_v2.py b/rllib/evaluation/env_runner_v2.py index 0de816f2f2c..2c095fe4b12 100644 --- a/rllib/evaluation/env_runner_v2.py +++ b/rllib/evaluation/env_runner_v2.py @@ -15,7 +15,7 @@ from ray.rllib.models.preprocessors import Preprocessor from ray.rllib.policy.policy import Policy from ray.rllib.policy.sample_batch import MultiAgentBatch, SampleBatch, concat_samples -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.filter import Filter from ray.rllib.utils.numpy import convert_to_numpy from ray.rllib.utils.spaces.space_utils import unbatch, get_original_space @@ -52,6 +52,7 @@ MS_TO_SEC = 1000.0 +@OldAPIStack class _PerfStats: """Sampler perf stats that will be included in rollout metrics.""" @@ -123,12 +124,14 @@ def get(self): return self._get_ema() +@OldAPIStack class _NewDefaultDict(defaultdict): def __missing__(self, env_id): ret = self[env_id] = self.default_factory(env_id) return ret +@OldAPIStack def _build_multi_agent_batch( episode_id: int, batch_builder: _PolicyCollectorGroup, @@ -200,6 +203,7 @@ def _build_multi_agent_batch( return MultiAgentBatch(policy_batches=ma_batch, env_steps=batch_builder.env_steps) +@OldAPIStack def _batch_inference_sample_batches(eval_data: List[SampleBatch]) -> SampleBatch: """Batch a list of input SampleBatches into a single SampleBatch. @@ -216,7 +220,7 @@ def _batch_inference_sample_batches(eval_data: List[SampleBatch]) -> SampleBatch return inference_batch -@DeveloperAPI +@OldAPIStack class EnvRunnerV2: """Collect experiences from user environment using Connectors.""" diff --git a/rllib/evaluation/episode.py b/rllib/evaluation/episode.py index 018009d32bf..bb07537d8fa 100644 --- a/rllib/evaluation/episode.py +++ b/rllib/evaluation/episode.py @@ -7,7 +7,7 @@ from ray.rllib.env.base_env import _DUMMY_AGENT_ID from ray.rllib.policy.policy_map import PolicyMap -from ray.rllib.utils.annotations import Deprecated, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.spaces.space_utils import flatten_to_single_ndarray from ray.rllib.utils.typing import ( @@ -26,7 +26,7 @@ from ray.rllib.evaluation.sample_batch_builder import MultiAgentSampleBatchBuilder -@DeveloperAPI +@OldAPIStack class Episode: """Tracks the current state of a (possibly multi-agent) episode. @@ -120,7 +120,6 @@ def __init__( self._agent_to_prev_action: Dict[AgentID, EnvActionType] = {} self._agent_reward_history: Dict[AgentID, List[int]] = defaultdict(list) - @DeveloperAPI def policy_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> PolicyID: """Returns and stores the policy ID for the specified agent. @@ -171,7 +170,6 @@ def policy_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> PolicyID: ) return policy_id - @DeveloperAPI def last_observation_for( self, agent_id: AgentID = _DUMMY_AGENT_ID ) -> Optional[EnvObsType]: @@ -187,7 +185,6 @@ def last_observation_for( return self._agent_to_last_obs.get(agent_id) - @DeveloperAPI def last_raw_obs_for( self, agent_id: AgentID = _DUMMY_AGENT_ID ) -> Optional[EnvObsType]: @@ -204,7 +201,6 @@ def last_raw_obs_for( """ return self._agent_to_last_raw_obs.get(agent_id) - @DeveloperAPI def last_info_for( self, agent_id: AgentID = _DUMMY_AGENT_ID ) -> Optional[EnvInfoDict]: @@ -220,7 +216,6 @@ def last_info_for( """ return self._agent_to_last_info.get(agent_id) - @DeveloperAPI def last_action_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> EnvActionType: """Returns the last action for the specified AgentID, or zeros. @@ -258,7 +253,6 @@ def last_action_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> EnvActionType: return np.zeros_like(flat, dtype=policy.action_space.dtype) return np.zeros_like(flat) - @DeveloperAPI def prev_action_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> EnvActionType: """Returns the previous action for the specified agent, or zeros. @@ -294,7 +288,6 @@ def prev_action_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> EnvActionType: else: return np.zeros_like(self.last_action_for(agent_id)) - @DeveloperAPI def last_reward_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> float: """Returns the last reward for the specified agent, or zero. @@ -317,7 +310,6 @@ def last_reward_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> float: else: return 0.0 - @DeveloperAPI def prev_reward_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> float: """Returns the previous reward for the specified agent, or zero. @@ -341,7 +333,6 @@ def prev_reward_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> float: else: return 0.0 - @DeveloperAPI def rnn_state_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> List[Any]: """Returns the last RNN state for the specified agent. @@ -358,7 +349,6 @@ def rnn_state_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> List[Any]: self._agent_to_rnn_state[agent_id] = policy.get_initial_state() return self._agent_to_rnn_state[agent_id] - @DeveloperAPI def last_terminated_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> bool: """Returns the last `terminated` flag for the specified AgentID. @@ -372,7 +362,6 @@ def last_terminated_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> bool: self._agent_to_last_terminated[agent_id] = False return self._agent_to_last_terminated[agent_id] - @DeveloperAPI def last_truncated_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> bool: """Returns the last `truncated` flag for the specified AgentID. @@ -386,7 +375,6 @@ def last_truncated_for(self, agent_id: AgentID = _DUMMY_AGENT_ID) -> bool: self._agent_to_last_truncated[agent_id] = False return self._agent_to_last_truncated[agent_id] - @DeveloperAPI def last_extra_action_outs_for( self, agent_id: AgentID = _DUMMY_AGENT_ID, @@ -405,7 +393,6 @@ def last_extra_action_outs_for( """ return self._agent_to_last_extra_action_outs[agent_id] - @DeveloperAPI def get_agents(self) -> List[AgentID]: """Returns list of agent IDs that have appeared in this episode. @@ -462,7 +449,3 @@ def _policy_mapping_fn(self): error=True, ) return self.policy_mapping_fn - - @Deprecated(new="Episode.last_extra_action_outs_for", error=True) - def last_pi_info_for(self, *args, **kwargs): - return self.last_extra_action_outs_for(*args, **kwargs) diff --git a/rllib/evaluation/episode_v2.py b/rllib/evaluation/episode_v2.py index c68a579c857..25ed5a36a71 100644 --- a/rllib/evaluation/episode_v2.py +++ b/rllib/evaluation/episode_v2.py @@ -11,7 +11,7 @@ from ray.rllib.evaluation.collectors.agent_collector import AgentCollector from ray.rllib.policy.policy_map import PolicyMap from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import AgentID, EnvID, EnvInfoDict, PolicyID, TensorType if TYPE_CHECKING: @@ -19,7 +19,7 @@ from ray.rllib.evaluation.rollout_worker import RolloutWorker -@DeveloperAPI +@OldAPIStack class EpisodeV2: """Tracks the current state of a (possibly multi-agent) episode.""" @@ -97,7 +97,6 @@ def __init__( # us something. self._last_infos: Dict[AgentID, Dict] = {} - @DeveloperAPI def policy_for( self, agent_id: AgentID = _DUMMY_AGENT_ID, refresh: bool = False ) -> PolicyID: @@ -133,7 +132,6 @@ def policy_for( ) return policy_id - @DeveloperAPI def get_agents(self) -> List[AgentID]: """Returns list of agent IDs that have appeared in this episode. diff --git a/rllib/evaluation/observation_function.py b/rllib/evaluation/observation_function.py index e50c003f306..fe33b3d6066 100644 --- a/rllib/evaluation/observation_function.py +++ b/rllib/evaluation/observation_function.py @@ -3,10 +3,12 @@ from ray.rllib.env import BaseEnv from ray.rllib.policy import Policy from ray.rllib.evaluation import Episode, RolloutWorker +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import TensorType from ray.rllib.utils.typing import AgentID, PolicyID +@OldAPIStack class ObservationFunction: """Interceptor function for rewriting observations from the environment. diff --git a/rllib/evaluation/postprocessing.py b/rllib/evaluation/postprocessing.py index d9de8403505..65ebfc3350c 100644 --- a/rllib/evaluation/postprocessing.py +++ b/rllib/evaluation/postprocessing.py @@ -5,7 +5,7 @@ from ray.rllib.evaluation.episode import Episode from ray.rllib.policy.policy import Policy from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import DeveloperAPI, OldAPIStack from ray.rllib.utils.nested_dict import NestedDict from ray.rllib.utils.numpy import convert_to_numpy from ray.rllib.utils.torch_utils import convert_to_torch_tensor @@ -21,7 +21,7 @@ class Postprocessing: VALUE_TARGETS = "value_targets" -@DeveloperAPI +@OldAPIStack def adjust_nstep(n_step: int, gamma: float, batch: SampleBatch) -> None: """Rewrites `batch` to encode n-step rewards, terminateds, truncateds, and next-obs. @@ -85,7 +85,7 @@ def adjust_nstep(n_step: int, gamma: float, batch: SampleBatch) -> None: ) -@DeveloperAPI +@OldAPIStack def compute_advantages( rollout: SampleBatch, last_r: float, @@ -154,7 +154,7 @@ def compute_advantages( return rollout -@DeveloperAPI +@OldAPIStack def compute_gae_for_sample_batch( policy: Policy, sample_batch: SampleBatch, @@ -222,7 +222,7 @@ def compute_gae_for_sample_batch( return batch -@DeveloperAPI +@OldAPIStack def compute_bootstrap_value(sample_batch: SampleBatch, policy: Policy) -> SampleBatch: """Performs a value function computation at the end of a trajectory. @@ -327,7 +327,7 @@ def compute_bootstrap_value(sample_batch: SampleBatch, policy: Policy) -> Sample return sample_batch -@DeveloperAPI +@OldAPIStack def discount_cumsum(x: np.ndarray, gamma: float) -> np.ndarray: """Calculates the discounted cumulative sum over a reward sequence `x`. diff --git a/rllib/evaluation/postprocessing_v2.py b/rllib/evaluation/postprocessing_v2.py index 1ad383c1c37..2ee6aed77c8 100644 --- a/rllib/evaluation/postprocessing_v2.py +++ b/rllib/evaluation/postprocessing_v2.py @@ -1,6 +1,9 @@ import numpy as np +from ray.rllib.utils.annotations import OldAPIStack + +@OldAPIStack def compute_value_targets( values, rewards, diff --git a/rllib/evaluation/rollout_worker.py b/rllib/evaluation/rollout_worker.py index caa94e59240..8351f882e56 100644 --- a/rllib/evaluation/rollout_worker.py +++ b/rllib/evaluation/rollout_worker.py @@ -66,7 +66,7 @@ from ray.rllib.policy.torch_policy import TorchPolicy from ray.rllib.policy.torch_policy_v2 import TorchPolicyV2 from ray.rllib.utils import check_env, force_list -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.debug import summarize, update_global_seed_if_necessary from ray.rllib.utils.deprecation import DEPRECATED_VALUE, deprecation_warning from ray.rllib.utils.error import ERR_MSG_NO_GPUS, HOWTO_CHANGE_CONFIG @@ -112,7 +112,7 @@ _global_worker: Optional["RolloutWorker"] = None -@DeveloperAPI +@OldAPIStack def get_global_worker() -> "RolloutWorker": """Returns a handle to the active rollout worker in this process.""" @@ -155,7 +155,7 @@ def _update_env_seed_if_necessary( ) -@DeveloperAPI +@OldAPIStack class RolloutWorker(ParallelIteratorWorker, EnvRunner): """Common experience collection class. @@ -1670,17 +1670,14 @@ def creation_args(self) -> dict: """Returns the kwargs dict used to create this worker.""" return self._original_kwargs - @DeveloperAPI def get_host(self) -> str: """Returns the hostname of the process running this evaluator.""" return platform.node() - @DeveloperAPI def get_node_ip(self) -> str: """Returns the IP address of the node that this worker runs on.""" return ray.util.get_node_ip_address() - @DeveloperAPI def find_free_port(self) -> int: """Finds a free port on the node that this worker runs on.""" from ray.air._internal.util import find_free_port diff --git a/rllib/evaluation/sample_batch_builder.py b/rllib/evaluation/sample_batch_builder.py index 85ece5df40c..7b7f2847477 100644 --- a/rllib/evaluation/sample_batch_builder.py +++ b/rllib/evaluation/sample_batch_builder.py @@ -7,8 +7,7 @@ from ray.rllib.evaluation.episode import Episode from ray.rllib.policy.policy import Policy from ray.rllib.policy.sample_batch import SampleBatch, MultiAgentBatch -from ray.rllib.utils.annotations import DeveloperAPI -from ray.rllib.utils.deprecation import Deprecated +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.debug import summarize from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.typing import PolicyID, AgentID @@ -27,7 +26,7 @@ def _to_float_array(v: List[Any]) -> np.ndarray: return arr -@Deprecated(new="a child class of `SampleCollector`", error=False) +@OldAPIStack class SampleBatchBuilder: """Util to build a SampleBatch incrementally. @@ -69,9 +68,7 @@ def build_and_reset(self) -> SampleBatch: return batch -# Deprecated class: Use a child class of `SampleCollector` instead -# (which handles multi-agent setups as well). -@DeveloperAPI +@OldAPIStack class MultiAgentSampleBatchBuilder: """Util to build SampleBatches for each policy in a multi-agent env. @@ -131,7 +128,6 @@ def has_pending_agent_data(self) -> bool: return len(self.agent_builders) > 0 - @DeveloperAPI def add_values(self, agent_id: AgentID, policy_id: AgentID, **values: Any) -> None: """Add the given dictionary (row) of values to this batch. @@ -244,7 +240,6 @@ def check_missing_dones(self) -> None: "to True. " ) - @DeveloperAPI def build_and_reset(self, episode: Optional[Episode] = None) -> MultiAgentBatch: """Returns the accumulated sample batches for each policy. diff --git a/rllib/evaluation/sampler.py b/rllib/evaluation/sampler.py index ed1b9a75883..9e34fd237ee 100644 --- a/rllib/evaluation/sampler.py +++ b/rllib/evaluation/sampler.py @@ -35,7 +35,7 @@ from ray.rllib.policy.policy import Policy from ray.rllib.policy.policy_map import PolicyMap from ray.rllib.policy.sample_batch import SampleBatch, concat_samples -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.debug import summarize from ray.rllib.utils.deprecation import deprecation_warning, DEPRECATED_VALUE from ray.rllib.utils.framework import try_import_tf @@ -82,7 +82,7 @@ def __missing__(self, env_id): return ret -@DeveloperAPI +@OldAPIStack class SamplerInput(InputReader, metaclass=ABCMeta): """Reads input experiences from an existing sampler.""" @@ -95,7 +95,6 @@ def next(self) -> SampleBatchType: return concat_samples(batches) @abstractmethod - @DeveloperAPI def get_data(self) -> SampleBatchType: """Called by `self.next()` to return the next batch of data. @@ -107,7 +106,6 @@ def get_data(self) -> SampleBatchType: raise NotImplementedError @abstractmethod - @DeveloperAPI def get_metrics(self) -> List[RolloutMetrics]: """Returns list of episode metrics since the last call to this method. @@ -120,7 +118,6 @@ def get_metrics(self) -> List[RolloutMetrics]: raise NotImplementedError @abstractmethod - @DeveloperAPI def get_extra_batches(self) -> List[SampleBatchType]: """Returns list of extra batches since the last call to this method. @@ -139,7 +136,7 @@ def get_extra_batches(self) -> List[SampleBatchType]: raise NotImplementedError -@DeveloperAPI +@OldAPIStack class SyncSampler(SamplerInput): """Sync SamplerInput that collects experiences when `get_data()` is called.""" @@ -304,6 +301,7 @@ def get_extra_batches(self) -> List[SampleBatchType]: return extra +@OldAPIStack def _env_runner( worker: "RolloutWorker", base_env: BaseEnv, @@ -476,6 +474,7 @@ def _new_episode(env_id): perf_stats.incr("env_render_time", time.time() - t5) +@OldAPIStack def _process_observations( *, worker: "RolloutWorker", @@ -894,6 +893,7 @@ def _process_observations( return active_envs, to_eval, outputs +@OldAPIStack def _do_policy_eval( *, to_eval: Dict[PolicyID, List[_PolicyEvalData]], @@ -952,6 +952,7 @@ def _do_policy_eval( return eval_results +@OldAPIStack def _process_policy_eval_results( *, to_eval: Dict[PolicyID, List[_PolicyEvalData]], @@ -1048,6 +1049,7 @@ def _process_policy_eval_results( return actions_to_send +@OldAPIStack def _create_episode(active_episodes, env_id, callbacks, worker, base_env): # Make sure we are really creating a new episode here. assert env_id not in active_episodes @@ -1066,6 +1068,7 @@ def _create_episode(active_episodes, env_id, callbacks, worker, base_env): return new_episode +@OldAPIStack def _call_on_episode_start(episode, env_id, callbacks, worker, base_env): # Call each policy's Exploration.on_episode_start method. # Note: This may break the exploration (e.g. ParameterNoise) of diff --git a/rllib/execution/buffers/mixin_replay_buffer.py b/rllib/execution/buffers/mixin_replay_buffer.py index b6dd623f432..6f897ac06f5 100644 --- a/rllib/execution/buffers/mixin_replay_buffer.py +++ b/rllib/execution/buffers/mixin_replay_buffer.py @@ -6,12 +6,13 @@ from ray.util.timer import _Timer from ray.rllib.execution.replay_ops import SimpleReplayBuffer from ray.rllib.policy.sample_batch import DEFAULT_POLICY_ID, concat_samples -from ray.rllib.utils.deprecation import Deprecated +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.replay_buffers.multi_agent_replay_buffer import ReplayMode from ray.rllib.utils.replay_buffers.replay_buffer import _ALL_POLICIES from ray.rllib.utils.typing import PolicyID, SampleBatchType +@OldAPIStack class MixInMultiAgentReplayBuffer: """This buffer adds replayed samples to a stream of new experiences. @@ -170,7 +171,3 @@ def get_host(self) -> str: name could not be determined. """ return platform.node() - - @Deprecated(new="MixInMultiAgentReplayBuffer.add()", error=True) - def add_batch(self, *args, **kwargs): - pass diff --git a/rllib/execution/learner_thread.py b/rllib/execution/learner_thread.py index 78643aa1578..49340a972c3 100644 --- a/rllib/execution/learner_thread.py +++ b/rllib/execution/learner_thread.py @@ -6,6 +6,7 @@ from ray.util.timer import _Timer from ray.rllib.evaluation.rollout_worker import RolloutWorker from ray.rllib.execution.minibatch_buffer import MinibatchBuffer +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.metrics.learner_info import LearnerInfoBuilder, LEARNER_INFO from ray.rllib.utils.metrics.window_stat import WindowStat @@ -14,8 +15,7 @@ tf1, tf, tfv = try_import_tf() -# TODO (sven): Deprecate once all algos are only available via the new API stack -# (learner API). +@OldAPIStack class LearnerThread(threading.Thread): """Background thread that updates the local model from sample trajectories. diff --git a/rllib/execution/minibatch_buffer.py b/rllib/execution/minibatch_buffer.py index 68b5e387c15..29b224ce2e3 100644 --- a/rllib/execution/minibatch_buffer.py +++ b/rllib/execution/minibatch_buffer.py @@ -1,8 +1,10 @@ from typing import Any, Tuple import queue +from ray.rllib.utils.annotations import OldAPIStack -# TODO (sven): Deprecate MinibatchBuffer. + +@OldAPIStack class MinibatchBuffer: """Ring buffer of recent data batches for minibatch SGD. diff --git a/rllib/execution/multi_gpu_learner_thread.py b/rllib/execution/multi_gpu_learner_thread.py index bb7a0c6e2e1..aacf797b32b 100644 --- a/rllib/execution/multi_gpu_learner_thread.py +++ b/rllib/execution/multi_gpu_learner_thread.py @@ -6,7 +6,7 @@ from ray.rllib.execution.learner_thread import LearnerThread from ray.rllib.execution.minibatch_buffer import MinibatchBuffer from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.metrics.learner_info import LearnerInfoBuilder @@ -17,8 +17,7 @@ logger = logging.getLogger(__name__) -# TODO (sven): Deprecate once all algos are only available via the new API stack -# (learner API). +@OldAPIStack class MultiGPULearnerThread(LearnerThread): """Learner that can use multiple GPUs and parallel loading. diff --git a/rllib/execution/replay_ops.py b/rllib/execution/replay_ops.py index 24d24e04eb1..bcd1f026cf1 100644 --- a/rllib/execution/replay_ops.py +++ b/rllib/execution/replay_ops.py @@ -1,11 +1,12 @@ from typing import Optional import random +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.replay_buffers.replay_buffer import warn_replay_capacity from ray.rllib.utils.typing import SampleBatchType -# TODO(sven) deprecate this class. +@OldAPIStack class SimpleReplayBuffer: """Simple replay buffer that operates over batches.""" diff --git a/rllib/execution/rollout_ops.py b/rllib/execution/rollout_ops.py index 033f846b522..1d599e5bfae 100644 --- a/rllib/execution/rollout_ops.py +++ b/rllib/execution/rollout_ops.py @@ -8,7 +8,7 @@ DEFAULT_POLICY_ID, concat_samples, ) -from ray.rllib.utils.annotations import ExperimentalAPI +from ray.rllib.utils.annotations import ExperimentalAPI, OldAPIStack from ray.rllib.utils.sgd import standardized from ray.rllib.utils.typing import EpisodeType, SampleBatchType @@ -127,6 +127,7 @@ def synchronous_parallel_sample( return sample_batches_or_episodes +@OldAPIStack def standardize_fields(samples: SampleBatchType, fields: List[str]) -> SampleBatchType: """Standardize fields of the given SampleBatch""" wrapped = False diff --git a/rllib/execution/train_ops.py b/rllib/execution/train_ops.py index 09a1986defa..1053d49499a 100644 --- a/rllib/execution/train_ops.py +++ b/rllib/execution/train_ops.py @@ -4,7 +4,7 @@ from typing import Dict from ray.rllib.policy.sample_batch import DEFAULT_POLICY_ID -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.metrics import ( @@ -22,7 +22,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack def train_one_step(algorithm, train_batch, policies_to_train=None) -> Dict: """Function that improves the all policies in `train_batch` on the local worker. @@ -82,7 +82,7 @@ def train_one_step(algorithm, train_batch, policies_to_train=None) -> Dict: return info -@DeveloperAPI +@OldAPIStack def multi_gpu_train_one_step(algorithm, train_batch) -> Dict: """Multi-GPU version of train_one_step. diff --git a/rllib/models/action_dist.py b/rllib/models/action_dist.py index e32ce2d07d0..53bd8bb84ec 100644 --- a/rllib/models/action_dist.py +++ b/rllib/models/action_dist.py @@ -2,11 +2,11 @@ import gymnasium as gym from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import TensorType, List, Union, ModelConfigDict -@DeveloperAPI +@OldAPIStack class ActionDistribution: """The policy action distribution of an agent. @@ -15,7 +15,6 @@ class ActionDistribution: model (ModelV2): reference to model producing the inputs. """ - @DeveloperAPI def __init__(self, inputs: List[TensorType], model: ModelV2): """Initializes an ActionDist object. @@ -29,12 +28,10 @@ def __init__(self, inputs: List[TensorType], model: ModelV2): self.inputs = inputs self.model = model - @DeveloperAPI def sample(self) -> TensorType: """Draw a sample from the action distribution.""" raise NotImplementedError - @DeveloperAPI def deterministic_sample(self) -> TensorType: """ Get the deterministic "sampling" output from the distribution. @@ -43,22 +40,18 @@ def deterministic_sample(self) -> TensorType: """ raise NotImplementedError - @DeveloperAPI def sampled_action_logp(self) -> TensorType: """Returns the log probability of the last sampled action.""" raise NotImplementedError - @DeveloperAPI def logp(self, x: TensorType) -> TensorType: """The log-likelihood of the action distribution.""" raise NotImplementedError - @DeveloperAPI def kl(self, other: "ActionDistribution") -> TensorType: """The KL-divergence between two action distributions.""" raise NotImplementedError - @DeveloperAPI def entropy(self) -> TensorType: """The entropy of the action distribution.""" raise NotImplementedError @@ -80,7 +73,7 @@ def multi_entropy(self) -> TensorType: return self.entropy() @staticmethod - @DeveloperAPI + @OldAPIStack def required_model_output_shape( action_space: gym.Space, model_config: ModelConfigDict ) -> Union[int, np.ndarray]: diff --git a/rllib/models/modelv2.py b/rllib/models/modelv2.py index 18a464782bb..c4451aed324 100644 --- a/rllib/models/modelv2.py +++ b/rllib/models/modelv2.py @@ -10,8 +10,7 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement from ray.rllib.utils import NullContextManager -from ray.rllib.utils.annotations import DeveloperAPI, PublicAPI -from ray.rllib.utils.deprecation import Deprecated +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf, try_import_torch, TensorType from ray.rllib.utils.spaces.repeated import Repeated from ray.rllib.utils.typing import ModelConfigDict, ModelInputDict, TensorStructType @@ -20,7 +19,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class ModelV2: r"""Defines an abstract neural network model for use with RLlib. @@ -71,9 +70,6 @@ def __init__( SampleBatch.OBS: ViewRequirement(shift=0, space=self.obs_space), } - # TODO: (sven): Get rid of `get_initial_state` once Trajectory - # View API is supported across all of RLlib. - @PublicAPI def get_initial_state(self) -> List[TensorType]: """Get the initial recurrent state values for the model. @@ -96,7 +92,6 @@ def get_initial_state(self): """ return [] - @PublicAPI def forward( self, input_dict: Dict[str, TensorType], @@ -142,7 +137,6 @@ def forward(self, input_dict, state, seq_lens): """ raise NotImplementedError - @PublicAPI def value_function(self) -> TensorType: """Returns the value function output for the most recent forward pass. @@ -155,7 +149,6 @@ def value_function(self) -> TensorType: """ raise NotImplementedError - @PublicAPI def custom_loss( self, policy_loss: TensorType, loss_inputs: Dict[str, TensorType] ) -> Union[List[TensorType], TensorType]: @@ -178,7 +171,6 @@ def custom_loss( """ return policy_loss - @PublicAPI def metrics(self) -> Dict[str, TensorType]: """Override to return custom metrics from your model. @@ -297,17 +289,14 @@ def import_from_h5(self, h5_file: str) -> None: """ raise NotImplementedError - @PublicAPI def last_output(self) -> TensorType: """Returns the last output returned from calling the model.""" return self._last_output - @PublicAPI def context(self) -> contextlib.AbstractContextManager: """Returns a contextmanager for the current forward pass.""" return NullContextManager() - @PublicAPI def variables( self, as_dict: bool = False ) -> Union[List[TensorType], Dict[str, TensorType]]: @@ -323,7 +312,6 @@ def variables( """ raise NotImplementedError - @PublicAPI def trainable_variables( self, as_dict: bool = False ) -> Union[List[TensorType], Dict[str, TensorType]]: @@ -339,7 +327,6 @@ def trainable_variables( """ raise NotImplementedError - @PublicAPI def is_time_major(self) -> bool: """If True, data for calling this ModelV2 must be in time-major format. @@ -349,28 +336,8 @@ def is_time_major(self) -> bool: """ return self.time_major is True - @Deprecated(new="ModelV2.__call__()", error=True) - def from_batch( - self, train_batch: SampleBatch, is_training: bool = True - ) -> (TensorType, List[TensorType]): - """Convenience function that calls this model with a tensor batch. - - All this does is unpack the tensor batch to call this model with the - right input dict, state, and seq len arguments. - """ - - input_dict = train_batch.copy() - input_dict.set_training(is_training) - states = [] - i = 0 - while "state_in_{}".format(i) in input_dict: - states.append(input_dict["state_in_{}".format(i)]) - i += 1 - ret = self.__call__(input_dict, states, input_dict.get(SampleBatch.SEQ_LENS)) - return ret - -@DeveloperAPI +@OldAPIStack def flatten(obs: TensorType, framework: str) -> TensorType: """Flatten the given tensor.""" if framework in ["tf2", "tf"]: @@ -382,7 +349,7 @@ def flatten(obs: TensorType, framework: str) -> TensorType: raise NotImplementedError("flatten", framework) -@DeveloperAPI +@OldAPIStack def restore_original_dimensions( obs: TensorType, obs_space: Space, tensorlib: Any = tf ) -> TensorStructType: @@ -421,6 +388,7 @@ def restore_original_dimensions( _cache = {} +@OldAPIStack def _unpack_obs(obs: TensorType, space: Space, tensorlib: Any = tf) -> TensorStructType: """Unpack a flattened Dict or Tuple observation array/tensor. diff --git a/rllib/models/preprocessors.py b/rllib/models/preprocessors.py index 1d3f0c19e18..f6578b58ec9 100644 --- a/rllib/models/preprocessors.py +++ b/rllib/models/preprocessors.py @@ -4,7 +4,7 @@ import gymnasium as gym from typing import Any, List -from ray.rllib.utils.annotations import override, PublicAPI, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.spaces.repeated import Repeated from ray.rllib.utils.typing import TensorType from ray.rllib.utils.images import resize @@ -20,7 +20,7 @@ logger = logging.getLogger(__name__) -@PublicAPI +@OldAPIStack class Preprocessor: """Defines an abstract observation preprocessor function. @@ -28,7 +28,6 @@ class Preprocessor: shape (List[int]): Shape of the preprocessed output. """ - @PublicAPI def __init__(self, obs_space: gym.Space, options: dict = None): _legacy_patch_shapes(obs_space) self._obs_space = obs_space @@ -43,12 +42,10 @@ def __init__(self, obs_space: gym.Space, options: dict = None): self._i = 0 self._obs_for_type_matching = self._obs_space.sample() - @PublicAPI def _init_shape(self, obs_space: gym.Space, options: dict) -> List[int]: """Returns the shape after preprocessing.""" raise NotImplementedError - @PublicAPI def transform(self, observation: TensorType) -> np.ndarray: """Returns the preprocessed observation.""" raise NotImplementedError @@ -89,12 +86,10 @@ def check_shape(self, observation: Any) -> None: self._i += 1 @property - @PublicAPI def size(self) -> int: return self._size @property - @PublicAPI def observation_space(self) -> gym.Space: obs_space = gym.spaces.Box(-1.0, 1.0, self.shape, dtype=np.float32) # Stash the unwrapped space so that we can unwrap dict and tuple spaces @@ -112,7 +107,7 @@ def observation_space(self) -> gym.Space: return obs_space -@DeveloperAPI +@OldAPIStack class GenericPixelPreprocessor(Preprocessor): """Generic image preprocessor. @@ -155,7 +150,7 @@ def transform(self, observation: TensorType) -> np.ndarray: return scaled -@DeveloperAPI +@OldAPIStack class AtariRamPreprocessor(Preprocessor): @override(Preprocessor) def _init_shape(self, obs_space: gym.Space, options: dict) -> List[int]: @@ -167,7 +162,7 @@ def transform(self, observation: TensorType) -> np.ndarray: return (observation.astype("float32") - 128) / 128 -@DeveloperAPI +@OldAPIStack class OneHotPreprocessor(Preprocessor): """One-hot preprocessor for Discrete and MultiDiscrete spaces. @@ -207,7 +202,7 @@ def write(self, observation: TensorType, array: np.ndarray, offset: int) -> None array[offset : offset + self.size] = self.transform(observation) -@PublicAPI +@OldAPIStack class NoPreprocessor(Preprocessor): @override(Preprocessor) def _init_shape(self, obs_space: gym.Space, options: dict) -> List[int]: @@ -228,7 +223,7 @@ def observation_space(self) -> gym.Space: return self._obs_space -@PublicAPI +@OldAPIStack class MultiBinaryPreprocessor(Preprocessor): """Preprocessor that turns a MultiBinary space into a Box. @@ -260,7 +255,7 @@ def observation_space(self) -> gym.Space: return obs_space -@DeveloperAPI +@OldAPIStack class TupleFlatteningPreprocessor(Preprocessor): """Preprocesses each tuple element, then flattens it all into a vector. @@ -300,7 +295,7 @@ def write(self, observation: TensorType, array: np.ndarray, offset: int) -> None offset += p.size -@DeveloperAPI +@OldAPIStack class DictFlatteningPreprocessor(Preprocessor): """Preprocesses each dict value, then flattens it all into a vector. @@ -344,7 +339,7 @@ def write(self, observation: TensorType, array: np.ndarray, offset: int) -> None offset += p.size -@DeveloperAPI +@OldAPIStack class RepeatedValuesPreprocessor(Preprocessor): """Pads and batches the variable-length list value.""" @@ -389,7 +384,7 @@ def write(self, observation: TensorType, array: np.ndarray, offset: int) -> None self.child_preprocessor.write(elem, array, offset_i) -@PublicAPI +@OldAPIStack def get_preprocessor(space: gym.Space, include_multi_binary=False) -> type: """Returns an appropriate preprocessor class for the given space.""" diff --git a/rllib/models/repeated_values.py b/rllib/models/repeated_values.py index 6c5d4930191..7ecef777f66 100644 --- a/rllib/models/repeated_values.py +++ b/rllib/models/repeated_values.py @@ -1,10 +1,10 @@ from typing import List -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.typing import TensorType, TensorStructType -@PublicAPI +@OldAPIStack class RepeatedValues: """Represents a variable-length list of items from spaces.Repeated. diff --git a/rllib/models/tf/attention_net.py b/rllib/models/tf/attention_net.py index 92bb331e0dd..15e09688755 100644 --- a/rllib/models/tf/attention_net.py +++ b/rllib/models/tf/attention_net.py @@ -24,7 +24,7 @@ from ray.rllib.models.tf.recurrent_net import RecurrentNetwork from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.tf_utils import flatten_inputs_to_1d_tensor, one_hot @@ -35,7 +35,7 @@ tf1, tf, tfv = try_import_tf() -# TODO(sven): Use RLlib's FCNet instead. +@OldAPIStack class PositionwiseFeedforward(tf.keras.layers.Layer if tf else object): """A 2x linear layer with ReLU activation in between described in [1]. @@ -71,6 +71,7 @@ def call(self, inputs: TensorType, **kwargs) -> TensorType: return self._output_layer(output) +@OldAPIStack class TrXLNet(RecurrentNetwork): """A TrXL net Model described in [1].""" diff --git a/rllib/models/tf/complex_input_net.py b/rllib/models/tf/complex_input_net.py index 4c37c3540cc..0ed5e7c91aa 100644 --- a/rllib/models/tf/complex_input_net.py +++ b/rllib/models/tf/complex_input_net.py @@ -8,7 +8,7 @@ from ray.rllib.models.tf.tf_modelv2 import TFModelV2 from ray.rllib.models.utils import get_filter_config from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.spaces.space_utils import flatten_space from ray.rllib.utils.tf_utils import one_hot @@ -17,6 +17,7 @@ # __sphinx_doc_begin__ +@OldAPIStack class ComplexInputNetwork(TFModelV2): """TFModelV2 concat'ing CNN outputs to flat input(s), followed by FC(s). diff --git a/rllib/models/tf/fcnet.py b/rllib/models/tf/fcnet.py index 52faa86b284..bb59110124c 100644 --- a/rllib/models/tf/fcnet.py +++ b/rllib/models/tf/fcnet.py @@ -5,15 +5,14 @@ from ray.rllib.models.tf.misc import normc_initializer from ray.rllib.models.tf.tf_modelv2 import TFModelV2 from ray.rllib.models.utils import get_activation_fn +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.typing import TensorType, List, ModelConfigDict -from ray.rllib.utils.annotations import DeveloperAPI tf1, tf, tfv = try_import_tf() -# TODO: (sven) obsolete this class. -@DeveloperAPI +@OldAPIStack class FullyConnectedNetwork(TFModelV2): """Generic fully connected network implemented in ModelV2 API.""" diff --git a/rllib/models/tf/noop.py b/rllib/models/tf/noop.py index 83e6453202f..30d91988e3f 100644 --- a/rllib/models/tf/noop.py +++ b/rllib/models/tf/noop.py @@ -1,11 +1,12 @@ from ray.rllib.models.modelv2 import ModelV2 from ray.rllib.models.tf.tf_modelv2 import TFModelV2 -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf _, tf, _ = try_import_tf() +@OldAPIStack class NoopModel(TFModelV2): """Trivial model that just returns the obs flattened. diff --git a/rllib/models/tf/recurrent_net.py b/rllib/models/tf/recurrent_net.py index f04b829930c..31f1763850b 100644 --- a/rllib/models/tf/recurrent_net.py +++ b/rllib/models/tf/recurrent_net.py @@ -10,7 +10,7 @@ from ray.rllib.policy.rnn_sequencing import add_time_dimension from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement -from ray.rllib.utils.annotations import override, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.tf_utils import flatten_inputs_to_1d_tensor, one_hot @@ -22,7 +22,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class RecurrentNetwork(TFModelV2): """Helper class to simplify implementing RNN models with TFModelV2. @@ -129,7 +129,7 @@ def get_initial_state(self): raise NotImplementedError("You must implement this for a RNN model") -@DeveloperAPI +@OldAPIStack class LSTMWrapper(RecurrentNetwork): """An LSTM wrapper serving as an interface for ModelV2s that set use_lstm.""" diff --git a/rllib/models/tf/tf_action_dist.py b/rllib/models/tf/tf_action_dist.py index a28ac4cce42..683d1939776 100644 --- a/rllib/models/tf/tf_action_dist.py +++ b/rllib/models/tf/tf_action_dist.py @@ -8,7 +8,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.modelv2 import ModelV2 from ray.rllib.utils import MIN_LOG_NN_OUTPUT, MAX_LOG_NN_OUTPUT, SMALL_NUMBER -from ray.rllib.utils.annotations import override, DeveloperAPI, ExperimentalAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf, try_import_tfp from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.typing import TensorType, List, Union, Tuple, ModelConfigDict @@ -17,7 +17,7 @@ tfp = try_import_tfp() -@DeveloperAPI +@OldAPIStack class TFActionDistribution(ActionDistribution): """TF-specific extensions for building action distributions.""" @@ -46,7 +46,7 @@ def sampled_action_logp(self) -> TensorType: return self.sampled_action_logp_op -@DeveloperAPI +@OldAPIStack class Categorical(TFActionDistribution): """Categorical distribution for discrete action spaces.""" @@ -97,7 +97,7 @@ def required_model_output_shape(action_space, model_config): return action_space.n -@DeveloperAPI +@OldAPIStack def get_categorical_class_with_temperature(t: float): """Categorical distribution class that has customized default temperature.""" @@ -108,7 +108,7 @@ def __init__(self, inputs, model=None, temperature=t): return CategoricalWithTemperature -@DeveloperAPI +@OldAPIStack class MultiCategorical(TFActionDistribution): """MultiCategorical distribution for MultiDiscrete action spaces.""" @@ -204,7 +204,7 @@ def required_model_output_shape( return np.sum(action_space.nvec) -@ExperimentalAPI +@OldAPIStack class SlateMultiCategorical(Categorical): """MultiCategorical distribution for MultiDiscrete action spaces. @@ -247,7 +247,7 @@ def logp(self, x: TensorType) -> TensorType: return tf.ones_like(self.inputs[:, 0]) -@DeveloperAPI +@OldAPIStack class GumbelSoftmax(TFActionDistribution): """GumbelSoftmax distr. (for differentiable sampling in discr. actions @@ -319,7 +319,7 @@ def required_model_output_shape( return action_space.n -@DeveloperAPI +@OldAPIStack class DiagGaussian(TFActionDistribution): """Action distribution where each vector element is a gaussian. @@ -391,7 +391,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class SquashedGaussian(TFActionDistribution): """A tanh-squashed Gaussian distribution defined by: mean, std, low, high. @@ -490,7 +490,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class Beta(TFActionDistribution): """ A Beta distribution is defined on the interval [0, 1] and parameterized by @@ -546,7 +546,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class Deterministic(TFActionDistribution): """Action distribution that returns the input values directly. @@ -574,7 +574,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) -@DeveloperAPI +@OldAPIStack class MultiActionDistribution(TFActionDistribution): """Action distribution that operates on a set of actions. @@ -678,7 +678,7 @@ def required_model_output_shape(self, action_space, model_config): return np.sum(self.input_lens, dtype=np.int32) -@DeveloperAPI +@OldAPIStack class Dirichlet(TFActionDistribution): """Dirichlet distribution for continuous actions that are between [0,1] and sum to 1. diff --git a/rllib/models/tf/tf_modelv2.py b/rllib/models/tf/tf_modelv2.py index 580c8bef237..74387969442 100644 --- a/rllib/models/tf/tf_modelv2.py +++ b/rllib/models/tf/tf_modelv2.py @@ -5,7 +5,7 @@ from ray.util import log_once from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.typing import ModelConfigDict, TensorType @@ -13,7 +13,7 @@ tf1, tf, tfv = try_import_tf() -@PublicAPI +@OldAPIStack class TFModelV2(ModelV2): """TF version of ModelV2, which should contain a tf keras Model. diff --git a/rllib/models/tf/visionnet.py b/rllib/models/tf/visionnet.py index 44a74283276..69124c9e2e6 100644 --- a/rllib/models/tf/visionnet.py +++ b/rllib/models/tf/visionnet.py @@ -1,18 +1,17 @@ import gymnasium as gym from typing import Dict, List -from ray.rllib.utils.annotations import DeveloperAPI from ray.rllib.models.tf.tf_modelv2 import TFModelV2 from ray.rllib.models.tf.misc import normc_initializer from ray.rllib.models.utils import get_activation_fn, get_filter_config +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.typing import ModelConfigDict, TensorType tf1, tf, tfv = try_import_tf() -# TODO: (sven) obsolete this class once we only support native keras models. -@DeveloperAPI +@OldAPIStack class VisionNetwork(TFModelV2): """Generic vision network implemented in ModelV2 API. diff --git a/rllib/models/torch/attention_net.py b/rllib/models/torch/attention_net.py index 574074bf7d3..5efdcd9434d 100644 --- a/rllib/models/torch/attention_net.py +++ b/rllib/models/torch/attention_net.py @@ -25,7 +25,7 @@ from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.torch_utils import flatten_inputs_to_1d_tensor, one_hot @@ -36,6 +36,7 @@ torch, nn = try_import_torch() +@OldAPIStack class GTrXLNet(RecurrentNetwork, nn.Module): """A GTrXL net Model described in [2]. diff --git a/rllib/models/torch/complex_input_net.py b/rllib/models/torch/complex_input_net.py index f3cb4311521..90ce2cdf78d 100644 --- a/rllib/models/torch/complex_input_net.py +++ b/rllib/models/torch/complex_input_net.py @@ -13,7 +13,7 @@ from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 from ray.rllib.models.utils import get_filter_config from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.spaces.space_utils import flatten_space from ray.rllib.utils.torch_utils import one_hot @@ -21,6 +21,7 @@ torch, nn = try_import_torch() +@OldAPIStack class ComplexInputNetwork(TorchModelV2, nn.Module): """TorchModelV2 concat'ing CNN outputs to flat input(s), followed by FC(s). diff --git a/rllib/models/torch/fcnet.py b/rllib/models/torch/fcnet.py index 97bb9096bb6..0a78cbc51e1 100644 --- a/rllib/models/torch/fcnet.py +++ b/rllib/models/torch/fcnet.py @@ -4,7 +4,7 @@ from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 from ray.rllib.models.torch.misc import SlimFC, AppendBiasLayer, normc_initializer -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.typing import Dict, TensorType, List, ModelConfigDict @@ -13,6 +13,7 @@ logger = logging.getLogger(__name__) +@OldAPIStack class FullyConnectedNetwork(TorchModelV2, nn.Module): """Generic fully connected network.""" diff --git a/rllib/models/torch/modules/convtranspose2d_stack.py b/rllib/models/torch/modules/convtranspose2d_stack.py index f991400d3df..7740c461cd4 100644 --- a/rllib/models/torch/modules/convtranspose2d_stack.py +++ b/rllib/models/torch/modules/convtranspose2d_stack.py @@ -2,6 +2,7 @@ from ray.rllib.models.torch.misc import Reshape from ray.rllib.models.utils import get_activation_fn, get_initializer +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch torch, nn = try_import_torch() @@ -9,6 +10,7 @@ import torch.distributions as td +@OldAPIStack class ConvTranspose2DStack(nn.Module): """ConvTranspose2D decoder generating an image distribution from a vector.""" diff --git a/rllib/models/torch/modules/gru_gate.py b/rllib/models/torch/modules/gru_gate.py index 7eee53534d6..7bb6eee8454 100644 --- a/rllib/models/torch/modules/gru_gate.py +++ b/rllib/models/torch/modules/gru_gate.py @@ -1,9 +1,11 @@ +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch -from ray.rllib.utils.framework import TensorType +from ray.rllib.utils.typing import TensorType torch, nn = try_import_torch() +@OldAPIStack class GRUGate(nn.Module): """Implements a gated recurrent unit for use in AttentionNet""" diff --git a/rllib/models/torch/modules/multi_head_attention.py b/rllib/models/torch/modules/multi_head_attention.py index 68413bde025..cf4dfb50b26 100644 --- a/rllib/models/torch/modules/multi_head_attention.py +++ b/rllib/models/torch/modules/multi_head_attention.py @@ -5,12 +5,14 @@ """ from ray.rllib.utils.framework import try_import_torch from ray.rllib.models.torch.misc import SlimFC +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.torch_utils import sequence_mask from ray.rllib.utils.framework import TensorType torch, nn = try_import_torch() +@OldAPIStack class MultiHeadAttention(nn.Module): """A multi-head attention layer described in [1].""" diff --git a/rllib/models/torch/modules/relative_multi_head_attention.py b/rllib/models/torch/modules/relative_multi_head_attention.py index d3ff9cf59ee..aa19207e024 100644 --- a/rllib/models/torch/modules/relative_multi_head_attention.py +++ b/rllib/models/torch/modules/relative_multi_head_attention.py @@ -1,13 +1,15 @@ from typing import Union -from ray.rllib.utils.framework import try_import_torch from ray.rllib.models.torch.misc import SlimFC +from ray.rllib.utils.annotations import OldAPIStack +from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.torch_utils import sequence_mask from ray.rllib.utils.typing import TensorType torch, nn = try_import_torch() +@OldAPIStack class RelativePositionEmbedding(nn.Module): """Creates a [seq_length x seq_length] matrix for rel. pos encoding. diff --git a/rllib/models/torch/modules/skip_connection.py b/rllib/models/torch/modules/skip_connection.py index 8bc155eda9c..444c1680686 100644 --- a/rllib/models/torch/modules/skip_connection.py +++ b/rllib/models/torch/modules/skip_connection.py @@ -1,3 +1,4 @@ +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.typing import TensorType from typing import Optional @@ -5,6 +6,7 @@ torch, nn = try_import_torch() +@OldAPIStack class SkipConnection(nn.Module): """Skip connection layer. diff --git a/rllib/models/torch/noop.py b/rllib/models/torch/noop.py index 8b0705b1187..1a59b6165f7 100644 --- a/rllib/models/torch/noop.py +++ b/rllib/models/torch/noop.py @@ -1,8 +1,9 @@ from ray.rllib.models.modelv2 import ModelV2 from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override +@OldAPIStack class TorchNoopModel(TorchModelV2): """Trivial model that just returns the obs flattened. diff --git a/rllib/models/torch/recurrent_net.py b/rllib/models/torch/recurrent_net.py index 6a636e032b5..01fd0a0757b 100644 --- a/rllib/models/torch/recurrent_net.py +++ b/rllib/models/torch/recurrent_net.py @@ -10,7 +10,7 @@ from ray.rllib.policy.rnn_sequencing import add_time_dimension from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement -from ray.rllib.utils.annotations import override, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space from ray.rllib.utils.torch_utils import flatten_inputs_to_1d_tensor, one_hot @@ -21,7 +21,7 @@ torch, nn = try_import_torch() -@DeveloperAPI +@OldAPIStack class RecurrentNetwork(TorchModelV2): """Helper class to simplify implementing RNN models with TorchModelV2. @@ -123,7 +123,7 @@ def forward_rnn(self, inputs, state, seq_lens): raise NotImplementedError("You must implement this for an RNN model") -@DeveloperAPI +@OldAPIStack class LSTMWrapper(RecurrentNetwork, nn.Module): """An LSTM wrapper serving as an interface for ModelV2s that set use_lstm.""" diff --git a/rllib/models/torch/torch_action_dist.py b/rllib/models/torch/torch_action_dist.py index 743cd323e7e..91c69180070 100644 --- a/rllib/models/torch/torch_action_dist.py +++ b/rllib/models/torch/torch_action_dist.py @@ -7,7 +7,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 -from ray.rllib.utils.annotations import override, DeveloperAPI, ExperimentalAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.numpy import SMALL_NUMBER, MIN_LOG_NN_OUTPUT, MAX_LOG_NN_OUTPUT from ray.rllib.utils.spaces.space_utils import get_base_struct_from_space @@ -16,7 +16,7 @@ torch, nn = try_import_torch() -@DeveloperAPI +@OldAPIStack class TorchDistributionWrapper(ActionDistribution): """Wrapper class for torch.distributions.""" @@ -55,7 +55,7 @@ def sampled_action_logp(self) -> TensorType: return self.logp(self.last_sample) -@DeveloperAPI +@OldAPIStack class TorchCategorical(TorchDistributionWrapper): """Wrapper class for PyTorch Categorical distribution.""" @@ -85,7 +85,7 @@ def required_model_output_shape( return action_space.n -@DeveloperAPI +@OldAPIStack def get_torch_categorical_class_with_temperature(t: float): """TorchCategorical distribution class that has customized default temperature.""" @@ -96,7 +96,7 @@ def __init__(self, inputs, model=None, temperature=t): return TorchCategoricalWithTemperature -@DeveloperAPI +@OldAPIStack class TorchMultiCategorical(TorchDistributionWrapper): """MultiCategorical distribution for MultiDiscrete action spaces.""" @@ -189,7 +189,7 @@ def required_model_output_shape( return np.sum(action_space.nvec) -@ExperimentalAPI +@OldAPIStack class TorchSlateMultiCategorical(TorchCategorical): """MultiCategorical distribution for MultiDiscrete action spaces. @@ -232,7 +232,7 @@ def logp(self, x: TensorType) -> TensorType: return torch.ones_like(self.inputs[:, 0]) -@DeveloperAPI +@OldAPIStack class TorchDiagGaussian(TorchDistributionWrapper): """Wrapper class for PyTorch Normal distribution.""" @@ -283,7 +283,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class TorchSquashedGaussian(TorchDistributionWrapper): """A tanh-squashed Gaussian distribution defined by: mean, std, low, high. @@ -389,7 +389,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class TorchBeta(TorchDistributionWrapper): """ A Beta distribution is defined on the interval [0, 1] and parameterized by @@ -449,7 +449,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) * 2 -@DeveloperAPI +@OldAPIStack class TorchDeterministic(TorchDistributionWrapper): """Action distribution that returns the input values directly. @@ -477,7 +477,7 @@ def required_model_output_shape( return np.prod(action_space.shape, dtype=np.int32) -@DeveloperAPI +@OldAPIStack class TorchMultiActionDistribution(TorchDistributionWrapper): """Action distribution that operates on multiple, possibly nested actions.""" @@ -597,7 +597,7 @@ def required_model_output_shape(self, action_space, model_config): return np.sum(self.input_lens, dtype=np.int32) -@DeveloperAPI +@OldAPIStack class TorchDirichlet(TorchDistributionWrapper): """Dirichlet distribution for continuous actions that are between [0,1] and sum to 1. diff --git a/rllib/models/torch/torch_modelv2.py b/rllib/models/torch/torch_modelv2.py index 2c5f4b947bb..dd473c70de3 100644 --- a/rllib/models/torch/torch_modelv2.py +++ b/rllib/models/torch/torch_modelv2.py @@ -2,14 +2,14 @@ from typing import Dict, List, Union from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.typing import ModelConfigDict, TensorType _, nn = try_import_torch() -@PublicAPI +@OldAPIStack class TorchModelV2(ModelV2): """Torch version of ModelV2. diff --git a/rllib/models/torch/visionnet.py b/rllib/models/torch/visionnet.py index f3003d1c83c..748ba5796e3 100644 --- a/rllib/models/torch/visionnet.py +++ b/rllib/models/torch/visionnet.py @@ -10,13 +10,14 @@ SlimFC, ) from ray.rllib.models.utils import get_activation_fn, get_filter_config -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.typing import ModelConfigDict, TensorType torch, nn = try_import_torch() +@OldAPIStack class VisionNetwork(TorchModelV2, nn.Module): """Generic vision network.""" diff --git a/rllib/policy/__init__.py b/rllib/policy/__init__.py index 67868182a07..23ae3c0f6e1 100644 --- a/rllib/policy/__init__.py +++ b/rllib/policy/__init__.py @@ -2,7 +2,6 @@ from ray.rllib.policy.torch_policy import TorchPolicy from ray.rllib.policy.tf_policy import TFPolicy from ray.rllib.policy.policy_template import build_policy_class -from ray.rllib.policy.torch_policy_template import build_torch_policy from ray.rllib.policy.tf_policy_template import build_tf_policy __all__ = [ @@ -11,5 +10,4 @@ "TorchPolicy", "build_policy_class", "build_tf_policy", - "build_torch_policy", ] diff --git a/rllib/policy/dynamic_tf_policy.py b/rllib/policy/dynamic_tf_policy.py index 46745fd0c18..edda8c818b5 100644 --- a/rllib/policy/dynamic_tf_policy.py +++ b/rllib/policy/dynamic_tf_policy.py @@ -14,7 +14,7 @@ from ray.rllib.policy.view_requirement import ViewRequirement from ray.rllib.models.catalog import ModelCatalog from ray.rllib.utils import force_list -from ray.rllib.utils.annotations import override, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.debug import summarize from ray.rllib.utils.deprecation import ( deprecation_warning, @@ -42,7 +42,7 @@ TOWER_SCOPE_NAME = "tower" -@DeveloperAPI +@OldAPIStack class DynamicTFPolicy(TFPolicy): """A TFPolicy that auto-defines placeholders dynamically at runtime. @@ -51,7 +51,6 @@ class DynamicTFPolicy(TFPolicy): to generate your custom tf (graph-mode or eager) Policy classes. """ - @DeveloperAPI def __init__( self, obs_space: gym.spaces.Space, @@ -480,7 +479,6 @@ def __init__( self.get_session().run(tf1.global_variables_initializer()) @override(TFPolicy) - @DeveloperAPI def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> TFPolicy: """Creates a copy of self using existing input placeholders.""" @@ -554,7 +552,6 @@ def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> TFPolicy return instance @override(Policy) - @DeveloperAPI def get_initial_state(self) -> List[TensorType]: if self.model: return self.model.get_initial_state() @@ -562,7 +559,6 @@ def get_initial_state(self) -> List[TensorType]: return [] @override(Policy) - @DeveloperAPI def load_batch_into_buffer( self, batch: SampleBatch, @@ -595,7 +591,6 @@ def load_batch_into_buffer( ) @override(Policy) - @DeveloperAPI def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: # Shortcut for 1 CPU only: Batch should already be stored in # `self._loaded_single_cpu_batch`. @@ -610,7 +605,6 @@ def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: return self.multi_gpu_tower_stacks[buffer_index].num_tuples_loaded @override(Policy) - @DeveloperAPI def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): # Shortcut for 1 CPU only: Batch should already be stored in # `self._loaded_single_cpu_batch`. @@ -920,7 +914,7 @@ def _do_loss_init(self, train_batch: SampleBatch): return losses -@DeveloperAPI +@OldAPIStack class TFMultiGPUTowerStack: """Optimizer that runs in parallel across multiple local devices. diff --git a/rllib/policy/dynamic_tf_policy_v2.py b/rllib/policy/dynamic_tf_policy_v2.py index 9a51d8e6331..1f1d41aa176 100644 --- a/rllib/policy/dynamic_tf_policy_v2.py +++ b/rllib/policy/dynamic_tf_policy_v2.py @@ -15,7 +15,7 @@ from ray.rllib.policy.view_requirement import ViewRequirement from ray.rllib.utils import force_list from ray.rllib.utils.annotations import ( - DeveloperAPI, + OldAPIStack, OverrideToImplementCustomLogic, OverrideToImplementCustomLogic_CallToSuperRecommended, is_overridden, @@ -46,14 +46,13 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class DynamicTFPolicyV2(TFPolicy): """A TFPolicy that auto-defines placeholders dynamically at runtime. This class is intended to be used and extended by sub-classing. """ - @DeveloperAPI def __init__( self, obs_space: gym.spaces.Space, @@ -137,14 +136,12 @@ def __init__( timestep=timestep, ) - @DeveloperAPI @staticmethod def enable_eager_execution_if_necessary(): # This is static graph TF policy. # Simply do nothing. pass - @DeveloperAPI @OverrideToImplementCustomLogic def validate_spaces( self, @@ -154,7 +151,6 @@ def validate_spaces( ): return {} - @DeveloperAPI @OverrideToImplementCustomLogic @override(Policy) def loss( @@ -175,7 +171,6 @@ def loss( """ raise NotImplementedError - @DeveloperAPI @OverrideToImplementCustomLogic def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """Stats function. Returns a dict of statistics. @@ -188,7 +183,6 @@ def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic def grad_stats_fn( self, train_batch: SampleBatch, grads: ModelGradients @@ -203,7 +197,6 @@ def grad_stats_fn( """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic def make_model(self) -> ModelV2: """Build underlying model for this Policy. @@ -223,7 +216,6 @@ def make_model(self) -> ModelV2: framework="tf", ) - @DeveloperAPI @OverrideToImplementCustomLogic def compute_gradients_fn( self, optimizer: LocalOptimizer, loss: TensorType @@ -244,7 +236,6 @@ def compute_gradients_fn( """ return None - @DeveloperAPI @OverrideToImplementCustomLogic def apply_gradients_fn( self, @@ -263,7 +254,6 @@ def apply_gradients_fn( """ return None - @DeveloperAPI @OverrideToImplementCustomLogic def action_sampler_fn( self, @@ -288,7 +278,6 @@ def action_sampler_fn( """ return None, None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def action_distribution_fn( self, @@ -312,7 +301,6 @@ def action_distribution_fn( """ return None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def get_batch_divisibility_req(self) -> int: """Get batch divisibility request. @@ -324,7 +312,6 @@ def get_batch_divisibility_req(self) -> int: return 1 @override(TFPolicy) - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_action_out_fn(self) -> Dict[str, TensorType]: """Extra values to fetch and return from compute_actions(). @@ -337,7 +324,6 @@ def extra_action_out_fn(self) -> Dict[str, TensorType]: extra_action_fetches.update(self._policy_extra_action_fetches) return extra_action_fetches - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_learn_fetches_fn(self) -> Dict[str, TensorType]: """Extra stats to be reported after gradient computation. @@ -880,7 +866,6 @@ def _do_loss_init(self, train_batch: SampleBatch): return losses @override(TFPolicy) - @DeveloperAPI def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> TFPolicy: """Creates a copy of self using existing input placeholders.""" @@ -953,7 +938,6 @@ def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> TFPolicy return instance @override(Policy) - @DeveloperAPI def get_initial_state(self) -> List[TensorType]: if self.model: return self.model.get_initial_state() @@ -961,7 +945,6 @@ def get_initial_state(self) -> List[TensorType]: return [] @override(Policy) - @DeveloperAPI def load_batch_into_buffer( self, batch: SampleBatch, @@ -994,7 +977,6 @@ def load_batch_into_buffer( ) @override(Policy) - @DeveloperAPI def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: # Shortcut for 1 CPU only: Batch should already be stored in # `self._loaded_single_cpu_batch`. @@ -1009,7 +991,6 @@ def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: return self.multi_gpu_tower_stacks[buffer_index].num_tuples_loaded @override(Policy) - @DeveloperAPI def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): # Shortcut for 1 CPU only: Batch should already be stored in # `self._loaded_single_cpu_batch`. diff --git a/rllib/policy/eager_tf_policy.py b/rllib/policy/eager_tf_policy.py index c0eda028a4b..94f71653035 100644 --- a/rllib/policy/eager_tf_policy.py +++ b/rllib/policy/eager_tf_policy.py @@ -17,7 +17,7 @@ from ray.rllib.policy.rnn_sequencing import pad_batch_to_sequences_of_same_size from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import add_mixins, force_list -from ray.rllib.utils.annotations import override, DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.deprecation import ( DEPRECATED_VALUE, deprecation_warning, @@ -144,7 +144,7 @@ def _func(self_, *args, **kwargs): return _func -@DeveloperAPI +@OldAPIStack class EagerTFPolicy(Policy): """Dummy class to recognize any eagerized TFPolicy by its inheritance.""" @@ -301,6 +301,7 @@ def compute_gradients(self, loss, var_list): return list(zip(self.tape.gradient(loss, var_list), var_list)) +@OldAPIStack def _build_eager_tf_policy( name, loss_fn, diff --git a/rllib/policy/eager_tf_policy_v2.py b/rllib/policy/eager_tf_policy_v2.py index 5f67676e107..b3e357d539a 100644 --- a/rllib/policy/eager_tf_policy_v2.py +++ b/rllib/policy/eager_tf_policy_v2.py @@ -29,11 +29,10 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import force_list from ray.rllib.utils.annotations import ( - DeveloperAPI, + is_overridden, + OldAPIStack, OverrideToImplementCustomLogic, OverrideToImplementCustomLogic_CallToSuperRecommended, - is_overridden, - ExperimentalAPI, override, ) from ray.rllib.utils.error import ERR_MSG_TF_POLICY_CANNOT_SAVE_KERAS_MODEL @@ -60,7 +59,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class EagerTFPolicyV2(Policy): """A TF-eager / TF2 based tensorflow policy. @@ -149,7 +148,6 @@ def __init__( # traced function after that. self._re_trace_counter = 0 - @DeveloperAPI @staticmethod def enable_eager_execution_if_necessary(): # If this class runs as a @ray.remote actor, eager mode may not @@ -157,7 +155,6 @@ def enable_eager_execution_if_necessary(): if tf1 and not tf1.executing_eagerly(): tf1.enable_eager_execution() - @ExperimentalAPI @override(Policy) def maybe_remove_time_dimension(self, input_dict: Dict[str, TensorType]): assert self.config.get( @@ -189,7 +186,6 @@ def fold_mapping(item): else: return input_dict - @DeveloperAPI @OverrideToImplementCustomLogic def validate_spaces( self, @@ -199,7 +195,6 @@ def validate_spaces( ): return {} - @DeveloperAPI @OverrideToImplementCustomLogic @override(Policy) def loss( @@ -233,7 +228,6 @@ def loss( else: raise NotImplementedError - @DeveloperAPI @OverrideToImplementCustomLogic def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """Stats function. Returns a dict of statistics. @@ -246,7 +240,6 @@ def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic def grad_stats_fn( self, train_batch: SampleBatch, grads: ModelGradients @@ -261,7 +254,6 @@ def grad_stats_fn( """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic def make_model(self) -> ModelV2: """Build underlying model for this Policy. @@ -281,7 +273,6 @@ def make_model(self) -> ModelV2: framework=self.framework, ) - @DeveloperAPI @OverrideToImplementCustomLogic def compute_gradients_fn( self, policy: Policy, optimizer: LocalOptimizer, loss: TensorType @@ -302,7 +293,6 @@ def compute_gradients_fn( """ return None - @DeveloperAPI @OverrideToImplementCustomLogic def apply_gradients_fn( self, @@ -321,7 +311,6 @@ def apply_gradients_fn( """ return None - @DeveloperAPI @OverrideToImplementCustomLogic def action_sampler_fn( self, @@ -346,7 +335,6 @@ def action_sampler_fn( """ return None, None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def action_distribution_fn( self, @@ -370,7 +358,6 @@ def action_distribution_fn( """ return None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def get_batch_divisibility_req(self) -> int: """Get batch divisibility request. @@ -381,7 +368,6 @@ def get_batch_divisibility_req(self) -> int: # By default, any sized batch is ok, so simply return 1. return 1 - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_action_out_fn(self) -> Dict[str, TensorType]: """Extra values to fetch and return from compute_actions(). @@ -392,7 +378,6 @@ def extra_action_out_fn(self) -> Dict[str, TensorType]: """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_learn_fetches_fn(self) -> Dict[str, TensorType]: """Extra stats to be reported after gradient computation. diff --git a/rllib/policy/policy.py b/rllib/policy/policy.py index fd7e22e94a5..38738094e0e 100644 --- a/rllib/policy/policy.py +++ b/rllib/policy/policy.py @@ -35,8 +35,7 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.view_requirement import ViewRequirement from ray.rllib.utils.annotations import ( - DeveloperAPI, - ExperimentalAPI, + OldAPIStack, OverrideToImplementCustomLogic, OverrideToImplementCustomLogic_CallToSuperRecommended, is_overridden, @@ -47,7 +46,6 @@ try_import_msgpack, ) from ray.rllib.utils.deprecation import ( - Deprecated, DEPRECATED_VALUE, deprecation_warning, ) @@ -78,7 +76,6 @@ TensorStructType, TensorType, ) -from ray.util.annotations import PublicAPI tf1, tf, tfv = try_import_tf() torch, _ = try_import_torch() @@ -91,7 +88,7 @@ logger = logging.getLogger(__name__) -@PublicAPI +@OldAPIStack class PolicySpec: """A policy spec used in the "config.multiagent.policies" specification dict. @@ -173,7 +170,7 @@ def deserialize(cls, spec: Dict) -> "PolicySpec": ) -@DeveloperAPI +@OldAPIStack class Policy(metaclass=ABCMeta): """RLlib's base class for all Policy implementations. @@ -198,7 +195,6 @@ class Policy(metaclass=ABCMeta): (`load_batch_into_buffer` + `learn_on_loaded_batch`). """ - @DeveloperAPI def __init__( self, observation_space: gym.Space, @@ -400,7 +396,6 @@ def from_state(state: PolicyState) -> "Policy": # Return the new policy. return new_policy - @ExperimentalAPI @OverrideToImplementCustomLogic def make_rl_module(self) -> "RLModule": """Returns the RL Module (only for when RLModule API is enabled.) @@ -434,7 +429,6 @@ def make_rl_module(self) -> "RLModule": return module - @DeveloperAPI def init_view_requirements(self): """Maximal view requirements dict for `learn_on_batch()` and `compute_actions` calls. @@ -472,7 +466,6 @@ def reset_connectors(self, env_id) -> None: self.agent_connectors.reset(env_id=env_id) self.action_connectors.reset(env_id=env_id) - @DeveloperAPI def compute_single_action( self, obs: Optional[TensorStructType] = None, @@ -583,7 +576,6 @@ def compute_single_action( tree.map_structure(lambda x: x[0], info), ) - @DeveloperAPI def compute_actions_from_input_dict( self, input_dict: Union[SampleBatch, Dict[str, TensorStructType]], @@ -639,7 +631,6 @@ def compute_actions_from_input_dict( ) @abstractmethod - @DeveloperAPI def compute_actions( self, obs_batch: Union[List[TensorStructType], TensorStructType], @@ -683,7 +674,6 @@ def compute_actions( """ raise NotImplementedError - @DeveloperAPI def compute_log_likelihoods( self, actions: Union[List[TensorType], TensorType], @@ -718,7 +708,6 @@ def compute_log_likelihoods( """ raise NotImplementedError - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def postprocess_trajectory( self, @@ -752,7 +741,6 @@ def postprocess_trajectory( # The default implementation just returns the same, unaltered batch. return sample_batch - @ExperimentalAPI @OverrideToImplementCustomLogic def loss( self, model: ModelV2, dist_class: ActionDistribution, train_batch: SampleBatch @@ -772,7 +760,6 @@ def loss( """ raise NotImplementedError - @DeveloperAPI def learn_on_batch(self, samples: SampleBatch) -> Dict[str, TensorType]: """Perform one learning update, given `samples`. @@ -797,7 +784,6 @@ def learn_on_batch(self, samples: SampleBatch) -> Dict[str, TensorType]: self.apply_gradients(grads) return grad_info - @ExperimentalAPI def learn_on_batch_from_replay_buffer( self, replay_actor: ActorHandle, policy_id: PolicyID ) -> Dict[str, TensorType]: @@ -827,7 +813,6 @@ def learn_on_batch_from_replay_buffer( else: return self.learn_on_batch(batch) - @DeveloperAPI def load_batch_into_buffer(self, batch: SampleBatch, buffer_index: int = 0) -> int: """Bulk-loads the given SampleBatch into the devices' memories. @@ -846,7 +831,6 @@ def load_batch_into_buffer(self, batch: SampleBatch, buffer_index: int = 0) -> i """ raise NotImplementedError - @DeveloperAPI def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: """Returns the number of currently loaded samples in the given buffer. @@ -861,7 +845,6 @@ def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: """ raise NotImplementedError - @DeveloperAPI def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): """Runs a single step of SGD on an already loaded data in a buffer. @@ -885,7 +868,6 @@ def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): """ raise NotImplementedError - @DeveloperAPI def compute_gradients( self, postprocessed_batch: SampleBatch ) -> Tuple[ModelGradients, Dict[str, TensorType]]: @@ -904,7 +886,6 @@ def compute_gradients( """ raise NotImplementedError - @DeveloperAPI def apply_gradients(self, gradients: ModelGradients) -> None: """Applies the (previously) computed gradients. @@ -917,7 +898,6 @@ def apply_gradients(self, gradients: ModelGradients) -> None: """ raise NotImplementedError - @DeveloperAPI def get_weights(self) -> ModelWeights: """Returns model weights. @@ -932,7 +912,6 @@ def get_weights(self) -> ModelWeights: """ raise NotImplementedError - @DeveloperAPI def set_weights(self, weights: ModelWeights) -> None: """Sets this Policy's model's weights. @@ -945,7 +924,6 @@ def set_weights(self, weights: ModelWeights) -> None: """ raise NotImplementedError - @DeveloperAPI def get_exploration_state(self) -> Dict[str, TensorType]: """Returns the state of this Policy's exploration component. @@ -954,7 +932,6 @@ def get_exploration_state(self) -> Dict[str, TensorType]: """ return self.exploration.get_state() - @DeveloperAPI def is_recurrent(self) -> bool: """Whether this Policy holds a recurrent Model. @@ -963,7 +940,6 @@ def is_recurrent(self) -> bool: """ return False - @DeveloperAPI def num_state_tensors(self) -> int: """The number of internal states needed by the RNN-Model of the Policy. @@ -972,7 +948,6 @@ def num_state_tensors(self) -> int: """ return 0 - @DeveloperAPI def get_initial_state(self) -> List[TensorType]: """Returns initial RNN state for the current policy. @@ -981,7 +956,6 @@ def get_initial_state(self) -> List[TensorType]: """ return [] - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def get_state(self) -> PolicyState: """Returns the entire current state of this Policy. @@ -1027,7 +1001,6 @@ def get_state(self) -> PolicyState: return state - @PublicAPI(stability="alpha") def restore_connectors(self, state: PolicyState): """Restore agent and action connectors if configs available. @@ -1056,7 +1029,6 @@ def restore_connectors(self, state: PolicyState): logger.debug("restoring action connectors:") logger.debug(self.action_connectors.__str__(indentation=4)) - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def set_state(self, state: PolicyState) -> None: """Restores the entire current state of this Policy from `state`. @@ -1094,7 +1066,6 @@ def set_state(self, state: PolicyState) -> None: self.set_weights(state["weights"]) self.restore_connectors(state) - @ExperimentalAPI def apply( self, func: Callable[["Policy", Optional[Any], Optional[Any]], T], @@ -1118,7 +1089,6 @@ def apply( """ return func(self, *args, **kwargs) - @DeveloperAPI def on_global_var_update(self, global_vars: Dict[str, TensorType]) -> None: """Called on an update to global vars. @@ -1139,7 +1109,6 @@ def on_global_var_update(self, global_vars: Dict[str, TensorType]) -> None: if num_grad_updates is not None: self.num_grad_updates = num_grad_updates - @DeveloperAPI def export_checkpoint( self, export_dir: str, @@ -1219,7 +1188,6 @@ def export_checkpoint( if self.config["export_native_model_files"]: self.export_model(os.path.join(export_dir, "model")) - @DeveloperAPI def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: """Exports the Policy's Model to local directory for serving. @@ -1238,7 +1206,6 @@ def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: """ raise NotImplementedError - @DeveloperAPI def import_model_from_h5(self, import_file: str) -> None: """Imports Policy from local file. @@ -1247,7 +1214,6 @@ def import_model_from_h5(self, import_file: str) -> None: """ raise NotImplementedError - @DeveloperAPI def get_session(self) -> Optional["tf1.Session"]: """Returns tf.Session object to use for computing actions or None. @@ -1621,7 +1587,6 @@ def _initialize_loss_from_dummy_batch( "but is of type {}.".format(self, type(self.global_timestep)) ) - @ExperimentalAPI def maybe_add_time_dimension( self, input_dict: Dict[str, TensorType], @@ -1704,7 +1669,6 @@ def _add_state_out_time_dimension(inputs): else: return input_dict - @ExperimentalAPI def maybe_remove_time_dimension(self, input_dict: Dict[str, TensorType]): """Removes a time dimension for recurrent RLModules. @@ -1841,16 +1805,11 @@ def _update_model_view_requirements_from_init_state(self): space=space, used_for_training=True ) - @DeveloperAPI def __repr__(self): return type(self).__name__ - @Deprecated(new="get_exploration_state", error=True) - def get_exploration_info(self) -> Dict[str, TensorType]: - return self.get_exploration_state() - -@DeveloperAPI +@OldAPIStack def get_gym_space_from_struct_of_tensors( value: Union[Mapping, Tuple, List, TensorType], batched_input=True, @@ -1866,7 +1825,7 @@ def get_gym_space_from_struct_of_tensors( return space -@DeveloperAPI +@OldAPIStack def get_gym_space_from_struct_of_spaces(value: Union[Dict, Tuple]) -> gym.spaces.Dict: if isinstance(value, Mapping): return gym.spaces.Dict( diff --git a/rllib/policy/policy_map.py b/rllib/policy/policy_map.py index 638bac0f26a..b14b2a27056 100644 --- a/rllib/policy/policy_map.py +++ b/rllib/policy/policy_map.py @@ -5,18 +5,17 @@ import ray from ray.rllib.policy.policy import Policy -from ray.rllib.utils.annotations import override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.threading import with_lock from ray.rllib.utils.typing import PolicyID -from ray.util.annotations import PublicAPI tf1, tf, tfv = try_import_tf() logger = logging.getLogger(__name__) -@PublicAPI(stability="beta") +@OldAPIStack class PolicyMap(dict): """Maps policy IDs to Policy objects. diff --git a/rllib/policy/policy_template.py b/rllib/policy/policy_template.py index 135c8b60418..e2fb1a1a3c2 100644 --- a/rllib/policy/policy_template.py +++ b/rllib/policy/policy_template.py @@ -20,7 +20,7 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.torch_policy import TorchPolicy from ray.rllib.utils import add_mixins, NullContextManager -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch, try_import_jax from ray.rllib.utils.metrics.learner_info import LEARNER_STATS_KEY from ray.rllib.utils.numpy import convert_to_numpy @@ -33,7 +33,7 @@ torch, _ = try_import_torch() -@DeveloperAPI +@OldAPIStack def build_policy_class( name: str, framework: str, diff --git a/rllib/policy/rnn_sequencing.py b/rllib/policy/rnn_sequencing.py index ca7172f4472..0f852261402 100644 --- a/rllib/policy/rnn_sequencing.py +++ b/rllib/policy/rnn_sequencing.py @@ -18,7 +18,7 @@ import functools from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.debug import summarize from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.typing import TensorType, ViewRequirementsDict @@ -31,7 +31,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack def pad_batch_to_sequences_of_same_size( batch: SampleBatch, max_seq_len: int, @@ -190,7 +190,7 @@ def pad_batch_to_sequences_of_same_size( ) -@DeveloperAPI +@OldAPIStack def add_time_dimension( padded_inputs: TensorType, *, @@ -264,7 +264,7 @@ def add_time_dimension( return padded_outputs -@DeveloperAPI +@OldAPIStack def chop_into_sequences( *, feature_columns, @@ -452,7 +452,7 @@ def chop_into_sequences( return feature_sequences, initial_states, seq_lens -@DeveloperAPI +@OldAPIStack def timeslice_along_seq_lens_with_overlap( sample_batch: SampleBatchType, seq_lens: Optional[List[int]] = None, @@ -601,7 +601,7 @@ def timeslice_along_seq_lens_with_overlap( return timeslices -@DeveloperAPI +@OldAPIStack def get_fold_unfold_fns(b_dim: int, t_dim: int, framework: str): """Produces two functions to fold/unfold any Tensors in a struct. diff --git a/rllib/policy/tf_mixins.py b/rllib/policy/tf_mixins.py index 385dbc0e386..9a1869a4b76 100644 --- a/rllib/policy/tf_mixins.py +++ b/rllib/policy/tf_mixins.py @@ -10,7 +10,7 @@ from ray.rllib.policy.policy import Policy, PolicyState from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.tf_policy import TFPolicy -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import get_variable, try_import_tf from ray.rllib.utils.schedules import PiecewiseSchedule from ray.rllib.utils.tf_utils import make_tf_callable @@ -26,7 +26,7 @@ tf1, tf, tfv = try_import_tf() -@DeveloperAPI +@OldAPIStack class LearningRateSchedule: """Mixin for TFPolicy that adds a learning rate schedule.""" @@ -72,7 +72,7 @@ def optimizer(self): return tf.keras.optimizers.Adam(self.cur_lr) -@DeveloperAPI +@OldAPIStack class EntropyCoeffSchedule: """Mixin for TFPolicy that adds entropy coeff decay.""" @@ -130,7 +130,7 @@ def on_global_var_update(self, global_vars): self.entropy_coeff.assign(new_val, read_value=False) -@DeveloperAPI +@OldAPIStack class KLCoeffMixin: """Assigns the `update_kl()` and other KL-related methods to a TFPolicy. @@ -205,7 +205,7 @@ def set_state(self, state: PolicyState) -> None: super().set_state(state) -@DeveloperAPI +@OldAPIStack class TargetNetworkMixin: """Assign the `update_target` method to the policy. @@ -281,7 +281,7 @@ def set_weights(self, weights): self.update_target(self.config.get("tau", 1.0)) -@DeveloperAPI +@OldAPIStack class ValueNetworkMixin: """Assigns the `_value()` method to a TFPolicy. @@ -362,7 +362,7 @@ def extra_action_out_fn(self) -> Dict[str, TensorType]: return self._cached_extra_action_fetches -@DeveloperAPI +@OldAPIStack class GradStatsMixin: def __init__(self): pass diff --git a/rllib/policy/tf_policy.py b/rllib/policy/tf_policy.py index 0025ac9db89..cceca81dd5d 100644 --- a/rllib/policy/tf_policy.py +++ b/rllib/policy/tf_policy.py @@ -13,7 +13,7 @@ from ray.rllib.policy.rnn_sequencing import pad_batch_to_sequences_of_same_size from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import force_list -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.debug import summarize from ray.rllib.utils.deprecation import Deprecated from ray.rllib.utils.error import ERR_MSG_TF_POLICY_CANNOT_SAVE_KERAS_MODEL @@ -42,7 +42,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class TFPolicy(Policy): """An agent policy and loss implemented in TensorFlow. @@ -89,7 +89,6 @@ def next_tf_var_scope_name(): TFPolicy.tf_var_creation_scope_counter += 1 return f"var_scope_{TFPolicy.tf_var_creation_scope_counter}" - @DeveloperAPI def __init__( self, observation_space: gym.spaces.Space, @@ -437,7 +436,6 @@ def compute_log_likelihoods( return builder.get(fetches)[0] @override(Policy) - @DeveloperAPI def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorType]: assert self.loss_initialized() @@ -473,7 +471,6 @@ def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorTy return stats @override(Policy) - @DeveloperAPI def compute_gradients( self, postprocessed_batch: SampleBatch ) -> Tuple[ModelGradients, Dict[str, TensorType]]: @@ -525,7 +522,6 @@ def _tf1_from_state_helper(state: PolicyState) -> "Policy": return new_policy @override(Policy) - @DeveloperAPI def apply_gradients(self, gradients: ModelGradients) -> None: assert self.loss_initialized() builder = _TFRunBuilder(self.get_session(), "apply_gradients") @@ -533,17 +529,14 @@ def apply_gradients(self, gradients: ModelGradients) -> None: builder.get(fetches) @override(Policy) - @DeveloperAPI def get_weights(self) -> Union[Dict[str, TensorType], List[TensorType]]: return self._variables.get_weights() @override(Policy) - @DeveloperAPI def set_weights(self, weights) -> None: return self._variables.set_weights(weights) @override(Policy) - @DeveloperAPI def get_exploration_state(self) -> Dict[str, TensorType]: return self.exploration.get_state(sess=self.get_session()) @@ -552,17 +545,14 @@ def get_exploration_info(self) -> Dict[str, TensorType]: return self.get_exploration_state() @override(Policy) - @DeveloperAPI def is_recurrent(self) -> bool: return len(self._state_inputs) > 0 @override(Policy) - @DeveloperAPI def num_state_tensors(self) -> int: return len(self._state_inputs) @override(Policy) - @DeveloperAPI def get_state(self) -> PolicyState: # For tf Policies, return Policy weights and optimizer var values. state = super().get_state() @@ -576,7 +566,6 @@ def get_state(self) -> PolicyState: return state @override(Policy) - @DeveloperAPI def set_state(self, state: PolicyState) -> None: # Set optimizer vars first. optimizer_vars = state.get("_optimizer_variables", None) @@ -595,7 +584,6 @@ def set_state(self, state: PolicyState) -> None: super().set_state(state) @override(Policy) - @DeveloperAPI def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: """Export tensorflow graph to export_dir for serving.""" if onnx: @@ -653,7 +641,6 @@ def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: logger.warning(ERR_MSG_TF_POLICY_CANNOT_SAVE_KERAS_MODEL) @override(Policy) - @DeveloperAPI def import_model_from_h5(self, import_file: str) -> None: """Imports weights into tf model.""" if self.model is None: @@ -798,7 +785,6 @@ def _initialize_loss( [v for o in self._optimizers for v in o.variables()], self.get_session() ) - @DeveloperAPI def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> "TFPolicy": """Creates a copy of self using existing input placeholders. @@ -814,7 +800,6 @@ def copy(self, existing_inputs: List[Tuple[str, "tf1.placeholder"]]) -> "TFPolic """ raise NotImplementedError - @DeveloperAPI def extra_compute_action_feed_dict(self) -> Dict[TensorType, TensorType]: """Extra dict to pass to the compute actions session run. @@ -824,7 +809,6 @@ def extra_compute_action_feed_dict(self) -> Dict[TensorType, TensorType]: """ return {} - @DeveloperAPI def extra_compute_action_fetches(self) -> Dict[str, TensorType]: # Cache graph fetches for action computation for better # performance. @@ -834,7 +818,6 @@ def extra_compute_action_fetches(self) -> Dict[str, TensorType]: self._cached_extra_action_out = self.extra_action_out_fn() return self._cached_extra_action_out - @DeveloperAPI def extra_action_out_fn(self) -> Dict[str, TensorType]: """Extra values to fetch and return from compute_actions(). @@ -855,7 +838,6 @@ def extra_action_out_fn(self) -> Dict[str, TensorType]: extra_fetches[SampleBatch.ACTION_DIST_INPUTS] = self._dist_inputs return extra_fetches - @DeveloperAPI def extra_compute_grad_feed_dict(self) -> Dict[TensorType, TensorType]: """Extra dict to pass to the compute gradients session run. @@ -865,7 +847,6 @@ def extra_compute_grad_feed_dict(self) -> Dict[TensorType, TensorType]: """ return {} # e.g, kl_coeff - @DeveloperAPI def extra_compute_grad_fetches(self) -> Dict[str, any]: """Extra values to fetch and return from compute_gradients(). @@ -875,7 +856,6 @@ def extra_compute_grad_fetches(self) -> Dict[str, any]: """ return {LEARNER_STATS_KEY: {}} # e.g, stats, td error, etc. - @DeveloperAPI def optimizer(self) -> "tf.keras.optimizers.Optimizer": """TF optimizer to use for policy optimization. @@ -888,7 +868,6 @@ def optimizer(self) -> "tf.keras.optimizers.Optimizer": else: return tf1.train.AdamOptimizer() - @DeveloperAPI def gradients( self, optimizer: Union[LocalOptimizer, List[LocalOptimizer]], @@ -924,7 +903,6 @@ def gradients( else: return optimizers[0].compute_gradients(losses[0]) - @DeveloperAPI def build_apply_op( self, optimizer: Union[LocalOptimizer, List[LocalOptimizer]], diff --git a/rllib/policy/tf_policy_template.py b/rllib/policy/tf_policy_template.py index c84f223b03d..83b132de5ce 100644 --- a/rllib/policy/tf_policy_template.py +++ b/rllib/policy/tf_policy_template.py @@ -9,7 +9,7 @@ from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.tf_policy import TFPolicy from ray.rllib.utils import add_mixins, force_list -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.deprecation import ( deprecation_warning, DEPRECATED_VALUE, @@ -29,7 +29,7 @@ tf1, tf, tfv = try_import_tf() -@DeveloperAPI +@OldAPIStack def build_tf_policy( name: str, *, diff --git a/rllib/policy/torch_mixins.py b/rllib/policy/torch_mixins.py index 36f8d18ac70..a8cb18a9cf3 100644 --- a/rllib/policy/torch_mixins.py +++ b/rllib/policy/torch_mixins.py @@ -1,14 +1,14 @@ from ray.rllib.policy.policy import Policy, PolicyState from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.policy.torch_policy import TorchPolicy -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.schedules import PiecewiseSchedule torch, nn = try_import_torch() -@DeveloperAPI +@OldAPIStack class LearningRateSchedule: """Mixin for TorchPolicy that adds a learning rate schedule.""" @@ -34,7 +34,7 @@ def on_global_var_update(self, global_vars): p["lr"] = self.cur_lr -@DeveloperAPI +@OldAPIStack class EntropyCoeffSchedule: """Mixin for TorchPolicy that adds entropy coeff decay.""" @@ -72,7 +72,7 @@ def on_global_var_update(self, global_vars): ) -@DeveloperAPI +@OldAPIStack class KLCoeffMixin: """Assigns the `update_kl()` method to a TorchPolicy. @@ -111,7 +111,7 @@ def set_state(self, state: PolicyState) -> None: super().set_state(state) -@DeveloperAPI +@OldAPIStack class ValueNetworkMixin: """Assigns the `_value()` method to a TorchPolicy. @@ -171,7 +171,7 @@ def extra_action_out(self, input_dict, state_batches, model, action_dist): } -@DeveloperAPI +@OldAPIStack class TargetNetworkMixin: """Mixin class adding a method for (soft) target net(s) synchronizations. diff --git a/rllib/policy/torch_policy.py b/rllib/policy/torch_policy.py index 88cd1a73807..31bce76119a 100644 --- a/rllib/policy/torch_policy.py +++ b/rllib/policy/torch_policy.py @@ -31,7 +31,7 @@ from ray.rllib.policy.rnn_sequencing import pad_batch_to_sequences_of_same_size from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import NullContextManager, force_list -from ray.rllib.utils.annotations import DeveloperAPI, override +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.error import ERR_MSG_TORCH_POLICY_CANNOT_SAVE_MODEL from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.metrics import ( @@ -61,11 +61,10 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class TorchPolicy(Policy): """PyTorch specific Policy class to use with RLlib.""" - @DeveloperAPI def __init__( self, observation_space: gym.spaces.Space, @@ -326,7 +325,6 @@ def compute_actions_from_input_dict( ) @override(Policy) - @DeveloperAPI def compute_actions( self, obs_batch: Union[List[TensorStructType], TensorStructType], @@ -360,7 +358,6 @@ def compute_actions( @with_lock @override(Policy) - @DeveloperAPI def compute_log_likelihoods( self, actions: Union[List[TensorStructType], TensorStructType], @@ -447,7 +444,6 @@ def compute_log_likelihoods( @with_lock @override(Policy) - @DeveloperAPI def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorType]: # Set Model to train mode. if self.model: @@ -487,7 +483,6 @@ def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorTy return fetches @override(Policy) - @DeveloperAPI def load_batch_into_buffer( self, batch: SampleBatch, @@ -538,14 +533,12 @@ def load_batch_into_buffer( return len(slices[0]) @override(Policy) - @DeveloperAPI def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: if len(self.devices) == 1 and self.devices[0] == "/cpu:0": assert buffer_index == 0 return sum(len(b) for b in self._loaded_batches[buffer_index]) @override(Policy) - @DeveloperAPI def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): if not self._loaded_batches[buffer_index]: raise ValueError( @@ -641,7 +634,6 @@ def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): @with_lock @override(Policy) - @DeveloperAPI def compute_gradients(self, postprocessed_batch: SampleBatch) -> ModelGradients: assert len(self.devices) == 1 @@ -671,7 +663,6 @@ def compute_gradients(self, postprocessed_batch: SampleBatch) -> ModelGradients: return all_grads, dict(fetches, **{LEARNER_STATS_KEY: grad_info}) @override(Policy) - @DeveloperAPI def apply_gradients(self, gradients: ModelGradients) -> None: if gradients == _directStepOptimizerSingleton: for i, opt in enumerate(self._optimizers): @@ -688,7 +679,6 @@ def apply_gradients(self, gradients: ModelGradients) -> None: self._optimizers[0].step() - @DeveloperAPI def get_tower_stats(self, stats_name: str) -> List[TensorStructType]: """Returns list of per-tower stats, copied to this Policy's device. @@ -720,33 +710,27 @@ def get_tower_stats(self, stats_name: str) -> List[TensorStructType]: return data @override(Policy) - @DeveloperAPI def get_weights(self) -> ModelWeights: return {k: v.cpu().detach().numpy() for k, v in self.model.state_dict().items()} @override(Policy) - @DeveloperAPI def set_weights(self, weights: ModelWeights) -> None: weights = convert_to_torch_tensor(weights, device=self.device) self.model.load_state_dict(weights) @override(Policy) - @DeveloperAPI def is_recurrent(self) -> bool: return self._is_recurrent @override(Policy) - @DeveloperAPI def num_state_tensors(self) -> int: return len(self.model.get_initial_state()) @override(Policy) - @DeveloperAPI def get_initial_state(self) -> List[TensorType]: return [s.detach().cpu().numpy() for s in self.model.get_initial_state()] @override(Policy) - @DeveloperAPI def get_state(self) -> PolicyState: state = super().get_state() @@ -762,7 +746,6 @@ def get_state(self) -> PolicyState: return state @override(Policy) - @DeveloperAPI def set_state(self, state: PolicyState) -> None: # Set optimizer vars first. optimizer_vars = state.get("_optimizer_variables", None) @@ -787,7 +770,6 @@ def set_state(self, state: PolicyState) -> None: # Then the Policy's (NN) weights and connectors. super().set_state(state) - @DeveloperAPI def extra_grad_process( self, optimizer: "torch.optim.Optimizer", loss: TensorType ) -> Dict[str, TensorType]: @@ -806,7 +788,6 @@ def extra_grad_process( """ return {} - @DeveloperAPI def extra_compute_grad_fetches(self) -> Dict[str, Any]: """Extra values to fetch and return from compute_gradients(). @@ -816,7 +797,6 @@ def extra_compute_grad_fetches(self) -> Dict[str, Any]: """ return {LEARNER_STATS_KEY: {}} # e.g, stats, td error, etc. - @DeveloperAPI def extra_action_out( self, input_dict: Dict[str, TensorType], @@ -839,7 +819,6 @@ def extra_action_out( """ return {} - @DeveloperAPI def extra_grad_info(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """Return dict of extra grad info. @@ -852,7 +831,6 @@ def extra_grad_info(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """ return {} - @DeveloperAPI def optimizer( self, ) -> Union[List["torch.optim.Optimizer"], "torch.optim.Optimizer"]: @@ -872,7 +850,6 @@ def optimizer( return optimizers @override(Policy) - @DeveloperAPI def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: """Exports the Policy's Model to local directory for serving. @@ -935,7 +912,6 @@ def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: logger.warning(ERR_MSG_TORCH_POLICY_CANNOT_SAVE_MODEL) @override(Policy) - @DeveloperAPI def import_model_from_h5(self, import_file: str) -> None: """Imports weights into torch model.""" return self.model.import_from_h5(import_file) @@ -1202,7 +1178,7 @@ def _worker(shard_idx, model, sample_batch, device): return outputs -@DeveloperAPI +@OldAPIStack class DirectStepOptimizer: """Typesafe method for indicating `apply_gradients` can directly step the optimizers with in-place gradients. diff --git a/rllib/policy/torch_policy_template.py b/rllib/policy/torch_policy_template.py deleted file mode 100644 index 823765a244d..00000000000 --- a/rllib/policy/torch_policy_template.py +++ /dev/null @@ -1,6 +0,0 @@ -from ray.rllib.utils.deprecation import Deprecated - - -@Deprecated(new="build_policy_class(framework='torch')", error=True) -def build_torch_policy(*args, **kwargs): - pass diff --git a/rllib/policy/torch_policy_v2.py b/rllib/policy/torch_policy_v2.py index 59e2d5bbaad..0150fd293cf 100644 --- a/rllib/policy/torch_policy_v2.py +++ b/rllib/policy/torch_policy_v2.py @@ -24,13 +24,12 @@ from ray.rllib.policy.torch_policy import _directStepOptimizerSingleton from ray.rllib.utils import NullContextManager, force_list from ray.rllib.utils.annotations import ( - DeveloperAPI, + OldAPIStack, OverrideToImplementCustomLogic, OverrideToImplementCustomLogic_CallToSuperRecommended, is_overridden, override, ) -from ray.rllib.utils.annotations import ExperimentalAPI from ray.rllib.utils.error import ERR_MSG_TORCH_POLICY_CANNOT_SAVE_MODEL from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.metrics import ( @@ -61,11 +60,10 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class TorchPolicyV2(Policy): """PyTorch specific Policy class to use with RLlib.""" - @DeveloperAPI def __init__( self, observation_space: gym.spaces.Space, @@ -234,7 +232,6 @@ def __init__( def loss_initialized(self): return self._loss_initialized - @DeveloperAPI @OverrideToImplementCustomLogic @override(Policy) def loss( @@ -268,7 +265,6 @@ def loss( else: raise NotImplementedError - @DeveloperAPI @OverrideToImplementCustomLogic def action_sampler_fn( self, @@ -293,7 +289,6 @@ def action_sampler_fn( """ return None, None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def action_distribution_fn( self, @@ -317,7 +312,6 @@ def action_distribution_fn( """ return None, None, None - @DeveloperAPI @OverrideToImplementCustomLogic def make_model(self) -> ModelV2: """Create model. @@ -330,7 +324,6 @@ def make_model(self) -> ModelV2: """ return None - @ExperimentalAPI @override(Policy) def maybe_remove_time_dimension(self, input_dict: Dict[str, TensorType]): assert self.config.get( @@ -360,7 +353,6 @@ def fold_mapping(item): else: return input_dict - @DeveloperAPI @OverrideToImplementCustomLogic def make_model_and_action_dist( self, @@ -373,7 +365,6 @@ def make_model_and_action_dist( """ return None, None - @DeveloperAPI @OverrideToImplementCustomLogic def get_batch_divisibility_req(self) -> int: """Get batch divisibility request. @@ -384,7 +375,6 @@ def get_batch_divisibility_req(self) -> int: # By default, any sized batch is ok, so simply return 1. return 1 - @DeveloperAPI @OverrideToImplementCustomLogic def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """Stats function. Returns a dict of statistics. @@ -397,7 +387,6 @@ def stats_fn(self, train_batch: SampleBatch) -> Dict[str, TensorType]: """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_grad_process( self, optimizer: "torch.optim.Optimizer", loss: TensorType @@ -417,7 +406,6 @@ def extra_grad_process( """ return {} - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_compute_grad_fetches(self) -> Dict[str, Any]: """Extra values to fetch and return from compute_gradients(). @@ -428,7 +416,6 @@ def extra_compute_grad_fetches(self) -> Dict[str, Any]: """ return {LEARNER_STATS_KEY: {}} # e.g, stats, td error, etc. - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def extra_action_out( self, @@ -453,7 +440,6 @@ def extra_action_out( return {} @override(Policy) - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def postprocess_trajectory( self, @@ -484,7 +470,6 @@ def postprocess_trajectory( """ return sample_batch - @DeveloperAPI @OverrideToImplementCustomLogic def optimizer( self, @@ -574,7 +559,6 @@ def compute_actions_from_input_dict( ) @override(Policy) - @DeveloperAPI def compute_actions( self, obs_batch: Union[List[TensorStructType], TensorStructType], @@ -609,7 +593,6 @@ def compute_actions( @with_lock @override(Policy) - @DeveloperAPI def compute_log_likelihoods( self, actions: Union[List[TensorStructType], TensorStructType], @@ -713,7 +696,6 @@ def compute_log_likelihoods( @with_lock @override(Policy) - @DeveloperAPI def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorType]: # Set Model to train mode. @@ -755,7 +737,6 @@ def learn_on_batch(self, postprocessed_batch: SampleBatch) -> Dict[str, TensorTy return fetches @override(Policy) - @DeveloperAPI def load_batch_into_buffer( self, batch: SampleBatch, @@ -814,14 +795,12 @@ def load_batch_into_buffer( return len(slices[0]) @override(Policy) - @DeveloperAPI def get_num_samples_loaded_into_buffer(self, buffer_index: int = 0) -> int: if len(self.devices) == 1 and self.devices[0] == "/cpu:0": assert buffer_index == 0 return sum(len(b) for b in self._loaded_batches[buffer_index]) @override(Policy) - @DeveloperAPI def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): if not self._loaded_batches[buffer_index]: raise ValueError( @@ -920,7 +899,6 @@ def learn_on_loaded_batch(self, offset: int = 0, buffer_index: int = 0): @with_lock @override(Policy) - @DeveloperAPI def compute_gradients(self, postprocessed_batch: SampleBatch) -> ModelGradients: assert len(self.devices) == 1 @@ -955,7 +933,6 @@ def compute_gradients(self, postprocessed_batch: SampleBatch) -> ModelGradients: return all_grads, dict(fetches, **{LEARNER_STATS_KEY: grad_info}) @override(Policy) - @DeveloperAPI def apply_gradients(self, gradients: ModelGradients) -> None: if gradients == _directStepOptimizerSingleton: for i, opt in enumerate(self._optimizers): @@ -972,7 +949,6 @@ def apply_gradients(self, gradients: ModelGradients) -> None: self._optimizers[0].step() - @DeveloperAPI def get_tower_stats(self, stats_name: str) -> List[TensorStructType]: """Returns list of per-tower stats, copied to this Policy's device. @@ -1010,12 +986,10 @@ def get_tower_stats(self, stats_name: str) -> List[TensorStructType]: return data @override(Policy) - @DeveloperAPI def get_weights(self) -> ModelWeights: return {k: v.cpu().detach().numpy() for k, v in self.model.state_dict().items()} @override(Policy) - @DeveloperAPI def set_weights(self, weights: ModelWeights) -> None: weights = convert_to_torch_tensor(weights, device=self.device) if self.config.get("_enable_new_api_stack", False): @@ -1024,17 +998,14 @@ def set_weights(self, weights: ModelWeights) -> None: self.model.load_state_dict(weights) @override(Policy) - @DeveloperAPI def is_recurrent(self) -> bool: return self._is_recurrent @override(Policy) - @DeveloperAPI def num_state_tensors(self) -> int: return len(self.model.get_initial_state()) @override(Policy) - @DeveloperAPI def get_initial_state(self) -> List[TensorType]: if self.config.get("_enable_new_api_stack", False): # convert the tree of tensors to a tree to numpy arrays @@ -1045,7 +1016,6 @@ def get_initial_state(self) -> List[TensorType]: return [s.detach().cpu().numpy() for s in self.model.get_initial_state()] @override(Policy) - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def get_state(self) -> PolicyState: # Legacy Policy state (w/o torch.nn.Module and w/o PolicySpec). @@ -1065,7 +1035,6 @@ def get_state(self) -> PolicyState: return state @override(Policy) - @DeveloperAPI @OverrideToImplementCustomLogic_CallToSuperRecommended def set_state(self, state: PolicyState) -> None: # Set optimizer vars first. @@ -1092,7 +1061,6 @@ def set_state(self, state: PolicyState) -> None: super().set_state(state) @override(Policy) - @DeveloperAPI def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: """Exports the Policy's Model to local directory for serving. @@ -1160,7 +1128,6 @@ def export_model(self, export_dir: str, onnx: Optional[int] = None) -> None: logger.warning(ERR_MSG_TORCH_POLICY_CANNOT_SAVE_MODEL) @override(Policy) - @DeveloperAPI def import_model_from_h5(self, import_file: str) -> None: """Imports weights into torch model.""" return self.model.import_from_h5(import_file) diff --git a/rllib/policy/view_requirement.py b/rllib/policy/view_requirement.py index 083a50f122e..ef360e3ddf3 100644 --- a/rllib/policy/view_requirement.py +++ b/rllib/policy/view_requirement.py @@ -3,7 +3,7 @@ from typing import Dict, List, Optional, Union import numpy as np -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.serialization import ( gym_space_to_dict, @@ -13,7 +13,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack @dataclasses.dataclass class ViewRequirement: """Single view requirement (for one column in an SampleBatch/input_dict). diff --git a/rllib/utils/annotations.py b/rllib/utils/annotations.py index 3b7a1f441a1..4f513f27284 100644 --- a/rllib/utils/annotations.py +++ b/rllib/utils/annotations.py @@ -117,6 +117,19 @@ def loss(self, model, action_dist, train_batch): return obj +def OldAPIStack(obj): + """Decorator for classes/methods/functions belonging to the old API stack. + + These should be deprecated at some point after Ray 3.0 (RLlib GA). + It is recommended for users to start exploring (and coding against) the new API + stack instead. + """ + # No effect yet. + + _mark_annotated(obj) + return obj + + def OverrideToImplementCustomLogic(obj): """Users should override this in their sub-classes to implement custom logic. diff --git a/rllib/utils/exploration/curiosity.py b/rllib/utils/exploration/curiosity.py index 2391e2e6683..2fa44c237ab 100644 --- a/rllib/utils/exploration/curiosity.py +++ b/rllib/utils/exploration/curiosity.py @@ -14,7 +14,7 @@ from ray.rllib.models.utils import get_activation_fn from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import NullContextManager -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.from_config import from_config @@ -29,7 +29,7 @@ F = nn.functional -@PublicAPI +@OldAPIStack class Curiosity(Exploration): """Implementation of: [1] Curiosity-driven Exploration by Self-supervised Prediction diff --git a/rllib/utils/exploration/epsilon_greedy.py b/rllib/utils/exploration/epsilon_greedy.py index 577b39f4a11..40a307bfbb3 100644 --- a/rllib/utils/exploration/epsilon_greedy.py +++ b/rllib/utils/exploration/epsilon_greedy.py @@ -6,7 +6,7 @@ from ray.rllib.models.torch.torch_action_dist import TorchMultiActionDistribution from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import override, OldAPIStack from ray.rllib.utils.exploration.exploration import Exploration, TensorType from ray.rllib.utils.framework import try_import_tf, try_import_torch, get_variable from ray.rllib.utils.from_config import from_config @@ -18,7 +18,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class EpsilonGreedy(Exploration): """Epsilon-greedy Exploration class that produces exploration actions. diff --git a/rllib/utils/exploration/exploration.py b/rllib/utils/exploration/exploration.py index 13f8870d962..9cbb494ef30 100644 --- a/rllib/utils/exploration/exploration.py +++ b/rllib/utils/exploration/exploration.py @@ -5,7 +5,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.modelv2 import ModelV2 from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import DeveloperAPI, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_torch, TensorType from ray.rllib.utils.typing import LocalOptimizer, AlgorithmConfigDict @@ -18,7 +18,7 @@ _, nn = try_import_torch() -@PublicAPI +@OldAPIStack class Exploration: """Implements an exploration strategy for Policies. @@ -60,7 +60,6 @@ def __init__( if params: self.device = params[0].device - @DeveloperAPI def before_compute_actions( self, *, @@ -82,7 +81,6 @@ def before_compute_actions( # fmt: off # __sphinx_doc_begin_get_exploration_action__ - @DeveloperAPI def get_exploration_action(self, *, action_distribution: ActionDistribution, @@ -113,7 +111,6 @@ def get_exploration_action(self, # __sphinx_doc_end_get_exploration_action__ # fmt: on - @DeveloperAPI def on_episode_start( self, policy: "Policy", @@ -132,7 +129,6 @@ def on_episode_start( """ pass - @DeveloperAPI def on_episode_end( self, policy: "Policy", @@ -151,7 +147,6 @@ def on_episode_end( """ pass - @DeveloperAPI def postprocess_trajectory( self, policy: "Policy", @@ -170,7 +165,6 @@ def postprocess_trajectory( """ return sample_batch - @DeveloperAPI def get_exploration_optimizer( self, optimizers: List[LocalOptimizer] ) -> List[LocalOptimizer]: @@ -189,7 +183,6 @@ def get_exploration_optimizer( """ return optimizers - @DeveloperAPI def get_state(self, sess: Optional["tf.Session"] = None) -> Dict[str, TensorType]: """Returns the current exploration state. @@ -201,7 +194,6 @@ def get_state(self, sess: Optional["tf.Session"] = None) -> Dict[str, TensorType """ return {} - @DeveloperAPI def set_state(self, state: object, sess: Optional["tf.Session"] = None) -> None: """Sets the Exploration object's state to the given values. diff --git a/rllib/utils/exploration/gaussian_noise.py b/rllib/utils/exploration/gaussian_noise.py index c0edc189535..385ac377d84 100644 --- a/rllib/utils/exploration/gaussian_noise.py +++ b/rllib/utils/exploration/gaussian_noise.py @@ -4,7 +4,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.exploration.random import Random from ray.rllib.utils.framework import ( @@ -22,7 +22,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class GaussianNoise(Exploration): """An exploration that adds white noise to continuous actions. diff --git a/rllib/utils/exploration/ornstein_uhlenbeck_noise.py b/rllib/utils/exploration/ornstein_uhlenbeck_noise.py index 441f6914e65..4bf1bce7108 100644 --- a/rllib/utils/exploration/ornstein_uhlenbeck_noise.py +++ b/rllib/utils/exploration/ornstein_uhlenbeck_noise.py @@ -2,7 +2,7 @@ from typing import Optional, Union from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.gaussian_noise import GaussianNoise from ray.rllib.utils.framework import ( try_import_tf, @@ -18,7 +18,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class OrnsteinUhlenbeckNoise(GaussianNoise): """An exploration that adds Ornstein-Uhlenbeck noise to continuous actions. diff --git a/rllib/utils/exploration/parameter_noise.py b/rllib/utils/exploration/parameter_noise.py index 2544349c3e8..94f1d978f72 100644 --- a/rllib/utils/exploration/parameter_noise.py +++ b/rllib/utils/exploration/parameter_noise.py @@ -11,7 +11,7 @@ TorchDeterministic, ) from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.framework import get_variable, try_import_tf, try_import_torch from ray.rllib.utils.from_config import from_config @@ -25,7 +25,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class ParameterNoise(Exploration): """An exploration that changes a Model's parameters. diff --git a/rllib/utils/exploration/per_worker_epsilon_greedy.py b/rllib/utils/exploration/per_worker_epsilon_greedy.py index ff3ecf24cee..1acdc124cad 100644 --- a/rllib/utils/exploration/per_worker_epsilon_greedy.py +++ b/rllib/utils/exploration/per_worker_epsilon_greedy.py @@ -1,12 +1,12 @@ from gymnasium.spaces import Space from typing import Optional -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.exploration.epsilon_greedy import EpsilonGreedy from ray.rllib.utils.schedules import ConstantSchedule -@PublicAPI +@OldAPIStack class PerWorkerEpsilonGreedy(EpsilonGreedy): """A per-worker epsilon-greedy class for distributed algorithms. diff --git a/rllib/utils/exploration/per_worker_gaussian_noise.py b/rllib/utils/exploration/per_worker_gaussian_noise.py index e3e5446f8f0..97efa73e97e 100644 --- a/rllib/utils/exploration/per_worker_gaussian_noise.py +++ b/rllib/utils/exploration/per_worker_gaussian_noise.py @@ -1,12 +1,12 @@ from gymnasium.spaces import Space from typing import Optional -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.exploration.gaussian_noise import GaussianNoise from ray.rllib.utils.schedules import ConstantSchedule -@PublicAPI +@OldAPIStack class PerWorkerGaussianNoise(GaussianNoise): """A per-worker Gaussian noise class for distributed algorithms. diff --git a/rllib/utils/exploration/per_worker_ornstein_uhlenbeck_noise.py b/rllib/utils/exploration/per_worker_ornstein_uhlenbeck_noise.py index 52aa7cff119..87b77aa2503 100644 --- a/rllib/utils/exploration/per_worker_ornstein_uhlenbeck_noise.py +++ b/rllib/utils/exploration/per_worker_ornstein_uhlenbeck_noise.py @@ -1,12 +1,12 @@ from gymnasium.spaces import Space from typing import Optional -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.exploration.ornstein_uhlenbeck_noise import OrnsteinUhlenbeckNoise from ray.rllib.utils.schedules import ConstantSchedule -@PublicAPI +@OldAPIStack class PerWorkerOrnsteinUhlenbeckNoise(OrnsteinUhlenbeckNoise): """A per-worker Ornstein Uhlenbeck noise class for distributed algorithms. diff --git a/rllib/utils/exploration/random.py b/rllib/utils/exploration/random.py index 1ef839e4f05..34d067990e2 100644 --- a/rllib/utils/exploration/random.py +++ b/rllib/utils/exploration/random.py @@ -5,7 +5,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils import force_tuple from ray.rllib.utils.framework import try_import_tf, try_import_torch, TensorType @@ -17,7 +17,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class Random(Exploration): """A random action selector (deterministic/greedy for explore=False). diff --git a/rllib/utils/exploration/random_encoder.py b/rllib/utils/exploration/random_encoder.py index 8741187ef29..567eb17447d 100644 --- a/rllib/utils/exploration/random_encoder.py +++ b/rllib/utils/exploration/random_encoder.py @@ -6,7 +6,7 @@ from ray.rllib.models.catalog import ModelCatalog from ray.rllib.models.modelv2 import ModelV2 from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.from_config import from_config @@ -78,7 +78,7 @@ def std(self) -> float: return np.sqrt(self.var) -@PublicAPI +@OldAPIStack def update_beta(beta_schedule: str, beta: float, rho: float, step: int) -> float: """Update beta based on schedule and training step. @@ -96,7 +96,7 @@ def update_beta(beta_schedule: str, beta: float, rho: float, step: int) -> float return beta -@PublicAPI +@OldAPIStack def compute_states_entropy( obs_embeds: np.ndarray, embed_dim: int, k_nn: int ) -> np.ndarray: @@ -116,7 +116,7 @@ def compute_states_entropy( return dist.argsort(axis=-1)[:, :k_nn][:, -1].astype(np.float32) -@PublicAPI +@OldAPIStack class RE3(Exploration): """Random Encoder for Efficient Exploration. diff --git a/rllib/utils/exploration/slate_epsilon_greedy.py b/rllib/utils/exploration/slate_epsilon_greedy.py index 6c0df5597eb..30f72dc853f 100644 --- a/rllib/utils/exploration/slate_epsilon_greedy.py +++ b/rllib/utils/exploration/slate_epsilon_greedy.py @@ -1,7 +1,7 @@ from typing import Union from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.epsilon_greedy import EpsilonGreedy from ray.rllib.utils.exploration.exploration import TensorType from ray.rllib.utils.framework import try_import_tf, try_import_torch @@ -10,7 +10,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class SlateEpsilonGreedy(EpsilonGreedy): @override(EpsilonGreedy) def _get_tf_exploration_action_op( diff --git a/rllib/utils/exploration/slate_soft_q.py b/rllib/utils/exploration/slate_soft_q.py index 05e6c08963f..5ed2205482e 100644 --- a/rllib/utils/exploration/slate_soft_q.py +++ b/rllib/utils/exploration/slate_soft_q.py @@ -1,7 +1,7 @@ from typing import Union from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import TensorType from ray.rllib.utils.exploration.soft_q import SoftQ from ray.rllib.utils.framework import try_import_tf, try_import_torch @@ -10,7 +10,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class SlateSoftQ(SoftQ): @override(SoftQ) def get_exploration_action( diff --git a/rllib/utils/exploration/soft_q.py b/rllib/utils/exploration/soft_q.py index 70fb4289227..b6d6fff5337 100644 --- a/rllib/utils/exploration/soft_q.py +++ b/rllib/utils/exploration/soft_q.py @@ -4,12 +4,12 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.tf.tf_action_dist import Categorical from ray.rllib.models.torch.torch_action_dist import TorchCategorical -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.stochastic_sampling import StochasticSampling from ray.rllib.utils.framework import TensorType -@PublicAPI +@OldAPIStack class SoftQ(StochasticSampling): """Special case of StochasticSampling w/ Categorical and temperature param. diff --git a/rllib/utils/exploration/stochastic_sampling.py b/rllib/utils/exploration/stochastic_sampling.py index 2b2e155b202..d083d6ddd80 100644 --- a/rllib/utils/exploration/stochastic_sampling.py +++ b/rllib/utils/exploration/stochastic_sampling.py @@ -5,7 +5,7 @@ from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.modelv2 import ModelV2 -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.exploration.random import Random from ray.rllib.utils.framework import ( @@ -20,7 +20,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class StochasticSampling(Exploration): """An exploration that simply samples from a distribution. diff --git a/rllib/utils/exploration/thompson_sampling.py b/rllib/utils/exploration/thompson_sampling.py index a12fdece5ae..3d4700790ed 100644 --- a/rllib/utils/exploration/thompson_sampling.py +++ b/rllib/utils/exploration/thompson_sampling.py @@ -1,7 +1,7 @@ from typing import Union from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.framework import ( TensorType, @@ -11,7 +11,7 @@ tf1, tf, tfv = try_import_tf() -@PublicAPI +@OldAPIStack class ThompsonSampling(Exploration): @override(Exploration) def get_exploration_action( diff --git a/rllib/utils/exploration/upper_confidence_bound.py b/rllib/utils/exploration/upper_confidence_bound.py index a1f0304c797..7e7e71efe18 100644 --- a/rllib/utils/exploration/upper_confidence_bound.py +++ b/rllib/utils/exploration/upper_confidence_bound.py @@ -1,7 +1,7 @@ from typing import Union from ray.rllib.models.action_dist import ActionDistribution -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.exploration.exploration import Exploration from ray.rllib.utils.framework import ( TensorType, @@ -11,7 +11,7 @@ tf1, tf, tfv = try_import_tf() -@PublicAPI +@OldAPIStack class UpperConfidenceBound(Exploration): @override(Exploration) def get_exploration_action( diff --git a/rllib/utils/filter.py b/rllib/utils/filter.py index 900284e98d3..390283ec8e5 100644 --- a/rllib/utils/filter.py +++ b/rllib/utils/filter.py @@ -4,7 +4,7 @@ import numpy as np import tree # pip install dm_tree -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.deprecation import Deprecated from ray.rllib.utils.numpy import SMALL_NUMBER from ray.rllib.utils.typing import TensorStructType @@ -14,7 +14,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class Filter: """Processes input, possibly statefully.""" @@ -46,7 +46,7 @@ def clear_buffer(self): pass -@DeveloperAPI +@OldAPIStack class NoFilter(Filter): is_concurrent = True @@ -77,7 +77,7 @@ def as_serializable(self) -> "NoFilter": # http://www.johndcook.com/blog/standard_deviation/ -@DeveloperAPI +@OldAPIStack class RunningStat: def __init__(self, shape=()): self.num_pushes = 0 @@ -177,7 +177,7 @@ def from_state(state): return running_stats -@DeveloperAPI +@OldAPIStack class MeanStdFilter(Filter): """Keeps track of a running mean for seen states""" @@ -357,7 +357,7 @@ def _helper(x, rs, buffer, shape): return _helper(x, self.running_stats, self.buffer, self.shape) -@DeveloperAPI +@OldAPIStack class ConcurrentMeanStdFilter(MeanStdFilter): is_concurrent = True @@ -406,7 +406,7 @@ def __repr__(self) -> str: ) -@DeveloperAPI +@OldAPIStack def get_filter(filter_config, shape): # TODO(rliaw): move this into filter manager if filter_config == "MeanStdFilter": diff --git a/rllib/utils/filter_manager.py b/rllib/utils/filter_manager.py index 8bcba097934..351d6dae766 100644 --- a/rllib/utils/filter_manager.py +++ b/rllib/utils/filter_manager.py @@ -2,19 +2,18 @@ from typing import Optional import ray -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack class FilterManager: """Manages filters and coordination across remote evaluators that expose `get_filters` and `sync_filters`. """ @staticmethod - @DeveloperAPI def synchronize( local_filters, worker_set, diff --git a/rllib/utils/postprocessing/episodes.py b/rllib/utils/postprocessing/episodes.py index 8730b39ea9f..3f5c61b2db9 100644 --- a/rllib/utils/postprocessing/episodes.py +++ b/rllib/utils/postprocessing/episodes.py @@ -3,8 +3,10 @@ import numpy as np from ray.rllib.env.single_agent_episode import SingleAgentEpisode +from ray.util.annotations import DeveloperAPI +@DeveloperAPI def add_one_ts_to_episodes_and_truncate(episodes: List[SingleAgentEpisode]): """Adds an artificial timestep to an episode at the end. @@ -57,6 +59,7 @@ def add_one_ts_to_episodes_and_truncate(episodes: List[SingleAgentEpisode]): return orig_truncateds +@DeveloperAPI def remove_last_ts_from_data( episode_lens: List[int], *data: Tuple[np._typing.NDArray], @@ -114,6 +117,7 @@ def remove_last_ts_from_data( return tuple(ret) +@DeveloperAPI def remove_last_ts_from_episodes_and_restore_truncateds( episodes: List[SingleAgentEpisode], orig_truncateds: List[bool], diff --git a/rllib/utils/postprocessing/value_predictions.py b/rllib/utils/postprocessing/value_predictions.py index f8eafcdb0a6..7bf2200b1f2 100644 --- a/rllib/utils/postprocessing/value_predictions.py +++ b/rllib/utils/postprocessing/value_predictions.py @@ -1,6 +1,9 @@ import numpy as np +from ray.util.annotations import DeveloperAPI + +@DeveloperAPI def compute_value_targets( values, rewards, @@ -11,7 +14,7 @@ def compute_value_targets( ): """Computes value function (vf) targets given vf predictions and rewards. - Note that advantages can then easily be computeed via the formula: + Note that advantages can then easily be computed via the formula: advantages = targets - vf_predictions """ # Force-set all values at terminals (not at truncations!) to 0.0. diff --git a/rllib/utils/postprocessing/zero_padding.py b/rllib/utils/postprocessing/zero_padding.py index 41328f20488..8cde924a99a 100644 --- a/rllib/utils/postprocessing/zero_padding.py +++ b/rllib/utils/postprocessing/zero_padding.py @@ -2,7 +2,10 @@ import numpy as np +from ray.util.annotations import DeveloperAPI + +@DeveloperAPI def create_mask_and_seq_lens( episode_lens: List[int], T: int, @@ -99,6 +102,7 @@ def split_and_pad(data_chunks: List[np._typing.NDArray], T: int) -> np._typing.N return result +@DeveloperAPI def split_and_pad_single_record( data: np._typing.NDArray, episode_lengths: List[int], T: int ): @@ -135,15 +139,16 @@ def split_and_pad_single_record( return split_and_pad(episodes_data, T) +@DeveloperAPI def unpad_data_if_necessary(episode_lens, data): """Removes right-side zero-padding from data based on `episode_lens`. ..testcode:: - from ray.rllib.algorithms.ppo.ppo_learner import PPOLearner + from ray.rllib.utils.postprocessing.zero_padding import unpad_data_if_necessary import numpy as np - unpadded = PPOLearner._unpad_data_if_necessary( + unpadded = unpad_data_if_necessary( episode_lens=[4, 2], data=np.array([ [2, 4, 5, 3, 0, 0, 0, 0], @@ -152,7 +157,7 @@ def unpad_data_if_necessary(episode_lens, data): ) assert (unpadded == [2, 4, 5, 3, -1, 3]).all() - unpadded = PPOLearner._unpad_data_if_necessary( + unpadded = unpad_data_if_necessary( episode_lens=[1, 5], data=np.array([ [2, 0, 0, 0, 0], diff --git a/rllib/utils/schedules/constant_schedule.py b/rllib/utils/schedules/constant_schedule.py index 9681c516704..5ad58e266f8 100644 --- a/rllib/utils/schedules/constant_schedule.py +++ b/rllib/utils/schedules/constant_schedule.py @@ -1,6 +1,6 @@ from typing import Optional -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.schedules.schedule import Schedule from ray.rllib.utils.typing import TensorType @@ -8,7 +8,7 @@ tf1, tf, tfv = try_import_tf() -@PublicAPI +@OldAPIStack class ConstantSchedule(Schedule): """A Schedule where the value remains constant over time.""" diff --git a/rllib/utils/schedules/exponential_schedule.py b/rllib/utils/schedules/exponential_schedule.py index 0c6571d928a..4aafac19470 100644 --- a/rllib/utils/schedules/exponential_schedule.py +++ b/rllib/utils/schedules/exponential_schedule.py @@ -1,6 +1,6 @@ from typing import Optional -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.schedules.schedule import Schedule from ray.rllib.utils.typing import TensorType @@ -8,7 +8,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class ExponentialSchedule(Schedule): """Exponential decay schedule from `initial_p` to `final_p`. diff --git a/rllib/utils/schedules/linear_schedule.py b/rllib/utils/schedules/linear_schedule.py index df892548eac..d23647b9bbe 100644 --- a/rllib/utils/schedules/linear_schedule.py +++ b/rllib/utils/schedules/linear_schedule.py @@ -1,8 +1,8 @@ -from ray.rllib.utils.annotations import PublicAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.schedules.polynomial_schedule import PolynomialSchedule -@PublicAPI +@OldAPIStack class LinearSchedule(PolynomialSchedule): """Linear interpolation between `initial_p` and `final_p`. diff --git a/rllib/utils/schedules/piecewise_schedule.py b/rllib/utils/schedules/piecewise_schedule.py index 8dd7db220be..6c4b15478b3 100644 --- a/rllib/utils/schedules/piecewise_schedule.py +++ b/rllib/utils/schedules/piecewise_schedule.py @@ -1,9 +1,10 @@ from typing import Callable, List, Optional, Tuple -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import override from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.schedules.schedule import Schedule from ray.rllib.utils.typing import TensorType +from ray.util.annotations import DeveloperAPI tf1, tf, tfv = try_import_tf() @@ -12,7 +13,7 @@ def _linear_interpolation(left, right, alpha): return left + alpha * (right - left) -@PublicAPI +@DeveloperAPI class PiecewiseSchedule(Schedule): """Implements a Piecewise Scheduler.""" diff --git a/rllib/utils/schedules/polynomial_schedule.py b/rllib/utils/schedules/polynomial_schedule.py index cd72b69cea5..17a2820d9af 100644 --- a/rllib/utils/schedules/polynomial_schedule.py +++ b/rllib/utils/schedules/polynomial_schedule.py @@ -1,6 +1,6 @@ from typing import Optional -from ray.rllib.utils.annotations import override, PublicAPI +from ray.rllib.utils.annotations import OldAPIStack, override from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.schedules.schedule import Schedule from ray.rllib.utils.typing import TensorType @@ -9,7 +9,7 @@ torch, _ = try_import_torch() -@PublicAPI +@OldAPIStack class PolynomialSchedule(Schedule): """Polynomial interpolation between `initial_p` and `final_p`. diff --git a/rllib/utils/schedules/schedule.py b/rllib/utils/schedules/schedule.py index 6b600380f87..fa5b113ff5a 100644 --- a/rllib/utils/schedules/schedule.py +++ b/rllib/utils/schedules/schedule.py @@ -1,14 +1,14 @@ from abc import ABCMeta, abstractmethod from typing import Any, Union -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.typing import TensorType tf1, tf, tfv = try_import_tf() -@DeveloperAPI +@OldAPIStack class Schedule(metaclass=ABCMeta): """Schedule classes implement various time-dependent scheduling schemas. @@ -45,7 +45,6 @@ def __call__(self, t: Union[int, TensorType]) -> Any: """Simply calls self.value(t). Implemented to make Schedules callable.""" return self.value(t) - @DeveloperAPI @abstractmethod def _value(self, t: Union[int, TensorType]) -> Any: """ @@ -59,7 +58,6 @@ def _value(self, t: Union[int, TensorType]) -> Any: """ raise NotImplementedError - @DeveloperAPI def _tf_value_op(self, t: TensorType) -> TensorType: """ Returns the tf-op that calculates the value based on a time step input. diff --git a/rllib/utils/schedules/scheduler.py b/rllib/utils/schedules/scheduler.py index fa02b009a44..231b25c71ea 100644 --- a/rllib/utils/schedules/scheduler.py +++ b/rllib/utils/schedules/scheduler.py @@ -3,12 +3,14 @@ from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.schedules.piecewise_schedule import PiecewiseSchedule from ray.rllib.utils.typing import LearningRateOrSchedule, TensorType +from ray.util.annotations import DeveloperAPI _, tf, _ = try_import_tf() torch, _ = try_import_torch() +@DeveloperAPI class Scheduler: """Class to manage a scheduled (framework-dependent) tensor variable. diff --git a/rllib/utils/sgd.py b/rllib/utils/sgd.py index 0187b4a9105..3e126c0a2f4 100644 --- a/rllib/utils/sgd.py +++ b/rllib/utils/sgd.py @@ -4,14 +4,14 @@ import numpy as np import random -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.policy.sample_batch import SampleBatch, MultiAgentBatch from ray.rllib.utils.metrics.learner_info import LearnerInfoBuilder logger = logging.getLogger(__name__) -@DeveloperAPI +@OldAPIStack def standardized(array: np.ndarray): """Normalize the values in an array. @@ -24,7 +24,7 @@ def standardized(array: np.ndarray): return (array - array.mean()) / max(1e-4, array.std()) -@DeveloperAPI +@OldAPIStack def minibatches(samples: SampleBatch, sgd_minibatch_size: int, shuffle: bool = True): """Return a generator yielding minibatches from a sample batch. @@ -68,7 +68,7 @@ def minibatches(samples: SampleBatch, sgd_minibatch_size: int, shuffle: bool = T yield samples.slice(i, j, si, sj) -@DeveloperAPI +@OldAPIStack def do_minibatch_sgd( samples, policies, diff --git a/rllib/utils/tf_run_builder.py b/rllib/utils/tf_run_builder.py index 2a652fef306..1a4116f2452 100644 --- a/rllib/utils/tf_run_builder.py +++ b/rllib/utils/tf_run_builder.py @@ -3,12 +3,14 @@ import time from ray.util.debug import log_once +from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.framework import try_import_tf tf1, tf, tfv = try_import_tf() logger = logging.getLogger(__name__) +@OldAPIStack class _TFRunBuilder: """Used to incrementally build up a TensorFlow run. diff --git a/rllib/utils/threading.py b/rllib/utils/threading.py index 409bf37fa9c..a9a4461dadb 100644 --- a/rllib/utils/threading.py +++ b/rllib/utils/threading.py @@ -1,9 +1,9 @@ from typing import Callable -from ray.rllib.utils.annotations import DeveloperAPI +from ray.rllib.utils.annotations import OldAPIStack -@DeveloperAPI +@OldAPIStack def with_lock(func: Callable) -> Callable: """Use as decorator (@withlock) around object methods that need locking. diff --git a/rllib/utils/window_stat.py b/rllib/utils/window_stat.py deleted file mode 100644 index 7f4a3874673..00000000000 --- a/rllib/utils/window_stat.py +++ /dev/null @@ -1,9 +0,0 @@ -from ray.rllib.utils.deprecation import deprecation_warning -from ray.rllib.utils.metrics.window_stat import WindowStat - -deprecation_warning( - old="ray.rllib.utils.window_stat.WindowStat", - new="ray.rllib.utils.metrics.window_stat.WindowStat", - error=True, -) -WindowStat = WindowStat