Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions vortex-datafusion/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ object_store = { workspace = true }
tokio = { workspace = true, features = ["rt-multi-thread", "fs"] }
tokio-stream = { workspace = true }
tracing = { workspace = true }
url = { workspace = true }
vortex = { workspace = true, features = ["object_store", "tokio"] }
vortex-utils = { workspace = true, features = ["dashmap"] }

Expand Down
149 changes: 149 additions & 0 deletions vortex-datafusion/tests/schema_evolution.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright the Vortex contributors

//! Test that checks we can evolve schemas in a cmpatible way across files.

use arrow_schema::{DataType, Field, Schema, SchemaRef};
use datafusion::arrow::array::StringViewArray;
use datafusion::arrow::compute::concat_batches;
use datafusion::datasource::listing::{ListingOptions, ListingTable, ListingTableConfig};
use datafusion::execution::SessionStateBuilder;
use datafusion::execution::context::SessionContext;
use datafusion_common::arrow::array::{ArrayRef as ArrowArrayRef, RecordBatch};
use datafusion_common::record_batch;
use datafusion_datasource::ListingTableUrl;
use datafusion_expr::col;
use object_store::ObjectStore;
use object_store::memory::InMemory;
use object_store::path::Path;
use std::sync::{Arc, LazyLock};
use url::Url;
use vortex::arrow::FromArrowArray;
use vortex::file::WriteOptionsSessionExt;
use vortex::io::{ObjectStoreWriter, VortexWrite};
use vortex::session::VortexSession;
use vortex::{ArrayRef, VortexSessionDefault};
use vortex_datafusion::{VortexFormat, VortexFormatFactory};

static SESSION: LazyLock<VortexSession> = LazyLock::new(|| VortexSession::default());

fn register_vortex_format_factory(
factory: VortexFormatFactory,
session_state_builder: &mut SessionStateBuilder,
) {
if let Some(table_factories) = session_state_builder.table_factories() {
table_factories.insert(
datafusion::common::GetExt::get_ext(&factory).to_uppercase(), // Has to be uppercase
Arc::new(datafusion::datasource::provider::DefaultTableFactory::new()),
);
}

if let Some(file_formats) = session_state_builder.file_formats() {
file_formats.push(Arc::new(factory));
}
}

fn make_session_ctx() -> (SessionContext, Arc<dyn ObjectStore>) {
let factory: VortexFormatFactory = VortexFormatFactory::new();
let mut session_state_builder = SessionStateBuilder::new().with_default_features();
register_vortex_format_factory(factory, &mut session_state_builder);
let ctx = SessionContext::new_with_state(session_state_builder.build());
let store = Arc::new(InMemory::new());
ctx.register_object_store(&Url::parse("s3://in-memory/").unwrap(), store.clone());

(ctx, store)
}

async fn write_file(store: &Arc<dyn ObjectStore>, path: &str, records: &RecordBatch) {
let array = ArrayRef::from_arrow(records, false);
let path = Path::from_url_path(path).unwrap();
let mut write = ObjectStoreWriter::new(store.clone(), &path).await.unwrap();
SESSION
.write_options()
.write(&mut write, array.to_array_stream())
.await
.unwrap();
write.shutdown().await.unwrap();
}

#[tokio::test]
async fn test_filter_with_schema_evolution() {
let (ctx, store) = make_session_ctx();

// file1 only contains field "a"
write_file(
&store,
"files/file1.vortex",
&record_batch!(("a", Utf8, vec![Some("one"), Some("two"), Some("three")])).unwrap(),
)
.await;

// file2 only contains field "b"
write_file(
&store,
"files/file2.vortex",
&record_batch!(("b", Utf8, vec![Some("four"), Some("five"), Some("six")])).unwrap(),
)
.await;

// Read the table back as Vortex
let table_url = ListingTableUrl::parse("s3://in-memory/files").unwrap();
let list_opts = ListingOptions::new(Arc::new(VortexFormat::new(SESSION.clone())))
.with_session_config_options(ctx.state().config())
.with_file_extension("vortex");

let table = ListingTable::try_new(
ListingTableConfig::new(table_url)
.with_listing_options(list_opts)
.infer_schema(&ctx.state())
.await
.unwrap(),
)
.unwrap();

let table = Arc::new(table);

let df = ctx.read_table(table).unwrap();

let table_schema = Arc::new(df.schema().as_arrow().clone());

assert_eq!(
table_schema.as_ref(),
&Schema::new(vec![
Field::new("a", DataType::Utf8View, true),
Field::new("b", DataType::Utf8View, true),
])
);

// Filter the result to only the values from the first file
let result = df
.filter(col("a").is_not_null())
.unwrap()
.collect()
.await
.unwrap();
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

test fails here with

thread 'test_filter_with_schema_evolution' panicked at vortex-datafusion/tests/schema_evolution.rs:124:10:
called `Result::unwrap()` on an `Err` value: ArrowError(SchemaError("Unable to get field named \"a\". Valid fields: [\"b\"]"), Some(""))

let table = concat_batches(&table_schema, result.iter()).unwrap();

// We read back the full table, with nulls filled in for missing fields
assert_eq!(
table,
record_batch(
&table_schema,
vec![
Arc::new(StringViewArray::from(vec![
Some("one"),
Some("two"),
Some("three"),
])) as ArrowArrayRef,
Arc::new(StringViewArray::from(vec![Option::<&str>::None, None, None])) as ArrowArrayRef,
]
)
);
}

fn record_batch(
schema: &SchemaRef,
fields: impl IntoIterator<Item = ArrowArrayRef>,
) -> RecordBatch {
RecordBatch::try_new(schema.clone(), fields.into_iter().collect()).unwrap()
}
Loading