alamb commented on code in PR #6000:
URL: https://github.com/apache/arrow-rs/pull/6000#discussion_r1676807214
##########
parquet/src/file/writer.rs:
##########
@@ -301,93 +277,36 @@ impl<W: Write + Send> SerializedFileWriter<W> {
Ok(())
}
- /// Serialize all the column index to the file
- fn write_column_indexes(&mut self, row_groups: &mut [RowGroup]) ->
Result<()> {
- // iter row group
- // iter each column
- // write column index to the file
- for (row_group_idx, row_group) in row_groups.iter_mut().enumerate() {
- for (column_idx, column_metadata) in
row_group.columns.iter_mut().enumerate() {
- match &self.column_indexes[row_group_idx][column_idx] {
- Some(column_index) => {
- let start_offset = self.buf.bytes_written();
- let mut protocol = TCompactOutputProtocol::new(&mut
self.buf);
- column_index.write_to_out_protocol(&mut protocol)?;
- let end_offset = self.buf.bytes_written();
- // set offset and index for offset index
- column_metadata.column_index_offset =
Some(start_offset as i64);
- column_metadata.column_index_length =
- Some((end_offset - start_offset) as i32);
- }
- None => {}
- }
- }
- }
- Ok(())
- }
-
/// Assembles and writes metadata at the end of the file.
fn write_metadata(&mut self) -> Result<parquet::FileMetaData> {
self.finished = true;
- let num_rows = self.row_groups.iter().map(|x| x.num_rows()).sum();
+
+ let key_value_metadata = match self.props.key_value_metadata() {
+ Some(kv) =>
Some(kv.iter().chain(&self.kv_metadatas).cloned().collect()),
+ None if self.kv_metadatas.is_empty() => None,
+ None => Some(self.kv_metadatas.clone()),
+ };
let mut row_groups = self
.row_groups
- .as_slice()
.iter()
.map(|v| v.to_thrift())
.collect::<Vec<_>>();
self.write_bloom_filters(&mut row_groups)?;
- // Write column indexes and offset indexes
- self.write_column_indexes(&mut row_groups)?;
- self.write_offset_indexes(&mut row_groups)?;
- let key_value_metadata = match self.props.key_value_metadata() {
- Some(kv) =>
Some(kv.iter().chain(&self.kv_metadatas).cloned().collect()),
- None if self.kv_metadatas.is_empty() => None,
- None => Some(self.kv_metadatas.clone()),
- };
-
- // We only include ColumnOrder for leaf nodes.
- // Currently only supported ColumnOrder is TypeDefinedOrder so we set
this
- // for all leaf nodes.
- // Even if the column has an undefined sort order, such as INTERVAL,
this
- // is still technically the defined TYPEORDER so it should still be
set.
- let column_orders = (0..self.schema_descr().num_columns())
- .map(|_| parquet::ColumnOrder::TYPEORDER(parquet::TypeDefinedOrder
{}))
- .collect();
- // This field is optional, perhaps in cases where no min/max fields
are set
- // in any Statistics or ColumnIndex object in the whole file.
- // But for simplicity we always set this field.
- let column_orders = Some(column_orders);
-
- let file_metadata = parquet::FileMetaData {
- num_rows,
+ let encoder = ThriftMetadataWriter::new(
+ &mut self.buf,
+ &self.schema,
+ &self.descr,
row_groups,
- key_value_metadata,
- version: self.props.writer_version().as_num(),
- schema: types::to_thrift(self.schema.as_ref())?,
- created_by: Some(self.props.created_by().to_owned()),
- column_orders,
- encryption_algorithm: None,
- footer_signing_key_metadata: None,
- };
-
- // Write file metadata
- let start_pos = self.buf.bytes_written();
- {
- let mut protocol = TCompactOutputProtocol::new(&mut self.buf);
- file_metadata.write_to_out_protocol(&mut protocol)?;
- }
- let end_pos = self.buf.bytes_written();
-
- // Write footer
- let metadata_len = (end_pos - start_pos) as u32;
-
- self.buf.write_all(&metadata_len.to_le_bytes())?;
- self.buf.write_all(&PARQUET_MAGIC)?;
- Ok(file_metadata)
+ &self.column_indexes,
+ &self.offset_indexes,
+ &key_value_metadata,
+ Some(self.props.created_by().to_string()),
+ self.props.writer_version().as_num(),
+ );
+ encoder.finish()
Review Comment:
👍
##########
parquet/src/file/writer.rs:
##########
@@ -791,23 +710,274 @@ impl<'a, W: Write + Send> PageWriter for
SerializedPageWriter<'a, W> {
}
}
+struct ThriftMetadataWriter<'a, W: Write> {
Review Comment:
I always get confused when reading the parquet code between what are the
generated Thrift structures from the structures in
https://docs.rs/parquet/latest/parquet/file/metadata/index.html
I like how you have split out writing of the thrift structures here from the
writing of the `parquet::file` structures
##########
parquet/src/file/writer.rs:
##########
@@ -791,23 +710,274 @@ impl<'a, W: Write + Send> PageWriter for
SerializedPageWriter<'a, W> {
}
}
+struct ThriftMetadataWriter<'a, W: Write> {
+ buf: &'a mut TrackedWrite<W>,
+ schema: &'a TypePtr,
+ schema_descr: &'a SchemaDescPtr,
+ row_groups: Vec<RowGroup>,
+ column_indexes: &'a [Vec<Option<ColumnIndex>>],
+ offset_indexes: &'a [Vec<Option<OffsetIndex>>],
+ key_value_metadata: &'a Option<Vec<KeyValue>>,
+ created_by: Option<String>,
+ writer_version: i32,
+}
+
+impl<'a, W: Write> ThriftMetadataWriter<'a, W> {
+ /// Serialize all the offset index to the file
+ fn write_offset_indexes(&mut self) -> Result<()> {
+ // iter row group
+ // iter each column
+ // write offset index to the file
+ for (row_group_idx, row_group) in
self.row_groups.iter_mut().enumerate() {
+ for (column_idx, column_metadata) in
row_group.columns.iter_mut().enumerate() {
+ match &self.offset_indexes[row_group_idx][column_idx] {
+ Some(offset_index) => {
+ let start_offset = self.buf.bytes_written();
+ let mut protocol = TCompactOutputProtocol::new(&mut
self.buf);
+ offset_index.write_to_out_protocol(&mut protocol)?;
+ let end_offset = self.buf.bytes_written();
+ // set offset and index for offset index
+ column_metadata.offset_index_offset =
Some(start_offset as i64);
+ column_metadata.offset_index_length =
+ Some((end_offset - start_offset) as i32);
+ }
+ None => {}
+ }
+ }
+ }
+ Ok(())
+ }
+
+ /// Serialize all the column index to the file
+ fn write_column_indexes(&mut self) -> Result<()> {
+ // iter row group
+ // iter each column
+ // write column index to the file
+ for (row_group_idx, row_group) in
self.row_groups.iter_mut().enumerate() {
+ for (column_idx, column_metadata) in
row_group.columns.iter_mut().enumerate() {
+ match &self.column_indexes[row_group_idx][column_idx] {
+ Some(column_index) => {
+ let start_offset = self.buf.bytes_written();
+ let mut protocol = TCompactOutputProtocol::new(&mut
self.buf);
+ column_index.write_to_out_protocol(&mut protocol)?;
+ let end_offset = self.buf.bytes_written();
+ // set offset and index for offset index
+ column_metadata.column_index_offset =
Some(start_offset as i64);
+ column_metadata.column_index_length =
+ Some((end_offset - start_offset) as i32);
+ }
+ None => {}
+ }
+ }
+ }
+ Ok(())
+ }
+
+ /// Assembles and writes metadata at the end of the file.
+ pub fn finish(mut self) -> Result<parquet::FileMetaData> {
+ let num_rows = self.row_groups.iter().map(|x| x.num_rows).sum();
+
+ // Write column indexes and offset indexes
+ self.write_column_indexes()?;
+ self.write_offset_indexes()?;
+
+ // We only include ColumnOrder for leaf nodes.
+ // Currently only supported ColumnOrder is TypeDefinedOrder so we set
this
+ // for all leaf nodes.
+ // Even if the column has an undefined sort order, such as INTERVAL,
this
+ // is still technically the defined TYPEORDER so it should still be
set.
+ let column_orders = (0..self.schema_descr.num_columns())
+ .map(|_| parquet::ColumnOrder::TYPEORDER(parquet::TypeDefinedOrder
{}))
+ .collect();
+ // This field is optional, perhaps in cases where no min/max fields
are set
+ // in any Statistics or ColumnIndex object in the whole file.
+ // But for simplicity we always set this field.
+ let column_orders = Some(column_orders);
+
+ let file_metadata = parquet::FileMetaData {
+ num_rows,
+ row_groups: self.row_groups,
+ key_value_metadata: self.key_value_metadata.clone(),
+ version: self.writer_version,
+ schema: types::to_thrift(self.schema.as_ref())?,
+ created_by: self.created_by.clone(),
+ column_orders,
+ encryption_algorithm: None,
+ footer_signing_key_metadata: None,
+ };
+
+ // Write file metadata
+ let start_pos = self.buf.bytes_written();
+ {
+ let mut protocol = TCompactOutputProtocol::new(&mut self.buf);
+ file_metadata.write_to_out_protocol(&mut protocol)?;
+ }
+ let end_pos = self.buf.bytes_written();
+
+ // Write footer
+ let metadata_len = (end_pos - start_pos) as u32;
+
+ self.buf.write_all(&metadata_len.to_le_bytes())?;
+ self.buf.write_all(&PARQUET_MAGIC)?;
+ Ok(file_metadata)
+ }
+
+ pub(self) fn new(
+ buf: &'a mut TrackedWrite<W>,
+ schema: &'a TypePtr,
+ schema_descr: &'a SchemaDescPtr,
+ row_groups: Vec<RowGroup>,
+ column_indexes: &'a [Vec<Option<ColumnIndex>>],
+ offset_indexes: &'a [Vec<Option<OffsetIndex>>],
+ key_value_metadata: &'a Option<Vec<KeyValue>>,
+ created_by: Option<String>,
+ writer_version: i32,
+ ) -> Self {
+ Self {
+ buf,
+ schema,
+ schema_descr,
+ row_groups,
+ column_indexes,
+ offset_indexes,
+ key_value_metadata,
+ created_by,
+ writer_version,
+ }
+ }
+}
+
+pub struct ParquetMetadataWriter<'a, W: Write> {
+ buf: TrackedWrite<W>,
+ write_page_index: bool,
+ metadata: &'a ParquetMetaData,
+}
+
+impl<'a, W: Write> ParquetMetadataWriter<'a, W> {
+ pub fn new(buf: W, metadata: &'a ParquetMetaData) -> Self {
+ Self {
+ buf: TrackedWrite::new(buf),
+ write_page_index: true,
+ metadata,
+ }
+ }
+
+ pub fn write_page_index(&mut self, write_page_index: bool) -> &mut Self {
+ self.write_page_index = write_page_index;
+ self
+ }
+
+ pub fn finish(&mut self) -> Result<()> {
+ let file_metadata = self.metadata.file_metadata();
+
+ let schema = Arc::new(file_metadata.schema().clone());
+ let schema_descr = Arc::new(SchemaDescriptor::new(schema.clone()));
+ let created_by = file_metadata.created_by().map(str::to_string);
+
+ let row_groups = self
+ .metadata
+ .row_groups()
+ .iter()
+ .map(|rg| rg.to_thrift())
+ .collect::<Vec<_>>();
+
+ let key_value_metadata = file_metadata.key_value_metadata().cloned();
+
+ let column_indexes = self.convert_column_indexes();
+ let offset_indexes = self.convert_offset_index();
+
+ let encoder = ThriftMetadataWriter::new(
+ &mut self.buf,
+ &schema,
+ &schema_descr,
+ row_groups,
+ &column_indexes,
+ &offset_indexes,
+ &key_value_metadata,
+ created_by,
+ file_metadata.version(),
+ );
+ encoder.finish()?;
+
+ Ok(())
+ }
+
+ fn convert_column_indexes(&self) -> Vec<Vec<Option<ColumnIndex>>> {
Review Comment:
I was looking around for another copy of this code and I now see that this
is the first time we are going from `Index` --> `ColumnIndex`
Makes sense to me. I think this type of structure could really help clean up
some of the tests too (but I am getting ahead of myself)
##########
parquet/src/file/writer.rs:
##########
@@ -301,93 +277,36 @@ impl<W: Write + Send> SerializedFileWriter<W> {
Ok(())
}
- /// Serialize all the column index to the file
- fn write_column_indexes(&mut self, row_groups: &mut [RowGroup]) ->
Result<()> {
- // iter row group
- // iter each column
- // write column index to the file
- for (row_group_idx, row_group) in row_groups.iter_mut().enumerate() {
- for (column_idx, column_metadata) in
row_group.columns.iter_mut().enumerate() {
- match &self.column_indexes[row_group_idx][column_idx] {
- Some(column_index) => {
- let start_offset = self.buf.bytes_written();
- let mut protocol = TCompactOutputProtocol::new(&mut
self.buf);
- column_index.write_to_out_protocol(&mut protocol)?;
- let end_offset = self.buf.bytes_written();
- // set offset and index for offset index
- column_metadata.column_index_offset =
Some(start_offset as i64);
- column_metadata.column_index_length =
- Some((end_offset - start_offset) as i32);
- }
- None => {}
- }
- }
- }
- Ok(())
- }
-
/// Assembles and writes metadata at the end of the file.
fn write_metadata(&mut self) -> Result<parquet::FileMetaData> {
self.finished = true;
- let num_rows = self.row_groups.iter().map(|x| x.num_rows()).sum();
+
+ let key_value_metadata = match self.props.key_value_metadata() {
+ Some(kv) =>
Some(kv.iter().chain(&self.kv_metadatas).cloned().collect()),
+ None if self.kv_metadatas.is_empty() => None,
+ None => Some(self.kv_metadatas.clone()),
+ };
let mut row_groups = self
.row_groups
- .as_slice()
.iter()
.map(|v| v.to_thrift())
.collect::<Vec<_>>();
self.write_bloom_filters(&mut row_groups)?;
Review Comment:
FWIW https://github.com/apache/arrow-rs/pull/5933 also contains changes for
bloom filter writing
##########
parquet/src/file/writer.rs:
##########
@@ -301,93 +277,36 @@ impl<W: Write + Send> SerializedFileWriter<W> {
Ok(())
}
- /// Serialize all the column index to the file
- fn write_column_indexes(&mut self, row_groups: &mut [RowGroup]) ->
Result<()> {
- // iter row group
- // iter each column
- // write column index to the file
- for (row_group_idx, row_group) in row_groups.iter_mut().enumerate() {
- for (column_idx, column_metadata) in
row_group.columns.iter_mut().enumerate() {
- match &self.column_indexes[row_group_idx][column_idx] {
- Some(column_index) => {
- let start_offset = self.buf.bytes_written();
- let mut protocol = TCompactOutputProtocol::new(&mut
self.buf);
- column_index.write_to_out_protocol(&mut protocol)?;
- let end_offset = self.buf.bytes_written();
- // set offset and index for offset index
- column_metadata.column_index_offset =
Some(start_offset as i64);
- column_metadata.column_index_length =
- Some((end_offset - start_offset) as i32);
- }
- None => {}
- }
- }
- }
- Ok(())
- }
-
/// Assembles and writes metadata at the end of the file.
fn write_metadata(&mut self) -> Result<parquet::FileMetaData> {
self.finished = true;
- let num_rows = self.row_groups.iter().map(|x| x.num_rows()).sum();
+
+ let key_value_metadata = match self.props.key_value_metadata() {
+ Some(kv) =>
Some(kv.iter().chain(&self.kv_metadatas).cloned().collect()),
+ None if self.kv_metadatas.is_empty() => None,
+ None => Some(self.kv_metadatas.clone()),
+ };
let mut row_groups = self
.row_groups
- .as_slice()
.iter()
.map(|v| v.to_thrift())
.collect::<Vec<_>>();
self.write_bloom_filters(&mut row_groups)?;
- // Write column indexes and offset indexes
- self.write_column_indexes(&mut row_groups)?;
- self.write_offset_indexes(&mut row_groups)?;
- let key_value_metadata = match self.props.key_value_metadata() {
- Some(kv) =>
Some(kv.iter().chain(&self.kv_metadatas).cloned().collect()),
- None if self.kv_metadatas.is_empty() => None,
- None => Some(self.kv_metadatas.clone()),
- };
-
- // We only include ColumnOrder for leaf nodes.
- // Currently only supported ColumnOrder is TypeDefinedOrder so we set
this
- // for all leaf nodes.
- // Even if the column has an undefined sort order, such as INTERVAL,
this
- // is still technically the defined TYPEORDER so it should still be
set.
- let column_orders = (0..self.schema_descr().num_columns())
- .map(|_| parquet::ColumnOrder::TYPEORDER(parquet::TypeDefinedOrder
{}))
- .collect();
- // This field is optional, perhaps in cases where no min/max fields
are set
- // in any Statistics or ColumnIndex object in the whole file.
- // But for simplicity we always set this field.
- let column_orders = Some(column_orders);
-
- let file_metadata = parquet::FileMetaData {
- num_rows,
+ let encoder = ThriftMetadataWriter::new(
Review Comment:
This might read nicer like this:
```rust
let encoder = ThriftMetadataWriter::new()
.with_schema(&self.schema)
.with_descr(&self.descr)
.with_row_groups(row_groups)
...
);
// encode the data to buf
encoder.encode(&mut buf)
```
Though I realize many of these fields are required
Maybe something like
```rust
let encoder = ThriftMetadataWriter::new(
&self.schema,
&self.descr,
...
)
.with_column_indexes(&self.column_indexes)
.with_offset_indexes(&self.offset_indexes);
encoder.encode(&mut buf)
```
--
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]