-
Notifications
You must be signed in to change notification settings - Fork 476
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
delete_timeline: request coalescing #4159
Conversation
Co-authored-by: Joonas Koivunen <[email protected]>
- rename to test_timeline_delete_fail_before_local_delete - assert not just filesystem state but also pageserver's idea - assert other timelines are still present - comment assertions
- add test description comment - fix test name in enable_remote_storage
…ify assertions - clearly distinguish main and branch timeline id - use concise notation to assert list of timelines
…est timeout The test fails because the assert trips in the second call.
fails at the second call with Error processing HTTP request: InternalServerError(timeline is deleting, deleted_at: 2023-04-27T13:58:21.497414458
…safe this fixes the test added in the previous commit
…eleted_flag() This pushes the (unlikely) possibility of failure to serialize metadata out of stop(). That in turn leaves us with only one case of how stop() can fail. There are two callsites of stop(): 1. perform_upload_task: here, we can safely say "unreachable", and I think any future refactorings that might violate that invariant would notice, because the unreachable!() is close to the code that would likely be refactored. The unreachable!() is desirable there because otherwise we'd need to think about how to handle the error. Maybe the previous code would have done the right thing, maybe not. 2. delete_timeline: this is the new one, and, it's far away from the code that initializes the upload queue. Putting an unreachable!() there seems risky. So, bail out with an error. It will become a 500 status code, which console shall retry according to the openapi spec. We have test coverage that the retry can succeed.
pageserver/src/tenant.rs
Outdated
// FIXME: should this really fail the delete, probably not? | ||
// #[error("directory remove failed: {0:#}")] | ||
// TimelineDirectoryRemoveFailed(std::io::Error), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could restore this, std::io::Error
is not needed however to maintain Clone
ability. We can just log it after filtering for NotFound
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 this is now a label in InnerDeleteTimelineError.
…' into joonas/dkr/deleted-flag-in-remote-index-cont Conflict was because both added a comment on the persist_index_part_with_deleted_flag_after_set_before_upload_pause failoint.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- Can this request-coalescing, including MaybeDone, be abstracted away into a library?
- As remarked in various review comments, I don't think this is the silver-bullet to all the race conditions (=> e.g., race with detach not addressed)
I think we should definitely have this PR, but, I don't want to spend the time polishing it up today + waiting for another review round, etc.
I'll merge #4147 to fix the immediate issue with the deleted_at is already Some()
and rebase this PR by using
- revert of the 4147
- the current code of this PR
We can merge this PR some time next week.
I don't think we need it to release, but, let's have that discussion separately.
pageserver/src/tenant.rs
Outdated
fn is_permanent(&self) -> bool { | ||
use InnerDeleteTimelineError::*; | ||
|
||
match self { | ||
StopUploadQueue(_) => true, | ||
#[cfg(feature = "testing")] | ||
Failpoint(_) => false, | ||
UploadFailed => false, | ||
ChildAppearedAfterRemoveDir => true, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The child could go away again. So, we should be retrying these.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have already poisoned the Tenant::timelines
, so I don't see how or why.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 this is the same, tenant::timelines
is still being poisoned.
return Err(DeleteTimelineError::from(e)) | ||
} | ||
MaybeDone::Pending(rx) => { | ||
// important: upgrading and later resubscription happens while holding the lock |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
need to explain why it's important
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 there are internal correctness assertions but this has also been expanded for both the upgrade failure and more consistent naming around strong
and "strong must not leak".
The comments are in SharedRetryable::decide_to_retry_or_join
.
pub(super) delete_self: | ||
tokio::sync::Mutex<Option<MaybeDone<Result<(), super::InnerDeleteTimelineError>>>>, | ||
} | ||
|
||
/// MaybeDone handles synchronization for multiple requests and the single actual task. | ||
/// | ||
/// If request handlers witness `Pending` which they are able to upgrade, they are guaranteed a | ||
/// useful `recv().await`, where useful means "value" or "disconnect" arrives. If upgrade fails, | ||
/// this means that "disconnect" has happened in the past. | ||
/// | ||
/// On successful execution the one executing task will set this to `Done` variant, with the actual | ||
/// resulting value. | ||
#[derive(Debug)] | ||
pub(super) enum MaybeDone<V> { | ||
Pending(std::sync::Weak<tokio::sync::broadcast::Receiver<V>>), | ||
Done(V), | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not push the Option::None case of delete_self
into a third MaybeDone variant?
That way, there's no wondering about what delete_self: None
means.
Suggestion for the variant name: MaybeDone::NeverStarted
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 this was not changed. I don't think the benefit of pushing the None case into the enum gives much.
} | ||
.instrument(tracing::info_span!("unique_delete_timeline")) | ||
}); | ||
*g = Some(timeline::MaybeDone::Pending(Arc::downgrade(&rx))); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think for readability purposes, it would make sense to move this *g = ...
assignment upwards.
As close to the
if let Some(rx) = maybe_rx {
rx
} else {
let (tx, rx) = tokio::sync::broadcast::channel(1);
}
as possible.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Moved in #4194.
// subscribe while the mutex is held, so we are guaranteed to receive the message, | ||
// unless a panic comes. | ||
.resubscribe() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I haven't used tokio::sync::broadcast in a while. The resubscribe API has a weird name. I understand how they got there, but, when reading this function here top to bottom, the resubscribe is plain confusing, espectially because there is a comment above it, so, brain goes "oh this must be a special operation".
Feel free to ignore this suggestion if you feel it's required implicit knowledge.
// subscribe while the mutex is held, so we are guaranteed to receive the message, | |
// unless a panic comes. | |
.resubscribe() | |
// subscribe while the mutex is held, so we are guaranteed to receive the message, | |
// unless a panic comes. | |
.resubscribe() // NB: we would use Sender::subscribe, but, we don't have a Sender here |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think those apis are just the same, on receiver it's just called resubscribe. I always found the "sharing a sender to create receivers" weird, but of course the underlying impl allows for that. I think this weirdness is one of the reasons there is now a resubscribe
.
I don't think this comment is necessary, but if you find I've misunderstood resubscribe
semantics then perhaps we should change to sharing the sender.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
but if you find I've misunderstood resubscribe semantics then perhaps we should change to sharing the sender.
No, you got them right. It's just the weirdness of the name.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is "resubscribe" because the receiver in Weak<Receiver>
is by itself locked in to receive all messages due to it being created at the same time as sender at tokio::sync::broadcast::channel(..)
.
resubscribe
is not clone where clone would mean "cloned receiver sees messages from same point on as the original." clone is not offered because this would be surprising, I assume.
I think this is good to resolve.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 this required no changes.
// TODO: this could be tenant scoped task_mgr task? | ||
tokio::spawn({ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes it should be.
Otherwise, we'll not wait for this task, e.g., during pageserver shutdown / tenant detach.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done, at quite more horrible internal impl in #4194.
pageserver/src/tenant.rs
Outdated
if let Err(e) = std::fs::remove_dir_all(local_timeline_directory) { | ||
warn!("failed to remove timeline files after uploading tombstoned index_part.json: {e}"); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Unacceptable to treat failure to remove the files as a success case.
This is completely retryable.
See how I did it in #4147
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In which cases is it actually retryable? To support directory permission changes?
In general, I guess it could be done, I just failed to think of the retryable cases. No problem adding new variants.
Your pr could most likely target this branch though I haven't looked at after yesterday.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The problem is that we would call the delete_timeline operation as succeeded, but, there's actually still state on the local pageserver.
IMO that is an unacceptable outcome.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we have remote storage configured, we retry the delete upon pageserver restart, due to the is_deleted flag being present. If the delete fails again, the tenant transitions into Broken state.
So, we'd be ACKing the delete timeline as succeeded, but on next restart, the tenant is Broken.
Not an acceptable outcome.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
// we use panicking to poison the tenant if we have grown a new branch while deleting. | ||
// FIXME: don't allow branch_timeline while delete_timeline | ||
let res = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { | ||
// Remove the timeline from the map. | ||
let mut timelines = self.timelines.lock().unwrap(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
// we use panicking to poison the tenant if we have grown a new branch while deleting. | |
// FIXME: don't allow branch_timeline while delete_timeline | |
let res = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { | |
// Remove the timeline from the map. | |
let mut timelines = self.timelines.lock().unwrap(); | |
// we use panicking to poison the timeline if we have grown a new branch while deleting. | |
// FIXME: don't allow branch_timeline while delete_timeline | |
let res = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { | |
// Remove the timeline from the map. | |
let mut timelines = self.timelines.lock().unwrap(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. It's the tenants timelines, which is a std mutex over a hashmap or similar, not a single timeline. At least this was my understanding of the existing code.
Confirmed now. Breaking tenant might work, but unsure if we should do piecewise break introduction.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Right, it should say ... poison the Tenant::timelines mutex
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No comment in #4194 for this.
let mut timelines = self.timelines.lock().unwrap(); | ||
let children_exist = timelines | ||
.iter() | ||
.any(|(_, entry)| entry.get_ancestor_timeline_id() == Some(timeline.timeline_id)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Avoid the churn by introducing a timeline_id variable at the start of the function.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done in #4194.
// not going to continue unwind, just fail. | ||
// the panick has been already logged and reported, hopefully nothing got to start | ||
// with this new timeline. | ||
Err(InnerDeleteTimelineError::ChildAppearedAfterRemoveDir) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should explain here what happens on retry.
My understanding is we'll try to lock the ()self
) timeline lock, likely unwrap, and panic?
Also, suggestion on the comment to avoid confusion which timeline is being referred to:
// not going to continue unwind, just fail. | |
// the panick has been already logged and reported, hopefully nothing got to start | |
// with this new timeline. | |
Err(InnerDeleteTimelineError::ChildAppearedAfterRemoveDir) | |
// not going to continue unwind, just fail. | |
// the panick has been already logged and reported, hopefully nothing got to start | |
// with the child timeline. | |
Err(InnerDeleteTimelineError::ChildAppearedAfterRemoveDir) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a permanent error, should not be retried. Yes in general with a new callsite, if we would get to starting over and not panic at the HasChildren checks, we would get a poisonerror unwrap within this catch_unwind.
Unsure if this needs to be explained, because this should not be retried.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In #4194 this is now DeletedGrewChildren
, continues to be terminal state.
Walking up the call graph, there is a tenant_id set, but not timeline_id. Not sure why Dmitry removed the entire #[instrument] in the original code.
I need to start be more careful about scope creep because otherwise my PRs just end up closed because I don't want to use my free time to continue working on them. So no. If this approach would work on 2-3 different cases, then yes, maybe on the 3rd use.
This does not expand on what we were trying to solve so no magic exclusion with previous unexcluded parts should be here, only that the requests are not racing with each other and can be retried. Some earlier simplifications I made are still present like the timeline directory removal, that was before I figured out the two layer approach and not needing to report the anyhow error to the user. They don't need to be. |
…deleting the files (#4147) Before this patch, timeline delete was not retryable if it failed with an error earlier. Previously, - if we had failed after uploading the index part but before remove_dir_all, the retry would have failed because `IndexPart::deleted_at` is already `Some()`. - if we had failed after remove_dir_all, the retry would have failed because the directory doesn't exist. This PR adds a test case and fixes these two issues.
Doing this right now. |
…' into joonas/dkr/deleted-flag-in-remote-index-cont The two main conflicts: - They had introduced the distinguished error for `persist_index_part_with_deleted_flag` to detect the case where persisting the index part was already done. We solved this using the `is_permanent()` method. Choose our method. - They also added the distinguished enum for deleted_at state. Remove it, we don't need it in our method. - They had introduced handling for NotFound at the remove_dir_all call. We had changed that to a warn!(). Resolution: - Keep the NotFound handling - Replace the warn!() with another InnerDeleteTimelineError variant: TimelineDirectoryRemoveFailed - It can't hold the error because std::io::Error is not Clone. So, log it and keep the TimelineDirectoryRemoveFailed high-level
As part of the merge, we reverted the upstream change that adds a distinguished error type to persist_index_part_with_deleted_flag. That error type allowed the caller to detect the case where a prior call to the delete method already succeeded. That is important to support retrying deletes, as show by the test case test_delete_timeline_post_rm_failure which failed as of the merge. This patch brings back the distinguished error type & handling of it, thereby getting the test case green again. Original commit that introduced the distinguished error type: 1312cbc
3b18f79
to
6b28e1c
Compare
Finished the merge using following conflict resolution strategy (also in commit messages):
Then noticed the following problem (with this branch / the merge strategy) and addressed it in commit
|
/// resulting value. | ||
#[derive(Debug)] | ||
pub(super) enum MaybeDone<V> { | ||
Pending(std::sync::Weak<tokio::sync::broadcast::Receiver<V>>), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reconsidering the weakness here. As there's currently the upgrade failure => retry logic, it essentially means that a panic had happened, which should now be retried.
Perhaps it is all right, we cannot communicate the "previous attempt panicked" otherwise.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As there's currently the upgrade failure => retry logic, it essentially means that a panic had happened, which should now be retried.
I'm not following, can you elaborate? I don't get the connection why upgrade failure implies that a panic happened.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The #4194 test case for SharedRetry (cancelling_spawner_is_fine
) now should explain this out. The Weak
is still needed for this one case (panick or not-spawned-but-dropped).
My imaginary questions and answers. Q: Why is taking an async mutex lock for the duration of the request handling not enough? |
…ne resurrection (#3919) Before this patch, the following sequence would lead to the resurrection of a deleted timeline: - create timeline - wait for its index part to reach s3 - delete timeline - wait an arbitrary amount of time, including 0 seconds - detach tenant - attach tenant - the timeline is there and Active again This happens because we only kept track of the deletion in the tenant dir (by deleting the timeline dir) but not in S3. The solution is to turn the deleted timeline's IndexPart into a tombstone. The deletion status of the timeline is expressed in the `deleted_at: Option<NativeDateTime>` field of IndexPart. It's `None` while the timeline is alive and `Some(deletion time stamp)` if it is deleted. We change the timeline deletion handler to upload this tombstoned IndexPart. The handler does not return success if the upload fails. Coincidentally, this fixes the long-stanging TODO about the `std::fs::remove_dir_all` being not atomic. It need not be atomic anymore because we set the `deleted_at=Some()` before starting the `remove_dir_all`. The tombstone is in the IndexPart only, not in the `metadata`. So, we only have the tombstone and the `remove_dir_all` benefits mentioned above if remote storage is configured. This was a conscious trade-off because there's no good format evolution story for the current metadata file format. The introduction of this additional step into `delete_timeline` was painful because delete_timeline needs to be 1. cancel-safe 2. idempotent 3. safe to call concurrently These are mostly self-inflicted limitations that can be avoided by using request-coalescing. PR #4159 will do that. fixes #3560 refs #3889 (part of tenant relocation) Co-authored-by: Joonas Koivunen <[email protected]> Co-authored-by: Christian Schwarz <[email protected]>
Uses request coalescing and spawned task instead of the previous hopefully idempotent
delete_timeline
, which I don't think is possible, see: https://github.com/neondatabase/neon/pull/3919/files#r1185357135. Buiilds upon #4156.Originally I was thinking this should be done with:
tokio::sync::watch
but the disconnected event cannot be received unless already waiting for it -- unsuitable for "relaunching"futures::future::FutureExt::shared
around theJoinHandle
-- cannot do, requiresJoinError: Clone
MaybeDone<V>
which encapsulates theWeak<broadcast::Receiver<V>>
to ready shareable value transformationImportant point is that we don't try in a loop. We could, but right now the HasChildren check is only being done once.
Notable differences to before:
Stopping
remove_dir_all
-- just log itDefunct tests:
test_concurrent_timeline_delete_if_first_stuck_at_index_upload
-- always fails due to how the implementation has now changedtest_delete_timeline_client_hangup
-- flaky due to implementation changes