Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[core] Support generators to allow tasks to return a dynamic number of objects #28291

Merged
merged 37 commits into from
Sep 21, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
5636eb9
Worker side
stephanie-wang Sep 4, 2022
3457a46
Owner side, works except for when spilling?
stephanie-wang Sep 5, 2022
031640b
now it works for spilling/in-plasma objects
stephanie-wang Sep 5, 2022
e59dd65
recovery test. TODO:
stephanie-wang Sep 5, 2022
775ff3e
Sort of fix nondeterminism
stephanie-wang Sep 6, 2022
b316b9f
Update src/ray/protobuf/node_manager.proto
stephanie-wang Sep 6, 2022
2d0f5d4
Update src/ray/protobuf/pubsub.proto
stephanie-wang Sep 6, 2022
faf64cb
C++
stephanie-wang Sep 6, 2022
5d92940
doc
stephanie-wang Sep 6, 2022
7c69b32
fixes
stephanie-wang Sep 7, 2022
c1ecf26
Update python/ray/_raylet.pyx
stephanie-wang Sep 8, 2022
422e4b6
minor
stephanie-wang Sep 9, 2022
62b427c
refactor
stephanie-wang Sep 9, 2022
636954c
ref counting during recovery
stephanie-wang Sep 10, 2022
0b73319
ref counting fix
stephanie-wang Sep 10, 2022
450d428
num_returns=dynamic
stephanie-wang Sep 10, 2022
a0bca43
Return generator instead of ObjRef
stephanie-wang Sep 12, 2022
6511a04
doc
stephanie-wang Sep 12, 2022
7ea1404
lint
stephanie-wang Sep 12, 2022
6e4663a
docs
stephanie-wang Sep 12, 2022
7bba5b7
doc
stephanie-wang Sep 12, 2022
c31b1c4
fixes
stephanie-wang Sep 13, 2022
167c6be
update
stephanie-wang Sep 13, 2022
44d9d85
fix
stephanie-wang Sep 13, 2022
39ba48f
Merge remote-tracking branch 'upstream/master' into generators-forreal
stephanie-wang Sep 13, 2022
805a088
x
stephanie-wang Sep 14, 2022
3360d63
cpp
stephanie-wang Sep 14, 2022
2ed252a
fix
stephanie-wang Sep 14, 2022
3a811dc
x
stephanie-wang Sep 14, 2022
285d98d
x
stephanie-wang Sep 15, 2022
7a86f19
Merge remote-tracking branch 'upstream/master' into generators-forreal
stephanie-wang Sep 17, 2022
b045d34
options
stephanie-wang Sep 17, 2022
7b3cf27
x
stephanie-wang Sep 19, 2022
92a8491
experimental
stephanie-wang Sep 21, 2022
492b95e
experimental
stephanie-wang Sep 21, 2022
07ca73d
x
stephanie-wang Sep 21, 2022
c196c67
fix
stephanie-wang Sep 21, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
x
Signed-off-by: Stephanie Wang <[email protected]>
  • Loading branch information
stephanie-wang committed Sep 21, 2022
commit 07ca73d3a876938e1bd2a237fab6e262ab9ed75d
4 changes: 2 additions & 2 deletions python/ray/_private/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -2264,9 +2264,9 @@ def get(
if is_individual_id:
object_refs = [object_refs]

if not (isinstance(object_refs, list) or isinstance(object_refs)):
if not isinstance(object_refs, list):
raise ValueError(
"'object_refs' must either be an ObjectRef or " "a list of ObjectRefs."
"'object_refs' must either be an ObjectRef or a list of ObjectRefs."
)

# TODO(ujvl): Consider how to allow user to retrieve the ready objects.
Expand Down
11 changes: 5 additions & 6 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -860,15 +860,10 @@ cdef execute_task(
worker, errors,
returns)
if dynamic_returns != NULL:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a bit confused. Does dynamic_returns == NULL mean that we are doing reconstruction? From line 815, seems it's not.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It means that the caller set num_returns="dynamic".

# We generated dynamic objects during the first execution and
# we are now re-executing the task during object
# reconstruction. Store the error for the dynamically generated
# objects too.
# Store errors for any dynamically generated objects too.
dynamic_errors = []
for _ in range(dynamic_returns[0].size()):
dynamic_errors.append(failure_object)
# We pass is_dynamic=False because we have a fixed number of
# return objects to populate.
core_worker.store_task_outputs(
worker, dynamic_errors,
dynamic_returns)
Expand Down Expand Up @@ -2184,6 +2179,10 @@ cdef class CoreWorker:
else:
# This is the first execution of the task, so we don't know how
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't necessarily mean it's the first execution of the task? It can also mean the generator is empty?

Are we able to catch this case: the first execution returns an empty generator but re-execution returns a non-empty generator.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm good catch, let me check. We'll probably have to resolve this case as a follow-up.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually it works since we don't reconstruct empty ObjectRefGenerators. Added a test.

# many return objects it should have yet.
# NOTE(swang): returns could also be empty if the task returned
# an empty generator and was re-executed. However, this should
# not happen because we never reconstruct empty
# ObjectRefGenerators (since these are not stored in plasma).
num_returns = -1
else:
# The task specified how many return values it should have.
Expand Down
55 changes: 55 additions & 0 deletions python/ray/tests/test_generators.py
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,61 @@ def fetch(x):
# ray.get(ref)


Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can generator itself be passed around as task args?

Can generator_obj_ref be passed around like other obj refs?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, and yes. I'll add test cases for these.

def test_dynamic_empty_generator_reconstruction_nondeterministic(ray_start_cluster):
config = {
"num_heartbeats_timeout": 10,
"raylet_heartbeat_period_milliseconds": 100,
"max_direct_call_object_size": 100,
"task_retry_delay_ms": 100,
"object_timeout_milliseconds": 200,
"fetch_warn_timeout_milliseconds": 1000,
}
cluster = ray_start_cluster
# Head node with no resources.
cluster.add_node(
num_cpus=0,
_system_config=config,
enable_object_reconstruction=True,
resources={"head": 1},
)
ray.init(address=cluster.address)
# Node to place the initial object.
node_to_kill = cluster.add_node(num_cpus=1, object_store_memory=10 ** 8)
cluster.wait_for_nodes()

@ray.remote(num_cpus=0, resources={"head": 1})
class ExecutionCounter:
def __init__(self):
self.count = 0

def inc(self):
self.count += 1
return self.count

def get_count(self):
return self.count

@ray.remote(num_returns="dynamic")
def maybe_empty_generator(exec_counter):
if ray.get(exec_counter.inc.remote()) > 1:
for i in range(3):
yield np.ones(1_000_000, dtype=np.int8) * i

@ray.remote
def check(empty_generator):
return len(empty_generator) == 0

exec_counter = ExecutionCounter.remote()
gen = maybe_empty_generator.remote(exec_counter)
assert ray.get(check.remote(gen))
cluster.remove_node(node_to_kill, allow_graceful=False)
node_to_kill = cluster.add_node(num_cpus=1, object_store_memory=10 ** 8)
assert ray.get(check.remote(gen))

# We should never reconstruct an empty generator.
assert exec_counter.get_count.remote() == 1


if __name__ == "__main__":
import os

Expand Down