ray: a distributed framework for emerging ai...

18
Ray: A Distributed Framework for Emerging AI Applications Philipp Moritz , Robert Nishihara , Stephanie Wang, Alexey Tumanov, Richard Liaw, Eric Liang, Melih Elibol, Zongheng Yang, William Paul, Michael I. Jordan, Ion Stoica University of California, Berkeley Abstract The next generation of AI applications will continuously interact with the environment and learn from these inter- actions. These applications impose new and demanding systems requirements, both in terms of performance and flexibility. In this paper, we consider these requirements and present Ray—a distributed system to address them. Ray implements a unified interface that can express both task-parallel and actor-based computations, supported by a single dynamic execution engine. To meet the perfor- mance requirements, Ray employs a distributed scheduler and a distributed and fault-tolerant store to manage the system’s control state. In our experiments, we demon- strate scaling beyond 1.8 million tasks per second and better performance than existing specialized systems for several challenging reinforcement learning applications. 1 Introduction Over the past two decades, many organizations have been collecting—and aiming to exploit—ever-growing quanti- ties of data. This has led to the development of a plethora of frameworks for distributed data analysis, including batch [20, 64, 28], streaming [15, 39, 31], and graph [34, 35, 24] processing systems. The success of these frame- works has made it possible for organizations to analyze large data sets as a core part of their business or scientific strategy, and has ushered in the age of “Big Data.” More recently, the scope of data-focused applications has expanded to encompass more complex artificial intel- ligence (AI) or machine learning (ML) techniques [30]. The paradigm case is that of supervised learning, where data points are accompanied by labels, and where the workhorse technology for mapping data points to labels is provided by deep neural networks. The complexity of these deep networks has led to another flurry of frame- works that focus on the training of deep neural networks equal contribution and their use in prediction. These frameworks often lever- age specialized hardware (e.g., GPUs and TPUs), with the goal of reducing training time in a batch setting. Examples include TensorFlow [7], MXNet [18], and PyTorch [46]. The promise of AI is, however, far broader than classi- cal supervised learning. Emerging AI applications must increasingly operate in dynamic environments, react to changes in the environment, and take sequences of ac- tions to accomplish long-term goals [8, 43]. They must aim not only to exploit the data gathered, but also to ex- plore the space of possible actions. These broader require- ments are naturally framed within the paradigm of rein- forcement learning (RL). RL deals with learning to oper- ate continuously within an uncertain environment based on delayed and limited feedback [56]. RL-based systems have already yielded remarkable results, such as Google’s AlphaGo beating a human world champion [54], and are beginning to find their way into dialogue systems, UAVs [42], and robotic manipulation [25, 60]. The central goal of an RL application is to learn a policy—a mapping from the state of the environment to a choice of action—that yields effective performance over time, e.g., winning a game or piloting a drone. Finding ef- fective policies in large-scale applications requires three main capabilities. First, RL methods often rely on simula- tion to evaluate policies. Simulations make it possible to explore many different choices of action sequences and to learn about the long-term consequences of those choices. Second, like their supervised learning counterparts, RL al- gorithms need to perform distributed training to improve the policy based on data generated through simulations or interactions with the physical environment. Third, poli- cies are intended to provide solutions to control problems, and thus it is necessary to serve the policy in interactive closed-loop and open-loop control scenarios. These characteristics drive new systems requirements: a system for RL must support fine-grained computations (e.g., rendering actions in milliseconds when interacting with the real world, and performing vast numbers of sim- USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 561

Upload: others

Post on 29-Dec-2020

0 views

Category:

Documents


0 download

TRANSCRIPT

Page 1: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Ray: A Distributed Framework for Emerging AI Applications

Philipp Moritz⇤, Robert Nishihara⇤, Stephanie Wang, Alexey Tumanov, Richard Liaw,Eric Liang, Melih Elibol, Zongheng Yang, William Paul, Michael I. Jordan, Ion Stoica

University of California, Berkeley

AbstractThe next generation of AI applications will continuouslyinteract with the environment and learn from these inter-actions. These applications impose new and demandingsystems requirements, both in terms of performance andflexibility. In this paper, we consider these requirementsand present Ray—a distributed system to address them.Ray implements a unified interface that can express bothtask-parallel and actor-based computations, supported bya single dynamic execution engine. To meet the perfor-mance requirements, Ray employs a distributed schedulerand a distributed and fault-tolerant store to manage thesystem’s control state. In our experiments, we demon-strate scaling beyond 1.8 million tasks per second andbetter performance than existing specialized systems forseveral challenging reinforcement learning applications.

1 Introduction

Over the past two decades, many organizations have beencollecting—and aiming to exploit—ever-growing quanti-ties of data. This has led to the development of a plethoraof frameworks for distributed data analysis, includingbatch [20, 64, 28], streaming [15, 39, 31], and graph [34,35, 24] processing systems. The success of these frame-works has made it possible for organizations to analyzelarge data sets as a core part of their business or scientificstrategy, and has ushered in the age of “Big Data.”

More recently, the scope of data-focused applicationshas expanded to encompass more complex artificial intel-ligence (AI) or machine learning (ML) techniques [30].The paradigm case is that of supervised learning, wheredata points are accompanied by labels, and where theworkhorse technology for mapping data points to labelsis provided by deep neural networks. The complexity ofthese deep networks has led to another flurry of frame-works that focus on the training of deep neural networks

⇤equal contribution

and their use in prediction. These frameworks often lever-age specialized hardware (e.g., GPUs and TPUs), with thegoal of reducing training time in a batch setting. Examplesinclude TensorFlow [7], MXNet [18], and PyTorch [46].

The promise of AI is, however, far broader than classi-cal supervised learning. Emerging AI applications mustincreasingly operate in dynamic environments, react tochanges in the environment, and take sequences of ac-tions to accomplish long-term goals [8, 43]. They mustaim not only to exploit the data gathered, but also to ex-plore the space of possible actions. These broader require-ments are naturally framed within the paradigm of rein-forcement learning (RL). RL deals with learning to oper-ate continuously within an uncertain environment basedon delayed and limited feedback [56]. RL-based systemshave already yielded remarkable results, such as Google’sAlphaGo beating a human world champion [54], and arebeginning to find their way into dialogue systems, UAVs[42], and robotic manipulation [25, 60].

The central goal of an RL application is to learn apolicy—a mapping from the state of the environment to achoice of action—that yields effective performance overtime, e.g., winning a game or piloting a drone. Finding ef-fective policies in large-scale applications requires threemain capabilities. First, RL methods often rely on simula-tion to evaluate policies. Simulations make it possible toexplore many different choices of action sequences and tolearn about the long-term consequences of those choices.Second, like their supervised learning counterparts, RL al-gorithms need to perform distributed training to improvethe policy based on data generated through simulations orinteractions with the physical environment. Third, poli-cies are intended to provide solutions to control problems,and thus it is necessary to serve the policy in interactiveclosed-loop and open-loop control scenarios.

These characteristics drive new systems requirements:a system for RL must support fine-grained computations(e.g., rendering actions in milliseconds when interactingwith the real world, and performing vast numbers of sim-

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 561

Page 2: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

ulations), must support heterogeneity both in time (e.g.,a simulation may take milliseconds or hours) and in re-source usage (e.g., GPUs for training and CPUs for simu-lations), and must support dynamic execution, as resultsof simulations or interactions with the environment canchange future computations. Thus, we need a dynamiccomputation framework that handles millions of hetero-geneous tasks per second at millisecond-level latencies.

Existing frameworks that have been developed forBig Data workloads or for supervised learning work-loads fall short of satisfying these new requirements forRL. Bulk-synchronous parallel systems such as Map-Reduce [20], Apache Spark [64], and Dryad [28] do notsupport fine-grained simulation or policy serving. Task-parallel systems such as CIEL [40] and Dask [48] providelittle support for distributed training and serving. Thesame is true for streaming systems such as Naiad [39]and Storm [31]. Distributed deep-learning frameworkssuch as TensorFlow [7] and MXNet [18] do not naturallysupport simulation and serving. Finally, model-servingsystems such as TensorFlow Serving [6] and Clipper [19]support neither training nor simulation.

While in principle one could develop an end-to-end so-lution by stitching together several existing systems (e.g.,Horovod [53] for distributed training, Clipper [19] forserving, and CIEL [40] for simulation), in practice this ap-proach is untenable due to the tight coupling of these com-ponents within applications. As a result, researchers andpractitioners today build one-off systems for specializedRL applications [58, 41, 54, 44, 49, 5]. This approach im-poses a massive systems engineering burden on the devel-opment of distributed applications by essentially pushingstandard systems challenges like scheduling, fault toler-ance, and data movement onto each application.

In this paper, we propose Ray, a general-purposecluster-computing framework that enables simulation,training, and serving for RL applications. The require-ments of these workloads range from lightweight andstateless computations, such as for simulation, to long-running and stateful computations, such as for training.To satisfy these requirements, Ray implements a unifiedinterface that can express both task-parallel and actor-based computations. Tasks enable Ray to efficiently anddynamically load balance simulations, process large in-puts and state spaces (e.g., images, video), and recoverfrom failures. In contrast, actors enable Ray to efficientlysupport stateful computations, such as model training, andexpose shared mutable state to clients, (e.g., a parameterserver). Ray implements the actor and the task abstrac-tions on top of a single dynamic execution engine that ishighly scalable and fault tolerant.

To meet the performance requirements, Ray distributestwo components that are typically centralized in existingframeworks [64, 28, 40]: (1) the task scheduler and (2) a

state (si+1) (observation)

reward (ri+1)

action (ai)Policy

improvement(e.g., SGD)

trajectory: s0, (s1, r1), …, (sn, rn)

policyTraining Serving Simulation

Policyevaluation

EnvironmentAgent

Figure 1: Example of an RL system.

metadata store which maintains the computation lineageand a directory for data objects. This allows Ray to sched-ule millions of tasks per second with millisecond-levellatencies. Furthermore, Ray provides lineage-based faulttolerance for tasks and actors, and replication-based faulttolerance for the metadata store.

While Ray supports serving, training, and simulationin the context of RL applications, this does not mean thatit should be viewed as a replacement for systems that pro-vide solutions for these workloads in other contexts. Inparticular, Ray does not aim to substitute for serving sys-tems like Clipper [19] and TensorFlow Serving [6], asthese systems address a broader set of challenges in de-ploying models, including model management, testing,and model composition. Similarly, despite its flexibility,Ray is not a substitute for generic data-parallel frame-works, such as Spark [64], as it currently lacks the richfunctionality and APIs (e.g., straggler mitigation, queryoptimization) that these frameworks provide.

We make the following contributions:

• We design and build the first distributed frame-work that unifies training, simulation, and serving—necessary components of emerging RL applications.

• To support these workloads, we unify the actor andtask-parallel abstractions on top of a dynamic taskexecution engine.

• To achieve scalability and fault tolerance, we pro-pose a system design principle in which control stateis stored in a sharded metadata store and all othersystem components are stateless.

• To achieve scalability, we propose a bottom-up dis-tributed scheduling strategy.

2 Motivation and Requirements

We begin by considering the basic components of an RLsystem and fleshing out the key requirements for Ray. Asshown in Figure 1, in an RL setting, an agent interactsrepeatedly with the environment. The goal of the agentis to learn a policy that maximizes a reward. A policy is

562 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 3: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

// evaluate policy by interacting with env. (e.g., simulator)rollout(policy, environment):

trajectory = []state = environment.initial_state()while (not environment.has_terminated()):

action = policy.compute(state) // Servingstate, reward = environment.step(action) // Simulationtrajectory.append(state, reward)

return trajectory

// improve policy iteratively until it convergestrain_policy(environment):

policy = initial_policy()while (policy has not converged):

trajectories = []for i from 1 to k:

// evaluate policy by generating k rolloutstrajectories.append(rollout(policy, environment))// improve policypolicy = policy.update(trajectories) // Training

return policy

Figure 2: Typical RL pseudocode for learning a policy.

a mapping from the state of the environment to a choiceof action. The precise definitions of environment, agent,state, action, and reward are application-specific.

To learn a policy, an agent typically employs a two-stepprocess: (1) policy evaluation and (2) policy improvement.To evaluate the policy, the agent interacts with the envi-ronment (e.g., with a simulation of the environment) togenerate trajectories, where a trajectory consists of a se-quence of (state, reward) tuples produced by the currentpolicy. Then, the agent uses these trajectories to improvethe policy; i.e., to update the policy in the direction of thegradient that maximizes the reward. Figure 2 shows anexample of the pseudocode used by an agent to learn apolicy. This pseudocode evaluates the policy by invok-ing rollout(environment, policy) to generate trajectories.train policy() then uses these trajectories to improve thecurrent policy via policy.update(trajectories). This pro-cess repeats until the policy converges.

Thus, a framework for RL applications must provideefficient support for training, serving, and simulation(Figure 1). Next, we briefly describe these workloads.

Training typically involves running stochastic gradientdescent (SGD), often in a distributed setting, to update thepolicy. Distributed SGD typically relies on an allreduceaggregation step or a parameter server [32].

Serving uses the trained policy to render an action basedon the current state of the environment. A serving systemaims to minimize latency, and maximize the number ofdecisions per second. To scale, load is typically balancedacross multiple nodes serving the policy.

Finally, most existing RL applications use simulationsto evaluate the policy—current RL algorithms are not

sample-efficient enough to rely solely on data obtainedfrom interactions with the physical world. These simula-tions vary widely in complexity. They might take a few ms(e.g., simulate a move in a chess game) to minutes (e.g.,simulate a realistic environment for a self-driving car).

In contrast with supervised learning, in which train-ing and serving can be handled separately by differentsystems, in RL all three of these workloads are tightlycoupled in a single application, with stringent latency re-quirements between them. Currently, no framework sup-ports this coupling of workloads. In theory, multiple spe-cialized frameworks could be stitched together to providethe overall capabilities, but in practice, the resulting datamovement and latency between systems is prohibitive inthe context of RL. As a result, researchers and practition-ers have been building their own one-off systems.

This state of affairs calls for the development of newdistributed frameworks for RL that can efficiently supporttraining, serving, and simulation. In particular, such aframework should satisfy the following requirements:

Fine-grained, heterogeneous computations. The dura-tion of a computation can range from milliseconds (e.g.,taking an action) to hours (e.g., training a complex pol-icy). Additionally, training often requires heterogeneoushardware (e.g., CPUs, GPUs, or TPUs).

Flexible computation model. RL applications requireboth stateless and stateful computations. Stateless compu-tations can be executed on any node in the system, whichmakes it easy to achieve load balancing and movementof computation to data, if needed. Thus stateless com-putations are a good fit for fine-grained simulation anddata processing, such as extracting features from imagesor videos. In contrast stateful computations are a good fitfor implementing parameter servers, performing repeatedcomputation on GPU-backed data, or running third-partysimulators that do not expose their state.

Dynamic execution. Several components of RL appli-cations require dynamic execution, as the order in whichcomputations finish is not always known in advance (e.g.,the order in which simulations finish), and the results of acomputation can determine future computations (e.g., theresults of a simulation will determine whether we need toperform more simulations).

We make two final comments. First, to achieve highutilization in large clusters, such a framework must handlemillions of tasks per second.⇤ Second, such a frameworkis not intended for implementing deep neural networksor complex simulators from scratch. Instead, it shouldenable seamless integration with existing simulators [13,11, 59] and deep learning frameworks [7, 18, 46, 29].

⇤Assume 5ms single-core tasks and a cluster of 200 32-core nodes.This cluster can run (1s/5ms)⇥32⇥200 = 1.28M tasks/sec.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 563

Page 4: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Name Descriptionfutures = f.remote(args) Execute function f remotely. f.remote() can take objects or futures as inputs

and returns one or more futures. This is non-blocking.objects = ray.get(futures) Return the values associated with one or more futures. This is blocking.ready futures = ray.wait(futures,k, timeout) Return the futures whose corresponding tasks have completed as soon as either

k have completed or the timeout expires.actor = Class.remote(args) Instantiate class Class as a remote actor, and return a handle to it. Call a methodfutures = actor.method.remote(args) on the remote actor and return one or more futures. Both are non-blocking.

Table 1: Ray API

3 Programming and Computation Model

Ray implements a dynamic task graph computationmodel, i.e., it models an application as a graph of depen-dent tasks that evolves during execution. On top of thismodel, Ray provides both an actor and a task-parallelprogramming abstraction. This unification differentiatesRay from related systems like CIEL, which only pro-vides a task-parallel abstraction, and from Orleans [14] orAkka [1], which primarily provide an actor abstraction.

3.1 Programming ModelTasks. A task represents the execution of a remote func-tion on a stateless worker. When a remote function isinvoked, a future representing the result of the task isreturned immediately. Futures can be retrieved usingray.get() and passed as arguments into other remote func-tions without waiting for their result. This allows the userto express parallelism while capturing data dependencies.Table 1 shows Ray’s API.

Remote functions operate on immutable objects andare expected to be stateless and side-effect free: theiroutputs are determined solely by their inputs. This impliesidempotence, which simplifies fault tolerance throughfunction re-execution on failure.Actors. An actor represents a stateful computation. Eachactor exposes methods that can be invoked remotely andare executed serially. A method execution is similar to atask, in that it executes remotely and returns a future, butdiffers in that it executes on a stateful worker. A handleto an actor can be passed to other actors or tasks, makingit possible for them to invoke methods on that actor.

Tasks (stateless) Actors (stateful)Fine-grained load balancing Coarse-grained load balancingSupport for object locality Poor locality support

High overhead for small updates Low overhead for small updatesEfficient failure handling Overhead from checkpointing

Table 2: Tasks vs. actors tradeoffs.

Table 2 summarizes the properties of tasks and actors.Tasks enable fine-grained load balancing through leverag-ing load-aware scheduling at task granularity, input datalocality, as each task can be scheduled on the node stor-ing its inputs, and low recovery overhead, as there is noneed to checkpoint and recover intermediate state. In con-trast, actors provide much more efficient fine-grained up-dates, as these updates are performed on internal ratherthan external state, which typically requires serializationand deserialization. For example, actors can be used toimplement parameter servers [32] and GPU-based itera-tive computations (e.g., training). In addition, actors canbe used to wrap third-party simulators and other opaquehandles that are hard to serialize.

To satisfy the requirements for heterogeneity and flex-ibility (Section 2), we augment the API in three ways.First, to handle concurrent tasks with heterogeneous du-rations, we introduce ray.wait(), which waits for thefirst k available results, instead of waiting for all resultslike ray.get(). Second, to handle resource-heterogeneoustasks, we enable developers to specify resource require-ments so that the Ray scheduler can efficiently manage re-sources. Third, to improve flexibility, we enable nested re-mote functions, meaning that remote functions can invokeother remote functions. This is also critical for achiev-ing high scalability (Section 4), as it enables multiple pro-cesses to invoke remote functions in a distributed fashion.

3.2 Computation ModelRay employs a dynamic task graph computationmodel [21], in which the execution of both remote func-tions and actor methods is automatically triggered by thesystem when their inputs become available. In this sec-tion, we describe how the computation graph (Figure 4) isconstructed from a user program (Figure 3). This programuses the API in Table 1 to implement the pseudocodefrom Figure 2.

Ignoring actors first, there are two types of nodes ina computation graph: data objects and remote functioninvocations, or tasks. There are also two types of edges:data edges and control edges. Data edges capture the de-

564 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 5: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

@ray.remotedef create_policy():

# Initialize the policy randomly.return policy

@ray.remote(num_gpus=1)class Simulator(object):

def __init__(self):# Initialize the environment.self.env = Environment()

def rollout(self, policy, num_steps):observations = []observation = self.env.current_state()for _ in range(num_steps):

action = policy(observation)observation = self.env.step(action)observations.append(observation)

return observations

@ray.remote(num_gpus=2)def update_policy(policy, *rollouts):

# Update the policy.return policy

@ray.remotedef train_policy():

# Create a policy.policy_id = create_policy.remote()# Create 10 actors.simulators = [Simulator.remote() for _ in range(10)]# Do 100 steps of training.for _ in range(100):

# Perform one rollout on each actor.rollout_ids = [s.rollout.remote(policy_id)

for s in simulators]# Update the policy with the rollouts.policy_id =

