alamb commented on code in PR #9613:
URL: https://github.com/apache/arrow-datafusion/pull/9613#discussion_r1528279784
##########
datafusion/common/src/scalar/mod.rs:
##########
@@ -1650,7 +1650,11 @@ impl ScalarValue {
| DataType::Duration(_)
| DataType::Union(_, _)
| DataType::Map(_, _)
- | DataType::RunEndEncoded(_, _) => {
+ | DataType::RunEndEncoded(_, _)
+ | DataType::Utf8View
+ | DataType::BinaryView
+ | DataType::ListView(_)
+ | DataType::LargeListView(_) => {
Review Comment:
🎉
##########
datafusion/common/src/file_options/parquet_writer.rs:
##########
@@ -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)]
Review Comment:
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?
##########
datafusion-examples/examples/deserialize_to_struct.rs:
##########
@@ -15,61 +15,61 @@
// specific language governing permissions and limitations
// under the License.
+use arrow::array::AsArray;
+use arrow::datatypes::{Float64Type, Int32Type};
use datafusion::error::Result;
use datafusion::prelude::*;
-use serde::Deserialize;
+use futures::StreamExt;
/// 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.
-/// Then, serde_json::Value is turned into Rust's struct.
-/// Any datatype with `Deserialize` implemeneted works.
#[tokio::main]
async fn main() -> Result<()> {
let data_list = Data::new().await?;
println!("{data_list:#?}");
Ok(())
}
-#[derive(Deserialize, Debug)]
+#[derive(Debug)]
struct Data {
#[allow(dead_code)]
- int_col: i64,
+ int_col: i32,
#[allow(dead_code)]
double_col: f64,
}
impl Data {
pub async fn new() -> Result<Vec<Self>> {
// this group is almost the same as the one you find it in
parquet_sql.rs
- let batches = {
- let ctx = SessionContext::new();
+ let ctx = SessionContext::new();
- let testdata = datafusion::test_util::parquet_test_data();
+ let testdata = datafusion::test_util::parquet_test_data();
- ctx.register_parquet(
- "alltypes_plain",
- &format!("{testdata}/alltypes_plain.parquet"),
- ParquetReadOptions::default(),
- )
- .await?;
+ ctx.register_parquet(
+ "alltypes_plain",
+ &format!("{testdata}/alltypes_plain.parquet"),
+ ParquetReadOptions::default(),
+ )
+ .await?;
- let df = ctx
- .sql("SELECT int_col, double_col FROM alltypes_plain")
- .await?;
+ let df = ctx
+ .sql("SELECT int_col, double_col FROM alltypes_plain")
+ .await?;
- df.clone().show().await?;
+ df.clone().show().await?;
- df.collect().await?
- };
- let batches: Vec<_> = batches.iter().collect();
+ let mut stream = df.execute_stream().await?;
+ let mut list = vec![];
+ while let Some(b) = stream.next().await.transpose()? {
+ let int_col = b.column(0).as_primitive::<Int32Type>();
+ let float_col = b.column(1).as_primitive::<Float64Type>();
- // converts it to serde_json type and then convert that into Rust type
Review Comment:
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
##########
datafusion/common/src/scalar/mod.rs:
##########
@@ -5769,7 +5773,7 @@ mod tests {
let batch = RecordBatch::try_from_iter(vec![("s", arr as _)]).unwrap();
#[rustfmt::skip]
- let expected = [
+ let expected = [
Review Comment:
This whitespace change seems unnecessary
##########
datafusion/core/src/datasource/file_format/parquet.rs:
##########
@@ -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;
Review Comment:
Due to https://github.com/apache/arrow-rs/pull/5485
##########
datafusion/functions/src/datetime/date_part.rs:
##########
@@ -157,28 +114,28 @@ impl ScalarUDFImpl for DatePartFunc {
ColumnarValue::Scalar(scalar) => scalar.to_array()?,
};
- let arr = match date_part.to_lowercase().as_str() {
- "year" => extract_date_part!(&array, temporal::year),
- "quarter" => extract_date_part!(&array, temporal::quarter),
- "month" => extract_date_part!(&array, temporal::month),
- "week" => extract_date_part!(&array, temporal::week),
- "day" => extract_date_part!(&array, temporal::day),
- "doy" => extract_date_part!(&array, temporal::doy),
- "dow" => extract_date_part!(&array,
temporal::num_days_from_sunday),
- "hour" => extract_date_part!(&array, temporal::hour),
- "minute" => extract_date_part!(&array, temporal::minute),
- "second" => extract_date_part!(&array, seconds),
- "millisecond" => extract_date_part!(&array, millis),
- "microsecond" => extract_date_part!(&array, micros),
- "nanosecond" => extract_date_part!(&array, nanos),
- "epoch" => extract_date_part!(&array, epoch),
- _ => exec_err!("Date part '{date_part}' not supported"),
- }?;
+ let arr = match part.to_lowercase().as_str() {
Review Comment:
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 https://github.com/apache/arrow-rs/pull/5319
and a cleanup of the code to use the `unary` kernel more effectively (the
changes look good to me)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]