mz_storage_operators/oneshot_source/
parquet.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
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! Parquet [`OneshotFormat`].

use std::fmt;
use std::sync::Arc;

use arrow::array::{Array, RecordBatch, StructArray};
use bytes::{Bytes, BytesMut};
use futures::future::BoxFuture;
use futures::stream::BoxStream;
use futures::{StreamExt, TryStreamExt};
use mz_persist_types::arrow::ProtoArrayData;
use mz_proto::{ProtoType, RustType};
use mz_repr::RelationDesc;
use parquet::arrow::async_reader::{AsyncFileReader, MetadataFetch};
use parquet::arrow::ParquetRecordBatchStreamBuilder;
use parquet::errors::ParquetError;
use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader};
use prost::Message;
use serde::de::Visitor;
use serde::{Deserialize, Deserializer, Serialize};
use smallvec::{smallvec, SmallVec};

use crate::oneshot_source::{
    OneshotFormat, OneshotObject, OneshotSource, StorageErrorX, StorageErrorXContext,
    StorageErrorXKind,
};

#[derive(Debug, Clone)]
pub struct ParquetFormat {
    desc: RelationDesc,
}

impl ParquetFormat {
    pub fn new(desc: RelationDesc) -> Self {
        ParquetFormat { desc }
    }
}

#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct ParquetWorkRequest<O, C> {
    object: O,
    checksum: C,
    row_groups: SmallVec<[usize; 1]>,
}

#[derive(Clone, Debug)]
pub struct ParquetRowGroup {
    record_batch: RecordBatch,
}

impl OneshotFormat for ParquetFormat {
    type WorkRequest<S>
        = ParquetWorkRequest<S::Object, S::Checksum>
    where
        S: OneshotSource;
    type RecordChunk = ParquetRowGroup;

    async fn split_work<S: OneshotSource + Send>(
        &self,
        source: S,
        object: S::Object,
        checksum: S::Checksum,
    ) -> Result<Vec<Self::WorkRequest<S>>, StorageErrorX> {
        let mut adapter = ParquetReaderAdapter::new(source, object.clone(), checksum.clone());
        let parquet_metadata = adapter.get_metadata().await?;

        tracing::info!(
            object = object.name(),
            row_groups = parquet_metadata.num_row_groups(),
            "splitting Parquet object"
        );

        // Split up the file by the number of RowGroups.
        //
        // TODO(cf3): Support splitting up large RowGroups.
        let work = (0..parquet_metadata.num_row_groups())
            .map(|row_group| ParquetWorkRequest {
                object: object.clone(),
                checksum: checksum.clone(),
                row_groups: smallvec![row_group],
            })
            .collect();

        Ok(work)
    }

    fn fetch_work<'a, S: OneshotSource + Sync + 'static>(
        &'a self,
        source: &'a S,
        request: Self::WorkRequest<S>,
    ) -> BoxStream<'a, Result<Self::RecordChunk, StorageErrorX>> {
        let ParquetWorkRequest {
            object,
            checksum,
            row_groups,
        } = request;

        let adapter = ParquetReaderAdapter::new(source.clone(), object.clone(), checksum.clone());

        let initial_work = async move {
            ParquetRecordBatchStreamBuilder::new(adapter)
                .await?
                .with_row_groups(row_groups.to_vec())
                .build()
        };

        futures::stream::once(initial_work)
            .try_flatten()
            .map_ok(|record_batch| ParquetRowGroup { record_batch })
            .err_into()
            .boxed()
    }

    fn decode_chunk(
        &self,
        chunk: Self::RecordChunk,
        rows: &mut Vec<mz_repr::Row>,
    ) -> Result<usize, StorageErrorX> {
        let ParquetRowGroup { record_batch } = chunk;

        let struct_array = StructArray::from(record_batch);
        let reader = mz_arrow_util::reader::ArrowReader::new(&self.desc, struct_array)
            .map_err(|err| StorageErrorXKind::ParquetError(err.to_string().into()))
            .context("reader")?;
        let rows_read = reader
            .read_all(rows)
            .map_err(|err| StorageErrorXKind::ParquetError(err.to_string().into()))
            .context("read_all")?;

        Ok(rows_read)
    }
}

