mito2/sst/parquet/
writer.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
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
// Copyright 2023 Greptime Team
//
// Licensed 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.

//! Parquet writer.

use std::future::Future;
use std::pin::Pin;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;
use std::task::{Context, Poll};

use common_time::Timestamp;
use datatypes::arrow::datatypes::SchemaRef;
use object_store::{FuturesAsyncWriter, ObjectStore};
use parquet::arrow::AsyncArrowWriter;
use parquet::basic::{Compression, Encoding, ZstdLevel};
use parquet::file::metadata::KeyValue;
use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder};
use parquet::schema::types::ColumnPath;
use smallvec::smallvec;
use snafu::ResultExt;
use store_api::metadata::RegionMetadataRef;
use store_api::storage::consts::SEQUENCE_COLUMN_NAME;
use store_api::storage::SequenceNumber;
use tokio::io::AsyncWrite;
use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt};

use crate::access_layer::{FilePathProvider, SstInfoArray};
use crate::error::{InvalidMetadataSnafu, OpenDalSnafu, Result, WriteParquetSnafu};
use crate::read::{Batch, Source};
use crate::sst::file::FileId;
use crate::sst::index::{Indexer, IndexerBuilder};
use crate::sst::parquet::format::WriteFormat;
use crate::sst::parquet::helper::parse_parquet_metadata;
use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY};
use crate::sst::{DEFAULT_WRITE_BUFFER_SIZE, DEFAULT_WRITE_CONCURRENCY};

/// Parquet SST writer.
pub struct ParquetWriter<F: WriterFactory, I: IndexerBuilder, P: FilePathProvider> {
    /// Path provider that creates SST and index file paths according to file id.
    path_provider: P,
    writer: Option<AsyncArrowWriter<SizeAwareWriter<F::Writer>>>,
    /// Current active file id.
    current_file: FileId,
    writer_factory: F,
    /// Region metadata of the source and the target SST.
    metadata: RegionMetadataRef,
    /// Indexer build that can create indexer for multiple files.
    indexer_builder: I,
    /// Current active indexer.
    current_indexer: Option<Indexer>,
    bytes_written: Arc<AtomicUsize>,
}

pub trait WriterFactory {
    type Writer: AsyncWrite + Send + Unpin;
    fn create(&mut self, file_path: &str) -> impl Future<Output = Result<Self::Writer>>;
}

pub struct ObjectStoreWriterFactory {
    object_store: ObjectStore,
}

impl WriterFactory for ObjectStoreWriterFactory {
    type Writer = Compat<FuturesAsyncWriter>;

    async fn create(&mut self, file_path: &str) -> Result<Self::Writer> {
        self.object_store
            .writer_with(file_path)
            .chunk(DEFAULT_WRITE_BUFFER_SIZE.as_bytes() as usize)
            .concurrent(DEFAULT_WRITE_CONCURRENCY)
            .await
            .map(|v| v.into_futures_async_write().compat_write())
            .context(OpenDalSnafu)
    }
}

impl<I, P> ParquetWriter<ObjectStoreWriterFactory, I, P>
where
    P: FilePathProvider,
    I: IndexerBuilder,
{
    pub async fn new_with_object_store(
        object_store: ObjectStore,
        metadata: RegionMetadataRef,
        indexer_builder: I,
        path_provider: P,
    ) -> ParquetWriter<ObjectStoreWriterFactory, I, P> {
        ParquetWriter::new(
            ObjectStoreWriterFactory { object_store },
            metadata,
            indexer_builder,
            path_provider,
        )
        .await
    }
}

