-
Notifications
You must be signed in to change notification settings - Fork 1.8k
fix: Substrait serializer clippy error: not calling truncate #14723
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
Changes from all commits
d9bcf6c
3696ff7
2b07ec4
5ce1ec8
13182cd
907b6ce
7cb405a
398d8e4
f7d2053
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -22,42 +22,59 @@ use datafusion::error::Result; | |||||||||
| use datafusion::prelude::*; | ||||||||||
|
|
||||||||||
| use prost::Message; | ||||||||||
| use std::path::Path; | ||||||||||
| use substrait::proto::Plan; | ||||||||||
| use tokio::{ | ||||||||||
| fs::OpenOptions, | ||||||||||
| io::{AsyncReadExt, AsyncWriteExt}, | ||||||||||
| }; | ||||||||||
|
|
||||||||||
| use std::fs::OpenOptions; | ||||||||||
| use std::io::{Read, Write}; | ||||||||||
| /// Plans a sql and serializes the generated logical plan to bytes. | ||||||||||
| /// The bytes are then written into a file at `path`. | ||||||||||
| /// | ||||||||||
| /// Returns an error if the file already exists. | ||||||||||
| pub async fn serialize( | ||||||||||
| sql: &str, | ||||||||||
| ctx: &SessionContext, | ||||||||||
| path: impl AsRef<Path>, | ||||||||||
| ) -> Result<()> { | ||||||||||
| let protobuf_out = serialize_bytes(sql, ctx).await?; | ||||||||||
|
|
||||||||||
| #[allow(clippy::suspicious_open_options)] | ||||||||||
| pub async fn serialize(sql: &str, ctx: &SessionContext, path: &str) -> Result<()> { | ||||||||||
| let protobuf_out = serialize_bytes(sql, ctx).await; | ||||||||||
| let mut file = OpenOptions::new().create(true).write(true).open(path)?; | ||||||||||
| file.write_all(&protobuf_out?)?; | ||||||||||
| let mut file = OpenOptions::new() | ||||||||||
| .write(true) | ||||||||||
| .create_new(true) | ||||||||||
| .open(path) | ||||||||||
| .await?; | ||||||||||
| file.write_all(&protobuf_out).await?; | ||||||||||
| Ok(()) | ||||||||||
| } | ||||||||||
|
|
||||||||||
| /// Plans a sql and serializes the generated logical plan to bytes. | ||||||||||
| pub async fn serialize_bytes(sql: &str, ctx: &SessionContext) -> Result<Vec<u8>> { | ||||||||||
| let df = ctx.sql(sql).await?; | ||||||||||
| let plan = df.into_optimized_plan()?; | ||||||||||
| let proto = producer::to_substrait_plan(&plan, &ctx.state())?; | ||||||||||
|
|
||||||||||
| let mut protobuf_out = Vec::<u8>::new(); | ||||||||||
| proto.encode(&mut protobuf_out).map_err(|e| { | ||||||||||
| DataFusionError::Substrait(format!("Failed to encode substrait plan: {e}")) | ||||||||||
| })?; | ||||||||||
| proto | ||||||||||
| .encode(&mut protobuf_out) | ||||||||||
| .map_err(|e| DataFusionError::Substrait(format!("Failed to encode plan: {e}")))?; | ||||||||||
| Ok(protobuf_out) | ||||||||||
| } | ||||||||||
|
|
||||||||||
| pub async fn deserialize(path: &str) -> Result<Box<Plan>> { | ||||||||||
| /// Reads the file at `path` and deserializes a plan from the bytes. | ||||||||||
| pub async fn deserialize(path: impl AsRef<Path>) -> Result<Box<Plan>> { | ||||||||||
|
Comment on lines
+65
to
+66
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Follow-up suggestion (breaking change): this function is marked
Suggested change
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I plan to contribute more to the datafusion-substrait crate. Personally, I would like to reserve the breaking change in one of the following PR. What do you think? @alamb
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agree -- let's discuss API changes in follow on PRs / issues |
||||||||||
| let mut protobuf_in = Vec::<u8>::new(); | ||||||||||
|
|
||||||||||
| let mut file = OpenOptions::new().read(true).open(path)?; | ||||||||||
| let mut file = OpenOptions::new().read(true).open(path).await?; | ||||||||||
| file.read_to_end(&mut protobuf_in).await?; | ||||||||||
|
|
||||||||||
| file.read_to_end(&mut protobuf_in)?; | ||||||||||
| deserialize_bytes(protobuf_in).await | ||||||||||
| } | ||||||||||
|
|
||||||||||
| /// Deserializes a plan from the bytes. | ||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Follow-up suggestion (breaking change): this function is marked |
||||||||||
| pub async fn deserialize_bytes(proto_bytes: Vec<u8>) -> Result<Box<Plan>> { | ||||||||||
| Ok(Box::new(Message::decode(&*proto_bytes).map_err(|e| { | ||||||||||
| DataFusionError::Substrait(format!("Failed to decode substrait plan: {e}")) | ||||||||||
| DataFusionError::Substrait(format!("Failed to decode plan: {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.
If we replace the
sqlarg with aDataFramearg, this function (andserialize) can be non-async.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.
Not sure which one is preferable for users.
@alamb cc
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 we should leave this one as a
strin this PR and we can potentially add aDataFramespecific one as a follow onHowever, given there is already an API to serialize
LogicalPlansdirectly I am not sure how much more value a DataFrame one would addhttps://docs.rs/datafusion-substrait/latest/datafusion_substrait/#example-serializing-logicalplans