chore: upgrade DataFusion family (#7558)

* chore: upgrade DataFusion family

Signed-off-by: luofucong <luofc@foxmail.com>

* use main proto

Signed-off-by: luofucong <luofc@foxmail.com>

* fix ci

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
This commit is contained in:
LFC
2026-01-14 22:02:31 +08:00
committed by GitHub
parent a5cb0116a2
commit e64c31e59a
96 changed files with 2003 additions and 1531 deletions

View File

@@ -27,13 +27,14 @@ common-recordbatch.workspace = true
common-runtime.workspace = true
common-telemetry.workspace = true
datafusion.workspace = true
datafusion-datasource.workspace = true
datafusion-orc.workspace = true
datatypes.workspace = true
futures.workspace = true
lazy_static.workspace = true
object-store.workspace = true
object_store_opendal.workspace = true
orc-rust = { version = "0.6.3", default-features = false, features = ["async"] }
orc-rust = { version = "0.7", default-features = false, features = ["async"] }
parquet.workspace = true
paste.workspace = true
regex.workspace = true

View File

@@ -14,7 +14,7 @@
use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use datafusion::parquet::format::FileMetaData;
use parquet::file::metadata::ParquetMetaData;
use crate::error::Result;
@@ -24,5 +24,5 @@ pub trait DfRecordBatchEncoder {
#[async_trait]
pub trait ArrowWriterCloser {
async fn close(mut self) -> Result<FileMetaData>;
async fn close(mut self) -> Result<ParquetMetaData>;
}

View File

@@ -40,7 +40,6 @@ use datafusion::datasource::physical_plan::{
use datafusion::error::{DataFusionError, Result as DataFusionResult};
use datafusion::physical_plan::SendableRecordBatchStream;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use datatypes::arrow::datatypes::SchemaRef;
use futures::{StreamExt, TryStreamExt};
use object_store::ObjectStore;
use object_store_opendal::OpendalStore;
@@ -303,24 +302,20 @@ where
pub async fn file_to_stream(
store: &ObjectStore,
filename: &str,
file_schema: SchemaRef,
file_source: Arc<dyn FileSource>,
projection: Option<Vec<usize>>,
compression_type: CompressionType,
) -> Result<DfSendableRecordBatchStream> {
let df_compression: DfCompressionType = compression_type.into();
let config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
file_schema,
file_source.clone(),
)
.with_file_group(FileGroup::new(vec![PartitionedFile::new(
filename.to_string(),
0,
)]))
.with_projection(projection)
.with_file_compression_type(df_compression)
.build();
let config =
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source.clone())
.with_file_group(FileGroup::new(vec![PartitionedFile::new(
filename.to_string(),
0,
)]))
.with_projection_indices(projection)
.with_file_compression_type(df_compression)
.build();
let store = Arc::new(OpendalStore::new(store.clone()));
let file_opener = file_source

View File

@@ -440,14 +440,11 @@ mod tests {
.await
.unwrap(),
);
let csv_source = CsvSource::new(true, b',', b'"')
.with_schema(schema.clone())
.with_batch_size(8192);
let csv_source = CsvSource::new(schema).with_batch_size(8192);
let stream = file_to_stream(
&store,
compressed_file_path_str,
schema.clone(),
csv_source.clone(),
None,
compression_type,

View File

@@ -347,14 +347,11 @@ mod tests {
.await
.unwrap(),
);
let json_source = JsonSource::new()
.with_schema(schema.clone())
.with_batch_size(8192);
let json_source = JsonSource::new(schema).with_batch_size(8192);
let stream = file_to_stream(
&store,
compressed_file_path_str,
schema.clone(),
json_source.clone(),
None,
compression_type,

View File

@@ -18,15 +18,15 @@ use std::sync::Arc;
use arrow::record_batch::RecordBatch;
use arrow_schema::Schema;
use async_trait::async_trait;
use datafusion::datasource::physical_plan::{FileMeta, ParquetFileReaderFactory};
use datafusion::datasource::physical_plan::ParquetFileReaderFactory;
use datafusion::error::Result as DatafusionResult;
use datafusion::parquet::arrow::async_reader::AsyncFileReader;
use datafusion::parquet::arrow::{ArrowWriter, parquet_to_arrow_schema};
use datafusion::parquet::errors::{ParquetError, Result as ParquetResult};
use datafusion::parquet::file::metadata::ParquetMetaData;
use datafusion::parquet::format::FileMetaData;
use datafusion::physical_plan::SendableRecordBatchStream;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion_datasource::PartitionedFile;
use datatypes::schema::SchemaRef;
use futures::StreamExt;
use futures::future::BoxFuture;
@@ -100,11 +100,11 @@ impl ParquetFileReaderFactory for DefaultParquetFileReaderFactory {
fn create_reader(
&self,
_partition_index: usize,
file_meta: FileMeta,
partitioned_file: PartitionedFile,
_metadata_size_hint: Option<usize>,
_metrics: &ExecutionPlanMetricsSet,
) -> DatafusionResult<Box<dyn AsyncFileReader + Send>> {
let path = file_meta.location().to_string();
let path = partitioned_file.path().to_string();
let object_store = self.object_store.clone();
Ok(Box::new(LazyParquetFileReader::new(object_store, path)))
@@ -180,7 +180,7 @@ impl DfRecordBatchEncoder for ArrowWriter<SharedBuffer> {
#[async_trait]
impl ArrowWriterCloser for ArrowWriter<SharedBuffer> {
async fn close(self) -> Result<FileMetaData> {
async fn close(self) -> Result<ParquetMetaData> {
self.close().context(error::EncodeRecordBatchSnafu)
}
}

View File

@@ -67,14 +67,14 @@ impl Test<'_> {
async fn test_json_opener() {
let store = test_store("/");
let schema = test_basic_schema();
let file_source = Arc::new(JsonSource::new()).with_batch_size(test_util::TEST_BATCH_SIZE);
let file_source = Arc::new(JsonSource::new(schema)).with_batch_size(test_util::TEST_BATCH_SIZE);
let path = &find_workspace_path("/src/common/datasource/tests/json/basic.json")
.display()
.to_string();
let tests = [
Test {
config: scan_config(schema.clone(), None, path, file_source.clone()),
config: scan_config(None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+-----+-------+",
@@ -87,7 +87,7 @@ async fn test_json_opener() {
],
},
Test {
config: scan_config(schema, Some(1), path, file_source.clone()),
config: scan_config(Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+-----+------+",
@@ -112,13 +112,11 @@ async fn test_csv_opener() {
.display()
.to_string();
let file_source = CsvSource::new(true, b',', b'"')
.with_batch_size(test_util::TEST_BATCH_SIZE)
.with_schema(schema.clone());
let file_source = CsvSource::new(schema).with_batch_size(test_util::TEST_BATCH_SIZE);
let tests = [
Test {
config: scan_config(schema.clone(), None, path, file_source.clone()),
config: scan_config(None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+-----+-------+---------------------+----------+------------+",
@@ -131,7 +129,7 @@ async fn test_csv_opener() {
],
},
Test {
config: scan_config(schema, Some(1), path, file_source.clone()),
config: scan_config(Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+-----+------+---------------------+----------+------------+",
@@ -158,10 +156,10 @@ async fn test_parquet_exec() {
.display()
.to_string();
let parquet_source = ParquetSource::default()
let parquet_source = ParquetSource::new(schema)
.with_parquet_file_reader_factory(Arc::new(DefaultParquetFileReaderFactory::new(store)));
let config = scan_config(schema, None, path, Arc::new(parquet_source));
let config = scan_config(None, path, Arc::new(parquet_source));
let exec = DataSourceExec::from_data_source(config);
let ctx = SessionContext::new();
@@ -197,11 +195,11 @@ async fn test_orc_opener() {
let store = test_store("/");
let schema = Arc::new(OrcFormat.infer_schema(&store, path).await.unwrap());
let file_source = Arc::new(OrcSource::default());
let file_source = Arc::new(OrcSource::new(schema.into()));
let tests = [
Test {
config: scan_config(schema.clone(), None, path, file_source.clone()),
config: scan_config(None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+",
@@ -216,7 +214,7 @@ async fn test_orc_opener() {
],
},
Test {
config: scan_config(schema.clone(), Some(1), path, file_source.clone()),
config: scan_config(Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+----------+-----+------+------------+---+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+-------------------------+-------------+",

View File

@@ -80,7 +80,6 @@ pub fn csv_basic_schema() -> SchemaRef {
}
pub(crate) fn scan_config(
file_schema: SchemaRef,
limit: Option<usize>,
filename: &str,
file_source: Arc<dyn FileSource>,
@@ -89,7 +88,7 @@ pub(crate) fn scan_config(
let filename = &filename.replace('\\', "/");
let file_group = FileGroup::new(vec![PartitionedFile::new(filename.clone(), 4096)]);
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema, file_source)
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source)
.with_file_group(file_group)
.with_limit(limit)
.build()
@@ -109,7 +108,7 @@ pub async fn setup_stream_to_json_test(origin_path: &str, threshold: impl Fn(usi
let size = store.read(origin_path).await.unwrap().len();
let config = scan_config(schema, None, origin_path, Arc::new(JsonSource::new()));
let config = scan_config(None, origin_path, Arc::new(JsonSource::new(schema)));
let stream = FileStream::new(
&config,
0,
@@ -151,10 +150,8 @@ pub async fn setup_stream_to_csv_test(
let schema = csv_basic_schema();
let csv_source = CsvSource::new(true, b',', b'"')
.with_schema(schema.clone())
.with_batch_size(TEST_BATCH_SIZE);
let config = scan_config(schema, None, origin_path, csv_source.clone());
let csv_source = CsvSource::new(schema).with_batch_size(TEST_BATCH_SIZE);
let config = scan_config(None, origin_path, csv_source.clone());
let size = store.read(origin_path).await.unwrap().len();
let csv_opener = csv_source.create_file_opener(