update_policy.remote(policy_id, *rollout_ids)return ray.get(policy_id)

Figure 3: Python code implementing the example in Figure 2in Ray. Note that @ray.remote indicates remote functions andactors. Invocations of remote functions and actor methods returnfutures, which can be passed to subsequent remote functions oractor methods to encode task dependencies. Each actor has anenvironment object self.env shared between all of its methods.

pendencies between data objects and tasks. More pre-cisely, if data object D is an output of task T , we add adata edge from T to D. Similarly, if D is an input to T ,we add a data edge from D to T . Control edges capturethe computation dependencies that result from nested re-mote functions (Section 3.1): if task T1 invokes task T2,then we add a control edge from T1 to T2.

Actor method invocations are also represented as nodesin the computation graph. They are identical to taskswith one key difference. To capture the state dependencyacross subsequent method invocations on the same actor,we add a third type of edge: a stateful edge. If methodMj is called right after method Mi on the same actor,then we add a stateful edge from Mi to Mj. Thus, all

policy1

T1create_policy

T2update_policy

A11rollout

A12rollout

policy2

T3update_policy

rollout11

rollout12

A21rollout

A22rollout

rollout22

A10Simulator

A20Simulator

… ……

dataedges stateful edgesobject task/method

controledges

rollout21

T0train_policy

