tustvold commented on code in PR #3103:
URL: https://github.com/apache/arrow-rs/pull/3103#discussion_r1022249268
##########
parquet/src/file/metadata.rs:
##########
@@ -353,6 +377,15 @@ impl RowGroupMetaDataBuilder {
self
}
+ /// Sets the sorting order for columns
+ pub fn set_sorting_columns(
+ mut self,
+ value: Option<&Vec<SortingColumnMetaData>>,
+ ) -> Self {
+ self.sorting_columns = value.cloned();
Review Comment:
```suggestion
value: Option<Vec<SortingColumnMetaData>>,
) -> Self {
self.sorting_columns = value;
```
Passing in a reference only to copy it is a little unusual
##########
parquet/tests/parquet_round_trip.rs:
##########
@@ -0,0 +1,100 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use parquet::data_type::BoolType;
+use parquet::file::metadata::SortingColumnMetaData;
+use parquet::file::properties::WriterProperties;
+use parquet::file::reader::FileReader;
+use parquet::file::serialized_reader::SerializedFileReader;
+use parquet::file::writer::SerializedFileWriter;
+use parquet::schema::parser::parse_message_type;
+use std::fs;
+use std::thread;
+use std::time::Duration;
+use std::{path::Path, sync::mpsc, sync::Arc};
+
+#[test]
+fn parquet_sorting_column_round_trip() {
+ let path = Path::new("it_writes_data_with_sorting_columns.parquet");
+
+ let message_type = "
+ message BooleanType {
+ REQUIRED BOOLEAN DIM0;
+ }
+";
+ let schema = Arc::new(parse_message_type(message_type).expect("parse
schema"));
+ let expected_result = Some(vec![SortingColumnMetaData {
+ column_index: 0,
+ descending: false,
+ nulls_first: true,
+ }]);
+ let props = Arc::new(
+ WriterProperties::builder()
+ .set_sorting_columns(expected_result.clone())
+ .build(),
+ );
+ let file = fs::File::create(path).expect("create file");
+ let mut writer =
+ SerializedFileWriter::new(file, schema, props).expect("create parquet
writer");
+ for _group in 0..1 {
+ let mut row_group_writer = writer.next_row_group().expect("get row
group writer");
+ let values: Vec<i64> = vec![0; 2049];
+ let my_bool_values: Vec<bool> = values
+ .iter()
+ .enumerate()
+ .map(|(count, _x)| count % 2 == 0)
+ .collect();
+ while let Some(mut col_writer) =
+ row_group_writer.next_column().expect("next column")
+ {
+ col_writer
+ .typed::<BoolType>()
+ .write_batch(&my_bool_values, None, None)
+ .expect("writing bool column");
+
+ col_writer.close().expect("close column");
+ }
+ let rg_md = row_group_writer.close().expect("close row group");
+ println!("total rows written: {}", rg_md.num_rows());
+ }
+ writer.close().expect("close writer");
+
+ // Now that we have written our data and are happy with it, make
+ // sure we can read it back in < 5 seconds...
Review Comment:
I'm not sure about this, timing tests are notoriously flaky... Perhaps we
could just have a unit test along the same lines of
`test_file_writer_with_metadata`.
##########
parquet/tests/parquet_round_trip.rs:
##########
@@ -0,0 +1,100 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use parquet::data_type::BoolType;
+use parquet::file::metadata::SortingColumnMetaData;
+use parquet::file::properties::WriterProperties;
+use parquet::file::reader::FileReader;
+use parquet::file::serialized_reader::SerializedFileReader;
+use parquet::file::writer::SerializedFileWriter;
+use parquet::schema::parser::parse_message_type;
+use std::fs;
+use std::thread;
+use std::time::Duration;
+use std::{path::Path, sync::mpsc, sync::Arc};
+
+#[test]
+fn parquet_sorting_column_round_trip() {
+ let path = Path::new("it_writes_data_with_sorting_columns.parquet");
Review Comment:
I think this should either use `tempfile::tempfile` or possibly even just
`Vec<u8>`. Otherwise it will potentially leave files in the working directory,
if the test is aborted or fails
##########
parquet/src/file/metadata.rs:
##########
@@ -940,6 +974,38 @@ impl OffsetIndexBuilder {
}
}
+/// Metadata for sorting order for columns
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct SortingColumnMetaData {
+ /// The column index (in this row group) *
+ pub column_index: i32,
+ /// If true, indicates this column is sorted in descending order. *
+ pub descending: bool,
+ /// If true, nulls will come before non-null values, otherwise,
+ /// nulls go at the end.
+ pub nulls_first: bool,
+}
+
+impl From<&SortingColumn> for SortingColumnMetaData {
+ fn from(sorting_column: &SortingColumn) -> Self {
+ SortingColumnMetaData {
Review Comment:
```suggestion
impl From<SortingColumn> for SortingColumnMetaData {
fn from(sorting_column: SortingColumn) -> Self {
SortingColumnMetaData {
```
As these are copy types
##########
parquet/src/file/metadata.rs:
##########
@@ -940,6 +974,38 @@ impl OffsetIndexBuilder {
}
}
+/// Metadata for sorting order for columns
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct SortingColumnMetaData {
+ /// The column index (in this row group) *
+ pub column_index: i32,
+ /// If true, indicates this column is sorted in descending order. *
+ pub descending: bool,
+ /// If true, nulls will come before non-null values, otherwise,
+ /// nulls go at the end.
+ pub nulls_first: bool,
+}
+
+impl From<&SortingColumn> for SortingColumnMetaData {
+ fn from(sorting_column: &SortingColumn) -> Self {
+ SortingColumnMetaData {
+ column_index: sorting_column.column_idx,
+ descending: sorting_column.descending,
+ nulls_first: sorting_column.nulls_first,
+ }
+ }
+}
+
+impl From<&SortingColumnMetaData> for SortingColumn {
+ fn from(sorting_column: &SortingColumnMetaData) -> SortingColumn {
+ SortingColumn {
Review Comment:
```suggestion
impl From<SortingColumnMetaData> for SortingColumn {
fn from(sorting_column: SortingColumnMetaData) -> SortingColumn {
SortingColumn {
```
##########
parquet/tests/parquet_round_trip.rs:
##########
@@ -0,0 +1,100 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use parquet::data_type::BoolType;
+use parquet::file::metadata::SortingColumnMetaData;
+use parquet::file::properties::WriterProperties;
+use parquet::file::reader::FileReader;
+use parquet::file::serialized_reader::SerializedFileReader;
+use parquet::file::writer::SerializedFileWriter;
+use parquet::schema::parser::parse_message_type;
+use std::fs;
+use std::thread;
+use std::time::Duration;
+use std::{path::Path, sync::mpsc, sync::Arc};
+
+#[test]
+fn parquet_sorting_column_round_trip() {
+ let path = Path::new("it_writes_data_with_sorting_columns.parquet");
+
+ let message_type = "
+ message BooleanType {
+ REQUIRED BOOLEAN DIM0;
+ }
+";
+ let schema = Arc::new(parse_message_type(message_type).expect("parse
schema"));
+ let expected_result = Some(vec![SortingColumnMetaData {
Review Comment:
Could we test multiple columns
##########
parquet/src/file/metadata.rs:
##########
@@ -940,6 +974,48 @@ impl OffsetIndexBuilder {
}
}
+/// Metadata for sorting order for columns
+#[derive(Debug, Clone, PartialEq, Eq)]
+pub struct SortingColumnMetaData {
Review Comment:
That is because they are actually metadata, i.e. RowGroupMetaData is
actually the metadata for the RowGroup, the thrift name is a little strange
imo. Calling this `SortColumn` is consistent with `Statistics`, `Encoding`,
`ConvertedType`,`LogicalType`, `PageType` etc...
##########
parquet/src/file/metadata.rs:
##########
@@ -303,9 +309,18 @@ impl RowGroupMetaData {
let cc = ColumnChunkMetaData::from_thrift(d.clone(), c)?;
columns.push(cc);
}
+ let sorting_columns: Option<Vec<SortingColumnMetaData>> = match
rg.sorting_columns
Review Comment:
```suggestion
let sorting_columns = match rg.sorting_columns
```
You shouldn't need to type hint this
##########
parquet/tests/parquet_round_trip.rs:
##########
@@ -0,0 +1,100 @@
+// Licensed to the Apache Software Foundation (ASF) under one
Review Comment:
I realise I asked for an integration test, which I appreciate in hindsight
was an ambiguous request. I simply meant a module level test like
test_file_writer_with_metadata that tests the integration of the various
components.
Top-level integration tests are fairly heavy, they are separate binary
targets, and best used for specific cases.
##########
parquet/src/file/metadata.rs:
##########
@@ -303,9 +309,18 @@ impl RowGroupMetaData {
let cc = ColumnChunkMetaData::from_thrift(d.clone(), c)?;
columns.push(cc);
}
+ let sorting_columns: Option<Vec<SortingColumnMetaData>> = match
rg.sorting_columns
+ {
+ Some(sorting_columns) => {
+ let result = sorting_columns.iter().map(Into::into).collect();
Review Comment:
```suggestion
let result =
sorting_columns.iter().copied().map(Into::into).collect();
```
To match suggested changes to From
--
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]