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

Refactor object restoration path #14821

Merged
merged 15 commits into from
Mar 26, 2021
Merged

Conversation

ericl
Copy link
Contributor

@ericl ericl commented Mar 21, 2021

Why are these changes needed?

This PR refactors the object restoration path to unify it with the normal object pull path, both eliminating the short polling and removing the need for separate restore RPCs. The new strategy is as follows:

  1. If an object has no locations, PullManager will
  • call Restore() directly if the object can be accessed directly (on local disk of the node, or remote storage), otherwise
  • treat the spilled_node_id as the singleton location and pull from there
  1. PushManager will call Restore() if it receives a request for a non in-memory object that is on local disk.

Note that this is latency-optimal, since PushManager will get a notification once the object becomes local again from Restore and immediately start the push.

TODO:

  • Fix and add unit tests

Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

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

This will definitely be way nicer once done, I left a few comments on an initial skim.

src/ray/object_manager/pull_manager.cc Show resolved Hide resolved
Comment on lines +346 to +351
// Issue a restore request if the object is on local disk. This is only relevant
// if the local filesystem storage type is being used.
auto object_url = get_spilled_object_url_(object_id);
if (!object_url.empty()) {
restore_spilled_object_(object_id, object_url, nullptr);
}
Copy link
Contributor

@clarkzinzow clarkzinzow Mar 21, 2021

Choose a reason for hiding this comment

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

Instead of restoring the object directly here, I think that we should issue a local pull through the pull manager so that the restoration is done under the pull manager's admission control.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm going to punt this for later, since #14817 will mitigate the issue and this isn't a regression.

In the long run, direct streaming from disk means we shouldn't need admission control here.

src/ray/raylet/local_object_manager.cc Show resolved Hide resolved
}

// TODO(ekl) should we more directly mark the object as lost in this case?
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 @stephanie-wang what should we do in this case? (this is an existing problem I guess). It seems we will be rapidly re-polling the object during Tick().

Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't it something that can happen before the spilled object URL / location is updated? (e.g., the object location is not available yet). If so, don't we just need to ignore this case?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm yeah it seems like we'll want to mark the object as failed. Or we can let the owner handle it now that we have the OBOD.

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 if we publish the URL update at the same time we remove the location, then I don't see how this could happen. The only case would be for node failure... I guess currently task dep manager will mark it as failed and cancel the pull at a higher level hopefully

Copy link
Contributor

Choose a reason for hiding this comment

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

Right now, the owner handles this case by marking the object as failed if it detects the failure of the primary location. It may already work for this case too (since the spilled location should also be the primary location), but we need to check.

@clarkzinzow clarkzinzow self-assigned this Mar 22, 2021
@ericl ericl changed the title [WIP] Refactor object restoration path Refactor object restoration path Mar 22, 2021
@ericl
Copy link
Contributor Author

ericl commented Mar 22, 2021

Ready for review.

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

This is a really nice refactoring!! Have some minor comments

@@ -60,6 +60,8 @@ def parse_url_with_offset(url_with_offset: str) -> Tuple[str, int, int]:
query_dict = urllib.parse.parse_qs(parsed_result.query)
# Split by ? to remove the query from the url.
base_url = parsed_result.geturl().split("?")[0]
if "offset" not in query_dict or "size" not in query_dict:
Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't this supposed to not happen ever? Why don't we use assert here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This gives a better error in case of mis-formatted URL (including the origin URL)

src/ray/object_manager/object_manager.h Show resolved Hide resolved
src/ray/object_manager/pull_manager.cc Outdated Show resolved Hide resolved
}

// TODO(ekl) should we more directly mark the object as lost in this case?
Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't it something that can happen before the spilled object URL / location is updated? (e.g., the object location is not available yet). If so, don't we just need to ignore this case?

std::function<void(const ray::Status &)> callback) {
if (objects_pending_restore_.count(object_id) > 0) {
// If the same object is restoring, we dedup here.
return;
}

if (is_external_storage_type_fs_ && node_id != self_node_id_) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we still need is_external_storage_type_fs_?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah, we still need it for a couple things like setting Node to Nil vs not nil, so we can't entirely remove it.

@rkooo567 rkooo567 added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Mar 23, 2021
Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

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

LGTM overall, although I think that is_external_storage_type_fs_ isn't being properly set.

src/ray/object_manager/pull_manager.cc Outdated Show resolved Hide resolved
src/ray/raylet/local_object_manager.h Show resolved Hide resolved
@ericl
Copy link
Contributor Author

ericl commented Mar 26, 2021

//python/ray/tune:test_convergence FAILED in 3 out of 3 in 287.4s

Test unrelated, merging

@ericl ericl merged commit 2157021 into ray-project:master Mar 26, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants