• Home
  • Features
  • Pricing
  • Docs
  • Announcements
  • Sign In

vortex-data / vortex / 16565982454

28 Jul 2025 09:57AM UTC coverage: 81.767% (-0.02%) from 81.789%
16565982454

Pull #4031

github

web-flow
Merge 73cac1802 into a0efcbe7a
Pull Request #4031: zip compute function

104 of 138 new or added lines in 1 file covered. (75.36%)

29 existing lines in 3 files now uncovered.

43326 of 52987 relevant lines covered (81.77%)

171359.09 hits per line

Source File
Press 'n' to go to next uncovered line, 'b' for previous

79.75
/vortex-datafusion/src/persistent/format.rs
1
// SPDX-License-Identifier: Apache-2.0
2
// SPDX-FileCopyrightText: Copyright the Vortex contributors
3

4
use std::any::Any;
5
use std::fmt::{Debug, Formatter};
6
use std::sync::Arc;
7

8
use async_trait::async_trait;
9
use datafusion::arrow::datatypes::{Schema, SchemaRef};
10
use datafusion::catalog::Session;
11
use datafusion::common::parsers::CompressionTypeVariant;
12
use datafusion::common::runtime::SpawnedTask;
13
use datafusion::common::stats::Precision;
14
use datafusion::common::{
15
    ColumnStatistics, DataFusionError, GetExt, Result as DFResult, Statistics,
16
    config_datafusion_err, not_impl_err,
17
};
18
use datafusion::datasource::file_format::file_compression_type::FileCompressionType;
19
use datafusion::datasource::file_format::{FileFormat, FileFormatFactory};
20
use datafusion::datasource::physical_plan::{
21
    FileScanConfig, FileScanConfigBuilder, FileSinkConfig, FileSource,
22
};
23
use datafusion::datasource::sink::DataSinkExec;
24
use datafusion::datasource::source::DataSourceExec;
25
use datafusion::logical_expr::dml::InsertOp;
26
use datafusion::physical_expr::LexRequirement;
27
use datafusion::physical_plan::ExecutionPlan;
28
use futures::{FutureExt, StreamExt as _, TryStreamExt as _, stream};
29
use itertools::Itertools;
30
use object_store::{ObjectMeta, ObjectStore};
31
use vortex::dtype::DType;
32
use vortex::dtype::arrow::FromArrowType;
33
use vortex::error::{VortexExpect, VortexResult, vortex_err};
34
use vortex::file::VORTEX_FILE_EXTENSION;
35
use vortex::metrics::VortexMetrics;
36
use vortex::session::VortexSession;
37
use vortex::stats;
38
use vortex::stats::{Stat, StatsProviderExt, StatsSet};
39

40
use super::cache::VortexFileCache;
41
use super::sink::VortexSink;
42
use super::source::VortexSource;
43
use crate::PrecisionExt as _;
44
use crate::convert::TryToDataFusion;
45

46
/// Vortex implementation of a DataFusion [`FileFormat`].
47
pub struct VortexFormat {
48
    session: Arc<VortexSession>,
49
    file_cache: VortexFileCache,
50
    opts: VortexFormatOptions,
51
}
52

53
impl Debug for VortexFormat {
54
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
×
55
        f.debug_struct("VortexFormat")
×
56
            .field("opts", &self.opts)
×
57
            .finish()
×
58
    }
×
59
}
60

61
/// Options to configure the [`VortexFormat`].
62
#[derive(Debug)]
63
pub struct VortexFormatOptions {
64
    /// The size of the in-memory [`vortex::file::Footer`] cache.
65
    pub footer_cache_size_mb: usize,
66
    /// The size of the in-memory segment cache.
67
    pub segment_cache_size_mb: usize,
68
}
69

70
impl Default for VortexFormatOptions {
71
    fn default() -> Self {
20✔
72
        Self {
20✔
73
            footer_cache_size_mb: 64,
20✔
74
            segment_cache_size_mb: 0,
20✔
75
        }
20✔
76
    }
20✔
77
}
78

79
/// Minimal factory to create [`VortexFormat`] instances.
80
#[derive(Default, Debug)]
81
pub struct VortexFormatFactory {
82
    session: Arc<VortexSession>,
83
}
84

85
impl GetExt for VortexFormatFactory {
86
    fn get_ext(&self) -> String {
18✔
87
        VORTEX_FILE_EXTENSION.to_string()
18✔
88
    }
18✔
89
}
90

