Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

File-based table provider for Datafusion #546

Merged
merged 25 commits into from
Aug 6, 2024
Merged
Show file tree
Hide file tree
Changes from 15 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
8 changes: 8 additions & 0 deletions Cargo.lock

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

2 changes: 1 addition & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ mimalloc = "0.1.42"
monoio = "0.2.3"
num-traits = "0.2.18"
num_enum = "0.7.2"
object_store = "0.10.1"
object_store = "0.10.2"
parquet = "52.0.0"
paste = "1.0.14"
pin-project = "1.1.5"
Expand Down
2 changes: 1 addition & 1 deletion bench-vortex/benches/datafusion_benchmark.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ use lazy_static::lazy_static;
use vortex::compress::CompressionStrategy;
use vortex::encoding::EncodingRef;
use vortex::{Array, Context, IntoArray, ToArrayData};
use vortex_datafusion::{VortexMemTable, VortexMemTableOptions};
use vortex_datafusion::memory::{VortexMemTable, VortexMemTableOptions};
use vortex_dict::DictEncoding;
use vortex_fastlanes::{BitPackedEncoding, DeltaEncoding, FoREncoding};
use vortex_sampling_compressor::compressors::bitpacked::BitPackedCompressor;
Expand Down
4 changes: 2 additions & 2 deletions bench-vortex/src/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -159,8 +159,8 @@ pub async fn read_vortex_footer_format<R: VortexReadAt>(
)
}

pub async fn take_vortex_object_store<O: ObjectStore>(
fs: &O,
pub async fn take_vortex_object_store(
fs: &Arc<dyn ObjectStore>,
path: &object_store::path::Path,
indices: &[u64],
) -> VortexResult<Array> {
Expand Down
3 changes: 2 additions & 1 deletion bench-vortex/src/tpch/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,8 @@ use datafusion::prelude::{CsvReadOptions, ParquetReadOptions, SessionContext};
use vortex::array::chunked::ChunkedArray;
use vortex::arrow::FromArrowArray;
use vortex::{Array, ArrayDType, ArrayData, IntoArray};
use vortex_datafusion::{SessionContextExt, VortexMemTableOptions};
use vortex_datafusion::memory::VortexMemTableOptions;
use vortex_datafusion::SessionContextExt;

use crate::idempotent_async;

Expand Down
21 changes: 17 additions & 4 deletions vortex-datafusion/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -10,17 +10,23 @@ include.workspace = true
edition.workspace = true
rust-version.workspace = true

[lib]
name = "vortex_datafusion"
path = "src/lib.rs"

[dependencies]
vortex-array = { path = "../vortex-array" }
vortex-dtype = { path = "../vortex-dtype" }
vortex-expr = { path = "../vortex-expr" }
vortex-error = { path = "../vortex-error" }
vortex-error = { path = "../vortex-error", features = ["datafusion"] }
vortex-serde = { path = "../vortex-serde", features = ["object_store"] }
vortex-scalar = { path = "../vortex-scalar", features = ["datafusion"] }

arrow-array = { workspace = true }
arrow-schema = { workspace = true }
async-trait = { workspace = true }

chrono = { workspace = true }
datafusion = { workspace = true }
datafusion-common = { workspace = true }
datafusion-expr = { workspace = true }
Expand All @@ -30,10 +36,17 @@ datafusion-physical-plan = { workspace = true }
futures = { workspace = true }
itertools = { workspace = true }
lazy_static = { workspace = true }
log = { workspace = true }
object_store = { workspace = true }
pin-project = { workspace = true }


[dev-dependencies]
tokio = { workspace = true, features = ["test-util"] }
anyhow = { workspace = true }
url = "2"
tempfile = "3"
AdamGS marked this conversation as resolved.
Show resolved Hide resolved
tokio = { workspace = true, features = ["test-util", "rt-multi-thread"] }


[lints]
workspace = true
# [lints]
# workspace = true
98 changes: 98 additions & 0 deletions vortex-datafusion/examples/table_provider.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
use std::env::temp_dir;
use std::sync::Arc;

use arrow_schema::{DataType, Field, Schema};
use datafusion::prelude::SessionContext;
use datafusion_execution::object_store::ObjectStoreUrl;
use object_store::local::LocalFileSystem;
use object_store::path::Path;
use object_store::ObjectStore;
use tokio::fs::OpenOptions;
use url::Url;
use vortex::array::chunked::ChunkedArray;
use vortex::array::primitive::PrimitiveArray;
use vortex::array::struct_::StructArray;
use vortex::array::varbin::VarBinArray;
use vortex::validity::Validity;
use vortex::IntoArray;
use vortex_datafusion::persistent::config::{VortexFile, VortexTableConfig};
use vortex_datafusion::persistent::provider::VortexFileTableProvider;
use vortex_serde::file::file_writer::FileWriter;

#[tokio::main]
async fn main() -> anyhow::Result<()> {
let tmp_path = temp_dir();
let strings = ChunkedArray::from_iter([
VarBinArray::from(vec!["ab", "foo", "bar", "baz"]).into_array(),
VarBinArray::from(vec!["ab", "foo", "bar", "baz"]).into_array(),
])
.into_array();

let numbers = ChunkedArray::from_iter([
PrimitiveArray::from(vec![1u32, 2, 3, 4]).into_array(),
PrimitiveArray::from(vec![5u32, 6, 7, 8]).into_array(),
])
.into_array();

let st = StructArray::try_new(
["strings".into(), "numbers".into()].into(),
vec![strings, numbers],
8,
Validity::NonNullable,
)
.unwrap();

let f = OpenOptions::new()
.write(true)
.truncate(true)
.open(tmp_path.join("a.vtx"))
.await?;

let writer = FileWriter::new(f);
let writer = writer.write_array_columns(st.into_array()).await?;
writer.finalize().await?;

let f = tokio::fs::File::open(tmp_path.join("a.vtx")).await?;
let file_size = f.metadata().await?.len();

let object_store: Arc<dyn ObjectStore> = Arc::new(LocalFileSystem::new());
let url = ObjectStoreUrl::local_filesystem();

let p = Path::from_filesystem_path(tmp_path.join("a.vtx"))?;

let config = VortexTableConfig::new(
Arc::new(Schema::new(vec![
Field::new("strings", DataType::Utf8, false),
Field::new("numbers", DataType::UInt32, false),
])),
vec![VortexFile::new(p, file_size)],
);

let provider = Arc::new(VortexFileTableProvider::try_new(url, config)?);

let ctx = SessionContext::new();
ctx.register_table("vortex_tbl", Arc::clone(&provider) as _)?;

let url = Url::try_from("file://").unwrap();
ctx.register_object_store(&url, object_store);

run_query(
&ctx,
"SELECT strings from vortex_tbl where numbers % 2 == 0",
)
.await?;

Ok(())
}

async fn run_query(ctx: &SessionContext, query_string: impl AsRef<str>) -> anyhow::Result<()> {
let query_string = query_string.as_ref();
ctx.sql(query_string).await?.show().await?;

ctx.sql(&format!("EXPLAIN {query_string}"))
.await?
.show()
.await?;

Ok(())
}
Loading
Loading