pub struct ArrowColumnWriter { /* private fields */ }
Expand description

Encodes ArrowLeafColumn to ArrowColumnChunk

Note: This is a low-level interface for applications that require fine-grained control of encoding, see ArrowWriter for a higher-level interface

// The arrow schema
let schema = Arc::new(Schema::new(vec![
    Field::new("i32", DataType::Int32, false),
    Field::new("f32", DataType::Float32, false),
]));

// Compute the parquet schema
let parquet_schema = arrow_to_parquet_schema(schema.as_ref()).unwrap();
let props = Arc::new(WriterProperties::default());

// Create writers for each of the leaf columns
let col_writers = get_column_writers(&parquet_schema, &props, &schema).unwrap();

// Spawn a worker thread for each column
// This is for demonstration purposes, a thread-pool e.g. rayon or tokio, would be better
let mut workers: Vec<_> = col_writers
    .into_iter()
    .map(|mut col_writer| {
        let (send, recv) = std::sync::mpsc::channel::<ArrowLeafColumn>();
        let handle = std::thread::spawn(move || {
            for col in recv {
                col_writer.write(&col)?;
            }
            col_writer.close()
        });
        (handle, send)
    })
    .collect();

// Create parquet writer
let root_schema = parquet_schema.root_schema_ptr();
let mut out = Vec::with_capacity(1024); // This could be a File
let mut writer = SerializedFileWriter::new(&mut out, root_schema, props.clone()).unwrap();

// Start row group
let mut row_group = writer.next_row_group().unwrap();

// Columns to encode
let to_write = vec![
    Arc::new(Int32Array::from_iter_values([1, 2, 3])) as _,
    Arc::new(Float32Array::from_iter_values([1., 45., -1.])) as _,
];

// Spawn work to encode columns
let mut worker_iter = workers.iter_mut();
for (arr, field) in to_write.iter().zip(&schema.fields) {
    for leaves in compute_leaves(field, arr).unwrap() {
        worker_iter.next().unwrap().1.send(leaves).unwrap();
    }
}

// Finish up parallel column encoding
for (handle, send) in workers {
    drop(send); // Drop send side to signal termination
    let chunk = handle.join().unwrap().unwrap();
    chunk.append_to_row_group(&mut row_group).unwrap();
}
row_group.close().unwrap();

let metadata = writer.close().unwrap();
assert_eq!(metadata.num_rows, 3);

Implementations§

source§

impl ArrowColumnWriter

source

pub fn write(&mut self, col: &ArrowLeafColumn) -> Result<()>

Write an ArrowLeafColumn

source

pub fn close(self) -> Result<ArrowColumnChunk>

Close this column returning the written ArrowColumnChunk

source

pub fn get_estimated_total_bytes(&self) -> usize

Returns the estimated total bytes for this column writer

Trait Implementations§

source§

impl Debug for ArrowColumnWriter

source§

fn fmt(&self, f: &mut Formatter<'_>) -> Result

Formats the value using the given formatter. Read more

Auto Trait Implementations§

Blanket Implementations§

source§

impl<T> Any for T
where T: 'static + ?Sized,

source§

fn type_id(&self) -> TypeId

Gets the TypeId of self. Read more
source§

impl<T> Borrow<T> for T
where T: ?Sized,

source§

fn borrow(&self) -> &T

Immutably borrows from an owned value. Read more
source§

impl<T> BorrowMut<T> for T
where T: ?Sized,

source§

fn borrow_mut(&mut self) -> &mut T

Mutably borrows from an owned value. Read more
source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

source§

impl<T, U> Into<U> for T
where U: From<T>,

source§

fn into(self) -> U

Calls U::from(self).

That is, this conversion is whatever the implementation of From<T> for U chooses to do.

source§

impl<T, U> TryFrom<U> for T
where U: Into<T>,

§

type Error = Infallible

The type returned in the event of a conversion error.
source§

fn try_from(value: U) -> Result<T, <T as TryFrom<U>>::Error>

Performs the conversion.
source§

impl<T, U> TryInto<U> for T
where U: TryFrom<T>,

§

type Error = <U as TryFrom<T>>::Error

The type returned in the event of a conversion error.
source§

fn try_into(self) -> Result<U, <U as TryFrom<T>>::Error>

Performs the conversion.