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][Spilled Object Leakage] More robust spilled object deletion #29014

Merged
merged 15 commits into from
Oct 7, 2022

Conversation

scv119
Copy link
Contributor

@scv119 scv119 commented Oct 3, 2022

Why are these changes needed?

We have noticed spilled objects not deleted even if the job creating those objects finished execution. After reading the code my theory is that the object delegation worker failed in the middle of deleting spilled files, which doesn't handle well in today's spilled object deletion logic.

Though I no longer get a reproduction, (which I suspect due to the fix #26395), we can enhance the failure handle logic when object deletion failed.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@scv119 scv119 linked an issue Oct 3, 2022 that may be closed by this pull request
@scv119
Copy link
Contributor Author

scv119 commented Oct 3, 2022

TBD, add tests.

@rkooo567 rkooo567 self-assigned this Oct 4, 2022
@scv119 scv119 marked this pull request as ready for review October 6, 2022 20:55
@rkooo567
Copy link
Contributor

rkooo567 commented Oct 6, 2022

my theory is that the object delegation worker failed in the middle of deleting spilled files, which doesn't handle well in today's spilled object deletion logic.

Is it something we can reproducing from cpp tests or by killing the spill worker in the middle of the test?

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.

Mostly LGTM!

/// \param num_retries Num of retries allowed in case of failure, zero or negative
/// means don't retry.
void DeleteSpilledObjects(std::vector<std::string> urls_to_delete,
int64_t num_retries = kDefaultSpilledObjectDeleteRetries);
Copy link
Contributor

Choose a reason for hiding this comment

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

num_retries_left?

@@ -365,6 +377,9 @@ class LocalObjectManager {
/// The last time a restore log finished.
int64_t last_restore_log_ns_ = 0;

/// The number of failed deletion requests.
std::atomic<int64_t> num_failed_deletion_requests_ = 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

Btw I think this doesn't have to be atomic (it is single threaded)

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 i thought the io_worker->rpc_client()->DeleteSpilledObjects callback happens in a different thread.

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we should start document explicitly where callbacks are run w.r.t threading model lol

@@ -613,6 +624,8 @@ void LocalObjectManager::RecordMetrics() const {
ray::stats::STATS_spill_manager_request_total.Record(spilled_objects_total_, "Spilled");
ray::stats::STATS_spill_manager_request_total.Record(restored_objects_total_,
"Restored");
ray::stats::STATS_spill_manager_request_total.Record(num_failed_deletion_requests_,
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm Wonder if we need more breakdown;;

type: spill/restore/delete
status: success/failed

Copy link
Contributor

Choose a reason for hiding this comment

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

i could work on this as part of 2.2.

python/ray/_private/external_storage.py Show resolved Hide resolved
Copy link
Contributor

@rickyyx rickyyx left a comment

Choose a reason for hiding this comment

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

Are there any release tests we need to trigger for this? Or it was originally from some other failures.

@@ -418,7 +425,11 @@ def restore_spilled_objects(
def delete_spilled_objects(self, urls: List[str]):
Copy link
Contributor

Choose a reason for hiding this comment

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

Feel the API could be better by returning failure urls (e.g. failure in parsing, failure in delete file etc...)

@@ -143,6 +148,39 @@ def wait_until_actor_dead():
assert_no_thrashing(address["address"])


@pytest.mark.skipif(platform.system() in ["Windows"], reason="Failing on Windows.")
Copy link
Contributor

Choose a reason for hiding this comment

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

<< status.ToString();
<< status.ToString() << ", retry count: " << num_retries;

if (num_retries > 0) {
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 need back-off for retry?

@@ -613,6 +624,8 @@ void LocalObjectManager::RecordMetrics() const {
ray::stats::STATS_spill_manager_request_total.Record(spilled_objects_total_, "Spilled");
ray::stats::STATS_spill_manager_request_total.Record(restored_objects_total_,
"Restored");
ray::stats::STATS_spill_manager_request_total.Record(num_failed_deletion_requests_,
Copy link
Contributor

Choose a reason for hiding this comment

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

i could work on this as part of 2.2.

@@ -365,6 +377,9 @@ class LocalObjectManager {
/// The last time a restore log finished.
int64_t last_restore_log_ns_ = 0;

/// The number of failed deletion requests.
std::atomic<int64_t> num_failed_deletion_requests_ = 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we should start document explicitly where callbacks are run w.r.t threading model lol

@scv119 scv119 merged commit c40632e into ray-project:master Oct 7, 2022
WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this pull request Dec 19, 2022
…ay-project#29014)

We have noticed spilled objects not deleted even if the job creating those objects finished execution. After reading the code my theory is that the object delegation worker failed in the middle of deleting spilled files, which doesn't handle well in today's spilled object deletion logic.

Though I no longer get a reproduction, (which I suspect due to the fix ray-project#26395), we can enhance the failure handle logic when object deletion failed.

Signed-off-by: Weichen Xu <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Core] spilled object leaked. [Core] raylet crashed due to unhandled std::length_error
4 participants