Figure 4: The task graph corresponding to an invocation oftrain policy.remote() in Figure 3. Remote function calls and theactor method calls correspond to tasks in the task graph. Thefigure shows two actors. The method invocations for each actor(the tasks labeled A1i and A2i) have stateful edges between themindicating that they share the mutable actor state. There are con-trol edges from train policy to the tasks that it invokes. To trainmultiple policies in parallel, we could call train policy.remote()multiple times.

methods invoked on the same actor object form a chainthat is connected by stateful edges (Figure 4). This chaincaptures the order in which these methods were invoked.

Stateful edges help us embed actors in an otherwisestateless task graph, as they capture the implicit data de-pendency between successive method invocations sharingthe internal state of an actor. Stateful edges also enableus to maintain lineage. As in other dataflow systems [64],we track data lineage to enable reconstruction. By explic-itly including stateful edges in the lineage graph, we caneasily reconstruct lost data, whether produced by remotefunctions or actor methods (Section 4.2.3).

4 Architecture

Ray’s architecture comprises (1) an application layer im-plementing the API, and (2) a system layer providing highscalability and fault tolerance.

4.1 Application LayerThe application layer consists of three types of processes:

• Driver: A process executing the user program.

• Worker: A stateless process that executes tasks(remote functions) invoked by a driver or another

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 565

Page 6: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Local Scheduler

Actor Driver

Object Store

Global Scheduler

Global Scheduler

Object Table

Task Table

Function Table

Event Logs

Global Control Store (GCS)

Local Scheduler

Driver Worker

Object Store

Node

Global Scheduler

Web UI

Debugging Tools

Profiling Tools

Error Diagnosis

Local Scheduler

Worker Worker

Object Store

Node NodeA

pp L

ayer

Sys

tem

Lay

er (

back

end)

Figure 5: Ray’s architecture consists of two parts: an applica-tion layer and a system layer. The application layer implementsthe API and the computation model described in Section 3, thesystem layer implements task scheduling and data managementto satisfy the performance and fault-tolerance requirements.

worker. Workers are started automatically and as-signed tasks by the system layer. When a remotefunction is declared, the function is automaticallypublished to all workers. A worker executes tasksserially, with no local state maintained across tasks.

• Actor: A stateful process that executes, when in-voked, only the methods it exposes. Unlike a worker,an actor is explicitly instantiated by a worker or adriver. Like workers, actors execute methods seri-ally, except that each method depends on the stateresulting from the previous method execution.

4.2 System LayerThe system layer consists of three major components: aglobal control store, a distributed scheduler, and a dis-tributed object store. All components are horizontallyscalable and fault-tolerant.

4.2.1 Global Control Store (GCS)

The global control store (GCS) maintains the entire con-trol state of the system, and it is a unique feature of ourdesign. At its core, GCS is a key-value store with pub-sub functionality. We use sharding to achieve scale, andper-shard chain replication [61] to provide fault tolerance.The primary reason for the GCS and its design is to main-tain fault tolerance and low latency for a system that candynamically spawn millions of tasks per second.

Fault tolerance in case of node failure requires a solu-tion to maintain lineage information. Existing lineage-based solutions [64, 63, 40, 28] focus on coarse-grainedparallelism and can therefore use a single node (e.g., mas-ter, driver) to store the lineage without impacting perfor-mance. However, this design is not scalable for a fine-grained and dynamic workload like simulation. Therefore,

we decouple the durable lineage storage from the othersystem components, allowing each to scale independently.

Maintaining low latency requires minimizing over-heads in task scheduling, which involves choosing whereto execute, and subsequently task dispatch, which in-volves retrieving remote inputs from other nodes. Manyexisting dataflow systems [64, 40, 48] couple these bystoring object locations and sizes in a centralized sched-uler, a natural design when the scheduler is not a bottle-neck. However, the scale and granularity that Ray targetsrequires keeping the centralized scheduler off the criticalpath. Involving the scheduler in each object transfer is pro-hibitively expensive for primitives important to distributedtraining like allreduce, which is both communication-intensive and latency-sensitive. Therefore, we store theobject metadata in the GCS rather than in the scheduler,fully decoupling task dispatch from task scheduling.

In summary, the GCS significantly simplifies Ray’soverall design, as it enables every component in the sys-tem to be stateless. This not only simplifies support forfault tolerance (i.e., on failure, components simply restartand read the lineage from the GCS), but also makes iteasy to scale the distributed object store and scheduler in-dependently, as all components share the needed state viathe GCS. An added benefit is the easy development of de-bugging, profiling, and visualization tools.

4.2.2 Bottom-Up Distributed Scheduler

As discussed in Section 2, Ray needs to dynamicallyschedule millions of tasks per second, tasks which maytake as little as a few milliseconds. None of the clus-ter schedulers we are aware of meet these requirements.Most cluster computing frameworks, such as Spark [64],CIEL [40], and Dryad [28] implement a centralized sched-uler, which can provide locality but at latencies in the tensof ms. Distributed schedulers such as work stealing [12],Sparrow [45] and Canary [47] can achieve high scale, butthey either don’t consider data locality [12], or assumetasks belong to independent jobs [45], or assume the com-putation graph is known [47].

To satisfy the above requirements, we design a two-level hierarchical scheduler consisting of a global sched-uler and per-node local schedulers. To avoid overloadingthe global scheduler, the tasks created at a node are sub-mitted first to the node’s local scheduler. A local sched-uler schedules tasks locally unless the node is overloaded(i.e., its local task queue exceeds a predefined threshold),or it cannot satisfy a task’s requirements (e.g., lacks aGPU). If a local scheduler decides not to schedule a tasklocally, it forwards it to the global scheduler. Since thisscheduler attempts to schedule tasks locally first (i.e., atthe leaves of the scheduling hierarchy), we call it a bottom-up scheduler.

566 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 7: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Global Scheduler

Local Scheduler

Global Scheduler

WorkerDriver Worker …

Global Control State (GCS)

Local Scheduler

WorkerWorker Worker

Submit tasks

Schedule tasks

Loadinfo

Node 1 Node N

Figure 6: Bottom-up distributed scheduler. Tasks are submittedbottom-up, from drivers and workers to a local scheduler andforwarded to the global scheduler only if needed (Section 4.2.2).The thickness of each arrow is proportional to its request rate.

The global scheduler considers each node’s load andtask’s constraints to make scheduling decisions. More pre-cisely, the global scheduler identifies the set of nodes thathave enough resources of the type requested by the task,and of these nodes selects the node which provides thelowest estimated waiting time. At a given node, this timeis the sum of (i) the estimated time the task will be queuedat that node (i.e., task queue size times average task ex-ecution), and (ii) the estimated transfer time of task’sremote inputs (i.e., total size of remote inputs divided byaverage bandwidth). The global scheduler gets the queuesize at each node and the node resource availability viaheartbeats, and the location of the task’s inputs and theirsizes from GCS. Furthermore, the global scheduler com-putes the average task execution and the average transferbandwidth using simple exponential averaging. If theglobal scheduler becomes a bottleneck, we can instantiatemore replicas all sharing the same information via GCS.This makes our scheduler architecture highly scalable.

4.2.3 In-Memory Distributed Object Store

To minimize task latency, we implement an in-memorydistributed storage system to store the inputs and outputsof every task, or stateless computation. On each node, weimplement the object store via shared memory. This al-lows zero-copy data sharing between tasks running on thesame node. As a data format, we use Apache Arrow [2].

If a task’s inputs are not local, the inputs are replicatedto the local object store before execution. Also, a taskwrites its outputs to the local object store. Replicationeliminates the potential bottleneck due to hot data objectsand minimizes task execution time as a task only read-s/writes data from/to the local memory. This increasesthroughput for computation-bound workloads, a profileshared by many AI applications. For low latency, we keepobjects entirely in memory and evict them as needed to

disk using an LRU policy.As with existing cluster computing frameworks, such

as Spark [64], and Dryad [28], the object store is limitedto immutable data. This obviates the need for complexconsistency protocols (as objects are not updated), andsimplifies support for fault tolerance. In the case of nodefailure, Ray recovers any needed objects through lineagere-execution. The lineage stored in the GCS tracks bothstateless tasks and stateful actors during initial execution;we use the former to reconstruct objects in the store.

For simplicity, our object store does not support dis-tributed objects, i.e., each object fits on a single node. Dis-tributed objects like large matrices or trees can be imple-mented at the application level as collections of futures.

4.2.4 Implementation

Ray is an active open source project† developed at the Uni-versity of California, Berkeley. Ray fully integrates withthe Python environment and is easy to install by simplyrunning pip install ray. The implementation com-prises ⇡ 40K lines of code (LoC), 72% in C++ for thesystem layer, 28% in Python for the application layer. TheGCS uses one Redis [50] key-value store per shard, withentirely single-key operations. GCS tables are shardedby object and task IDs to scale, and every shard is chain-replicated [61] for fault tolerance. We implement boththe local and global schedulers as event-driven, single-threaded processes. Internally, local schedulers maintaincached state for local object metadata, tasks waiting forinputs, and tasks ready for dispatch to a worker. To trans-fer large objects between different object stores, we stripethe object across multiple TCP connections.