impl<F, I, P> ParquetWriter<F, I, P>
where
    F: WriterFactory,
    I: IndexerBuilder,
    P: FilePathProvider,
{
    /// Creates a new parquet SST writer.
    pub async fn new(
        factory: F,
        metadata: RegionMetadataRef,
        indexer_builder: I,
        path_provider: P,
    ) -> ParquetWriter<F, I, P> {
        let init_file = FileId::random();
        let indexer = indexer_builder.build(init_file).await;

        ParquetWriter {
            path_provider,
            writer: None,
            current_file: init_file,
            writer_factory: factory,
            metadata,
            indexer_builder,
            current_indexer: Some(indexer),
            bytes_written: Arc::new(AtomicUsize::new(0)),
        }
    }

    async fn get_or_create_indexer(&mut self) -> &mut Indexer {
        match self.current_indexer {
            None => {
                self.current_file = FileId::random();
                let indexer = self.indexer_builder.build(self.current_file).await;
                self.current_indexer = Some(indexer);
                // safety: self.current_indexer already set above.
                self.current_indexer.as_mut().unwrap()
            }
            Some(ref mut indexer) => indexer,
        }
    }

    /// Iterates source and writes all rows to Parquet file.
    ///
    /// Returns the [SstInfo] if the SST is written.
    pub async fn write_all(
        &mut self,
        mut source: Source,
        override_sequence: Option<SequenceNumber>, // override the `sequence` field from `Source`
        opts: &WriteOptions,
    ) -> Result<SstInfoArray> {
        let write_format =
            WriteFormat::new(self.metadata.clone()).with_override_sequence(override_sequence);
        let mut stats = SourceStats::default();

        while let Some(res) = self
            .write_next_batch(&mut source, &write_format, opts)
            .await
            .transpose()
        {
            match res {
                Ok(mut batch) => {
                    stats.update(&batch);
                    self.get_or_create_indexer().await.update(&mut batch).await;
                }
                Err(e) => {
                    self.get_or_create_indexer().await.abort().await;
                    return Err(e);
                }
            }
        }

        let index_output = self.get_or_create_indexer().await.finish().await;

        if stats.num_rows == 0 {
            return Ok(smallvec![]);
        }

        let Some(mut arrow_writer) = self.writer.take() else {
            // No batch actually written.
            return Ok(smallvec![]);
        };

        arrow_writer.flush().await.context(WriteParquetSnafu)?;

        let file_meta = arrow_writer.close().await.context(WriteParquetSnafu)?;
        let file_size = self.bytes_written.load(Ordering::Relaxed) as u64;

        // Safety: num rows > 0 so we must have min/max.
        let time_range = stats.time_range.unwrap();

        // convert FileMetaData to ParquetMetaData
        let parquet_metadata = parse_parquet_metadata(file_meta)?;

        let file_id = self.current_file;

        // object_store.write will make sure all bytes are written or an error is raised.
        Ok(smallvec![SstInfo {
            file_id,
            time_range,
            file_size,
            num_rows: stats.num_rows,
            num_row_groups: parquet_metadata.num_row_groups() as u64,
            file_metadata: Some(Arc::new(parquet_metadata)),
            index_metadata: index_output,
        }])
    }

    /// Customizes per-column config according to schema and maybe column cardinality.
    fn customize_column_config(
        builder: WriterPropertiesBuilder,
        region_metadata: &RegionMetadataRef,
    ) -> WriterPropertiesBuilder {
        let ts_col = ColumnPath::new(vec![region_metadata
            .time_index_column()
            .column_schema
            .name
            .clone()]);
        let seq_col = ColumnPath::new(vec![SEQUENCE_COLUMN_NAME.to_string()]);

        builder
            .set_column_encoding(seq_col.clone(), Encoding::DELTA_BINARY_PACKED)
            .set_column_dictionary_enabled(seq_col, false)
            .set_column_encoding(ts_col.clone(), Encoding::DELTA_BINARY_PACKED)
            .set_column_dictionary_enabled(ts_col, false)
    }

    async fn write_next_batch(
        &mut self,
        source: &mut Source,
        write_format: &WriteFormat,
        opts: &WriteOptions,
    ) -> Result<Option<Batch>> {
        let Some(batch) = source.next_batch().await? else {
            return Ok(None);
        };

        let arrow_batch = write_format.convert_batch(&batch)?;
        self.maybe_init_writer(write_format.arrow_schema(), opts)
            .await?
            .write(&arrow_batch)
            .await
            .context(WriteParquetSnafu)?;
        Ok(Some(batch))
    }

    async fn maybe_init_writer(
        &mut self,
        schema: &SchemaRef,
        opts: &WriteOptions,
    ) -> Result<&mut AsyncArrowWriter<SizeAwareWriter<F::Writer>>> {
        if let Some(ref mut w) = self.writer {
            Ok(w)
        } else {
            let json = self.metadata.to_json().context(InvalidMetadataSnafu)?;
            let key_value_meta = KeyValue::new(PARQUET_METADATA_KEY.to_string(), json);

            // TODO(yingwen): Find and set proper column encoding for internal columns: op type and tsid.
            let props_builder = WriterProperties::builder()
                .set_key_value_metadata(Some(vec![key_value_meta]))
                .set_compression(Compression::ZSTD(ZstdLevel::default()))
                .set_encoding(Encoding::PLAIN)
                .set_max_row_group_size(opts.row_group_size);

            let props_builder = Self::customize_column_config(props_builder, &self.metadata);
            let writer_props = props_builder.build();

            let sst_file_path = self.path_provider.build_sst_file_path(self.current_file);
            let writer = SizeAwareWriter::new(
                self.writer_factory.create(&sst_file_path).await?,
                self.bytes_written.clone(),
            );
            let arrow_writer =
                AsyncArrowWriter::try_new(writer, schema.clone(), Some(writer_props))
                    .context(WriteParquetSnafu)?;
            self.writer = Some(arrow_writer);
            // safety: self.writer is assigned above
            Ok(self.writer.as_mut().unwrap())
        }
    }
}

