@@ -21,22 +21,37 @@ use vortex_layout::layouts::repartition::{
21
21
} ;
22
22
use vortex_layout:: layouts:: stats:: writer:: { StatsLayoutOptions , StatsLayoutWriter } ;
23
23
use vortex_layout:: layouts:: struct_:: writer:: StructLayoutWriter ;
24
- use vortex_layout:: segments:: SegmentWriter ;
24
+ use vortex_layout:: scan:: TaskExecutor ;
25
+ use vortex_layout:: segments:: ConcurrentSegmentWriter ;
25
26
use vortex_layout:: { Layout , LayoutStrategy , LayoutWriter , LayoutWriterExt } ;
26
27
27
28
const ROW_BLOCK_SIZE : usize = 8192 ;
28
29
29
30
/// The default Vortex file layout strategy.
30
- #[ derive( Clone , Debug , Default ) ]
31
- pub struct VortexLayoutStrategy ;
31
+ #[ derive( Clone , Default ) ]
32
+ pub struct VortexLayoutStrategy {
33
+ executor : Option < Arc < dyn TaskExecutor > > ,
34
+ }
35
+
36
+ impl VortexLayoutStrategy {
37
+ #[ cfg( feature = "tokio" ) ]
38
+ pub fn with_tokio_executor ( mut self , handle : tokio:: runtime:: Handle ) -> Self {
39
+ self . executor = Some ( Arc :: new ( handle) ) ;
40
+ self
41
+ }
42
+ }
32
43
33
44
impl LayoutStrategy for VortexLayoutStrategy {
34
45
fn new_writer ( & self , ctx : & ArrayContext , dtype : & DType ) -> VortexResult < Box < dyn LayoutWriter > > {
35
46
// First, we unwrap struct arrays into their components.
36
47
if dtype. is_struct ( ) {
37
- return Ok (
38
- StructLayoutWriter :: try_new_with_strategy ( ctx, dtype, self . clone ( ) ) ?. boxed ( ) ,
39
- ) ;
48
+ return Ok ( StructLayoutWriter :: try_new_with_strategy (
49
+ ctx,
50
+ dtype,
51
+ self . executor . clone ( ) ,
52
+ self . clone ( ) ,
53
+ ) ?
54
+ . boxed ( ) ) ;
40
55
}
41
56
42
57
// We buffer arrays per column, before flushing them into a chunked layout.
@@ -135,7 +150,7 @@ struct BtrBlocksCompressedWriter {
135
150
impl LayoutWriter for BtrBlocksCompressedWriter {
136
151
async fn push_chunk (
137
152
& mut self ,
138
- segment_writer : & mut dyn SegmentWriter ,
153
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
139
154
chunk : ArrayRef ,
140
155
) -> VortexResult < ( ) > {
141
156
// Compute the stats for the chunk prior to compression
@@ -204,11 +219,17 @@ impl LayoutWriter for BtrBlocksCompressedWriter {
204
219
. await
205
220
}
206
221
207
- async fn flush ( & mut self , segment_writer : & mut dyn SegmentWriter ) -> VortexResult < ( ) > {
222
+ async fn flush (
223
+ & mut self ,
224
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
225
+ ) -> VortexResult < ( ) > {
208
226
self . child . flush ( segment_writer) . await
209
227
}
210
228
211
- async fn finish ( & mut self , segment_writer : & mut dyn SegmentWriter ) -> VortexResult < Layout > {
229
+ async fn finish (
230
+ & mut self ,
231
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
232
+ ) -> VortexResult < Layout > {
212
233
self . child . finish ( segment_writer) . await
213
234
}
214
235
}
@@ -242,7 +263,7 @@ struct BufferedWriter {
242
263
impl LayoutWriter for BufferedWriter {
243
264
async fn push_chunk (
244
265
& mut self ,
245
- segment_writer : & mut dyn SegmentWriter ,
266
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
246
267
chunk : ArrayRef ,
247
268
) -> VortexResult < ( ) > {
248
269
self . nbytes += chunk. nbytes ( ) as u64 ;
@@ -262,14 +283,20 @@ impl LayoutWriter for BufferedWriter {
262
283
Ok ( ( ) )
263
284
}
264
285
265
- async fn flush ( & mut self , segment_writer : & mut dyn SegmentWriter ) -> VortexResult < ( ) > {
286
+ async fn flush (
287
+ & mut self ,
288
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
289
+ ) -> VortexResult < ( ) > {
266
290
for chunk in self . chunks . drain ( ..) {
267
291
self . child . push_chunk ( segment_writer, chunk) . await ?;
268
292
}
269
293
self . child . flush ( segment_writer) . await
270
294
}
271
295
272
- async fn finish ( & mut self , segment_writer : & mut dyn SegmentWriter ) -> VortexResult < Layout > {
296
+ async fn finish (
297
+ & mut self ,
298
+ segment_writer : & mut dyn ConcurrentSegmentWriter ,
299
+ ) -> VortexResult < Layout > {
273
300
self . child . finish ( segment_writer) . await
274
301
}
275
302
}
0 commit comments