91
impl FileFormatFactory for VortexFormatFactory {
92
    #[allow(clippy::disallowed_types)]
93
    fn create(
3✔
94
        &self,
3✔
95
        _state: &dyn Session,
3✔
96
        format_options: &std::collections::HashMap<String, String>,
3✔
97
    ) -> DFResult<Arc<dyn FileFormat>> {
3✔
98
        if !format_options.is_empty() {
3✔
99
            return Err(config_datafusion_err!(
1✔
100
                "Vortex tables don't support any options"
1✔
101
            ));
1✔
102
        }
2✔
103

104
        Ok(Arc::new(VortexFormat::new(self.session.clone())))
2✔
105
    }
3✔
106

107
    fn default(&self) -> Arc<dyn FileFormat> {
×
108
        Arc::new(VortexFormat::default())
×
109
    }
×
110

111
    fn as_any(&self) -> &dyn Any {
×
112
        self
×
113
    }
×
114
}
115

116
impl Default for VortexFormat {
117
    fn default() -> Self {
18✔
118
        Self::new(Arc::new(VortexSession::default()))
18✔
119
    }
18✔
120
}
121

122
impl VortexFormat {
123
    /// Create a new instance of the [`VortexFormat`].
124
    pub fn new(session: Arc<VortexSession>) -> Self {
20✔
125
        let opts = VortexFormatOptions::default();
20✔
126
        Self {
20✔
127
            session: session.clone(),
20✔
128
            file_cache: VortexFileCache::new(
20✔
129
                opts.footer_cache_size_mb,
20✔
130
                opts.segment_cache_size_mb,
20✔
131
                session,
20✔
132
            ),
20✔
133
            opts,
20✔
134
        }
20✔
135
    }
20✔
136

137
    /// Return the format specific configuration
138
    pub fn options(&self) -> &VortexFormatOptions {
×
139
        &self.opts
×
140
    }
×
141
}
142

143
#[async_trait]
144
impl FileFormat for VortexFormat {
145
    fn as_any(&self) -> &dyn Any {
×
146
        self
×
147
    }
×
148

UNCOV
149
    fn compression_type(&self) -> Option<FileCompressionType> {
×
UNCOV
150
        None
×
UNCOV
151
    }
×
152

153
    fn get_ext(&self) -> String {
22✔
154
        VORTEX_FILE_EXTENSION.to_string()
22✔
155
    }
22✔
156

157
    fn get_ext_with_compression(
×
158
        &self,
×
159
        file_compression_type: &FileCompressionType,
×
160
    ) -> DFResult<String> {
×
161
        match file_compression_type.get_variant() {
×
UNCOV
162
            CompressionTypeVariant::UNCOMPRESSED => Ok(self.get_ext()),
×
163
            _ => Err(DataFusionError::Internal(
×
UNCOV
164
                "Vortex does not support file level compression.".into(),
×
UNCOV
165
            )),
×
166
        }
UNCOV
167
    }
×
168

169
    async fn infer_schema(
170
        &self,
171
        state: &dyn Session,
172
        store: &Arc<dyn ObjectStore>,
173
        objects: &[ObjectMeta],
174
    ) -> DFResult<SchemaRef> {
4✔
175
        let mut file_schemas = stream::iter(objects.iter().cloned())
2✔
176
            .map(|o| {
2✔
177
                let store = store.clone();
2✔
178
                let cache = self.file_cache.clone();
2✔
179
                SpawnedTask::spawn(async move {
2✔
180
                    let vxf = cache.try_get(&o, store).await?;
2✔
181
                    let inferred_schema = vxf.dtype().to_arrow_schema()?;
2✔
182
                    VortexResult::Ok((o.location, inferred_schema))
2✔
183
                })
2✔
184
                .map(|f| f.vortex_expect("Failed to spawn infer_schema"))
2✔
185
            })
2✔
186
            .buffer_unordered(state.config_options().execution.meta_fetch_concurrency)
2✔
187
            .try_collect::<Vec<_>>()
2✔
188
            .await
2✔
189
            .map_err(|e| DataFusionError::Execution(format!("Failed to infer schema: {e}")))?;
2✔
190

191
        // Get consistent order of schemas for `Schema::try_merge`, as some filesystems don't have deterministic listing orders
192
        file_schemas.sort_by(|(l1, _), (l2, _)| l1.cmp(l2));
2✔
193
        let file_schemas = file_schemas.into_iter().map(|(_, schema)| schema);
2✔
194

195
        Ok(Arc::new(Schema::try_merge(file_schemas)?))
2✔
196
    }
4✔
197

198
    #[cfg_attr(feature = "tracing", tracing::instrument(skip_all, fields(location = object.location.as_ref()
199
    )))]
