This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/master by this push:
     new b9384f1  Add Parquet writer example to docs (#797)
b9384f1 is described below

commit b9384f1a30f3dc808a91a44846f552ba8e0ed383
Author: Matthew Turner <[email protected]>
AuthorDate: Thu Sep 30 15:10:13 2021 -0400

    Add Parquet writer example to docs (#797)
    
    * First example parquet writer
    
    * Add WriterProp examples
    
    * Add missing imports
    
    * Remove options and run doctest
    
    * One more section to run
    
    * no_run on read example
    
    * Make reader run test
    
    * Fix get_schema_by_cols
---
 parquet/src/arrow/mod.rs | 75 ++++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 72 insertions(+), 3 deletions(-)

diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs
index afb1fdc..d2a1daa 100644
--- a/parquet/src/arrow/mod.rs
+++ b/parquet/src/arrow/mod.rs
@@ -20,22 +20,91 @@
 //!
 //! This mod provides API for converting between arrow and parquet.
 //!
+//!# Example of writing Arrow record batch to Parquet file
+//!
+//!```rust
+//! use arrow::array::Int32Array;
+//! use arrow::datatypes::{DataType, Field, Schema};
+//! use arrow::record_batch::RecordBatch;
+//! use parquet::arrow::arrow_writer::ArrowWriter;
+//! use parquet::file::properties::WriterProperties;
+//! use std::fs::File;
+//! use std::sync::Arc;
+//! let ids = Int32Array::from(vec![1, 2, 3, 4]);
+//! let vals = Int32Array::from(vec![5, 6, 7, 8]);
+//! let schema = Arc::new(Schema::new(vec![
+//!     Field::new("id", DataType::Int32, false),
+//!     Field::new("val", DataType::Int32, false),
+//! ]));
+//!
+//! let file = File::create("data.parquet").unwrap();
+//!
+//! let batch =
+//!     RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(ids), 
Arc::new(vals)]).unwrap();
+//! let batches = vec![batch];
+//!
+//! // Default writer properties
+//! let props = WriterProperties::builder().build();
+//!
+//! let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), 
Some(props)).unwrap();
+//!
+//! for batch in batches {
+//!     writer.write(&batch).expect("Writing batch");
+//! }
+//! writer.close().unwrap();
+//! ```
+
+//! `WriterProperties` can be used to set Parquet file options
+//! ```rust
+//! use parquet::file::properties::WriterProperties;
+//! use parquet::basic::{ Compression, Encoding };
+//! use parquet::file::properties::WriterVersion;
+//!
+//! // File compression
+//! let props = WriterProperties::builder()
+//!     .set_compression(Compression::SNAPPY)
+//!     .build();
+//! ```
+//!
 //! # Example of reading parquet file into arrow record batch
 //!
-//! ```rust, no_run
+//! ```rust
 //! use arrow::record_batch::RecordBatchReader;
 //! use parquet::file::reader::SerializedFileReader;
 //! use parquet::arrow::{ParquetFileArrowReader, ArrowReader};
 //! use std::sync::Arc;
 //! use std::fs::File;
 //!
-//! let file = File::open("parquet.file").unwrap();
+//! # use arrow::array::Int32Array;
+//! # use arrow::datatypes::{DataType, Field, Schema};
+//! # use arrow::record_batch::RecordBatch;
+//! # use parquet::arrow::arrow_writer::ArrowWriter;
+//! # let ids = Int32Array::from(vec![1, 2, 3, 4]);
+//! # let schema = Arc::new(Schema::new(vec![
+//! #    Field::new("id", DataType::Int32, false),
+//! # ]));
+//! #
+//! # // Write to a memory buffer (can also write to a File)
+//! # let file = File::create("data.parquet").unwrap();
+//! #
+//! # let batch =
+//! #    RecordBatch::try_new(Arc::clone(&schema), 
vec![Arc::new(ids)]).unwrap();
+//! # let batches = vec![batch];
+//! #
+//! # let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), 
None).unwrap();
+//! #
+//! # for batch in batches {
+//! #     writer.write(&batch).expect("Writing batch");
+//! # }
+//! # writer.close().unwrap();
+//!
+//! let file = File::open("data.parquet").unwrap();
 //! let file_reader = SerializedFileReader::new(file).unwrap();
 //! let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader));
 //!
 //! println!("Converted arrow schema is: {}", 
arrow_reader.get_schema().unwrap());
 //! println!("Arrow schema after projection is: {}",
-//!    arrow_reader.get_schema_by_columns(vec![2, 4, 6], true).unwrap());
+//!    arrow_reader.get_schema_by_columns(vec![0], true).unwrap());
 //!
 //! let mut record_batch_reader = 
arrow_reader.get_record_reader(2048).unwrap();
 //!

Reply via email to