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
use std::{
io::{Read, Seek},
sync::Arc,
};
use futures::{
future::{try_join_all, BoxFuture},
AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt,
};
use parquet2::{
metadata::ColumnChunkMetaData,
read::{BasicDecompressor, PageIterator},
};
use crate::{
array::Array, chunk::Chunk, datatypes::Field, error::Result,
io::parquet::read::column_iter_to_arrays,
};
use super::ArrayIter;
use super::RowGroupMetaData;
pub struct RowGroupDeserializer {
num_rows: usize,
remaining_rows: usize,
column_chunks: Vec<ArrayIter<'static>>,
}
impl RowGroupDeserializer {
pub fn new(
column_chunks: Vec<ArrayIter<'static>>,
num_rows: usize,
limit: Option<usize>,
) -> Self {
Self {
num_rows,
remaining_rows: limit.unwrap_or(usize::MAX).min(num_rows),
column_chunks,
}
}
pub fn num_rows(&self) -> usize {
self.num_rows
}
}
impl Iterator for RowGroupDeserializer {
type Item = Result<Chunk<Arc<dyn Array>>>;
fn next(&mut self) -> Option<Self::Item> {
if self.remaining_rows == 0 {
return None;
}
let chunk = self
.column_chunks
.iter_mut()
.map(|iter| {
let array = iter.next().unwrap()?;
Ok(if array.len() > self.remaining_rows {
array.slice(0, array.len() - self.remaining_rows).into()
} else {
array
})
})
.collect::<Result<Vec<_>>>()
.map(Chunk::new);
self.remaining_rows -= chunk
.as_ref()
.map(|x| x.len())
.unwrap_or(self.remaining_rows);
Some(chunk)
}
}
pub(super) fn get_field_columns<'a>(
columns: &'a [ColumnChunkMetaData],
field_name: &str,
) -> Vec<&'a ColumnChunkMetaData> {
columns
.iter()
.filter(|x| x.descriptor().path_in_schema()[0] == field_name)
.collect()
}
pub fn read_columns<'a, R: Read + Seek>(
reader: &mut R,
columns: &'a [ColumnChunkMetaData],
field_name: &str,
) -> Result<Vec<(&'a ColumnChunkMetaData, Vec<u8>)>> {
get_field_columns(columns, field_name)
.into_iter()
.map(|meta| _read_single_column(reader, meta))
.collect()
}
fn _read_single_column<'a, R>(
reader: &mut R,
meta: &'a ColumnChunkMetaData,
) -> Result<(&'a ColumnChunkMetaData, Vec<u8>)>
where
R: Read + Seek,
{
let (start, len) = meta.byte_range();
reader.seek(std::io::SeekFrom::Start(start))?;
let mut chunk = vec![0; len as usize];
reader.read_exact(&mut chunk)?;
Ok((meta, chunk))
}
async fn _read_single_column_async<'b, R, F>(
factory: F,
meta: &ColumnChunkMetaData,
) -> Result<(&ColumnChunkMetaData, Vec<u8>)>
where
R: AsyncRead + AsyncSeek + Send + Unpin,
F: Fn() -> BoxFuture<'b, std::io::Result<R>>,
{
let mut reader = factory().await?;
let (start, len) = meta.byte_range();
reader.seek(std::io::SeekFrom::Start(start)).await?;
let mut chunk = vec![0; len as usize];
reader.read_exact(&mut chunk).await?;
Result::Ok((meta, chunk))
}
pub async fn read_columns_async<
'a,
'b,
R: AsyncRead + AsyncSeek + Send + Unpin,
F: Fn() -> BoxFuture<'b, std::io::Result<R>> + Clone,
>(
factory: F,
columns: &'a [ColumnChunkMetaData],
field_name: &str,
) -> Result<Vec<(&'a ColumnChunkMetaData, Vec<u8>)>> {
let futures = get_field_columns(columns, field_name)
.into_iter()
.map(|meta| async { _read_single_column_async(factory.clone(), meta).await });
try_join_all(futures).await
}
pub fn to_deserializer<'a>(
columns: Vec<(&ColumnChunkMetaData, Vec<u8>)>,
field: Field,
num_rows: usize,
chunk_size: Option<usize>,
) -> Result<ArrayIter<'a>> {
let chunk_size = chunk_size.unwrap_or(usize::MAX).min(num_rows);
let (columns, types): (Vec<_>, Vec<_>) = columns
.into_iter()
.map(|(column_meta, chunk)| {
let pages = PageIterator::new(
std::io::Cursor::new(chunk),
column_meta.num_values(),
column_meta.compression(),
column_meta.descriptor().clone(),
Arc::new(|_, _| true),
vec![],
);
(
BasicDecompressor::new(pages, vec![]),
column_meta.descriptor().type_(),
)
})
.unzip();
column_iter_to_arrays(columns, types, field, chunk_size)
}
pub fn read_columns_many<'a, R: Read + Seek>(
reader: &mut R,
row_group: &RowGroupMetaData,
fields: Vec<Field>,
chunk_size: Option<usize>,
) -> Result<Vec<ArrayIter<'a>>> {
let field_columns = fields
.iter()
.map(|field| read_columns(reader, row_group.columns(), &field.name))
.collect::<Result<Vec<_>>>()?;
field_columns
.into_iter()
.zip(fields.into_iter())
.map(|(columns, field)| {
to_deserializer(columns, field, row_group.num_rows() as usize, chunk_size)
})
.collect()
}
pub async fn read_columns_many_async<
'a,
'b,
R: AsyncRead + AsyncSeek + Send + Unpin,
F: Fn() -> BoxFuture<'b, std::io::Result<R>> + Clone,
>(
factory: F,
row_group: &RowGroupMetaData,
fields: Vec<Field>,
chunk_size: Option<usize>,
) -> Result<Vec<ArrayIter<'a>>> {
let futures = fields
.iter()
.map(|field| read_columns_async(factory.clone(), row_group.columns(), &field.name));
let field_columns = try_join_all(futures).await?;
field_columns
.into_iter()
.zip(fields.into_iter())
.map(|(columns, field)| {
to_deserializer(columns, field, row_group.num_rows() as usize, chunk_size)
})
.collect()
}