4.3 Putting Everything TogetherFigure 7 illustrates how Ray works end-to-end with asimple example that adds two objects a and b, whichcould be scalars or matrices, and returns result c. Theremote function add() is automatically registered with theGCS upon initialization and distributed to every workerin the system (step 0 in Figure 7a).

Figure 7a shows the step-by-step operations triggeredby a driver invoking add.remote(a,b), where a and b arestored on nodes N1 and N2, respectively. The driver sub-mits add(a, b) to the local scheduler (step 1), which for-wards it to a global scheduler (step 2).‡ Next, the globalscheduler looks up the locations of add(a, b)’s argumentsin the GCS (step 3) and decides to schedule the task onnode N2, which stores argument b (step 4). The localscheduler at node N2 checks whether the local objectstore contains add(a, b)’s arguments (step 5). Since the

†https://github.com/ray-project/ray‡Note that N1 could also decide to schedule the task locally.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 567

Page 8: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Object store

@ray.remotedef add(a, b):

return a + b

@ray.remotedef add(a, b):

return a + b

idc = add.remote(a, b)c = ray.get(idc)

N1

Driver

Object Table

Function Table

@ray.remotedef add(a, b):

return a + b

N2

Worker

ida N1idb N2

Global Control Store (GCS)

4

5

6

Local Scheduler

Object store

ida a1

2

8

Global Scheduler

7

9

ida a idb b

0

3

Local Scheduler

(a) Executing a task remotely

Local Scheduler

idb b

@ray.remotedef add(a, b):

return a + b

@ray.remotedef add(a, b):

return a + b

idc = add.remote(a, b)c = ray.get(idc)

N1

Driver

Object Table

Function Table

@ray.remotedef add(a, b):

return a + b

N2

Worker

ida N1idb N2

Global Control Store (GCS)

Local Scheduler

ida a1

ida aidc cidc N2, N1 4

2

5

7 3

Global Scheduler

idc c6

(b) Returning the result of a remote task

Figure 7: An end-to-end example that adds a and b and returnsc. Solid lines are data plane operations and dotted lines arecontrol plane operations. (a) The function add() is registeredwith the GCS by node 1 (N1), invoked on N1, and executedon N2. (b) N1 gets add()’s result using ray.get(). The ObjectTable entry for c is created in step 4 and updated in step 6 afterc is copied to N1.

local store doesn’t have object a, it looks up a’s locationin the GCS (step 6). Learning that a is stored at N1, N2’sobject store replicates it locally (step 7). As all argumentsof add() are now stored locally, the local scheduler in-vokes add() at a local worker (step 8), which accesses thearguments via shared memory (step 9).

Figure 7b shows the step-by-step operations triggeredby the execution of ray.get() at N1, and of add() at N2,respectively. Upon ray.get(idc)’s invocation, the driverchecks the local object store for the value c, using thefuture idc returned by add() (step 1). Since the localobject store doesn’t store c, it looks up its location in theGCS. At this time, there is no entry for c, as c has notbeen created yet. As a result, N1’s object store registers acallback with the Object Table to be triggered when c’sentry has been created (step 2). Meanwhile, at N2, add()completes its execution, stores the result c in the localobject store (step 3), which in turn adds c’s entry to theGCS (step 4). As a result, the GCS triggers a callbackto N1’s object store with c’s entry (step 5). Next, N1replicates c from N2 (step 6), and returns c to ray.get()(step 7), which finally completes the task.

While this example involves a large number of RPCs,

100KB 10B 100B 1000B

2bMect sLze

10-5

10-4

10-3

10-2

10-1

0ean

task

Oate

ncy

(s)

LocaOLty AwaUe

UnawaUe

(a) Ray locality scheduling

10 20 30 40 50 60 100

number of nodes

0.0

0.4

0.8

1.2

1.6

0ill

ion

s of

task

s/s

(b) Ray scalability

Figure 8: (a) Tasks leverage locality-aware placement. 1000tasks with a random object dependency are scheduled onto oneof two nodes. With locality-aware policy, task latency remainsindependent of the size of task inputs instead of growing by 1-2orders of magnitude. (b) Near-linear scalability leveraging theGCS and bottom-up distributed scheduler. Ray reaches 1 milliontasks per second throughput with 60 nodes. x 2 {70,80,90}omitted due to cost.

in many cases this number is much smaller, as most tasksare scheduled locally, and the GCS replies are cached bythe global and local schedulers.

5 Evaluation

In our evaluation, we study the following questions:1. How well does Ray meet the latency, scalability,

and fault tolerance requirements listed in Section 2?(Section 5.1)

2. What overheads are imposed on distributed primi-tives (e.g., allreduce) written using Ray’s API? (Sec-tion 5.1)

3. In the context of RL workloads, how does Ray com-pare against specialized systems for training, serv-ing, and simulation? (Section 5.2)

4. What advantages does Ray provide for RL applica-tions, compared to custom systems? (Section 5.3)

All experiments were run on Amazon Web Services.Unless otherwise stated, we use m4.16xlarge CPU in-stances and p3.16xlarge GPU instances.

5.1 MicrobenchmarksLocality-aware task placement. Fine-grain load bal-

ancing and locality-aware placement are primary benefitsof tasks in Ray. Actors, once placed, are unable to movetheir computation to large remote objects, while tasks can.In Figure 8a, tasks placed without data locality awareness(as is the case for actor methods), suffer 1-2 orders ofmagnitude latency increase at 10-100MB input data sizes.Ray unifies tasks and actors through the shared objectstore, allowing developers to use tasks for e.g., expensivepostprocessing on output produced by simulation actors.

End-to-end scalability. One of the key benefits of

568 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 9: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

1KB 10KB100KB 10B 100B1000B 1GB

obMect size

0

5000

10000

15000

20000I2

P6

0

2

4

6

8

10

12

14

16

tKro

ug

KS

ut

(GB

/s)

Figure 9: Object store write throughput and IOPS. From asingle client, throughput exceeds 15GB/s (red) for large objectsand 18K IOPS (cyan) for small objects on a 16 core instance(m4.4xlarge). It uses 8 threads to copy objects larger than 0.5MBand 1 thread for small objects. Bar plots report throughput with1, 2, 4, 8, 16 threads. Results are averaged over 5 runs.

the Global Control Store (GCS) and the bottom-up dis-tributed scheduler is the ability to horizontally scale thesystem to support a high throughput of fine-grained tasks,while maintaining fault tolerance and low-latency taskscheduling. In Figure 8b, we evaluate this ability on anembarrassingly parallel workload of empty tasks, increas-ing the cluster size on the x-axis. We observe near-perfectlinearity in progressively increasing task throughput. Rayexceeds 1 million tasks per second throughput at 60 nodesand continues to scale linearly beyond 1.8 million tasksper second at 100 nodes. The rightmost datapoint showsthat Ray can process 100 million tasks in less than aminute (54s), with minimum variability. As expected, in-creasing task duration reduces throughput proportionallyto mean task duration, but the overall scalability remainslinear. While many realistic workloads may exhibit morelimited scalability due to object dependencies and inher-ent limits to application parallelism, this demonstrates thescalability of our overall architecture under high load.

Object store performance. To evaluate the perfor-mance of the object store (Section 4.2.3), we track twometrics: IOPS (for small objects) and write throughput(for large objects). In Figure 9, the write throughput froma single client exceeds 15GB/s as object size increases.For larger objects, memcpy dominates object creationtime. For smaller objects, the main overheads are in seri-alization and IPC between the client and object store.

GCS fault tolerance. To maintain low latency whileproviding strong consistency and fault tolerance, we builda lightweight chain replication [61] layer on top of Redis.Figure 10a simulates recording Ray tasks to and readingtasks from the GCS, where keys are 25 bytes and valuesare 512 bytes. The client sends requests as fast as it can,having at most one in-flight request at a time. Failures arereported to the chain master either from the client (havingreceived explicit errors, or timeouts despite retries) or

0 1 2 3 4 5 6 7 8 9 10

7Lme sLnce start (s)

103 103

104 104

Late

ncy

(μs)

wrLtereadnode dead

(a) A timeline for GCS read and write latencies as viewed froma client submitting tasks. The chain starts with 2 replicas. Wemanually trigger reconfiguration as follows. At t ⇡ 4.2s, a chainmember is killed; immediately after, a new chain member joins,initiates state transfer, and restores the chain to 2-way replication.The maximum client-observed latency is under 30ms despitereconfigurations.

