Skip to content

Conversation

@nathaniel-d-ef
Copy link
Contributor

Which issue does this PR close?

Rationale for this change

The arrow-avro crate currently uses ArrowError throughout. This lacks the level of precision other crates in the project, such as Parquet, have.

What changes are included in this PR?

  • A new AvroError enum
  • Application of AvroError on all internal methods where ArrowError was previously used. Errors on pub methods at the API boundary remain as ArrowError. A Result utility on the AvroError allows for Result<T, AvroError> to be written as Result.

Are these changes tested?

No new functionality has been introduced, all existing tests are passing.

Are there any user-facing changes?

There shouldn't be - the API signatures remain the same.

@github-actions github-actions bot added arrow Changes to the arrow crate arrow-avro arrow-avro crate labels Oct 31, 2025
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 @nathaniel-d-ef -- I think this makes sense to me, but it is a breaking API change and thus we will have to wait until the next major release in a few months:

FYI @jecsand838 and @mbrobbel

EOF(String),
/// Arrow error.
/// Returned when reading into arrow or writing from arrow.
ArrowError(String),
Copy link
Contributor

Choose a reason for hiding this comment

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

Could this instead keep the actual Arrow error rather than converting directly into a string?

Suggested change
ArrowError(String),
ArrowError(Box<ArrowError>)),

I realize that is what ParquetError::ArrowError does the same thing -- but I think we might want to change that Parquet error as well

}
}

impl From<cell::BorrowMutError> for AvroError {
Copy link
Contributor

Choose a reason for hiding this comment

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

this is a fairly specific conversion -- maybe it would be simpler to annotate the locations where this happens with map_err(|e| AvroError::From(Box::new(e)) 🤔

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Whoops, my mistake. This is unnecessary, I removed.

}
return out
.write_all(&src_be[extra..])
.map_err(|e| ArrowError::IoError(format!("write decimal fixed: {e}"), e));
Copy link
Contributor

Choose a reason for hiding this comment

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

This change does appear to lose some error context. Would it be better to keep the information that this came from write decimal fixed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, that's a good point. I adjusted this to use the General error and pass the contextual info.

Copy link
Member

@martin-g martin-g Nov 10, 2025

Choose a reason for hiding this comment

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

I don't think this addresses @alamb's concerns. Now e is string-ified and it losses the io::Error source. AFAIS only ::External and ::ArrowError preserve the source/cause (https://github.com/elastiflow/arrow-rs/blob/b40bb9c6b70ecefee8647e0c3dd9be77bc6ca2bf/arrow-avro/src/errors.rs#L54)

Copy link
Contributor

Choose a reason for hiding this comment

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

@nathaniel-d-ef This doesn't seem to be fully addressed imo. Several write_all(...) failures are still converted to AvroError::General(format!(...)), which drops the io::Error source entirely, recreating the original concern. This shows up specifically in write_sign_extended (padding writes) and write_optional_index.

@alamb alamb added api-change Changes to the arrow API next-major-release the PR has API changes and it waiting on the next major version labels Oct 31, 2025
@jecsand838
Copy link
Contributor

jecsand838 commented Oct 31, 2025

@nathaniel-d-ef I'll do a deeper review tonight, but if we plan to go this direction, should we also remove the AvroError variant from https://github.com/apache/arrow-rs/blob/main/arrow-schema/src/error.rs?

I never got around to wiring that up before public release. But the original intent was to align arrow-avro with arrow-csv and arrow-json.

CC: @alamb

@mbrobbel mbrobbel added this to the 58.0.0 milestone Nov 3, 2025
@nathaniel-d-ef
Copy link
Contributor Author

@nathaniel-d-ef I'll do a deeper review tonight, but if we plan to go this direction, should we also remove the AvroError variant from https://github.com/apache/arrow-rs/blob/main/arrow-schema/src/error.rs?

I never got around to wiring that up before public release. But the original intent was to align arrow-avro with arrow-csv and arrow-json.

CC: @alamb

I see what you mean. I don't feel strongly either way. On the whole we're probably unlikely to use all of the variants here, so if it's preferable to roll this back a bit and implement the ArrowError::AvroError instead, I'm happy to go that direction. I mainly didn't want it stuck somewhere in the middle. Your comment on the issue is a good one, perhaps Parquet is more of the odd one out.

}
return out
.write_all(&src_be[extra..])
.map_err(|e| ArrowError::IoError(format!("write decimal fixed: {e}"), e));
Copy link
Member

@martin-g martin-g Nov 10, 2025

Choose a reason for hiding this comment

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

I don't think this addresses @alamb's concerns. Now e is string-ified and it losses the io::Error source. AFAIS only ::External and ::ArrowError preserve the source/cause (https://github.com/elastiflow/arrow-rs/blob/b40bb9c6b70ecefee8647e0c3dd9be77bc6ca2bf/arrow-avro/src/errors.rs#L54)

@alamb
Copy link
Contributor

alamb commented Jan 9, 2026

main is now open for breaking API changes, if you would like to merge this one, we can do so.

I can't remember where we landed

@jecsand838
Copy link
Contributor

main is now open for breaking API changes, if you would like to merge this one, we can do so.

I can't remember where we landed

Not sure it was 100% decided.

My vote was to just use AvroError for now, but I'll support whichever direction @nathaniel-d-ef wants to go.

@nathaniel-d-ef
Copy link
Contributor Author

nathaniel-d-ef commented Jan 13, 2026

@jecsand838 I think the changes move things forward, so if you're good with them let's go ahead and merge. I should be able to clean up the conflicts in the next few days if no one beats me to it. I need to read up on what's changed; I've been out of the loop.

@alamb
Copy link
Contributor

alamb commented Jan 18, 2026

Sounds good -- I'll plan to merge the PR once it is ready

Copy link
Contributor

@jecsand838 jecsand838 left a comment

Choose a reason for hiding this comment

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

@nathaniel-d-ef

Just reviewed your latest changes. Overall this is looking really good and I love the idea of the custom AvroError enum. The major callouts in this review relate to:

  1. Whether we can return AvroError in the public api (like parquet).
  2. Fully addressing some previous comments.

Also it maybe a good idea to have @getChan take a look to ensure these changes work with the latest work in apache/datafusion#17861

decoder.decode(b"Ob").unwrap();
let err = decoder.decode(b"s").unwrap_err().to_string();
assert_eq!(err, "Parser error: Incorrect avro magic");
assert_eq!(err, "Parse error: Incorrect avro magic");
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there a reason we needed to change the test expectation 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.

No, the avro error implementation changed the phrase slightly. I fixed it there.

Comment on lines 181 to 183
.map_err(|e| ArrowError::IoError(format!("write decimal fixed: {e}"), e))?;
.map_err(|e| AvroError::General(format!("write decimal fixed: {e}")))?;
rem -= pad.len();
}
if rem > 0 {
out.write_all(&pad[..rem])
.map_err(|e| ArrowError::IoError(format!("write decimal fixed: {e}"), e))?;
.map_err(|e| AvroError::General(format!("write decimal fixed: {e}")))?;
}
out.write_all(src_be)
.map_err(|e| ArrowError::IoError(format!("write decimal fixed: {e}"), e))
.map_err(|e| AvroError::General(format!("write decimal fixed: {e}")))
Copy link
Contributor

Choose a reason for hiding this comment

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

We'll want to preserve the error context in these map_err 's as well imo.

magic: &[u8; MAGIC_LEN],
fingerprint_from: impl FnOnce([u8; N]) -> Fingerprint,
) -> Result<Option<usize>, ArrowError> {
) -> Result<Option<usize>> {
Copy link
Contributor

Choose a reason for hiding this comment

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

This maybe just me, but not specifying the error enum being passed back makes the code a bit trickier for me to grok. Especially now that we have multiple error enums.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Totally fair, I'm fine leaning more on the explicit side, it doesn't hurt. I'll adjust.

let batch = self.flush_and_reset();
self.apply_pending_schema();
batch
batch.map_err(ArrowError::from)
Copy link
Contributor

@jecsand838 jecsand838 Jan 23, 2026

Choose a reason for hiding this comment

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

I'm actually curious, why are we not following the pattern used by parquet and just return the pub AvroError? I thought that was the big reason for doing this PR and waiting for a major release?

I re-read the PR description and just caught this part, not sure I fully agree. I'm thinking we should either return AvroError or ArrowError::AvroError as part of the public API to align with the other crates and help downstream callers (if possible ofc).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's been a while since I kicked off this work, but I think it was a misinterpretation of the ask. I took another look at this and revised it.

}

