From 98a508d6ca9d870ab7b07a8b3712be2220fd51aa Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Sun, 31 Jul 2016 19:58:03 -0700 Subject: [PATCH] Terminology change Object Reference -> Object ID (#330) --- README.md | 12 +- doc/aliasing.md | 72 ++-- doc/reference-counting.md | 48 +-- doc/remote-functions.md | 8 +- doc/tutorial.md | 63 ++-- examples/alexnet/README.md | 8 +- examples/alexnet/alexnet.py | 46 +-- examples/alexnet/driver.py | 20 +- examples/hyperopt/README.md | 6 +- examples/hyperopt/driver.py | 4 +- examples/lbfgs/README.md | 28 +- examples/lbfgs/driver.py | 20 +- examples/rl_pong/README.md | 8 +- examples/rl_pong/driver.py | 8 +- include/ray/ray.h | 2 +- lib/python/ray/__init__.py | 2 +- lib/python/ray/array/distributed/core.py | 62 +-- lib/python/ray/array/distributed/linalg.py | 32 +- lib/python/ray/array/distributed/random.py | 6 +- lib/python/ray/graph.py | 6 +- lib/python/ray/serialization.py | 12 +- lib/python/ray/worker.py | 149 ++++---- protos/graph.proto | 8 +- protos/ray.proto | 70 ++-- protos/types.proto | 8 +- src/ipc.h | 10 +- src/objstore.cc | 140 +++---- src/objstore.h | 12 +- src/raylib.cc | 306 +++++++-------- src/scheduler.cc | 414 ++++++++++----------- src/scheduler.h | 104 +++--- src/worker.cc | 96 ++--- src/worker.h | 38 +- test/array_test.py | 32 +- test/memory_leak_deserialize.py | 2 +- test/runtest.py | 104 +++--- 36 files changed, 982 insertions(+), 984 deletions(-) diff --git a/README.md b/README.md index ef71f071e..4602b70a4 100644 --- a/README.md +++ b/README.md @@ -28,12 +28,12 @@ def estimate_pi(n): return 4 * np.mean(x ** 2 + y ** 2 < 1) # Launch 10 tasks, each of which estimates pi. -results = [] +result_ids = [] for _ in range(10): - results.append(estimate_pi.remote(100)) + result_ids.append(estimate_pi.remote(100)) # Fetch the results of the tasks and print their average. -estimate = np.mean([ray.get(ref) for ref in results]) +estimate = np.mean([ray.get(result_id) for result_id in result_ids]) print "Pi is approximately {}.".format(estimate) ``` @@ -41,10 +41,10 @@ Within the for loop, each call to `estimate_pi.remote(100)` sends a message to the scheduler asking it to schedule the task of running `estimate_pi` with the argument `100`. This call returns right away without waiting for the actual estimation of pi to take place. Instead of returning a float, it returns an -**object reference**, which represents the eventual output of the computation -(this is a similar to a Future). +**object ID**, which represents the eventual output of the computation (this is +a similar to a Future). -The call to `ray.get(ref)` takes an object reference and returns the actual +The call to `ray.get(result_id)` takes an object ID and returns the actual estimate of pi (waiting until the computation has finished if necessary). ## Next Steps diff --git a/doc/aliasing.md b/doc/aliasing.md index 38c5daa3b..1f227541e 100644 --- a/doc/aliasing.md +++ b/doc/aliasing.md @@ -1,13 +1,13 @@ # Aliasing An important feature of Ray is that a remote call sent to the scheduler -immediately returns object references to the outputs of the task, and the actual -outputs of the task are only associated with the relevant object references +immediately returns object ids for the outputs of the task, and the actual +outputs of the task are only associated with the relevant object ids after the task has been executed and the outputs have been computed. This allows the worker to continue without blocking. However, to provide a more flexible API, we allow tasks to not only return -values, but to also return object references to values. As an examples, consider +values, but to also return object ids to values. As an examples, consider the following code. ```python @ray.remote([], [np.ndarray]) @@ -22,44 +22,44 @@ def g() def h() return g() ``` -A call to `h` will immediate return an object reference `ref_h` for the return +A call to `h` will immediate return an object id `ref_h` for the return value of `h`. The task of executing `h` (call it `task_h`) will then be scheduled for execution. When `task_h` is executed, it will call `g`, which will -immediately return an object reference `ref_g` for the output of `g`. Then two -things will happen and can happen in any order: `AliasObjRefs(ref_h, ref_g)` +immediately return an object id `ref_g` for the output of `g`. Then two +things will happen and can happen in any order: `AliasObjectIDs(ref_h, ref_g)` will be called and `task_g` will be scheduled and executed. When `task_g` is -executed, it will call `f`, and immediately obtain an object reference `ref_f` +executed, it will call `f`, and immediately obtain an object id `ref_f` for the output of `f`. Then two things will happen and can happen in either -order, `AliasObjRefs(ref_g, ref_f)` will be called, and `f` will be executed. +order, `AliasObjectIDs(ref_g, ref_f)` will be called, and `f` will be executed. When `f` is executed, it will create an actual array and `put_object` will be called, which will store the array in the object store (it will also call -`SchedulerService::AddCanonicalObjRef(ref_f)`). +`SchedulerService::AddCanonicalObjectID(ref_f)`). From the scheduler's perspective, there are three important calls, -`AliasObjRefs(ref_h, ref_g)`, `AliasObjRefs(ref_g, ref_f)`, and -`AddCanonicalObjRef(ref_f)`. These three calls can happen in any order. +`AliasObjectIDs(ref_h, ref_g)`, `AliasObjectIDs(ref_g, ref_f)`, and +`AddCanonicalObjectID(ref_f)`. These three calls can happen in any order. -The scheduler maintains a data structure called `target_objrefs_`, which keeps -track of which object references have been aliased together (`target_objrefs_` +The scheduler maintains a data structure called `target_objectids_`, which keeps +track of which object ids have been aliased together (`target_objectids_` is a vector, but we can think of it as a graph). The call -`AliasObjRefs(ref_h, ref_g)` updates `target_objrefs_` with `ref_h -> ref_g`. -The call `AliasObjRefs(ref_g, ref_f)` updates it with `ref_g -> ref_f`, and the -call `AddCanonicalObjRef(ref_f)` updates it with `ref_f -> ref_f`. The data +`AliasObjectIDs(ref_h, ref_g)` updates `target_objectids_` with `ref_h -> ref_g`. +The call `AliasObjectIDs(ref_g, ref_f)` updates it with `ref_g -> ref_f`, and the +call `AddCanonicalObjectID(ref_f)` updates it with `ref_f -> ref_f`. The data structure is initialized with `ref -> UNINITIALIZED_ALIAS` for each object -reference `ref`. +id `ref`. -We refer to `ref_f` as a "canonical" object reference. And in a pair such as -`ref_h -> ref_g`, we refer to `ref_h` as the "alias" object reference and to -`ref_g` as the "target" object reference. These details are available to the -scheduler, but a worker process just has an object reference and doesn't know if +We refer to `ref_f` as a "canonical" object id. And in a pair such as +`ref_h -> ref_g`, we refer to `ref_h` as the "alias" object id and to +`ref_g` as the "target" object id. These details are available to the +scheduler, but a worker process just has an object id and doesn't know if it is canonical or not. -We also maintain a data structure `reverse_target_objrefs_`, which maps in the +We also maintain a data structure `reverse_target_objectids_`, which maps in the reverse direction (in the above example, we would have `ref_g -> ref_h`, `ref_f -> ref_g`, and `ref_h -> UNINITIALIZED_ALIAS`). This data structure is -not particuarly important for the task of aliasing, but when we do reference +not particuarly important for the task of aliasing, but when we do id counting and attempt to deallocate an object, we need to be able to determine -all of the object references that refer to the same object, and this data +all of the object ids that refer to the same object, and this data structure comes in handy for that purpose. ## Gets and Remote Calls @@ -67,29 +67,29 @@ structure comes in handy for that purpose. When a worker calls `ray.get(ref)`, it first sends a message to the scheduler asking the scheduler to ship the object referred to by `ref` to the worker's local object store. Then the worker asks its local object store for the object -referred to by `ref`. If `ref` is a canonical object reference, then that's all -there is too it. However, if `ref` is not a canonical object reference but -rather is an alias for the canonical object reference `c_ref`, then the +referred to by `ref`. If `ref` is a canonical object id, then that's all +there is too it. However, if `ref` is not a canonical object id but +rather is an alias for the canonical object id `c_ref`, then the scheduler also notifies the worker's local object store that `ref` is an alias for `c_ref`. This is important because the object store does not keep track of aliasing on its own (it only knows the bits about aliasing that the scheduler tells it). Lastly, if the scheduler does not yet have enough information to determine if `ref` is canonical, or if the scheduler cannot -yet determine what the canonical object reference for `ref` is, then the +yet determine what the canonical object id for `ref` is, then the scheduler will wait until it has the relevant information. Similar things happen when a worker performs a remote call. If an object -reference is passed to a remote call, the object referred to by that object -reference will be shipped to the local object store of the worker that executes +id is passed to a remote call, the object referred to by that object +id will be shipped to the local object store of the worker that executes the task. The scheduler will notify that object store about any aliasing that it needs to be aware of. -## Passing Object References by Value +## Passing Object ids by Value Currently, the graph of aliasing looks like a collection of chains, as in the above example with `ref_h -> ref_g -> ref_f -> ref_f`. In the future, we will -allow object references to be passed by value to remote calls (so the worker -has access to the object reference object and not the object that the object -reference refers to). If an object reference that is passed by value is then -returned by the task, it is possible that a given object reference could be -the target of multiple alias object references. In this case, the graph of +allow object ids to be passed by value to remote calls (so the worker +has access to the object id object and not the object that the object +id refers to). If an object id that is passed by value is then +returned by the task, it is possible that a given object id could be +the target of multiple alias object ids. In this case, the graph of aliasing will be a tree. diff --git a/doc/reference-counting.md b/doc/reference-counting.md index 9b6919acb..11819dc05 100644 --- a/doc/reference-counting.md +++ b/doc/reference-counting.md @@ -1,10 +1,10 @@ # Reference Counting -In Ray, each object is assigned a globally unique object reference by the +In Ray, each object is assigned a globally unique object ID by the scheduler (starting with 0 and incrementing upward). The objects are stored in object stores. In order to avoid running out of memory, the object stores must know when it is ok to deallocate an object. Since a worker on one node may have -an object reference for an object that lives in an object store on a different +an object ID for an object that lives in an object store on a different node, knowing when we can safely deallocate an object requires cluster-wide information. @@ -20,12 +20,12 @@ worker processes are Python processes). However, this could be made to work with worker processes that use garbage collection, for example, if each worker process is a Java Virtual Machine. -At a high level, the scheduler keeps track of the number of object references -that exist on the cluster for each object reference. When the number of object +At a high level, the scheduler keeps track of the number of object IDs +that exist on the cluster for each object ID. When the number of object references reaches 0 for a particular object, the scheduler notifies all of the object stores that contain that object to deallocate it. -Object references can exist in several places. +object IDs can exist in several places. 1. They can be Python objects on a worker. 2. They can be serialized within an object in an object store. @@ -37,41 +37,41 @@ to a remote procedure call). We handle these three cases by calling the SchedulerService methods `IncrementRefCount` and `DecrementRefCount` as follows: -1. To handle the first case, we increment in the ObjRef constructor and -decrement in the ObjRef destructor. +1. To handle the first case, we increment in the ObjectID constructor and +decrement in the ObjectID destructor. 2. To handle the second case, when an object is written to an object store with -a call to `put_object`, we call `IncrementRefCount` for each object reference +a call to `put_object`, we call `IncrementRefCount` for each object ID that is contained internally in the serialized object (for example, if we serialize a `DistArray`, we increment the reference counts for its blocks). This -will notify the scheduler that those object references are in the object store. +will notify the scheduler that those object IDs are in the object store. Then when the scheduler deallocates the object, we call `DecrementRefCount` for -the object references that it holds internally (the scheduler keeps track of -these internal object references in the `contained_objrefs_` data structure). +the object IDs that it holds internally (the scheduler keeps track of +these internal object IDs in the `contained_objectids_` data structure). 3. To handle the third case, we increment in the `serialize_task` method and decrement in the `deserialize_task` method. ## How to Handle Aliasing -Reference counting interacts with aliasing. Since multiple object references +Reference counting interacts with aliasing. Since multiple object IDs may refer to the same object, we cannot deallocate that object until all of the -object references that refer to it have reference counts of 0. We keep track of -the number of separate aliases separately. If two object references refer to the +object IDs that refer to it have reference counts of 0. We keep track of +the number of separate aliases separately. If two object IDs refer to the same object, the scheduler keeps track the number of occurrences of each of -those object references separately. This simplifies the scheduler's job because -it may not always know if two object references refer to the same object or not +those object IDs separately. This simplifies the scheduler's job because +it may not always know if two object IDs refer to the same object or not (since it assigns them before hearing back about what they refer to). -When we decrement the count for an object reference, if the count reaches 0, -we compute all object references that the scheduler knows to reference the same -object. If these object references all have count 0, then we deallocate the +When we decrement the count for an object ID, if the count reaches 0, +we compute all object IDs that the scheduler knows to reference the same +object. If these object IDs all have count 0, then we deallocate the object. Otherwise, we do not deallocate the object. -You may ask, what if there is some object reference with a nonzero count which +You may ask, what if there is some object ID with a nonzero count which refers to the same object, but the scheduler does not know it? This cannot happen because the following invariant holds. If `a` and `b` are object references that will be aliased together (through a call to -`AliasObjRefs(a, b)`), then either the call has already happened, or both `a` +`AliasObjectIDs(a, b)`), then either the call has already happened, or both `a` and `b` have positive reference counts (they must have positive reference counts -because they must be passed into `AliasObjRefs` at some point). +because they must be passed into `AliasObjectIDs` at some point). ## Complications The following problem has not yet been resolved. In the following code, the @@ -80,10 +80,10 @@ result `x` will be garbage. x = ray.get(ra.zeros([10, 10], "float")) ``` When `ra.zeros` is called, a worker will create an array of zeros and store -it in an object store. An object reference to the output is returned. The call +it in an object store. An object ID to the output is returned. The call to `ray.get` will not copy data from the object store process to the worker process, but will instead give the worker process a pointer to shared memory. -After the `ray.get` call completes, the object reference returned by +After the `ray.get` call completes, the object ID returned by `ra.zeros` will go out of scope, and the object it refers to will be deallocated from the object store. This will cause the memory that `x` points to to be garbage. diff --git a/doc/remote-functions.md b/doc/remote-functions.md index 7f28df99a..bfae67341 100644 --- a/doc/remote-functions.md +++ b/doc/remote-functions.md @@ -21,16 +21,16 @@ implementation is run). **Invocation:** To invoke a remote function, the user calls the remote function. ```python -ref = increment(1) +x_id = increment(1) ``` This line sends a message to the scheduler asking it to schedule the task of executing the function `increment` with the argument `1`. It then returns an -object reference for the eventual output of the task. This takes place almost +object id for the eventual output of the task. This takes place almost instantaneously and does not wait for the actual task to be executed. When calling a remote function, the return value always consists of one or more -object references. If you want the actual value, call `ray.get(ref)`, which will +object ids. If you want the actual value, call `ray.get(x_id)`, which will wait for the task to execute and will return the resulting value. **Execution:** Eventually, the scheduler will schedule the task on a worker. The @@ -49,7 +49,7 @@ types in the object store. **The serializable types are:** 1. Primitive data types (for example, `1`, `1.0`, `"hello"`, `True`) 2. Numpy arrays -3. Object references +3. Object IDs 4. Lists, tuples, and dictionaries of other serializable types, but excluding custom classes (for example, `[1, 1.0, "hello"]`, `{True: "hi", 1: ["hi"]}`) 5. Custom classes where the user has provided `serialize` and `desererialize` diff --git a/doc/tutorial.md b/doc/tutorial.md index 91bfbba43..541dacb8b 100644 --- a/doc/tutorial.md +++ b/doc/tutorial.md @@ -2,7 +2,7 @@ To use Ray, you need to understand the following: -- How Ray uses object references to represent immutable remote objects. +- How Ray uses object IDs to represent immutable remote objects. - How Ray constructs a computation graph using remote functions. ## Overview @@ -42,13 +42,13 @@ killall scheduler objstore python ## Immutable remote objects In Ray, we can create and manipulate objects. We refer to these objects as -**remote objects**, and we use **object references** to refer to them. Remote +**remote objects**, and we use **object IDs** to refer to them. Remote objects are stored in **object stores**, and there is one object store per node in the cluster. In the cluster setting, we may not actually know which machine each object lives on. -An **object reference** is essentially a unique ID that can be used to refer to -a remote object. If you're familiar with Futures, our object references are +An **object ID** is essentially a unique ID that can be used to refer to +a remote object. If you're familiar with Futures, our object IDs are conceptually similar. We assume that remote objects are immutable. That is, their values cannot be @@ -58,11 +58,11 @@ object stores without needing to synchronize the copies. ### Put and Get The commands `ray.get` and `ray.put` can be used to convert between Python -objects and object references, as shown in the example below. +objects and object IDs, as shown in the example below. ```python x = [1, 2, 3] -ray.put(x) # prints +ray.put(x) # prints ``` The command `ray.put(x)` would be run by a worker process or by the driver @@ -71,28 +71,28 @@ object and copies it to the local object store (here *local* means *on the same node*). Once the object has been stored in the object store, its value cannot be changed. -In addition, `ray.put(x)` returns an object reference, which is essentially an +In addition, `ray.put(x)` returns an object ID, which is essentially an ID that can be used to refer to the newly created remote object. If we save the -object reference in a variable with `ref = ray.put(x)`, then we can pass `ref` +object ID in a variable with `x_id = ray.put(x)`, then we can pass `x_id` into remote functions, and those remote functions will operate on the corresponding remote object. -The command `ray.get(ref)` takes an object reference and creates a Python object +The command `ray.get(x_id)` takes an object ID and creates a Python object from the corresponding remote object. For some objects like arrays, we can use shared memory and avoid copying the object. For other objects, this currently copies the object from the object store into the memory of the worker process. -If the remote object corresponding to the object reference `ref` does not live -on the same node as the worker that calls `ray.get(ref)`, then the remote object +If the remote object corresponding to the object ID `x_id` does not live +on the same node as the worker that calls `ray.get(x_id)`, then the remote object will first be copied from an object store that has it to the object store that needs it. ```python -ref = ray.put([1, 2, 3]) -ray.get(ref) # prints [1, 2, 3] +x_id = ray.put([1, 2, 3]) +ray.get(x_id) # prints [1, 2, 3] ``` -If the remote object corresponding to the object reference `ref` has not been -created yet, *the command `ray.get(ref)` will wait until the remote object has +If the remote object corresponding to the object ID `x_id` has not been +created yet, *the command `ray.get(id)` will wait until the remote object has been created.* ## Computation graphs in Ray @@ -122,23 +122,23 @@ function is called instead of catching them when the task is actually executed ### Remote functions Whereas in regular Python, calling `add(1, 2)` would return `3`, in Ray, calling -`add.remote(1, 2)` does not actually execute the task. Instead, it adds a task -to the computation graph and immediately returns an object reference to the -output of the computation. +`add.remote(1, 2)` does not actually execute the task. Instead, it adds a task to the +computation graph and immediately returns an object ID to the output of +the computation. ```python -ref = add.remote(1, 2) -ray.get(ref) # prints 3 +x_id = add.remote(1, 2) +ray.get(x_id) # prints 3 ``` There is a sharp distinction between *submitting a task* and *executing the task*. When a remote function is called, the task of executing that function is submitted to the scheduler, and the scheduler immediately returns object -references for the outputs of the task. However, the task will not be executed +ids for the outputs of the task. However, the task will not be executed until the scheduler actually schedules the task on a worker. When a task is submitted, each argument may be passed in by value or by object -reference. For example, these lines have the same behavior. +id. For example, these lines have the same behavior. ```python add.remote(1, 2) @@ -146,12 +146,11 @@ add.remote(1, ray.put(2)) add.remote(ray.put(1), ray.put(2)) ``` -Remote functions never return actual values, they always return object -references. +Remote functions never return actual values, they always return object IDs. When the remote function is actually executed, it operates on Python objects. -That is, if the remote function was called with any object references, the -Python objects corresponding to those object references will be retrieved and +That is, if the remote function was called with any object IDs, the +Python objects corresponding to those object IDs will be retrieved and passed into the actual execution of the remote function. ### Blocking computation @@ -195,12 +194,12 @@ Then we can write ```python # Submit ten tasks to the scheduler. This finishes almost immediately. -result_refs = [] +result_ids = [] for i in range(10): - result_refs.append(sleep.remote(5)) + result_ids.append(sleep.remote(5)) # Wait for the results. If we have at least ten workers, this takes 5 seconds. -[ray.get(ref) for ref in result_refs] # prints [0, 0, 0, 0, 0, 0, 0, 0, 0, 0] +[ray.get(id) for id in result_ids] # prints [0, 0, 0, 0, 0, 0, 0, 0, 0, 0] ``` The for loop simply adds ten tasks to the computation graph, with no @@ -248,9 +247,9 @@ def dot(a, b): Then we run ```python -aref = zeros.remote([10, 10]) -bref = zeros.remote([10, 10]) -cref = dot.remote(aref, bref) +a_id = zeros.remote([10, 10]) +b_id = zeros.remote([10, 10]) +c_id = dot.remote(a_id, b_id) ``` The corresponding computation graph looks like this. diff --git a/examples/alexnet/README.md b/examples/alexnet/README.md index a461c7ad0..23cde1440 100644 --- a/examples/alexnet/README.md +++ b/examples/alexnet/README.md @@ -68,7 +68,7 @@ def load_tarfile_from_s3(bucket, s3_key, size=[]): To load data in parallel, we simply call this function multiple times with the keys of all the objects that we want to retrieve. This returns a list of pairs -of object references, where the first object reference in each pair refers to a +of object IDs, where the first object ID in each pair refers to a batch of images and the second refers to the corresponding batch of labels. ```python @@ -100,9 +100,9 @@ We can parallelize the computation of the gradient over multiple batches by calling `compute_grad` multiple times in parallel. ```python -gradient_refs = [] +gradient_ids = [] for i in range(num_workers): # Choose a random batch and use it to compute the gradient of the loss. - x_ref, y_ref = batches[np.random.randint(len(batches))] - gradient_refs.append(compute_grad.remote(x_ref, y_ref, mean_ref, weights_ref)) + x_id, y_id = batches[np.random.randint(len(batches))] + gradient_ids.append(compute_grad.remote(x_id, y_id, mean_id, weights_id)) ``` diff --git a/examples/alexnet/alexnet.py b/examples/alexnet/alexnet.py index bb8dc1165..cb07aa211 100644 --- a/examples/alexnet/alexnet.py +++ b/examples/alexnet/alexnet.py @@ -67,11 +67,13 @@ def load_tarfiles_from_s3(bucket, s3_keys, size=[]): Args: bucket (str): Bucket holding the imagenet .tars. - s3_keys (List[str]): List of s3 keys from which the .tar files are being loaded. - size (List[int]): Resize the image to this size if size != []; len(size) == 2 required. + s3_keys (List[str]): List of s3 keys from which the .tar files are being + loaded. + size (List[int]): Resize the image to this size if size does not equal []. + The length of size must be 2. Returns: - np.ndarray: Contains object references to the chunks of the images (see load_chunk). + np.ndarray: Contains object IDs to the chunks of the images (see load_chunk). """ return [load_tarfile_from_s3.remote(bucket, s3_key, size) for s3_key in s3_keys] @@ -239,23 +241,23 @@ def num_images(batches): Returns: int: The number of images """ - shape_refs = [ra.shape.remote(batch) for batch in batches] - return sum([ray.get(shape_ref)[0] for shape_ref in shape_refs]) + shape_ids = [ra.shape.remote(batch) for batch in batches] + return sum([ray.get(shape_id)[0] for shape_id in shape_ids]) @ray.remote([List], [np.ndarray]) def compute_mean_image(batches): """Computes the mean image given a list of batches of images. Args: - batches (List[ObjRef]): A list of batches of images. + batches (List[ObjectID]): A list of batches of images. Returns: ndarray: The mean image """ if len(batches) == 0: raise Exception("No images were passed into `compute_mean_image`.") - sum_image_refs = [ra.sum.remote(batch, axis=0) for batch in batches] - sum_images = [ray.get(ref) for ref in sum_image_refs] + sum_image_ids = [ra.sum.remote(batch, axis=0) for batch in batches] + sum_images = [ray.get(sum_image_id) for sum_image_id in sum_image_ids] n_images = num_images.remote(batches) return np.sum(sum_images, axis=0).astype("float64") / ray.get(n_images) @@ -290,18 +292,16 @@ def shuffle_pair(first_batch, second_batch): """Shuffle two batches of data. Args: - first_batch (Tuple[ObjRef. ObjRef]): The first batch to be shuffled. The - first component is the object reference of a batch of images, and the - second component is the object reference of the corresponding batch of - labels. - second_batch (Tuple[ObjRef, ObjRef]): The second batch to be shuffled. The - first component is the object reference of a batch of images, and the - second component is the object reference of the corresponding batch of - labels. + first_batch (Tuple[ObjectID. ObjectID]): The first batch to be shuffled. The + first component is the object ID of a batch of images, and the second + component is the object ID of the corresponding batch of labels. + second_batch (Tuple[ObjectID, ObjectID]): The second batch to be shuffled. + The first component is the object ID of a batch of images, and the second + component is the object ID of the corresponding batch of labels. Returns: - Tuple[ObjRef, Objref]: The first batch of shuffled data. - Tuple[ObjRef, Objref]: Two second bach of shuffled data. + Tuple[ObjectID, ObjectID]: The first batch of shuffled data. + Tuple[ObjectID, ObjectID]: Two second bach of shuffled data. """ images1, labels1, images2, labels2 = shuffle_arrays.remote(first_batch[0], first_batch[1], second_batch[0], second_batch[1]) return (images1, labels1), (images2, labels2) @@ -361,13 +361,13 @@ def shuffle(batches): the data between the two members. Args: - batches (List[Tuple[ObjRef, ObjRef]]): This is a list of tuples, where each - tuple consists of two object references. The first component is an object - reference for a batch of images, and the second component is an object - reference for the corresponding batch of labels. + batches (List[Tuple[ObjectID, ObjectID]]): This is a list of tuples, where + each tuple consists of two object IDs. The first component is an object ID + for a batch of images, and the second component is an object ID for the + corresponding batch of labels. Returns: - List[Tuple[ObjRef, ObjRef]]: The shuffled data. + List[Tuple[ObjectID, ObjectID]]: The shuffled data. """ # Randomly permute the order of the batches. permuted_batches = np.random.permutation(batches) diff --git a/examples/alexnet/driver.py b/examples/alexnet/driver.py index 92eb1f196..457bf56f7 100644 --- a/examples/alexnet/driver.py +++ b/examples/alexnet/driver.py @@ -37,17 +37,17 @@ if __name__ == "__main__": filename_label_str = label_file["Body"].read().strip().split("\n") filename_label_pairs = [line.split(" ") for line in filename_label_str] filename_label_dict = dict([(os.path.basename(name), label) for name, label in filename_label_pairs]) - filename_label_dict_ref = ray.put(filename_label_dict) + filename_label_dict_id = ray.put(filename_label_dict) print "Labels extracted" # Download the imagenet dataset. imagenet_data = alexnet.load_tarfiles_from_s3(args.s3_bucket, image_tar_files, [256, 256]) # Convert the parsed filenames to integer labels and create batches. - batches = [(images, alexnet.filenames_to_labels.remote(filenames, filename_label_dict_ref)) for images, filenames in imagenet_data] + batches = [(images, alexnet.filenames_to_labels.remote(filenames, filename_label_dict_id)) for images, filenames in imagenet_data] # Compute the mean image. - mean_ref = alexnet.compute_mean_image.remote([images for images, labels in batches]) + mean_id = alexnet.compute_mean_image.remote([images for images, labels in batches]) # The data does not start out shuffled. Images of the same class all appear # together, so we shuffle it ourselves here. Each shuffle pairs up the batches @@ -67,24 +67,24 @@ if __name__ == "__main__": # Extract weights from the local copy of the network. weights = sess.run(parameters) # Put weights in the object store. - weights_ref = ray.put(weights) + weights_id = ray.put(weights) # Compute the accuracy on a random training batch. - x_ref, y_ref = batches[np.random.randint(len(batches))] - accuracy = alexnet.compute_accuracy.remote(x_ref, y_ref, weights_ref) + x_id, y_id = batches[np.random.randint(len(batches))] + accuracy = alexnet.compute_accuracy.remote(x_id, y_id, weights_id) # Launch tasks in parallel to compute the gradients for some batches. - gradient_refs = [] + gradient_ids = [] for i in range(num_workers - 1): # Choose a random batch and use it to compute the gradient of the loss. - x_ref, y_ref = batches[np.random.randint(len(batches))] - gradient_refs.append(alexnet.compute_grad.remote(x_ref, y_ref, mean_ref, weights_ref)) + x_id, y_id = batches[np.random.randint(len(batches))] + gradient_ids.append(alexnet.compute_grad.remote(x_id, y_id, mean_id, weights_id)) # Print the accuracy on a random training batch. print "Iteration {}: accuracy = {:.3}%".format(iteration, 100 * ray.get(accuracy)) # Fetch the gradients. This blocks until the gradients have been computed. - gradient_sets = [ray.get(ref) for ref in gradient_refs] + gradient_sets = [ray.get(gradient_id) for gradient_id in gradient_ids] # Average the gradients over all of the tasks. mean_gradients = [np.mean([gradient_set[i] for gradient_set in gradient_sets], axis=0) for i in range(len(weights))] # Use the gradients to update the network. diff --git a/examples/hyperopt/README.md b/examples/hyperopt/README.md index 18ea371a7..e8cb4e8d4 100644 --- a/examples/hyperopt/README.md +++ b/examples/hyperopt/README.md @@ -93,7 +93,7 @@ little bit of type information (the input is a dictionary along with some numpy arrays, and the return value is a float). Now a call to `train_cnn_and_compute_accuracy` does not execute the function. It -submits the task to the scheduler and returns an object reference for the output +submits the task to the scheduler and returns an object ID for the output of the eventual computation. The scheduler, at its leisure, will schedule the task on a worker (which may live on the same machine or on a different machine in the cluster). @@ -102,7 +102,7 @@ Now the for loop runs almost instantaneously because it does not do any actual computation. Instead, it simply submits a number of tasks to the scheduler. ```python -result_refs = [] +result_ids = [] for _ in range(100): params = generate_random_params() results.append((params, train_cnn_and_compute_accuracy.remote(params, epochs))) @@ -112,7 +112,7 @@ If we wish to wait until the results have all been retrieved, we can retrieve their values with `ray.get`. ```python -results = [(params, ray.get(ref)) for (params, ref) in result_refs] +results = [(params, ray.get(result_id)) for (params, result_id) in result_ids] ``` ## Additional notes diff --git a/examples/hyperopt/driver.py b/examples/hyperopt/driver.py index 1e5c9da88..c988b29a6 100644 --- a/examples/hyperopt/driver.py +++ b/examples/hyperopt/driver.py @@ -41,8 +41,8 @@ if __name__ == "__main__": # Fetch the results of the tasks and print the results. for i in range(trials): - params, ref = results[i] - accuracy = ray.get(ref) + params, result_id = results[i] + accuracy = ray.get(result_id) print """We achieve accuracy {:.3}% with learning_rate: {:.2} batch_size: {} diff --git a/examples/lbfgs/README.md b/examples/lbfgs/README.md index f7a0b779d..8e0cf8c04 100644 --- a/examples/lbfgs/README.md +++ b/examples/lbfgs/README.md @@ -81,7 +81,7 @@ on a number of workers or machines. First, let's turn the data into a collection of remote objects. ```python -batch_refs = [(ray.put(xs), ray.put(ys)) for (xs, ys) in batches] +batch_ids = [(ray.put(xs), ray.put(ys)) for (xs, ys) in batches] ``` We can load the data on the driver and distribute it this way because MNIST @@ -111,35 +111,35 @@ gradient. ```python def full_loss(theta): - theta_ref = ray.put(theta) - loss_refs = [loss.remote(theta_ref, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] - return sum([ray.get(loss_ref) for loss_ref in loss_refs]) + theta_id = ray.put(theta) + loss_ids = [loss.remote(theta_id, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] + return sum([ray.get(loss_id) for loss_id in loss_ids]) def full_grad(theta): - theta_ref = ray.put(theta) - grad_refs = [grad.remote(theta_ref, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] - return sum([ray.get(grad_ref) for grad_ref in grad_refs]).astype("float64") # This conversion is necessary for use with fmin_l_bfgs_b. + theta_id = ray.put(theta) + grad_ids = [grad.remote(theta_id, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] + return sum([ray.get(grad_id) for grad_id in grad_ids]).astype("float64") # This conversion is necessary for use with fmin_l_bfgs_b. ``` -Note that we turn `theta` into a remote object with the line `theta_ref = +Note that we turn `theta` into a remote object with the line `theta_id = ray.put(theta)` before passing it into the remote functions. If we had written ```python -[loss.remote(theta, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] +[loss.remote(theta, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] ``` instead of ```python -theta_ref = ray.put(theta) -[loss.remote(theta_ref, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] +theta_id = ray.put(theta) +[loss.remote(theta_id, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] ``` then each task that got sent to the scheduler (one for every element of -`batch_refs`) would have had a copy of `theta` serialized inside of it. Since +`batch_ids`) would have had a copy of `theta` serialized inside of it. Since `theta` here consists of the parameters of a potentially large model, this is -inefficient. *Large objects should be passed by object reference to remote -functions and not by value*. +inefficient. *Large objects should be passed by object ID to remote functions +and not by value*. We use remote functions and remote objects internally in the implementation of `full_loss` and `full_grad`, but the user-facing behavior of these methods is diff --git a/examples/lbfgs/driver.py b/examples/lbfgs/driver.py index cdb578596..b18afd655 100644 --- a/examples/lbfgs/driver.py +++ b/examples/lbfgs/driver.py @@ -78,24 +78,24 @@ if __name__ == "__main__": # Compute the loss on the entire dataset. def full_loss(theta): - theta_ref = ray.put(theta) - loss_refs = [loss.remote(theta_ref, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] - return sum([ray.get(loss_ref) for loss_ref in loss_refs]) + theta_id = ray.put(theta) + loss_ids = [loss.remote(theta_id, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] + return sum([ray.get(loss_id) for loss_id in loss_ids]) # Compute the gradient of the loss on the entire dataset. def full_grad(theta): - theta_ref = ray.put(theta) - grad_refs = [grad.remote(theta_ref, xs_ref, ys_ref) for (xs_ref, ys_ref) in batch_refs] - return sum([ray.get(grad_ref) for grad_ref in grad_refs]).astype("float64") # This conversion is necessary for use with fmin_l_bfgs_b. + theta_id = ray.put(theta) + grad_ids = [grad.remote(theta_id, xs_id, ys_id) for (xs_id, ys_id) in batch_ids] + return sum([ray.get(grad_id) for grad_id in grad_ids]).astype("float64") # This conversion is necessary for use with fmin_l_bfgs_b. # From the perspective of scipy.optimize.fmin_l_bfgs_b, full_loss is simply a # function which takes some parameters theta, and computes a loss. Similarly, # full_grad is a function which takes some parameters theta, and computes the # gradient of the loss. Internally, these functions use Ray to distribute the # computation of the loss and the gradient over the data that is represented - # by the remote object references is x_batches and y_batches and which is - # potentially distributed over a cluster. However, these details are hidden - # from scipy.optimize.fmin_l_bfgs_b, which simply uses it to run the L-BFGS + # by the remote object IDs x_batches and y_batches and which is potentially + # distributed over a cluster. However, these details are hidden from + # scipy.optimize.fmin_l_bfgs_b, which simply uses it to run the L-BFGS # algorithm. # Load the mnist data and turn the data into remote objects. @@ -104,7 +104,7 @@ if __name__ == "__main__": batch_size = 100 num_batches = mnist.train.num_examples / batch_size batches = [mnist.train.next_batch(batch_size) for _ in range(num_batches)] - batch_refs = [(ray.put(xs), ray.put(ys)) for (xs, ys) in batches] + batch_ids = [(ray.put(xs), ray.put(ys)) for (xs, ys) in batches] # Initialize the weights for the network to the vector of all zeros. theta_init = 1e-2 * np.random.normal(size=dim) diff --git a/examples/rl_pong/README.md b/examples/rl_pong/README.md index 9a761572e..907624fe0 100644 --- a/examples/rl_pong/README.md +++ b/examples/rl_pong/README.md @@ -50,13 +50,13 @@ perform rollouts and compute gradients. If we have at least ten worker processes, then these tasks will all be executed in parallel. ```python -model_ref = ray.put(model) +model_id = ray.put(model) grads, reward_sums = [], [] # Launch tasks to compute gradients from multiple rollouts in parallel. for i in range(10): - grad_ref, reward_sum_ref = compute_gradient.remote(model_ref) - grads.append(grad_ref) - reward_sums.append(reward_sum_ref) + grad_id, reward_sum_id = compute_gradient.remote(model_id) + grads.append(grad_id) + reward_sums.append(reward_sum_id) ``` ### Reusing the Gym environment diff --git a/examples/rl_pong/driver.py b/examples/rl_pong/driver.py index 331a4ca34..a83f8c6af 100644 --- a/examples/rl_pong/driver.py +++ b/examples/rl_pong/driver.py @@ -123,13 +123,13 @@ if __name__ == "__main__": rmsprop_cache = {k: np.zeros_like(v) for k, v in model.iteritems()} # rmsprop memory while True: - model_ref = ray.put(model) + model_id = ray.put(model) grads, reward_sums = [], [] # Launch tasks to compute gradients from multiple rollouts in parallel. for i in range(batch_size): - grad_ref, reward_sum_ref = compute_gradient.remote(model_ref) - grads.append(grad_ref) - reward_sums.append(reward_sum_ref) + grad_id, reward_sum_id = compute_gradient.remote(model_id) + grads.append(grad_id) + reward_sums.append(reward_sum_id) for i in range(batch_size): grad = ray.get(grads[i]) reward_sum = ray.get(reward_sums[i]) diff --git a/include/ray/ray.h b/include/ray/ray.h index 242541537..e2c32e98a 100644 --- a/include/ray/ray.h +++ b/include/ray/ray.h @@ -6,7 +6,7 @@ #include #include "logging.h" -typedef size_t ObjRef; +typedef size_t ObjectID; typedef size_t WorkerId; typedef size_t ObjStoreId; typedef size_t OperationId; diff --git a/lib/python/ray/__init__.py b/lib/python/ray/__init__.py index a824efeae..6ecf636f9 100644 --- a/lib/python/ray/__init__.py +++ b/lib/python/ray/__init__.py @@ -23,5 +23,5 @@ import libraylib as lib import serialization from worker import scheduler_info, visualize_computation_graph, task_info, register_module, init, connect, disconnect, get, put, remote, kill_workers, restart_workers_local from worker import Reusable, reusables -from libraylib import ObjRef +from libraylib import ObjectID import internal diff --git a/lib/python/ray/array/distributed/core.py b/lib/python/ray/array/distributed/core.py index da99f89f2..756464c20 100644 --- a/lib/python/ray/array/distributed/core.py +++ b/lib/python/ray/array/distributed/core.py @@ -9,20 +9,20 @@ __all__ = ["BLOCK_SIZE", "DistArray", "assemble", "zeros", "ones", "copy", BLOCK_SIZE = 10 class DistArray(object): - def construct(self, shape, objrefs=None): + def construct(self, shape, objectids=None): self.shape = shape self.ndim = len(shape) self.num_blocks = [int(np.ceil(1.0 * a / BLOCK_SIZE)) for a in self.shape] - self.objrefs = objrefs if objrefs is not None else np.empty(self.num_blocks, dtype=object) - if self.num_blocks != list(self.objrefs.shape): - raise Exception("The fields `num_blocks` and `objrefs` are inconsistent, `num_blocks` is {} and `objrefs` has shape {}".format(self.num_blocks, list(self.objrefs.shape))) + self.objectids = objectids if objectids is not None else np.empty(self.num_blocks, dtype=object) + if self.num_blocks != list(self.objectids.shape): + raise Exception("The fields `num_blocks` and `objectids` are inconsistent, `num_blocks` is {} and `objectids` has shape {}".format(self.num_blocks, list(self.objectids.shape))) def deserialize(self, primitives): - (shape, objrefs) = primitives - self.construct(shape, objrefs) + (shape, objectids) = primitives + self.construct(shape, objectids) def serialize(self): - return (self.shape, self.objrefs) + return (self.shape, self.objectids) def __init__(self, shape=None): if shape is not None: @@ -54,14 +54,14 @@ class DistArray(object): return [int(np.ceil(1.0 * a / BLOCK_SIZE)) for a in shape] def assemble(self): - """Assemble an array on this node from a distributed array object reference.""" - first_block = ray.get(self.objrefs[(0,) * self.ndim]) + """Assemble an array on this node from a distributed array of object IDs.""" + first_block = ray.get(self.objectids[(0,) * self.ndim]) dtype = first_block.dtype result = np.zeros(self.shape, dtype=dtype) for index in np.ndindex(*self.num_blocks): lower = DistArray.compute_block_lower(index, self.shape) upper = DistArray.compute_block_upper(index, self.shape) - result[[slice(l, u) for (l, u) in zip(lower, upper)]] = ray.get(self.objrefs[index]) + result[[slice(l, u) for (l, u) in zip(lower, upper)]] = ray.get(self.objectids[index]) return result def __getitem__(self, sliced): @@ -80,28 +80,28 @@ def numpy_to_dist(a): for index in np.ndindex(*result.num_blocks): lower = DistArray.compute_block_lower(index, a.shape) upper = DistArray.compute_block_upper(index, a.shape) - result.objrefs[index] = ray.put(a[[slice(l, u) for (l, u) in zip(lower, upper)]]) + result.objectids[index] = ray.put(a[[slice(l, u) for (l, u) in zip(lower, upper)]]) return result @ray.remote([List, str], [DistArray]) def zeros(shape, dtype_name="float"): result = DistArray(shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = ra.zeros.remote(DistArray.compute_block_shape(index, shape), dtype_name=dtype_name) + result.objectids[index] = ra.zeros.remote(DistArray.compute_block_shape(index, shape), dtype_name=dtype_name) return result @ray.remote([List, str], [DistArray]) def ones(shape, dtype_name="float"): result = DistArray(shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = ra.ones.remote(DistArray.compute_block_shape(index, shape), dtype_name=dtype_name) + result.objectids[index] = ra.ones.remote(DistArray.compute_block_shape(index, shape), dtype_name=dtype_name) return result @ray.remote([DistArray], [DistArray]) def copy(a): result = DistArray(a.shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = a.objrefs[index] # We don't need to actually copy the objects because cluster-level objects are assumed to be immutable. + result.objectids[index] = a.objectids[index] # We don't need to actually copy the objects because cluster-level objects are assumed to be immutable. return result @ray.remote([int, int, str], [DistArray]) @@ -112,9 +112,9 @@ def eye(dim1, dim2=-1, dtype_name="float"): for (i, j) in np.ndindex(*result.num_blocks): block_shape = DistArray.compute_block_shape([i, j], shape) if i == j: - result.objrefs[i, j] = ra.eye.remote(block_shape[0], block_shape[1], dtype_name=dtype_name) + result.objectids[i, j] = ra.eye.remote(block_shape[0], block_shape[1], dtype_name=dtype_name) else: - result.objrefs[i, j] = ra.zeros.remote(block_shape, dtype_name=dtype_name) + result.objectids[i, j] = ra.zeros.remote(block_shape, dtype_name=dtype_name) return result @ray.remote([DistArray], [DistArray]) @@ -124,11 +124,11 @@ def triu(a): result = DistArray(a.shape) for (i, j) in np.ndindex(*result.num_blocks): if i < j: - result.objrefs[i, j] = ra.copy.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.copy.remote(a.objectids[i, j]) elif i == j: - result.objrefs[i, j] = ra.triu.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.triu.remote(a.objectids[i, j]) else: - result.objrefs[i, j] = ra.zeros_like.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.zeros_like.remote(a.objectids[i, j]) return result @ray.remote([DistArray], [DistArray]) @@ -138,11 +138,11 @@ def tril(a): result = DistArray(a.shape) for (i, j) in np.ndindex(*result.num_blocks): if i > j: - result.objrefs[i, j] = ra.copy.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.copy.remote(a.objectids[i, j]) elif i == j: - result.objrefs[i, j] = ra.tril.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.tril.remote(a.objectids[i, j]) else: - result.objrefs[i, j] = ra.zeros_like.remote(a.objrefs[i, j]) + result.objectids[i, j] = ra.zeros_like.remote(a.objectids[i, j]) return result @ray.remote([np.ndarray], [np.ndarray]) @@ -167,8 +167,8 @@ def dot(a, b): shape = [a.shape[0], b.shape[1]] result = DistArray(shape) for (i, j) in np.ndindex(*result.num_blocks): - args = list(a.objrefs[i, :]) + list(b.objrefs[:, j]) - result.objrefs[i, j] = blockwise_dot.remote(*args) + args = list(a.objectids[i, :]) + list(b.objectids[:, j]) + result.objectids[i, j] = blockwise_dot.remote(*args) return result @ray.remote([DistArray, List], [DistArray]) @@ -176,9 +176,9 @@ def subblocks(a, *ranges): """ This function produces a distributed array from a subset of the blocks in the `a`. The result and `a` will have the same number of dimensions.For example, subblocks(a, [0, 1], [2, 4]) - will produce a DistArray whose objrefs are - [[a.objrefs[0, 2], a.objrefs[0, 4]], - [a.objrefs[1, 2], a.objrefs[1, 4]]] + will produce a DistArray whose objectids are + [[a.objectids[0, 2], a.objectids[0, 4]], + [a.objectids[1, 2], a.objectids[1, 4]]] We allow the user to pass in an empty list [] to indicate the full range. """ ranges = list(ranges) @@ -198,7 +198,7 @@ def subblocks(a, *ranges): shape = [(len(ranges[i]) - 1) * BLOCK_SIZE + last_block_shape[i] for i in range(a.ndim)] result = DistArray(shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = a.objrefs[tuple([ranges[i][index[i]] for i in range(a.ndim)])] + result.objectids[index] = a.objectids[tuple([ranges[i][index[i]] for i in range(a.ndim)])] return result @ray.remote([DistArray], [DistArray]) @@ -208,7 +208,7 @@ def transpose(a): result = DistArray([a.shape[1], a.shape[0]]) for i in range(result.num_blocks[0]): for j in range(result.num_blocks[1]): - result.objrefs[i, j] = ra.transpose.remote(a.objrefs[j, i]) + result.objectids[i, j] = ra.transpose.remote(a.objectids[j, i]) return result # TODO(rkn): support broadcasting? @@ -218,7 +218,7 @@ def add(x1, x2): raise Exception("add expects arguments `x1` and `x2` to have the same shape, but x1.shape = {}, and x2.shape = {}.".format(x1.shape, x2.shape)) result = DistArray(x1.shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = ra.add.remote(x1.objrefs[index], x2.objrefs[index]) + result.objectids[index] = ra.add.remote(x1.objectids[index], x2.objectids[index]) return result # TODO(rkn): support broadcasting? @@ -228,5 +228,5 @@ def subtract(x1, x2): raise Exception("subtract expects arguments `x1` and `x2` to have the same shape, but x1.shape = {}, and x2.shape = {}.".format(x1.shape, x2.shape)) result = DistArray(x1.shape) for index in np.ndindex(*result.num_blocks): - result.objrefs[index] = ra.subtract.remote(x1.objrefs[index], x2.objrefs[index]) + result.objectids[index] = ra.subtract.remote(x1.objectids[index], x2.objectids[index]) return result diff --git a/lib/python/ray/array/distributed/linalg.py b/lib/python/ray/array/distributed/linalg.py index f94712531..efbe206a9 100644 --- a/lib/python/ray/array/distributed/linalg.py +++ b/lib/python/ray/array/distributed/linalg.py @@ -32,7 +32,7 @@ def tsqr(a): q_tree = np.empty((num_blocks, K), dtype=object) current_rs = [] for i in range(num_blocks): - block = a.objrefs[i, 0] + block = a.objectids[i, 0] q, r = ra.linalg.qr.remote(block) q_tree[i, 0] = q current_rs.append(r) @@ -57,8 +57,8 @@ def tsqr(a): q_shape = [a.shape[0], a.shape[0]] q_num_blocks = DistArray.compute_num_blocks(q_shape) q_result = DistArray() - q_objrefs = np.empty(q_num_blocks, dtype=object) - q_result.construct(q_shape, q_objrefs) + q_objectids = np.empty(q_num_blocks, dtype=object) + q_result.construct(q_shape, q_objectids) # reconstruct output for i in range(num_blocks): @@ -73,7 +73,7 @@ def tsqr(a): upper = [2 * a.shape[1], BLOCK_SIZE] ith_index /= 2 q_block_current = ra.dot.remote(q_block_current, ra.subarray.remote(q_tree[ith_index, j], lower, upper)) - q_result.objrefs[i] = q_block_current + q_result.objectids[i] = q_block_current r = current_rs[0] return q_result, r @@ -126,7 +126,7 @@ def tsqr_hr(a): q, r_temp = tsqr.remote(a) y, u, s = modified_lu.remote(q) y_blocked = ray.get(y) - t, y_top = tsqr_hr_helper1.remote(u, s, y_blocked.objrefs[0, 0], a.shape[1]) + t, y_top = tsqr_hr_helper1.remote(u, s, y_blocked.objectids[0, 0], a.shape[1]) r = tsqr_hr_helper2.remote(s, r_temp) return y, t, y_top, r @@ -146,9 +146,9 @@ def qr(a): # we will store our scratch work in a_work a_work = DistArray() - a_work.construct(a.shape, np.copy(a.objrefs)) + a_work.construct(a.shape, np.copy(a.objectids)) - result_dtype = np.linalg.qr(ray.get(a.objrefs[0, 0]))[0].dtype.name + result_dtype = np.linalg.qr(ray.get(a.objectids[0, 0]))[0].dtype.name r_res = ray.get(zeros.remote([k, n], result_dtype)) # TODO(rkn): It would be preferable not to get this right after creating it. y_res = ray.get(zeros.remote([m, k], result_dtype)) # TODO(rkn): It would be preferable not to get this right after creating it. Ts = [] @@ -159,27 +159,27 @@ def qr(a): y_val = ray.get(y) for j in range(i, a.num_blocks[0]): - y_res.objrefs[j, i] = y_val.objrefs[j - i, 0] + y_res.objectids[j, i] = y_val.objectids[j - i, 0] if a.shape[0] > a.shape[1]: # in this case, R needs to be square R_shape = ray.get(ra.shape.remote(R)) eye_temp = ra.eye.remote(R_shape[1], R_shape[0], dtype_name=result_dtype) - r_res.objrefs[i, i] = ra.dot.remote(eye_temp, R) + r_res.objectids[i, i] = ra.dot.remote(eye_temp, R) else: - r_res.objrefs[i, i] = R + r_res.objectids[i, i] = R Ts.append(numpy_to_dist.remote(t)) for c in range(i + 1, a.num_blocks[1]): W_rcs = [] for r in range(i, a.num_blocks[0]): - y_ri = y_val.objrefs[r - i, 0] - W_rcs.append(qr_helper2.remote(y_ri, a_work.objrefs[r, c])) + y_ri = y_val.objectids[r - i, 0] + W_rcs.append(qr_helper2.remote(y_ri, a_work.objectids[r, c])) W_c = ra.sum_list.remote(*W_rcs) for r in range(i, a.num_blocks[0]): - y_ri = y_val.objrefs[r - i, 0] - A_rc = qr_helper1.remote(a_work.objrefs[r, c], y_ri, t, W_c) - a_work.objrefs[r, c] = A_rc - r_res.objrefs[i, c] = a_work.objrefs[i, c] + y_ri = y_val.objectids[r - i, 0] + A_rc = qr_helper1.remote(a_work.objectids[r, c], y_ri, t, W_c) + a_work.objectids[r, c] = A_rc + r_res.objectids[i, c] = a_work.objectids[i, c] # construct q_res from Ys and Ts q = eye.remote(m, k, dtype_name=result_dtype) diff --git a/lib/python/ray/array/distributed/random.py b/lib/python/ray/array/distributed/random.py index a8c48ad46..f966162be 100644 --- a/lib/python/ray/array/distributed/random.py +++ b/lib/python/ray/array/distributed/random.py @@ -9,9 +9,9 @@ from core import * @ray.remote([List], [DistArray]) def normal(shape): num_blocks = DistArray.compute_num_blocks(shape) - objrefs = np.empty(num_blocks, dtype=object) + objectids = np.empty(num_blocks, dtype=object) for index in np.ndindex(*num_blocks): - objrefs[index] = ra.random.normal.remote(DistArray.compute_block_shape(index, shape)) + objectids[index] = ra.random.normal.remote(DistArray.compute_block_shape(index, shape)) result = DistArray() - result.construct(shape, objrefs) + result.construct(shape, objectids) return result diff --git a/lib/python/ray/graph.py b/lib/python/ray/graph.py index f153a72c2..981f4ad99 100644 --- a/lib/python/ray/graph.py +++ b/lib/python/ray/graph.py @@ -22,13 +22,13 @@ def graph_to_graphviz(computation_graph): dot.edge("op" + str(i), str(res)) elif op.HasField("put"): dot.node("op" + str(i), shape="box", label=str(i) + "\n" + "put") - dot.edge("op" + str(i), str(op.put.objref)) + dot.edge("op" + str(i), str(op.put.objectid)) elif op.HasField("get"): dot.node("op" + str(i), shape="box", label=str(i) + "\n" + "get") creator_operationid = op.creator_operationid if op.creator_operationid != 2 ** 64 - 1 else "-root" dot.edge("op" + str(creator_operationid), "op" + str(i), style="dotted", constraint="false") for arg in op.task.arg: if not arg.HasField("obj"): - dot.node(str(arg.ref)) - dot.edge(str(arg.ref), "op" + str(i)) + dot.node(str(arg.id)) + dot.edge(str(arg.id), "op" + str(i)) return dot diff --git a/lib/python/ray/serialization.py b/lib/python/ray/serialization.py index e67b418b9..b62124c88 100644 --- a/lib/python/ray/serialization.py +++ b/lib/python/ray/serialization.py @@ -55,18 +55,18 @@ def is_arrow_serializable(value): def serialize(worker_capsule, obj): primitive_obj = to_primitive(obj) - obj_capsule, contained_objrefs = ray.lib.serialize_object(worker_capsule, primitive_obj) # contained_objrefs is a list of the objrefs contained in obj - return obj_capsule, contained_objrefs + obj_capsule, contained_objectids = ray.lib.serialize_object(worker_capsule, primitive_obj) # contained_objectids is a list of the objectids contained in obj + return obj_capsule, contained_objectids def deserialize(worker_capsule, capsule): primitive_obj = ray.lib.deserialize_object(worker_capsule, capsule) return from_primitive(primitive_obj) def serialize_task(worker_capsule, func_name, args): - primitive_args = [(arg if isinstance(arg, ray.lib.ObjRef) else to_primitive(arg)) for arg in args] + primitive_args = [(arg if isinstance(arg, ray.ObjectID) else to_primitive(arg)) for arg in args] return ray.lib.serialize_task(worker_capsule, func_name, primitive_args) def deserialize_task(worker_capsule, task): - func_name, primitive_args, return_objrefs = task - args = [(arg if isinstance(arg, ray.lib.ObjRef) else from_primitive(arg)) for arg in primitive_args] - return func_name, args, return_objrefs + func_name, primitive_args, return_objectids = task + args = [(arg if isinstance(arg, ray.ObjectID) else from_primitive(arg)) for arg in primitive_args] + return func_name, args, return_objectids diff --git a/lib/python/ray/worker.py b/lib/python/ray/worker.py index 8949f77b9..a9b1d10ed 100644 --- a/lib/python/ray/worker.py +++ b/lib/python/ray/worker.py @@ -75,12 +75,12 @@ class RayFailedObject(object): class RayDealloc(object): """An object used internally to properly implement reference counting. - When we call get_object with a particular object reference, we create a - RayDealloc object with the information necessary to properly handle closing - the relevant memory segment when the object is no longer needed by the worker. - The RayDealloc object is stored as a field in the object returned by - get_object so that its destructor is only called when the worker no longer has - any references to the object. + When we call get_object with a particular object ID, we create a RayDealloc + object with the information necessary to properly handle closing the relevant + memory segment when the object is no longer needed by the worker. The + RayDealloc object is stored as a field in the object returned by get_object so + that its destructor is only called when the worker no longer has any + references to the object. Attributes handle (worker capsule): A Python object wrapping a C++ Worker object. @@ -293,14 +293,14 @@ class Worker(object): self.mode = mode colorama.init() - def put_object(self, objref, value): - """Put value in the local object store with object reference objref. + def put_object(self, objectid, value): + """Put value in the local object store with object id objectid. - This assumes that the value for objref has not yet been placed in the + This assumes that the value for objectid has not yet been placed in the local object store. Args: - objref (ray.ObjRef): The object reference of the value to be put. + objectid (ray.ObjectID): The object ID of the value to be put. value (serializable object): The value to put in the object store. """ try: @@ -312,7 +312,7 @@ class Worker(object): # the len(schema) is for storing the metadata and the 4096 is for storing # the metadata in the batch (see INITIAL_METADATA_SIZE in arrow) size = size + 8 + len(schema) + 4096 - buff, segmentid = ray.lib.allocate_buffer(self.handle, objref, size) + buff, segmentid = ray.lib.allocate_buffer(self.handle, objectid, size) # write the metadata length np.frombuffer(buff, dtype="int64", count=1)[0] = len(schema) # metadata buffer @@ -321,25 +321,25 @@ class Worker(object): metadata[:] = schema data = np.frombuffer(buff, dtype="byte")[8 + len(schema):] metadata_offset = libnumbuf.write_to_buffer(serialized, memoryview(data)) - ray.lib.finish_buffer(self.handle, objref, segmentid, metadata_offset) + ray.lib.finish_buffer(self.handle, objectid, segmentid, metadata_offset) except: - # At the moment, custom object and objects that contain object references take this path + # At the moment, custom object and objects that contain object IDs take this path # TODO(pcm): Make sure that these are the only objects getting serialized to protobuf - object_capsule, contained_objrefs = serialization.serialize(self.handle, value) # contained_objrefs is a list of the objrefs contained in object_capsule - ray.lib.put_object(self.handle, objref, object_capsule, contained_objrefs) + object_capsule, contained_objectids = serialization.serialize(self.handle, value) # contained_objectids is a list of the objectids contained in object_capsule + ray.lib.put_object(self.handle, objectid, object_capsule, contained_objectids) - def get_object(self, objref): - """Get the value in the local object store associated with objref. + def get_object(self, objectid): + """Get the value in the local object store associated with objectid. - Return the value from the local object store for objref. This will block - until the value for objref has been written to the local object store. + Return the value from the local object store for objectid. This will block + until the value for objectid has been written to the local object store. Args: - objref (ray.ObjRef): The object reference of the value to retrieve. + objectid (ray.ObjectID): The object ID of the value to retrieve. """ - if ray.lib.is_arrow(self.handle, objref): + if ray.lib.is_arrow(self.handle, objectid): ## this is the new codepath - buff, segmentid, metadata_offset = ray.lib.get_buffer(self.handle, objref) + buff, segmentid, metadata_offset = ray.lib.get_buffer(self.handle, objectid) metadata_size = np.frombuffer(buff, dtype="int64", count=1)[0] metadata = np.frombuffer(buff, dtype="byte", offset=8, count=metadata_size) data = np.frombuffer(buff, dtype="byte")[8 + metadata_size:] @@ -349,9 +349,9 @@ class Worker(object): assert len(deserialized) == 1 result = deserialized[0] ## this is the old codepath - # result, segmentid = ray.lib.get_arrow(self.handle, objref) + # result, segmentid = ray.lib.get_arrow(self.handle, objectid) else: - object_capsule, segmentid = ray.lib.get_object(self.handle, objref) + object_capsule, segmentid = ray.lib.get_object(self.handle, objectid) result = serialization.deserialize(self.handle, object_capsule) if isinstance(result, int): @@ -379,13 +379,13 @@ class Worker(object): elif result == None: ray.lib.unmap_object(self.handle, segmentid) # need to unmap here because result is passed back "by value" and we have no reference to unmap later return None # can't subclass None and don't need to because there is a global None - result.ray_objref = objref # TODO(pcm): This could be done only for the "get" case in the future if we want to increase performance + result.ray_objectid = objectid # TODO(pcm): This could be done only for the "get" case in the future if we want to increase performance result.ray_deallocator = RayDealloc(self.handle, segmentid) return result - def alias_objrefs(self, alias_objref, target_objref): - """Make two object references refer to the same object.""" - ray.lib.alias_objrefs(self.handle, alias_objref, target_objref) + def alias_objectids(self, alias_objectid, target_objectid): + """Make two object IDs refer to the same object.""" + ray.lib.alias_objectids(self.handle, alias_objectid, target_objectid) def register_function(self, function): """Register a function with the scheduler. @@ -405,20 +405,20 @@ class Worker(object): """Submit a remote task to the scheduler. Tell the scheduler to schedule the execution of the function with name - func_name with arguments args. Retrieve object references for the outputs of + func_name with arguments args. Retrieve object IDs for the outputs of the function from the scheduler and immediately return them. Args: func_name (str): The name of the function to be executed. args (List[Any]): The arguments to pass into the function. Arguments can - be object references or they can be values. If they are values, they + be object IDs or they can be values. If they are values, they must be serializable objecs. """ task_capsule = serialization.serialize_task(self.handle, func_name, args) - objrefs = ray.lib.submit_task(self.handle, task_capsule) + objectids = ray.lib.submit_task(self.handle, task_capsule) if self.mode in [ray.SHELL_MODE, ray.SCRIPT_MODE]: print_task_info(ray.lib.task_info(self.handle), self.mode) - return objrefs + return objectids global_worker = Worker() """Worker: The global Worker object for this worker process. @@ -645,29 +645,29 @@ def disconnect(worker=global_worker): worker.cached_remote_functions = [] reusables._cached_reusables = [] -def get(objref, worker=global_worker): +def get(objectid, worker=global_worker): """Get a remote object from an object store. - This method blocks until the object corresponding to objref is available in + This method blocks until the object corresponding to objectid is available in the local object store. If this object is not in the local object store, it will be shipped from an object store that has it (once the object has been created). Args: - objref (ray.ObjRef): Object reference to the object to get. + objectid (ray.ObjectID): Object ID to the object to get. Returns: A Python object """ check_connected(worker) if worker.mode == ray.PYTHON_MODE: - return objref # In ray.PYTHON_MODE, ray.get is the identity operation (the input will actually be a value not an objref) - ray.lib.request_object(worker.handle, objref) + return objectid # In ray.PYTHON_MODE, ray.get is the identity operation (the input will actually be a value not an objectid) + ray.lib.request_object(worker.handle, objectid) if worker.mode in [ray.SHELL_MODE, ray.SCRIPT_MODE]: print_task_info(ray.lib.task_info(worker.handle), worker.mode) - value = worker.get_object(objref) + value = worker.get_object(objectid) if isinstance(value, RayFailedObject): - raise Exception("The task that created this object reference failed with error message:\n{}".format(value.error_message)) + raise Exception("The task that created this object ID failed with error message:\n{}".format(value.error_message)) return value def put(value, worker=global_worker): @@ -677,16 +677,16 @@ def put(value, worker=global_worker): value (serializable object): The Python object to be stored. Returns: - The object reference assigned to this value. + The object ID assigned to this value. """ check_connected(worker) if worker.mode == ray.PYTHON_MODE: return value # In ray.PYTHON_MODE, ray.put is the identity operation - objref = ray.lib.get_objref(worker.handle) - worker.put_object(objref, value) + objectid = ray.lib.get_objectid(worker.handle) + worker.put_object(objectid, value) if worker.mode in [ray.SHELL_MODE, ray.SCRIPT_MODE]: print_task_info(ray.lib.task_info(worker.handle), worker.mode) - return objref + return objectid def kill_workers(worker=global_worker): """Kill all of the workers in the cluster. This does not kill drivers. @@ -748,7 +748,7 @@ def main_loop(worker=global_worker): This method is an infinite loop. It waits to receive tasks from the scheduler. When it receives a task, it first deserializes the task. Then it retrieves the - values for any arguments that were passed in as object references. Then it + values for any arguments that were passed in as object IDs. Then it passes the arguments to the actual function. Then it stores the outputs of the function in the local object store. Then it notifies the scheduler that it completed the task. @@ -763,22 +763,22 @@ def main_loop(worker=global_worker): raise Exception("Worker is attempting to enter main_loop but has not been connected yet.") ray.lib.start_worker_service(worker.handle) def process_task(task): # wrapping these lines in a function should cause the local variables to go out of scope more quickly, which is useful for inspecting reference counts - func_name, args, return_objrefs = serialization.deserialize_task(worker.handle, task) + func_name, args, return_objectids = serialization.deserialize_task(worker.handle, task) try: arguments = get_arguments_for_execution(worker.functions[func_name], args, worker) # get args from objstore outputs = worker.functions[func_name].executor(arguments) # execute the function - if len(return_objrefs) == 1: + if len(return_objectids) == 1: outputs = (outputs,) except Exception: exception_message = format_error_message(traceback.format_exc()) # Here we are storing RayFailedObjects in the object store to indicate # failure (this is only interpreted by the worker). - failure_objects = [RayFailedObject(exception_message) for _ in range(len(return_objrefs))] - store_outputs_in_objstore(return_objrefs, failure_objects, worker) + failure_objects = [RayFailedObject(exception_message) for _ in range(len(return_objectids))] + store_outputs_in_objstore(return_objectids, failure_objects, worker) ray.lib.notify_task_completed(worker.handle, False, exception_message) # notify the scheduler that the task threw an exception _logger().info("Worker threw exception with message: \n\n{}\n, while running function {}.".format(exception_message, func_name)) else: - store_outputs_in_objstore(return_objrefs, outputs, worker) # store output in local object store + store_outputs_in_objstore(return_objectids, outputs, worker) # store output in local object store ray.lib.notify_task_completed(worker.handle, True, "") # notify the scheduler that the task completed successfully finally: # Reinitialize the values of reusable variables that were used in the task @@ -868,11 +868,11 @@ def remote(arg_types, return_types, worker=global_worker): # match the usual behavior of immutable remote objects. return func(*copy.deepcopy(args)) check_arguments(arg_types, has_vararg_param, func_name, args) # throws an exception if args are invalid - objrefs = _submit_task(func_name, args) - if len(objrefs) == 1: - return objrefs[0] - elif len(objrefs) > 1: - return objrefs + objectids = _submit_task(func_name, args) + if len(objectids) == 1: + return objectids[0] + elif len(objectids) > 1: + return objectids def func_executor(arguments): """This gets run when the remote function is executed.""" _logger().info("Calling function {}".format(func.__name__)) @@ -977,8 +977,8 @@ def check_return_values(function, result): # Here we do some limited type checking to make sure the return values have # the right types. for i in range(len(result)): - if (not issubclass(type(result[i]), function.return_types[i])) and (not isinstance(result[i], ray.lib.ObjRef)): - raise Exception("The {}th return value for function {} has type {}, but the @remote decorator expected a return value of type {} or an ObjRef.".format(i, function.__name__, type(result[i]), function.return_types[i])) + if (not issubclass(type(result[i]), function.return_types[i])) and (not isinstance(result[i], ray.lib.ObjectID)): + raise Exception("The {}th return value for function {} has type {}, but the @remote decorator expected a return value of type {} or an ObjectID.".format(i, function.__name__, type(result[i]), function.return_types[i])) def typecheck_arg(arg, expected_type, i, name): """Check that an argument has the expected type. @@ -1033,8 +1033,8 @@ def check_arguments(arg_types, has_vararg_param, name, args): else: assert False, "This code should be unreachable." - if isinstance(arg, ray.lib.ObjRef): - # TODO(rkn): When we have type information in the ObjRef, do type checking here. + if isinstance(arg, ray.ObjectID): + # TODO(rkn): When we have type information in the ObjectID, do type checking here. pass else: typecheck_arg(arg, expected_type, i, name) @@ -1043,9 +1043,9 @@ def get_arguments_for_execution(function, args, worker=global_worker): """Retrieve the arguments for the remote function. This retrieves the values for the arguments to the remote function that were - passed in as object references. Argumens that were passed by value are not - changed. This also does some type checking. This is called by the worker that - is executing the remote function. + passed in as object IDs. Argumens that were passed by value are not changed. + This also does some type checking. This is called by the worker that is + executing the remote function. Args: function (Callable): The remote function whose arguments are being @@ -1075,7 +1075,7 @@ def get_arguments_for_execution(function, args, worker=global_worker): else: assert False, "This code should be unreachable." - if isinstance(arg, ray.lib.ObjRef): + if isinstance(arg, ray.ObjectID): # get the object from the local object store _logger().info("Getting argument {} for function {}.".format(i, function.__name__)) argument = worker.get_object(arg) @@ -1088,31 +1088,30 @@ def get_arguments_for_execution(function, args, worker=global_worker): arguments.append(argument) return arguments -def store_outputs_in_objstore(objrefs, outputs, worker=global_worker): +def store_outputs_in_objstore(objectids, outputs, worker=global_worker): """Store the outputs of a remote function in the local object store. This stores the values that were returned by a remote function in the local - object store. If any of the return values are object references, then these - object references are aliased with the object references that the scheduler - assigned for the return values. This is called by the worker that executes the - remote function. + object store. If any of the return values are object IDs, then these object + IDs are aliased with the object IDs that the scheduler assigned for the return + values. This is called by the worker that executes the remote function. Note: - The arguments objrefs and outputs should have the same length. + The arguments objectids and outputs should have the same length. Args: - objrefs (List[ray.ObjRef]): The object references that were assigned to the + objectids (List[ray.ObjectID]): The object IDs that were assigned to the outputs of the remote function call. outputs (Tuple): The value returned by the remote function. If the remote function was supposed to only return one value, then its output was wrapped in a tuple with one element prior to being passed into this function. """ - for i in range(len(objrefs)): - if isinstance(outputs[i], ray.lib.ObjRef): - # An ObjRef is being returned, so we must alias objrefs[i] so that it refers to the same object that outputs[i] refers to - _logger().info("Aliasing objrefs {} and {}".format(objrefs[i].val, outputs[i].val)) - worker.alias_objrefs(objrefs[i], outputs[i]) + for i in range(len(objectids)): + if isinstance(outputs[i], ray.ObjectID): + # An ObjectID is being returned, so we must alias objectids[i] so that it refers to the same object that outputs[i] refers to + _logger().info("Aliasing objectids {} and {}".format(objectids[i].id, outputs[i].id)) + worker.alias_objectids(objectids[i], outputs[i]) pass else: - worker.put_object(objrefs[i], outputs[i]) + worker.put_object(objectids[i], outputs[i]) diff --git a/protos/graph.proto b/protos/graph.proto index 01badf03f..1ca6e7530 100644 --- a/protos/graph.proto +++ b/protos/graph.proto @@ -4,16 +4,16 @@ import "types.proto"; message Task { string name = 1; // Name of the function call. Must not be empty. - repeated Value arg = 2; // List of arguments, can be either object references or protobuf descriptions of object passed by value - repeated uint64 result = 3; // Object references for result + repeated Value arg = 2; // List of arguments, can be either object IDs or protobuf descriptions of object passed by value + repeated uint64 result = 3; // Object IDs for result } message Put { - uint64 objref = 1; // The objref for the object that was put + uint64 objectid = 1; // The objectid for the object that was put } message Get { - uint64 objref = 1; // The objref for the object that is retrieved + uint64 objectid = 1; // The objectid for the object that is retrieved } // This is used internally by the scheduler. From the scheduler's perspective, diff --git a/protos/ray.proto b/protos/ray.proto index 05c95797e..a7379aec1 100644 --- a/protos/ray.proto +++ b/protos/ray.proto @@ -7,7 +7,7 @@ // Object store: Typically there is one object store per node which holds the // objects locally stored on that node. // Scheduler: The scheduler process keeps track of a mapping from object -// references to object stores, orchestrates data transfer between object +// IDs to object stores, orchestrates data transfer between object // stores and assigns tasks to workers. syntax = "proto3"; @@ -24,26 +24,26 @@ service Scheduler { rpc RegisterObjStore(RegisterObjStoreRequest) returns (RegisterObjStoreReply); // Tell the scheduler that a worker can execute a certain function rpc RegisterFunction(RegisterFunctionRequest) returns (AckReply); - // Asks the scheduler to execute a task, immediately returns an object reference to the result + // Asks the scheduler to execute a task, immediately returns an object ID to the result rpc SubmitTask(SubmitTaskRequest) returns (SubmitTaskReply); - // Increment the count of the object reference + // Increment the count of the object ID rpc IncrementCount(ChangeCountRequest) returns (AckReply); - // Decrement the count of the object reference + // Decrement the count of the object ID rpc DecrementCount(ChangeCountRequest) returns (AckReply); - // Request an object reference for an object that will be put in an object store + // Request an object ID for an object that will be put in an object store rpc PutObj(PutObjRequest) returns (PutObjReply); // Request delivery of an object from an object store that holds the object to the local object store rpc RequestObj(RequestObjRequest) returns (AckReply); - // Used by the worker to tell the scheduler that two objrefs should refer to the same object - rpc AliasObjRefs(AliasObjRefsRequest) returns (AckReply); + // Used by the worker to tell the scheduler that two objectids should refer to the same object + rpc AliasObjectIDs(AliasObjectIDsRequest) returns (AckReply); // Used by an object store to tell the scheduler that an object is ready (i.e. has been finalized and can be shared) rpc ObjReady(ObjReadyRequest) returns (AckReply); - // Increments the reference count of a particular object reference + // Increments the reference count of a particular object ID rpc IncrementRefCount(IncrementRefCountRequest) returns (AckReply); - // Decrements the reference count of a particular object reference + // Decrements the reference count of a particular object ID rpc DecrementRefCount(DecrementRefCountRequest) returns (AckReply); - // Used by the worker to notify the scheduler about which objrefs a particular object contains - rpc AddContainedObjRefs(AddContainedObjRefsRequest) returns (AckReply); + // Used by the worker to notify the scheduler about which objectids a particular object contains + rpc AddContainedObjectIDs(AddContainedObjectIDsRequest) returns (AckReply); // Used by the worker to ask for work, this also returns the status of the previous task if there was one rpc ReadyForNewTask(ReadyForNewTaskRequest) returns (AckReply); // Get information about the scheduler state @@ -92,13 +92,13 @@ message SubmitTaskRequest { } message SubmitTaskReply { - repeated uint64 result = 1; // Object references of the function return values + repeated uint64 result = 1; // Object IDs of the function return values bool function_registered = 2; // True if the function was registered; false otherwise } message RequestObjRequest { uint64 workerid = 1; // Worker that tries to request the object - uint64 objref = 2; // Object reference of the object being requested + uint64 objectid = 2; // Object ID of the object being requested } message PutObjRequest { @@ -106,30 +106,30 @@ message PutObjRequest { } message PutObjReply { - uint64 objref = 1; // Object reference assigned by the scheduler to the object + uint64 objectid = 1; // Object ID assigned by the scheduler to the object } -message AliasObjRefsRequest { - uint64 alias_objref = 1; // ObjRef which will be aliased - uint64 target_objref = 2; // The target ObjRef +message AliasObjectIDsRequest { + uint64 alias_objectid = 1; // ObjectID which will be aliased + uint64 target_objectid = 2; // The target ObjectID } message ObjReadyRequest { - uint64 objref = 1; // Object reference of the object that has been finalized + uint64 objectid = 1; // Object ID of the object that has been finalized uint64 objstoreid = 2; // ID of the object store the object lives on } message IncrementRefCountRequest { - repeated uint64 objref = 1; // Object references whose reference count should be incremented. Duplicates will be incremented multiple times. + repeated uint64 objectid = 1; // Object IDs whose reference count should be incremented. Duplicates will be incremented multiple times. } -message AddContainedObjRefsRequest { - uint64 objref = 1; // The objref of the object in question - repeated uint64 contained_objref = 2; // Object references contained in the object +message AddContainedObjectIDsRequest { + uint64 objectid = 1; // The objectid of the object in question + repeated uint64 contained_objectid = 2; // Object IDs contained in the object } message DecrementRefCountRequest { - repeated uint64 objref = 1; // Object references whose reference count should be decremented. Duplicates will be decremented multiple times. + repeated uint64 objectid = 1; // Object IDs whose reference count should be decremented. Duplicates will be decremented multiple times. } message ReadyForNewTaskRequest { @@ -142,7 +142,7 @@ message ReadyForNewTaskRequest { } message ChangeCountRequest { - uint64 objref = 1; // Object reference of the object whose reference count is increased or decreased + uint64 objectid = 1; // Object ID of the object whose reference count is increased or decreased } // The following messages are used to get information about the scheduler state @@ -159,7 +159,7 @@ message SchedulerInfoReply { repeated uint64 operationid = 1; // OperationIds of the tasks on the task queue repeated uint64 avail_worker = 3; // List of workers waiting to get a task assigned map function_table = 2; // Table of all available remote function - repeated uint64 target_objref = 4; // The target_objrefs_ data structure + repeated uint64 target_objectid = 4; // The target_objectids_ data structure repeated uint64 reference_count = 5; // The reference_counts_ data structure CompGraph computation_graph = 6; // The computation graph constructed so far } @@ -171,7 +171,7 @@ service ObjStore { rpc StartDelivery(StartDeliveryRequest) returns (AckReply); // Accept incoming data from another object store, as a stream of object chunks rpc StreamObjTo(StreamObjToRequest) returns (stream ObjChunk); - // Notify the object store about objref aliasing. This is called by the scheduler + // Notify the object store about objectid aliasing. This is called by the scheduler rpc NotifyAlias(NotifyAliasRequest) returns (AckReply); // Tell the object store to deallocate an object held by the object store. This is called by the scheduler. rpc DeallocateObject(DeallocateObjectRequest) returns (AckReply); @@ -181,11 +181,11 @@ service ObjStore { message StartDeliveryRequest { string objstore_address = 1; // Object store to get the object from - uint64 objref = 2; // Reference of object that gets delivered + uint64 objectid = 2; // ID of object that gets delivered } message RegisterObjRequest { - uint64 objref = 1; // Reference of object that gets registered + uint64 objectid = 1; // ID of object that gets registered } message RegisterObjReply { @@ -193,7 +193,7 @@ message RegisterObjReply { } message StreamObjToRequest { - uint64 objref = 1; // Object reference of the object being streamed + uint64 objectid = 1; // Object ID of the object being streamed } message ObjChunk { @@ -203,16 +203,16 @@ message ObjChunk { } message NotifyAliasRequest { - uint64 alias_objref = 1; // The objref being aliased - uint64 canonical_objref = 2; // The canonical objref that points to the actual object + uint64 alias_objectid = 1; // The objectid being aliased + uint64 canonical_objectid = 2; // The canonical objectid that points to the actual object } message DeallocateObjectRequest { - uint64 canonical_objref = 1; // The canonical objref of the object to deallocate + uint64 canonical_objectid = 1; // The canonical objectid of the object to deallocate } message GetObjRequest { - uint64 objref = 1; // Object reference of the object being requested by the worker + uint64 objectid = 1; // Object ID of the object being requested by the worker } message TaskInfoRequest { @@ -246,11 +246,11 @@ message ExportReusableVariableRequest { // These messages are for getting information about the object store state message ObjStoreInfoRequest { - repeated uint64 objref = 1; // Object references we want to retrieve from the store for inspection + repeated uint64 objectid = 1; // Object IDs we want to retrieve from the store for inspection } message ObjStoreInfoReply { - repeated uint64 objref = 1; // List of object references in the store + repeated uint64 objectid = 1; // List of object IDs in the store repeated Obj obj = 2; // Protocol buffer objects that were requested } diff --git a/protos/types.proto b/protos/types.proto index 4d61884a7..37bbe9da4 100644 --- a/protos/types.proto +++ b/protos/types.proto @@ -24,7 +24,7 @@ message Bool { bool data = 1; } -message Ref { +message ObjID { uint64 data = 1; } @@ -55,7 +55,7 @@ message Obj { Dict dict_data = 8; Array array_data = 5; Empty empty_data = 9; - Ref objref_data = 11; + ObjID objectid_data = 11; PyObj pyobj_data = 6; } @@ -77,7 +77,7 @@ message Dict { } message Value { - uint64 ref = 1; // For pass by reference + uint64 id = 1; // For pass by object ID Obj obj = 2; // For pass by value } @@ -89,5 +89,5 @@ message Array { repeated float float_data = 4; repeated sint64 int_data = 5; repeated uint64 uint_data = 6; - repeated uint64 objref_data = 7; + repeated uint64 objectid_data = 7; } diff --git a/src/ipc.h b/src/ipc.h index 0b82532a3..fce649ae5 100644 --- a/src/ipc.h +++ b/src/ipc.h @@ -65,13 +65,13 @@ public: // For communicating between object store and workers, the following // messages can be sent: -// ALLOC: workerid, objref, size -> objhandle: +// ALLOC: workerid, objectid, size -> objhandle: // worker requests an allocation from the object store -// GET: workerid, objref -> objhandle: +// GET: workerid, objectid -> objhandle: // worker requests an object from the object store -// WORKER_DONE: workerid, objref -> (): +// WORKER_DONE: workerid, objectid -> (): // worker tells the object store that an object has been finalized -// ALIAS_DONE: objref -> (): +// ALIAS_DONE: objectid -> (): // objstore tells itself that it has finalized something (perhaps an alias) enum ObjRequestType {ALLOC = 0, GET = 1, WORKER_DONE = 2, ALIAS_DONE = 3}; @@ -79,7 +79,7 @@ enum ObjRequestType {ALLOC = 0, GET = 1, WORKER_DONE = 2, ALIAS_DONE = 3}; struct ObjRequest { WorkerId workerid; // worker that sends the request ObjRequestType type; // do we want to allocate a new object or get a handle? - ObjRef objref; // object reference of the object to be returned/allocated + ObjectID objectid; // object ID of the object to be returned/allocated int64_t size; // if allocate, that's the size of the object int64_t metadata_offset; // if sending 'WORKER_DONE', that's the location of the metadata relative to the beginning of the object }; diff --git a/src/objstore.cc b/src/objstore.cc index 42334e2e6..c5c120776 100644 --- a/src/objstore.cc +++ b/src/objstore.cc @@ -7,19 +7,19 @@ const size_t ObjStoreService::CHUNK_SIZE = 8 * 1024; // this method needs to be protected by a objstore_lock_ // TODO(rkn): Make sure that we do not in fact need the objstore_lock_. We want multiple deliveries to be able to happen simultaneously. -void ObjStoreService::get_data_from(ObjRef objref, ObjStore::Stub& stub) { - RAY_LOG(RAY_DEBUG, "Objstore " << objstoreid_ << " is beginning to get objref " << objref); +void ObjStoreService::get_data_from(ObjectID objectid, ObjStore::Stub& stub) { + RAY_LOG(RAY_DEBUG, "Objstore " << objstoreid_ << " is beginning to get objectid " << objectid); ObjChunk chunk; ClientContext context; StreamObjToRequest stream_request; - stream_request.set_objref(objref); + stream_request.set_objectid(objectid); std::unique_ptr > reader(stub.StreamObjTo(&context, stream_request)); size_t total_size = 0; ObjHandle handle; if (reader->Read(&chunk)) { total_size = chunk.total_size(); - handle = alloc(objref, total_size); + handle = alloc(objectid, total_size); } size_t num_bytes = 0; segmentpool_lock_.lock(); @@ -34,9 +34,9 @@ void ObjStoreService::get_data_from(ObjRef objref, ObjStore::Stub& stub) { Status status = reader->Finish(); // Right now we don't use the status. // finalize object - RAY_CHECK_EQ(num_bytes, total_size, "Streamed objref " << objref << ", but num_bytes != total_size"); - object_ready(objref, chunk.metadata_offset()); - RAY_LOG(RAY_DEBUG, "finished streaming data, objref was " << objref << " and size was " << num_bytes); + RAY_CHECK_EQ(num_bytes, total_size, "Streamed objectid " << objectid << ", but num_bytes != total_size"); + object_ready(objectid, chunk.metadata_offset()); + RAY_LOG(RAY_DEBUG, "finished streaming data, objectid was " << objectid << " and size was " << num_bytes); } ObjStoreService::ObjStoreService(const std::string& objstore_address, std::shared_ptr scheduler_channel) @@ -66,25 +66,25 @@ Status ObjStoreService::StartDelivery(ServerContext* context, const StartDeliver // because the scheduler holds a lock while DeliverObj is being called. The correct solution is to make DeliverObj // an asynchronous call (and similarly with the rest of the object store service methods). std::string address = request->objstore_address(); - ObjRef objref = request->objref(); + ObjectID objectid = request->objectid(); { std::lock_guard memory_lock(memory_lock_); - if (objref >= memory_.size()) { - memory_.resize(objref + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); + if (objectid >= memory_.size()) { + memory_.resize(objectid + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); } - if (memory_[objref].second == MemoryStatusType::NOT_PRESENT) { + if (memory_[objectid].second == MemoryStatusType::NOT_PRESENT) { } else { - RAY_CHECK_NEQ(memory_[objref].second, MemoryStatusType::DEALLOCATED, "Objstore " << objstoreid_ << " is attempting to get objref " << objref << ", but memory_[objref] == DEALLOCATED."); - RAY_LOG(RAY_DEBUG, "Objstore " << objstoreid_ << " already has objref " << objref << " or it is already being shipped, so no need to get it again."); + RAY_CHECK_NEQ(memory_[objectid].second, MemoryStatusType::DEALLOCATED, "Objstore " << objstoreid_ << " is attempting to get objectid " << objectid << ", but memory_[objectid] == DEALLOCATED."); + RAY_LOG(RAY_DEBUG, "Objstore " << objstoreid_ << " already has objectid " << objectid << " or it is already being shipped, so no need to get it again."); return Status::OK; } - memory_[objref].second = MemoryStatusType::PRE_ALLOCED; + memory_[objectid].second = MemoryStatusType::PRE_ALLOCED; } - delivery_threads_.push_back(std::make_shared([this, address, objref]() { + delivery_threads_.push_back(std::make_shared([this, address, objectid]() { std::lock_guard objstores_lock(objstores_lock_); ObjStore::Stub& stub = get_objstore_stub(address); - get_data_from(objref, stub); + get_data_from(objectid, stub); })); return Status::OK; } @@ -93,14 +93,14 @@ Status ObjStoreService::ObjStoreInfo(ServerContext* context, const ObjStoreInfoR std::lock_guard memory_lock(memory_lock_); for (size_t i = 0; i < memory_.size(); ++i) { if (memory_[i].second == MemoryStatusType::READY) { // is the object available? - reply->add_objref(i); + reply->add_objectid(i); } } /* - for (int i = 0; i < request->objref_size(); ++i) { - ObjRef objref = request->objref(i); + for (int i = 0; i < request->objectid_size(); ++i) { + ObjectID objectid = request->objectid(i); Obj* obj = new Obj(); - std::string data(memory_[objref].ptr.data, memory_[objref].ptr.len); // copies, but for debugging should be ok + std::string data(memory_[objectid].ptr.data, memory_[objectid].ptr.len); // copies, but for debugging should be ok obj->ParseFromString(data); reply->mutable_obj()->AddAllocated(obj); } @@ -111,11 +111,11 @@ Status ObjStoreService::ObjStoreInfo(ServerContext* context, const ObjStoreInfoR Status ObjStoreService::StreamObjTo(ServerContext* context, const StreamObjToRequest* request, ServerWriter* writer) { RAY_LOG(RAY_DEBUG, "begin to stream data from object store " << objstoreid_); ObjChunk chunk; - ObjRef objref = request->objref(); + ObjectID objectid = request->objectid(); memory_lock_.lock(); - RAY_CHECK_LT(objref, memory_.size(), "Objstore " << objstoreid_ << " is attempting to use objref " << objref << " in StreamObjTo, but this objref is not present in the object store."); - RAY_CHECK_EQ(memory_[objref].second, MemoryStatusType::READY, "Objstore " << objstoreid_ << " is attempting to stream objref " << objref << ", but memory_[objref].second != MemoryStatusType::READY."); - ObjHandle handle = memory_[objref].first; + RAY_CHECK_LT(objectid, memory_.size(), "Objstore " << objstoreid_ << " is attempting to use objectid " << objectid << " in StreamObjTo, but this objectid is not present in the object store."); + RAY_CHECK_EQ(memory_[objectid].second, MemoryStatusType::READY, "Objstore " << objstoreid_ << " is attempting to stream objectid " << objectid << ", but memory_[objectid].second != MemoryStatusType::READY."); + ObjHandle handle = memory_[objectid].first; memory_lock_.unlock(); // TODO(rkn): Make sure we don't still need to hold on to this lock. segmentpool_lock_.lock(); const uint8_t* head = segmentpool_->get_address(handle); @@ -131,39 +131,39 @@ Status ObjStoreService::StreamObjTo(ServerContext* context, const StreamObjToReq } Status ObjStoreService::NotifyAlias(ServerContext* context, const NotifyAliasRequest* request, AckReply* reply) { - // NotifyAlias assumes that the objstore already holds canonical_objref - ObjRef alias_objref = request->alias_objref(); - ObjRef canonical_objref = request->canonical_objref(); - RAY_LOG(RAY_DEBUG, "Aliasing objref " << alias_objref << " with objref " << canonical_objref); + // NotifyAlias assumes that the objstore already holds canonical_objectid + ObjectID alias_objectid = request->alias_objectid(); + ObjectID canonical_objectid = request->canonical_objectid(); + RAY_LOG(RAY_DEBUG, "Aliasing objectid " << alias_objectid << " with objectid " << canonical_objectid); { std::lock_guard memory_lock(memory_lock_); - RAY_CHECK_LT(canonical_objref, memory_.size(), "Attempting to alias objref " << alias_objref << " with objref " << canonical_objref << ", but objref " << canonical_objref << " is not in the objstore.") - RAY_CHECK_NEQ(memory_[canonical_objref].second, MemoryStatusType::NOT_READY, "Attempting to alias objref " << alias_objref << " with objref " << canonical_objref << ", but objref " << canonical_objref << " is not ready yet in the objstore.") - RAY_CHECK_NEQ(memory_[canonical_objref].second, MemoryStatusType::NOT_PRESENT, "Attempting to alias objref " << alias_objref << " with objref " << canonical_objref << ", but objref " << canonical_objref << " is not present in the objstore.") - RAY_CHECK_NEQ(memory_[canonical_objref].second, MemoryStatusType::DEALLOCATED, "Attempting to alias objref " << alias_objref << " with objref " << canonical_objref << ", but objref " << canonical_objref << " has already been deallocated.") - if (alias_objref >= memory_.size()) { - memory_.resize(alias_objref + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); + RAY_CHECK_LT(canonical_objectid, memory_.size(), "Attempting to alias objectid " << alias_objectid << " with objectid " << canonical_objectid << ", but objectid " << canonical_objectid << " is not in the objstore.") + RAY_CHECK_NEQ(memory_[canonical_objectid].second, MemoryStatusType::NOT_READY, "Attempting to alias objectid " << alias_objectid << " with objectid " << canonical_objectid << ", but objectid " << canonical_objectid << " is not ready yet in the objstore.") + RAY_CHECK_NEQ(memory_[canonical_objectid].second, MemoryStatusType::NOT_PRESENT, "Attempting to alias objectid " << alias_objectid << " with objectid " << canonical_objectid << ", but objectid " << canonical_objectid << " is not present in the objstore.") + RAY_CHECK_NEQ(memory_[canonical_objectid].second, MemoryStatusType::DEALLOCATED, "Attempting to alias objectid " << alias_objectid << " with objectid " << canonical_objectid << ", but objectid " << canonical_objectid << " has already been deallocated.") + if (alias_objectid >= memory_.size()) { + memory_.resize(alias_objectid + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); } - memory_[alias_objref].first = memory_[canonical_objref].first; - memory_[alias_objref].second = MemoryStatusType::READY; + memory_[alias_objectid].first = memory_[canonical_objectid].first; + memory_[alias_objectid].second = MemoryStatusType::READY; } ObjRequest done_request; done_request.type = ObjRequestType::ALIAS_DONE; - done_request.objref = alias_objref; + done_request.objectid = alias_objectid; RAY_CHECK(recv_queue_.send(&done_request), "error sending over IPC"); return Status::OK; } Status ObjStoreService::DeallocateObject(ServerContext* context, const DeallocateObjectRequest* request, AckReply* reply) { - ObjRef canonical_objref = request->canonical_objref(); - RAY_LOG(RAY_INFO, "Deallocating canonical_objref " << canonical_objref); + ObjectID canonical_objectid = request->canonical_objectid(); + RAY_LOG(RAY_INFO, "Deallocating canonical_objectid " << canonical_objectid); std::lock_guard memory_lock(memory_lock_); - RAY_CHECK_EQ(memory_[canonical_objref].second, MemoryStatusType::READY, "Attempting to deallocate canonical_objref " << canonical_objref << ", but memory_[canonical_objref].second = " << memory_[canonical_objref].second); - RAY_CHECK_LT(canonical_objref, memory_.size(), "Attempting to deallocate canonical_objref " << canonical_objref << ", but it is not in the objstore."); + RAY_CHECK_EQ(memory_[canonical_objectid].second, MemoryStatusType::READY, "Attempting to deallocate canonical_objectid " << canonical_objectid << ", but memory_[canonical_objectid].second = " << memory_[canonical_objectid].second); + RAY_CHECK_LT(canonical_objectid, memory_.size(), "Attempting to deallocate canonical_objectid " << canonical_objectid << ", but it is not in the objstore."); segmentpool_lock_.lock(); - segmentpool_->deallocate(memory_[canonical_objref].first); + segmentpool_->deallocate(memory_[canonical_objectid].first); segmentpool_lock_.unlock(); - memory_[canonical_objref].second = MemoryStatusType::DEALLOCATED; + memory_[canonical_objectid].second = MemoryStatusType::DEALLOCATED; return Status::OK; } @@ -180,7 +180,7 @@ Status ObjStoreService::DeallocateObject(ServerContext* context, const Deallocat void ObjStoreService::process_objstore_request(const ObjRequest request) { switch (request.type) { case ObjRequestType::ALIAS_DONE: { - process_gets_for_objref(request.objref); + process_gets_for_objectid(request.objectid); } break; default: { @@ -199,32 +199,32 @@ void ObjStoreService::process_worker_request(const ObjRequest request) { } { std::lock_guard memory_lock(memory_lock_); - if (request.objref >= memory_.size()) { - memory_.resize(request.objref + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); + if (request.objectid >= memory_.size()) { + memory_.resize(request.objectid + 1, std::make_pair(ObjHandle(), MemoryStatusType::NOT_PRESENT)); } } switch (request.type) { case ObjRequestType::ALLOC: { - ObjHandle handle = alloc(request.objref, request.size); // This method acquires memory_lock_ + ObjHandle handle = alloc(request.objectid, request.size); // This method acquires memory_lock_ RAY_CHECK(send_queues_[request.workerid].send(&handle), "error sending over IPC"); } break; case ObjRequestType::GET: { std::lock_guard memory_lock(memory_lock_); - std::pair& item = memory_[request.objref]; + std::pair& item = memory_[request.objectid]; if (item.second == MemoryStatusType::READY) { - RAY_LOG(RAY_DEBUG, "Responding to GET request: returning objref " << request.objref); + RAY_LOG(RAY_DEBUG, "Responding to GET request: returning objectid " << request.objectid); RAY_CHECK(send_queues_[request.workerid].send(&item.first), "error sending over IPC"); } else if (item.second == MemoryStatusType::NOT_READY || item.second == MemoryStatusType::NOT_PRESENT || item.second == MemoryStatusType::PRE_ALLOCED) { std::lock_guard lock(get_queue_lock_); - get_queue_.push_back(std::make_pair(request.workerid, request.objref)); + get_queue_.push_back(std::make_pair(request.workerid, request.objectid)); } else { - RAY_CHECK(false, "A worker requested objref " << request.objref << ", but memory_[objref].second = " << memory_[request.objref].second); + RAY_CHECK(false, "A worker requested objectid " << request.objectid << ", but memory_[objectid].second = " << memory_[request.objectid].second); } } break; case ObjRequestType::WORKER_DONE: { - object_ready(request.objref, request.metadata_offset); // This method acquires memory_lock_ + object_ready(request.objectid, request.metadata_offset); // This method acquires memory_lock_ } break; default: { @@ -240,17 +240,17 @@ void ObjStoreService::process_requests() { RAY_CHECK(recv_queue_.receive(&request), "error receiving over IPC"); switch (request.type) { case ObjRequestType::ALLOC: { - RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Allocate object with objref " << request.objref << " and size " << request.size); + RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Allocate object with objectid " << request.objectid << " and size " << request.size); process_worker_request(request); } break; case ObjRequestType::GET: { - RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Get object with objref " << request.objref); + RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Get object with objectid " << request.objectid); process_worker_request(request); } break; case ObjRequestType::WORKER_DONE: { - RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Finalize object with objref " << request.objref); + RAY_LOG(RAY_VERBOSE, "Request (worker " << request.workerid << " to objstore " << objstoreid_ << "): Finalize object with objectid " << request.objectid); process_worker_request(request); } break; @@ -265,12 +265,12 @@ void ObjStoreService::process_requests() { } } -void ObjStoreService::process_gets_for_objref(ObjRef objref) { - std::pair& item = memory_[objref]; +void ObjStoreService::process_gets_for_objectid(ObjectID objectid) { + std::pair& item = memory_[objectid]; std::lock_guard get_queue_lock(get_queue_lock_); for (size_t i = 0; i < get_queue_.size(); ++i) { - if (get_queue_[i].second == objref) { - ObjHandle& elem = memory_[objref].first; + if (get_queue_[i].second == objectid) { + ObjHandle& elem = memory_[objectid].first; RAY_CHECK(send_queues_[get_queue_[i].first].send(&item.first), "error sending over IPC"); // Remove the get task from the queue std::swap(get_queue_[i], get_queue_[get_queue_.size() - 1]); @@ -280,33 +280,33 @@ void ObjStoreService::process_gets_for_objref(ObjRef objref) { } } -ObjHandle ObjStoreService::alloc(ObjRef objref, size_t size) { +ObjHandle ObjStoreService::alloc(ObjectID objectid, size_t size) { segmentpool_lock_.lock(); ObjHandle handle = segmentpool_->allocate(size); segmentpool_lock_.unlock(); std::lock_guard memory_lock(memory_lock_); - RAY_LOG(RAY_VERBOSE, "Allocating space for objref " << objref << " on object store " << objstoreid_); - RAY_CHECK(memory_[objref].second == MemoryStatusType::NOT_PRESENT || memory_[objref].second == MemoryStatusType::PRE_ALLOCED, "Attempting to allocate space for objref " << objref << ", but memory_[objref].second = " << memory_[objref].second); - memory_[objref].first = handle; - memory_[objref].second = MemoryStatusType::NOT_READY; + RAY_LOG(RAY_VERBOSE, "Allocating space for objectid " << objectid << " on object store " << objstoreid_); + RAY_CHECK(memory_[objectid].second == MemoryStatusType::NOT_PRESENT || memory_[objectid].second == MemoryStatusType::PRE_ALLOCED, "Attempting to allocate space for objectid " << objectid << ", but memory_[objectid].second = " << memory_[objectid].second); + memory_[objectid].first = handle; + memory_[objectid].second = MemoryStatusType::NOT_READY; return handle; } -void ObjStoreService::object_ready(ObjRef objref, size_t metadata_offset) { +void ObjStoreService::object_ready(ObjectID objectid, size_t metadata_offset) { { - RAY_LOG(RAY_INFO, "Objref " << objref << " is ready."); + RAY_LOG(RAY_INFO, "Object with ObjectID " << objectid << " is ready."); std::lock_guard memory_lock(memory_lock_); - std::pair& item = memory_[objref]; - RAY_CHECK_EQ(item.second, MemoryStatusType::NOT_READY, "A worker notified the object store that objref " << objref << " has been written to the object store, but memory_[objref].second != NOT_READY."); + std::pair& item = memory_[objectid]; + RAY_CHECK_EQ(item.second, MemoryStatusType::NOT_READY, "A worker notified the object store that objectid " << objectid << " has been written to the object store, but memory_[objectid].second != NOT_READY."); item.first.set_metadata_offset(metadata_offset); item.second = MemoryStatusType::READY; } - process_gets_for_objref(objref); + process_gets_for_objectid(objectid); // Tell the scheduler that the object arrived // TODO(pcm): put this in a separate thread so we don't have to pay the latency here ClientContext objready_context; ObjReadyRequest objready_request; - objready_request.set_objref(objref); + objready_request.set_objectid(objectid); objready_request.set_objstoreid(objstoreid_); AckReply objready_reply; scheduler_stub_->ObjReady(&objready_context, objready_request, &objready_reply); diff --git a/src/objstore.h b/src/objstore.h index f0088781b..81a3531e2 100644 --- a/src/objstore.h +++ b/src/objstore.h @@ -46,27 +46,27 @@ public: Status ObjStoreInfo(ServerContext* context, const ObjStoreInfoRequest* request, ObjStoreInfoReply* reply) override; void start_objstore_service(); private: - void get_data_from(ObjRef objref, ObjStore::Stub& stub); + void get_data_from(ObjectID objectid, ObjStore::Stub& stub); // check if we already connected to the other objstore, if yes, return reference to connection, otherwise connect ObjStore::Stub& get_objstore_stub(const std::string& objstore_address); void process_worker_request(const ObjRequest request); void process_objstore_request(const ObjRequest request); void process_requests(); - void process_gets_for_objref(ObjRef objref); - ObjHandle alloc(ObjRef objref, size_t size); - void object_ready(ObjRef objref, size_t metadata_offset); + void process_gets_for_objectid(ObjectID objectid); + ObjHandle alloc(ObjectID objectid, size_t size); + void object_ready(ObjectID objectid, size_t metadata_offset); static const size_t CHUNK_SIZE; std::string objstore_address_; ObjStoreId objstoreid_; // id of this objectstore in the scheduler object store table std::shared_ptr segmentpool_; std::mutex segmentpool_lock_; - std::vector > memory_; // object reference -> (memory address, memory status) + std::vector > memory_; // object ID -> (memory address, memory status) std::mutex memory_lock_; std::unordered_map> objstores_; std::mutex objstores_lock_; std::unique_ptr scheduler_stub_; - std::vector > get_queue_; + std::vector > get_queue_; std::mutex get_queue_lock_; MessageQueue recv_queue_; // This queue is used by workers to send tasks to the object store. std::vector > send_queues_; // This maps workerid -> queue. The object store uses these queues to send replies to the relevant workers. diff --git a/src/raylib.cc b/src/raylib.cc index 5b4e0aa2e..bdc7a6cf4 100644 --- a/src/raylib.cc +++ b/src/raylib.cc @@ -22,79 +22,79 @@ static int PyObjectToWorker(PyObject* object, Worker **worker); typedef struct { PyObject_HEAD - ObjRef val; - // We give the PyObjRef object a reference to the worker capsule object to + ObjectID id; + // We give the PyObjectID object a reference to the worker capsule object to // make sure that the worker capsule does not go out of scope until all of the // object references have gone out of scope. The reason for this is that the // worker capsule destructor destroys the worker object. If the worker object // has been destroyed, then when the object reference tries to call // worker->decrement_reference_count, we can get a segfault. PyObject* worker_capsule; -} PyObjRef; +} PyObjectID; -static void PyObjRef_dealloc(PyObjRef *self) { +static void PyObjectID_dealloc(PyObjectID *self) { Worker* worker; PyObjectToWorker(self->worker_capsule, &worker); - std::vector objrefs; - objrefs.push_back(self->val); - worker->decrement_reference_count(objrefs); - Py_DECREF(self->worker_capsule); // The corresponding increment happens in PyObjRef_init. + std::vector objectids; + objectids.push_back(self->id); + worker->decrement_reference_count(objectids); + Py_DECREF(self->worker_capsule); // The corresponding increment happens in PyObjectID_init. self->ob_type->tp_free((PyObject*) self); } -static PyObject* PyObjRef_new(PyTypeObject *type, PyObject *args, PyObject *kwds) { - PyObjRef* self = (PyObjRef*) type->tp_alloc(type, 0); +static PyObject* PyObjectID_new(PyTypeObject *type, PyObject *args, PyObject *kwds) { + PyObjectID* self = (PyObjectID*) type->tp_alloc(type, 0); if (self != NULL) { - self->val = 0; + self->id = 0; } return (PyObject*) self; } -static int PyObjRef_init(PyObjRef *self, PyObject *args, PyObject *kwds) { - if (!PyArg_ParseTuple(args, "iO", &self->val, &self->worker_capsule)) { +static int PyObjectID_init(PyObjectID *self, PyObject *args, PyObject *kwds) { + if (!PyArg_ParseTuple(args, "iO", &self->id, &self->worker_capsule)) { return -1; } Worker* worker; PyObjectToWorker(self->worker_capsule, &worker); - Py_INCREF(self->worker_capsule); // The corresponding decrement happens in PyObjRef_dealloc. - std::vector objrefs; - objrefs.push_back(self->val); - RAY_LOG(RAY_REFCOUNT, "In PyObjRef_init, calling increment_reference_count for objref " << objrefs[0]); - worker->increment_reference_count(objrefs); + Py_INCREF(self->worker_capsule); // The corresponding decrement happens in PyObjectID_dealloc. + std::vector objectids; + objectids.push_back(self->id); + RAY_LOG(RAY_REFCOUNT, "In PyObjectID_init, calling increment_reference_count for objectid " << objectids[0]); + worker->increment_reference_count(objectids); return 0; }; -static int PyObjRef_compare(PyObject* a, PyObject* b) { - PyObjRef* A = (PyObjRef*) a; - PyObjRef* B = (PyObjRef*) b; - if (A->val < B->val) { +static int PyObjectID_compare(PyObject* a, PyObject* b) { + PyObjectID* A = (PyObjectID*) a; + PyObjectID* B = (PyObjectID*) b; + if (A->id < B->id) { return -1; } - if (A->val > B->val) { + if (A->id > B->id) { return 1; } return 0; } -char RAY_VAL_LITERAL[] = "val"; -char RAY_OBJECT_REFERENCE_LITERAL[] = "object reference"; +char RAY_ID_LITERAL[] = "id"; +char RAY_OBJECT_ID_LITERAL[] = "object id"; -static PyMemberDef PyObjRef_members[] = { - {RAY_VAL_LITERAL, T_INT, offsetof(PyObjRef, val), 0, RAY_OBJECT_REFERENCE_LITERAL}, +static PyMemberDef PyObjectID_members[] = { + {RAY_ID_LITERAL, T_INT, offsetof(PyObjectID, id), 0, RAY_OBJECT_ID_LITERAL}, {NULL} }; -static PyTypeObject PyObjRefType = { +static PyTypeObject PyObjectIDType = { PyObject_HEAD_INIT(NULL) 0, /* ob_size */ - "ray.ObjRef", /* tp_name */ - sizeof(PyObjRef), /* tp_basicsize */ + "ray.ObjectID", /* tp_name */ + sizeof(PyObjectID), /* tp_basicsize */ 0, /* tp_itemsize */ - (destructor)PyObjRef_dealloc, /* tp_dealloc */ + (destructor)PyObjectID_dealloc, /* tp_dealloc */ 0, /* tp_print */ 0, /* tp_getattr */ 0, /* tp_setattr */ - PyObjRef_compare, /* tp_compare */ + PyObjectID_compare, /* tp_compare */ 0, /* tp_repr */ 0, /* tp_as_number */ 0, /* tp_as_sequence */ @@ -106,7 +106,7 @@ static PyTypeObject PyObjRefType = { 0, /* tp_setattro */ 0, /* tp_as_buffer */ Py_TPFLAGS_DEFAULT, /* tp_flags */ - "Ray objects", /* tp_doc */ + "Ray objects", /* tp_doc */ 0, /* tp_traverse */ 0, /* tp_clear */ 0, /* tp_richcompare */ @@ -114,22 +114,22 @@ static PyTypeObject PyObjRefType = { 0, /* tp_iter */ 0, /* tp_iternext */ 0, /* tp_methods */ - PyObjRef_members, /* tp_members */ + PyObjectID_members, /* tp_members */ 0, /* tp_getset */ 0, /* tp_base */ 0, /* tp_dict */ 0, /* tp_descr_get */ 0, /* tp_descr_set */ 0, /* tp_dictoffset */ - (initproc)PyObjRef_init, /* tp_init */ + (initproc)PyObjectID_init, /* tp_init */ 0, /* tp_alloc */ - PyObjRef_new, /* tp_new */ + PyObjectID_new, /* tp_new */ }; -// create PyObjRef from C++ (could be made more efficient if neccessary) -PyObject* make_pyobjref(PyObject* worker_capsule, ObjRef objref) { - PyObject* arglist = Py_BuildValue("(iO)", objref, worker_capsule); - PyObject* result = PyObject_CallObject((PyObject*) &PyObjRefType, arglist); +// create PyObjectID from C++ (could be made more efficient if neccessary) +PyObject* make_pyobjectid(PyObject* worker_capsule, ObjectID objectid) { + PyObject* arglist = Py_BuildValue("(iO)", objectid, worker_capsule); + PyObject* result = PyObject_CallObject((PyObject*) &PyObjectIDType, arglist); Py_DECREF(arglist); return result; } @@ -171,9 +171,9 @@ static int PyObjectToWorker(PyObject* object, Worker **worker) { } } -static int PyObjectToObjRef(PyObject* object, ObjRef *objref) { - if (PyObject_IsInstance(object, (PyObject*)&PyObjRefType)) { - *objref = ((PyObjRef*) object)->val; +static int PyObjectToObjectID(PyObject* object, ObjectID *objectid) { + if (PyObject_IsInstance(object, (PyObject*)&PyObjectIDType)) { + *objectid = ((PyObjectID*) object)->id; return 1; } else { PyErr_SetString(PyExc_TypeError, "must be an object reference"); @@ -227,8 +227,8 @@ void set_dict_item_and_transfer_ownership(PyObject* dict, PyObject* key, PyObjec // object obj, returns 0 if successful and something else if not // NOTE: If some primitive types are added here, they may also need to be handled in serialization.py // FIXME(pcm): This currently only works for contiguous arrays -// This method will push all of the object references contained in `obj` to the `objrefs` vector. -int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vector &objrefs) { +// This method will push all of the object references contained in `obj` to the `objectids` vector. +int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vector &objectids) { if (PyBool_Check(val)) { // The bool case must precede the int case because PyInt_Check passes for bools Bool* data = obj->mutable_bool_data(); @@ -257,7 +257,7 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vectormutable_tuple_data(); for (size_t i = 0, size = PyTuple_Size(val); i < size; ++i) { Obj* elem = data->add_elem(); - if (serialize(worker_capsule, PyTuple_GetItem(val, i), elem, objrefs) != 0) { + if (serialize(worker_capsule, PyTuple_GetItem(val, i), elem, objectids) != 0) { return -1; } } @@ -265,7 +265,7 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vectormutable_list_data(); for (size_t i = 0, size = PyList_Size(val); i < size; ++i) { Obj* elem = data->add_elem(); - if (serialize(worker_capsule, PyList_GetItem(val, i), elem, objrefs) != 0) { + if (serialize(worker_capsule, PyList_GetItem(val, i), elem, objectids) != 0) { return -1; } } @@ -276,11 +276,11 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vectoradd_elem(); Obj* key = elem->mutable_key(); - if (serialize(worker_capsule, pykey, key, objrefs) != 0) { + if (serialize(worker_capsule, pykey, key, objectids) != 0) { return -1; } Obj* value = elem->mutable_value(); - if (serialize(worker_capsule, pyvalue, value, objrefs) != 0) { + if (serialize(worker_capsule, pyvalue, value, objectids) != 0) { return -1; } } @@ -291,11 +291,11 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vectormutable_string_data()->set_data(buffer, length); } else if (val == Py_None) { obj->mutable_empty_data(); // allocate an Empty object, this is a None - } else if (PyObject_IsInstance(val, (PyObject*) &PyObjRefType)) { - ObjRef objref = ((PyObjRef*) val)->val; - Ref* data = obj->mutable_objref_data(); - data->set_data(objref); - objrefs.push_back(objref); + } else if (PyObject_IsInstance(val, (PyObject*) &PyObjectIDType)) { + ObjectID objectid = ((PyObjectID*) val)->id; + ObjID* data = obj->mutable_objectid_data(); + data->set_data(objectid); + objectids.push_back(objectid); } else if (PyArray_Check(val) || PyArray_CheckScalar(val)) { // Python int and float already handled Array* data = obj->mutable_array_data(); PyArrayObject* array; // will be deallocated at the end @@ -324,19 +324,19 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vectorval; + ObjectID objectid; + if (PyObject_IsInstance(*item, (PyObject*) &PyObjectIDType)) { + objectid = ((PyObjectID*) (*item))->id; } else { PyErr_SetString(PyExc_TypeError, "must be an object reference"); // TODO: improve error message return -1; } - data->add_objref_data(objref); - objrefs.push_back(objref); + data->add_objectid_data(objectid); + objectids.push_back(objectid); PyArray_ITER_NEXT(iter); } Py_XDECREF(iter); @@ -364,8 +364,8 @@ int serialize(PyObject* worker_capsule, PyObject* val, Obj* obj, std::vector &objrefs) { +// This method will push all of the object references contained in `obj` to the `objectids` vector. +static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vector &objectids) { if (obj.has_int_data()) { return PyInt_FromLong(obj.int_data().data()); } else if (obj.has_long_data()) { @@ -383,7 +383,7 @@ static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vect size_t size = data.elem_size(); PyObject* tuple = PyTuple_New(size); for (size_t i = 0; i < size; ++i) { - PyTuple_SetItem(tuple, i, deserialize(worker_capsule, data.elem(i), objrefs)); + PyTuple_SetItem(tuple, i, deserialize(worker_capsule, data.elem(i), objectids)); } return tuple; } else if (obj.has_list_data()) { @@ -391,7 +391,7 @@ static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vect size_t size = data.elem_size(); PyObject* list = PyList_New(size); for (size_t i = 0; i < size; ++i) { - PyList_SetItem(list, i, deserialize(worker_capsule, data.elem(i), objrefs)); + PyList_SetItem(list, i, deserialize(worker_capsule, data.elem(i), objectids)); } return list; } else if (obj.has_dict_data()) { @@ -399,8 +399,8 @@ static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vect PyObject* dict = PyDict_New(); size_t size = data.elem_size(); for (size_t i = 0; i < size; ++i) { - PyObject* pykey = deserialize(worker_capsule, data.elem(i).key(), objrefs); - PyObject* pyval = deserialize(worker_capsule, data.elem(i).value(), objrefs); + PyObject* pykey = deserialize(worker_capsule, data.elem(i).key(), objectids); + PyObject* pyval = deserialize(worker_capsule, data.elem(i).value(), objectids); set_dict_item_and_transfer_ownership(dict, pykey, pyval); } return dict; @@ -410,9 +410,9 @@ static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vect return PyString_FromStringAndSize(buffer, length); } else if (obj.has_empty_data()) { Py_RETURN_NONE; - } else if (obj.has_objref_data()) { - objrefs.push_back(obj.objref_data().data()); - return make_pyobjref(worker_capsule, obj.objref_data().data()); + } else if (obj.has_objectid_data()) { + objectids.push_back(obj.objectid_data().data()); + return make_pyobjectid(worker_capsule, obj.objectid_data().data()); } else if (obj.has_array_data()) { const Array& array = obj.array_data(); std::vector dims; @@ -432,11 +432,11 @@ static PyObject* deserialize(PyObject* worker_capsule, const Obj& obj, std::vect RAYLIB_DESERIALIZE_NPY(UINT32, npy_uint32, uint) RAYLIB_DESERIALIZE_NPY(UINT64, npy_uint64, uint) case NPY_OBJECT: { - npy_intp size = array.objref_data_size(); + npy_intp size = array.objectid_data_size(); PyObject** buffer = (PyObject**) PyArray_DATA(pyarray); for (npy_intp i = 0; i < size; ++i) { - buffer[i] = make_pyobjref(worker_capsule, array.objref_data(i)); - objrefs.push_back(array.objref_data(i)); + buffer[i] = make_pyobjectid(worker_capsule, array.objectid_data(i)); + objectids.push_back(array.objectid_data(i)); } } break; @@ -463,31 +463,31 @@ static PyObject* serialize_object(PyObject* self, PyObject* args) { if (!PyArg_ParseTuple(args, "OO", &worker_capsule, &pyval)) { return NULL; } - std::vector objrefs; - if (serialize(worker_capsule, pyval, obj, objrefs) != 0) { + std::vector objectids; + if (serialize(worker_capsule, pyval, obj, objectids) != 0) { return NULL; } Worker* worker; PyObjectToWorker(worker_capsule, &worker); - PyObject* contained_objrefs = PyList_New(objrefs.size()); - for (int i = 0; i < objrefs.size(); ++i) { - PyList_SetItem(contained_objrefs, i, make_pyobjref(worker_capsule, objrefs[i])); + PyObject* contained_objectids = PyList_New(objectids.size()); + for (int i = 0; i < objectids.size(); ++i) { + PyList_SetItem(contained_objectids, i, make_pyobjectid(worker_capsule, objectids[i])); } PyObject* t = PyTuple_New(2); // We set the items of the tuple using PyTuple_SetItem, because that transfers ownership to the tuple. PyTuple_SetItem(t, 0, PyCapsule_New(static_cast(obj), "obj", &ObjCapsule_Destructor)); - PyTuple_SetItem(t, 1, contained_objrefs); + PyTuple_SetItem(t, 1, contained_objectids); return t; } static PyObject* allocate_buffer(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; + ObjectID objectid; SegmentId segmentid; long size; - if (!PyArg_ParseTuple(args, "O&O&l", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref, &size)) { + if (!PyArg_ParseTuple(args, "O&O&l", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid, &size)) { return NULL; } - void* address = reinterpret_cast(const_cast(worker->allocate_buffer(objref, size, segmentid))); + void* address = reinterpret_cast(const_cast(worker->allocate_buffer(objectid, size, segmentid))); std::vector dim({size}); PyObject* t = PyTuple_New(2); PyTuple_SetItem(t, 0, PyArray_SimpleNewFromData(1, dim.data(), NPY_BYTE, address)); @@ -497,25 +497,25 @@ static PyObject* allocate_buffer(PyObject* self, PyObject* args) { static PyObject* finish_buffer(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; + ObjectID objectid; long segmentid; long metadata_offset; - if (!PyArg_ParseTuple(args, "O&O&ll", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref, &segmentid, &metadata_offset)) { + if (!PyArg_ParseTuple(args, "O&O&ll", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid, &segmentid, &metadata_offset)) { return NULL; } - return worker->finish_buffer(objref, segmentid, metadata_offset); + return worker->finish_buffer(objectid, segmentid, metadata_offset); } static PyObject* get_buffer(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; + ObjectID objectid; int64_t size; SegmentId segmentid; int64_t metadata_offset; - if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref)) { + if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid)) { return NULL; } - void* address = reinterpret_cast(const_cast(worker->get_buffer(objref, size, segmentid, metadata_offset))); + void* address = reinterpret_cast(const_cast(worker->get_buffer(objectid, size, segmentid, metadata_offset))); std::vector dim({static_cast(size)}); PyObject* t = PyTuple_New(3); PyTuple_SetItem(t, 0, PyArray_SimpleNewFromData(1, dim.data(), NPY_BYTE, address)); @@ -526,11 +526,11 @@ static PyObject* get_buffer(PyObject* self, PyObject* args) { static PyObject* is_arrow(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; - if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref)) { + ObjectID objectid; + if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid)) { return NULL; } - if (worker->is_arrow(objref)) + if (worker->is_arrow(objectid)) Py_RETURN_TRUE; else Py_RETURN_FALSE; @@ -552,9 +552,9 @@ static PyObject* deserialize_object(PyObject* self, PyObject* args) { if (!PyArg_ParseTuple(args, "OO&", &worker_capsule, &PyObjectToObj, &obj)) { return NULL; } - std::vector objrefs; // This is a vector of all the objrefs that are serialized in this task, including objrefs that are contained in Python objects that are passed by value. - return deserialize(worker_capsule, *obj, objrefs); - // TODO(rkn): Should we do anything with objrefs? + std::vector objectids; // This is a vector of all the objectids that are serialized in this task, including objectids that are contained in Python objects that are passed by value. + return deserialize(worker_capsule, *obj, objectids); + // TODO(rkn): Should we do anything with objectids? } static PyObject* serialize_task(PyObject* self, PyObject* args) { @@ -567,17 +567,17 @@ static PyObject* serialize_task(PyObject* self, PyObject* args) { return NULL; } task->set_name(name, len); - std::vector objrefs; // This is a vector of all the objrefs that are serialized in this task, including objrefs that are contained in Python objects that are passed by value. + std::vector objectids; // This is a vector of all the objectids that are serialized in this task, including objectids that are contained in Python objects that are passed by value. if (PyList_Check(arguments)) { for (size_t i = 0, size = PyList_Size(arguments); i < size; ++i) { PyObject* element = PyList_GetItem(arguments, i); - if (PyObject_IsInstance(element, (PyObject*)&PyObjRefType)) { - ObjRef objref = ((PyObjRef*) element)->val; - task->add_arg()->set_ref(objref); - objrefs.push_back(objref); + if (PyObject_IsInstance(element, (PyObject*)&PyObjectIDType)) { + ObjectID objectid = ((PyObjectID*) element)->id; + task->add_arg()->set_id(objectid); + objectids.push_back(objectid); } else { Obj* arg = task->add_arg()->mutable_obj(); - serialize(worker_capsule, PyList_GetItem(arguments, i), arg, objrefs); + serialize(worker_capsule, PyList_GetItem(arguments, i), arg, objectids); } } } else { @@ -586,9 +586,9 @@ static PyObject* serialize_task(PyObject* self, PyObject* args) { } Worker* worker; PyObjectToWorker(worker_capsule, &worker); - if (objrefs.size() > 0) { - RAY_LOG(RAY_REFCOUNT, "In serialize_task, calling increment_reference_count for contained objrefs"); - worker->increment_reference_count(objrefs); + if (objectids.size() > 0) { + RAY_LOG(RAY_REFCOUNT, "In serialize_task, calling increment_reference_count for contained objectids"); + worker->increment_reference_count(objectids); } std::string output; task->SerializeToString(&output); @@ -608,30 +608,30 @@ static PyObject* serialize_task(PyObject* self, PyObject* args) { } static PyObject* deserialize_task(PyObject* worker_capsule, const Task& task) { - std::vector objrefs; // This is a vector of all the objrefs that were serialized in this task, including objrefs that are contained in Python objects that are passed by value. + std::vector objectids; // This is a vector of all the objectids that were serialized in this task, including objectids that are contained in Python objects that are passed by value. PyObject* string = PyString_FromStringAndSize(task.name().c_str(), task.name().size()); int argsize = task.arg_size(); PyObject* arglist = PyList_New(argsize); for (int i = 0; i < argsize; ++i) { const Value& val = task.arg(i); if (!val.has_obj()) { - PyList_SetItem(arglist, i, make_pyobjref(worker_capsule, val.ref())); - objrefs.push_back(val.ref()); + PyList_SetItem(arglist, i, make_pyobjectid(worker_capsule, val.id())); + objectids.push_back(val.id()); } else { - PyList_SetItem(arglist, i, deserialize(worker_capsule, val.obj(), objrefs)); + PyList_SetItem(arglist, i, deserialize(worker_capsule, val.obj(), objectids)); } } Worker* worker; PyObjectToWorker(worker_capsule, &worker); - worker->decrement_reference_count(objrefs); + worker->decrement_reference_count(objectids); int resultsize = task.result_size(); - std::vector result_objrefs; + std::vector result_objectids; PyObject* resultlist = PyList_New(resultsize); for (int i = 0; i < resultsize; ++i) { - PyList_SetItem(resultlist, i, make_pyobjref(worker_capsule, task.result(i))); - result_objrefs.push_back(task.result(i)); + PyList_SetItem(resultlist, i, make_pyobjectid(worker_capsule, task.result(i))); + result_objectids.push_back(task.result(i)); } - worker->decrement_reference_count(result_objrefs); // The corresponding increment is done in SubmitTask in the scheduler. + worker->decrement_reference_count(result_objectids); // The corresponding increment is done in SubmitTask in the scheduler. PyObject* t = PyTuple_New(3); // We set the items of the tuple using PyTuple_SetItem, because that transfers ownership to the tuple. PyTuple_SetItem(t, 0, string); PyTuple_SetItem(t, 1, arglist); @@ -764,12 +764,12 @@ static PyObject* submit_task(PyObject* self, PyObject* args) { request.release_task(); // TODO: Make sure that task is not moved, otherwise capsule pointer needs to be updated int size = reply.result_size(); PyObject* list = PyList_New(size); - std::vector result_objrefs; + std::vector result_objectids; for (int i = 0; i < size; ++i) { - PyList_SetItem(list, i, make_pyobjref(worker_capsule, reply.result(i))); - result_objrefs.push_back(reply.result(i)); + PyList_SetItem(list, i, make_pyobjectid(worker_capsule, reply.result(i))); + result_objectids.push_back(reply.result(i)); } - worker->decrement_reference_count(result_objrefs); // The corresponding increment is done in SubmitTask in the scheduler. + worker->decrement_reference_count(result_objectids); // The corresponding increment is done in SubmitTask in the scheduler. return list; } @@ -797,45 +797,45 @@ static PyObject* register_function(PyObject* self, PyObject* args) { Py_RETURN_NONE; } -static PyObject* get_objref(PyObject* self, PyObject* args) { +static PyObject* get_objectid(PyObject* self, PyObject* args) { PyObject* worker_capsule; if (!PyArg_ParseTuple(args, "O", &worker_capsule)) { return NULL; } Worker* worker; PyObjectToWorker(worker_capsule, &worker); - ObjRef objref = worker->get_objref(); - return make_pyobjref(worker_capsule, objref); + ObjectID objectid = worker->get_objectid(); + return make_pyobjectid(worker_capsule, objectid); } static PyObject* put_object(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; + ObjectID objectid; Obj* obj; - PyObject* contained_objrefs; - if (!PyArg_ParseTuple(args, "O&O&O&O", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref, &PyObjectToObj, &obj, &contained_objrefs)) { + PyObject* contained_objectids; + if (!PyArg_ParseTuple(args, "O&O&O&O", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid, &PyObjectToObj, &obj, &contained_objectids)) { return NULL; } - RAY_CHECK(PyList_Check(contained_objrefs), "The contained_objrefs argument must be a list.") - std::vector vec_contained_objrefs; - size_t size = PyList_Size(contained_objrefs); + RAY_CHECK(PyList_Check(contained_objectids), "The contained_objectids argument must be a list.") + std::vector vec_contained_objectids; + size_t size = PyList_Size(contained_objectids); for (size_t i = 0; i < size; ++i) { - ObjRef contained_objref; - PyObjectToObjRef(PyList_GetItem(contained_objrefs, i), &contained_objref); - vec_contained_objrefs.push_back(contained_objref); + ObjectID contained_objectid; + PyObjectToObjectID(PyList_GetItem(contained_objectids, i), &contained_objectid); + vec_contained_objectids.push_back(contained_objectid); } - worker->put_object(objref, obj, vec_contained_objrefs); + worker->put_object(objectid, obj, vec_contained_objectids); Py_RETURN_NONE; } static PyObject* get_object(PyObject* self, PyObject* args) { - // get_object assumes that objref is a canonical objref + // get_object assumes that objectid is a canonical objectid Worker* worker; - ObjRef objref; - if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref)) { + ObjectID objectid; + if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid)) { return NULL; } - slice s = worker->get_object(objref); + slice s = worker->get_object(objectid); Obj* obj = new Obj(); // TODO: Make sure this will get deleted obj->ParseFromString(std::string(reinterpret_cast(s.data), s.len)); PyObject* result = PyList_New(2); @@ -846,22 +846,22 @@ static PyObject* get_object(PyObject* self, PyObject* args) { static PyObject* request_object(PyObject* self, PyObject* args) { Worker* worker; - ObjRef objref; - if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjRef, &objref)) { + ObjectID objectid; + if (!PyArg_ParseTuple(args, "O&O&", &PyObjectToWorker, &worker, &PyObjectToObjectID, &objectid)) { return NULL; } - worker->request_object(objref); + worker->request_object(objectid); Py_RETURN_NONE; } -static PyObject* alias_objrefs(PyObject* self, PyObject* args) { +static PyObject* alias_objectids(PyObject* self, PyObject* args) { Worker* worker; - ObjRef alias_objref; - ObjRef target_objref; - if (!PyArg_ParseTuple(args, "O&O&O&", &PyObjectToWorker, &worker, &PyObjectToObjRef, &alias_objref, &PyObjectToObjRef, &target_objref)) { + ObjectID alias_objectid; + ObjectID target_objectid; + if (!PyArg_ParseTuple(args, "O&O&O&", &PyObjectToWorker, &worker, &PyObjectToObjectID, &alias_objectid, &PyObjectToObjectID, &target_objectid)) { return NULL; } - worker->alias_objrefs(alias_objref, target_objref); + worker->alias_objectids(alias_objectid, target_objectid); Py_RETURN_NONE; } @@ -884,9 +884,9 @@ static PyObject* scheduler_info(PyObject* self, PyObject* args) { SchedulerInfoReply reply; worker->scheduler_info(context, request, reply); - PyObject* target_objref_list = PyList_New(reply.target_objref_size()); - for (size_t i = 0; i < reply.target_objref_size(); ++i) { - PyList_SetItem(target_objref_list, i, PyInt_FromLong(reply.target_objref(i))); + PyObject* target_objectid_list = PyList_New(reply.target_objectid_size()); + for (size_t i = 0; i < reply.target_objectid_size(); ++i) { + PyList_SetItem(target_objectid_list, i, PyInt_FromLong(reply.target_objectid(i))); } PyObject* reference_count_list = PyList_New(reply.reference_count_size()); for (size_t i = 0; i < reply.reference_count_size(); ++i) { @@ -894,7 +894,7 @@ static PyObject* scheduler_info(PyObject* self, PyObject* args) { } PyObject* dict = PyDict_New(); - set_dict_item_and_transfer_ownership(dict, PyString_FromString("target_objrefs"), target_objref_list); + set_dict_item_and_transfer_ownership(dict, PyString_FromString("target_objectids"), target_objectid_list); set_dict_item_and_transfer_ownership(dict, PyString_FromString("reference_counts"), reference_count_list); return dict; } @@ -979,9 +979,9 @@ static PyObject* kill_workers(PyObject* self, PyObject* args) { static PyMethodDef RayLibMethods[] = { { "serialize_object", serialize_object, METH_VARARGS, "serialize an object to protocol buffers" }, { "deserialize_object", deserialize_object, METH_VARARGS, "deserialize an object from protocol buffers" }, - { "allocate_buffer", allocate_buffer, METH_VARARGS, "Allocates and returns buffer for objref."}, - { "finish_buffer", finish_buffer, METH_VARARGS, "Makes the buffer immutable and closes memory segment of objref."}, - { "get_buffer", get_buffer, METH_VARARGS, "Gets buffer for objref"}, + { "allocate_buffer", allocate_buffer, METH_VARARGS, "Allocates and returns buffer for objectid."}, + { "finish_buffer", finish_buffer, METH_VARARGS, "Makes the buffer immutable and closes memory segment of objectid."}, + { "get_buffer", get_buffer, METH_VARARGS, "Gets buffer for objectid"}, { "is_arrow", is_arrow, METH_VARARGS, "is the object in the local object store an arrow object?"}, { "unmap_object", unmap_object, METH_VARARGS, "unmap the object from the client's shared memory pool"}, { "serialize_task", serialize_task, METH_VARARGS, "serialize a task to protocol buffers" }, @@ -991,9 +991,9 @@ static PyMethodDef RayLibMethods[] = { { "register_function", register_function, METH_VARARGS, "register a function with the scheduler" }, { "put_object", put_object, METH_VARARGS, "put a protocol buffer object (given as a capsule) on the local object store" }, { "get_object", get_object, METH_VARARGS, "get protocol buffer object from the local object store" }, - { "get_objref", get_objref, METH_VARARGS, "register a new object reference with the scheduler" }, + { "get_objectid", get_objectid, METH_VARARGS, "register a new object reference with the scheduler" }, { "request_object" , request_object, METH_VARARGS, "request an object to be delivered to the local object store" }, - { "alias_objrefs", alias_objrefs, METH_VARARGS, "make two objrefs refer to the same object" }, + { "alias_objectids", alias_objectids, METH_VARARGS, "make two objectids refer to the same object" }, { "wait_for_next_message", wait_for_next_message, METH_VARARGS, "get next message from scheduler (blocking)" }, { "submit_task", submit_task, METH_VARARGS, "call a remote function" }, { "notify_task_completed", notify_task_completed, METH_VARARGS, "notify the scheduler that a task has been completed" }, @@ -1010,13 +1010,13 @@ static PyMethodDef RayLibMethods[] = { PyMODINIT_FUNC initlibraylib(void) { PyObject* m; - PyObjRefType.tp_new = PyType_GenericNew; - if (PyType_Ready(&PyObjRefType) < 0) { + PyObjectIDType.tp_new = PyType_GenericNew; + if (PyType_Ready(&PyObjectIDType) < 0) { return; } m = Py_InitModule3("libraylib", RayLibMethods, "Python C Extension for Ray"); - Py_INCREF(&PyObjRefType); - PyModule_AddObject(m, "ObjRef", (PyObject *)&PyObjRefType); + Py_INCREF(&PyObjectIDType); + PyModule_AddObject(m, "ObjectID", (PyObject *)&PyObjectIDType); char ray_error[] = "ray.error"; char ray_size_error[] = "ray_size.error"; RayError = PyErr_NewException(ray_error, NULL, NULL); diff --git a/src/scheduler.cc b/src/scheduler.cc index 9071b5808..ac875c47a 100644 --- a/src/scheduler.cc +++ b/src/scheduler.cc @@ -128,17 +128,17 @@ Status SchedulerService::SubmitTask(ServerContext* context, const SubmitTaskRequ } } if (reply->function_registered()) { - std::vector result_objrefs; + std::vector result_objectids; for (size_t i = 0; i < num_return_vals; ++i) { - ObjRef result = register_new_object(); + ObjectID result = register_new_object(); reply->add_result(result); task->add_result(result); - result_objrefs.push_back(result); + result_objectids.push_back(result); } { auto reference_counts = GET(reference_counts_); - increment_ref_count(result_objrefs, reference_counts); // We increment once so the objrefs don't go out of scope before we reply to the worker that called SubmitTask. The corresponding decrement will happen in submit_task in raylib. - increment_ref_count(result_objrefs, reference_counts); // We increment once so the objrefs don't go out of scope before the task is scheduled on the worker. The corresponding decrement will happen in deserialize_task in raylib. + increment_ref_count(result_objectids, reference_counts); // We increment once so the objectids don't go out of scope before we reply to the worker that called SubmitTask. The corresponding decrement will happen in submit_task in raylib. + increment_ref_count(result_objectids, reference_counts); // We increment once so the objectids don't go out of scope before the task is scheduled on the worker. The corresponding decrement will happen in deserialize_task in raylib. } auto operation = std::unique_ptr(new Operation()); @@ -153,48 +153,48 @@ Status SchedulerService::SubmitTask(ServerContext* context, const SubmitTaskRequ } Status SchedulerService::PutObj(ServerContext* context, const PutObjRequest* request, PutObjReply* reply) { - ObjRef objref = register_new_object(); + ObjectID objectid = register_new_object(); auto operation = std::unique_ptr(new Operation()); - operation->mutable_put()->set_objref(objref); + operation->mutable_put()->set_objectid(objectid); operation->set_creator_operationid((*GET(workers_))[request->workerid()].current_task); GET(computation_graph_)->add_operation(std::move(operation)); - reply->set_objref(objref); + reply->set_objectid(objectid); schedule(); return Status::OK; } Status SchedulerService::RequestObj(ServerContext* context, const RequestObjRequest* request, AckReply* reply) { size_t size = GET(objtable_)->size(); - ObjRef objref = request->objref(); - RAY_CHECK_LT(objref, size, "internal error: no object with objref " << objref << " exists"); + ObjectID objectid = request->objectid(); + RAY_CHECK_LT(objectid, size, "internal error: no object with objectid " << objectid << " exists"); auto operation = std::unique_ptr(new Operation()); - operation->mutable_get()->set_objref(objref); + operation->mutable_get()->set_objectid(objectid); operation->set_creator_operationid((*GET(workers_))[request->workerid()].current_task); GET(computation_graph_)->add_operation(std::move(operation)); - GET(get_queue_)->push_back(std::make_pair(request->workerid(), objref)); + GET(get_queue_)->push_back(std::make_pair(request->workerid(), objectid)); schedule(); return Status::OK; } -Status SchedulerService::AliasObjRefs(ServerContext* context, const AliasObjRefsRequest* request, AckReply* reply) { - ObjRef alias_objref = request->alias_objref(); - ObjRef target_objref = request->target_objref(); - RAY_LOG(RAY_ALIAS, "Aliasing objref " << alias_objref << " with objref " << target_objref); - RAY_CHECK_NEQ(alias_objref, target_objref, "internal error: attempting to alias objref " << alias_objref << " with itself."); +Status SchedulerService::AliasObjectIDs(ServerContext* context, const AliasObjectIDsRequest* request, AckReply* reply) { + ObjectID alias_objectid = request->alias_objectid(); + ObjectID target_objectid = request->target_objectid(); + RAY_LOG(RAY_ALIAS, "Aliasing objectid " << alias_objectid << " with objectid " << target_objectid); + RAY_CHECK_NEQ(alias_objectid, target_objectid, "internal error: attempting to alias objectid " << alias_objectid << " with itself."); size_t size = GET(objtable_)->size(); - RAY_CHECK_LT(alias_objref, size, "internal error: no object with objref " << alias_objref << " exists"); - RAY_CHECK_LT(target_objref, size, "internal error: no object with objref " << target_objref << " exists"); + RAY_CHECK_LT(alias_objectid, size, "internal error: no object with objectid " << alias_objectid << " exists"); + RAY_CHECK_LT(target_objectid, size, "internal error: no object with objectid " << target_objectid << " exists"); { - auto target_objrefs = GET(target_objrefs_); - RAY_CHECK_EQ((*target_objrefs)[alias_objref], UNITIALIZED_ALIAS, "internal error: attempting to alias objref " << alias_objref << " with objref " << target_objref << ", but objref " << alias_objref << " has already been aliased with objref " << (*target_objrefs)[alias_objref]); - (*target_objrefs)[alias_objref] = target_objref; + auto target_objectids = GET(target_objectids_); + RAY_CHECK_EQ((*target_objectids)[alias_objectid], UNITIALIZED_ALIAS, "internal error: attempting to alias objectid " << alias_objectid << " with objectid " << target_objectid << ", but objectid " << alias_objectid << " has already been aliased with objectid " << (*target_objectids)[alias_objectid]); + (*target_objectids)[alias_objectid] = target_objectid; } - (*GET(reverse_target_objrefs_))[target_objref].push_back(alias_objref); + (*GET(reverse_target_objectids_))[target_objectid].push_back(alias_objectid); { // The corresponding increment was done in register_new_object. auto reference_counts = GET(reference_counts_); // we grab this lock because decrement_ref_count assumes it has been acquired - auto contained_objrefs = GET(contained_objrefs_); // we grab this lock because decrement_ref_count assumes it has been acquired - decrement_ref_count(std::vector({alias_objref}), reference_counts, contained_objrefs); + auto contained_objectids = GET(contained_objectids_); // we grab this lock because decrement_ref_count assumes it has been acquired + decrement_ref_count(std::vector({alias_objectid}), reference_counts, contained_objectids); } schedule(); return Status::OK; @@ -210,7 +210,7 @@ Status SchedulerService::RegisterObjStore(ServerContext* context, const Register (*objstores)[objstoreid].channel = channel; (*objstores)[objstoreid].objstore_stub = ObjStore::NewStub(channel); reply->set_objstoreid(objstoreid); - objects_in_transit_.push_back(std::vector()); + objects_in_transit_.push_back(std::vector()); return Status::OK; } @@ -233,18 +233,18 @@ Status SchedulerService::RegisterFunction(ServerContext* context, const Register } Status SchedulerService::ObjReady(ServerContext* context, const ObjReadyRequest* request, AckReply* reply) { - ObjRef objref = request->objref(); - RAY_LOG(RAY_DEBUG, "object " << objref << " ready on store " << request->objstoreid()); - add_canonical_objref(objref); - add_location(objref, request->objstoreid()); + ObjectID objectid = request->objectid(); + RAY_LOG(RAY_DEBUG, "object " << objectid << " ready on store " << request->objstoreid()); + add_canonical_objectid(objectid); + add_location(objectid, request->objstoreid()); { - // If this is the first time that ObjReady has been called for this objref, + // If this is the first time that ObjReady has been called for this objectid, // the corresponding increment was done in register_new_object in the // scheduler. For all subsequent calls to ObjReady, the corresponding // increment was done in deliver_object_if_necessary in the scheduler. auto reference_counts = GET(reference_counts_); // we grab this lock because decrement_ref_count assumes it has been acquired - auto contained_objrefs = GET(contained_objrefs_); // we grab this lock because decrement_ref_count assumes it has been acquired - decrement_ref_count(std::vector({objref}), reference_counts, contained_objrefs); + auto contained_objectids = GET(contained_objectids_); // we grab this lock because decrement_ref_count assumes it has been acquired + decrement_ref_count(std::vector({objectid}), reference_counts, contained_objectids); } schedule(); return Status::OK; @@ -296,40 +296,40 @@ Status SchedulerService::ReadyForNewTask(ServerContext* context, const ReadyForN } Status SchedulerService::IncrementRefCount(ServerContext* context, const IncrementRefCountRequest* request, AckReply* reply) { - int num_objrefs = request->objref_size(); - RAY_CHECK_NEQ(num_objrefs, 0, "Scheduler received IncrementRefCountRequest with 0 objrefs."); - std::vector objrefs; - for (int i = 0; i < num_objrefs; ++i) { - objrefs.push_back(request->objref(i)); + int num_objectids = request->objectid_size(); + RAY_CHECK_NEQ(num_objectids, 0, "Scheduler received IncrementRefCountRequest with 0 objectids."); + std::vector objectids; + for (int i = 0; i < num_objectids; ++i) { + objectids.push_back(request->objectid(i)); } auto reference_counts = GET(reference_counts_); - increment_ref_count(objrefs, reference_counts); + increment_ref_count(objectids, reference_counts); return Status::OK; } Status SchedulerService::DecrementRefCount(ServerContext* context, const DecrementRefCountRequest* request, AckReply* reply) { - int num_objrefs = request->objref_size(); - RAY_CHECK_NEQ(num_objrefs, 0, "Scheduler received DecrementRefCountRequest with 0 objrefs."); - std::vector objrefs; - for (int i = 0; i < num_objrefs; ++i) { - objrefs.push_back(request->objref(i)); + int num_objectids = request->objectid_size(); + RAY_CHECK_NEQ(num_objectids, 0, "Scheduler received DecrementRefCountRequest with 0 objectids."); + std::vector objectids; + for (int i = 0; i < num_objectids; ++i) { + objectids.push_back(request->objectid(i)); } auto reference_counts = GET(reference_counts_); // we grab this lock, because decrement_ref_count assumes it has been acquired - auto contained_objrefs = GET(contained_objrefs_); // we grab this lock because decrement_ref_count assumes it has been acquired - decrement_ref_count(objrefs, reference_counts, contained_objrefs); + auto contained_objectids = GET(contained_objectids_); // we grab this lock because decrement_ref_count assumes it has been acquired + decrement_ref_count(objectids, reference_counts, contained_objectids); return Status::OK; } -Status SchedulerService::AddContainedObjRefs(ServerContext* context, const AddContainedObjRefsRequest* request, AckReply* reply) { - ObjRef objref = request->objref(); - // if (!is_canonical(objref)) { +Status SchedulerService::AddContainedObjectIDs(ServerContext* context, const AddContainedObjectIDsRequest* request, AckReply* reply) { + ObjectID objectid = request->objectid(); + // if (!is_canonical(objectid)) { // TODO(rkn): Perhaps we don't need this check. It won't work because the objstore may not have called ObjReady yet. - // RAY_LOG(RAY_FATAL, "Attempting to add contained objrefs for non-canonical objref " << objref); + // RAY_LOG(RAY_FATAL, "Attempting to add contained objectids for non-canonical objectid " << objectid); // } - auto contained_objrefs = GET(contained_objrefs_); - RAY_CHECK_EQ((*contained_objrefs)[objref].size(), 0, "Attempting to add contained objrefs for objref " << objref << ", but contained_objrefs_[objref].size() != 0."); - for (int i = 0; i < request->contained_objref_size(); ++i) { - (*contained_objrefs)[objref].push_back(request->contained_objref(i)); + auto contained_objectids = GET(contained_objectids_); + RAY_CHECK_EQ((*contained_objectids)[objectid].size(), 0, "Attempting to add contained objectids for objectid " << objectid << ", but contained_objectids_[objectid].size() != 0."); + for (int i = 0; i < request->contained_objectid_size(); ++i) { + (*contained_objectids)[objectid].push_back(request->contained_objectid(i)); } return Status::OK; } @@ -436,21 +436,21 @@ Status SchedulerService::ExportReusableVariable(ServerContext* context, const Ex return Status::OK; } -void SchedulerService::deliver_object_async_if_necessary(ObjRef canonical_objref, ObjStoreId from, ObjStoreId to) { +void SchedulerService::deliver_object_async_if_necessary(ObjectID canonical_objectid, ObjStoreId from, ObjStoreId to) { bool object_present_or_in_transit; { auto objtable = GET(objtable_); - auto &locations = (*objtable)[canonical_objref]; + auto &locations = (*objtable)[canonical_objectid]; bool object_present = std::binary_search(locations.begin(), locations.end(), to); auto &objects_in_flight = objects_in_transit_[to]; - bool object_in_transit = (std::find(objects_in_flight.begin(), objects_in_flight.end(), canonical_objref) != objects_in_flight.end()); + bool object_in_transit = (std::find(objects_in_flight.begin(), objects_in_flight.end(), canonical_objectid) != objects_in_flight.end()); object_present_or_in_transit = object_present || object_in_transit; if (!object_present_or_in_transit) { - objects_in_flight.push_back(canonical_objref); + objects_in_flight.push_back(canonical_objectid); } } if (!object_present_or_in_transit) { - deliver_object_async(canonical_objref, from, to); + deliver_object_async(canonical_objectid, from, to); } } @@ -460,21 +460,21 @@ void SchedulerService::deliver_object_async_if_necessary(ObjRef canonical_objref // delivery once. However, we may want to handle it in the scheduler in the // future. // -// deliver_object_async assumes that the aliasing for objref has already been completed. That is, has_canonical_objref(objref) == true -void SchedulerService::deliver_object_async(ObjRef canonical_objref, ObjStoreId from, ObjStoreId to) { - RAY_CHECK_NEQ(from, to, "attempting to deliver canonical_objref " << canonical_objref << " from objstore " << from << " to itself."); - RAY_CHECK(is_canonical(canonical_objref), "attempting to deliver objref " << canonical_objref << ", but this objref is not a canonical objref."); +// deliver_object_async assumes that the aliasing for objectid has already been completed. That is, has_canonical_objectid(objectid) == true +void SchedulerService::deliver_object_async(ObjectID canonical_objectid, ObjStoreId from, ObjStoreId to) { + RAY_CHECK_NEQ(from, to, "attempting to deliver canonical_objectid " << canonical_objectid << " from objstore " << from << " to itself."); + RAY_CHECK(is_canonical(canonical_objectid), "attempting to deliver objectid " << canonical_objectid << ", but this objectid is not a canonical objectid."); { - // We increment once so the objref doesn't go out of scope before the ObjReady + // We increment once so the objectid doesn't go out of scope before the ObjReady // method is called. The corresponding decrement will happen in ObjReady in // the scheduler. auto reference_counts = GET(reference_counts_); // we grab this lock because increment_ref_count assumes it has been acquired - increment_ref_count(std::vector({canonical_objref}), reference_counts); + increment_ref_count(std::vector({canonical_objectid}), reference_counts); } ClientContext context; AckReply reply; StartDeliveryRequest request; - request.set_objref(canonical_objref); + request.set_objectid(canonical_objectid); auto objstores = GET(objstores_); request.set_objstore_address((*objstores)[from].address); (*objstores)[to].objstore_stub->StartDelivery(&context, request, &reply); @@ -493,7 +493,7 @@ void SchedulerService::schedule() { perform_notify_aliases(); // See what we can do in alias_notification_queue_ } -// assign_task assumes that the canonical objrefs for its arguments are all ready, that is has_canonical_objref() is true for all of the call's arguments +// assign_task assumes that the canonical objectids for its arguments are all ready, that is has_canonical_objectid() is true for all of the call's arguments void SchedulerService::assign_task(OperationId operationid, WorkerId workerid, const MySynchronizedPtr &computation_graph) { // assign_task takes computation_graph as an argument, which is obtained by // GET(computation_graph_), so we know that the data structure has been @@ -506,13 +506,13 @@ void SchedulerService::assign_task(OperationId operationid, WorkerId workerid, c RAY_LOG(RAY_INFO, "starting to send arguments"); for (size_t i = 0; i < task.arg_size(); ++i) { if (!task.arg(i).has_obj()) { - ObjRef objref = task.arg(i).ref(); - ObjRef canonical_objref = get_canonical_objref(objref); + ObjectID objectid = task.arg(i).id(); + ObjectID canonical_objectid = get_canonical_objectid(objectid); // Notify the relevant objstore about potential aliasing when it's ready - GET(alias_notification_queue_)->push_back(std::make_pair(objstoreid, std::make_pair(objref, canonical_objref))); - attempt_notify_alias(objstoreid, objref, canonical_objref); - RAY_LOG(RAY_DEBUG, "task contains object ref " << canonical_objref); - deliver_object_async_if_necessary(canonical_objref, pick_objstore(canonical_objref), objstoreid); + GET(alias_notification_queue_)->push_back(std::make_pair(objstoreid, std::make_pair(objectid, canonical_objectid))); + attempt_notify_alias(objstoreid, objectid, canonical_objectid); + RAY_LOG(RAY_DEBUG, "task contains object ref " << canonical_objectid); + deliver_object_async_if_necessary(canonical_objectid, pick_objstore(canonical_objectid), objstoreid); } } { @@ -527,12 +527,12 @@ bool SchedulerService::can_run(const Task& task) { auto objtable = GET(objtable_); for (int i = 0; i < task.arg_size(); ++i) { if (!task.arg(i).has_obj()) { - ObjRef objref = task.arg(i).ref(); - if (!has_canonical_objref(objref)) { + ObjectID objectid = task.arg(i).id(); + if (!has_canonical_objectid(objectid)) { return false; } - ObjRef canonical_objref = get_canonical_objref(objref); - if (canonical_objref >= objtable->size() || (*objtable)[canonical_objref].size() == 0) { + ObjectID canonical_objectid = get_canonical_objectid(objectid); + if (canonical_objectid >= objtable->size() || (*objtable)[canonical_objectid].size() == 0) { return false; } } @@ -576,58 +576,58 @@ std::pair SchedulerService::register_worker(const std::str return std::make_pair(workerid, objstoreid); } -ObjRef SchedulerService::register_new_object() { - // If we don't simultaneously lock objtable_ and target_objrefs_, we will probably get errors. - // TODO(rkn): increment/decrement_reference_count also acquire reference_counts_lock_ and target_objrefs_lock_ (through has_canonical_objref()), which caused deadlock in the past +ObjectID SchedulerService::register_new_object() { + // If we don't simultaneously lock objtable_ and target_objectids_, we will probably get errors. + // TODO(rkn): increment/decrement_reference_count also acquire reference_counts_lock_ and target_objectids_lock_ (through has_canonical_objectid()), which caused deadlock in the past auto reference_counts = GET(reference_counts_); - auto contained_objrefs = GET(contained_objrefs_); + auto contained_objectids = GET(contained_objectids_); auto objtable = GET(objtable_); - auto target_objrefs = GET(target_objrefs_); - auto reverse_target_objrefs = GET(reverse_target_objrefs_); - ObjRef objtable_size = objtable->size(); - ObjRef target_objrefs_size = target_objrefs->size(); - ObjRef reverse_target_objrefs_size = reverse_target_objrefs->size(); - ObjRef reference_counts_size = reference_counts->size(); - ObjRef contained_objrefs_size = contained_objrefs->size(); - RAY_CHECK_EQ(objtable_size, target_objrefs_size, "objtable_ and target_objrefs_ should have the same size, but objtable_.size() = " << objtable_size << " and target_objrefs_.size() = " << target_objrefs_size); - RAY_CHECK_EQ(objtable_size, reverse_target_objrefs_size, "objtable_ and reverse_target_objrefs_ should have the same size, but objtable_.size() = " << objtable_size << " and reverse_target_objrefs_.size() = " << reverse_target_objrefs_size); + auto target_objectids = GET(target_objectids_); + auto reverse_target_objectids = GET(reverse_target_objectids_); + ObjectID objtable_size = objtable->size(); + ObjectID target_objectids_size = target_objectids->size(); + ObjectID reverse_target_objectids_size = reverse_target_objectids->size(); + ObjectID reference_counts_size = reference_counts->size(); + ObjectID contained_objectids_size = contained_objectids->size(); + RAY_CHECK_EQ(objtable_size, target_objectids_size, "objtable_ and target_objectids_ should have the same size, but objtable_.size() = " << objtable_size << " and target_objectids_.size() = " << target_objectids_size); + RAY_CHECK_EQ(objtable_size, reverse_target_objectids_size, "objtable_ and reverse_target_objectids_ should have the same size, but objtable_.size() = " << objtable_size << " and reverse_target_objectids_.size() = " << reverse_target_objectids_size); RAY_CHECK_EQ(objtable_size, reference_counts_size, "objtable_ and reference_counts_ should have the same size, but objtable_.size() = " << objtable_size << " and reference_counts_.size() = " << reference_counts_size); - RAY_CHECK_EQ(objtable_size, contained_objrefs_size, "objtable_ and contained_objrefs_ should have the same size, but objtable_.size() = " << objtable_size << " and contained_objrefs_.size() = " << contained_objrefs_size); + RAY_CHECK_EQ(objtable_size, contained_objectids_size, "objtable_ and contained_objectids_ should have the same size, but objtable_.size() = " << objtable_size << " and contained_objectids_.size() = " << contained_objectids_size); objtable->push_back(std::vector()); - target_objrefs->push_back(UNITIALIZED_ALIAS); - reverse_target_objrefs->push_back(std::vector()); + target_objectids->push_back(UNITIALIZED_ALIAS); + reverse_target_objectids->push_back(std::vector()); reference_counts->push_back(0); - contained_objrefs->push_back(std::vector()); + contained_objectids->push_back(std::vector()); { - // We increment once so the objref doesn't go out of scope before the ObjReady + // We increment once so the objectid doesn't go out of scope before the ObjReady // method is called. The corresponding decrement will happen either in - // ObjReady in the scheduler or in AliasObjRefs in the scheduler. - increment_ref_count(std::vector({objtable_size}), reference_counts); // Note that reference_counts_lock_ is acquired above, as assumed by increment_ref_count + // ObjReady in the scheduler or in AliasObjectIDs in the scheduler. + increment_ref_count(std::vector({objtable_size}), reference_counts); // Note that reference_counts_lock_ is acquired above, as assumed by increment_ref_count } return objtable_size; } -void SchedulerService::add_location(ObjRef canonical_objref, ObjStoreId objstoreid) { - // add_location must be called with a canonical objref - RAY_CHECK_NEQ((*GET(reference_counts_))[canonical_objref], DEALLOCATED, "Calling ObjReady with canonical_objref " << canonical_objref << ", but this objref has already been deallocated"); - RAY_CHECK(is_canonical(canonical_objref), "Attempting to call add_location with a non-canonical objref (objref " << canonical_objref << ")"); +void SchedulerService::add_location(ObjectID canonical_objectid, ObjStoreId objstoreid) { + // add_location must be called with a canonical objectid + RAY_CHECK_NEQ((*GET(reference_counts_))[canonical_objectid], DEALLOCATED, "Calling ObjReady with canonical_objectid " << canonical_objectid << ", but this objectid has already been deallocated"); + RAY_CHECK(is_canonical(canonical_objectid), "Attempting to call add_location with a non-canonical objectid (objectid " << canonical_objectid << ")"); auto objtable = GET(objtable_); - RAY_CHECK_LT(canonical_objref, objtable->size(), "trying to put an object in the object store that was not registered with the scheduler (objref " << canonical_objref << ")"); + RAY_CHECK_LT(canonical_objectid, objtable->size(), "trying to put an object in the object store that was not registered with the scheduler (objectid " << canonical_objectid << ")"); // do a binary search - auto &locations = (*objtable)[canonical_objref]; + auto &locations = (*objtable)[canonical_objectid]; auto pos = std::lower_bound(locations.begin(), locations.end(), objstoreid); if (pos == locations.end() || objstoreid < *pos) { locations.insert(pos, objstoreid); } auto &objects_in_flight = objects_in_transit_[objstoreid]; - objects_in_flight.erase(std::remove(objects_in_flight.begin(), objects_in_flight.end(), canonical_objref), objects_in_flight.end()); + objects_in_flight.erase(std::remove(objects_in_flight.begin(), objects_in_flight.end(), canonical_objectid), objects_in_flight.end()); } -void SchedulerService::add_canonical_objref(ObjRef objref) { - auto target_objrefs = GET(target_objrefs_); - RAY_CHECK_LT(objref, target_objrefs->size(), "internal error: attempting to insert objref " << objref << " in target_objrefs_, but target_objrefs_.size() is " << target_objrefs->size()); - RAY_CHECK((*target_objrefs)[objref] == UNITIALIZED_ALIAS || (*target_objrefs)[objref] == objref, "internal error: attempting to declare objref " << objref << " as a canonical objref, but target_objrefs_[objref] is already aliased with objref " << (*target_objrefs)[objref]); - (*target_objrefs)[objref] = objref; +void SchedulerService::add_canonical_objectid(ObjectID objectid) { + auto target_objectids = GET(target_objectids_); + RAY_CHECK_LT(objectid, target_objectids->size(), "internal error: attempting to insert objectid " << objectid << " in target_objectids_, but target_objectids_.size() is " << target_objectids->size()); + RAY_CHECK((*target_objectids)[objectid] == UNITIALIZED_ALIAS || (*target_objectids)[objectid] == objectid, "internal error: attempting to declare objectid " << objectid << " as a canonical objectid, but target_objectids_[objectid] is already aliased with objectid " << (*target_objectids)[objectid]); + (*target_objectids)[objectid] = objectid; } ObjStoreId SchedulerService::get_store(WorkerId workerid) { @@ -649,13 +649,13 @@ void SchedulerService::get_info(const SchedulerInfoRequest& request, SchedulerIn auto avail_workers = GET(avail_workers_); auto task_queue = GET(task_queue_); auto reference_counts = GET(reference_counts_); - auto target_objrefs = GET(target_objrefs_); + auto target_objectids = GET(target_objectids_); auto function_table = reply->mutable_function_table(); for (int i = 0; i < reference_counts->size(); ++i) { reply->add_reference_count((*reference_counts)[i]); } - for (int i = 0; i < target_objrefs->size(); ++i) { - reply->add_target_objref((*target_objrefs)[i]); + for (int i = 0; i < target_objectids->size(); ++i) { + reply->add_target_objectid((*target_objectids)[i]); } for (const auto& entry : *fntable) { (*function_table)[entry.first].set_num_return_vals(entry.second.num_return_vals()); @@ -672,41 +672,41 @@ void SchedulerService::get_info(const SchedulerInfoRequest& request, SchedulerIn computation_graph->to_protobuf(reply->mutable_computation_graph()); } -// pick_objstore must be called with a canonical_objref -ObjStoreId SchedulerService::pick_objstore(ObjRef canonical_objref) { +// pick_objstore must be called with a canonical_objectid +ObjStoreId SchedulerService::pick_objstore(ObjectID canonical_objectid) { std::mt19937 rng; - RAY_CHECK(is_canonical(canonical_objref), "Attempting to call pick_objstore with a non-canonical objref, (objref " << canonical_objref << ")"); + RAY_CHECK(is_canonical(canonical_objectid), "Attempting to call pick_objstore with a non-canonical objectid, (objectid " << canonical_objectid << ")"); auto objtable = GET(objtable_); - std::uniform_int_distribution uni(0, (*objtable)[canonical_objref].size() - 1); - ObjStoreId objstoreid = (*objtable)[canonical_objref][uni(rng)]; + std::uniform_int_distribution uni(0, (*objtable)[canonical_objectid].size() - 1); + ObjStoreId objstoreid = (*objtable)[canonical_objectid][uni(rng)]; return objstoreid; } -bool SchedulerService::is_canonical(ObjRef objref) { - auto target_objrefs = GET(target_objrefs_); - RAY_CHECK_NEQ((*target_objrefs)[objref], UNITIALIZED_ALIAS, "Attempting to call is_canonical on an objref for which aliasing is not complete or the object is not ready, target_objrefs_[objref] == UNITIALIZED_ALIAS for objref " << objref << "."); - return objref == (*target_objrefs)[objref]; +bool SchedulerService::is_canonical(ObjectID objectid) { + auto target_objectids = GET(target_objectids_); + RAY_CHECK_NEQ((*target_objectids)[objectid], UNITIALIZED_ALIAS, "Attempting to call is_canonical on an objectid for which aliasing is not complete or the object is not ready, target_objectids_[objectid] == UNITIALIZED_ALIAS for objectid " << objectid << "."); + return objectid == (*target_objectids)[objectid]; } void SchedulerService::perform_gets() { auto get_queue = GET(get_queue_); // Complete all get tasks that can be completed. for (int i = 0; i < get_queue->size(); ++i) { - const std::pair& get_request = (*get_queue)[i]; - ObjRef objref = get_request.second; + const std::pair& get_request = (*get_queue)[i]; + ObjectID objectid = get_request.second; WorkerId workerid = get_request.first; ObjStoreId objstoreid = get_store(workerid); - if (!has_canonical_objref(objref)) { - RAY_LOG(RAY_ALIAS, "objref " << objref << " does not have a canonical_objref, so continuing"); + if (!has_canonical_objectid(objectid)) { + RAY_LOG(RAY_ALIAS, "objectid " << objectid << " does not have a canonical_objectid, so continuing"); continue; } - ObjRef canonical_objref = get_canonical_objref(objref); - RAY_LOG(RAY_DEBUG, "attempting to get objref " << get_request.second << " with canonical objref " << canonical_objref << " to objstore " << objstoreid); - int num_stores = (*GET(objtable_))[canonical_objref].size(); + ObjectID canonical_objectid = get_canonical_objectid(objectid); + RAY_LOG(RAY_DEBUG, "attempting to get objectid " << get_request.second << " with canonical objectid " << canonical_objectid << " to objstore " << objstoreid); + int num_stores = (*GET(objtable_))[canonical_objectid].size(); if (num_stores > 0) { - deliver_object_async_if_necessary(canonical_objref, pick_objstore(canonical_objref), objstoreid); + deliver_object_async_if_necessary(canonical_objectid, pick_objstore(canonical_objectid), objstoreid); // Notify the relevant objstore about potential aliasing when it's ready - GET(alias_notification_queue_)->push_back(std::make_pair(objstoreid, std::make_pair(objref, canonical_objref))); + GET(alias_notification_queue_)->push_back(std::make_pair(objstoreid, std::make_pair(objectid, canonical_objectid))); // Remove the get task from the queue std::swap((*get_queue)[i], (*get_queue)[get_queue->size() - 1]); get_queue->pop_back(); @@ -762,13 +762,13 @@ void SchedulerService::schedule_tasks_location_aware() { size_t num_shipped_objects = 0; for (int j = 0; j < task.arg_size(); ++j) { if (!task.arg(j).has_obj()) { - ObjRef objref = task.arg(j).ref(); - RAY_CHECK(has_canonical_objref(objref), "no canonical object ref found even though task is ready; that should not be possible!"); - ObjRef canonical_objref = get_canonical_objref(objref); + ObjectID objectid = task.arg(j).id(); + RAY_CHECK(has_canonical_objectid(objectid), "no canonical object ref found even though task is ready; that should not be possible!"); + ObjectID canonical_objectid = get_canonical_objectid(objectid); { // check if the object is already in the local object store auto objtable = GET(objtable_); - if (!std::binary_search((*objtable)[canonical_objref].begin(), (*objtable)[canonical_objref].end(), objstoreid)) { + if (!std::binary_search((*objtable)[canonical_objectid].begin(), (*objtable)[canonical_objectid].end(), objstoreid)) { num_shipped_objects += 1; } } @@ -794,12 +794,12 @@ void SchedulerService::schedule_tasks_location_aware() { void SchedulerService::perform_notify_aliases() { auto alias_notification_queue = GET(alias_notification_queue_); for (int i = 0; i < alias_notification_queue->size(); ++i) { - const std::pair > alias_notification = (*alias_notification_queue)[i]; + const std::pair > alias_notification = (*alias_notification_queue)[i]; ObjStoreId objstoreid = alias_notification.first; - ObjRef alias_objref = alias_notification.second.first; - ObjRef canonical_objref = alias_notification.second.second; - if (attempt_notify_alias(objstoreid, alias_objref, canonical_objref)) { // this locks both the objstore_ and objtable_ - // the attempt to notify the objstore of the objref aliasing succeeded, so remove the notification task from the queue + ObjectID alias_objectid = alias_notification.second.first; + ObjectID canonical_objectid = alias_notification.second.second; + if (attempt_notify_alias(objstoreid, alias_objectid, canonical_objectid)) { // this locks both the objstore_ and objtable_ + // the attempt to notify the objstore of the objectid aliasing succeeded, so remove the notification task from the queue std::swap((*alias_notification_queue)[i], (*alias_notification_queue)[alias_notification_queue->size() - 1]); alias_notification_queue->pop_back(); i -= 1; @@ -807,145 +807,145 @@ void SchedulerService::perform_notify_aliases() { } } -bool SchedulerService::has_canonical_objref(ObjRef objref) { - auto target_objrefs = GET(target_objrefs_); - ObjRef objref_temp = objref; +bool SchedulerService::has_canonical_objectid(ObjectID objectid) { + auto target_objectids = GET(target_objectids_); + ObjectID objectid_temp = objectid; while (true) { - RAY_CHECK_LT(objref_temp, target_objrefs->size(), "Attempting to index target_objrefs_ with objref " << objref_temp << ", but target_objrefs_.size() = " << target_objrefs->size()); - if ((*target_objrefs)[objref_temp] == UNITIALIZED_ALIAS) { + RAY_CHECK_LT(objectid_temp, target_objectids->size(), "Attempting to index target_objectids_ with objectid " << objectid_temp << ", but target_objectids_.size() = " << target_objectids->size()); + if ((*target_objectids)[objectid_temp] == UNITIALIZED_ALIAS) { return false; } - if ((*target_objrefs)[objref_temp] == objref_temp) { + if ((*target_objectids)[objectid_temp] == objectid_temp) { return true; } - objref_temp = (*target_objrefs)[objref_temp]; + objectid_temp = (*target_objectids)[objectid_temp]; } } -ObjRef SchedulerService::get_canonical_objref(ObjRef objref) { - // get_canonical_objref assumes that has_canonical_objref(objref) is true - auto target_objrefs = GET(target_objrefs_); - ObjRef objref_temp = objref; +ObjectID SchedulerService::get_canonical_objectid(ObjectID objectid) { + // get_canonical_objectid assumes that has_canonical_objectid(objectid) is true + auto target_objectids = GET(target_objectids_); + ObjectID objectid_temp = objectid; while (true) { - RAY_CHECK_LT(objref_temp, target_objrefs->size(), "Attempting to index target_objrefs_ with objref " << objref_temp << ", but target_objrefs_.size() = " << target_objrefs->size()); - RAY_CHECK_NEQ((*target_objrefs)[objref_temp], UNITIALIZED_ALIAS, "Attempting to get canonical objref for objref " << objref << ", which aliases, objref " << objref_temp << ", but target_objrefs_[objref_temp] == UNITIALIZED_ALIAS for objref_temp = " << objref_temp << "."); - if ((*target_objrefs)[objref_temp] == objref_temp) { - return objref_temp; + RAY_CHECK_LT(objectid_temp, target_objectids->size(), "Attempting to index target_objectids_ with objectid " << objectid_temp << ", but target_objectids_.size() = " << target_objectids->size()); + RAY_CHECK_NEQ((*target_objectids)[objectid_temp], UNITIALIZED_ALIAS, "Attempting to get canonical objectid for objectid " << objectid << ", which aliases, objectid " << objectid_temp << ", but target_objectids_[objectid_temp] == UNITIALIZED_ALIAS for objectid_temp = " << objectid_temp << "."); + if ((*target_objectids)[objectid_temp] == objectid_temp) { + return objectid_temp; } - objref_temp = (*target_objrefs)[objref_temp]; - RAY_LOG(RAY_ALIAS, "Looping in get_canonical_objref."); + objectid_temp = (*target_objectids)[objectid_temp]; + RAY_LOG(RAY_ALIAS, "Looping in get_canonical_objectid."); } } -bool SchedulerService::attempt_notify_alias(ObjStoreId objstoreid, ObjRef alias_objref, ObjRef canonical_objref) { +bool SchedulerService::attempt_notify_alias(ObjStoreId objstoreid, ObjectID alias_objectid, ObjectID canonical_objectid) { // return true if successful and false otherwise - if (alias_objref == canonical_objref) { + if (alias_objectid == canonical_objectid) { // no need to do anything return true; } { auto objtable = GET(objtable_); - if (!std::binary_search((*objtable)[canonical_objref].begin(), (*objtable)[canonical_objref].end(), objstoreid)) { - // the objstore doesn't have the object for canonical_objref yet, so it's too early to notify the objstore about the alias + if (!std::binary_search((*objtable)[canonical_objectid].begin(), (*objtable)[canonical_objectid].end(), objstoreid)) { + // the objstore doesn't have the object for canonical_objectid yet, so it's too early to notify the objstore about the alias return false; } } ClientContext context; AckReply reply; NotifyAliasRequest request; - request.set_alias_objref(alias_objref); - request.set_canonical_objref(canonical_objref); + request.set_alias_objectid(alias_objectid); + request.set_canonical_objectid(canonical_objectid); (*GET(objstores_))[objstoreid].objstore_stub->NotifyAlias(&context, request, &reply); return true; } -void SchedulerService::deallocate_object(ObjRef canonical_objref, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objrefs) { +void SchedulerService::deallocate_object(ObjectID canonical_objectid, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objectids) { // deallocate_object should only be called from decrement_ref_count (note that // deallocate_object also recursively calls decrement_ref_count). Both of - // these methods take reference_counts and contained_objrefs as argumens, - // which are obtained by GET(reference_counts) and GET(contained_objrefs_), + // these methods take reference_counts and contained_objectids as argumens, + // which are obtained by GET(reference_counts) and GET(contained_objectids_), // so we know that those data structures have been locked - RAY_LOG(RAY_REFCOUNT, "Deallocating canonical_objref " << canonical_objref << "."); + RAY_LOG(RAY_REFCOUNT, "Deallocating canonical_objectid " << canonical_objectid << "."); { auto objtable = GET(objtable_); - auto &locations = (*objtable)[canonical_objref]; + auto &locations = (*objtable)[canonical_objectid]; auto objstores = GET(objstores_); // TODO(rkn): Should this be inside the for loop instead? for (int i = 0; i < locations.size(); ++i) { ClientContext context; AckReply reply; DeallocateObjectRequest request; - request.set_canonical_objref(canonical_objref); + request.set_canonical_objectid(canonical_objectid); ObjStoreId objstoreid = locations[i]; - RAY_LOG(RAY_REFCOUNT, "Attempting to deallocate canonical_objref " << canonical_objref << " from objstore " << objstoreid); + RAY_LOG(RAY_REFCOUNT, "Attempting to deallocate canonical_objectid " << canonical_objectid << " from objstore " << objstoreid); (*objstores)[objstoreid].objstore_stub->DeallocateObject(&context, request, &reply); } locations.clear(); } - decrement_ref_count((*contained_objrefs)[canonical_objref], reference_counts, contained_objrefs); + decrement_ref_count((*contained_objectids)[canonical_objectid], reference_counts, contained_objectids); } -void SchedulerService::increment_ref_count(const std::vector &objrefs, const MySynchronizedPtr > &reference_counts) { +void SchedulerService::increment_ref_count(const std::vector &objectids, const MySynchronizedPtr > &reference_counts) { // increment_ref_count takes reference_counts as an argument, which is // obtained by GET(reference_counts_), so we know that the data structure has // been locked - for (int i = 0; i < objrefs.size(); ++i) { - ObjRef objref = objrefs[i]; - RAY_CHECK_NEQ((*reference_counts)[objref], DEALLOCATED, "Attempting to increment the reference count for objref " << objref << ", but this object appears to have been deallocated already."); - (*reference_counts)[objref] += 1; - RAY_LOG(RAY_REFCOUNT, "Incremented ref count for objref " << objref <<". New reference count is " << (*reference_counts)[objref]); + for (int i = 0; i < objectids.size(); ++i) { + ObjectID objectid = objectids[i]; + RAY_CHECK_NEQ((*reference_counts)[objectid], DEALLOCATED, "Attempting to increment the reference count for objectid " << objectid << ", but this object appears to have been deallocated already."); + (*reference_counts)[objectid] += 1; + RAY_LOG(RAY_REFCOUNT, "Incremented ref count for objectid " << objectid <<". New reference count is " << (*reference_counts)[objectid]); } } -void SchedulerService::decrement_ref_count(const std::vector &objrefs, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objrefs) { - // decrement_ref_count takes reference_counts and contained_objrefs as +void SchedulerService::decrement_ref_count(const std::vector &objectids, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objectids) { + // decrement_ref_count takes reference_counts and contained_objectids as // arguments, which are obtained by GET(reference_counts_) and - // GET(contained_objrefs_), so we know that those data structures have been + // GET(contained_objectids_), so we know that those data structures have been // locked - for (int i = 0; i < objrefs.size(); ++i) { - ObjRef objref = objrefs[i]; - RAY_CHECK_NEQ((*reference_counts)[objref], DEALLOCATED, "Attempting to decrement the reference count for objref " << objref << ", but this object appears to have been deallocated already."); - RAY_CHECK_NEQ((*reference_counts)[objref], 0, "Attempting to decrement the reference count for objref " << objref << ", but the reference count for this object is already 0."); - (*reference_counts)[objref] -= 1; - RAY_LOG(RAY_REFCOUNT, "Decremented ref count for objref " << objref << ". New reference count is " << (*reference_counts)[objref]); + for (int i = 0; i < objectids.size(); ++i) { + ObjectID objectid = objectids[i]; + RAY_CHECK_NEQ((*reference_counts)[objectid], DEALLOCATED, "Attempting to decrement the reference count for objectid " << objectid << ", but this object appears to have been deallocated already."); + RAY_CHECK_NEQ((*reference_counts)[objectid], 0, "Attempting to decrement the reference count for objectid " << objectid << ", but the reference count for this object is already 0."); + (*reference_counts)[objectid] -= 1; + RAY_LOG(RAY_REFCOUNT, "Decremented ref count for objectid " << objectid << ". New reference count is " << (*reference_counts)[objectid]); // See if we can deallocate the object - std::vector equivalent_objrefs; - get_equivalent_objrefs(objref, equivalent_objrefs); + std::vector equivalent_objectids; + get_equivalent_objectids(objectid, equivalent_objectids); bool can_deallocate = true; - for (int j = 0; j < equivalent_objrefs.size(); ++j) { - if ((*reference_counts)[equivalent_objrefs[j]] != 0) { + for (int j = 0; j < equivalent_objectids.size(); ++j) { + if ((*reference_counts)[equivalent_objectids[j]] != 0) { can_deallocate = false; break; } } if (can_deallocate) { - ObjRef canonical_objref = equivalent_objrefs[0]; - RAY_CHECK(is_canonical(canonical_objref), "canonical_objref is not canonical."); - deallocate_object(canonical_objref, reference_counts, contained_objrefs); - for (int j = 0; j < equivalent_objrefs.size(); ++j) { - (*reference_counts)[equivalent_objrefs[j]] = DEALLOCATED; + ObjectID canonical_objectid = equivalent_objectids[0]; + RAY_CHECK(is_canonical(canonical_objectid), "canonical_objectid is not canonical."); + deallocate_object(canonical_objectid, reference_counts, contained_objectids); + for (int j = 0; j < equivalent_objectids.size(); ++j) { + (*reference_counts)[equivalent_objectids[j]] = DEALLOCATED; } } } } -void SchedulerService::upstream_objrefs(ObjRef objref, std::vector &objrefs, const MySynchronizedPtr > > &reverse_target_objrefs) { - // upstream_objrefs takes reverse_target_objrefs as an argument, which is - // obtained by GET(reverse_target_objrefs_), so we know the data structure +void SchedulerService::upstream_objectids(ObjectID objectid, std::vector &objectids, const MySynchronizedPtr > > &reverse_target_objectids) { + // upstream_objectids takes reverse_target_objectids as an argument, which is + // obtained by GET(reverse_target_objectids_), so we know the data structure // has been locked. - objrefs.push_back(objref); - for (int i = 0; i < (*reverse_target_objrefs)[objref].size(); ++i) { - upstream_objrefs((*reverse_target_objrefs)[objref][i], objrefs, reverse_target_objrefs); + objectids.push_back(objectid); + for (int i = 0; i < (*reverse_target_objectids)[objectid].size(); ++i) { + upstream_objectids((*reverse_target_objectids)[objectid][i], objectids, reverse_target_objectids); } } -void SchedulerService::get_equivalent_objrefs(ObjRef objref, std::vector &equivalent_objrefs) { - auto target_objrefs = GET(target_objrefs_); - ObjRef downstream_objref = objref; - while ((*target_objrefs)[downstream_objref] != downstream_objref && (*target_objrefs)[downstream_objref] != UNITIALIZED_ALIAS) { - RAY_LOG(RAY_ALIAS, "Looping in get_equivalent_objrefs"); - downstream_objref = (*target_objrefs)[downstream_objref]; +void SchedulerService::get_equivalent_objectids(ObjectID objectid, std::vector &equivalent_objectids) { + auto target_objectids = GET(target_objectids_); + ObjectID downstream_objectid = objectid; + while ((*target_objectids)[downstream_objectid] != downstream_objectid && (*target_objectids)[downstream_objectid] != UNITIALIZED_ALIAS) { + RAY_LOG(RAY_ALIAS, "Looping in get_equivalent_objectids"); + downstream_objectid = (*target_objectids)[downstream_objectid]; } - upstream_objrefs(downstream_objref, equivalent_objrefs, GET(reverse_target_objrefs_)); + upstream_objectids(downstream_objectid, equivalent_objectids, GET(reverse_target_objectids_)); } diff --git a/src/scheduler.h b/src/scheduler.h index 31b216b00..f8136a5dd 100644 --- a/src/scheduler.h +++ b/src/scheduler.h @@ -29,7 +29,7 @@ using grpc::Channel; typedef size_t RefCount; -const ObjRef UNITIALIZED_ALIAS = std::numeric_limits::max(); +const ObjectID UNITIALIZED_ALIAS = std::numeric_limits::max(); const RefCount DEALLOCATED = std::numeric_limits::max(); struct WorkerHandle { @@ -62,7 +62,7 @@ public: Status SubmitTask(ServerContext* context, const SubmitTaskRequest* request, SubmitTaskReply* reply) override; Status PutObj(ServerContext* context, const PutObjRequest* request, PutObjReply* reply) override; Status RequestObj(ServerContext* context, const RequestObjRequest* request, AckReply* reply) override; - Status AliasObjRefs(ServerContext* context, const AliasObjRefsRequest* request, AckReply* reply) override; + Status AliasObjectIDs(ServerContext* context, const AliasObjectIDsRequest* request, AckReply* reply) override; Status RegisterObjStore(ServerContext* context, const RegisterObjStoreRequest* request, RegisterObjStoreReply* reply) override; Status RegisterWorker(ServerContext* context, const RegisterWorkerRequest* request, RegisterWorkerReply* reply) override; Status RegisterFunction(ServerContext* context, const RegisterFunctionRequest* request, AckReply* reply) override; @@ -70,7 +70,7 @@ public: Status ReadyForNewTask(ServerContext* context, const ReadyForNewTaskRequest* request, AckReply* reply) override; Status IncrementRefCount(ServerContext* context, const IncrementRefCountRequest* request, AckReply* reply) override; Status DecrementRefCount(ServerContext* context, const DecrementRefCountRequest* request, AckReply* reply) override; - Status AddContainedObjRefs(ServerContext* context, const AddContainedObjRefsRequest* request, AckReply* reply) override; + Status AddContainedObjectIDs(ServerContext* context, const AddContainedObjectIDsRequest* request, AckReply* reply) override; Status SchedulerInfo(ServerContext* context, const SchedulerInfoRequest* request, SchedulerInfoReply* reply) override; Status TaskInfo(ServerContext* context, const TaskInfoRequest* request, TaskInfoReply* reply) override; Status KillWorkers(ServerContext* context, const KillWorkersRequest* request, KillWorkersReply* reply) override; @@ -90,23 +90,23 @@ public: // This will ask an object store to send an object to another object store if // the object is not already present in that object store and is not already // being transmitted. - void deliver_object_async_if_necessary(ObjRef objref, ObjStoreId from, ObjStoreId to); + void deliver_object_async_if_necessary(ObjectID objectid, ObjStoreId from, ObjStoreId to); // ask an object store to send object to another object store - void deliver_object_async(ObjRef objref, ObjStoreId from, ObjStoreId to); + void deliver_object_async(ObjectID objectid, ObjStoreId from, ObjStoreId to); // assign a task to a worker void schedule(); - // execute a task on a worker and ship required object references + // execute a task on a worker and ship required object IDs void assign_task(OperationId operationid, WorkerId workerid, const MySynchronizedPtr &computation_graph); // checks if the dependencies of the task are met bool can_run(const Task& task); // register a worker and its object store (if it has not been registered yet) std::pair register_worker(const std::string& worker_address, const std::string& objstore_address, bool is_driver); - // register a new object with the scheduler and return its object reference - ObjRef register_new_object(); - // register the location of the object reference in the object table - void add_location(ObjRef objref, ObjStoreId objstoreid); - // indicate that objref is a canonical objref - void add_canonical_objref(ObjRef objref); + // register a new object with the scheduler and return its object ID + ObjectID register_new_object(); + // register the location of the object ID in the object table + void add_location(ObjectID objectid, ObjStoreId objstoreid); + // indicate that objectid is a canonical objectid + void add_canonical_objectid(ObjectID objectid); // get object store associated with a workerid ObjStoreId get_store(WorkerId workerid); // register a function with the scheduler @@ -115,34 +115,34 @@ public: void get_info(const SchedulerInfoRequest& request, SchedulerInfoReply* reply); private: // pick an objectstore that holds a given object (needs protection by objects_lock_) - ObjStoreId pick_objstore(ObjRef objref); - // checks if objref is a canonical objref - bool is_canonical(ObjRef objref); + ObjStoreId pick_objstore(ObjectID objectid); + // checks if objectid is a canonical objectid + bool is_canonical(ObjectID objectid); void perform_gets(); // schedule tasks using the naive algorithm void schedule_tasks_naively(); // schedule tasks using a scheduling algorithm that takes into account data locality void schedule_tasks_location_aware(); void perform_notify_aliases(); - // checks if aliasing for objref has been completed - bool has_canonical_objref(ObjRef objref); - // get the canonical objref for an objref - ObjRef get_canonical_objref(ObjRef objref); - // attempt to notify the objstore about potential objref aliasing, returns true if successful, if false then retry later - bool attempt_notify_alias(ObjStoreId objstoreid, ObjRef alias_objref, ObjRef canonical_objref); - // tell all of the objstores holding canonical_objref to deallocate it, the + // checks if aliasing for objectid has been completed + bool has_canonical_objectid(ObjectID objectid); + // get the canonical objectid for an objectid + ObjectID get_canonical_objectid(ObjectID objectid); + // attempt to notify the objstore about potential objectid aliasing, returns true if successful, if false then retry later + bool attempt_notify_alias(ObjStoreId objstoreid, ObjectID alias_objectid, ObjectID canonical_objectid); + // tell all of the objstores holding canonical_objectid to deallocate it, the // data structures are passed into ensure that the appropriate locks are held. - void deallocate_object(ObjRef canonical_objref, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objrefs); - // increment the ref counts for the object references in objrefs, the data + void deallocate_object(ObjectID canonical_objectid, const MySynchronizedPtr > &reference_counts, const MySynchronizedPtr > > &contained_objectids); + // increment the ref counts for the object IDs in objectids, the data // structures are passed into ensure that the appropriate locks are held. - void increment_ref_count(const std::vector &objrefs, const MySynchronizedPtr > &reference_count); - // decrement the ref counts for the object references in objrefs, the data + void increment_ref_count(const std::vector &objectids, const MySynchronizedPtr > &reference_count); + // decrement the ref counts for the object IDs in objectids, the data // structures are passed into ensure that the appropriate locks are held. - void decrement_ref_count(const std::vector &objrefs, const MySynchronizedPtr > &reference_count, const MySynchronizedPtr > > &contained_objrefs); - // Find all of the object references which are upstream of objref (including objref itself). That is, you can get from everything in objrefs to objref by repeatedly indexing in target_objrefs_. - void upstream_objrefs(ObjRef objref, std::vector &objrefs, const MySynchronizedPtr > > &reverse_target_objrefs); - // Find all of the object references that refer to the same object as objref (as best as we can determine at the moment). The information may be incomplete because not all of the aliases may be known. - void get_equivalent_objrefs(ObjRef objref, std::vector &equivalent_objrefs); + void decrement_ref_count(const std::vector &objectids, const MySynchronizedPtr > &reference_count, const MySynchronizedPtr > > &contained_objectids); + // Find all of the object IDs which are upstream of objectid (including objectid itself). That is, you can get from everything in objectids to objectid by repeatedly indexing in target_objectids_. + void upstream_objectids(ObjectID objectid, std::vector &objectids, const MySynchronizedPtr > > &reverse_target_objectids); + // Find all of the object IDs that refer to the same object as objectid (as best as we can determine at the moment). The information may be incomplete because not all of the aliases may be known. + void get_equivalent_objectids(ObjectID objectid, std::vector &equivalent_objectids); // Export a remote function to a worker. void export_function_to_worker(WorkerId workerid, int function_index, MySynchronizedPtr > &workers, const MySynchronizedPtr > > &exported_functions); // Export a reusable variable to a worker @@ -175,7 +175,7 @@ private: // List of failed tasks Synchronized > failed_tasks_; // List of pending get calls. - Synchronized > > get_queue_; + Synchronized > > get_queue_; // The computation graph tracks the operations that have been submitted to the // scheduler and is mostly used for fault tolerance. Synchronized computation_graph_; @@ -185,42 +185,42 @@ private: Synchronized > avail_workers_; // List of pending tasks. Synchronized > task_queue_; - // Reference counts. Currently, reference_counts_[objref] is the number of - // existing references held to objref. This is done for all objrefs, not just - // canonical_objrefs. This data structure completely ignores aliasing. If the - // object corresponding to objref has been deallocated, then - // reference_counts[objref] will equal DEALLOCATED. + // Reference counts. Currently, reference_counts_[objectid] is the number of + // existing references held to objectid. This is done for all objectids, not just + // canonical_objectids. This data structure completely ignores aliasing. If the + // object corresponding to objectid has been deallocated, then + // reference_counts[objectid] will equal DEALLOCATED. Synchronized > reference_counts_; - // contained_objrefs_[objref] is a vector of all of the objrefs contained inside the object referred to by objref - Synchronized > > contained_objrefs_; + // contained_objectids_[objectid] is a vector of all of the objectids contained inside the object referred to by objectid + Synchronized > > contained_objectids_; // Vector of all workers registered in the system. Their index in this vector // is the workerid. Synchronized > workers_; - // List of pending alias notifications. Each element consists of (objstoreid, (alias_objref, canonical_objref)). - Synchronized > > > alias_notification_queue_; - // Mapping from canonical objref to list of object stores where the object is stored. Non-canonical (aliased) objrefs should not be used to index objtable_. + // List of pending alias notifications. Each element consists of (objstoreid, (alias_objectid, canonical_objectid)). + Synchronized > > > alias_notification_queue_; + // Mapping from canonical objectid to list of object stores where the object is stored. Non-canonical (aliased) objectids should not be used to index objtable_. Synchronized objtable_; // This lock protects objtable_ and objects_in_transit_ // Vector of all object stores registered in the system. Their index in this // vector is the objstoreid. Synchronized > objstores_; - // Mapping from an aliased objref to the objref it is aliased with. If an - // objref is a canonical objref (meaning it is not aliased), then - // target_objrefs_[objref] == objref. For each objref, target_objrefs_[objref] + // Mapping from an aliased objectid to the objectid it is aliased with. If an + // objectid is a canonical objectid (meaning it is not aliased), then + // target_objectids_[objectid] == objectid. For each objectid, target_objectids_[objectid] // is initialized to UNITIALIZED_ALIAS and the correct value is filled later // when it is known. - Synchronized > target_objrefs_; - // This data structure maps an objref to all of the objrefs that alias it (there could be multiple such objrefs). - Synchronized > > reverse_target_objrefs_; + Synchronized > target_objectids_; + // This data structure maps an objectid to all of the objectids that alias it (there could be multiple such objectids). + Synchronized > > reverse_target_objectids_; // For each object store objstoreid, objects_in_transit_[objstoreid] is a - // vector of the canonical object references that are being streamed to that - // object store but are not yet present. Object references are added to this + // vector of the canonical object IDs that are being streamed to that + // object store but are not yet present. object IDs are added to this // in deliver_object_async_if_necessary (to ensure that we do not attempt to deliver - // the same object to a given object store twice), and object references are + // the same object to a given object store twice), and object IDs are // removed when add_location is called (from ObjReady), and they are moved to // the objtable_. Note that objects_in_transit_ and objtable_ share the same // lock (objects_lock_). // TODO(rkn): Consider making this part of the // objtable data structure. - std::vector > objects_in_transit_; + std::vector > objects_in_transit_; // All of the remote functions that have been exported to the workers. Synchronized > > exported_functions_; // All of the reusable variables that have been exported to the workers. diff --git a/src/worker.cc b/src/worker.cc index 9da8a7c3e..b0886c29b 100644 --- a/src/worker.cc +++ b/src/worker.cc @@ -114,35 +114,35 @@ void Worker::register_worker(const std::string& worker_address, const std::strin return; } -void Worker::request_object(ObjRef objref) { +void Worker::request_object(ObjectID objectid) { RAY_CHECK(connected_, "Attempted to perform request_object but failed."); RequestObjRequest request; request.set_workerid(workerid_); - request.set_objref(objref); + request.set_objectid(objectid); AckReply reply; ClientContext context; Status status = scheduler_stub_->RequestObj(&context, request, &reply); return; } -ObjRef Worker::get_objref() { - // first get objref for the new object - RAY_CHECK(connected_, "Attempted to perform get_objref but failed."); +ObjectID Worker::get_objectid() { + // first get objectid for the new object + RAY_CHECK(connected_, "Attempted to perform get_objectid but failed."); PutObjRequest request; request.set_workerid(workerid_); PutObjReply reply; ClientContext context; Status status = scheduler_stub_->PutObj(&context, request, &reply); - return reply.objref(); + return reply.objectid(); } -slice Worker::get_object(ObjRef objref) { - // get_object assumes that objref is a canonical objref +slice Worker::get_object(ObjectID objectid) { + // get_object assumes that objectid is a canonical objectid RAY_CHECK(connected_, "Attempted to perform get_object but failed."); ObjRequest request; request.workerid = workerid_; request.type = ObjRequestType::GET; - request.objref = objref; + request.objectid = objectid; RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); ObjHandle result; RAY_CHECK(receive_obj_queue_.receive(&result), "error receiving over IPC"); @@ -154,20 +154,20 @@ slice Worker::get_object(ObjRef objref) { } // TODO(pcm): More error handling -// contained_objrefs is a vector of all the objrefs contained in obj -void Worker::put_object(ObjRef objref, const Obj* obj, std::vector &contained_objrefs) { +// contained_objectids is a vector of all the objectids contained in obj +void Worker::put_object(ObjectID objectid, const Obj* obj, std::vector &contained_objectids) { RAY_CHECK(connected_, "Attempted to perform put_object but failed."); std::string data; obj->SerializeToString(&data); // TODO(pcm): get rid of this serialization ObjRequest request; request.workerid = workerid_; request.type = ObjRequestType::ALLOC; - request.objref = objref; + request.objectid = objectid; request.size = data.size(); RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); - if (contained_objrefs.size() > 0) { - RAY_LOG(RAY_REFCOUNT, "In put_object, calling increment_reference_count for contained objrefs"); - increment_reference_count(contained_objrefs); // Notify the scheduler that some object references are serialized in the objstore. + if (contained_objectids.size() > 0) { + RAY_LOG(RAY_REFCOUNT, "In put_object, calling increment_reference_count for contained objectids"); + increment_reference_count(contained_objectids); // Notify the scheduler that some object references are serialized in the objstore. } ObjHandle result; RAY_CHECK(receive_obj_queue_.receive(&result), "error receiving over IPC"); @@ -180,15 +180,15 @@ void Worker::put_object(ObjRef objref, const Obj* obj, std::vector &cont request.metadata_offset = 0; RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); - // Notify the scheduler about the objrefs that we are serializing in the objstore. - AddContainedObjRefsRequest contained_objrefs_request; - contained_objrefs_request.set_objref(objref); - for (int i = 0; i < contained_objrefs.size(); ++i) { - contained_objrefs_request.add_contained_objref(contained_objrefs[i]); // TODO(rkn): The naming here is bad + // Notify the scheduler about the objectids that we are serializing in the objstore. + AddContainedObjectIDsRequest contained_objectids_request; + contained_objectids_request.set_objectid(objectid); + for (int i = 0; i < contained_objectids.size(); ++i) { + contained_objectids_request.add_contained_objectid(contained_objectids[i]); // TODO(rkn): The naming here is bad } AckReply reply; ClientContext context; - scheduler_stub_->AddContainedObjRefs(&context, contained_objrefs_request, &reply); + scheduler_stub_->AddContainedObjectIDs(&context, contained_objectids_request, &reply); } #define CHECK_ARROW_STATUS(s, msg) \ @@ -201,12 +201,12 @@ void Worker::put_object(ObjRef objref, const Obj* obj, std::vector &cont } \ } while (0); -const char* Worker::allocate_buffer(ObjRef objref, int64_t size, SegmentId& segmentid) { +const char* Worker::allocate_buffer(ObjectID objectid, int64_t size, SegmentId& segmentid) { RAY_CHECK(connected_, "Attempted to perform put_arrow but failed."); ObjRequest request; request.workerid = workerid_; request.type = ObjRequestType::ALLOC; - request.objref = objref; + request.objectid = objectid; request.size = size; RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); ObjHandle result; @@ -216,23 +216,23 @@ const char* Worker::allocate_buffer(ObjRef objref, int64_t size, SegmentId& segm return address; } -PyObject* Worker::finish_buffer(ObjRef objref, SegmentId segmentid, int64_t metadata_offset) { +PyObject* Worker::finish_buffer(ObjectID objectid, SegmentId segmentid, int64_t metadata_offset) { segmentpool_->unmap_segment(segmentid); ObjRequest request; request.workerid = workerid_; - request.objref = objref; + request.objectid = objectid; request.type = ObjRequestType::WORKER_DONE; request.metadata_offset = metadata_offset; RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); Py_RETURN_NONE; } -const char* Worker::get_buffer(ObjRef objref, int64_t &size, SegmentId& segmentid, int64_t& metadata_offset) { +const char* Worker::get_buffer(ObjectID objectid, int64_t &size, SegmentId& segmentid, int64_t& metadata_offset) { RAY_CHECK(connected_, "Attempted to perform get_arrow but failed."); ObjRequest request; request.workerid = workerid_; request.type = ObjRequestType::GET; - request.objref = objref; + request.objectid = objectid; RAY_CHECK(request_obj_queue_.send(&request), "error sending over IPC"); ObjHandle result; RAY_CHECK(receive_obj_queue_.receive(&result), "error receiving over IPC"); @@ -243,64 +243,64 @@ const char* Worker::get_buffer(ObjRef objref, int64_t &size, SegmentId& segmenti return address; } -bool Worker::is_arrow(ObjRef objref) { +bool Worker::is_arrow(ObjectID objectid) { RAY_CHECK(connected_, "Attempted to perform is_arrow but failed."); ObjRequest request; request.workerid = workerid_; request.type = ObjRequestType::GET; - request.objref = objref; + request.objectid = objectid; request_obj_queue_.send(&request); ObjHandle result; RAY_CHECK(receive_obj_queue_.receive(&result), "error receiving over IPC"); return result.metadata_offset() != 0; } -void Worker::unmap_object(ObjRef objref) { +void Worker::unmap_object(ObjectID objectid) { if (!connected_) { RAY_LOG(RAY_DEBUG, "Attempted to perform unmap_object but failed."); return; } - segmentpool_->unmap_segment(objref); + segmentpool_->unmap_segment(objectid); } -void Worker::alias_objrefs(ObjRef alias_objref, ObjRef target_objref) { - RAY_CHECK(connected_, "Attempted to perform alias_objrefs but failed."); +void Worker::alias_objectids(ObjectID alias_objectid, ObjectID target_objectid) { + RAY_CHECK(connected_, "Attempted to perform alias_objectids but failed."); ClientContext context; - AliasObjRefsRequest request; - request.set_alias_objref(alias_objref); - request.set_target_objref(target_objref); + AliasObjectIDsRequest request; + request.set_alias_objectid(alias_objectid); + request.set_target_objectid(target_objectid); AckReply reply; - scheduler_stub_->AliasObjRefs(&context, request, &reply); + scheduler_stub_->AliasObjectIDs(&context, request, &reply); } -void Worker::increment_reference_count(std::vector &objrefs) { +void Worker::increment_reference_count(std::vector &objectids) { if (!connected_) { - RAY_LOG(RAY_DEBUG, "Attempting to increment_reference_count for objrefs, but connected_ = " << connected_ << " so returning instead."); + RAY_LOG(RAY_DEBUG, "Attempting to increment_reference_count for objectids, but connected_ = " << connected_ << " so returning instead."); return; } - if (objrefs.size() > 0) { + if (objectids.size() > 0) { ClientContext context; IncrementRefCountRequest request; - for (int i = 0; i < objrefs.size(); ++i) { - RAY_LOG(RAY_REFCOUNT, "Incrementing reference count for objref " << objrefs[i]); - request.add_objref(objrefs[i]); + for (int i = 0; i < objectids.size(); ++i) { + RAY_LOG(RAY_REFCOUNT, "Incrementing reference count for objectid " << objectids[i]); + request.add_objectid(objectids[i]); } AckReply reply; scheduler_stub_->IncrementRefCount(&context, request, &reply); } } -void Worker::decrement_reference_count(std::vector &objrefs) { +void Worker::decrement_reference_count(std::vector &objectids) { if (!connected_) { RAY_LOG(RAY_DEBUG, "Attempting to decrement_reference_count, but connected_ = " << connected_ << " so returning instead."); return; } - if (objrefs.size() > 0) { + if (objectids.size() > 0) { ClientContext context; DecrementRefCountRequest request; - for (int i = 0; i < objrefs.size(); ++i) { - RAY_LOG(RAY_REFCOUNT, "Decrementing reference count for objref " << objrefs[i]); - request.add_objref(objrefs[i]); + for (int i = 0; i < objectids.size(); ++i) { + RAY_LOG(RAY_REFCOUNT, "Decrementing reference count for objectid " << objectids[i]); + request.add_objectid(objectids[i]); } AckReply reply; scheduler_stub_->DecrementRefCount(&context, request, &reply); diff --git a/src/worker.h b/src/worker.h index 945550d87..d49f99045 100644 --- a/src/worker.h +++ b/src/worker.h @@ -47,30 +47,30 @@ class Worker { bool kill_workers(ClientContext &context); // send request to the scheduler to register this worker void register_worker(const std::string& worker_address, const std::string& objstore_address, bool is_driver); - // get a new object reference that is registered with the scheduler - ObjRef get_objref(); + // get a new object ID that is registered with the scheduler + ObjectID get_objectid(); // request an object to be delivered to the local object store - void request_object(ObjRef objref); + void request_object(ObjectID objectid); // stores an object to the local object store - void put_object(ObjRef objref, const Obj* obj, std::vector &contained_objrefs); + void put_object(ObjectID objectid, const Obj* obj, std::vector &contained_objectids); // retrieve serialized object from local object store - slice get_object(ObjRef objref); - // Allocates buffer for objref with size of size - const char* allocate_buffer(ObjRef objref, int64_t size, SegmentId& segmentid); + slice get_object(ObjectID objectid); + // Allocates buffer for objectid with size of size + const char* allocate_buffer(ObjectID objectid, int64_t size, SegmentId& segmentid); // Finishes buffer with segmentid and an offset of metadata_ofset - PyObject* finish_buffer(ObjRef objref, SegmentId segmentid, int64_t metadata_offset); - // Gets the buffer for objref - const char* get_buffer(ObjRef objref, int64_t& size, SegmentId& segmentid, int64_t& metadata_offset); - // determine if the object stored in objref is an arrow object // TODO(pcm): more general mechanism for this? - bool is_arrow(ObjRef objref); + PyObject* finish_buffer(ObjectID objectid, SegmentId segmentid, int64_t metadata_offset); + // Gets the buffer for objectid + const char* get_buffer(ObjectID objectid, int64_t& size, SegmentId& segmentid, int64_t& metadata_offset); + // determine if the object stored in objectid is an arrow object // TODO(pcm): more general mechanism for this? + bool is_arrow(ObjectID objectid); // unmap the segment containing an object from the local address space - void unmap_object(ObjRef objref); - // make `alias_objref` refer to the same object that `target_objref` refers to - void alias_objrefs(ObjRef alias_objref, ObjRef target_objref); - // increment the reference count for objref - void increment_reference_count(std::vector &objref); - // decrement the reference count for objref - void decrement_reference_count(std::vector &objref); + void unmap_object(ObjectID objectid); + // make `alias_objectid` refer to the same object that `target_objectid` refers to + void alias_objectids(ObjectID alias_objectid, ObjectID target_objectid); + // increment the reference count for objectid + void increment_reference_count(std::vector &objectid); + // decrement the reference count for objectid + void decrement_reference_count(std::vector &objectid); // register function with scheduler void register_function(const std::string& name, size_t num_return_vals); // start the worker server which accepts tasks from the scheduler and stores diff --git a/test/array_test.py b/test/array_test.py index 411b0efda..ba5f65fd8 100644 --- a/test/array_test.py +++ b/test/array_test.py @@ -16,29 +16,29 @@ class RemoteArrayTest(unittest.TestCase): ray.init(start_ray_local=True) # test eye - ref = ra.eye.remote(3) - val = ray.get(ref) + object_id = ra.eye.remote(3) + val = ray.get(object_id) self.assertTrue(np.alltrue(val == np.eye(3))) # test zeros - ref = ra.zeros.remote([3, 4, 5]) - val = ray.get(ref) + object_id = ra.zeros.remote([3, 4, 5]) + val = ray.get(object_id) self.assertTrue(np.alltrue(val == np.zeros([3, 4, 5]))) # test qr - pass by value - val_a = np.random.normal(size=[10, 11]) - ref_q, ref_r = ra.linalg.qr.remote(val_a) - val_q = ray.get(ref_q) - val_r = ray.get(ref_r) - self.assertTrue(np.allclose(np.dot(val_q, val_r), val_a)) + a_val = np.random.normal(size=[10, 11]) + q_id, r_id = ra.linalg.qr.remote(a_val) + q_val = ray.get(q_id) + r_val = ray.get(r_id) + self.assertTrue(np.allclose(np.dot(q_val, r_val), a_val)) - # test qr - pass by objref + # test qr - pass by objectid a = ra.random.normal.remote([10, 13]) - ref_q, ref_r = ra.linalg.qr.remote(a) - val_a = ray.get(a) - val_q = ray.get(ref_q) - val_r = ray.get(ref_r) - self.assertTrue(np.allclose(np.dot(val_q, val_r), val_a)) + q_id, r_id = ra.linalg.qr.remote(a) + a_val = ray.get(a) + q_val = ray.get(q_id) + r_val = ray.get(r_id) + self.assertTrue(np.allclose(np.dot(q_val, r_val), a_val)) ray.services.cleanup() @@ -54,7 +54,7 @@ class DistributedArrayTest(unittest.TestCase): capsule, _ = ray.serialization.serialize(ray.worker.global_worker.handle, x) y = ray.serialization.deserialize(ray.worker.global_worker.handle, capsule) self.assertEqual(x.shape, y.shape) - self.assertEqual(x.objrefs[0, 0, 0].val, y.objrefs[0, 0, 0].val) + self.assertEqual(x.objectids[0, 0, 0].id, y.objectids[0, 0, 0].id) ray.services.cleanup() diff --git a/test/memory_leak_deserialize.py b/test/memory_leak_deserialize.py index e6549297d..cbc20e98d 100644 --- a/test/memory_leak_deserialize.py +++ b/test/memory_leak_deserialize.py @@ -8,7 +8,7 @@ ray.init(start_ray_local=True, num_workers=1) d = {"w": np.zeros(1000000)} -obj_capsule, contained_objrefs = ray.lib.serialize_object(ray.worker.global_worker.handle, d) +obj_capsule, contained_objectids = ray.lib.serialize_object(ray.worker.global_worker.handle, d) while True: ray.lib.deserialize_object(ray.worker.global_worker.handle, obj_capsule) diff --git a/test/runtest.py b/test/runtest.py index 27fd75038..af9008257 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -97,50 +97,50 @@ class ObjStoreTest(unittest.TestCase): # putting and getting an object shouldn't change it for data in ["h", "h" * 10000, 0, 0.0]: - objref = ray.put(data, w1) - result = ray.get(objref, w1) + objectid = ray.put(data, w1) + result = ray.get(objectid, w1) self.assertEqual(result, data) # putting an object, shipping it to another worker, and getting it shouldn't change it for data in ["h", "h" * 10000, 0, 0.0, [1, 2, 3, "a", (1, 2)], ("a", ("b", 3))]: - objref = ray.put(data, w1) - result = ray.get(objref, w2) + objectid = ray.put(data, w1) + result = ray.get(objectid, w2) self.assertEqual(result, data) # putting an array, shipping it to another worker, and getting it shouldn't change it for data in [np.zeros([10, 20]), np.random.normal(size=[45, 25])]: - objref = ray.put(data, w1) - result = ray.get(objref, w2) + objectid = ray.put(data, w1) + result = ray.get(objectid, w2) self.assertTrue(np.alltrue(result == data)) # getting multiple times shouldn't matter # for data in [np.zeros([10, 20]), np.random.normal(size=[45, 25]), np.zeros([10, 20], dtype=np.dtype("float64")), np.zeros([10, 20], dtype=np.dtype("float32")), np.zeros([10, 20], dtype=np.dtype("int64")), np.zeros([10, 20], dtype=np.dtype("int32"))]: - # objref = worker.put(data, w1) - # result = worker.get(objref, w2) - # result = worker.get(objref, w2) - # result = worker.get(objref, w2) + # objectid = worker.put(data, w1) + # result = worker.get(objectid, w2) + # result = worker.get(objectid, w2) + # result = worker.get(objectid, w2) # self.assertTrue(np.alltrue(result == data)) # shipping a numpy array inside something else should be fine data = ("a", np.random.normal(size=[10, 10])) - objref = ray.put(data, w1) - result = ray.get(objref, w2) + objectid = ray.put(data, w1) + result = ray.get(objectid, w2) self.assertEqual(data[0], result[0]) self.assertTrue(np.alltrue(data[1] == result[1])) # shipping a numpy array inside something else should be fine data = ["a", np.random.normal(size=[10, 10])] - objref = ray.put(data, w1) - result = ray.get(objref, w2) + objectid = ray.put(data, w1) + result = ray.get(objectid, w2) self.assertEqual(data[0], result[0]) self.assertTrue(np.alltrue(data[1] == result[1])) # Getting a buffer after modifying it before it finishes should return updated buffer - objref = ray.lib.get_objref(w1.handle) - buf = ray.lib.allocate_buffer(w1.handle, objref, 100) + objectid = ray.lib.get_objectid(w1.handle) + buf = ray.lib.allocate_buffer(w1.handle, objectid, 100) buf[0][0] = 1 - ray.lib.finish_buffer(w1.handle, objref, buf[1], 0) - completedbuffer = ray.lib.get_buffer(w1.handle, objref) + ray.lib.finish_buffer(w1.handle, objectid, buf[1], 0) + completedbuffer = ray.lib.get_buffer(w1.handle, objectid) self.assertEqual(completedbuffer[0][0], 1) ray.services.cleanup() @@ -152,33 +152,33 @@ class WorkerTest(unittest.TestCase): for i in range(100): value_before = i * 10 ** 6 - objref = ray.put(value_before) - value_after = ray.get(objref) + objectid = ray.put(value_before) + value_after = ray.get(objectid) self.assertEqual(value_before, value_after) for i in range(100): value_before = i * 10 ** 6 * 1.0 - objref = ray.put(value_before) - value_after = ray.get(objref) + objectid = ray.put(value_before) + value_after = ray.get(objectid) self.assertEqual(value_before, value_after) for i in range(100): value_before = "h" * i - objref = ray.put(value_before) - value_after = ray.get(objref) + objectid = ray.put(value_before) + value_after = ray.get(objectid) self.assertEqual(value_before, value_after) for i in range(100): value_before = [1] * i - objref = ray.put(value_before) - value_after = ray.get(objref) + objectid = ray.put(value_before) + value_after = ray.get(objectid) self.assertEqual(value_before, value_after) ray.services.cleanup() class APITest(unittest.TestCase): - def testObjRefAliasing(self): + def testObjectIDAliasing(self): reload(test_functions) ray.init(start_ray_local=True, num_workers=3, driver_mode=ray.SILENT_MODE) @@ -396,12 +396,12 @@ class TaskStatusTest(unittest.TestCase): def check_get_deallocated(data): x = ray.put(data) ray.get(x) - return x.val + return x.id def check_get_not_deallocated(data): x = ray.put(data) y = ray.get(x) - return y, x.val + return y, x.id class ReferenceCountingTest(unittest.TestCase): @@ -414,46 +414,46 @@ class ReferenceCountingTest(unittest.TestCase): x = test_functions.test_alias_f.remote() ray.get(x) time.sleep(0.1) - objref_val = x.val - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val], 1) + objectid_val = x.id + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val], 1) del x - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val], -1) # -1 indicates deallocated + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val], -1) # -1 indicates deallocated y = test_functions.test_alias_h.remote() ray.get(y) time.sleep(0.1) - objref_val = y.val - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [1, 0, 0]) + objectid_val = y.id + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [1, 0, 0]) del y - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [-1, -1, -1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [-1, -1, -1]) z = da.zeros.remote([da.BLOCK_SIZE, 2 * da.BLOCK_SIZE]) time.sleep(0.1) - objref_val = z.val - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [1, 1, 1]) + objectid_val = z.id + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [1, 1, 1]) del z time.sleep(0.1) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [-1, -1, -1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [-1, -1, -1]) x = ra.zeros.remote([10, 10]) y = ra.zeros.remote([10, 10]) z = ra.dot.remote(x, y) - objref_val = x.val + objectid_val = x.id time.sleep(0.1) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [1, 1, 1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [1, 1, 1]) del x time.sleep(0.1) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [-1, 1, 1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [-1, 1, 1]) del y time.sleep(0.1) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [-1, -1, 1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [-1, -1, 1]) del z time.sleep(0.1) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val:(objref_val + 3)], [-1, -1, -1]) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val:(objectid_val + 3)], [-1, -1, -1]) ray.services.cleanup() @@ -461,20 +461,20 @@ class ReferenceCountingTest(unittest.TestCase): ray.init(start_ray_local=True, num_workers=3) for val in RAY_TEST_OBJECTS + [np.zeros((2, 2)), UserDefinedType()]: - objref_val = check_get_deallocated(val) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val], -1) + objectid_val = check_get_deallocated(val) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val], -1) if not isinstance(val, bool) and not isinstance(val, np.generic) and val is not None: - x, objref_val = check_get_not_deallocated(val) - self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val], 1) + x, objectid_val = check_get_not_deallocated(val) + self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val], 1) # The following currently segfaults: The second "result = " closes the # memory segment as soon as the assignment is done (and the first result # goes out of scope). # data = np.zeros([10, 20]) - # objref = ray.put(data) - # result = worker.get(objref) - # result = worker.get(objref) + # objectid = ray.put(data) + # result = worker.get(objectid) + # result = worker.get(objectid) # self.assertTrue(np.alltrue(result == data)) ray.services.cleanup() @@ -490,8 +490,8 @@ class ReferenceCountingTest(unittest.TestCase): # # None are returned by get "by value" and therefore can be reclaimed from # # the object store safely. # for val in [True, False, None]: - # x, objref_val = check_get_not_deallocated(val) - # self.assertEqual(ray.scheduler_info()["reference_counts"][objref_val], 1) + # x, objectid_val = check_get_not_deallocated(val) + # self.assertEqual(ray.scheduler_info()["reference_counts"][objectid_val], 1) # ray.services.cleanup()