0 10000 20000 30000 40000 50000 60000(lasSeG Time (secRnGs)

0

2000

4000

6000

8000

GC6

8se

G 0

emRU

y (0

B)

50 milliRn nR-RS tasks

5ay, nR GC6 flush5ay, GC6 flush

(b) The Ray GCS maintains a constant memory footprint withGCS flushing. Without GCS flushing, the memory footprintreaches a maximum capacity and the workload fails to completewithin a predetermined duration (indicated by the red cross).

Figure 10: Ray GCS fault tolerance and flushing.

from any server in the chain (having received expliciterrors). Overall, reconfigurations caused a maximumclient-observed delay of under 30ms (this includes bothfailure detection and recovery delays).

GCS flushing. Ray is equipped to periodically flushthe contents of GCS to disk. In Figure 10b we submit 50million empty tasks sequentially and monitor GCS mem-ory consumption. As expected, it grows linearly with thenumber of tasks tracked and eventually reaches the mem-ory capacity of the system. At that point, the system be-comes stalled and the workload fails to finish within a rea-sonable amount of time. With periodic GCS flushing, weachieve two goals. First, the memory footprint is cappedat a user-configurable level (in the microbenchmark weemploy an aggressive strategy where consumed memoryis kept as low as possible). Second, the flushing mecha-nism provides a natural way to snapshot lineage to diskfor long-running Ray applications.

Recovering from task failures. In Figure 11a, we

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 569

Page 10: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

0 50 100 150 200Time since start (s)

0

500

1000

1500

2000

ThrR

ughp

ut (t

asNs

/s)

0

20

40

60

1um

ber R

f nRd

es2riginaO tasNs5e-executed tasNs

(a) Task reconstruction

100 200 300 400 500 6007ime since start (s)

0100200300400500600700

7hrR

ughp

ut (t

asks

/s)

2riginaO tasks5e-executed tasksCheckpRint tasks

(b) Actor reconstruction

Figure 11: Ray fault-tolerance. (a) Ray reconstructs lost taskdependencies as nodes are removed (dotted line), and recoversto original throughput when nodes are added back. Each taskis 100ms and depends on an object generated by a previouslysubmitted task. (b) Actors are reconstructed from their lastcheckpoint. At t = 200s, we kill 2 of the 10 nodes, causing 400of the 2000 actors in the cluster to be recovered on the remainingnodes (t = 200–270s).

demonstrate Ray’s ability to transparently recover fromworker node failures and elastically scale, using thedurable GCS lineage storage. The workload, run onm4.xlarge instances, consists of linear chains of 100mstasks submitted by the driver. As nodes are removed (at25s, 50s, 100s), the local schedulers reconstruct previousresults in the chain in order to continue execution. Over-all per-node throughput remains stable throughout.

Recovering from actor failures. By encoding actormethod calls as stateful edges directly in the dependencygraph, we can reuse the same object reconstruction mech-anism as in Figure 11a to provide transparent fault tol-erance for stateful computation. Ray additionally lever-ages user-defined checkpoint functions to bound the re-construction time for actors (Figure 11b). With minimaloverhead, checkpointing enables only 500 methods to bere-executed, versus 10k re-executions without checkpoint-ing. In the future, we hope to further reduce actor recon-struction time, e.g., by allowing users to annotate meth-ods that do not mutate state.

Allreduce. Allreduce is a distributed communication

100B 1000B 1*B

2bMect size

100

101

102

103

104

Itera

tiRn

tiP

e (

PiOO

isecR

nG

s)

2pen03I

Ray*

Ray

(a) Ray vs OpenMPI

+0 +1 +5 +10

Added scheduler latency (ms)

0

100

200

300

400

500

600

700

800

,tera

tiRn

tim

e (

mill

isecR

nd

s)

5ay ring reduce latency(16 nRdes, 1000B)

(b) Ray scheduler ablation

Figure 12: (a) Mean execution time of allreduce on 16 m4.16xlnodes. Each worker runs on a distinct node. Ray* restricts Rayto 1 thread for sending and 1 thread for receiving. (b) Ray’s low-latency scheduling is critical for allreduce.

primitive important to many machine learning workloads.Here, we evaluate whether Ray can natively support aring allreduce [57] implementation with low enough over-head to match existing implementations [53]. We find thatRay completes allreduce across 16 nodes on 100MB in⇠200ms and 1GB in ⇠1200ms, surprisingly outperform-ing OpenMPI (v1.10), a popular MPI implementation,by 1.5⇥ and 2⇥ respectively (Figure 12a). We attributeRay’s performance to its use of multiple threads for net-work transfers, taking full advantage of the 25Gbps con-nection between nodes on AWS, whereas OpenMPI se-quentially sends and receives data on a single thread [22].For smaller objects, OpenMPI outperforms Ray by switch-ing to a lower overhead algorithm, an optimization weplan to implement in the future.

Ray’s scheduler performance is critical to implement-ing primitives such as allreduce. In Figure 12b, we injectartificial task execution delays and show that performancedrops nearly 2⇥ with just a few ms of extra latency. Sys-tems with centralized schedulers like Spark and CIEL typ-ically have scheduler overheads in the tens of millisec-onds [62, 38], making such workloads impractical. Sched-uler throughput also becomes a bottleneck since the num-ber of tasks required by ring reduce scales quadraticallywith the number of participants.

5.2 Building blocks

End-to-end applications (e.g., AlphaGo [54]) require atight coupling of training, serving, and simulation. In thissection, we isolate each of these workloads to a settingthat illustrates a typical RL application’s requirements.Due to a flexible programming model targeted to RL, anda system designed to support this programming model,Ray matches and sometimes exceeds the performance ofdedicated systems for these individual workloads.

570 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 11: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

4 8 16 32 64

1uP G38V (V100)

01000200030004000500060007000

0HDn

iP

Dg

HV

/ V

+RURvRG + 7)

DiVtUibutHG 7)

5Dy + 7)

Figure 13: Images per second reached when distributing thetraining of a ResNet-101 TensorFlow model (from the officialTF benchmark). All experiments were run on p3.16xl instancesconnected by 25Gbps Ethernet, and workers allocated 4 GPUsper node as done in Horovod [53]. We note some measurementdeviations from previously reported, likely due to hardwaredifferences and recent TensorFlow performance improvements.We used OpenMPI 3.0, TF 1.8, and NCCL2 for all runs.

5.2.1 Distributed Training

We implement data-parallel synchronous SGD leverag-ing the Ray actor abstraction to represent model replicas.Model weights are synchronized via allreduce (5.1) or pa-rameter server, both implemented on top of the Ray API.

In Figure 13, we evaluate the performance of theRay (synchronous) parameter-server SGD implementa-tion against state-of-the-art implementations [53], us-ing the same TensorFlow model and synthetic data gen-erator for each experiment. We compare only againstTensorFlow-based systems to accurately measure the over-head imposed by Ray, rather than differences between thedeep learning frameworks themselves. In each iteration,model replica actors compute gradients in parallel, sendthe gradients to a sharded parameter server, then read thesummed gradients from the parameter server for the nextiteration.

Figure 13 shows that Ray matches the performance ofHorovod and is within 10% of distributed TensorFlow(in distributed replicated mode). This is due tothe ability to express the same application-level optimiza-tions found in these specialized systems in Ray’s general-purpose API. A key optimization is the pipelining of gra-dient computation, transfer, and summation within a sin-gle iteration. To overlap GPU computation with networktransfer, we use a custom TensorFlow operator to writetensors directly to Ray’s object store.

5.2.2 Serving

Model serving is an important component of end-to-endapplications. Ray focuses primarily on the embeddedserving of models to simulators running within the samedynamic task graph (e.g., within an RL application onRay). In contrast, systems like Clipper [19] focus onserving predictions to external clients.

In this setting, low latency is critical for achieving highutilization. To show this, in Table 3 we compare the

System Small Input Larger InputClipper 4400 ± 15 states/sec 290 ± 1.3 states/sec

Ray 6200 ± 21 states/sec 6900 ± 150 states/sec

Table 3: Throughput comparisons for Clipper [19], a dedicatedserving system, and Ray for two embedded serving workloads.We use a residual network and a small fully connected network,taking 10ms and 5ms to evaluate, respectively. The server isqueried by clients that each send states of size 4KB and 100KBrespectively in batches of 64.

server throughput achieved using a Ray actor to servea policy versus using the open source Clipper systemover REST. Here, both client and server processes are co-located on the same machine (a p3.8xlarge instance). Thisis often the case for RL applications but not for the generalweb serving workloads addressed by systems like Clipper.Due to its low-overhead serialization and shared memoryabstractions, Ray achieves an order of magnitude higherthroughput for a small fully connected policy model thattakes in a large input and is also faster on a more expensiveresidual network policy model, similar to one used inAlphaGo Zero, that takes smaller input.

5.2.3 Simulation

Simulators used in RL produce results with variablelengths (“timesteps”) that, due to the tight loop with train-ing, must be used as soon as they are available. The taskheterogeneity and timeliness requirements make simu-lations hard to support efficiently in BSP-style systems.To demonstrate, we compare (1) an MPI implementationthat submits 3n parallel simulation runs on n cores in 3rounds, with a global barrier between rounds§, to (2) aRay program that issues the same 3n tasks while concur-rently gathering simulation results back to the driver. Ta-ble 4 shows that both systems scale well, yet Ray achievesup to 1.8⇥ throughput. This motivates a programmingmodel that can dynamically spawn and collect the resultsof fine-grained simulation tasks.

System, programming model 1 CPU 16 CPUs 256 CPUsMPI, bulk synchronous 22.6K 208K 2.16M

Ray, asynchronous tasks 22.3K 290K 4.03M

Table 4: Timesteps per second for the Pendulum-v0 simulatorin OpenAI Gym [13]. Ray allows for better utilization whenrunning heterogeneous simulations at scale.

§Note that experts can use MPI’s asynchronous primitives to getaround barriers—at the expense of increased program complexity —wenonetheless chose such an implementation to simulate BSP.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 571

Page 12: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

5.3 RL ApplicationsWithout a system that can tightly couple the training, sim-ulation, and serving steps, reinforcement learning algo-rithms today are implemented as one-off solutions thatmake it difficult to incorporate optimizations that, for ex-ample, require a different computation structure or thatutilize different architectures. Consequently, with imple-mentations of two representative reinforcement learningapplications in Ray, we are able to match and even out-perform custom systems built specifically for these algo-rithms. The primary reason is the flexibility of Ray’s pro-gramming model, which can express application-level op-timizations that would require substantial engineering ef-fort to port to custom-built systems, but are transparentlysupported by Ray’s dynamic task graph execution engine.

5.3.1 Evolution Strategies

To evaluate Ray on large-scale RL workloads, we imple-ment the evolution strategies (ES) algorithm and com-pare to the reference implementation [49]—a system spe-cially built for this algorithm that relies on Redis for mes-saging and low-level multiprocessing libraries for data-sharing. The algorithm periodically broadcasts a new pol-icy to a pool of workers and aggregates the results ofroughly 10000 tasks (each performing 10 to 1000 simula-tion steps).

As shown in Figure 14a, an implementation on Rayscales to 8192 cores. Doubling the cores available yieldsan average completion time speedup of 1.6⇥. Conversely,the special-purpose system fails to complete at 2048 cores,where the work in the system exceeds the processingcapacity of the application driver. To avoid this issue, theRay implementation uses an aggregation tree of actors,reaching a median time of 3.7 minutes, more than twiceas fast as the best published result (10 minutes).

