Skip to content

Commit 0af3169

Browse files
logan-keedealamb
andauthored
Move FileSinkConfig out of Core (#14585)
Co-authored-by: Andrew Lamb <[email protected]>
1 parent c0e78d2 commit 0af3169

File tree

9 files changed

+150
-127
lines changed

9 files changed

+150
-127
lines changed

Cargo.lock

+2
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

datafusion/catalog-listing/Cargo.toml

+3-1
Original file line numberDiff line numberDiff line change
@@ -40,11 +40,13 @@ async-compression = { version = "0.4.0", features = [
4040
"zstd",
4141
"tokio",
4242
], optional = true }
43+
async-trait = { workspace = true }
4344
bytes = { workspace = true }
4445
bzip2 = { version = "0.5.0", optional = true }
4546
chrono = { workspace = true }
4647
datafusion-catalog = { workspace = true }
4748
datafusion-common = { workspace = true, features = ["object_store"] }
49+
datafusion-common-runtime = { workspace = true }
4850
datafusion-execution = { workspace = true }
4951
datafusion-expr = { workspace = true }
5052
datafusion-physical-expr = { workspace = true }
@@ -56,14 +58,14 @@ glob = "0.3.0"
5658
itertools = { workspace = true }
5759
log = { workspace = true }
5860
object_store = { workspace = true }
61+
rand = { workspace = true }
5962
tokio = { workspace = true }
6063
tokio-util = { version = "0.7.4", features = ["io"], optional = true }
6164
url = { workspace = true }
6265
xz2 = { version = "0.1", optional = true, features = ["static"] }
6366
zstd = { version = "0.13", optional = true, default-features = false }
6467

6568
[dev-dependencies]
66-
async-trait = { workspace = true }
6769
tempfile = { workspace = true }
6870

6971
[lints]
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,114 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
use crate::write::demux::{start_demuxer_task, DemuxedStreamReceiver};
19+
use crate::{ListingTableUrl, PartitionedFile};
20+
use arrow::datatypes::{DataType, SchemaRef};
21+
use async_trait::async_trait;
22+
use datafusion_common::Result;
23+
use datafusion_common_runtime::SpawnedTask;
24+
use datafusion_execution::object_store::ObjectStoreUrl;
25+
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
26+
use datafusion_expr::dml::InsertOp;
27+
use datafusion_physical_plan::insert::DataSink;
28+
use object_store::ObjectStore;
29+
use std::sync::Arc;
30+
31+
/// General behaviors for files that do `DataSink` operations
32+
#[async_trait]
33+
pub trait FileSink: DataSink {
34+
/// Retrieves the file sink configuration.
35+
fn config(&self) -> &FileSinkConfig;
36+
37+
/// Spawns writer tasks and joins them to perform file writing operations.
38+
/// Is a critical part of `FileSink` trait, since it's the very last step for `write_all`.
39+
///
40+
/// This function handles the process of writing data to files by:
41+
/// 1. Spawning tasks for writing data to individual files.
42+
/// 2. Coordinating the tasks using a demuxer to distribute data among files.
43+
/// 3. Collecting results using `tokio::join`, ensuring that all tasks complete successfully.
44+
///
45+
/// # Parameters
46+
/// - `context`: The execution context (`TaskContext`) that provides resources
47+
/// like memory management and runtime environment.
48+
/// - `demux_task`: A spawned task that handles demuxing, responsible for splitting
49+
/// an input [`SendableRecordBatchStream`] into dynamically determined partitions.
50+
/// See `start_demuxer_task()`
51+
/// - `file_stream_rx`: A receiver that yields streams of record batches and their
52+
/// corresponding file paths for writing. See `start_demuxer_task()`
53+
/// - `object_store`: A handle to the object store where the files are written.
54+
///
55+
/// # Returns
56+
/// - `Result<u64>`: Returns the total number of rows written across all files.
57+
async fn spawn_writer_tasks_and_join(
58+
&self,
59+
context: &Arc<TaskContext>,
60+
demux_task: SpawnedTask<Result<()>>,
61+
file_stream_rx: DemuxedStreamReceiver,
62+
object_store: Arc<dyn ObjectStore>,
63+
) -> Result<u64>;
64+
65+
/// File sink implementation of the [`DataSink::write_all`] method.
66+
async fn write_all(
67+
&self,
68+
data: SendableRecordBatchStream,
69+
context: &Arc<TaskContext>,
70+
) -> Result<u64> {
71+
let config = self.config();
72+
let object_store = context
73+
.runtime_env()
74+
.object_store(&config.object_store_url)?;
75+
let (demux_task, file_stream_rx) = start_demuxer_task(config, data, context);
76+
self.spawn_writer_tasks_and_join(
77+
context,
78+
demux_task,
79+
file_stream_rx,
80+
object_store,
81+
)
82+
.await
83+
}
84+
}
85+
86+
/// The base configurations to provide when creating a physical plan for
87+
/// writing to any given file format.
88+
pub struct FileSinkConfig {
89+
/// Object store URL, used to get an ObjectStore instance
90+
pub object_store_url: ObjectStoreUrl,
91+
/// A vector of [`PartitionedFile`] structs, each representing a file partition
92+
pub file_groups: Vec<PartitionedFile>,
93+
/// Vector of partition paths
94+
pub table_paths: Vec<ListingTableUrl>,
95+
/// The schema of the output file
96+
pub output_schema: SchemaRef,
97+
/// A vector of column names and their corresponding data types,
98+
/// representing the partitioning columns for the file
99+
pub table_partition_cols: Vec<(String, DataType)>,
100+
/// Controls how new data should be written to the file, determining whether
101+
/// to append to, overwrite, or replace records in existing files.
102+
pub insert_op: InsertOp,
103+
/// Controls whether partition columns are kept for the file
104+
pub keep_partition_by_columns: bool,
105+
/// File extension without a dot(.)
106+
pub file_extension: String,
107+
}
108+
109+
impl FileSinkConfig {
110+
/// Get output schema
111+
pub fn output_schema(&self) -> &SchemaRef {
112+
&self.output_schema
113+
}
114+
}

datafusion/catalog-listing/src/mod.rs

+2
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,10 @@
2020
2121
pub mod file_compression_type;
2222
pub mod file_groups;
23+
pub mod file_sink_config;
2324
pub mod helpers;
2425
pub mod url;
26+
pub mod write;
2527
use chrono::TimeZone;
2628
use datafusion_common::Result;
2729
use datafusion_common::{ScalarValue, Statistics};

datafusion/core/src/datasource/file_format/write/demux.rs datafusion/catalog-listing/src/write/demux.rs

+4-4
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,10 @@ use std::borrow::Cow;
2222
use std::collections::HashMap;
2323
use std::sync::Arc;
2424

25-
use crate::datasource::listing::ListingTableUrl;
26-
use crate::datasource::physical_plan::FileSinkConfig;
27-
use crate::error::Result;
28-
use crate::physical_plan::SendableRecordBatchStream;
25+
use crate::url::ListingTableUrl;
26+
use crate::write::FileSinkConfig;
27+
use datafusion_common::error::Result;
28+
use datafusion_physical_plan::SendableRecordBatchStream;
2929

3030
use arrow::array::{
3131
builder::UInt64Builder, cast::AsArray, downcast_dictionary_array, RecordBatch,

datafusion/core/src/datasource/file_format/write/mod.rs datafusion/catalog-listing/src/write/mod.rs

+9-9
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,9 @@
2121
use std::io::Write;
2222
use std::sync::Arc;
2323

24-
use crate::datasource::file_format::file_compression_type::FileCompressionType;
25-
use crate::datasource::physical_plan::FileSinkConfig;
26-
use crate::error::Result;
24+
use crate::file_compression_type::FileCompressionType;
25+
use crate::file_sink_config::FileSinkConfig;
26+
use datafusion_common::error::Result;
2727

2828
use arrow::array::RecordBatch;
2929
use arrow::datatypes::Schema;
@@ -33,18 +33,18 @@ use object_store::path::Path;
3333
use object_store::ObjectStore;
3434
use tokio::io::AsyncWrite;
3535

36-
pub(crate) mod demux;
37-
pub(crate) mod orchestration;
36+
pub mod demux;
37+
pub mod orchestration;
3838

3939
/// A buffer with interior mutability shared by the SerializedFileWriter and
4040
/// ObjectStore writer
4141
#[derive(Clone)]
42-
pub(crate) struct SharedBuffer {
42+
pub struct SharedBuffer {
4343
/// The inner buffer for reading and writing
4444
///
4545
/// The lock is used to obtain internal mutability, so no worry about the
4646
/// lock contention.
47-
pub(crate) buffer: Arc<futures::lock::Mutex<Vec<u8>>>,
47+
pub buffer: Arc<futures::lock::Mutex<Vec<u8>>>,
4848
}
4949

5050
impl SharedBuffer {
@@ -79,7 +79,7 @@ pub trait BatchSerializer: Sync + Send {
7979
/// with the specified compression.
8080
/// We drop the `AbortableWrite` struct and the writer will not try to cleanup on failure.
8181
/// Users can configure automatic cleanup with their cloud provider.
82-
pub(crate) async fn create_writer(
82+
pub async fn create_writer(
8383
file_compression_type: FileCompressionType,
8484
location: &Path,
8585
object_store: Arc<dyn ObjectStore>,
@@ -91,7 +91,7 @@ pub(crate) async fn create_writer(
9191
/// Converts table schema to writer schema, which may differ in the case
9292
/// of hive style partitioning where some columns are removed from the
9393
/// underlying files.
94-
pub(crate) fn get_writer_schema(config: &FileSinkConfig) -> Arc<Schema> {
94+
pub fn get_writer_schema(config: &FileSinkConfig) -> Arc<Schema> {
9595
if !config.table_partition_cols.is_empty() && !config.keep_partition_by_columns {
9696
let schema = config.output_schema();
9797
let partition_names: Vec<_> =

datafusion/core/src/datasource/file_format/write/orchestration.rs datafusion/catalog-listing/src/write/orchestration.rs

+3-3
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,8 @@ use std::sync::Arc;
2323

2424
use super::demux::DemuxedStreamReceiver;
2525
use super::{create_writer, BatchSerializer};
26-
use crate::datasource::file_format::file_compression_type::FileCompressionType;
27-
use crate::error::Result;
26+
use crate::file_compression_type::FileCompressionType;
27+
use datafusion_common::error::Result;
2828

2929
use arrow::array::RecordBatch;
3030
use datafusion_common::{internal_datafusion_err, internal_err, DataFusionError};
@@ -237,7 +237,7 @@ pub(crate) async fn stateless_serialize_and_write_files(
237237
/// Orchestrates multipart put of a dynamic number of output files from a single input stream
238238
/// for any statelessly serialized file type. That is, any file type for which each [RecordBatch]
239239
/// can be serialized independently of all other [RecordBatch]s.
240-
pub(crate) async fn spawn_writer_tasks_and_join(
240+
pub async fn spawn_writer_tasks_and_join(
241241
context: &Arc<TaskContext>,
242242
serializer: Arc<dyn BatchSerializer>,
243243
compression: FileCompressionType,

datafusion/core/src/datasource/file_format/mod.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,8 @@ pub mod json;
2828
pub mod options;
2929
#[cfg(feature = "parquet")]
3030
pub mod parquet;
31-
pub mod write;
3231
pub use datafusion_catalog_listing::file_compression_type;
32+
pub use datafusion_catalog_listing::write;
3333

3434
use std::any::Any;
3535
use std::collections::{HashMap, VecDeque};

0 commit comments

Comments
 (0)