polars_parquet/parquet/write/
compression.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
use crate::parquet::compression::CompressionOptions;
use crate::parquet::error::{ParquetError, ParquetResult};
use crate::parquet::page::{
    CompressedDataPage, CompressedDictPage, CompressedPage, DataPage, DataPageHeader, DictPage,
    Page,
};
use crate::parquet::{compression, CowBuffer, FallibleStreamingIterator};

/// Compresses a [`DataPage`] into a [`CompressedDataPage`].
fn compress_data(
    page: DataPage,
    mut compressed_buffer: Vec<u8>,
    compression: CompressionOptions,
) -> ParquetResult<CompressedDataPage> {
    let DataPage {
        mut buffer,
        header,
        descriptor,
        num_rows,
    } = page;
    let uncompressed_page_size = buffer.len();
    let num_rows = num_rows.expect("We should have num_rows when we are writing");
    if compression != CompressionOptions::Uncompressed {
        match &header {
            DataPageHeader::V1(_) => {
                compression::compress(compression, &buffer, &mut compressed_buffer)?;
            },
            DataPageHeader::V2(header) => {
                let levels_byte_length = (header.repetition_levels_byte_length
                    + header.definition_levels_byte_length)
                    as usize;
                compressed_buffer.extend_from_slice(&buffer[..levels_byte_length]);
                compression::compress(
                    compression,
                    &buffer[levels_byte_length..],
                    &mut compressed_buffer,
                )?;
            },
        };
    } else {
        std::mem::swap(buffer.to_mut(), &mut compressed_buffer);
    }

    Ok(CompressedDataPage::new(
        header,
        CowBuffer::Owned(compressed_buffer),
        compression.into(),
        uncompressed_page_size,
        descriptor,
        num_rows,
    ))
}

fn compress_dict(
    page: DictPage,
    mut compressed_buffer: Vec<u8>,
    compression: CompressionOptions,
) -> ParquetResult<CompressedDictPage> {
    let DictPage {
        buffer,
        num_values,
        is_sorted,
    } = page;

    let uncompressed_page_size = buffer.len();
    let compressed_buffer = if compression != CompressionOptions::Uncompressed {
        compression::compress(compression, &buffer, &mut compressed_buffer)?;
        CowBuffer::Owned(compressed_buffer)
    } else {
        buffer
    };

    Ok(CompressedDictPage::new(
        compressed_buffer,
        compression.into(),
        uncompressed_page_size,
        num_values,
        is_sorted,
    ))
}

/// Compresses an [`EncodedPage`] into a [`CompressedPage`] using `compressed_buffer` as the
/// intermediary buffer.
///
/// `compressed_buffer` is taken by value because it becomes owned by [`CompressedPage`]
///
/// # Errors
/// Errors if the compressor fails
pub fn compress(
    page: Page,
    compressed_buffer: Vec<u8>,
    compression: CompressionOptions,
) -> ParquetResult<CompressedPage> {
    match page {
        Page::Data(page) => {
            compress_data(page, compressed_buffer, compression).map(CompressedPage::Data)
        },
        Page::Dict(page) => {
            compress_dict(page, compressed_buffer, compression).map(CompressedPage::Dict)
        },
    }
}

/// A [`FallibleStreamingIterator`] that consumes [`Page`] and yields [`CompressedPage`]
/// holding a reusable buffer ([`Vec<u8>`]) for compression.
pub struct Compressor<I: Iterator<Item = ParquetResult<Page>>> {
    iter: I,
    compression: CompressionOptions,
    buffer: Vec<u8>,
    current: Option<CompressedPage>,
}

impl<I: Iterator<Item = ParquetResult<Page>>> Compressor<I> {
    /// Creates a new [`Compressor`]
    pub fn new(iter: I, compression: CompressionOptions, buffer: Vec<u8>) -> Self {
        Self {
            iter,
            compression,
            buffer,
            current: None,
        }
    }

    /// Creates a new [`Compressor`] (same as `new`)
    pub fn new_from_vec(iter: I, compression: CompressionOptions, buffer: Vec<u8>) -> Self {
        Self::new(iter, compression, buffer)
    }

    /// Deconstructs itself into its iterator and scratch buffer.
    pub fn into_inner(mut self) -> (I, Vec<u8>) {
        let mut buffer = if let Some(page) = self.current.as_mut() {
            std::mem::take(page.buffer_mut())
        } else {
            std::mem::take(&mut self.buffer)
        };
        buffer.clear();
        (self.iter, buffer)
    }
}

impl<I: Iterator<Item = ParquetResult<Page>>> FallibleStreamingIterator for Compressor<I> {
    type Item = CompressedPage;
    type Error = ParquetError;

    fn advance(&mut self) -> std::result::Result<(), Self::Error> {
        let mut compressed_buffer = if let Some(page) = self.current.as_mut() {
            std::mem::take(page.buffer_mut())
        } else {
            std::mem::take(&mut self.buffer)
        };
        compressed_buffer.clear();

        let next = self
            .iter
            .next()
            .map(|x| x.and_then(|page| compress(page, compressed_buffer, self.compression)))
            .transpose()?;
        self.current = next;
        Ok(())
    }

    fn get(&self) -> Option<&Self::Item> {
        self.current.as_ref()
    }
}