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

Update Arrow/Parquet to 51.0.0, tonic to 0.11 #9613

Merged
merged 11 commits into from
Mar 19, 2024

Conversation

tustvold
Copy link
Contributor

Which issue does this PR close?

Closes #.

Rationale for this change

What changes are included in this PR?

Are these changes tested?

Are there any user-facing changes?

@github-actions github-actions bot added the core Core DataFusion crate label Mar 14, 2024
@github-actions github-actions bot added the sql SQL Planner label Mar 14, 2024
@tustvold tustvold changed the title Prepare for arrow 51 Update Arrow 51.0.0 Mar 18, 2024

/// This example shows that it is possible to convert query results into Rust structs .
/// It will collect the query results into RecordBatch, then convert it to serde_json::Value.
Copy link
Contributor Author

Choose a reason for hiding this comment

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

apache/arrow-rs#5318 deprecated the serde_json based APIs

@@ -156,6 +156,7 @@ pub(crate) fn parse_encoding_string(
"plain" => Ok(parquet::basic::Encoding::PLAIN),
"plain_dictionary" => Ok(parquet::basic::Encoding::PLAIN_DICTIONARY),
"rle" => Ok(parquet::basic::Encoding::RLE),
#[allow(deprecated)]
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't understand the reference (to the JSON writer) when this is for parquet encoding. Is there some other encoding/compression scheme that was deprecated too?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a copypasta meant to link to apache/arrow-rs#5348

"epoch" => extract_date_part!(&array, epoch),
_ => exec_err!("Date part '{date_part}' not supported"),
}?;
let arr = match part.to_lowercase().as_str() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

As above, I don't understand the reference to the JSON writer PR

The changes in this module look more like switching to use the date_part kernels that @Jefffrey added in apache/arrow-rs#5319 and a cleanup of the code to use the unary kernel more effectively (the changes look good to me)

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 this was a copypasta

@tustvold tustvold marked this pull request as ready for review March 18, 2024 08:01
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.