200
    async fn infer_stats(
201
        &self,
202
        _state: &dyn Session,
203
        store: &Arc<dyn ObjectStore>,
204
        table_schema: SchemaRef,
205
        object: &ObjectMeta,
206
    ) -> DFResult<Statistics> {
52✔
207
        let object = object.clone();
28✔
208
        let store = store.clone();
28✔
209
        let cache = self.file_cache.clone();
28✔
210
        SpawnedTask::spawn(async move {
28✔
211
            let vxf = cache.try_get(&object, store.clone()).await.map_err(|e| {
28✔
212
                DataFusionError::Execution(format!(
×
UNCOV
213
                    "Failed to open Vortex file {}: {e}",
×
UNCOV
214
                    object.location
×
UNCOV
215
                ))
×
UNCOV
216
            })?;
×
217

218
            let struct_dtype = vxf
28✔
219
                .dtype()
28✔
220
                .as_struct()
28✔
221
                .vortex_expect("dtype is not a struct");
28✔
222

223
            // Evaluate the statistics for each column that we are able to return to DataFusion.
224
            let Some(file_stats) = vxf.file_stats() else {
28✔
225
                // If the file has no column stats, the best we can do is return a row count.
226
                return Ok(Statistics {
227
                    num_rows: Precision::Exact(
228
                        usize::try_from(vxf.row_count())
×
229
                            .map_err(|_| vortex_err!("Row count overflow"))
×
UNCOV
230
                            .vortex_expect("Row count overflow"),
×
231
                    ),
UNCOV
232
                    total_byte_size: Precision::Absent,
×
UNCOV
233
                    column_statistics: vec![ColumnStatistics::default(); struct_dtype.nfields()],
×
234
                });
235
            };
236

237
            let stats = table_schema
28✔
238
                .fields()
28✔
239
                .iter()
28✔
240
                .map(|field| struct_dtype.find(field.name()))
168✔
241
                .map(|idx| match idx {
168✔
UNCOV
242
                    None => StatsSet::default(),
×
243
                    Some(id) => file_stats[id].clone(),
168✔
244
                })
168✔
245
                .collect_vec();
28✔
246

247
            let total_byte_size = stats
28✔
248
                .iter()
28✔
249
                .map(|stats_set| {
168✔
250
                    stats_set
168✔
251
                        .get_as::<usize>(Stat::UncompressedSizeInBytes)
168✔
252
                        .unwrap_or_else(|| stats::Precision::inexact(0_usize))
168✔
253
                })
168✔
254
                .fold(stats::Precision::exact(0_usize), |acc, stats_set| {
168✔
255
                    acc.zip(stats_set).map(|(acc, stats_set)| acc + stats_set)
168✔
256
                });
168✔
257

258
            // Sum up the total byte size across all the columns.
259
            let total_byte_size = total_byte_size.to_df();
28✔
260

261
            let column_statistics = stats
28✔
262
                .into_iter()
28✔
263
                .zip(table_schema.fields().iter())
28✔
264
                .map(|(stats_set, field)| {
168✔
265
                    let null_count = stats_set.get_as::<usize>(Stat::NullCount);
168✔
266
                    let min = stats_set
168✔
267
                        .get_scalar(Stat::Min, &DType::from_arrow(field.as_ref()))
168✔
268
                        .and_then(|n| n.map(|n| n.try_to_df().ok()).transpose());
168✔
269

270
                    let max = stats_set
168✔
271
                        .get_scalar(Stat::Max, &DType::from_arrow(field.as_ref()))
168✔
272
                        .and_then(|n| n.map(|n| n.try_to_df().ok()).transpose());
168✔
273

274
                    ColumnStatistics {
275
                        null_count: null_count.to_df(),
168✔
276
                        max_value: max.to_df(),
168✔
277
                        min_value: min.to_df(),
168✔
278
                        sum_value: Precision::Absent,
168✔
279
                        distinct_count: stats_set
168✔
280
                            .get_as::<bool>(Stat::IsConstant)
168✔
281
                            .and_then(|is_constant| {
168✔
UNCOV
282
                                is_constant.as_exact().map(|_| Precision::Exact(1))
×
UNCOV
283
                            })
×
284
                            .unwrap_or(Precision::Absent),
168✔
285
                    }
286
                })
168✔
287
                .collect::<Vec<_>>();
28✔
288

289
            Ok(Statistics {
290
                num_rows: Precision::Exact(
291
                    usize::try_from(vxf.row_count())
28✔
292
                        .map_err(|_| vortex_err!("Row count overflow"))
28✔
293
                        .vortex_expect("Row count overflow"),
28✔
294
                ),
295
                total_byte_size,
28✔
296
                column_statistics,
28✔
297
            })
298
        })
28✔
299
        .await
28✔
300
        .vortex_expect("Failed to spawn infer_stats")
28✔
301
    }
52✔
302

