Skip to content

Conversation

@viirya
Copy link
Member

@viirya viirya commented Sep 8, 2023

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?

@github-actions github-actions bot added the core Core DataFusion crate label Sep 8, 2023
Ok(_) => continue,
// This means a blocking task error
Err(e) => {
return Some(internal_err!("Spawned Task error: {e}"));
Copy link
Member Author

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")

Copy link
Contributor

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.

Copy link
Contributor

@alamb alamb left a 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() {
Copy link
Contributor

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

Copy link
Member Author

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}"));
Copy link
Contributor

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.

@viirya viirya merged commit c3898cf into apache:main Sep 9, 2023
@viirya
Copy link
Member Author

viirya commented Sep 9, 2023

Thank you @alamb

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Core DataFusion crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Errors cannot be propagated from spawned task reading spills

2 participants