Initial parallelization of a serial implementation usingRay required modifying only 7 lines of code. Performanceimprovement through hierarchical aggregation was easyto realize with Ray’s support for nested tasks and actors.In contrast, the reference implementation had several hun-dred lines of code dedicated to a protocol for communi-cating tasks and data between workers, and would requirefurther engineering to support optimizations like hierar-chical aggregation.

5.3.2 Proximal Policy Optimization

We implement Proximal Policy Optimization (PPO) [51]in Ray and compare to a highly-optimized reference im-plementation [5] that uses OpenMPI communication prim-itives. The algorithm is an asynchronous scatter-gather,where new tasks are assigned to simulation actors as they

256 1024 8192

1uPEeU Rf C38s

0

10

20

30

40

50

60

70

80

90

0ean

tiP

e t

R s

Rlv

e (

Pin

ute

s)

x x x

5efeUence (6

5ay (6

(a) Evolution Strategies

8x1 64x8 512x64

C38s x G38s

0

100

200

300

400

500

0ean

tiP

e t

R s

ROv

e (

Pin

ute

s)

03I 332

5ay 332

(b) PPO

Figure 14: Time to reach a score of 6000 in the Humanoid-v1 task [13]. (a) The Ray ES implementation scales well to8192 cores and achieves a median time of 3.7 minutes, overtwice as fast as the best published result. The special-purposesystem failed to run beyond 1024 cores. ES is faster than PPOon this benchmark, but shows greater runtime variance. (b)The Ray PPO implementation outperforms a specialized MPIimplementation [5] with fewer GPUs, at a fraction of the cost.The MPI implementation required 1 GPU for every 8 CPUs,whereas the Ray version required at most 8 GPUs (and nevermore than 1 GPU per 8 CPUs).

return rollouts to the driver. Tasks are submitted un-til 320000 simulation steps are collected (each task pro-duces between 10 and 1000 steps). The policy update per-forms 20 steps of SGD with a batch size of 32768. Themodel parameters in this example are roughly 350KB.These experiments were run using p2.16xlarge (GPU) andm4.16xlarge (high CPU) instances.

As shown in Figure 14b, the Ray implementation out-performs the optimized MPI implementation in all exper-iments, while using a fraction of the GPUs. The reasonis that Ray is heterogeneity-aware and allows the user toutilize asymmetric architectures by expressing resourcerequirements at the granularity of a task or actor. The Rayimplementation can then leverage TensorFlow’s single-process multi-GPU support and can pin objects in GPUmemory when possible. This optimization cannot be eas-ily ported to MPI due to the need to asynchronously gatherrollouts to a single GPU process. Indeed, [5] includes twocustom implementations of PPO, one using MPI for largeclusters and one that is optimized for GPUs but that is re-stricted to a single node. Ray allows for an implementa-tion suitable for both scenarios.

Ray’s ability to handle resource heterogeneity also de-creased PPO’s cost by a factor of 4.5 [4], since CPU-onlytasks can be scheduled on cheaper high-CPU instances.In contrast, MPI applications often exhibit symmetric ar-chitectures, in which all processes run the same code andrequire identical resources, in this case preventing theuse of CPU-only machines for scale-out. Furthermore,the MPI implementation requires on-demand instancessince it does not transparently handle failure. Assum-ing 4⇥ cheaper spot instances, Ray’s fault tolerance andresource-aware scheduling together cut costs by 18⇥.

572 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 13: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

6 Related Work

Dynamic task graphs. Ray is closely related toCIEL [40] and Dask [48]. All three support dynamictask graphs with nested tasks and implement the futuresabstraction. CIEL also provides lineage-based fault toler-ance, while Dask, like Ray, fully integrates with Python.However, Ray differs in two aspects that have importantperformance consequences. First, Ray extends the taskmodel with an actor abstraction. This is necessary forefficient stateful computation in distributed training andserving, to keep the model data collocated with the com-putation. Second, Ray employs a fully distributed and de-coupled control plane and scheduler, instead of relying ona single master storing all metadata. This is critical for ef-ficiently supporting primitives like allreduce without sys-tem modification. At peak performance for 100MB on 16nodes, allreduce on Ray (Section 5.1) submits 32 roundsof 16 tasks in 200ms. Meanwhile, Dask reports a maxi-mum scheduler throughput of 3k tasks/s on 512 cores [3].With a centralized scheduler, each round of allreducewould then incur a minimum of ⇠5ms of schedulingdelay, translating to up to 2⇥ worse completion time (Fig-ure 12b). Even with a decentralized scheduler, couplingthe control plane information with the scheduler leavesthe latter on the critical path for data transfer, adding anextra roundtrip to every round of allreduce.

Dataflow systems. Popular dataflow systems, suchas MapReduce [20], Spark [65], and Dryad [28] havewidespread adoption for analytics and ML workloads,but their computation model is too restrictive for a fine-grained and dynamic simulation workload. Spark andMapReduce implement the BSP execution model, whichassumes that tasks within the same stage perform thesame computation and take roughly the same amount oftime. Dryad relaxes this restriction but lacks support fordynamic task graphs. Furthermore, none of these systemsprovide an actor abstraction, nor implement a distributedscalable control plane and scheduler. Finally, Naiad [39]is a dataflow system that provides improved scalabilityfor some workloads, but only supports static task graphs.

Machine learning frameworks. TensorFlow [7] andMXNet [18] target deep learning workloads and effi-ciently leverage both CPUs and GPUs. While theyachieve great performance for training workloads consist-ing of static DAGs of linear algebra operations, they havelimited support for the more general computation requiredto tightly couple training with simulation and embeddedserving. TensorFlow Fold [33] provides some support fordynamic task graphs, as well as MXNet through its inter-nal C++ APIs, but neither fully supports the ability to mod-ify the DAG during execution in response to task progress,task completion times, or faults. TensorFlow and MXNetin principle achieve generality by allowing the program-

mer to simulate low-level message-passing and synchro-nization primitives, but the pitfalls and user experience inthis case are similar to those of MPI. OpenMPI [22] canachieve high performance, but it is relatively hard to pro-gram as it requires explicit coordination to handle hetero-geneous and dynamic task graphs. Furthermore, it forcesthe programmer to explicitly handle fault tolerance.

Actor systems. Orleans [14] and Akka [1] are two ac-tor frameworks well suited to developing highly availableand concurrent distributed systems. However, comparedto Ray, they provide less support for recovery from dataloss. To recover stateful actors, the Orleans developermust explicitly checkpoint actor state and intermediate re-sponses. Stateless actors in Orleans can be replicated forscale-out, and could therefore act as tasks, but unlike inRay, they have no lineage. Similarly, while Akka explic-itly supports persisting actor state across failures, it doesnot provide efficient fault tolerance for stateless computa-tion (i.e., tasks). For message delivery, Orleans providesat-least-once and Akka provides at-most-once semantics.In contrast, Ray provides transparent fault tolerance andexactly-once semantics, as each method call is logged inthe GCS and both arguments and results are immutable.We find that in practice these limitations do not affect theperformance of our applications. Erlang [10] and C++ Ac-tor Framework [17], two other actor-based systems, havesimilarly limited support for fault tolerance.

Global control store and scheduling. The conceptof logically centralizing the control plane has been pre-viously proposed in software defined networks (SDNs)[16], distributed file systems (e.g., GFS [23]), resourcemanagement (e.g., Omega [52]), and distributed frame-works (e.g., MapReduce [20], BOOM [9]), to name afew. Ray draws inspiration from these pioneering efforts,but provides significant improvements. In contrast withSDNs, BOOM, and GFS, Ray decouples the storage ofthe control plane information (e.g., GCS) from the logicimplementation (e.g., schedulers). This allows both stor-age and computation layers to scale independently, whichis key to achieving our scalability targets. Omega usesa distributed architecture in which schedulers coordinatevia globally shared state. To this architecture, Ray addsglobal schedulers to balance load across local schedulers,and targets ms-level, not second-level, task scheduling.

Ray implements a unique distributed bottom-up sched-uler that is horizontally scalable, and can handle dynami-cally constructed task graphs. Unlike Ray, most existingcluster computing systems [20, 64, 40] use a centralizedscheduler architecture. While Sparrow [45] is decentral-ized, its schedulers make independent decisions, limitingthe possible scheduling policies, and all tasks of a job arehandled by the same global scheduler. Mesos [26] im-plements a two-level hierarchical scheduler, but its top-level scheduler manages frameworks, not individual tasks.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 573

Page 14: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

Canary [47] achieves impressive performance by havingeach scheduler instance handle a portion of the task graph,but does not handle dynamic computation graphs.

Cilk [12] is a parallel programming language whosework-stealing scheduler achieves provably efficient load-balancing for dynamic task graphs. However, with nocentral coordinator like Ray’s global scheduler, this fullyparallel design is also difficult to extend to support datalocality and resource heterogeneity in a distributed setting.

7 Discussion and Experiences

Building Ray has been a long journey. It started two yearsago with a Spark library to perform distributed trainingand simulations. However, the relative inflexibility of theBSP model, the high per-task overhead, and the lack of anactor abstraction led us to develop a new system. Since wereleased Ray roughly one year ago, several hundreds ofpeople have used it and several companies are running itin production. Here we discuss our experience developingand using Ray, and some early user feedback.

API. In designing the API, we have emphasized mini-malism. Initially we started with a basic task abstraction.Later, we added the wait() primitive to accommodate roll-outs with heterogeneous durations and the actor abstrac-tion to accommodate third-party simulators and amortizethe overhead of expensive initializations. While the re-sulting API is relatively low-level, it has proven both pow-erful and simple to use. We have already used this API toimplement many state-of-the-art RL algorithms on top ofRay, including A3C [36], PPO [51], DQN [37], ES [49],DDPG [55], and Ape-X [27]. In most cases it took usjust a few tens of lines of code to port these algorithms toRay. Based on early user feedback, we are consideringenhancing the API to include higher level primitives andlibraries, which could also inform scheduling decisions.

Limitations. Given the workload generality, special-ized optimizations are hard. For example, we must makescheduling decisions without full knowledge of the com-putation graph. Scheduling optimizations in Ray mightrequire more complex runtime profiling. In addition, stor-ing lineage for each task requires the implementation ofgarbage collection policies to bound storage costs in theGCS, a feature we are actively developing.

Fault tolerance. We are often asked if fault toleranceis really needed for AI applications. After all, due to thestatistical nature of many AI algorithms, one could sim-ply ignore failed rollouts. Based on our experience, ouranswer is “yes”. First, the ability to ignore failures makesapplications much easier to write and reason about. Sec-ond, our particular implementation of fault tolerance viadeterministic replay dramatically simplifies debugging asit allows us to easily reproduce most errors. This is par-ticularly important since, due to their stochasticity, AI al-

gorithms are notoriously hard to debug. Third, fault toler-ance helps save money since it allows us to run on cheapresources like spot instances on AWS. Of course, thiscomes at the price of some overhead. However, we foundthis overhead to be minimal for our target workloads.

GCS and Horizontal Scalability. The GCS dramati-cally simplified Ray development and debugging. It en-abled us to query the entire system state while debuggingRay itself, instead of having to manually expose internalcomponent state. In addition, the GCS is also the backendfor our timeline visualization tool, used for application-level debugging.

The GCS was also instrumental to Ray’s horizontalscalability. In Section 5, we were able to scale by addingmore shards whenever the GCS became a bottleneck. TheGCS also enabled the global scheduler to scale by sim-ply adding more replicas. Due to these advantages, webelieve that centralizing control state will be a key designcomponent of future distributed systems.

8 Conclusion

No general-purpose system today can efficiently supportthe tight loop of training, serving, and simulation. To ex-press these core building blocks and meet the demands ofemerging AI applications, Ray unifies task-parallel andactor programming models in a single dynamic task graphand employs a scalable architecture enabled by the globalcontrol store and a bottom-up distributed scheduler. Theprogramming flexibility, high throughput, and low laten-cies simultaneously achieved by this architecture is partic-ularly important for emerging artificial intelligence work-loads, which produce tasks diverse in their resource re-quirements, duration, and functionality. Our evaluationdemonstrates linear scalability up to 1.8 million tasks persecond, transparent fault tolerance, and substantial perfor-mance improvements on several contemporary RL work-loads. Thus, Ray provides a powerful combination of flex-ibility, performance, and ease of use for the developmentof future AI applications.

9 Acknowledgments

This research is supported in part by NSF CISE Expedi-tions Award CCF-1730628 and gifts from Alibaba, Ama-zon Web Services, Ant Financial, Arm, CapitalOne, Eric-sson, Facebook, Google, Huawei, Intel, Microsoft, Sco-tiabank, Splunk and VMware as well as by NSF grantDGE-1106400. We are grateful to our anonymous review-ers and our shepherd, Miguel Castro, for thoughtful feed-back, which helped improve the quality of this paper.

574 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 15: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

References[1] Akka. https://akka.io/.

[2] Apache Arrow. https://arrow.apache.org/.

[3] Dask Benchmarks. http://matthewrocklin.com/blog/work/2017/07/03/scaling.

[4] EC2 Instance Pricing. https://aws.amazon.com/ec2/pricing/on-demand/.

[5] OpenAI Baselines: high-quality implementations of reinforce-ment learning algorithms. https://github.com/openai/baselines.

[6] TensorFlow Serving. https://www.tensorflow.org/serving/.

[7] ABADI, M., BARHAM, P., CHEN, J., CHEN, Z., DAVIS, A.,DEAN, J., DEVIN, M., GHEMAWAT, S., IRVING, G., ISARD, M.,ET AL. TensorFlow: A system for large-scale machine learning.In Proceedings of the 12th USENIX Symposium on OperatingSystems Design and Implementation (OSDI). Savannah, Georgia,USA (2016).

[8] AGARWAL, A., BIRD, S., COZOWICZ, M., HOANG, L., LANG-FORD, J., LEE, S., LI, J., MELAMED, D., OSHRI, G., RIBAS,O., SEN, S., AND SLIVKINS, A. A multiworld testing decisionservice. arXiv preprint arXiv:1606.03966 (2016).

[9] ALVARO, P., CONDIE, T., CONWAY, N., ELMELEEGY, K.,HELLERSTEIN, J. M., AND SEARS, R. BOOM Analytics: ex-ploring data-centric, declarative programming for the cloud. InProceedings of the 5th European conference on Computer systems(2010), ACM, pp. 223–236.

[10] ARMSTRONG, J., VIRDING, R., WIKSTROM, C., ANDWILLIAMS, M. Concurrent programming in ERLANG.

[11] BEATTIE, C., LEIBO, J. Z., TEPLYASHIN, D., WARD, T.,WAINWRIGHT, M., KUTTLER, H., LEFRANCQ, A., GREEN, S.,VALDES, V., SADIK, A., ET AL. DeepMind Lab. arXiv preprintarXiv:1612.03801 (2016).

[12] BLUMOFE, R. D., AND LEISERSON, C. E. Scheduling mul-tithreaded computations by work stealing. J. ACM 46, 5 (Sept.1999), 720–748.

[13] BROCKMAN, G., CHEUNG, V., PETTERSSON, L., SCHNEIDER,J., SCHULMAN, J., TANG, J., AND ZAREMBA, W. OpenAI gym.arXiv preprint arXiv:1606.01540 (2016).

[14] BYKOV, S., GELLER, A., KLIOT, G., LARUS, J. R., PANDYA,R., AND THELIN, J. Orleans: Cloud computing for everyone. InProceedings of the 2nd ACM Symposium on Cloud Computing(2011), ACM, p. 16.

[15] CARBONE, P., EWEN, S., FORA, G., HARIDI, S., RICHTER,S., AND TZOUMAS, K. State management in Apache Flink:Consistent stateful distributed stream processing. Proc. VLDBEndow. 10, 12 (Aug. 2017), 1718–1729.

[16] CASADO, M., FREEDMAN, M. J., PETTIT, J., LUO, J., MCKE-OWN, N., AND SHENKER, S. Ethane: Taking control of the enter-prise. SIGCOMM Comput. Commun. Rev. 37, 4 (Aug. 2007), 1–12.

[17] CHAROUSSET, D., SCHMIDT, T. C., HIESGEN, R., ANDWAHLISCH, M. Native actors: A scalable software platform fordistributed, heterogeneous environments. In Proceedings of the2013 workshop on Programming based on actors, agents, and de-centralized control (2013), ACM, pp. 87–96.

[18] CHEN, T., LI, M., LI, Y., LIN, M., WANG, N., WANG, M.,XIAO, T., XU, B., ZHANG, C., AND ZHANG, Z. MXNet: Aflexible and efficient machine learning library for heterogeneousdistributed systems. In NIPS Workshop on Machine LearningSystems (LearningSys’16) (2016).

[19] CRANKSHAW, D., WANG, X., ZHOU, G., FRANKLIN, M. J.,GONZALEZ, J. E., AND STOICA, I. Clipper: A low-latencyonline prediction serving system. In 14th USENIX Symposiumon Networked Systems Design and Implementation (NSDI 17)(Boston, MA, 2017), USENIX Association, pp. 613–627.

[20] DEAN, J., AND GHEMAWAT, S. MapReduce: Simplified dataprocessing on large clusters. Commun. ACM 51, 1 (Jan. 2008),107–113.

[21] DENNIS, J. B., AND MISUNAS, D. P. A preliminary architecturefor a basic data-flow processor. In Proceedings of the 2Nd An-nual Symposium on Computer Architecture (New York, NY, USA,1975), ISCA ’75, ACM, pp. 126–132.

[22] GABRIEL, E., FAGG, G. E., BOSILCA, G., ANGSKUN, T., DON-GARRA, J. J., SQUYRES, J. M., SAHAY, V., KAMBADUR, P.,BARRETT, B., LUMSDAINE, A., CASTAIN, R. H., DANIEL,D. J., GRAHAM, R. L., AND WOODALL, T. S. Open MPI: Goals,concept, and design of a next generation MPI implementation. InProceedings, 11th European PVM/MPI Users’ Group Meeting(Budapest, Hungary, September 2004), pp. 97–104.

[23] GHEMAWAT, S., GOBIOFF, H., AND LEUNG, S.-T. The Googlefile system. 29–43.

[24] GONZALEZ, J. E., XIN, R. S., DAVE, A., CRANKSHAW, D.,FRANKLIN, M. J., AND STOICA, I. GraphX: Graph processingin a distributed dataflow framework. In Proceedings of the 11thUSENIX Conference on Operating Systems Design and Implemen-tation (Berkeley, CA, USA, 2014), OSDI’14, USENIX Associa-tion, pp. 599–613.

[25] GU*, S., HOLLY*, E., LILLICRAP, T., AND LEVINE, S. Deep re-inforcement learning for robotic manipulation with asynchronousoff-policy updates. In IEEE International Conference on Roboticsand Automation (ICRA 2017) (2017).

[26] HINDMAN, B., KONWINSKI, A., ZAHARIA, M., GHODSI, A.,JOSEPH, A. D., KATZ, R., SHENKER, S., AND STOICA, I.Mesos: A platform for fine-grained resource sharing in the datacenter. In Proceedings of the 8th USENIX Conference on Net-worked Systems Design and Implementation (Berkeley, CA, USA,2011), NSDI’11, USENIX Association, pp. 295–308.

[27] HORGAN, D., QUAN, J., BUDDEN, D., BARTH-MARON, G.,HESSEL, M., VAN HASSELT, H., AND SILVER, D. Distributedprioritized experience replay. International Conference on Learn-ing Representations (2018).

[28] ISARD, M., BUDIU, M., YU, Y., BIRRELL, A., AND FETTERLY,D. Dryad: Distributed data-parallel programs from sequentialbuilding blocks. In Proceedings of the 2nd ACM SIGOPS/EuroSysEuropean Conference on Computer Systems 2007 (New York, NY,USA, 2007), EuroSys ’07, ACM, pp. 59–72.

[29] JIA, Y., SHELHAMER, E., DONAHUE, J., KARAYEV, S., LONG,J., GIRSHICK, R., GUADARRAMA, S., AND DARRELL, T. Caffe:Convolutional architecture for fast feature embedding. arXivpreprint arXiv:1408.5093 (2014).

[30] JORDAN, M. I., AND MITCHELL, T. M. Machine learning:Trends, perspectives, and prospects. Science 349, 6245 (2015),255–260.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 575

Page 16: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

[31] LEIBIUSKY, J., EISBRUCH, G., AND SIMONASSI, D. GettingStarted with Storm. O’Reilly Media, Inc., 2012.

[32] LI, M., ANDERSEN, D. G., PARK, J. W., SMOLA, A. J.,AHMED, A., JOSIFOVSKI, V., LONG, J., SHEKITA, E. J., ANDSU, B.-Y. Scaling distributed machine learning with the parame-ter server. In Proceedings of the 11th USENIX Conference on Op-erating Systems Design and Implementation (Berkeley, CA, USA,2014), OSDI’14, pp. 583–598.

[33] LOOKS, M., HERRESHOFF, M., HUTCHINS, D., AND NORVIG,P. Deep learning with dynamic computation graphs. arXiv preprintarXiv:1702.02181 (2017).

[34] LOW, Y., GONZALEZ, J., KYROLA, A., BICKSON, D.,GUESTRIN, C., AND HELLERSTEIN, J. GraphLab: A new frame-work for parallel machine learning. In Proceedings of the Twenty-Sixth Conference on Uncertainty in Artificial Intelligence (Arling-ton, Virginia, United States, 2010), UAI’10, pp. 340–349.

[35] MALEWICZ, G., AUSTERN, M. H., BIK, A. J., DEHNERT, J. C.,HORN, I., LEISER, N., AND CZAJKOWSKI, G. Pregel: A systemfor large-scale graph processing. In Proceedings of the 2010 ACMSIGMOD International Conference on Management of Data (NewYork, NY, USA, 2010), SIGMOD ’10, ACM, pp. 135–146.

[36] MNIH, V., BADIA, A. P., MIRZA, M., GRAVES, A., LILLICRAP,T. P., HARLEY, T., SILVER, D., AND KAVUKCUOGLU, K. Asyn-chronous methods for deep reinforcement learning. In Interna-tional Conference on Machine Learning (2016).

[37] MNIH, V., KAVUKCUOGLU, K., SILVER, D., RUSU, A. A.,VENESS, J., BELLEMARE, M. G., GRAVES, A., RIEDMILLER,M., FIDJELAND, A. K., OSTROVSKI, G., ET AL. Human-levelcontrol through deep reinforcement learning. Nature 518, 7540(2015), 529–533.

[38] MURRAY, D. A Distributed Execution Engine Supporting Data-dependent Control Flow. University of Cambridge, 2012.

[39] MURRAY, D. G., MCSHERRY, F., ISAACS, R., ISARD, M.,BARHAM, P., AND ABADI, M. Naiad: A timely dataflow system.In Proceedings of the Twenty-Fourth ACM Symposium on Operat-ing Systems Principles (New York, NY, USA, 2013), SOSP ’13,ACM, pp. 439–455.

[40] MURRAY, D. G., SCHWARZKOPF, M., SMOWTON, C., SMITH,S., MADHAVAPEDDY, A., AND HAND, S. CIEL: A universal exe-cution engine for distributed data-flow computing. In Proceedingsof the 8th USENIX Conference on Networked Systems Design andImplementation (Berkeley, CA, USA, 2011), NSDI’11, USENIXAssociation, pp. 113–126.

[41] NAIR, A., SRINIVASAN, P., BLACKWELL, S., ALCICEK, C.,FEARON, R., MARIA, A. D., PANNEERSHELVAM, V., SULEY-MAN, M., BEATTIE, C., PETERSEN, S., LEGG, S., MNIH, V.,KAVUKCUOGLU, K., AND SILVER, D. Massively parallel meth-ods for deep reinforcement learning, 2015.

[42] NG, A., COATES, A., DIEL, M., GANAPATHI, V., SCHULTE, J.,TSE, B., BERGER, E., AND LIANG, E. Autonomous inverted he-licopter flight via reinforcement learning. Experimental RoboticsIX (2006), 363–372.

[43] NISHIHARA, R., MORITZ, P., WANG, S., TUMANOV, A., PAUL,W., SCHLEIER-SMITH, J., LIAW, R., NIKNAMI, M., JORDAN,M. I., AND STOICA, I. Real-time machine learning: The missingpieces. In Workshop on Hot Topics in Operating Systems (2017).

[44] OPENAI. OpenAI Dota 2 1v1 bot. https://openai.com/the-international/, 2017.

[45] OUSTERHOUT, K., WENDELL, P., ZAHARIA, M., AND STOICA,I. Sparrow: Distributed, low latency scheduling. In Proceedingsof the Twenty-Fourth ACM Symposium on Operating SystemsPrinciples (New York, NY, USA, 2013), SOSP ’13, ACM, pp. 69–84.

[46] PASZKE, A., GROSS, S., CHINTALA, S., CHANAN, G., YANG,E., DEVITO, Z., LIN, Z., DESMAISON, A., ANTIGA, L., ANDLERER, A. Automatic differentiation in PyTorch.

[47] QU, H., MASHAYEKHI, O., TEREI, D., AND LEVIS, P. Canary:A scheduling architecture for high performance cloud computing.arXiv preprint arXiv:1602.01412 (2016).

[48] ROCKLIN, M. Dask: Parallel computation with blocked algo-rithms and task scheduling. In Proceedings of the 14th Python inScience Conference (2015), K. Huff and J. Bergstra, Eds., pp. 130– 136.

[49] SALIMANS, T., HO, J., CHEN, X., AND SUTSKEVER, I. Evolu-tion strategies as a scalable alternative to reinforcement learning.arXiv preprint arXiv:1703.03864 (2017).

[50] SANFILIPPO, S. Redis: An open source, in-memory data structurestore. https://redis.io/, 2009.

[51] SCHULMAN, J., WOLSKI, F., DHARIWAL, P., RADFORD, A.,AND KLIMOV, O. Proximal policy optimization algorithms. arXivpreprint arXiv:1707.06347 (2017).

[52] SCHWARZKOPF, M., KONWINSKI, A., ABD-EL-MALEK, M.,AND WILKES, J. Omega: Flexible, scalable schedulers for largecompute clusters. In Proceedings of the 8th ACM European Con-ference on Computer Systems (New York, NY, USA, 2013), Eu-roSys ’13, ACM, pp. 351–364.

[53] SERGEEV, A., AND DEL BALSO, M. Horovod: fast andeasy distributed deep learning in tensorflow. arXiv preprintarXiv:1802.05799 (2018).

[54] SILVER, D., HUANG, A., MADDISON, C. J., GUEZ, A.,SIFRE, L., VAN DEN DRIESSCHE, G., SCHRITTWIESER, J.,ANTONOGLOU, I., PANNEERSHELVAM, V., LANCTOT, M.,ET AL. Mastering the game of Go with deep neural networks andtree search. Nature 529, 7587 (2016), 484–489.

[55] SILVER, D., LEVER, G., HEESS, N., DEGRIS, T., WIERSTRA,D., AND RIEDMILLER, M. Deterministic policy gradient algo-rithms. In ICML (2014).

[56] SUTTON, R. S., AND BARTO, A. G. Reinforcement Learning:An Introduction. MIT press Cambridge, 1998.

[57] THAKUR, R., RABENSEIFNER, R., AND GROPP, W. Optimiza-tion of collective communication operations in MPICH. The Inter-national Journal of High Performance Computing Applications19, 1 (2005), 49–66.

[58] TIAN, Y., GONG, Q., SHANG, W., WU, Y., AND ZITNICK, C. L.ELF: An extensive, lightweight and flexible research platformfor real-time strategy games. Advances in Neural InformationProcessing Systems (NIPS) (2017).

[59] TODOROV, E., EREZ, T., AND TASSA, Y. Mujoco: A physicsengine for model-based control. In Intelligent Robots and Systems(IROS), 2012 IEEE/RSJ International Conference on (2012), IEEE,pp. 5026–5033.

576 13th USENIX Symposium on Operating Systems Design and Implementation USENIX Association

Page 17: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the

[60] VAN DEN BERG, J., MILLER, S., DUCKWORTH, D., HU, H.,WAN, A., FU, X.-Y., GOLDBERG, K., AND ABBEEL, P. Su-perhuman performance of surgical tasks by robots using iterativelearning from human-guided demonstrations. In Robotics and Au-tomation (ICRA), 2010 IEEE International Conference on (2010),IEEE, pp. 2074–2081.

[61] VAN RENESSE, R., AND SCHNEIDER, F. B. Chain replication forsupporting high throughput and availability. In Proceedings of the6th Conference on Symposium on Opearting Systems Design &Implementation - Volume 6 (Berkeley, CA, USA, 2004), OSDI’04,USENIX Association.

[62] VENKATARAMAN, S., PANDA, A., OUSTERHOUT, K., GHODSI,A., ARMBRUST, M., RECHT, B., FRANKLIN, M., AND STOICA,I. Drizzle: Fast and adaptable stream processing at scale. InProceedings of the Twenty-Sixth ACM Symposium on OperatingSystems Principles (2017), SOSP ’17, ACM.

[63] WHITE, T. Hadoop: The Definitive Guide. O’Reilly Media, Inc.,2012.

[64] ZAHARIA, M., CHOWDHURY, M., DAS, T., DAVE, A., MA, J.,MCCAULEY, M., FRANKLIN, M. J., SHENKER, S., AND STO-ICA, I. Resilient distributed datasets: A fault-tolerant abstrac-tion for in-memory cluster computing. In Proceedings of the 9thUSENIX conference on Networked Systems Design and Implemen-tation (2012), USENIX Association, pp. 2–2.

[65] ZAHARIA, M., XIN, R. S., WENDELL, P., DAS, T., ARMBRUST,M., DAVE, A., MENG, X., ROSEN, J., VENKATARAMAN, S.,FRANKLIN, M. J., GHODSI, A., GONZALEZ, J., SHENKER, S.,AND STOICA, I. Apache Spark: A unified engine for big dataprocessing. Commun. ACM 59, 11 (Oct. 2016), 56–65.

USENIX Association 13th USENIX Symposium on Operating Systems Design and Implementation 577

Page 18: Ray: A Distributed Framework for Emerging AI Applicationsweb.eecs.umich.edu/~mosharaf/Readings/Ray.pdf · The next generation of AI applications will continuously interact with the