/// A newtype wrapper around a [`OneshotSource`] that allows us to implement
/// [`AsyncFileReader`] and [`MetadataFetch`] for all types that implement
/// [`OneshotSource`].
#[derive(Clone)]
struct ParquetReaderAdapter<S: OneshotSource> {
    source: S,
    object: S::Object,
    checksum: S::Checksum,
}

impl<S: OneshotSource> fmt::Debug for ParquetReaderAdapter<S> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("ObjectStoreAdapter").finish()
    }
}

impl<S: OneshotSource> ParquetReaderAdapter<S> {
    fn new(source: S, object: S::Object, checksum: S::Checksum) -> Self {
        ParquetReaderAdapter {
            source,
            object,
            checksum,
        }
    }
}

impl<S: OneshotSource> MetadataFetch for ParquetReaderAdapter<S> {
    fn fetch(
        &mut self,
        range: std::ops::Range<usize>,
    ) -> BoxFuture<'_, parquet::errors::Result<Bytes>> {
        let inclusive_end = std::cmp::max(range.start, range.end.saturating_sub(1));

        Box::pin(async move {
            // Fetch the specified range.
            let result: Result<Vec<_>, _> = self
                .source
                .get(
                    self.object.clone(),
                    self.checksum.clone(),
                    Some(range.start..=inclusive_end),
                )
                .try_collect()
                .await;
            let bytes = match result {
                Err(e) => return Err(ParquetError::General(e.to_string())),
                Ok(bytes) => bytes,
            };

            // Join the stream into a single chunk.
            let total_length = inclusive_end.saturating_sub(range.start);
            let mut joined_bytes = BytesMut::with_capacity(total_length);
            joined_bytes.extend(bytes);

            Ok(joined_bytes.freeze())
        })
    }
}

impl<S: OneshotSource> AsyncFileReader for ParquetReaderAdapter<S> {
    fn get_bytes(
        &mut self,
        range: std::ops::Range<usize>,
    ) -> BoxFuture<'_, parquet::errors::Result<Bytes>> {
        MetadataFetch::fetch(self, range)
    }

    fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result<Arc<ParquetMetaData>>> {
        Box::pin(async move {
            let mut reader = ParquetMetaDataReader::new();
            let object_size = self.object.size();
            reader.try_load(self, object_size).await?;
            reader.finish().map(Arc::new)
        })
    }
}

// Note(parkmycar): Instead of a manual implementation of Serialize and Deserialize we could
// change `ParquetRowGroup` to have a type which we can derive the impl for. But no types from the
// `arrow` crate do, and we'd prefer not to use a Vec<u8> since serialization is only required when
// Timely workers span multiple processes.
impl Serialize for ParquetRowGroup {
    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
    where
        S: serde::Serializer,
    {
        // Note: This implementation isn't very efficient, but it should only be rarely used so
        // it's not too much of a concern.
        let struct_array = StructArray::from(self.record_batch.clone());
        let proto_array: ProtoArrayData = struct_array.into_data().into_proto();
        let encoded_proto = proto_array.encode_to_vec();
        encoded_proto.serialize(serializer)
    }
}

impl<'de> Deserialize<'de> for ParquetRowGroup {
    fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
    where
        D: serde::Deserializer<'de>,
    {
        fn struct_array<'de: 'a, 'a, D: Deserializer<'de>>(
            deserializer: D,
        ) -> Result<StructArray, D::Error> {
            struct StructArrayVisitor;

            impl<'a> Visitor<'a> for StructArrayVisitor {
                type Value = StructArray;

                fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result {
                    formatter.write_str("binary data")
                }

                fn visit_bytes<E>(self, v: &[u8]) -> Result<Self::Value, E>
                where
                    E: serde::de::Error,
                {
                    let serde_err =
                        || serde::de::Error::invalid_value(serde::de::Unexpected::Bytes(v), &self);

                    let array_data = ProtoArrayData::decode(v)
                        .map_err(|_| serde_err())
                        .and_then(|proto_array| proto_array.into_rust().map_err(|_| serde_err()))?;
                    let array_ref = arrow::array::make_array(array_data);
                    let struct_array = array_ref
                        .as_any()
                        .downcast_ref::<StructArray>()
                        .ok_or_else(serde_err)?;

                    Ok(struct_array.clone())
                }
            }

            deserializer.deserialize_bytes(StructArrayVisitor)
        }

        let struct_array = struct_array(deserializer)?;
        let record_batch = RecordBatch::from(struct_array);

        Ok(ParquetRowGroup { record_batch })
    }
}