Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
3e6a570
Infer stream ipc format for arrow data sources
corasaurus-hex Nov 2, 2025
0ad62ed
Allow FileOpener for ArrowSource to open both IPC formats
corasaurus-hex Nov 2, 2025
34ccba4
Split reading file vs stream because repartitioning + ranges
corasaurus-hex Nov 3, 2025
99ebe62
Fix rewind bug
corasaurus-hex Nov 3, 2025
936b2e3
Remove a comment that isn't needed anymore
corasaurus-hex Nov 3, 2025
a8bc19d
Stray reference left over from Rename Symbol fail
corasaurus-hex Nov 3, 2025
93d26b1
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 3, 2025
21320cf
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 4, 2025
3c00395
Address clippy error
corasaurus-hex Nov 4, 2025
917c6c3
Address additional clippy errors
corasaurus-hex Nov 4, 2025
0f5642a
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 4, 2025
8941014
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 4, 2025
ffeca09
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 4, 2025
07593b4
Pull out the stream format check into an independent function
corasaurus-hex Nov 6, 2025
0446c32
Refactor schema inference
corasaurus-hex Nov 7, 2025
7409462
Let's move the `into()` outside the parens
corasaurus-hex Nov 7, 2025
3f72b0c
Err, no, on the inside
corasaurus-hex Nov 7, 2025
9e63fc7
Merge branch 'main' into cs--register-arrow-ipc-stream-format-files
corasaurus-hex Nov 7, 2025
c6d4a06
Also include a test for arrow stream source
corasaurus-hex Nov 7, 2025
4e28ef9
Add a bunch more tests
corasaurus-hex Nov 7, 2025
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
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ pub use datafusion_datasource_parquet::{ParquetFileMetrics, ParquetFileReaderFac

pub use json::{JsonOpener, JsonSource};

pub use arrow::{ArrowOpener, ArrowSource};
pub use arrow::{ArrowFileOpener, ArrowFileSource, ArrowStreamOpener, ArrowStreamSource};
pub use csv::{CsvOpener, CsvSource};
pub use datafusion_datasource::file::FileSource;
pub use datafusion_datasource::file_groups::FileGroup;
Expand Down
Binary file not shown.
Binary file not shown.
1 change: 1 addition & 0 deletions datafusion/core/tests/execution/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,3 +18,4 @@
mod coop;
mod datasource_split;
mod logical_plan;
mod register_arrow;
90 changes: 90 additions & 0 deletions datafusion/core/tests/execution/register_arrow.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

//! Integration tests for register_arrow API

use datafusion::{execution::options::ArrowReadOptions, prelude::*};
use datafusion_common::Result;

#[tokio::test]
async fn test_register_arrow_auto_detects_format() -> Result<()> {
let ctx = SessionContext::new();

ctx.register_arrow(
"file_format",
"../../datafusion/datasource-arrow/tests/data/example.arrow",
ArrowReadOptions::default(),
)
.await?;

ctx.register_arrow(
"stream_format",
"../../datafusion/datasource-arrow/tests/data/example_stream.arrow",
ArrowReadOptions::default(),
)
.await?;

let file_result = ctx.sql("SELECT * FROM file_format ORDER BY f0").await?;
let stream_result = ctx.sql("SELECT * FROM stream_format ORDER BY f0").await?;

let file_batches = file_result.collect().await?;
let stream_batches = stream_result.collect().await?;

assert_eq!(file_batches.len(), stream_batches.len());
assert_eq!(file_batches[0].schema(), stream_batches[0].schema());

let file_rows: usize = file_batches.iter().map(|b| b.num_rows()).sum();
let stream_rows: usize = stream_batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(file_rows, stream_rows);

Ok(())
}

#[tokio::test]
async fn test_register_arrow_join_file_and_stream() -> Result<()> {
let ctx = SessionContext::new();

ctx.register_arrow(
"file_table",
"../../datafusion/datasource-arrow/tests/data/example.arrow",
ArrowReadOptions::default(),
)
.await?;

ctx.register_arrow(
"stream_table",
"../../datafusion/datasource-arrow/tests/data/example_stream.arrow",
ArrowReadOptions::default(),
)
.await?;

let result = ctx
.sql(
"SELECT a.f0, a.f1, b.f0, b.f1
FROM file_table a
JOIN stream_table b ON a.f0 = b.f0
WHERE a.f0 <= 2
ORDER BY a.f0",
)
.await?;
let batches = result.collect().await?;

let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(total_rows, 2);

Ok(())
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use bytes::{BufMut, BytesMut};
use datafusion::common::Result;
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::physical_plan::{
ArrowSource, CsvSource, FileSource, JsonSource, ParquetSource,
ArrowFileSource, ArrowStreamSource, CsvSource, FileSource, JsonSource, ParquetSource,
};
use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::SessionContext;
Expand Down Expand Up @@ -282,9 +282,28 @@ async fn test_multi_source_schema_adapter_reuse() -> Result<()> {
// Create a test factory
let factory = Arc::new(UppercaseAdapterFactory {});

// Test ArrowSource
// Test ArrowFileSource
{
let source = ArrowSource::default();
let source = ArrowFileSource::default();
let source_with_adapter = source
.clone()
.with_schema_adapter_factory(factory.clone())
.unwrap();

let base_source: Arc<dyn FileSource> = source.into();
assert!(base_source.schema_adapter_factory().is_none());
assert!(source_with_adapter.schema_adapter_factory().is_some());

let retrieved_factory = source_with_adapter.schema_adapter_factory().unwrap();
assert_eq!(
format!("{:?}", retrieved_factory.as_ref()),
format!("{:?}", factory.as_ref())
);
}

// Test ArrowStreamSource
{
let source = ArrowStreamSource::default();
let source_with_adapter = source
.clone()
.with_schema_adapter_factory(factory.clone())
Expand Down
Loading