303
    async fn create_physical_plan(
304
        &self,
305
        _state: &dyn Session,
306
        file_scan_config: FileScanConfig,
307
    ) -> DFResult<Arc<dyn ExecutionPlan>> {
360✔
308
        if file_scan_config
180✔
309
            .file_groups
180✔
310
            .iter()
180✔
311
            .flat_map(|fg| fg.files())
182✔
312
            .any(|f| f.range.is_some())
182✔
313
        {
314
            return not_impl_err!("File level partitioning isn't implemented yet for Vortex");
×
315
        }
180✔
316

317
        if !file_scan_config.table_partition_cols.is_empty() {
180✔
318
            return not_impl_err!("Hive style partitioning isn't implemented yet for Vortex");
×
319
        }
180✔
320

321
        if !file_scan_config.output_ordering.is_empty() {
180✔
UNCOV
322
            return not_impl_err!("Vortex doesn't support output ordering");
×
323
        }
180✔
324

325
        let source = VortexSource::new(self.file_cache.clone(), self.session.metrics().clone());
180✔
326
        Ok(DataSourceExec::from_data_source(
180✔
327
            FileScanConfigBuilder::from(file_scan_config)
180✔
328
                .with_source(Arc::new(source))
180✔
329
                .build(),
180✔
330
        ))
180✔
331
    }
360✔
332

333
    async fn create_writer_physical_plan(
334
        &self,
335
        input: Arc<dyn ExecutionPlan>,
336
        _state: &dyn Session,
337
        conf: FileSinkConfig,
338
        order_requirements: Option<LexRequirement>,
339
    ) -> DFResult<Arc<dyn ExecutionPlan>> {
4✔
340
        if conf.insert_op != InsertOp::Append {
2✔
341
            return not_impl_err!("Overwrites are not implemented yet for Vortex");
×
342
        }
2✔
343

344
        if !conf.table_partition_cols.is_empty() {
2✔
UNCOV
345
            return not_impl_err!("Hive style partitioning isn't implemented yet for Vortex");
×
346
        }
2✔
347

348
        let schema = conf.output_schema().clone();
2✔
349
        let sink = Arc::new(VortexSink::new(conf, schema));
2✔
350

351
        Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _)
2✔
352
    }
4✔
353

354
    fn file_source(&self) -> Arc<dyn FileSource> {
180✔
355
        Arc::new(VortexSource::new(
180✔
356
            self.file_cache.clone(),
180✔
357
            VortexMetrics::default(),
180✔
358
        ))
180✔
359
    }
180✔
360
}
361

362
#[cfg(test)]
363
mod tests {
364
    use datafusion::execution::SessionStateBuilder;
365
    use datafusion::prelude::SessionContext;
366
    use tempfile::TempDir;
367

368
    use super::*;
369
    use crate::persistent::register_vortex_format_factory;
370

371
    #[tokio::test]
372
    async fn create_table() {
1✔
373
        let dir = TempDir::new().unwrap();
1✔
374

375
        let factory: VortexFormatFactory = Default::default();
1✔
376
        let mut session_state_builder = SessionStateBuilder::new().with_default_features();
1✔
377
        register_vortex_format_factory(factory, &mut session_state_builder);
1✔
378
        let session = SessionContext::new_with_state(session_state_builder.build());
1✔
379

380
        let df = session
1✔
381
            .sql(&format!(
1✔
382
                "CREATE EXTERNAL TABLE my_tbl \
1✔
383
                (c1 VARCHAR NOT NULL, c2 INT NOT NULL) \
1✔
384
                STORED AS vortex LOCATION '{}'",
1✔
385
                dir.path().to_str().unwrap()
1✔
386
            ))
1✔
387
            .await
1✔
388
            .unwrap();
1✔
389

390
        assert_eq!(df.count().await.unwrap(), 0);
1✔
391
    }
1✔
392

393
    #[tokio::test]
394
    #[should_panic]
395
    async fn fail_table_config() {
1✔
396
        let dir = TempDir::new().unwrap();
1✔
397

398
        let factory: VortexFormatFactory = Default::default();
1✔
399
        let mut session_state_builder = SessionStateBuilder::new().with_default_features();
1✔
400
        register_vortex_format_factory(factory, &mut session_state_builder);
1✔
401
        let session = SessionContext::new_with_state(session_state_builder.build());
1✔
402

403
        session
1✔
404
            .sql(&format!(
1✔
405
                "CREATE EXTERNAL TABLE my_tbl \
1✔
406
                (c1 VARCHAR NOT NULL, c2 INT NOT NULL) \
1✔
407
                STORED AS vortex LOCATION '{}' \
1✔
408
                OPTIONS( some_key 'value' );",
1✔
409
                dir.path().to_str().unwrap()
1✔
410
            ))
1✔
411
            .await
1✔
412
            .unwrap()
1✔
413
            .collect()
1✔
414
            .await
1✔
415
            .unwrap();
1✔
416
    }
1✔
417
}
STATUS · Troubleshooting · Open an Issue · Sales · Support · CAREERS · ENTERPRISE · START FREE · SCHEDULE DEMO
ANNOUNCEMENTS · TWITTER · TOS & SLA · Supported CI Services · What's a CI service? · Automated Testing

© 2026 Coveralls, Inc