forked from vortex-data/vortex
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathwriter.rs
More file actions
546 lines (476 loc) · 18.5 KB
/
writer.rs
File metadata and controls
546 lines (476 loc) · 18.5 KB
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
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright the Vortex contributors
use std::io;
use std::io::Write;
use std::sync::Arc;
use std::sync::atomic::AtomicU64;
use futures::future::{Fuse, LocalBoxFuture, ready};
use futures::{FutureExt, StreamExt, TryStreamExt, pin_mut, select};
use vortex_array::iter::{ArrayIterator, ArrayIteratorExt};
use vortex_array::stats::{PRUNING_STATS, Stat};
use vortex_array::stream::{ArrayStream, ArrayStreamAdapter, ArrayStreamExt, SendableArrayStream};
use vortex_array::{ArrayContext, ArrayRef};
use vortex_buffer::ByteBuffer;
use vortex_dtype::DType;
use vortex_error::{
VortexError, VortexExpect, VortexResult, vortex_bail, vortex_err, vortex_panic,
};
use vortex_io::kanal_ext::KanalExt;
use vortex_io::runtime::{BlockingRuntime, Handle};
use vortex_io::{IoBuf, VortexWrite};
use vortex_layout::LayoutStrategy;
use vortex_layout::layouts::file_stats::accumulate_stats;
use vortex_layout::sequence::{SequenceId, SequentialStreamAdapter, SequentialStreamExt};
use crate::counting::CountingVortexWrite;
use crate::footer::FileStatistics;
use crate::segments::writer::BufferedSegmentSink;
use crate::{Footer, MAGIC_BYTES, WriteStrategyBuilder};
const DEFAULT_EXCLUDE_DTYPE: bool = false;
const DEFAULT_MAX_VARIABLE_LENGTH_STATISTICS_SIZE: usize = 64;
const DEFAULT_FILE_STATISTICS: &[Stat] = PRUNING_STATS;
/// Factory for creating [`VortexWriteOptions`] with custom defaults.
///
/// This can be used to configure writer options before acquiring a handle, where we later reuse the options but need to source an available handle.
///
/// This factory maintains the default behaviour of [`VortexWriteOptions::default`].
#[derive(Clone)]
pub struct VortexWriteOptionsFactory {
strategy: Option<Arc<dyn LayoutStrategy>>,
exclude_dtype: Option<bool>,
max_variable_length_statistics_size: Option<usize>,
file_statistics: Option<Vec<Stat>>,
}
impl std::fmt::Debug for VortexWriteOptionsFactory {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("VortexWriteOptions")
.field("exclude_dtype", &self.exclude_dtype)
.field(
"max_variable_length_statistics_size",
&self.max_variable_length_statistics_size,
)
.field("file_statistics", &self.file_statistics)
.finish()
}
}
impl Default for VortexWriteOptionsFactory {
fn default() -> Self {
Self {
strategy: None,
exclude_dtype: None,
max_variable_length_statistics_size: None,
file_statistics: None,
}
}
}
impl VortexWriteOptionsFactory {
/// Create a new builder with default settings.
pub fn new() -> Self {
Self::default()
}
/// Replace the default layout strategy with the provided one.
pub fn with_strategy(mut self, strategy: Arc<dyn LayoutStrategy>) -> Self {
self.strategy = Some(strategy);
self
}
/// Exclude the DType from the Vortex file. You must provide the DType to the reader.
///
/// See [`VortexWriteOptions::exclude_dtype`] for details.
pub fn exclude_dtype(mut self) -> Self {
self.exclude_dtype = Some(true);
self
}
pub fn with_max_variable_length_statistics_size(mut self, size: usize) -> Self {
self.max_variable_length_statistics_size = Some(size);
self
}
/// Configure which statistics to compute at the file-level.
///
/// See [`VortexWriteOptions::with_file_statistics`] for details.
pub fn with_file_statistics(mut self, stats: Vec<Stat>) -> Self {
self.file_statistics = Some(stats);
self
}
/// Build the [`VortexWriteOptions`] with the configured settings.
///
/// Finds an appropriate [`Handle`] automatically.
pub fn build(&self) -> VortexWriteOptions {
VortexWriteOptions {
strategy: self
.strategy
.clone()
.unwrap_or_else(|| WriteStrategyBuilder::new().build()),
exclude_dtype: self.exclude_dtype.clone().unwrap_or(DEFAULT_EXCLUDE_DTYPE),
max_variable_length_statistics_size: self
.max_variable_length_statistics_size
.clone()
.unwrap_or(DEFAULT_MAX_VARIABLE_LENGTH_STATISTICS_SIZE),
file_statistics: self
.file_statistics
.clone()
.unwrap_or_else(|| DEFAULT_FILE_STATISTICS.to_vec()),
handle: Handle::find(),
}
}
}
/// Configure a new writer, which can eventually be used to write an [`ArrayStream`] into a sink that implements [`VortexWrite`].
///
/// Unless overridden, the default [write strategy][crate::WriteStrategyBuilder] will be used with no
/// additional configuration.
pub struct VortexWriteOptions {
strategy: Arc<dyn LayoutStrategy>,
exclude_dtype: bool,
max_variable_length_statistics_size: usize,
file_statistics: Vec<Stat>,
handle: Option<Handle>,
}
impl Default for VortexWriteOptions {
fn default() -> Self {
Self {
strategy: WriteStrategyBuilder::new().build(),
exclude_dtype: DEFAULT_EXCLUDE_DTYPE,
file_statistics: DEFAULT_FILE_STATISTICS.to_vec(),
max_variable_length_statistics_size: DEFAULT_MAX_VARIABLE_LENGTH_STATISTICS_SIZE,
handle: Handle::find(),
}
}
}
impl VortexWriteOptions {
/// Configure a [`Handle`] for driving async tasks.
///
/// If not provided, a handle will try to be inferred from [`Handle::find`].
pub fn with_handle(mut self, handle: Handle) -> Self {
self.handle = Some(handle);
self
}
/// See [`VortexWriteOptions::with_handle`].
pub fn with_some_handle(mut self, handle: Option<Handle>) -> Self {
self.handle = handle.or(self.handle);
self
}
/// Replace the default layout strategy with the provided one.
pub fn with_strategy(mut self, strategy: Arc<dyn LayoutStrategy>) -> Self {
self.strategy = strategy;
self
}
/// Exclude the DType from the Vortex file. You must provide the DType to the reader.
// TODO(ngates): Should we store some sort of DType checksum to make sure the one passed at
// read-time is sane? I guess most layouts will have some reasonable validation.
pub fn exclude_dtype(mut self) -> Self {
self.exclude_dtype = true;
self
}
/// Configure which statistics to compute at the file-level.
pub fn with_file_statistics(mut self, file_statistics: Vec<Stat>) -> Self {
self.file_statistics = file_statistics;
self
}
}
impl VortexWriteOptions {
/// Drop into the blocking writer API using the given runtime.
pub fn blocking<B: BlockingRuntime + Default>(self) -> BlockingWrite<B> {
self.with_blocking(B::default())
}
/// Drop into the blocking writer API using the given runtime.
pub fn with_blocking<B: BlockingRuntime>(self, runtime: B) -> BlockingWrite<B> {
if self.handle.is_some() {
vortex_panic!("Must not provide or infer a Handle when using the blocking writer API")
}
BlockingWrite {
options: self,
runtime,
}
}
/// Write an [`ArrayStream`] as a Vortex file.
///
/// Note that buffers are flushed as soon as they are available with no buffering, the caller
/// is responsible for deciding how to configure buffering on the underlying `Write` sink.
pub async fn write<W: VortexWrite + Unpin, S: ArrayStream + Send + 'static>(
self,
write: W,
stream: S,
) -> VortexResult<WriteSummary> {
self.write_internal(write, ArrayStreamExt::boxed(stream))
.await
}
async fn write_internal<W: VortexWrite + Unpin>(
self,
mut write: W,
stream: SendableArrayStream,
) -> VortexResult<WriteSummary> {
let Some(handle) = self.handle else {
vortex_panic!("Must provide a Handle to use the async writer API");
};
// Set up a Context to capture the encodings used in the file.
let ctx = ArrayContext::empty();
let dtype = stream.dtype().clone();
let (mut ptr, eof) = SequenceId::root().split();
let stream = SequentialStreamAdapter::new(
dtype.clone(),
stream
.try_filter(|chunk| ready(!chunk.is_empty()))
.map(move |result| result.map(|chunk| (ptr.advance(), chunk))),
)
.sendable();
let (file_stats, stream) = accumulate_stats(
stream,
self.file_statistics.clone().into(),
self.max_variable_length_statistics_size,
);
// First, write the magic bytes.
write.write_all(ByteBuffer::copy_from(MAGIC_BYTES)).await?;
let mut position = MAGIC_BYTES.len() as u64;
// Create a channel to send buffers from the segment sink to the output stream.
let (send, recv) = kanal::bounded_async(1);
let segments = Arc::new(BufferedSegmentSink::new(send, position));
// We spawn the layout future so it is driven in the background while we write the
// buffer stream, so we don't need to poll it until all buffers have been drained.
let ctx2 = ctx.clone();
let layout_fut = handle.spawn_nested(|h| async move {
let layout = self
.strategy
.write_stream(ctx2, segments.clone(), stream, eof, h)
.await?;
Ok::<_, VortexError>((layout, segments.segment_specs()))
});
// Flush buffers as they arrive
let recv_stream = recv.into_stream();
pin_mut!(recv_stream);
while let Some(buffer) = recv_stream.next().await {
if buffer.is_empty() {
continue;
}
position += buffer.len() as u64;
write.write_all(buffer).await?;
}
let (layout, segment_specs) = layout_fut.await?;
// Assemble the Footer object now that we have all the segments.
let footer = Footer::new(
layout.clone(),
segment_specs,
if self.file_statistics.is_empty() {
None
} else {
Some(FileStatistics(file_stats.stats_sets().into()))
},
ctx,
);
// Emit the footer buffers and EOF.
let footer_buffers = footer
.clone()
.into_serializer()
.with_offset(position)
.with_exclude_dtype(self.exclude_dtype)
.serialize()?;
for buffer in footer_buffers {
position += buffer.len() as u64;
write.write_all(buffer).await?;
}
write.flush().await?;
Ok(WriteSummary {
footer,
size: position,
})
}
/// Create a push-based [`Writer`] that can be used to incrementally write arrays to the file.
pub fn writer<'w, W: VortexWrite + Unpin + 'w>(self, write: W, dtype: DType) -> Writer<'w> {
// Create a channel for sending arrays to the layout task.
let (arrays_send, arrays_recv) = kanal::bounded_async(1);
let arrays =
ArrayStreamExt::boxed(ArrayStreamAdapter::new(dtype, arrays_recv.into_stream()));
let write = CountingVortexWrite::new(write);
let bytes_written = write.counter();
let strategy = self.strategy.clone();
let future = self.write(write, arrays).boxed_local().fuse();
Writer {
arrays: Some(arrays_send),
future,
bytes_written,
strategy,
}
}
}
/// An async API for writing Vortex files.
pub struct Writer<'w> {
// The input channel for sending arrays to the writer.
arrays: Option<kanal::AsyncSender<VortexResult<ArrayRef>>>,
// The writer task that ultimately produces the footer.
future: Fuse<LocalBoxFuture<'w, VortexResult<WriteSummary>>>,
// The bytes written so far.
bytes_written: Arc<AtomicU64>,
// The layout strategy that is being used for the write.
strategy: Arc<dyn LayoutStrategy>,
}
impl Writer<'_> {
/// Push a new chunk into the writer.
pub async fn push(&mut self, chunk: ArrayRef) -> VortexResult<()> {
let arrays = self.arrays.clone().vortex_expect("missing arrays sender");
let send_fut = async move { arrays.send(Ok(chunk)).await }.fuse();
pin_mut!(send_fut);
// We poll the writer future to continue writing bytes to the output, while waiting for
// enough room to push the next chunk into the channel.
select! {
result = send_fut => {
// If the send future failed, the writer has failed or panicked.
if result.is_err() {
return Err(self.handle_failed_task().await);
}
},
result = &mut self.future => {
// Under normal operation, the writer future should never complete until
// finish() is called. Therefore, we can assume the writer has failed.
// The writer future has failed, we need to propagate the error.
match result {
Ok(_) => vortex_bail!("Internal error: writer future completed early"),
Err(e) => return Err(e),
}
}
}
Ok(())
}
/// Push an entire [`ArrayStream`] into the writer, consuming it.
///
/// A task is spawned to consume the stream and push it into the writer, with the current
/// thread being used to write buffers to the output.
pub async fn push_stream(&mut self, mut stream: SendableArrayStream) -> VortexResult<()> {
let arrays = self.arrays.clone().vortex_expect("missing arrays sender");
let stream_fut = async move {
while let Some(chunk) = stream.next().await {
arrays.send(chunk).await?;
}
Ok::<_, kanal::SendError>(())
}
.fuse();
pin_mut!(stream_fut);
// We poll the writer future to continue writing bytes to the output, while waiting for
// enough room to push the stream into the channel.
select! {
result = stream_fut => {
if let Err(_send_err) = result {
// If the send future failed, the writer has failed or panicked.
return Err(self.handle_failed_task().await);
}
}
result = &mut self.future => {
// Under normal operation, the writer future should never complete until
// finish() is called. Therefore, we can assume the writer has failed.
// The writer future has failed, we need to propagate the error.
match result {
Ok(_) => vortex_bail!("Internal error: writer future completed early"),
Err(e) => return Err(e),
}
}
}
Ok(())
}
/// Returns the number of bytes written to the file so far.
pub fn bytes_written(&self) -> u64 {
self.bytes_written
.load(std::sync::atomic::Ordering::Relaxed)
}
/// Returns the number of bytes currently buffered by the layout writers.
pub fn buffered_bytes(&self) -> u64 {
self.strategy.buffered_bytes()
}
/// Finish writing the Vortex file, flushing any remaining buffers and returning the
/// new file's footer.
pub async fn finish(mut self) -> VortexResult<WriteSummary> {
// Drop the input channel to signal EOF.
drop(self.arrays.take());
// Await the future task.
self.future.await
}
/// Assuming the writer task has failed, await it to get the error.
async fn handle_failed_task(&mut self) -> VortexError {
match (&mut self.future).await {
Ok(_) => vortex_err!(
"Internal error: writer task completed successfully but write future finished early"
),
Err(e) => e,
}
}
}
/// A blocking API for writing Vortex files.
pub struct BlockingWrite<B: BlockingRuntime> {
options: VortexWriteOptions,
runtime: B,
}
impl<B: BlockingRuntime> BlockingWrite<B> {
/// Write a Vortex file into the given `Write` sink.
pub fn write<W: Write + Unpin>(
self,
write: W,
iter: impl ArrayIterator + Send + 'static,
) -> VortexResult<WriteSummary> {
self.runtime.block_on(|handle| async move {
self.options
.with_handle(handle)
.write(BlockingWriteAdapter(write), iter.into_array_stream())
.await
})
}
pub fn writer<'w, W: Write + Unpin + 'w>(
self,
write: W,
dtype: DType,
) -> BlockingWriter<'w, B> {
BlockingWriter {
writer: self
.options
.with_handle(self.runtime.handle())
.writer(BlockingWriteAdapter(write), dtype),
runtime: self.runtime,
}
}
}
/// A blocking adapter around a [`Writer`], allowing incremental writing of arrays to a Vortex file.
pub struct BlockingWriter<'w, B: BlockingRuntime> {
runtime: B,
writer: Writer<'w>,
}
impl<B: BlockingRuntime> BlockingWriter<'_, B> {
pub fn push(&mut self, chunk: ArrayRef) -> VortexResult<()> {
self.runtime.block_on(|_| self.writer.push(chunk))
}
pub fn bytes_written(&self) -> u64 {
self.writer.bytes_written()
}
pub fn buffered_bytes(&self) -> u64 {
self.writer.buffered_bytes()
}
pub fn finish(self) -> VortexResult<WriteSummary> {
self.runtime.block_on(|_| self.writer.finish())
}
}
// TODO(ngates): this blocking API may change, for now we just run blocking I/O inline.
struct BlockingWriteAdapter<W>(W);
impl<W: Write + Unpin> VortexWrite for BlockingWriteAdapter<W> {
async fn write_all<B: IoBuf>(&mut self, buffer: B) -> io::Result<B> {
self.0.write_all(buffer.as_slice())?;
Ok(buffer)
}
fn flush(&mut self) -> impl Future<Output = io::Result<()>> {
ready(self.0.flush())
}
fn shutdown(&mut self) -> impl Future<Output = io::Result<()>> {
ready(Ok(()))
}
}
pub struct WriteSummary {
footer: Footer,
size: u64,
// TODO(ngates): add a checksum
}
impl WriteSummary {
/// The footer of the written Vortex file.
pub fn footer(&self) -> &Footer {
&self.footer
}
/// The total size of the written Vortex file in bytes.
pub fn size(&self) -> u64 {
self.size
}
/// The footer of the written Vortex file.
pub fn row_count(&self) -> u64 {
self.footer.row_count()
}
}