Thank you for the contribution @tustvold -- I think this looks good overall (though I didn't understand several ticket references)

I also took the liberty of merging this PR up from main and running cargo update in datafusion-cli to get a clean CI run


// converts it to serde_json type and then convert that into Rust type
Copy link
Contributor

Choose a reason for hiding this comment

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

I do think showing how to use serde to convert arrow --> rust structs is important. While I am well aware its performance is not good, the serde concept is widely understood and supported in the rust Ecosystem.

Is there any API that can do serde into Rust structs in the core arrow crates anymore?

If not, perhaps we can point in comments at a crate like https://github.com/chmp/serde_arrow (or bring an example that parses the JSON back to Json::Value and then serde's)

We/I can do this as a follow on PR

Copy link
Contributor Author

@tustvold tustvold Mar 18, 2024

Choose a reason for hiding this comment

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

You can serialize to JSON and parse it, but I would rather encourage people towards the performant way of doing things

Is there any API that can do serde into Rust structs in the core arrow crates anymore?

I'd dispute that we ever really had a way to do this, going via serde_json::Value is more of a hack than anything else. Serializing to a JSON string and back will likely be faster

Copy link
Contributor

Choose a reason for hiding this comment

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

I'd dispute that we ever really had a way to do this, going via serde_json::Value is more of a hack than anything else. Serializing to a JSON string and back will likely be faster

The key thing in my mind is to make it easy / quick for new users to get something working quickly. I am well aware that custom array -> struct will be the fastest performance, but I think it takes non trivial expertise in manipulating the arrow-rs API (especially when it comes to StructArray and ListArray) -- so offering them a fast way to get started with a slower API is important I think

Copy link
Contributor

Choose a reason for hiding this comment

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

I think since this is an example, we can always update / improve it as a follow on PR

@@ -156,6 +156,7 @@ pub(crate) fn parse_encoding_string(
"plain" => Ok(parquet::basic::Encoding::PLAIN),
"plain_dictionary" => Ok(parquet::basic::Encoding::PLAIN_DICTIONARY),
"rle" => Ok(parquet::basic::Encoding::RLE),
#[allow(deprecated)]
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't understand the reference (to the JSON writer) when this is for parquet encoding. Is there some other encoding/compression scheme that was deprecated too?

@@ -5769,7 +5773,7 @@ mod tests {
let batch = RecordBatch::try_from_iter(vec![("s", arr as _)]).unwrap();

#[rustfmt::skip]
let expected = [
let expected = [
Copy link
Contributor

Choose a reason for hiding this comment

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

This whitespace change seems unnecessary

| DataType::Utf8View
| DataType::BinaryView
| DataType::ListView(_)
| DataType::LargeListView(_) => {
Copy link
Contributor

Choose a reason for hiding this comment

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

🎉

@@ -78,9 +78,6 @@ use hashbrown::HashMap;
use object_store::path::Path;
use object_store::{ObjectMeta, ObjectStore};

/// Size of the buffer for [`AsyncArrowWriter`].
const PARQUET_WRITER_BUFFER_SIZE: usize = 10485760;
Copy link
Contributor

Choose a reason for hiding this comment

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

"epoch" => extract_date_part!(&array, epoch),
_ => exec_err!("Date part '{date_part}' not supported"),
}?;
let arr = match part.to_lowercase().as_str() {
Copy link
Contributor

Choose a reason for hiding this comment

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

As above, I don't understand the reference to the JSON writer PR

The changes in this module look more like switching to use the date_part kernels that @Jefffrey added in apache/arrow-rs#5319 and a cleanup of the code to use the unary kernel more effectively (the changes look good to me)

@alamb
Copy link
Contributor

alamb commented Mar 18, 2024

Let's leave this one open for another day or two so there is at least one work day for for people to comment

@tustvold
Copy link
Contributor Author

Apologies it was late and it would appear rather than linking to the appropriate tickets I just repeatedly linked to the same one

Copy link
Contributor

@Jefffrey Jefffrey left a comment

Choose a reason for hiding this comment

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

👍

@@ -75,6 +75,6 @@ serde_json = { workspace = true }
tempfile = { workspace = true }
tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] }
# 0.10 and 0.11 are incompatible. Need to upgrade tonic to 0.11 when upgrading to arrow 51
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
# 0.10 and 0.11 are incompatible. Need to upgrade tonic to 0.11 when upgrading to arrow 51

No longer need this comment

Copy link
Contributor

Choose a reason for hiding this comment

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

Removed in 456e2fe

Comment on lines +152 to +160
fn seconds(array: &dyn Array, unit: TimeUnit) -> Result<ArrayRef> {
let sf = match unit {
Second => 1_f64,
Millisecond => 1_000_f64,
Microsecond => 1_000_000_f64,
Nanosecond => 1_000_000_000_f64,
};
let secs = date_part(array, DatePart::Second)?;
let secs = as_int32_array(secs.as_ref())?;
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it worth making a note somewhere here that array must be a PrimitiveArray? Otherwise the as_int32_array() can panic if it is a dictionary, and maybe want to make this clear to anyone viewing the code hoping to make it work for dictionary in the future?

(Previous code encoded this by having array be a &PrimitiveArray<T>)

Copy link
Contributor

Choose a reason for hiding this comment

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

Added in f550b64

@leoyvens leoyvens mentioned this pull request Mar 19, 2024
8 tasks
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.

I merged up from main to resolve a conflict and addressed comments. I plan to merge this in later today


// converts it to serde_json type and then convert that into Rust type
Copy link
Contributor

Choose a reason for hiding this comment

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

I think since this is an example, we can always update / improve it as a follow on PR

Comment on lines +152 to +160
fn seconds(array: &dyn Array, unit: TimeUnit) -> Result<ArrayRef> {
let sf = match unit {
Second => 1_f64,
Millisecond => 1_000_f64,
Microsecond => 1_000_000_f64,
Nanosecond => 1_000_000_000_f64,
};
let secs = date_part(array, DatePart::Second)?;
let secs = as_int32_array(secs.as_ref())?;
Copy link
Contributor

Choose a reason for hiding this comment

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

Added in f550b64

@@ -75,6 +75,6 @@ serde_json = { workspace = true }
tempfile = { workspace = true }
tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] }
# 0.10 and 0.11 are incompatible. Need to upgrade tonic to 0.11 when upgrading to arrow 51
Copy link
Contributor

Choose a reason for hiding this comment

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

Removed in 456e2fe

@alamb alamb changed the title Update Arrow 51.0.0 Update Arrow/Paruqet to 51.0.0, tonic to 0.11 Mar 19, 2024
@alamb alamb changed the title Update Arrow/Paruqet to 51.0.0, tonic to 0.11 Update Arrow/Parquet to 51.0.0, tonic to 0.11 Mar 19, 2024
@alamb
Copy link
Contributor

alamb commented Mar 19, 2024

🚀

@alamb alamb merged commit 7af69f9 into apache:main Mar 19, 2024
26 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Core DataFusion crate sql SQL Planner
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants