ZENOTME commented on code in PR #176:
URL: https://github.com/apache/iceberg-rust/pull/176#discussion_r1475452336


##########
crates/iceberg/src/writer/file_writer/parquet_writer.rs:
##########
@@ -0,0 +1,390 @@
+// 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.
+
+//! The module contains the file writer for parquet file format.
+
+use std::{
+    collections::HashMap,
+    sync::{atomic::AtomicI64, Arc},
+};
+
+use crate::Result;
+use crate::{
+    io::{FileIO, OutputFile},
+    spec::{DataFileBuilder, DataFileFormat},
+    writer::CurrentFileStatus,
+    Error,
+};
+use arrow_schema::SchemaRef;
+use parquet::{arrow::AsyncArrowWriter, format::FileMetaData};
+use parquet::{arrow::PARQUET_FIELD_ID_META_KEY, 
file::properties::WriterProperties};
+
+use super::{
+    location_generator::LocationGenerator, track_writer::TrackWriter, 
FileWriter, FileWriterBuilder,
+};
+
+/// ParquetWriterBuilder is used to builder a [`ParquetWriter`]
+#[derive(Clone)]
+pub struct ParquetWriterBuilder {
+    /// `buffer_size` determines the initial size of the intermediate buffer.
+    /// The intermediate buffer will automatically be resized if necessary
+    init_buffer_size: usize,
+    props: WriterProperties,
+    schema: SchemaRef,
+
+    file_io: FileIO,
+    location_generator: Arc<dyn LocationGenerator>,
+}
+
+impl ParquetWriterBuilder {
+    /// To avoid EntiryTooSmall error, we set the minimum buffer size to 8MB 
if the given buffer size is smaller than it.
+    const MIN_BUFFER_SIZE: usize = 8 * 1024 * 1024;
+
+    /// Create a new `ParquetWriterBuilder`
+    /// To construct the write result, the schema should contain the 
`PARQUET_FIELD_ID_META_KEY` metadata for each field.
+    pub fn new(
+        init_buffer_size: usize,
+        props: WriterProperties,
+        schema: SchemaRef,
+        file_io: FileIO,
+        location_generator: Arc<dyn LocationGenerator>,
+    ) -> Self {
+        Self {
+            init_buffer_size,
+            props,
+            schema,
+            file_io,
+            location_generator,
+        }
+    }
+}
+
+impl FileWriterBuilder for ParquetWriterBuilder {
+    type R = ParquetWriter;
+
+    async fn build(self) -> crate::Result<Self::R> {
+        // Fetch field id from schema
+        let field_ids = self
+            .schema
+            .fields()
+            .iter()
+            .map(|field| {
+                field
+                    .metadata()
+                    .get(PARQUET_FIELD_ID_META_KEY)
+                    .ok_or_else(|| {
+                        Error::new(
+                            crate::ErrorKind::UnexpectedParameter,
+                            "field id not found in schema",
+                        )
+                    })?
+                    .parse::<i32>()
+                    .map_err(|err| {
+                        Error::new(
+                            crate::ErrorKind::UnexpectedParameter,
+                            "field id parse error",
+                        )
+                        .with_source(err)
+                    })
+            })
+            .collect::<crate::Result<Vec<_>>>()?;
+
+        let written_size = Arc::new(AtomicI64::new(0));
+        let file_path = self.location_generator.generate_name();
+        let out_file = self.file_io.new_output(file_path)?;
+        let inner_writer = TrackWriter::new(out_file.writer().await?, 
written_size.clone());
+        let init_buffer_size = if self.init_buffer_size < 
Self::MIN_BUFFER_SIZE {
+            Self::MIN_BUFFER_SIZE
+        } else {
+            self.init_buffer_size
+        };
+        let writer = AsyncArrowWriter::try_new(
+            inner_writer,
+            self.schema.clone(),
+            init_buffer_size,
+            Some(self.props),
+        )
+        .map_err(|err| {
+            Error::new(
+                crate::ErrorKind::Unexpected,
+                "build error from parquet writer",
+            )
+            .with_source(err)
+        })?;
+
+        Ok(ParquetWriter {
+            writer,
+            written_size,
+            current_row_num: 0,
+            out_file,
+            file_io: self.file_io,
+            field_ids,
+        })
+    }
+}
+
+/// `ParquetWriter`` is used to write arrow data into parquet file on storage.
+pub struct ParquetWriter {

Review Comment:
   I'm not sure what this means. ParquetWriter doesn't need another FileWriter. 
It's the lowest-level writer.



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to