#[derive(Default)]
struct SourceStats {
    /// Number of rows fetched.
    num_rows: usize,
    /// Time range of fetched batches.
    time_range: Option<(Timestamp, Timestamp)>,
}

impl SourceStats {
    fn update(&mut self, batch: &Batch) {
        if batch.is_empty() {
            return;
        }

        self.num_rows += batch.num_rows();
        // Safety: batch is not empty.
        let (min_in_batch, max_in_batch) = (
            batch.first_timestamp().unwrap(),
            batch.last_timestamp().unwrap(),
        );
        if let Some(time_range) = &mut self.time_range {
            time_range.0 = time_range.0.min(min_in_batch);
            time_range.1 = time_range.1.max(max_in_batch);
        } else {
            self.time_range = Some((min_in_batch, max_in_batch));
        }
    }
}

/// Workaround for [AsyncArrowWriter] does not provide a method to
/// get total bytes written after close.
struct SizeAwareWriter<W> {
    inner: W,
    size: Arc<AtomicUsize>,
}

impl<W> SizeAwareWriter<W> {
    fn new(inner: W, size: Arc<AtomicUsize>) -> Self {
        Self {
            inner,
            size: size.clone(),
        }
    }
}

impl<W> AsyncWrite for SizeAwareWriter<W>
where
    W: AsyncWrite + Unpin,
{
    fn poll_write(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
        buf: &[u8],
    ) -> Poll<std::result::Result<usize, std::io::Error>> {
        let this = self.as_mut().get_mut();

        match Pin::new(&mut this.inner).poll_write(cx, buf) {
            Poll::Ready(Ok(bytes_written)) => {
                this.size.fetch_add(bytes_written, Ordering::Relaxed);
                Poll::Ready(Ok(bytes_written))
            }
            other => other,
        }
    }

    fn poll_flush(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<std::result::Result<(), std::io::Error>> {
        Pin::new(&mut self.inner).poll_flush(cx)
    }

    fn poll_shutdown(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<std::result::Result<(), std::io::Error>> {
        Pin::new(&mut self.inner).poll_shutdown(cx)
    }
}