-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Propagate error from spawned task reading spills #7510
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
Conversation
| Ok(_) => continue, | ||
| // This means a blocking task error | ||
| Err(e) => { | ||
| return Some(internal_err!("Spawned Task error: {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.
For example, after this, if the file doesn't exist, you will get an error:
Error: Internal("Spawned Task error: Execution error: Spill file \"/var/folders/zq/2tdnn5955wvdcw7qfy6qhvk00000gn/T/.tmpb5EGzS/.tmptpkruZ\" does not exist")
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.
ExecutionError might be more appropriate than InternalError given an error in a spill file is likely not a bug in DataFusion, but rather some issue in the environment.
alamb
left a comment
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.
Thanks @viirya -- this looks great. The only thing I am not quite sure about is erroring if a message can't be sent via channel.
|
|
||
| builder.spawn(async move { | ||
| while let Some(item) = input.next().await { | ||
| if sender.send(item).await.is_err() { |
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 only reason an item can't be sent via a channel I think is if the other end (the Receiver) was dropped. This can certainly happen if some other part of the query errors, but I also think it can happen with the plan shuts down early due to a LIMIT or something similar.
Thus I am not sure we should propagate an error in this case
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.
Makes sense. Removed the error.
| Ok(_) => continue, | ||
| // This means a blocking task error | ||
| Err(e) => { | ||
| return Some(internal_err!("Spawned Task error: {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.
ExecutionError might be more appropriate than InternalError given an error in a spill file is likely not a bug in DataFusion, but rather some issue in the environment.
|
Thank you @alamb |
Which issue does this PR close?
Closes #7509.
Rationale for this change
Any error happened in the spawned task reading spills cannot propagate out currently. So these errors will be ignored and it doesn't look correct as the sorted result will be incorrect.
What changes are included in this PR?
Propagating errors from spawned task reading spills.
Are these changes tested?
Are there any user-facing changes?