/// A specialized `Result` for Avro errors.
pub type Result<T, E = AvroError> = result::Result<T, E>;
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm thinking we should remove this imo. From a reviewer perspective, the custom Result caused some confusion in correctly understanding the diff.

Suggested change
pub type Result<T, E = AvroError> = result::Result<T, E>;

return Err(ArrowError::ParseError(
"Unexpected EOF reading float".to_string(),
));
return Err(AvroError::EOF("Unexpected EOF reading float".to_string()));
Copy link
Contributor

Choose a reason for hiding this comment

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

I know this was pre-existing, but maybe worth taking the chance to correct?

Suggested change
return Err(AvroError::EOF("Unexpected EOF reading float".to_string()));
return Err(AvroError::EOF("Unexpected EOF reading double".to_string()));

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh good catch, yes

#[derive(Debug)]
#[non_exhaustive]
pub enum AvroError {
Copy link
Contributor

Choose a reason for hiding this comment

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

If these errors are internal only, then why make this pub?

My preference would be to return pub if possible (like in the parquet crate), but if we can't shouldn't we make this pub(crate)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Left this alone having changed the boundary type.

pub mod codec;

/// AvroError variants
pub mod errors;
Copy link
Contributor

Choose a reason for hiding this comment

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

Same here, if internal only, then should we make this change?

Suggested change
pub mod errors;
pub(crate) mod errors;

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See above

let writer_schema = hdr
.schema()
.map_err(|e| ArrowError::ExternalError(Box::new(e)))?
.map_err(|e| AvroError::External(Box::new(e)))?
Copy link
Contributor

Choose a reason for hiding this comment

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

Because schema returns Result<Option<Schema<'_>>> // Result<_, AvroError> (the custom Result made this less readable for me), the closure |e| AvroError::External(Box::new(e)) is still wrapping an AvroError inside AvroError::External.

I think @martin-g called this out in another comment as well.

You can probably improve this just by something like this:

            let writer_schema = hdr
                .schema()?
                .ok_or_else(|| AvroError::ParseError("No Avro schema present in file header".into()))?;

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

Labels

api-change Changes to the arrow API arrow Changes to the arrow crate arrow-avro arrow-avro crate next-major-release the PR has API changes and it waiting on the next major version

Projects

None yet

Development

Successfully merging this pull request may close these issues.

AvroError enum for arrow-avro crate

5 participants